=str #17089 stream testkit

This commit is contained in:
Martynas Mickevičius 2015-04-24 11:45:03 +03:00
parent 7b4a640147
commit 8e2cc3e70f
96 changed files with 1411 additions and 1131 deletions

View file

@ -17,6 +17,6 @@ Streams
stream-error
stream-io
stream-parallelism
stream-testkit
stream-cookbook
../stream-configuration

View file

@ -0,0 +1,25 @@
.. _stream-testkit-java:
###############
Testing streams
###############
Akka Streams comes with an :mod:`akka-stream-testkit` module that provides tools which can be used for controlling and asserting various parts of the stream pipeline.
Probe Sink
==========
Using probe as a `Sink` allows manual control over demand and assertions over elements coming downstream. Streams testkit provides a sink that materializes to a :class:`TestSubscriber.Probe`.
.. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/StreamTestKitDocTest.java#test-sink-probe
Probe Source
============
A source that materializes to :class:`TestPublisher.Probe` can be used for asserting demand or controlling when stream is completed or ended with an error.
.. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/StreamTestKitDocTest.java#test-source-probe
*TODO*
List by example various operations on probes. Using probes without a sink.

View file

@ -5,9 +5,8 @@ package docs.stream
import akka.stream.ActorFlowMaterializer
import akka.stream.scaladsl.Flow
import akka.stream.testkit.AkkaSpec
import akka.stream.testkit._
import akka.stream.scaladsl.Sink
import akka.stream.testkit.StreamTestKit.SubscriberProbe
import akka.stream.scaladsl.Source
class ReactiveStreamsDocSpec extends AkkaSpec {
@ -45,12 +44,12 @@ class ReactiveStreamsDocSpec extends AkkaSpec {
override def tweets: Publisher[Tweet] =
TwitterStreamQuickstartDocSpec.tweets.runWith(Sink.publisher)
override def storage = SubscriberProbe[Author]
override def storage = TestSubscriber.manualProbe[Author]
override def alert = SubscriberProbe[Author]
override def alert = TestSubscriber.manualProbe[Author]
}
def assertResult(storage: SubscriberProbe[Author]): Unit = {
def assertResult(storage: TestSubscriber.ManualProbe[Author]): Unit = {
val sub = storage.expectSubscription()
sub.request(10)
storage.expectNext(Author("rolandkuhn"))

View file

@ -0,0 +1,38 @@
/**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/
package docs.stream
import akka.stream._
import akka.stream.scaladsl._
import akka.stream.testkit._
import akka.stream.testkit.scaladsl._
class StreamTestKitDocSpec extends AkkaSpec {
implicit val mat = ActorFlowMaterializer()
"test source probe" in {
//#test-source-probe
TestSource.probe[Int]
.toMat(Sink.cancelled)(Keep.left)
.run()
.expectCancellation()
//#test-source-probe
}
"test sink probe" in {
//#test-sink-probe
Source(1 to 4)
.filter(_ % 2 == 0)
.map(_ * 2)
.runWith(TestSink.probe[Int])
.request(2)
.expectNext(4, 8)
.expectComplete()
//#test-sink-probe
}
}

View file

@ -2,8 +2,7 @@ package docs.stream.cookbook
import akka.stream.{ ActorFlowMaterializerSettings, ActorFlowMaterializer }
import akka.stream.scaladsl._
import akka.stream.testkit.StreamTestKit
import akka.stream.testkit.StreamTestKit.{ SubscriberProbe, PublisherProbe }
import akka.stream.testkit._
import scala.collection.immutable
import scala.concurrent.Await
@ -18,8 +17,8 @@ class RecipeCollectingMetrics extends RecipeSpec {
"work" in {
// type Tick = Unit
//
// val loadPub = PublisherProbe[Int]()
// val tickPub = PublisherProbe[Tick]()
// val loadPub = TestPublisher.manualProbe[Int]()
// val tickPub = TestPublisher.manualProbe[Tick]()
// val reportTicks = Source(tickPub)
// val loadUpdates = Source(loadPub)
// val futureSink = Sink.head[immutable.Seq[String]]

View file

@ -2,7 +2,7 @@ package docs.stream.cookbook
import akka.stream.OverflowStrategy
import akka.stream.scaladsl._
import akka.stream.testkit.StreamTestKit.SubscriberProbe
import akka.stream.testkit._
import scala.collection.immutable
import scala.concurrent.Await
@ -14,8 +14,8 @@ class RecipeDroppyBroadcast extends RecipeSpec {
"work" in {
val myElements = Source(immutable.Iterable.tabulate(100)(_ + 1))
val sub1 = SubscriberProbe[Int]()
val sub2 = SubscriberProbe[Int]()
val sub1 = TestSubscriber.manualProbe[Int]()
val sub2 = TestSubscriber.manualProbe[Int]()
val futureSink = Sink.head[Seq[Int]]
val mySink1 = Sink(sub1)
val mySink2 = Sink(sub2)

View file

@ -3,7 +3,7 @@ package docs.stream.cookbook
import akka.actor.{ Props, ActorRef, Actor }
import akka.actor.Actor.Receive
import akka.stream.scaladsl._
import akka.stream.testkit.StreamTestKit.SubscriberProbe
import akka.stream.testkit._
import scala.collection.immutable
import scala.concurrent.duration._
@ -95,7 +95,7 @@ class RecipeGlobalRateLimit extends RecipeSpec {
val source1 = Source(() => Iterator.continually("E1")).via(limitGlobal(limiter, 2.seconds))
val source2 = Source(() => Iterator.continually("E2")).via(limitGlobal(limiter, 2.seconds))
val probe = SubscriberProbe[String]()
val probe = TestSubscriber.manualProbe[String]()
FlowGraph.closed() { implicit b =>
import FlowGraph.Implicits._

View file

@ -1,8 +1,7 @@
package docs.stream.cookbook
import akka.stream.scaladsl.{ Sink, Source }
import akka.stream.testkit.StreamTestKit
import akka.stream.testkit.StreamTestKit.{ SubscriberProbe, PublisherProbe }
import akka.stream.testkit._
import scala.concurrent.duration._
@ -53,15 +52,13 @@ class RecipeHold extends RecipeSpec {
"work for version 1" in {
val pub = PublisherProbe[Int]()
val sub = SubscriberProbe[Int]()
val pub = TestPublisher.probe[Int]()
val sub = TestSubscriber.manualProbe[Int]()
val source = Source(pub)
val sink = Sink(sub)
source.transform(() => new HoldWithInitial(0)).to(sink).run()
val manualSource = new StreamTestKit.AutoPublisher(pub)
val subscription = sub.expectSubscription()
sub.expectNoMsg(100.millis)
@ -71,46 +68,44 @@ class RecipeHold extends RecipeSpec {
subscription.request(1)
sub.expectNext(0)
manualSource.sendNext(1)
manualSource.sendNext(2)
pub.sendNext(1)
pub.sendNext(2)
subscription.request(2)
sub.expectNext(2)
sub.expectNext(2)
manualSource.sendComplete()
pub.sendComplete()
subscription.request(1)
sub.expectComplete()
}
"work for version 2" in {
val pub = PublisherProbe[Int]()
val sub = SubscriberProbe[Int]()
val pub = TestPublisher.probe[Int]()
val sub = TestSubscriber.manualProbe[Int]()
val source = Source(pub)
val sink = Sink(sub)
source.transform(() => new HoldWithWait).to(sink).run()
val manualSource = new StreamTestKit.AutoPublisher(pub)
val subscription = sub.expectSubscription()
sub.expectNoMsg(100.millis)
subscription.request(1)
sub.expectNoMsg(100.millis)
manualSource.sendNext(1)
pub.sendNext(1)
sub.expectNext(1)
manualSource.sendNext(2)
manualSource.sendNext(3)
pub.sendNext(2)
pub.sendNext(3)
subscription.request(2)
sub.expectNext(3)
sub.expectNext(3)
manualSource.sendComplete()
pub.sendComplete()
subscription.request(1)
sub.expectComplete()
}

View file

@ -1,8 +1,7 @@
package docs.stream.cookbook
import akka.stream.scaladsl._
import akka.stream.testkit.StreamTestKit
import akka.stream.testkit.StreamTestKit.{ SubscriberProbe, PublisherProbe }
import akka.stream.testkit._
import akka.util.ByteString
class RecipeKeepAlive extends RecipeSpec {
@ -13,9 +12,9 @@ class RecipeKeepAlive extends RecipeSpec {
type Tick = Unit
val tickPub = PublisherProbe[Tick]()
val dataPub = PublisherProbe[ByteString]()
val sub = SubscriberProbe[ByteString]()
val tickPub = TestPublisher.probe[Tick]()
val dataPub = TestPublisher.probe[ByteString]()
val sub = TestSubscriber.manualProbe[ByteString]()
val ticks = Source(tickPub)
val dataStream = Source(dataPub)
@ -38,17 +37,14 @@ class RecipeKeepAlive extends RecipeSpec {
graph.run()
val manualTicks = new StreamTestKit.AutoPublisher(tickPub)
val manualData = new StreamTestKit.AutoPublisher(dataPub)
val subscription = sub.expectSubscription()
manualTicks.sendNext(())
tickPub.sendNext(())
// pending data will overcome the keepalive
manualData.sendNext(ByteString(1))
manualData.sendNext(ByteString(2))
manualData.sendNext(ByteString(3))
dataPub.sendNext(ByteString(1))
dataPub.sendNext(ByteString(2))
dataPub.sendNext(ByteString(3))
subscription.request(1)
sub.expectNext(ByteString(1))
@ -60,11 +56,11 @@ class RecipeKeepAlive extends RecipeSpec {
sub.expectNext(keepaliveMessage)
subscription.request(1)
manualTicks.sendNext(())
tickPub.sendNext(())
sub.expectNext(keepaliveMessage)
manualData.sendComplete()
manualTicks.sendComplete()
dataPub.sendComplete()
tickPub.sendComplete()
sub.expectComplete()

View file

@ -1,8 +1,7 @@
package docs.stream.cookbook
import akka.stream.scaladsl._
import akka.stream.testkit.StreamTestKit
import akka.stream.testkit.StreamTestKit.{ SubscriberProbe, PublisherProbe }
import akka.stream.testkit._
import scala.concurrent.duration._
class RecipeManualTrigger extends RecipeSpec {
@ -12,8 +11,8 @@ class RecipeManualTrigger extends RecipeSpec {
"work" in {
val elements = Source(List("1", "2", "3", "4"))
val pub = PublisherProbe[Trigger]()
val sub = SubscriberProbe[Message]()
val pub = TestPublisher.probe[Trigger]()
val sub = TestSubscriber.manualProbe[Message]()
val triggerSource = Source(pub)
val sink = Sink(sub)
@ -28,22 +27,21 @@ class RecipeManualTrigger extends RecipeSpec {
//#manually-triggered-stream
graph.run()
val manualSource = new StreamTestKit.AutoPublisher(pub)
sub.expectSubscription().request(1000)
sub.expectNoMsg(100.millis)
manualSource.sendNext(())
pub.sendNext(())
sub.expectNext("1")
sub.expectNoMsg(100.millis)
manualSource.sendNext(())
manualSource.sendNext(())
pub.sendNext(())
pub.sendNext(())
sub.expectNext("2")
sub.expectNext("3")
sub.expectNoMsg(100.millis)
manualSource.sendNext(())
pub.sendNext(())
sub.expectNext("4")
sub.expectComplete()
}
@ -51,8 +49,8 @@ class RecipeManualTrigger extends RecipeSpec {
"work with ZipWith" in {
val elements = Source(List("1", "2", "3", "4"))
val pub = PublisherProbe[Trigger]()
val sub = SubscriberProbe[Message]()
val pub = TestPublisher.probe[Trigger]()
val sub = TestSubscriber.manualProbe[Message]()
val triggerSource = Source(pub)
val sink = Sink(sub)
@ -68,22 +66,21 @@ class RecipeManualTrigger extends RecipeSpec {
//#manually-triggered-stream-zipwith
graph.run()
val manualSource = new StreamTestKit.AutoPublisher(pub)
sub.expectSubscription().request(1000)
sub.expectNoMsg(100.millis)
manualSource.sendNext(())
pub.sendNext(())
sub.expectNext("1")
sub.expectNoMsg(100.millis)
manualSource.sendNext(())
manualSource.sendNext(())
pub.sendNext(())
pub.sendNext(())
sub.expectNext("2")
sub.expectNext("3")
sub.expectNoMsg(100.millis)
manualSource.sendNext(())
pub.sendNext(())
sub.expectNext("4")
sub.expectComplete()
}

View file

@ -1,8 +1,7 @@
package docs.stream.cookbook
import akka.stream.scaladsl.{ Sink, Source }
import akka.stream.testkit.StreamTestKit
import akka.stream.testkit.StreamTestKit.{ SubscriberProbe, PublisherProbe }
import akka.stream.testkit._
import scala.concurrent.duration._
@ -13,8 +12,8 @@ class RecipeMissedTicks extends RecipeSpec {
"work" in {
type Tick = Unit
val pub = PublisherProbe[Tick]()
val sub = SubscriberProbe[Int]()
val pub = TestPublisher.probe[Tick]()
val sub = TestSubscriber.manualProbe[Int]()
val tickStream = Source(pub)
val sink = Sink(sub)
@ -26,12 +25,11 @@ class RecipeMissedTicks extends RecipeSpec {
//#missed-ticks
missedTicks.to(sink).run()
val manualSource = new StreamTestKit.AutoPublisher(pub)
manualSource.sendNext(())
manualSource.sendNext(())
manualSource.sendNext(())
manualSource.sendNext(())
pub.sendNext(())
pub.sendNext(())
pub.sendNext(())
pub.sendNext(())
val subscription = sub.expectSubscription()
subscription.request(1)
@ -40,10 +38,10 @@ class RecipeMissedTicks extends RecipeSpec {
subscription.request(1)
sub.expectNoMsg(100.millis)
manualSource.sendNext(())
pub.sendNext(())
sub.expectNext(0)
manualSource.sendComplete()
pub.sendComplete()
subscription.request(1)
sub.expectComplete()
}

View file

@ -1,8 +1,7 @@
package docs.stream.cookbook
import akka.stream.scaladsl.{ Flow, Sink, Source }
import akka.stream.testkit.StreamTestKit
import akka.stream.testkit.StreamTestKit.{ SubscriberProbe, PublisherProbe }
import akka.stream.testkit._
import scala.concurrent.duration._
@ -17,26 +16,24 @@ class RecipeSimpleDrop extends RecipeSpec {
Flow[Message].conflate(seed = identity)((lastMessage, newMessage) => newMessage)
//#simple-drop
val pub = PublisherProbe[Message]()
val sub = SubscriberProbe[Message]()
val pub = TestPublisher.probe[Message]()
val sub = TestSubscriber.manualProbe[Message]()
val messageSource = Source(pub)
val sink = Sink(sub)
messageSource.via(droppyStream).to(sink).run()
val manualSource = new StreamTestKit.AutoPublisher(pub)
val subscription = sub.expectSubscription()
sub.expectNoMsg(100.millis)
manualSource.sendNext("1")
manualSource.sendNext("2")
manualSource.sendNext("3")
pub.sendNext("1")
pub.sendNext("2")
pub.sendNext("3")
subscription.request(1)
sub.expectNext("3")
manualSource.sendComplete()
pub.sendComplete()
subscription.request(1)
sub.expectComplete()
}

View file

@ -8,11 +8,11 @@ import java.io.File
import akka.stream._
import akka.stream.io.SynchronousFileSource
import akka.stream.io.SynchronousFileSink
import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.StreamTestKit
import akka.stream.testkit._
import akka.stream.testkit.Utils._
import akka.util.ByteString
class StreamFileDocSpec extends AkkaSpec(StreamTestKit.UnboundedMailboxConfig) {
class StreamFileDocSpec extends AkkaSpec(UnboundedMailboxConfig) {
implicit val ec = system.dispatcher
implicit val mat = ActorFlowMaterializer()

View file

@ -17,6 +17,6 @@ Streams
stream-error
stream-io
stream-parallelism
stream-testkit
stream-cookbook
../stream-configuration

View file

@ -0,0 +1,25 @@
.. _stream-testkit-scala:
###############
Testing streams
###############
Akka Streams comes with an :mod:`akka-stream-testkit` module that provides tools which can be used for controlling and asserting various parts of the stream pipeline.
Probe Sink
==========
Using probe as a `Sink` allows manual control over demand and assertions over elements coming downstream. Streams testkit provides a sink that materializes to a :class:`TestSubscriber.Probe`.
.. includecode:: code/docs/stream/StreamTestKitDocSpec.scala#test-sink-probe
Probe Source
============
A source that materializes to :class:`TestPublisher.Probe` can be used for asserting demand or controlling when stream is completed or ended with an error.
.. includecode:: code/docs/stream/StreamTestKitDocSpec.scala#test-source-probe
*TODO*
List by example various operations on probes. Using probes without a sink.

View file

@ -21,8 +21,7 @@ import akka.http.model.headers._
import akka.http.util._
import akka.stream.{ ActorFlowMaterializer, BindFailedException }
import akka.stream.scaladsl._
import akka.stream.testkit.StreamTestKit
import akka.stream.testkit.StreamTestKit.{ PublisherProbe, SubscriberProbe }
import akka.stream.testkit._
import com.typesafe.config.{ Config, ConfigFactory }
import org.scalatest.{ BeforeAndAfterAll, Matchers, WordSpec }
@ -40,7 +39,7 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
"properly bind a server" in {
val (_, hostname, port) = temporaryServerHostnameAndPort()
val probe = StreamTestKit.SubscriberProbe[Http.IncomingConnection]()
val probe = TestSubscriber.manualProbe[Http.IncomingConnection]()
val binding = Http().bind(hostname, port).toMat(Sink(probe))(Keep.left).run()
val sub = probe.expectSubscription() // if we get it we are bound
val address = Await.result(binding, 1.second).localAddress
@ -50,16 +49,16 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
"report failure if bind fails" in {
val (_, hostname, port) = temporaryServerHostnameAndPort()
val binding = Http().bind(hostname, port)
val probe1 = StreamTestKit.SubscriberProbe[Http.IncomingConnection]()
val probe1 = TestSubscriber.manualProbe[Http.IncomingConnection]()
// Bind succeeded, we have a local address
val b1 = Await.result(binding.to(Sink(probe1)).run(), 3.seconds)
probe1.expectSubscription()
val probe2 = StreamTestKit.SubscriberProbe[Http.IncomingConnection]()
val probe2 = TestSubscriber.manualProbe[Http.IncomingConnection]()
an[BindFailedException] shouldBe thrownBy { Await.result(binding.to(Sink(probe2)).run(), 3.seconds) }
probe2.expectSubscriptionAndError()
val probe3 = StreamTestKit.SubscriberProbe[Http.IncomingConnection]()
val probe3 = TestSubscriber.manualProbe[Http.IncomingConnection]()
an[BindFailedException] shouldBe thrownBy { Await.result(binding.to(Sink(probe3)).run(), 3.seconds) }
probe3.expectSubscriptionAndError()
@ -68,7 +67,7 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
probe1.expectComplete()
if (!akka.util.Helpers.isWindows) {
val probe4 = StreamTestKit.SubscriberProbe[Http.IncomingConnection]()
val probe4 = TestSubscriber.manualProbe[Http.IncomingConnection]()
// Bind succeeded, we have a local address
val b2 = Await.result(binding.to(Sink(probe4)).run(), 3.seconds)
probe4.expectSubscription()
@ -221,15 +220,15 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
val connSource = {
val settings = configOverrides.toOption.fold(ServerSettings(system))(ServerSettings(_))
val binding = Http().bind(hostname, port, settings = settings)
val probe = StreamTestKit.SubscriberProbe[Http.IncomingConnection]
val probe = TestSubscriber.manualProbe[Http.IncomingConnection]
binding.runWith(Sink(probe))
probe
}
val connSourceSub = connSource.expectSubscription()
def openNewClientConnection(settings: ClientConnectionSettings = ClientConnectionSettings(system)) = {
val requestPublisherProbe = StreamTestKit.PublisherProbe[HttpRequest]()
val responseSubscriberProbe = StreamTestKit.SubscriberProbe[HttpResponse]()
val requestPublisherProbe = TestPublisher.manualProbe[HttpRequest]()
val responseSubscriberProbe = TestSubscriber.manualProbe[HttpResponse]()
val connectionFuture = Source(requestPublisherProbe)
.viaMat(Http().outgoingConnection(hostname, port, settings = settings))(Keep.right)
@ -242,7 +241,7 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
requestPublisherProbe -> responseSubscriberProbe
}
def acceptConnection(): (SubscriberProbe[HttpRequest], PublisherProbe[HttpResponse]) = {
def acceptConnection(): (TestSubscriber.ManualProbe[HttpRequest], TestPublisher.ManualProbe[HttpResponse]) = {
connSourceSub.request(1)
val incomingConnection = connSource.expectNext()
val sink = Sink.publisher[HttpRequest]
@ -254,8 +253,8 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
}
val (pub, sub) = incomingConnection.handleWith(handler)
val requestSubscriberProbe = StreamTestKit.SubscriberProbe[HttpRequest]()
val responsePublisherProbe = StreamTestKit.PublisherProbe[HttpResponse]()
val requestSubscriberProbe = TestSubscriber.manualProbe[HttpRequest]()
val responsePublisherProbe = TestPublisher.manualProbe[HttpResponse]()
pub.subscribe(requestSubscriberProbe)
responsePublisherProbe.subscribe(sub)

View file

@ -4,6 +4,7 @@
package akka.http.engine.client
import java.net.InetAddress
import java.util.concurrent.atomic.AtomicInteger
import akka.http.engine.client.PoolGateway
@ -14,7 +15,7 @@ import akka.http.engine.server.ServerSettings
import akka.http.util.{ SingletonException, StreamUtils }
import akka.util.ByteString
import akka.stream.{ BidiShape, ActorFlowMaterializer }
import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
import akka.stream.testkit._
import akka.stream.scaladsl._
import akka.http.{ Http, TestUtils }
import akka.http.model.headers._
@ -33,7 +34,7 @@ class ConnectionPoolSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = O
acceptIncomingConnection()
responseOutSub.request(1)
val (Success(response), 42) = responseOut.expectNext()
response.headers should contain(RawHeader("Req-Host", s"localhost:$serverPort"))
response.headers should contain(RawHeader("Req-Host", s"$serverHostName:$serverPort"))
}
"open a second connection if the first one is loaded" in new TestSetup {
@ -179,7 +180,7 @@ class ConnectionPoolSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = O
val request = HttpRequest(uri = "/abc", headers = List(Host(serverHostName, serverPort)))
val responseFuture = Http().singleRequest(request)
val responseHeaders = Await.result(responseFuture, 1.second).headers
responseHeaders should contain(RawHeader("Req-Uri", s"http://localhost:$serverPort/abc"))
responseHeaders should contain(RawHeader("Req-Uri", s"http://$serverHostName:$serverPort/abc"))
responseHeaders should contain(RawHeader("Req-Raw-Request-URI", "/abc"))
}
@ -188,7 +189,7 @@ class ConnectionPoolSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = O
val responseFuture = Http().singleRequest(request)
val responseHeaders = Await.result(responseFuture, 1.second).headers
responseHeaders should contain(RawHeader("Req-Raw-Request-URI", "/abc?query"))
responseHeaders should contain(RawHeader("Req-Host", s"localhost:$serverPort"))
responseHeaders should contain(RawHeader("Req-Host", s"$serverHostName:$serverPort"))
}
"support absolute request URIs without path component" in new LocalTestSetup {
@ -202,7 +203,7 @@ class ConnectionPoolSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = O
val request = HttpRequest(uri = s"http://$serverHostName:$serverPort//foo")
val responseFuture = Http().singleRequest(request)
val responseHeaders = Await.result(responseFuture, 1.second).headers
responseHeaders should contain(RawHeader("Req-Uri", s"http://localhost:$serverPort//foo"))
responseHeaders should contain(RawHeader("Req-Uri", s"http://$serverHostName:$serverPort//foo"))
responseHeaders should contain(RawHeader("Req-Raw-Request-URI", "//foo"))
}
@ -249,7 +250,7 @@ class ConnectionPoolSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = O
def mapServerSideOutboundRawBytes(bytes: ByteString): ByteString = bytes
val incomingConnectionCounter = new AtomicInteger
val incomingConnections = StreamTestKit.SubscriberProbe[Http.IncomingConnection]
val incomingConnections = TestSubscriber.manualProbe[Http.IncomingConnection]
val incomingConnectionsSub = {
val rawBytesInjection = BidiFlow() { b
val top = b.add(Flow[ByteString].map(mapServerSideOutboundRawBytes)
@ -298,11 +299,11 @@ class ConnectionPoolSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = O
}
def flowTestBench[T, Mat](poolFlow: Flow[(HttpRequest, T), (Try[HttpResponse], T), Mat]) = {
val requestIn = StreamTestKit.PublisherProbe[(HttpRequest, T)]
val responseOut = StreamTestKit.SubscriberProbe[(Try[HttpResponse], T)]
val requestIn = TestPublisher.probe[(HttpRequest, T)]()
val responseOut = TestSubscriber.manualProbe[(Try[HttpResponse], T)]
val hcp = Source(requestIn).viaMat(poolFlow)(Keep.right).toMat(Sink(responseOut))(Keep.left).run()
val responseOutSub = responseOut.expectSubscription()
(new StreamTestKit.AutoPublisher(requestIn), responseOut, responseOutSub, hcp)
(requestIn, responseOut, responseOutSub, hcp)
}
def connNr(r: HttpResponse): Int = r.headers.find(_ is "conn-nr").get.value.toInt

View file

@ -9,7 +9,7 @@ import org.scalatest.Inside
import akka.util.ByteString
import akka.event.NoLogging
import akka.stream.ActorFlowMaterializer
import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
import akka.stream.testkit._
import akka.stream.scaladsl._
import akka.http.model.HttpEntity._
import akka.http.model.HttpMethods._
@ -50,7 +50,7 @@ class LowLevelOutgoingConnectionSpec extends AkkaSpec("akka.loggers = []\n akka.
}
"has a request with default entity" in new TestSetup {
val probe = StreamTestKit.PublisherProbe[ByteString]()
val probe = TestPublisher.manualProbe[ByteString]()
requestsSub.sendNext(HttpRequest(PUT, entity = HttpEntity(ContentTypes.`application/octet-stream`, 8, Source(probe))))
expectWireData(
"""PUT / HTTP/1.1
@ -106,7 +106,7 @@ class LowLevelOutgoingConnectionSpec extends AkkaSpec("akka.loggers = []\n akka.
val HttpResponse(_, _, HttpEntity.Chunked(ct, chunks), _) = responses.expectNext()
ct shouldEqual ContentTypes.`application/octet-stream`
val probe = StreamTestKit.SubscriberProbe[ChunkStreamPart]()
val probe = TestSubscriber.manualProbe[ChunkStreamPart]()
chunks.runWith(Sink(probe))
val sub = probe.expectSubscription()
@ -175,7 +175,7 @@ class LowLevelOutgoingConnectionSpec extends AkkaSpec("akka.loggers = []\n akka.
responsesSub.request(1)
val HttpResponse(_, _, HttpEntity.Chunked(ct, chunks), _) = responses.expectNext()
val probe = StreamTestKit.SubscriberProbe[ChunkStreamPart]()
val probe = TestSubscriber.manualProbe[ChunkStreamPart]()
chunks.runWith(Sink(probe))
val sub = probe.expectSubscription()
@ -216,7 +216,7 @@ class LowLevelOutgoingConnectionSpec extends AkkaSpec("akka.loggers = []\n akka.
"produce proper errors" which {
"catch the entity stream being shorter than the Content-Length" in new TestSetup {
val probe = StreamTestKit.PublisherProbe[ByteString]()
val probe = TestPublisher.manualProbe[ByteString]()
requestsSub.sendNext(HttpRequest(PUT, entity = HttpEntity(ContentTypes.`application/octet-stream`, 8, Source(probe))))
expectWireData(
"""PUT / HTTP/1.1
@ -242,7 +242,7 @@ class LowLevelOutgoingConnectionSpec extends AkkaSpec("akka.loggers = []\n akka.
}
"catch the entity stream being longer than the Content-Length" in new TestSetup {
val probe = StreamTestKit.PublisherProbe[ByteString]()
val probe = TestPublisher.manualProbe[ByteString]()
requestsSub.sendNext(HttpRequest(PUT, entity = HttpEntity(ContentTypes.`application/octet-stream`, 8, Source(probe))))
expectWireData(
"""PUT / HTTP/1.1
@ -308,7 +308,7 @@ class LowLevelOutgoingConnectionSpec extends AkkaSpec("akka.loggers = []\n akka.
val HttpResponse(_, _, HttpEntity.Chunked(ct, chunks), _) = responses.expectNext()
ct shouldEqual ContentTypes.`application/octet-stream`
val probe = StreamTestKit.SubscriberProbe[ChunkStreamPart]()
val probe = TestSubscriber.manualProbe[ChunkStreamPart]()
chunks.runWith(Sink(probe))
val sub = probe.expectSubscription()
@ -348,16 +348,16 @@ class LowLevelOutgoingConnectionSpec extends AkkaSpec("akka.loggers = []\n akka.
}
class TestSetup {
val requests = StreamTestKit.PublisherProbe[HttpRequest]
val responses = StreamTestKit.SubscriberProbe[HttpResponse]
val requests = TestPublisher.manualProbe[HttpRequest]
val responses = TestSubscriber.manualProbe[HttpResponse]
val remoteAddress = new InetSocketAddress("example.com", 80)
def settings = ClientConnectionSettings(system)
.copy(userAgentHeader = Some(`User-Agent`(List(ProductVersion("akka-http", "test")))))
val (netOut, netIn) = {
val netOut = StreamTestKit.SubscriberProbe[ByteString]
val netIn = StreamTestKit.PublisherProbe[ByteString]
val netOut = TestSubscriber.manualProbe[ByteString]
val netIn = TestPublisher.manualProbe[ByteString]
FlowGraph.closed(OutgoingConnectionBlueprint(remoteAddress, settings, NoLogging)) { implicit b
client

View file

@ -12,7 +12,7 @@ import akka.event.NoLogging
import akka.util.ByteString
import akka.stream.scaladsl._
import akka.stream.ActorFlowMaterializer
import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
import akka.stream.testkit._
import akka.http.model._
import akka.http.util._
import headers._
@ -43,7 +43,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
inside(expectRequest) {
case HttpRequest(POST, _, _, HttpEntity.Default(_, 12, data), _)
val dataProbe = StreamTestKit.SubscriberProbe[ByteString]
val dataProbe = TestSubscriber.manualProbe[ByteString]
data.to(Sink(dataProbe)).run()
val sub = dataProbe.expectSubscription()
sub.request(10)
@ -87,7 +87,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
inside(expectRequest) {
case HttpRequest(POST, _, _, HttpEntity.Chunked(_, data), _)
val dataProbe = StreamTestKit.SubscriberProbe[ChunkStreamPart]
val dataProbe = TestSubscriber.manualProbe[ChunkStreamPart]
data.to(Sink(dataProbe)).run()
val sub = dataProbe.expectSubscription()
sub.request(10)
@ -139,7 +139,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
inside(expectRequest) {
case HttpRequest(POST, _, _, HttpEntity.Default(_, 12, data), _)
val dataProbe = StreamTestKit.SubscriberProbe[ByteString]
val dataProbe = TestSubscriber.manualProbe[ByteString]
data.to(Sink(dataProbe)).run()
val sub = dataProbe.expectSubscription()
sub.request(10)
@ -162,7 +162,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
inside(expectRequest) {
case HttpRequest(POST, _, _, HttpEntity.Chunked(_, data), _)
val dataProbe = StreamTestKit.SubscriberProbe[ChunkStreamPart]
val dataProbe = TestSubscriber.manualProbe[ChunkStreamPart]
data.to(Sink(dataProbe)).run()
val sub = dataProbe.expectSubscription()
sub.request(10)
@ -211,7 +211,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
inside(expectRequest) {
case HttpRequest(POST, _, _, HttpEntity.Default(_, 12, data), _)
val dataProbe = StreamTestKit.SubscriberProbe[ByteString]
val dataProbe = TestSubscriber.manualProbe[ByteString]
data.to(Sink(dataProbe)).run()
val sub = dataProbe.expectSubscription()
sub.request(10)
@ -248,7 +248,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
inside(expectRequest) {
case HttpRequest(POST, _, _, HttpEntity.Chunked(_, data), _)
val dataProbe = StreamTestKit.SubscriberProbe[ChunkStreamPart]
val dataProbe = TestSubscriber.manualProbe[ChunkStreamPart]
data.to(Sink(dataProbe)).run()
val sub = dataProbe.expectSubscription()
sub.request(10)
@ -284,7 +284,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
inside(expectRequest) {
case HttpRequest(POST, _, _, HttpEntity.Default(_, 12, data), _)
val dataProbe = StreamTestKit.SubscriberProbe[ByteString]
val dataProbe = TestSubscriber.manualProbe[ByteString]
data.to(Sink(dataProbe)).run()
val sub = dataProbe.expectSubscription()
sub.request(10)
@ -307,7 +307,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
inside(expectRequest) {
case HttpRequest(POST, _, _, HttpEntity.Chunked(_, data), _)
val dataProbe = StreamTestKit.SubscriberProbe[ChunkStreamPart]
val dataProbe = TestSubscriber.manualProbe[ChunkStreamPart]
data.to(Sink(dataProbe)).run()
val sub = dataProbe.expectSubscription()
sub.request(10)
@ -328,7 +328,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
|abcdef""".stripMarginWithNewline("\r\n"))
inside(expectRequest) {
case HttpRequest(POST, _, _, HttpEntity.Default(_, 12, data), _)
val dataProbe = StreamTestKit.SubscriberProbe[ByteString]
val dataProbe = TestSubscriber.manualProbe[ByteString]
data.to(Sink(dataProbe)).run()
val sub = dataProbe.expectSubscription()
sub.request(10)
@ -349,7 +349,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
|""".stripMarginWithNewline("\r\n"))
inside(expectRequest) {
case HttpRequest(POST, _, _, HttpEntity.Chunked(_, data), _)
val dataProbe = StreamTestKit.SubscriberProbe[ChunkStreamPart]
val dataProbe = TestSubscriber.manualProbe[ChunkStreamPart]
data.to(Sink(dataProbe)).run()
val sub = dataProbe.expectSubscription()
sub.request(10)
@ -403,7 +403,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
|Host: example.com
|
|""".stripMarginWithNewline("\r\n"))
val data = StreamTestKit.PublisherProbe[ByteString]
val data = TestPublisher.manualProbe[ByteString]
inside(expectRequest) {
case HttpRequest(GET, _, _, _, _)
responsesSub.sendNext(HttpResponse(entity = HttpEntity.Default(ContentTypes.`text/plain`, 4, Source(data))))
@ -426,7 +426,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
|Host: example.com
|
|""".stripMarginWithNewline("\r\n"))
val data = StreamTestKit.PublisherProbe[ByteString]
val data = TestPublisher.manualProbe[ByteString]
inside(expectRequest) {
case HttpRequest(GET, _, _, _, _)
responsesSub.sendNext(HttpResponse(entity = HttpEntity.CloseDelimited(ContentTypes.`text/plain`, Source(data))))
@ -450,7 +450,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
|Host: example.com
|
|""".stripMarginWithNewline("\r\n"))
val data = StreamTestKit.PublisherProbe[ChunkStreamPart]
val data = TestPublisher.manualProbe[ChunkStreamPart]
inside(expectRequest) {
case HttpRequest(GET, _, _, _, _)
responsesSub.sendNext(HttpResponse(entity = HttpEntity.Chunked(ContentTypes.`text/plain`, Source(data))))
@ -474,7 +474,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
|Connection: close
|
|""".stripMarginWithNewline("\r\n"))
val data = StreamTestKit.PublisherProbe[ByteString]
val data = TestPublisher.manualProbe[ByteString]
inside(expectRequest) {
case HttpRequest(GET, _, _, _, _)
responsesSub.sendNext(HttpResponse(entity = CloseDelimited(ContentTypes.`text/plain`, Source(data))))
@ -495,7 +495,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
|""".stripMarginWithNewline("\r\n"))
inside(expectRequest) {
case HttpRequest(POST, _, _, Default(ContentType(`application/octet-stream`, None), 16, data), _)
val dataProbe = StreamTestKit.SubscriberProbe[ByteString]
val dataProbe = TestSubscriber.manualProbe[ByteString]
data.to(Sink(dataProbe)).run()
val dataSub = dataProbe.expectSubscription()
netOutSub.request(2)
@ -532,7 +532,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
|""".stripMarginWithNewline("\r\n"))
inside(expectRequest) {
case HttpRequest(POST, _, _, Chunked(ContentType(`application/octet-stream`, None), data), _)
val dataProbe = StreamTestKit.SubscriberProbe[ChunkStreamPart]
val dataProbe = TestSubscriber.manualProbe[ChunkStreamPart]
data.to(Sink(dataProbe)).run()
val dataSub = dataProbe.expectSubscription()
netOutSub.request(2)
@ -660,14 +660,14 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
}
class TestSetup {
val requests = StreamTestKit.SubscriberProbe[HttpRequest]
val responses = StreamTestKit.PublisherProbe[HttpResponse]
val requests = TestSubscriber.manualProbe[HttpRequest]
val responses = TestPublisher.manualProbe[HttpResponse]
def settings = ServerSettings(system).copy(serverHeader = Some(Server(List(ProductVersion("akka-http", "test")))))
val (netIn, netOut) = {
val netIn = StreamTestKit.PublisherProbe[ByteString]
val netOut = StreamTestKit.SubscriberProbe[ByteString]
val netIn = TestPublisher.manualProbe[ByteString]
val netOut = TestSubscriber.manualProbe[ByteString]
FlowGraph.closed(HttpServerBluePrint(settings, NoLogging)) { implicit b
server

View file

@ -9,7 +9,7 @@ import scala.concurrent.duration._
import akka.stream.FlowShape
import akka.stream.scaladsl._
import akka.stream.testkit.StreamTestKit
import akka.stream.testkit._
import akka.util.ByteString
import org.scalatest.{ Matchers, FreeSpec }
@ -41,7 +41,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
pushInput(header ++ data1)
val BinaryMessage.Streamed(dataSource) = expectMessage()
val sub = StreamTestKit.SubscriberProbe[ByteString]
val sub = TestSubscriber.manualProbe[ByteString]
dataSource.runWith(Sink(sub))
val s = sub.expectSubscription()
s.request(2)
@ -53,7 +53,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
pushInput(header)
val BinaryMessage.Streamed(dataSource) = expectMessage()
val sub = StreamTestKit.SubscriberProbe[ByteString]
val sub = TestSubscriber.manualProbe[ByteString]
dataSource.runWith(Sink(sub))
val s = sub.expectSubscription()
s.request(2)
@ -72,7 +72,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
pushInput(header1 ++ data1)
val BinaryMessage.Streamed(dataSource) = expectMessage()
val sub = StreamTestKit.SubscriberProbe[ByteString]
val sub = TestSubscriber.manualProbe[ByteString]
dataSource.runWith(Sink(sub))
val s = sub.expectSubscription()
s.request(2)
@ -90,7 +90,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
pushInput(header1 ++ data1)
val BinaryMessage.Streamed(dataSource) = expectMessage()
val sub = StreamTestKit.SubscriberProbe[ByteString]
val sub = TestSubscriber.manualProbe[ByteString]
dataSource.runWith(Sink(sub))
val s = sub.expectSubscription()
s.request(2)
@ -105,7 +105,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
sub.expectComplete()
val BinaryMessage.Streamed(dataSource2) = expectMessage()
val sub2 = StreamTestKit.SubscriberProbe[ByteString]
val sub2 = TestSubscriber.manualProbe[ByteString]
dataSource2.runWith(Sink(sub2))
val s2 = sub2.expectSubscription()
s2.request(2)
@ -125,7 +125,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
pushInput(header ++ data1)
val BinaryMessage.Streamed(dataSource) = expectMessage()
val sub = StreamTestKit.SubscriberProbe[ByteString]
val sub = TestSubscriber.manualProbe[ByteString]
dataSource.runWith(Sink(sub))
val s = sub.expectSubscription()
s.request(2)
@ -161,7 +161,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
pushInput(input)
val TextMessage.Streamed(parts) = expectMessage()
val sub = StreamTestKit.SubscriberProbe[String]
val sub = TestSubscriber.manualProbe[String]
parts.runWith(Sink(sub))
val s = sub.expectSubscription()
s.request(4)
@ -179,7 +179,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
pushInput(header0 ++ data0)
val TextMessage.Streamed(parts) = expectMessage()
val sub = StreamTestKit.SubscriberProbe[String]
val sub = TestSubscriber.manualProbe[String]
parts.runWith(Sink(sub))
val s = sub.expectSubscription()
s.request(4)
@ -198,7 +198,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
pushInput(header ++ data1)
val BinaryMessage.Streamed(dataSource) = expectMessage()
val sub = StreamTestKit.SubscriberProbe[ByteString]
val sub = TestSubscriber.manualProbe[ByteString]
dataSource.runWith(Sink(sub))
val s = sub.expectSubscription()
s.request(2)
@ -223,7 +223,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
"for a strict message larger than configured maximum frame size" in pending
"for a streamed message" in new ServerTestSetup {
val data = ByteString("abcdefg", "ASCII")
val pub = StreamTestKit.PublisherProbe[ByteString]
val pub = TestPublisher.manualProbe[ByteString]
val msg = BinaryMessage.Streamed(Source(pub))
netOutSub.request(6)
pushMessage(msg)
@ -246,7 +246,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
"for a streamed message with a chunk being larger than configured maximum frame size" in pending
"and mask input on the client side" in new ClientTestSetup {
val data = ByteString("abcdefg", "ASCII")
val pub = StreamTestKit.PublisherProbe[ByteString]
val pub = TestPublisher.manualProbe[ByteString]
val msg = BinaryMessage.Streamed(Source(pub))
netOutSub.request(7)
pushMessage(msg)
@ -279,7 +279,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
"for a strict message larger than configured maximum frame size" in pending
"for a streamed message" in new ServerTestSetup {
val text = "äbcd€fg"
val pub = StreamTestKit.PublisherProbe[String]
val pub = TestPublisher.manualProbe[String]
val msg = TextMessage.Streamed(Source(pub))
netOutSub.request(6)
pushMessage(msg)
@ -311,7 +311,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
println(half1(0).toInt.toHexString)
println(half2(0).toInt.toHexString)
val pub = StreamTestKit.PublisherProbe[String]
val pub = TestPublisher.manualProbe[String]
val msg = TextMessage.Streamed(Source(pub))
netOutSub.request(6)
@ -328,7 +328,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
"for a streamed message with a chunk being larger than configured maximum frame size" in pending
"and mask input on the client side" in new ClientTestSetup {
val text = "abcdefg"
val pub = StreamTestKit.PublisherProbe[String]
val pub = TestPublisher.manualProbe[String]
val msg = TextMessage.Streamed(Source(pub))
netOutSub.request(5)
pushMessage(msg)
@ -377,13 +377,13 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
pushInput(input1)
val BinaryMessage.Streamed(dataSource) = expectMessage()
val sub = StreamTestKit.SubscriberProbe[ByteString]
val sub = TestSubscriber.manualProbe[ByteString]
dataSource.runWith(Sink(sub))
val s = sub.expectSubscription()
s.request(2)
sub.expectNext(ByteString("123", "ASCII"))
val outPub = StreamTestKit.PublisherProbe[ByteString]
val outPub = TestPublisher.manualProbe[ByteString]
val msg = BinaryMessage.Streamed(Source(outPub))
netOutSub.request(10)
pushMessage(msg)
@ -460,7 +460,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
messageIn.expectComplete()
// sending another message is allowed before closing (inherently racy)
val pub = StreamTestKit.PublisherProbe[ByteString]
val pub = TestPublisher.manualProbe[ByteString]
val msg = BinaryMessage.Streamed(Source(pub))
pushMessage(msg)
expectFrameOnNetwork(Opcode.Binary, ByteString.empty, fin = false)
@ -485,7 +485,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
pushInput(frameHeader(Protocol.Opcode.Binary, 0, fin = false))
val BinaryMessage.Streamed(dataSource) = messageIn.expectNext()
val inSubscriber = StreamTestKit.SubscriberProbe[ByteString]
val inSubscriber = TestSubscriber.manualProbe[ByteString]
dataSource.runWith(Sink(inSubscriber))
val inSub = inSubscriber.expectSubscription()
@ -502,7 +502,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
// sending another message is allowed before closing (inherently racy)
val pub = StreamTestKit.PublisherProbe[ByteString]
val pub = TestPublisher.manualProbe[ByteString]
val msg = BinaryMessage.Streamed(Source(pub))
pushMessage(msg)
expectFrameOnNetwork(Opcode.Binary, ByteString.empty, fin = false)
@ -547,7 +547,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
messageInSub.request(10)
// send half a message
val pub = StreamTestKit.PublisherProbe[ByteString]
val pub = TestPublisher.manualProbe[ByteString]
val msg = BinaryMessage.Streamed(Source(pub))
pushMessage(msg)
expectFrameOnNetwork(Opcode.Binary, ByteString.empty, fin = false)
@ -763,11 +763,11 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
protected def serverSide: Boolean
protected def closeTimeout: FiniteDuration = 1.second
val netIn = StreamTestKit.PublisherProbe[ByteString]
val netOut = StreamTestKit.SubscriberProbe[ByteString]
val netIn = TestPublisher.manualProbe[ByteString]
val netOut = TestSubscriber.manualProbe[ByteString]
val messageIn = StreamTestKit.SubscriberProbe[Message]
val messageOut = StreamTestKit.PublisherProbe[Message]
val messageIn = TestSubscriber.manualProbe[Message]
val messageOut = TestPublisher.manualProbe[Message]
val messageHandler: Flow[Message, Message, Unit] =
Flow.wrap {

View file

@ -9,7 +9,7 @@ import java.util.concurrent.TimeUnit
import scala.concurrent.duration._
import akka.stream.ActorFlowMaterializer
import akka.stream.scaladsl.{ Flow, Sink, Source }
import akka.stream.testkit.StreamTestKit
import akka.stream.testkit.TestPublisher
import org.reactivestreams.{ Subscriber, Subscription, Processor, Publisher }
import org.reactivestreams.tck.IdentityProcessorVerification
import org.reactivestreams.tck.TestEnvironment
@ -26,7 +26,7 @@ abstract class AkkaIdentityProcessorVerification[T](env: TestEnvironment, publis
def this() = this(false)
override def createFailedPublisher(): Publisher[T] =
StreamTestKit.errorPublisher(new Exception("Unable to serve subscribers right now!"))
TestPublisher.error(new Exception("Unable to serve subscribers right now!"))
def processorFromFlow(flow: Flow[T, T, _])(implicit mat: ActorFlowMaterializer): Processor[T, T] = {
val (sub: Subscriber[T], pub: Publisher[T]) = flow.runWith(Source.subscriber[T], Sink.publisher[T])

View file

@ -10,7 +10,7 @@ import akka.actor.ActorSystem
import akka.stream.ActorFlowMaterializerSettings
import akka.stream.ActorFlowMaterializer
import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.StreamTestKit
import akka.stream.testkit.TestPublisher
import org.reactivestreams.Publisher
import org.reactivestreams.tck.{ PublisherVerification, TestEnvironment }
import org.scalatest.testng.TestNGSuiteLike
@ -30,7 +30,7 @@ abstract class AkkaPublisherVerification[T](val env: TestEnvironment, publisherS
implicit lazy val materializer = ActorFlowMaterializer(ActorFlowMaterializerSettings(system).copy(maxInputBufferSize = 512))(system)
override def createFailedPublisher(): Publisher[T] =
StreamTestKit.errorPublisher(new Exception("Unable to serve subscribers right now!"))
TestPublisher.error(new Exception("Unable to serve subscribers right now!"))
def iterable(elements: Long): immutable.Iterable[Int] =
if (elements > Int.MaxValue)

View file

@ -9,7 +9,8 @@ import akka.actor.ActorSystem
import akka.event.Logging
import akka.stream.io.SynchronousFileSource
import akka.stream.scaladsl.Sink
import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
import akka.stream.testkit._
import akka.stream.testkit.Utils._
import akka.testkit.{ EventFilter, TestEvent }
import akka.util.ByteString
import org.reactivestreams.Publisher
@ -22,7 +23,7 @@ class SynchronousFilePublisherTest extends AkkaPublisherVerification[ByteString]
@BeforeClass
override def createActorSystem(): Unit = {
_system = ActorSystem(Logging.simpleName(getClass), StreamTestKit.UnboundedMailboxConfig.withFallback(AkkaSpec.testConf))
_system = ActorSystem(Logging.simpleName(getClass), UnboundedMailboxConfig.withFallback(AkkaSpec.testConf))
_system.eventStream.publish(TestEvent.Mute(EventFilter[RuntimeException]("Test exception")))
}
@ -45,4 +46,3 @@ class SynchronousFilePublisherTest extends AkkaPublisherVerification[ByteString]
override def maxElementsFromPublisher(): Long = Elements
}

View file

@ -0,0 +1,420 @@
/**
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.stream.testkit
import scala.language.existentials
import akka.actor.ActorSystem
import akka.actor.DeadLetterSuppression
import akka.stream._
import akka.stream.impl._
import akka.stream.impl.StreamLayout.Module
import akka.stream.scaladsl._
import akka.testkit.TestProbe
import org.reactivestreams.{ Publisher, Subscriber, Subscription }
import scala.collection.immutable
import scala.concurrent.duration._
/**
* Provides factory methods for various Publishers.
*/
object TestPublisher {
import StreamTestKit._
/**
* 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.
*/
def manualProbe[T]()(implicit system: ActorSystem): ManualProbe[T] = new ManualProbe()
/**
* 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.
*/
class ManualProbe[I] private[TestPublisher] ()(implicit system: ActorSystem) extends Publisher[I] {
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)
subscriber.onSubscribe(subscription)
}
/**
* 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]])
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()
/**
* 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 {
import StreamTestKit._
/**
* 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.
*/
class ManualProbe[I] private[TestSubscriber] ()(implicit system: ActorSystem) extends Subscriber[I] {
type Self <: ManualProbe[I]
private val probe = TestProbe()
private val self = this.asInstanceOf[Self]
/**
* Expect and return a Subscription.
*/
def expectSubscription(): Subscription = probe.expectMsgType[OnSubscribe].subscription
/**
* Expect [[SubscriberEvent]].
*/
def expectEvent(event: SubscriberEvent): Self = {
probe.expectMsg(event)
self
}
/**
* Expect a data element.
*/
def expectNext(element: I): Self = {
probe.expectMsg(OnNext(element))
self
}
/**
* Expect multiple data elements.
*/
@annotation.varargs def expectNext(e1: I, e2: I, es: I*): Self =
expectNextN((e1 +: e2 +: es).map(identity)(collection.breakOut))
@annotation.varargs def expectNextUnordered(e1: I, e2: I, es: I*): Self =
expectNextUnorderedN((e1 +: e2 +: es).map(identity)(collection.breakOut))
/**
* Expect and return a data element.
*/
def expectNext(): I = probe.expectMsgType[OnNext[I]].element
def expectNextN(all: immutable.Seq[I]): Self = {
all.foreach(e probe.expectMsg(OnNext(e)))
self
}
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
}
/**
* Expect completion.
*/
def expectComplete(): Self = {
probe.expectMsg(OnComplete)
self
}
/**
* Expect given [[Throwable]].
*/
def expectError(cause: Throwable): Self = {
probe.expectMsg(OnError(cause))
self
}
/**
* Expect and return a [[Throwable]].
*/
def expectError(): Throwable = probe.expectMsgType[OnError].cause
def expectSubscriptionAndError(cause: Throwable): Self = {
val sub = expectSubscription()
sub.request(1)
expectError(cause)
self
}
def expectSubscriptionAndError(): Throwable = {
val sub = expectSubscription()
sub.request(1)
expectError()
}
def expectSubscriptionAndComplete(): Self = {
val sub = expectSubscription()
sub.request(1)
expectComplete()
self
}
def expectNextOrError(element: I, cause: Throwable): Either[Throwable, I] = {
probe.fishForMessage(hint = s"OnNext($element) or ${cause.getClass.getName}") {
case OnNext(n) true
case OnError(`cause`) true
} match {
case OnNext(n: I @unchecked) Right(n)
case OnError(err) Left(err)
}
}
def expectNextOrComplete(element: I): Self = {
probe.fishForMessage(hint = s"OnNext($element) or OnComplete") {
case OnNext(n) true
case OnComplete true
}
self
}
def expectNoMsg(): Self = {
probe.expectNoMsg()
self
}
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[SubscriberEvent, T]): immutable.Seq[T] =
probe.receiveWhile(max, idle, messages)(f.asInstanceOf[PartialFunction[AnyRef, T]])
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()
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
}
}
}
/**
* INTERNAL API
*/
private[testkit] object StreamTestKit {
import TestPublisher._
import TestSubscriber._
sealed trait PublisherEvent extends DeadLetterSuppression
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
sealed trait SubscriberEvent extends DeadLetterSuppression
final case class OnSubscribe(subscription: Subscription) extends SubscriberEvent
final case class OnNext[I](element: I) extends SubscriberEvent
final case object OnComplete extends SubscriberEvent
final case class OnError(cause: Throwable) extends SubscriberEvent
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)
}
final class ProbeSource[T](val attributes: OperationAttributes, shape: SourceShape[T])(implicit system: ActorSystem) extends SourceModule[T, TestPublisher.Probe[T]](shape) {
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)
override def withAttributes(attr: OperationAttributes): Module = new ProbeSource[T](attr, amendShape(attr))
}
final class ProbeSink[T](val attributes: OperationAttributes, shape: SinkShape[T])(implicit system: ActorSystem) extends SinkModule[T, TestSubscriber.Probe[T]](shape) {
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)
override def withAttributes(attr: OperationAttributes): Module = new ProbeSink[T](attr, amendShape(attr))
}
}

View file

@ -0,0 +1,19 @@
/**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.stream.testkit.javadsl
import akka.actor.ActorSystem
import akka.stream.javadsl.Sink
import akka.stream.testkit._
/** Java API */
object TestSink {
/**
* A Sink that materialized to a [[TestSubscriber.Probe]].
*/
def probe[T](system: ActorSystem): Sink[T, TestSubscriber.Probe[T]] =
new Sink(scaladsl.TestSink.probe[T]()(system))
}

View file

@ -0,0 +1,19 @@
/**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.stream.testkit.javadsl
import akka.actor.ActorSystem
import akka.stream.javadsl.Source
import akka.stream.testkit._
/** Java API */
object TestSource {
/**
* A Source that materializes to a [[TestPublisher.Probe]].
*/
def probe[T](system: ActorSystem): Source[T, TestPublisher.Probe[T]] =
new Source(scaladsl.TestSource.probe[T]()(system))
}

View file

@ -0,0 +1,24 @@
/**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.stream.testkit.scaladsl
import akka.stream._
import akka.stream.impl._
import akka.stream.OperationAttributes.none
import akka.stream.scaladsl._
import akka.stream.testkit._
import akka.actor.ActorSystem;
/**
* Factory methods for test sinks.
*/
object TestSink {
/**
* A Sink that materialized to a [[TestSubscriber.Probe]].
*/
def probe[T]()(implicit system: ActorSystem) = new Sink[T, TestSubscriber.Probe[T]](new StreamTestKit.ProbeSink(none, SinkShape(new Inlet("ProbeSink.in"))))
}

View file

@ -0,0 +1,24 @@
/**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.stream.testkit.scaladsl
import akka.stream._
import akka.stream.impl._
import akka.stream.OperationAttributes.none
import akka.stream.scaladsl._
import akka.stream.testkit._
import akka.actor.ActorSystem;
/**
* Factory methods for test sources.
*/
object TestSource {
/**
* A Source that materializes to a [[TestPublisher.Probe]].
*/
def probe[T]()(implicit system: ActorSystem) = new Source[T, TestPublisher.Probe[T]](new StreamTestKit.ProbeSource(none, SourceShape(new Outlet("ProbeSource.out"))))
}

View file

@ -22,8 +22,8 @@ class ChainSetup[In, Out](
toPublisher: (Source[Out, _], ActorFlowMaterializer) Publisher[Out])(implicit system: ActorSystem) =
this(stream, settings, materializerCreator(settings, system), toPublisher)(system)
val upstream = StreamTestKit.PublisherProbe[In]()
val downstream = StreamTestKit.SubscriberProbe[Out]()
val upstream = TestPublisher.manualProbe[In]()
val downstream = TestSubscriber.probe[Out]()
private val s = Source(upstream).via(stream(Flow[In].map(x x).named("buh")))
val publisher = toPublisher(s, materializer)
val upstreamSubscription = upstream.expectSubscription()

View file

@ -6,6 +6,7 @@ package akka.stream.testkit
import akka.actor.ActorSystem
import akka.stream.ActorFlowMaterializerSettings
import akka.stream.scaladsl.{ Sink, Source, Flow }
import akka.stream.testkit._
import akka.stream.testkit.StreamTestKit._
import org.reactivestreams.Publisher
import org.scalatest.Matchers
@ -121,14 +122,14 @@ trait ScriptedTest extends Matchers {
def mayRequestMore: Boolean = remainingDemand > 0
def shakeIt(): Boolean = {
val u = upstream.probe.receiveWhile(1.milliseconds) {
val u = upstream.receiveWhile(1.milliseconds) {
case RequestMore(_, n)
debugLog(s"operation requests $n")
pendingRequests += n
true
case _ false // Ignore
}
val d = downstream.probe.receiveWhile(1.milliseconds) {
val d = downstream.receiveWhile(1.milliseconds) {
case OnNext(elem: Out @unchecked)
debugLog(s"operation produces [$elem]")
if (outstandingDemand == 0) fail("operation produced while there was no demand")

View file

@ -1,206 +0,0 @@
/**
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.stream.testkit
import akka.stream.FlowMaterializer
import com.typesafe.config.ConfigFactory
import scala.language.existentials
import scala.concurrent.duration._
import akka.actor.ActorSystem
import akka.stream.impl.{ StreamSupervisor, ActorFlowMaterializerImpl, EmptyPublisher, ErrorPublisher }
import akka.testkit.TestProbe
import org.reactivestreams.{ Publisher, Subscriber, Subscription }
import scala.concurrent.duration.FiniteDuration
import akka.actor.DeadLetterSuppression
import scala.util.control.NoStackTrace
import akka.stream.FlowMaterializer
import akka.stream.impl.ActorFlowMaterializerImpl
import akka.stream.impl.StreamSupervisor
import akka.actor.ActorRef
object StreamTestKit {
/** Sets the default-mailbox to the usual [[akka.dispatch.UnboundedMailbox]] instead of [[StreamTestDefaultMailbox]]. */
val UnboundedMailboxConfig = ConfigFactory.parseString("""akka.actor.default-mailbox.mailbox-type = "akka.dispatch.UnboundedMailbox"""")
/**
* Subscribes the subscriber and completes after the first request.
*/
def lazyEmptyPublisher[T]: Publisher[T] = new Publisher[T] {
override def subscribe(subscriber: Subscriber[_ >: T]): Unit =
subscriber.onSubscribe(CompletedSubscription(subscriber))
}
/**
* Signals error to subscribers immediately, before handing out subscription.
*/
def errorPublisher[T](cause: Throwable): Publisher[T] = ErrorPublisher(cause, "error").asInstanceOf[Publisher[T]]
def emptyPublisher[T](): Publisher[T] = EmptyPublisher[T]
/**
* Subscribes the subscriber and signals error after the first request.
*/
def lazyErrorPublisher[T](cause: Throwable): Publisher[T] = new Publisher[T] {
override def subscribe(subscriber: Subscriber[_ >: T]): Unit =
subscriber.onSubscribe(FailedSubscription(subscriber, cause))
}
private 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 = ()
}
private final case class CompletedSubscription[T](subscriber: Subscriber[T]) extends Subscription {
override def request(elements: Long): Unit = subscriber.onComplete()
override def cancel(): Unit = ()
}
class AutoPublisher[T](probe: PublisherProbe[T], initialPendingRequests: Long = 0) {
val subscription = probe.expectSubscription()
var pendingRequests = initialPendingRequests
def sendNext(elem: T): Unit = {
if (pendingRequests == 0) pendingRequests = subscription.expectRequest()
pendingRequests -= 1
subscription.sendNext(elem)
}
def sendComplete(): Unit = subscription.sendComplete()
def sendError(cause: Exception): Unit = subscription.sendError(cause)
}
sealed trait SubscriberEvent extends DeadLetterSuppression
final case class OnSubscribe(subscription: Subscription) extends SubscriberEvent
final case class OnNext[I](element: I) extends SubscriberEvent
final case object OnComplete extends SubscriberEvent
final case class OnError(cause: Throwable) extends SubscriberEvent
sealed trait PublisherEvent extends DeadLetterSuppression
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
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)
}
object SubscriberProbe {
def apply[I]()(implicit system: ActorSystem): SubscriberProbe[I] = new SubscriberProbe[I]()
}
class SubscriberProbe[I]()(implicit system: ActorSystem) extends Subscriber[I] {
val probe = TestProbe()
def expectSubscription(): Subscription = probe.expectMsgType[OnSubscribe].subscription
def expectEvent(event: SubscriberEvent): Unit = probe.expectMsg(event)
def expectNext(element: I): Unit = probe.expectMsg(OnNext(element))
def expectNext(e1: I, e2: I, es: I*): Unit = {
val all = e1 +: e2 +: es
all.foreach(e probe.expectMsg(OnNext(e)))
}
def expectNext(): I = probe.expectMsgType[OnNext[I]].element
def expectComplete(): Unit = probe.expectMsg(OnComplete)
def expectError(cause: Throwable): Unit = probe.expectMsg(OnError(cause))
def expectError(): Throwable = probe.expectMsgType[OnError].cause
def expectSubscriptionAndError(cause: Throwable): Unit = {
val sub = expectSubscription()
sub.request(1)
expectError(cause)
}
def expectSubscriptionAndError(): Throwable = {
val sub = expectSubscription()
sub.request(1)
expectError()
}
def expectSubscriptionAndComplete(): Unit = {
val sub = expectSubscription()
sub.request(1)
expectComplete()
}
def expectNextOrError(element: I, cause: Throwable): Either[Throwable, I] = {
probe.fishForMessage(hint = s"OnNext($element) or ${cause.getClass.getName}") {
case OnNext(n) true
case OnError(`cause`) true
} match {
case OnNext(n: I @unchecked) Right(n)
case OnError(err) Left(err)
}
}
def expectNoMsg(): Unit = probe.expectNoMsg()
def expectNoMsg(max: FiniteDuration): Unit = probe.expectNoMsg(max)
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)
}
object PublisherProbe {
def apply[I]()(implicit system: ActorSystem): PublisherProbe[I] = new PublisherProbe[I]()
}
class PublisherProbe[I]()(implicit system: ActorSystem) extends Publisher[I] {
val probe: TestProbe = TestProbe()
def subscribe(subscriber: Subscriber[_ >: I]): Unit = {
val subscription: PublisherProbeSubscription[I] = new PublisherProbeSubscription[I](subscriber, probe)
probe.ref ! Subscribe(subscription)
subscriber.onSubscribe(subscription)
}
def expectSubscription(): PublisherProbeSubscription[I] =
probe.expectMsgType[Subscribe].subscription.asInstanceOf[PublisherProbeSubscription[I]]
def expectRequest(subscription: Subscription, n: Int): Unit = probe.expectMsg(RequestMore(subscription, n))
def expectNoMsg(): Unit = probe.expectNoMsg()
def expectNoMsg(max: FiniteDuration): Unit = probe.expectNoMsg(max)
def getPublisher: Publisher[I] = this
}
case class TE(message: String) extends RuntimeException(message) with NoStackTrace
def assertAllStagesStopped[T](block: T)(implicit materializer: FlowMaterializer): T =
materializer match {
case impl: ActorFlowMaterializerImpl
impl.supervisor ! StreamSupervisor.StopChildren
val result = block
val probe = TestProbe()(impl.system)
probe.within(5.seconds) {
probe.awaitAssert {
impl.supervisor.tell(StreamSupervisor.GetChildren, probe.ref)
val children = probe.expectMsgType[StreamSupervisor.Children].children
assert(children.isEmpty,
s"expected no StreamSupervisor children, but got [${children.mkString(", ")}]")
}
}
result
case _ block
}
}

View file

@ -5,7 +5,7 @@ import akka.stream.scaladsl._
import org.reactivestreams.Publisher
import scala.collection.immutable
import scala.util.control.NoStackTrace
import akka.stream.testkit.StreamTestKit.assertAllStagesStopped
import akka.stream.testkit.Utils._
abstract class TwoStreamsSetup extends AkkaSpec {
@ -27,7 +27,7 @@ abstract class TwoStreamsSetup extends AkkaSpec {
def fixture(b: FlowGraph.Builder[_]): Fixture
def setup(p1: Publisher[Int], p2: Publisher[Int]) = {
val subscriber = StreamTestKit.SubscriberProbe[Outputs]()
val subscriber = TestSubscriber.probe[Outputs]()
FlowGraph.closed() { implicit b
import FlowGraph.Implicits._
val f = fixture(b)
@ -41,15 +41,15 @@ abstract class TwoStreamsSetup extends AkkaSpec {
subscriber
}
def failedPublisher[T]: Publisher[T] = StreamTestKit.errorPublisher[T](TestException)
def failedPublisher[T]: Publisher[T] = TestPublisher.error[T](TestException)
def completedPublisher[T]: Publisher[T] = StreamTestKit.emptyPublisher[T]
def completedPublisher[T]: Publisher[T] = TestPublisher.empty[T]
def nonemptyPublisher[T](elems: immutable.Iterable[T]): Publisher[T] = Source(elems).runWith(Sink.publisher)
def soonToFailPublisher[T]: Publisher[T] = StreamTestKit.lazyErrorPublisher[T](TestException)
def soonToFailPublisher[T]: Publisher[T] = TestPublisher.lazyError[T](TestException)
def soonToCompletePublisher[T]: Publisher[T] = StreamTestKit.lazyEmptyPublisher[T]
def soonToCompletePublisher[T]: Publisher[T] = TestPublisher.lazyEmpty[T]
def commonTests() = {
"work with two immediately completed publishers" in assertAllStagesStopped {

View file

@ -0,0 +1,36 @@
package akka.stream.testkit
import akka.actor.ActorRef
import akka.stream.FlowMaterializer
import akka.stream.impl._
import akka.testkit.TestProbe
import com.typesafe.config.ConfigFactory
import scala.concurrent.duration._
import scala.util.control.NoStackTrace
object Utils {
/** Sets the default-mailbox to the usual [[akka.dispatch.UnboundedMailbox]] instead of [[StreamTestDefaultMailbox]]. */
val UnboundedMailboxConfig = ConfigFactory.parseString("""akka.actor.default-mailbox.mailbox-type = "akka.dispatch.UnboundedMailbox"""")
case class TE(message: String) extends RuntimeException(message) with NoStackTrace
def assertAllStagesStopped[T](block: T)(implicit materializer: FlowMaterializer): T =
materializer match {
case impl: ActorFlowMaterializerImpl
impl.supervisor ! StreamSupervisor.StopChildren
val result = block
val probe = TestProbe()(impl.system)
probe.within(5.seconds) {
probe.awaitAssert {
impl.supervisor.tell(StreamSupervisor.GetChildren, probe.ref)
val children = probe.expectMsgType[StreamSupervisor.Children].children
assert(children.isEmpty,
s"expected no StreamSupervisor children, but got [${children.mkString(", ")}]")
}
}
result
case _ block
}
}

View file

@ -6,7 +6,8 @@ package akka.stream.actor
import akka.actor.{ ActorRef, PoisonPill, Props }
import akka.stream.{ ActorFlowMaterializer, ActorFlowMaterializerSettings, ActorOperationAttributes }
import akka.stream.scaladsl._
import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
import akka.stream.testkit._
import akka.stream.testkit.Utils._
import akka.testkit.TestEvent.Mute
import akka.testkit.{ EventFilter, ImplicitSender, TestProbe }
@ -136,37 +137,35 @@ class ActorPublisherSpec extends AkkaSpec(ActorPublisherSpec.config) with Implic
val probe = TestProbe()
val ref = system.actorOf(testPublisherProps(probe.ref))
val p = ActorPublisher[String](ref)
val s = StreamTestKit.SubscriberProbe[String]()
val s = TestSubscriber.probe[String]()
p.subscribe(s)
val sub = s.expectSubscription
sub.request(2)
s.request(2)
probe.expectMsg(TotalDemand(2))
sub.request(3)
s.request(3)
probe.expectMsg(TotalDemand(5))
sub.cancel()
s.cancel()
}
"allow onNext up to requested elements, but not more" in {
val probe = TestProbe()
val ref = system.actorOf(testPublisherProps(probe.ref))
val p = ActorPublisher[String](ref)
val s = StreamTestKit.SubscriberProbe[String]()
val s = TestSubscriber.probe[String]()
p.subscribe(s)
val sub = s.expectSubscription
sub.request(2)
s.request(2)
ref ! Produce("elem-1")
ref ! Produce("elem-2")
ref ! Produce("elem-3")
s.expectNext("elem-1")
s.expectNext("elem-2")
s.expectNoMsg(300.millis)
sub.cancel()
s.cancel()
}
"signal error" in {
val probe = TestProbe()
val ref = system.actorOf(testPublisherProps(probe.ref))
val s = StreamTestKit.SubscriberProbe[String]()
val s = TestSubscriber.manualProbe[String]()
ActorPublisher[String](ref).subscribe(s)
ref ! Err("wrong")
s.expectSubscription
@ -176,7 +175,7 @@ class ActorPublisherSpec extends AkkaSpec(ActorPublisherSpec.config) with Implic
"not terminate after signalling onError" in {
val probe = TestProbe()
val ref = system.actorOf(testPublisherProps(probe.ref))
val s = StreamTestKit.SubscriberProbe[String]()
val s = TestSubscriber.manualProbe[String]()
ActorPublisher[String](ref).subscribe(s)
s.expectSubscription
probe.watch(ref)
@ -188,7 +187,7 @@ class ActorPublisherSpec extends AkkaSpec(ActorPublisherSpec.config) with Implic
"terminate after signalling onErrorThenStop" in {
val probe = TestProbe()
val ref = system.actorOf(testPublisherProps(probe.ref))
val s = StreamTestKit.SubscriberProbe[String]()
val s = TestSubscriber.manualProbe[String]()
ActorPublisher[String](ref).subscribe(s)
s.expectSubscription
probe.watch(ref)
@ -201,7 +200,7 @@ class ActorPublisherSpec extends AkkaSpec(ActorPublisherSpec.config) with Implic
val probe = TestProbe()
val ref = system.actorOf(testPublisherProps(probe.ref))
ref ! Err("early err")
val s = StreamTestKit.SubscriberProbe[String]()
val s = TestSubscriber.manualProbe[String]()
ActorPublisher[String](ref).subscribe(s)
s.expectSubscriptionAndError.getMessage should be("early err")
}
@ -210,12 +209,11 @@ class ActorPublisherSpec extends AkkaSpec(ActorPublisherSpec.config) with Implic
val probe = TestProbe()
val ref = system.actorOf(testPublisherProps(probe.ref))
val p = ActorPublisher[String](ref)
val s = StreamTestKit.SubscriberProbe[String]()
val s = TestSubscriber.probe[String]()
p.subscribe(s)
val sub = s.expectSubscription
sub.request(2)
s.request(2)
ref ! Produce("elem-1")
sub.cancel()
s.cancel()
ref ! Produce("elem-2")
s.expectNext("elem-1")
s.expectNoMsg(300.millis)
@ -225,28 +223,26 @@ class ActorPublisherSpec extends AkkaSpec(ActorPublisherSpec.config) with Implic
val probe = TestProbe()
val ref = system.actorOf(testPublisherProps(probe.ref))
val p = ActorPublisher[String](ref)
val s = StreamTestKit.SubscriberProbe[String]()
val s = TestSubscriber.probe[String]()
p.subscribe(s)
val sub = s.expectSubscription
sub.request(3)
s.request(3)
probe.expectMsg(TotalDemand(3))
ref ! Produce("elem-1")
ref ! Boom
ref ! Produce("elem-2")
s.expectNext("elem-1")
s.expectNext("elem-2")
sub.request(5)
s.request(5)
probe.expectMsg(TotalDemand(6))
sub.cancel()
s.cancel()
}
"signal onComplete" in {
val probe = TestProbe()
val ref = system.actorOf(testPublisherProps(probe.ref))
val s = StreamTestKit.SubscriberProbe[String]()
val s = TestSubscriber.probe[String]()
ActorPublisher[String](ref).subscribe(s)
val sub = s.expectSubscription
sub.request(3)
s.request(3)
ref ! Produce("elem-1")
ref ! Complete
s.expectNext("elem-1")
@ -256,7 +252,7 @@ class ActorPublisherSpec extends AkkaSpec(ActorPublisherSpec.config) with Implic
"not terminate after signalling onComplete" in {
val probe = TestProbe()
val ref = system.actorOf(testPublisherProps(probe.ref))
val s = StreamTestKit.SubscriberProbe[String]()
val s = TestSubscriber.manualProbe[String]()
ActorPublisher[String](ref).subscribe(s)
val sub = s.expectSubscription
sub.request(3)
@ -272,7 +268,7 @@ class ActorPublisherSpec extends AkkaSpec(ActorPublisherSpec.config) with Implic
"terminate after signalling onCompleteThenStop" in {
val probe = TestProbe()
val ref = system.actorOf(testPublisherProps(probe.ref))
val s = StreamTestKit.SubscriberProbe[String]()
val s = TestSubscriber.manualProbe[String]()
ActorPublisher[String](ref).subscribe(s)
val sub = s.expectSubscription
sub.request(3)
@ -289,7 +285,7 @@ class ActorPublisherSpec extends AkkaSpec(ActorPublisherSpec.config) with Implic
val probe = TestProbe()
val ref = system.actorOf(testPublisherProps(probe.ref))
ref ! Complete
val s = StreamTestKit.SubscriberProbe[String]()
val s = TestSubscriber.manualProbe[String]()
ActorPublisher[String](ref).subscribe(s)
s.expectSubscriptionAndComplete
}
@ -297,10 +293,10 @@ class ActorPublisherSpec extends AkkaSpec(ActorPublisherSpec.config) with Implic
"only allow one subscriber" in {
val probe = TestProbe()
val ref = system.actorOf(testPublisherProps(probe.ref))
val s = StreamTestKit.SubscriberProbe[String]()
val s = TestSubscriber.manualProbe[String]()
ActorPublisher[String](ref).subscribe(s)
s.expectSubscription
val s2 = StreamTestKit.SubscriberProbe[String]()
val s2 = TestSubscriber.manualProbe[String]()
ActorPublisher[String](ref).subscribe(s2)
s2.expectSubscriptionAndError.getClass should be(classOf[IllegalStateException])
}
@ -308,7 +304,7 @@ class ActorPublisherSpec extends AkkaSpec(ActorPublisherSpec.config) with Implic
"signal onCompete when actor is stopped" in {
val probe = TestProbe()
val ref = system.actorOf(testPublisherProps(probe.ref))
val s = StreamTestKit.SubscriberProbe[String]()
val s = TestSubscriber.manualProbe[String]()
ActorPublisher[String](ref).subscribe(s)
s.expectSubscription
ref ! PoisonPill
@ -317,7 +313,7 @@ class ActorPublisherSpec extends AkkaSpec(ActorPublisherSpec.config) with Implic
"work together with Flow and ActorSubscriber" in {
implicit val materializer = ActorFlowMaterializer()
StreamTestKit.assertAllStagesStopped {
assertAllStagesStopped {
val probe = TestProbe()
val source: Source[Int, ActorRef] = Source.actorPublisher(senderProps)
@ -383,7 +379,7 @@ class ActorPublisherSpec extends AkkaSpec(ActorPublisherSpec.config) with Implic
"be able to define a subscription-timeout, after which it should shut down" in {
implicit val materializer = ActorFlowMaterializer()
StreamTestKit.assertAllStagesStopped {
Utils.assertAllStagesStopped {
val timeout = 150.millis
val a = system.actorOf(timeoutingProps(testActor, timeout))
val pub = ActorPublisher(a)
@ -392,7 +388,7 @@ class ActorPublisherSpec extends AkkaSpec(ActorPublisherSpec.config) with Implic
expectMsg("timed-out")
// now subscribers will already be rejected, while the actor could perform some clean-up
val sub = StreamTestKit.SubscriberProbe()
val sub = TestSubscriber.manualProbe()
pub.subscribe(sub)
sub.expectSubscriptionAndError()
@ -406,7 +402,7 @@ class ActorPublisherSpec extends AkkaSpec(ActorPublisherSpec.config) with Implic
"be able to define a subscription-timeout, which is cancelled by the first incoming Subscriber" in {
implicit val materializer = ActorFlowMaterializer()
val timeout = 500.millis
val sub = StreamTestKit.SubscriberProbe[Int]()
val sub = TestSubscriber.manualProbe[Int]()
within(2 * timeout) {
val pub = ActorPublisher(system.actorOf(timeoutingProps(testActor, timeout)))
@ -422,7 +418,7 @@ class ActorPublisherSpec extends AkkaSpec(ActorPublisherSpec.config) with Implic
"use dispatcher from materializer settings" in {
implicit val materializer = ActorFlowMaterializer(
ActorFlowMaterializerSettings(system).withDispatcher("my-dispatcher1"))
val s = StreamTestKit.SubscriberProbe[String]()
val s = TestSubscriber.manualProbe[String]()
val ref = Source.actorPublisher(testPublisherProps(testActor, useTestDispatcher = false)).to(Sink(s)).run()
ref ! ThreadName
expectMsgType[String] should include("my-dispatcher1")
@ -430,7 +426,7 @@ class ActorPublisherSpec extends AkkaSpec(ActorPublisherSpec.config) with Implic
"use dispatcher from operation attributes" in {
implicit val materializer = ActorFlowMaterializer()
val s = StreamTestKit.SubscriberProbe[String]()
val s = TestSubscriber.manualProbe[String]()
val ref = Source.actorPublisher(testPublisherProps(testActor, useTestDispatcher = false))
.withAttributes(ActorOperationAttributes.dispatcher("my-dispatcher1"))
.to(Sink(s)).run()
@ -440,7 +436,7 @@ class ActorPublisherSpec extends AkkaSpec(ActorPublisherSpec.config) with Implic
"use dispatcher from props" in {
implicit val materializer = ActorFlowMaterializer()
val s = StreamTestKit.SubscriberProbe[String]()
val s = TestSubscriber.manualProbe[String]()
val ref = Source.actorPublisher(testPublisherProps(testActor, useTestDispatcher = false).withDispatcher("my-dispatcher1"))
.withAttributes(ActorOperationAttributes.dispatcher("my-dispatcher2"))
.to(Sink(s)).run()

View file

@ -6,7 +6,8 @@ package akka.stream.extra
import akka.stream.{ ActorFlowMaterializerSettings, ActorFlowMaterializer }
import akka.stream.scaladsl.{ Source, Flow }
import akka.stream.scaladsl.Sink
import akka.stream.testkit.{ AkkaSpec, ScriptedTest, StreamTestKit }
import akka.stream.testkit._
import akka.stream.testkit.Utils._
import akka.testkit.TestProbe
import org.reactivestreams.{ Publisher, Subscriber }
@ -72,12 +73,12 @@ class FlowTimedSpec extends AkkaSpec with ScriptedTest {
"Timed Flow" must {
import akka.stream.extra.Implicits.TimedFlowDsl
"measure time it between elements matching a predicate" in StreamTestKit.assertAllStagesStopped {
"measure time it between elements matching a predicate" in assertAllStagesStopped {
val probe = TestProbe()
val flow: Flow[Int, Long, _] = Flow[Int].map(_.toLong).timedIntervalBetween(in in % 2 == 1, d probe.ref ! d)
val c1 = StreamTestKit.SubscriberProbe[Long]()
val c1 = TestSubscriber.manualProbe[Long]()
Source(List(1, 2, 3)).via(flow).runWith(Sink(c1))
val s = c1.expectSubscription()
@ -91,7 +92,7 @@ class FlowTimedSpec extends AkkaSpec with ScriptedTest {
info(s"Got duration (first): $duration")
}
"measure time from start to complete, by wrapping operations" in StreamTestKit.assertAllStagesStopped {
"measure time from start to complete, by wrapping operations" in assertAllStagesStopped {
val probe = TestProbe()
// making sure the types come out as expected
@ -105,7 +106,7 @@ class FlowTimedSpec extends AkkaSpec with ScriptedTest {
val (flowIn: Subscriber[Int], flowOut: Publisher[String]) = flow.runWith(Source.subscriber[Int], Sink.publisher[String])
val c1 = StreamTestKit.SubscriberProbe[String]()
val c1 = TestSubscriber.manualProbe[String]()
val c2 = flowOut.subscribe(c1)
val p = Source(0 to 100).runWith(Sink.publisher)
@ -122,4 +123,3 @@ class FlowTimedSpec extends AkkaSpec with ScriptedTest {
}
}

View file

@ -7,7 +7,7 @@ import akka.stream.Supervision._
import akka.stream.testkit.AkkaSpec
import akka.stream._
import akka.stream.scaladsl._
import akka.stream.testkit.StreamTestKit._
import akka.stream.testkit._
import akka.stream.impl.fusing.ActorInterpreter
import akka.stream.stage.Stage
import akka.stream.stage.PushPullStage
@ -22,8 +22,8 @@ class ActorInterpreterSpec extends AkkaSpec {
implicit val mat = ActorFlowMaterializer()
class Setup(ops: List[Stage[_, _]] = List(fusing.Map({ x: Any x }, stoppingDecider))) {
val up = PublisherProbe[Int]
val down = SubscriberProbe[Int]
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)

View file

@ -6,13 +6,13 @@ package akka.stream.io
import java.io.InputStream
import akka.stream.scaladsl.Sink
import akka.stream.testkit.StreamTestKit._
import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
import akka.stream.testkit._
import akka.stream.testkit.Utils._
import akka.stream.{ ActorFlowMaterializer, ActorFlowMaterializerSettings }
import akka.util.ByteString
import org.scalatest.concurrent.ScalaFutures
class InputStreamSourceSpec extends AkkaSpec(StreamTestKit.UnboundedMailboxConfig) with ScalaFutures {
class InputStreamSourceSpec extends AkkaSpec(UnboundedMailboxConfig) with ScalaFutures {
val settings = ActorFlowMaterializerSettings(system).withDispatcher("akka.actor.default-dispatcher")
implicit val materializer = ActorFlowMaterializer(settings)
@ -39,4 +39,3 @@ class InputStreamSourceSpec extends AkkaSpec(StreamTestKit.UnboundedMailboxConfi
}
}

View file

@ -6,8 +6,8 @@ package akka.stream.io
import java.io.OutputStream
import akka.stream.scaladsl.Source
import akka.stream.testkit.StreamTestKit._
import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
import akka.stream.testkit._
import akka.stream.testkit.Utils._
import akka.stream.{ ActorFlowMaterializer, ActorFlowMaterializerSettings }
import akka.testkit.TestProbe
import akka.util.ByteString
@ -15,7 +15,7 @@ import akka.util.ByteString
import scala.concurrent.Await
import scala.concurrent.duration._
class OutputStreamSinkSpec extends AkkaSpec(StreamTestKit.UnboundedMailboxConfig) {
class OutputStreamSinkSpec extends AkkaSpec(UnboundedMailboxConfig) {
val settings = ActorFlowMaterializerSettings(system).withDispatcher("akka.actor.default-dispatcher")
implicit val materializer = ActorFlowMaterializer(settings)
@ -63,4 +63,3 @@ class OutputStreamSinkSpec extends AkkaSpec(StreamTestKit.UnboundedMailboxConfig
}
}

View file

@ -14,10 +14,10 @@ import scala.concurrent.Await
import scala.concurrent.duration._
import akka.util.ByteString
import akka.stream.scaladsl.Flow
import akka.stream.testkit.{ StreamTestKit, AkkaSpec }
import akka.stream.testkit._
import akka.stream.testkit.Utils._
import akka.stream.scaladsl._
import akka.stream.testkit.TestUtils.temporaryServerAddress
import akka.stream.testkit.StreamTestKit.assertAllStagesStopped
class StreamTcpSpec extends AkkaSpec with TcpHelper {
import akka.stream.io.TcpHelper._
@ -400,18 +400,18 @@ class StreamTcpSpec extends AkkaSpec with TcpHelper {
"bind and unbind correctly" in {
val address = temporaryServerAddress()
val probe1 = StreamTestKit.SubscriberProbe[StreamTcp.IncomingConnection]()
val probe1 = TestSubscriber.manualProbe[StreamTcp.IncomingConnection]()
val bind = StreamTcp(system).bind(address)
// Bind succeeded, we have a local address
val binding1 = Await.result(bind.to(Sink(probe1)).run(), 3.second)
probe1.expectSubscription()
val probe2 = StreamTestKit.SubscriberProbe[StreamTcp.IncomingConnection]()
val probe2 = TestSubscriber.manualProbe[StreamTcp.IncomingConnection]()
val binding2F = bind.to(Sink(probe2)).run()
probe2.expectSubscriptionAndError(BindFailedException)
val probe3 = StreamTestKit.SubscriberProbe[StreamTcp.IncomingConnection]()
val probe3 = TestSubscriber.manualProbe[StreamTcp.IncomingConnection]()
val binding3F = bind.to(Sink(probe3)).run()
probe3.expectSubscriptionAndError()
@ -422,7 +422,7 @@ class StreamTcpSpec extends AkkaSpec with TcpHelper {
Await.result(binding1.unbind(), 1.second)
probe1.expectComplete()
val probe4 = StreamTestKit.SubscriberProbe[StreamTcp.IncomingConnection]()
val probe4 = TestSubscriber.manualProbe[StreamTcp.IncomingConnection]()
// Bind succeeded, we have a local address
val binding4 = Await.result(bind.to(Sink(probe4)).run(), 3.second)
probe4.expectSubscription()

View file

@ -7,8 +7,8 @@ import java.io.File
import akka.actor.{ ActorCell, ActorSystem, RepointableActorRef }
import akka.stream.scaladsl.Source
import akka.stream.testkit.StreamTestKit._
import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
import akka.stream.testkit._
import akka.stream.testkit.Utils._
import akka.stream.{ ActorFlowMaterializer, ActorFlowMaterializerSettings, ActorOperationAttributes }
import akka.util.{ ByteString, Timeout }
@ -16,7 +16,7 @@ import scala.collection.mutable.ListBuffer
import scala.concurrent.Await
import scala.concurrent.duration._
class SynchronousFileSinkSpec extends AkkaSpec(StreamTestKit.UnboundedMailboxConfig) {
class SynchronousFileSinkSpec extends AkkaSpec(UnboundedMailboxConfig) {
val settings = ActorFlowMaterializerSettings(system).withDispatcher("akka.actor.default-dispatcher")
implicit val materializer = ActorFlowMaterializer(settings)
@ -86,7 +86,7 @@ class SynchronousFileSinkSpec extends AkkaSpec(StreamTestKit.UnboundedMailboxCon
"use dedicated file-io-dispatcher by default" in assertAllStagesStopped {
targetFile { f
val sys = ActorSystem("dispatcher-testing", StreamTestKit.UnboundedMailboxConfig)
val sys = ActorSystem("dispatcher-testing", UnboundedMailboxConfig)
val mat = ActorFlowMaterializer()(sys)
implicit val timeout = Timeout(3.seconds)
@ -101,7 +101,7 @@ class SynchronousFileSinkSpec extends AkkaSpec(StreamTestKit.UnboundedMailboxCon
"allow overriding the dispatcher using OperationAttributes" in assertAllStagesStopped {
targetFile { f
val sys = ActorSystem("dispatcher-testing", StreamTestKit.UnboundedMailboxConfig)
val sys = ActorSystem("dispatcher-testing", UnboundedMailboxConfig)
val mat = ActorFlowMaterializer()(sys)
implicit val timeout = Timeout(3.seconds)
@ -132,4 +132,3 @@ class SynchronousFileSinkSpec extends AkkaSpec(StreamTestKit.UnboundedMailboxCon
}
}

View file

@ -9,8 +9,8 @@ import java.util.Random
import akka.actor.{ ActorCell, RepointableActorRef, ActorSystem }
import akka.stream.io.SynchronousFileSourceSpec.Settings
import akka.stream.scaladsl.Sink
import akka.stream.testkit.StreamTestKit._
import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
import akka.stream.testkit._
import akka.stream.testkit.Utils._
import akka.stream.{ ActorOperationAttributes, ActorFlowMaterializer, ActorFlowMaterializerSettings, OperationAttributes }
import akka.util.{ Timeout, ByteString }
@ -21,7 +21,7 @@ object SynchronousFileSourceSpec {
final case class Settings(chunkSize: Int, readAhead: Int)
}
class SynchronousFileSourceSpec extends AkkaSpec(StreamTestKit.UnboundedMailboxConfig) {
class SynchronousFileSourceSpec extends AkkaSpec(UnboundedMailboxConfig) {
val settings = ActorFlowMaterializerSettings(system).withDispatcher("akka.actor.default-dispatcher")
implicit val materializer = ActorFlowMaterializer(settings)
@ -68,7 +68,7 @@ class SynchronousFileSourceSpec extends AkkaSpec(StreamTestKit.UnboundedMailboxC
val p = SynchronousFileSource(testFile, chunkSize)
.withAttributes(bufferAttributes)
.runWith(Sink.publisher)
val c = StreamTestKit.SubscriberProbe[ByteString]()
val c = TestSubscriber.manualProbe[ByteString]()
p.subscribe(c)
val sub = c.expectSubscription()
@ -106,7 +106,7 @@ class SynchronousFileSourceSpec extends AkkaSpec(StreamTestKit.UnboundedMailboxC
.withAttributes(bufferAttributes)
.runWith(Sink.publisher)
val c = StreamTestKit.SubscriberProbe[ByteString]()
val c = TestSubscriber.manualProbe[ByteString]()
p.subscribe(c)
val sub = c.expectSubscription()
@ -132,7 +132,7 @@ class SynchronousFileSourceSpec extends AkkaSpec(StreamTestKit.UnboundedMailboxC
"onError whent trying to read from file which does not exist" in assertAllStagesStopped {
val p = SynchronousFileSource(notExistingFile).runWith(Sink.publisher)
val c = StreamTestKit.SubscriberProbe[ByteString]()
val c = TestSubscriber.manualProbe[ByteString]()
p.subscribe(c)
c.expectSubscription()
@ -158,7 +158,7 @@ class SynchronousFileSourceSpec extends AkkaSpec(StreamTestKit.UnboundedMailboxC
}
"use dedicated file-io-dispatcher by default" in {
val sys = ActorSystem("dispatcher-testing", StreamTestKit.UnboundedMailboxConfig)
val sys = ActorSystem("dispatcher-testing", UnboundedMailboxConfig)
val mat = ActorFlowMaterializer()(sys)
implicit val timeout = Timeout(500.millis)
@ -171,7 +171,7 @@ class SynchronousFileSourceSpec extends AkkaSpec(StreamTestKit.UnboundedMailboxC
}
"allow overriding the dispatcher using OperationAttributes" in {
val sys = ActorSystem("dispatcher-testing", StreamTestKit.UnboundedMailboxConfig)
val sys = ActorSystem("dispatcher-testing", UnboundedMailboxConfig)
val mat = ActorFlowMaterializer()(sys)
implicit val timeout = Timeout(500.millis)
@ -192,4 +192,3 @@ class SynchronousFileSourceSpec extends AkkaSpec(StreamTestKit.UnboundedMailboxC
}
}

View file

@ -6,7 +6,7 @@ package akka.stream.io
import akka.actor.{ Actor, ActorRef, Props }
import akka.io.Tcp.{ ResumeReading, Register, ConnectionClosed, Closed }
import akka.io.{ IO, Tcp }
import akka.stream.testkit.StreamTestKit
import akka.stream.testkit._
import akka.stream.{ ActorFlowMaterializer, ActorFlowMaterializerSettings }
import akka.testkit.{ TestKitBase, TestProbe }
import akka.util.ByteString
@ -158,7 +158,7 @@ trait TcpHelper { this: TestKitBase ⇒
}
class TcpReadProbe() {
val subscriberProbe = StreamTestKit.SubscriberProbe[ByteString]()
val subscriberProbe = TestSubscriber.manualProbe[ByteString]()
lazy val tcpReadSubscription = subscriberProbe.expectSubscription()
def read(count: Int): ByteString = {
@ -174,7 +174,7 @@ trait TcpHelper { this: TestKitBase ⇒
}
class TcpWriteProbe() {
val publisherProbe = StreamTestKit.PublisherProbe[ByteString]()
val publisherProbe = TestPublisher.manualProbe[ByteString]()
lazy val tcpWriteSubscription = publisherProbe.expectSubscription()
var demand = 0L

View file

@ -4,12 +4,13 @@
package akka.stream.scaladsl
import akka.stream.ActorFlowMaterializer
import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.StreamTestKit
import akka.stream.testkit.StreamTestKit.assertAllStagesStopped
import akka.stream.testkit._
import akka.stream.testkit.Utils._
import akka.stream.testkit.scaladsl._
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.Props
import akka.stream.OverflowStrategy
object ActorRefSinkSpec {
case class Fw(ref: ActorRef) extends Actor {
@ -34,16 +35,12 @@ class ActorRefSinkSpec extends AkkaSpec {
}
"cancel stream when actor terminates" in assertAllStagesStopped {
val publisher = StreamTestKit.PublisherProbe[Int]()
val fw = system.actorOf(Props(classOf[Fw], testActor).withDispatcher("akka.test.stream-dispatcher"))
Source(publisher).runWith(Sink.actorRef(fw, onCompleteMessage = "done"))
val autoPublisher = new StreamTestKit.AutoPublisher(publisher)
autoPublisher.sendNext(1)
autoPublisher.sendNext(2)
val publisher = TestSource.probe[Int].to(Sink.actorRef(fw, onCompleteMessage = "done")).run().sendNext(1).sendNext(2)
expectMsg(1)
expectMsg(2)
system.stop(fw)
autoPublisher.subscription.expectCancellation()
publisher.expectCancellation()
}
}

View file

@ -6,11 +6,10 @@ package akka.stream.scaladsl
import scala.concurrent.duration._
import akka.stream.ActorFlowMaterializer
import akka.stream.OverflowStrategy
import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.StreamTestKit
import akka.stream.testkit._
import akka.stream.testkit.Utils._
import akka.actor.PoisonPill
import akka.actor.Status
import akka.stream.testkit.StreamTestKit.assertAllStagesStopped
class ActorRefSourceSpec extends AkkaSpec {
implicit val mat = ActorFlowMaterializer()
@ -18,7 +17,7 @@ class ActorRefSourceSpec extends AkkaSpec {
"A ActorRefSource" must {
"emit received messages to the stream" in {
val s = StreamTestKit.SubscriberProbe[Int]()
val s = TestSubscriber.manualProbe[Int]()
val ref = Source.actorRef(10, OverflowStrategy.fail).to(Sink(s)).run()
val sub = s.expectSubscription
sub.request(2)
@ -31,7 +30,7 @@ class ActorRefSourceSpec extends AkkaSpec {
}
"buffer when needed" in {
val s = StreamTestKit.SubscriberProbe[Int]()
val s = TestSubscriber.manualProbe[Int]()
val ref = Source.actorRef(100, OverflowStrategy.dropHead).to(Sink(s)).run()
val sub = s.expectSubscription
for (n 1 to 20) ref ! n
@ -46,7 +45,7 @@ class ActorRefSourceSpec extends AkkaSpec {
}
"terminate when the stream is cancelled" in assertAllStagesStopped {
val s = StreamTestKit.SubscriberProbe[Int]()
val s = TestSubscriber.manualProbe[Int]()
val ref = Source.actorRef(0, OverflowStrategy.fail).to(Sink(s)).run()
watch(ref)
val sub = s.expectSubscription
@ -55,7 +54,7 @@ class ActorRefSourceSpec extends AkkaSpec {
}
"complete the stream when receiving PoisonPill" in assertAllStagesStopped {
val s = StreamTestKit.SubscriberProbe[Int]()
val s = TestSubscriber.manualProbe[Int]()
val ref = Source.actorRef(10, OverflowStrategy.fail).to(Sink(s)).run()
val sub = s.expectSubscription
ref ! PoisonPill
@ -63,7 +62,7 @@ class ActorRefSourceSpec extends AkkaSpec {
}
"complete the stream when receiving Status.Success" in assertAllStagesStopped {
val s = StreamTestKit.SubscriberProbe[Int]()
val s = TestSubscriber.manualProbe[Int]()
val ref = Source.actorRef(10, OverflowStrategy.fail).to(Sink(s)).run()
val sub = s.expectSubscription
ref ! Status.Success("ok")
@ -71,10 +70,10 @@ class ActorRefSourceSpec extends AkkaSpec {
}
"fail the stream when receiving Status.Failure" in assertAllStagesStopped {
val s = StreamTestKit.SubscriberProbe[Int]()
val s = TestSubscriber.manualProbe[Int]()
val ref = Source.actorRef(10, OverflowStrategy.fail).to(Sink(s)).run()
val sub = s.expectSubscription
val exc = StreamTestKit.TE("testfailure")
val exc = TE("testfailure")
ref ! Status.Failure(exc)
s.expectError(exc)
}

View file

@ -4,6 +4,7 @@
package akka.stream.scaladsl
import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.Utils._
import org.scalactic.ConversionCheckedTripleEquals
import akka.util.ByteString
import akka.stream.BidiShape
@ -12,7 +13,6 @@ import scala.concurrent.Await
import scala.concurrent.duration._
import scala.collection.immutable
import akka.stream.OperationAttributes
import akka.stream.testkit.StreamTestKit.assertAllStagesStopped
class BidiFlowSpec extends AkkaSpec with ConversionCheckedTripleEquals {
import OperationAttributes._

View file

@ -6,7 +6,7 @@ package akka.stream.scaladsl
import akka.actor.ActorSystem
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
import akka.stream.testkit.{ AkkaSpec, TestSubscriber }
import org.reactivestreams.Subscriber
import org.scalatest.Matchers
@ -49,13 +49,15 @@ trait River { self: Matchers ⇒
val otherFlow = Flow[Int].map(_.toString)
def riverOf[T](flowConstructor: Subscriber[T] Unit)(implicit system: ActorSystem) = {
val subscriber = StreamTestKit.SubscriberProbe[T]()
val subscriber = TestSubscriber.manualProbe[T]()
flowConstructor(subscriber)
val subscription = subscriber.expectSubscription()
subscription.request(elements.size)
subscriber.probe.receiveN(elements.size) should be(elements.map(_.toString).map(StreamTestKit.OnNext(_)))
elements.foreach { el
subscriber.expectNext() shouldBe el.toString
}
subscription.request(1)
subscriber.expectComplete()
}

View file

@ -11,8 +11,8 @@ import akka.stream.ActorFlowMaterializer
import akka.stream.ActorFlowMaterializerSettings
import akka.stream.OverflowStrategy
import akka.stream.OverflowStrategy.Fail.BufferOverflowException
import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
import akka.stream.testkit.StreamTestKit.assertAllStagesStopped
import akka.stream.testkit._
import akka.stream.testkit.Utils._
class FlowBufferSpec extends AkkaSpec {
@ -50,16 +50,14 @@ class FlowBufferSpec extends AkkaSpec {
}
"accept elements that fit in the buffer while downstream is silent" in {
val publisher = StreamTestKit.PublisherProbe[Int]()
val subscriber = StreamTestKit.SubscriberProbe[Int]()
val publisher = TestPublisher.probe[Int]()
val subscriber = TestSubscriber.manualProbe[Int]()
Source(publisher).buffer(100, overflowStrategy = OverflowStrategy.backpressure).runWith(Sink(subscriber))
val autoPublisher = new StreamTestKit.AutoPublisher(publisher)
Source(publisher).buffer(100, overflowStrategy = OverflowStrategy.backpressure).to(Sink(subscriber)).run()
val sub = subscriber.expectSubscription()
// Fill up buffer
for (i 1 to 100) autoPublisher.sendNext(i)
for (i 1 to 100) publisher.sendNext(i)
// drain
for (i 1 to 100) {
@ -70,16 +68,14 @@ class FlowBufferSpec extends AkkaSpec {
}
"drop head elements if buffer is full and configured so" in {
val publisher = StreamTestKit.PublisherProbe[Int]()
val subscriber = StreamTestKit.SubscriberProbe[Int]()
val publisher = TestPublisher.probe[Int]()
val subscriber = TestSubscriber.manualProbe[Int]()
Source(publisher).buffer(100, overflowStrategy = OverflowStrategy.dropHead).runWith(Sink(subscriber))
val autoPublisher = new StreamTestKit.AutoPublisher(publisher)
Source(publisher).buffer(100, overflowStrategy = OverflowStrategy.dropHead).to(Sink(subscriber)).run()
val sub = subscriber.expectSubscription()
// Fill up buffer
for (i 1 to 200) autoPublisher.sendNext(i)
for (i 1 to 200) publisher.sendNext(i)
// drain
for (i 101 to 200) {
@ -90,7 +86,7 @@ class FlowBufferSpec extends AkkaSpec {
sub.request(1)
subscriber.expectNoMsg(1.seconds)
autoPublisher.sendNext(-1)
publisher.sendNext(-1)
sub.request(1)
subscriber.expectNext(-1)
@ -98,16 +94,14 @@ class FlowBufferSpec extends AkkaSpec {
}
"drop tail elements if buffer is full and configured so" in {
val publisher = StreamTestKit.PublisherProbe[Int]()
val subscriber = StreamTestKit.SubscriberProbe[Int]()
val publisher = TestPublisher.probe[Int]()
val subscriber = TestSubscriber.manualProbe[Int]()
Source(publisher).buffer(100, overflowStrategy = OverflowStrategy.dropTail).runWith(Sink(subscriber))
val autoPublisher = new StreamTestKit.AutoPublisher(publisher)
Source(publisher).buffer(100, overflowStrategy = OverflowStrategy.dropTail).to(Sink(subscriber)).run()
val sub = subscriber.expectSubscription()
// Fill up buffer
for (i 1 to 200) autoPublisher.sendNext(i)
for (i 1 to 200) publisher.sendNext(i)
// drain
for (i 1 to 99) {
@ -121,7 +115,7 @@ class FlowBufferSpec extends AkkaSpec {
sub.request(1)
subscriber.expectNoMsg(1.seconds)
autoPublisher.sendNext(-1)
publisher.sendNext(-1)
sub.request(1)
subscriber.expectNext(-1)
@ -129,16 +123,14 @@ class FlowBufferSpec extends AkkaSpec {
}
"drop all elements if buffer is full and configured so" in {
val publisher = StreamTestKit.PublisherProbe[Int]
val subscriber = StreamTestKit.SubscriberProbe[Int]()
val publisher = TestPublisher.probe[Int]()
val subscriber = TestSubscriber.manualProbe[Int]()
Source(publisher).buffer(100, overflowStrategy = OverflowStrategy.dropBuffer).runWith(Sink(subscriber))
val autoPublisher = new StreamTestKit.AutoPublisher(publisher)
Source(publisher).buffer(100, overflowStrategy = OverflowStrategy.dropBuffer).to(Sink(subscriber)).run()
val sub = subscriber.expectSubscription()
// Fill up buffer
for (i 1 to 150) autoPublisher.sendNext(i)
for (i 1 to 150) publisher.sendNext(i)
// drain
for (i 101 to 150) {
@ -149,7 +141,7 @@ class FlowBufferSpec extends AkkaSpec {
sub.request(1)
subscriber.expectNoMsg(1.seconds)
autoPublisher.sendNext(-1)
publisher.sendNext(-1)
sub.request(1)
subscriber.expectNext(-1)
@ -157,16 +149,14 @@ class FlowBufferSpec extends AkkaSpec {
}
"fail upstream if buffer is full and configured so" in assertAllStagesStopped {
val publisher = StreamTestKit.PublisherProbe[Int]
val subscriber = StreamTestKit.SubscriberProbe[Int]()
val publisher = TestPublisher.probe[Int]()
val subscriber = TestSubscriber.manualProbe[Int]()
Source(publisher).buffer(100, overflowStrategy = OverflowStrategy.fail).runWith(Sink(subscriber))
val autoPublisher = new StreamTestKit.AutoPublisher(publisher)
Source(publisher).buffer(100, overflowStrategy = OverflowStrategy.fail).to(Sink(subscriber)).run()
val sub = subscriber.expectSubscription()
// Fill up buffer
for (i 1 to 100) autoPublisher.sendNext(i)
for (i 1 to 100) publisher.sendNext(i)
// drain
for (i 1 to 10) {
@ -175,9 +165,9 @@ class FlowBufferSpec extends AkkaSpec {
}
// overflow the buffer
for (i 101 to 111) autoPublisher.sendNext(i)
for (i 101 to 111) publisher.sendNext(i)
autoPublisher.subscription.expectCancellation()
publisher.expectCancellation()
val error = new BufferOverflowException("Buffer overflow (max capacity was: 100)!")
subscriber.expectError(error)
}
@ -186,16 +176,14 @@ class FlowBufferSpec extends AkkaSpec {
s"work with $strategy if buffer size of one" in {
val publisher = StreamTestKit.PublisherProbe[Int]
val subscriber = StreamTestKit.SubscriberProbe[Int]()
val publisher = TestPublisher.probe[Int]()
val subscriber = TestSubscriber.manualProbe[Int]()
Source(publisher).buffer(1, overflowStrategy = strategy).runWith(Sink(subscriber))
val autoPublisher = new StreamTestKit.AutoPublisher(publisher)
Source(publisher).buffer(1, overflowStrategy = strategy).to(Sink(subscriber)).run()
val sub = subscriber.expectSubscription()
// Fill up buffer
for (i 1 to 200) autoPublisher.sendNext(i)
for (i 1 to 200) publisher.sendNext(i)
sub.request(1)
subscriber.expectNext(200)
@ -203,7 +191,7 @@ class FlowBufferSpec extends AkkaSpec {
sub.request(1)
subscriber.expectNoMsg(1.seconds)
autoPublisher.sendNext(-1)
publisher.sendNext(-1)
sub.request(1)
subscriber.expectNext(-1)

View file

@ -7,8 +7,8 @@ import scala.concurrent.duration._
import scala.util.control.NoStackTrace
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.{ StreamTestKit, AkkaSpec }
import akka.stream.testkit.StreamTestKit.assertAllStagesStopped
import akka.stream.testkit._
import akka.stream.testkit.Utils._
class FlowConcatAllSpec extends AkkaSpec {
@ -30,35 +30,37 @@ class FlowConcatAllSpec extends AkkaSpec {
val main = Source(List(s1, s2, s3, s4, s5))
val subscriber = StreamTestKit.SubscriberProbe[Int]()
val subscriber = TestSubscriber.manualProbe[Int]()
main.flatten(FlattenStrategy.concat).to(Sink(subscriber)).run()
val subscription = subscriber.expectSubscription()
subscription.request(10)
subscriber.probe.receiveN(10) should be((1 to 10).map(StreamTestKit.OnNext(_)))
for (i 1 to 10)
subscriber.expectNext() shouldBe i
subscription.request(1)
subscriber.expectComplete()
}
"work together with SplitWhen" in {
val subscriber = StreamTestKit.SubscriberProbe[Int]()
val subscriber = TestSubscriber.manualProbe[Int]()
Source(1 to 10).splitWhen(_ % 2 == 0).flatten(FlattenStrategy.concat).runWith(Sink(subscriber))
val subscription = subscriber.expectSubscription()
subscription.request(10)
subscriber.probe.receiveN(10) should be((1 to 10).map(StreamTestKit.OnNext(_)))
for (i (1 to 10))
subscriber.expectNext() shouldBe i
subscription.request(1)
subscriber.expectComplete()
}
"on onError on master stream cancel the current open substream and signal error" in assertAllStagesStopped {
val publisher = StreamTestKit.PublisherProbe[Source[Int, _]]()
val subscriber = StreamTestKit.SubscriberProbe[Int]()
val publisher = TestPublisher.manualProbe[Source[Int, _]]()
val subscriber = TestSubscriber.manualProbe[Int]()
Source(publisher).flatten(FlattenStrategy.concat).to(Sink(subscriber)).run()
val upstream = publisher.expectSubscription()
val downstream = subscriber.expectSubscription()
downstream.request(1000)
val substreamPublisher = StreamTestKit.PublisherProbe[Int]()
val substreamPublisher = TestPublisher.manualProbe[Int]()
val substreamFlow = Source(substreamPublisher)
upstream.expectRequest()
upstream.sendNext(substreamFlow)
@ -70,15 +72,15 @@ class FlowConcatAllSpec extends AkkaSpec {
}
"on onError on open substream, cancel the master stream and signal error " in assertAllStagesStopped {
val publisher = StreamTestKit.PublisherProbe[Source[Int, _]]()
val subscriber = StreamTestKit.SubscriberProbe[Int]()
val publisher = TestPublisher.manualProbe[Source[Int, _]]()
val subscriber = TestSubscriber.manualProbe[Int]()
Source(publisher).flatten(FlattenStrategy.concat).to(Sink(subscriber)).run()
val upstream = publisher.expectSubscription()
val downstream = subscriber.expectSubscription()
downstream.request(1000)
val substreamPublisher = StreamTestKit.PublisherProbe[Int]()
val substreamPublisher = TestPublisher.manualProbe[Int]()
val substreamFlow = Source(substreamPublisher)
upstream.expectRequest()
upstream.sendNext(substreamFlow)
@ -90,15 +92,15 @@ class FlowConcatAllSpec extends AkkaSpec {
}
"on cancellation cancel the current open substream and the master stream" in assertAllStagesStopped {
val publisher = StreamTestKit.PublisherProbe[Source[Int, _]]()
val subscriber = StreamTestKit.SubscriberProbe[Int]()
val publisher = TestPublisher.manualProbe[Source[Int, _]]()
val subscriber = TestSubscriber.manualProbe[Int]()
Source(publisher).flatten(FlattenStrategy.concat).to(Sink(subscriber)).run()
val upstream = publisher.expectSubscription()
val downstream = subscriber.expectSubscription()
downstream.request(1000)
val substreamPublisher = StreamTestKit.PublisherProbe[Int]()
val substreamPublisher = TestPublisher.manualProbe[Int]()
val substreamFlow = Source(substreamPublisher)
upstream.expectRequest()
upstream.sendNext(substreamFlow)
@ -111,8 +113,8 @@ class FlowConcatAllSpec extends AkkaSpec {
}
"pass along early cancellation" in assertAllStagesStopped {
val up = StreamTestKit.PublisherProbe[Source[Int, _]]()
val down = StreamTestKit.SubscriberProbe[Int]()
val up = TestPublisher.manualProbe[Source[Int, _]]()
val down = TestSubscriber.manualProbe[Int]()
val flowSubscriber = Source.subscriber[Source[Int, _]].flatten(FlattenStrategy.concat).to(Sink(down)).run()

View file

@ -7,7 +7,7 @@ import scala.concurrent.Await
import scala.concurrent.duration._
import scala.concurrent.forkjoin.ThreadLocalRandom
import akka.stream.{ OverflowStrategy, ActorFlowMaterializer, ActorFlowMaterializerSettings }
import akka.stream.testkit.{ StreamTestKit, AkkaSpec }
import akka.stream.testkit._
class FlowConflateSpec extends AkkaSpec {
@ -19,17 +19,15 @@ class FlowConflateSpec extends AkkaSpec {
"Conflate" must {
"pass-through elements unchanged when there is no rate difference" in {
val publisher = StreamTestKit.PublisherProbe[Int]()
val subscriber = StreamTestKit.SubscriberProbe[Int]()
val publisher = TestPublisher.probe[Int]()
val subscriber = TestSubscriber.manualProbe[Int]()
Source(publisher).conflate(seed = i i)(aggregate = (sum, i) sum + i).runWith(Sink(subscriber))
val autoPublisher = new StreamTestKit.AutoPublisher(publisher)
Source(publisher).conflate(seed = i i)(aggregate = (sum, i) sum + i).to(Sink(subscriber)).run()
val sub = subscriber.expectSubscription()
for (i 1 to 100) {
sub.request(1)
autoPublisher.sendNext(i)
publisher.sendNext(i)
subscriber.expectNext(i)
}
@ -37,16 +35,14 @@ class FlowConflateSpec extends AkkaSpec {
}
"conflate elements while downstream is silent" in {
val publisher = StreamTestKit.PublisherProbe[Int]()
val subscriber = StreamTestKit.SubscriberProbe[Int]()
val publisher = TestPublisher.probe[Int]()
val subscriber = TestSubscriber.manualProbe[Int]()
Source(publisher).conflate(seed = i i)(aggregate = (sum, i) sum + i).runWith(Sink(subscriber))
val autoPublisher = new StreamTestKit.AutoPublisher(publisher)
Source(publisher).conflate(seed = i i)(aggregate = (sum, i) sum + i).to(Sink(subscriber)).run()
val sub = subscriber.expectSubscription()
for (i 1 to 100) {
autoPublisher.sendNext(i)
publisher.sendNext(i)
}
subscriber.expectNoMsg(1.second)
sub.request(1)
@ -63,25 +59,23 @@ class FlowConflateSpec extends AkkaSpec {
}
"backpressure subscriber when upstream is slower" in {
val publisher = StreamTestKit.PublisherProbe[Int]()
val subscriber = StreamTestKit.SubscriberProbe[Int]()
val publisher = TestPublisher.probe[Int]()
val subscriber = TestSubscriber.manualProbe[Int]()
Source(publisher).conflate(seed = i i)(aggregate = (sum, i) sum + i).runWith(Sink(subscriber))
val autoPublisher = new StreamTestKit.AutoPublisher(publisher)
Source(publisher).conflate(seed = i i)(aggregate = (sum, i) sum + i).to(Sink(subscriber)).run()
val sub = subscriber.expectSubscription()
sub.request(1)
autoPublisher.sendNext(1)
publisher.sendNext(1)
subscriber.expectNext(1)
sub.request(1)
subscriber.expectNoMsg(1.second)
autoPublisher.sendNext(2)
publisher.sendNext(2)
subscriber.expectNext(2)
autoPublisher.sendNext(3)
autoPublisher.sendNext(4)
publisher.sendNext(3)
publisher.sendNext(4)
sub.request(1)
subscriber.expectNext(7)

View file

@ -7,9 +7,7 @@ import scala.concurrent.forkjoin.ThreadLocalRandom.{ current ⇒ random }
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.ScriptedTest
import akka.stream.testkit.StreamTestKit
import akka.stream.testkit._
class FlowDropSpec extends AkkaSpec with ScriptedTest {
@ -29,7 +27,7 @@ class FlowDropSpec extends AkkaSpec with ScriptedTest {
}
"not drop anything for negative n" in {
val probe = StreamTestKit.SubscriberProbe[Int]()
val probe = TestSubscriber.manualProbe[Int]()
Source(List(1, 2, 3)).drop(-1).to(Sink(probe)).run()
probe.expectSubscription().request(10)
probe.expectNext(1)

View file

@ -6,8 +6,7 @@ package akka.stream.scaladsl
import scala.concurrent.duration._
import akka.stream.ActorFlowMaterializer
import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.StreamTestKit
import akka.stream.testkit._
class FlowDropWithinSpec extends AkkaSpec {
@ -17,8 +16,8 @@ class FlowDropWithinSpec extends AkkaSpec {
"deliver elements after the duration, but not before" in {
val input = Iterator.from(1)
val p = StreamTestKit.PublisherProbe[Int]()
val c = StreamTestKit.SubscriberProbe[Int]()
val p = TestPublisher.manualProbe[Int]()
val c = TestSubscriber.manualProbe[Int]()
Source(p).dropWithin(1.second).to(Sink(c)).run()
val pSub = p.expectSubscription
val cSub = c.expectSubscription

View file

@ -10,7 +10,7 @@ import scala.concurrent.forkjoin.ThreadLocalRandom
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.{ StreamTestKit, AkkaSpec }
import akka.stream.testkit._
class FlowExpandSpec extends AkkaSpec {
@ -22,68 +22,54 @@ class FlowExpandSpec extends AkkaSpec {
"Expand" must {
"pass-through elements unchanged when there is no rate difference" in {
val publisher = StreamTestKit.PublisherProbe[Int]()
val subscriber = StreamTestKit.SubscriberProbe[Int]()
val publisher = TestPublisher.probe[Int]()
val subscriber = TestSubscriber.probe[Int]()
// Simply repeat the last element as an extrapolation step
Source(publisher).expand(seed = i i)(extrapolate = i (i, i)).runWith(Sink(subscriber))
val autoPublisher = new StreamTestKit.AutoPublisher(publisher)
val sub = subscriber.expectSubscription()
Source(publisher).expand(seed = i i)(extrapolate = i (i, i)).to(Sink(subscriber)).run()
for (i 1 to 100) {
// Order is important here: If the request comes first it will be extrapolated!
autoPublisher.sendNext(i)
sub.request(1)
subscriber.expectNext(i)
publisher.sendNext(i)
subscriber.requestNext(i)
}
sub.cancel()
subscriber.cancel()
}
"expand elements while upstream is silent" in {
val publisher = StreamTestKit.PublisherProbe[Int]()
val subscriber = StreamTestKit.SubscriberProbe[Int]()
val publisher = TestPublisher.probe[Int]()
val subscriber = TestSubscriber.probe[Int]()
// Simply repeat the last element as an extrapolation step
Source(publisher).expand(seed = i i)(extrapolate = i (i, i)).runWith(Sink(subscriber))
Source(publisher).expand(seed = i i)(extrapolate = i (i, i)).to(Sink(subscriber)).run()
val autoPublisher = new StreamTestKit.AutoPublisher(publisher)
val sub = subscriber.expectSubscription()
autoPublisher.sendNext(42)
publisher.sendNext(42)
for (i 1 to 100) {
sub.request(1)
subscriber.expectNext(42)
subscriber.requestNext(42)
}
autoPublisher.sendNext(-42)
sub.request(1)
subscriber.expectNext(-42)
publisher.sendNext(-42)
subscriber.requestNext(-42)
sub.cancel()
subscriber.cancel()
}
"do not drop last element" in {
val publisher = StreamTestKit.PublisherProbe[Int]()
val subscriber = StreamTestKit.SubscriberProbe[Int]()
val publisher = TestPublisher.probe[Int]()
val subscriber = TestSubscriber.probe[Int]()
// Simply repeat the last element as an extrapolation step
Source(publisher).expand(seed = i i)(extrapolate = i (i, i)).runWith(Sink(subscriber))
Source(publisher).expand(seed = i i)(extrapolate = i (i, i)).to(Sink(subscriber)).run()
val autoPublisher = new StreamTestKit.AutoPublisher(publisher)
val sub = subscriber.expectSubscription()
publisher.sendNext(1)
subscriber.requestNext(1)
autoPublisher.sendNext(1)
sub.request(1)
subscriber.expectNext(1)
publisher.sendNext(2)
publisher.sendComplete()
autoPublisher.sendNext(2)
autoPublisher.sendComplete()
sub.request(1)
subscriber.expectNext(2)
subscriber.requestNext(2)
subscriber.expectComplete()
}
@ -97,43 +83,38 @@ class FlowExpandSpec extends AkkaSpec {
}
"backpressure publisher when subscriber is slower" in {
val publisher = StreamTestKit.PublisherProbe[Int]()
val subscriber = StreamTestKit.SubscriberProbe[Int]()
val publisher = TestPublisher.probe[Int]()
val subscriber = TestSubscriber.probe[Int]()
Source(publisher).expand(seed = i i)(extrapolate = i (i, i)).runWith(Sink(subscriber))
Source(publisher).expand(seed = i i)(extrapolate = i (i, i)).to(Sink(subscriber)).run()
val autoPublisher = new StreamTestKit.AutoPublisher(publisher)
val sub = subscriber.expectSubscription()
publisher.sendNext(1)
subscriber.requestNext(1)
subscriber.requestNext(1)
autoPublisher.sendNext(1)
sub.request(1)
subscriber.expectNext(1)
sub.request(1)
subscriber.expectNext(1)
var pending = autoPublisher.pendingRequests
var pending = publisher.pending
// Deplete pending requests coming from input buffer
while (pending > 0) {
autoPublisher.subscription.sendNext(2)
publisher.unsafeSendNext(2)
pending -= 1
}
// The above sends are absorbed in the input buffer, and will result in two one-sized batch requests
pending += autoPublisher.subscription.expectRequest()
pending += autoPublisher.subscription.expectRequest()
pending += publisher.expectRequest()
pending += publisher.expectRequest()
while (pending > 0) {
autoPublisher.subscription.sendNext(2)
publisher.unsafeSendNext(2)
pending -= 1
}
publisher.expectNoMsg(1.second)
sub.request(2)
subscriber.request(2)
subscriber.expectNext(2)
subscriber.expectNext(2)
// Now production is resumed
autoPublisher.subscription.expectRequest()
publisher.expectRequest()
}
}

View file

@ -7,8 +7,7 @@ import scala.concurrent.forkjoin.ThreadLocalRandom.{ current ⇒ random }
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
import akka.stream.testkit.ScriptedTest
import akka.stream.testkit._
class FlowFilterSpec extends AkkaSpec with ScriptedTest {
@ -27,7 +26,7 @@ class FlowFilterSpec extends AkkaSpec with ScriptedTest {
.withInputBuffer(initialSize = 1, maxSize = 1)
implicit val materializer = ActorFlowMaterializer(settings)
val probe = StreamTestKit.SubscriberProbe[Int]()
val probe = TestSubscriber.manualProbe[Int]()
Source(List.fill(1000)(0) ::: List(1)).filter(_ != 0).runWith(Sink(probe))
val subscription = probe.expectSubscription()

View file

@ -8,7 +8,7 @@ import scala.util.control.NoStackTrace
import akka.stream.{ OverflowStrategy, ActorFlowMaterializer }
import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.StreamTestKit.assertAllStagesStopped
import akka.stream.testkit.Utils._
class FlowFoldSpec extends AkkaSpec {
implicit val mat = ActorFlowMaterializer()

View file

@ -5,9 +5,9 @@ package akka.stream.scaladsl
import scala.util.control.NoStackTrace
import akka.stream.ActorFlowMaterializer
import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
import akka.stream.testkit._
import akka.stream.testkit.Utils._
import scala.concurrent.Await
import akka.stream.testkit.StreamTestKit.assertAllStagesStopped
class FlowForeachSpec extends AkkaSpec {
@ -34,7 +34,7 @@ class FlowForeachSpec extends AkkaSpec {
}
"yield the first error" in assertAllStagesStopped {
val p = StreamTestKit.PublisherProbe[Int]()
val p = TestPublisher.manualProbe[Int]()
Source(p).runForeach(testActor ! _) onFailure {
case ex testActor ! ex
}

View file

@ -9,9 +9,8 @@ import scala.util.control.NoStackTrace
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.StreamTestKit.assertAllStagesStopped
import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
import akka.stream.testkit._
import akka.stream.testkit.Utils._
class FlowFromFutureSpec extends AkkaSpec {
@ -22,7 +21,7 @@ class FlowFromFutureSpec extends AkkaSpec {
"A Flow based on a Future" must {
"produce one element from already successful Future" in assertAllStagesStopped {
val p = Source(Future.successful(1)).runWith(Sink.publisher)
val c = StreamTestKit.SubscriberProbe[Int]()
val c = TestSubscriber.manualProbe[Int]()
p.subscribe(c)
val sub = c.expectSubscription()
c.expectNoMsg(100.millis)
@ -34,7 +33,7 @@ class FlowFromFutureSpec extends AkkaSpec {
"produce error from already failed Future" in assertAllStagesStopped {
val ex = new RuntimeException("test") with NoStackTrace
val p = Source(Future.failed[Int](ex)).runWith(Sink.publisher)
val c = StreamTestKit.SubscriberProbe[Int]()
val c = TestSubscriber.manualProbe[Int]()
p.subscribe(c)
c.expectSubscriptionAndError(ex)
}
@ -42,7 +41,7 @@ class FlowFromFutureSpec extends AkkaSpec {
"produce one element when Future is completed" in assertAllStagesStopped {
val promise = Promise[Int]()
val p = Source(promise.future).runWith(Sink.publisher)
val c = StreamTestKit.SubscriberProbe[Int]()
val c = TestSubscriber.manualProbe[Int]()
p.subscribe(c)
val sub = c.expectSubscription()
sub.request(1)
@ -56,7 +55,7 @@ class FlowFromFutureSpec extends AkkaSpec {
"produce one element when Future is completed but not before request" in {
val promise = Promise[Int]()
val p = Source(promise.future).runWith(Sink.publisher)
val c = StreamTestKit.SubscriberProbe[Int]()
val c = TestSubscriber.manualProbe[Int]()
p.subscribe(c)
val sub = c.expectSubscription()
promise.success(1)
@ -69,8 +68,8 @@ class FlowFromFutureSpec extends AkkaSpec {
"produce elements with multiple subscribers" in assertAllStagesStopped {
val promise = Promise[Int]()
val p = Source(promise.future).runWith(Sink.fanoutPublisher(1, 1))
val c1 = StreamTestKit.SubscriberProbe[Int]()
val c2 = StreamTestKit.SubscriberProbe[Int]()
val c1 = TestSubscriber.manualProbe[Int]()
val c2 = TestSubscriber.manualProbe[Int]()
p.subscribe(c1)
p.subscribe(c2)
val sub1 = c1.expectSubscription()
@ -87,8 +86,8 @@ class FlowFromFutureSpec extends AkkaSpec {
"allow cancel before receiving element" in {
val promise = Promise[Int]()
val p = Source(promise.future).runWith(Sink.fanoutPublisher(1, 1))
val keepAlive = StreamTestKit.SubscriberProbe[Int]()
val c = StreamTestKit.SubscriberProbe[Int]()
val keepAlive = TestSubscriber.manualProbe[Int]()
val c = TestSubscriber.manualProbe[Int]()
p.subscribe(keepAlive)
p.subscribe(c)
val sub = c.expectSubscription()

View file

@ -6,8 +6,7 @@ package akka.stream.scaladsl
import akka.stream.OperationAttributes._
import akka.stream.ActorFlowMaterializer
import akka.stream.OverflowStrategy
import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.StreamTestKit.{ PublisherProbe, SubscriberProbe }
import akka.stream.testkit._
import akka.stream.stage._
object FlowGraphCompileSpec {
@ -216,7 +215,7 @@ class FlowGraphCompileSpec extends AkkaSpec {
}
"build with variance" in {
val out = Sink(SubscriberProbe[Fruit]())
val out = Sink(TestSubscriber.manualProbe[Fruit]())
FlowGraph.closed() { b
val merge = b.add(Merge[Fruit](2))
b.addEdge(b add Source[Fruit](apples), Flow[Fruit], merge.in(0))
@ -227,10 +226,10 @@ class FlowGraphCompileSpec extends AkkaSpec {
"build with implicits and variance" in {
FlowGraph.closed() { implicit b
def appleSource = b.add(Source(PublisherProbe[Apple]))
def fruitSource = b.add(Source(PublisherProbe[Fruit]))
val outA = b add Sink(SubscriberProbe[Fruit]())
val outB = b add Sink(SubscriberProbe[Fruit]())
def appleSource = b.add(Source(TestPublisher.manualProbe[Apple]))
def fruitSource = b.add(Source(TestPublisher.manualProbe[Fruit]))
val outA = b add Sink(TestSubscriber.manualProbe[Fruit]())
val outB = b add Sink(TestSubscriber.manualProbe[Fruit]())
val merge = b add Merge[Fruit](11)
val unzip = b add Unzip[Int, String]()
val whatever = b add Sink.publisher[Any]

View file

@ -9,11 +9,10 @@ import akka.stream.ActorFlowMaterializer
import akka.stream.ActorFlowMaterializerSettings
import akka.stream.Supervision.resumingDecider
import akka.stream.testkit._
import akka.stream.testkit.StreamTestKit.TE
import akka.stream.testkit.Utils._
import org.reactivestreams.Publisher
import akka.stream.OperationAttributes
import akka.stream.ActorOperationAttributes
import akka.stream.testkit.StreamTestKit.assertAllStagesStopped
class FlowGroupBySpec extends AkkaSpec {
@ -23,7 +22,7 @@ class FlowGroupBySpec extends AkkaSpec {
implicit val materializer = ActorFlowMaterializer(settings)
case class StreamPuppet(p: Publisher[Int]) {
val probe = StreamTestKit.SubscriberProbe[Int]()
val probe = TestSubscriber.manualProbe[Int]()
p.subscribe(probe)
val subscription = probe.expectSubscription()
@ -38,7 +37,7 @@ class FlowGroupBySpec extends AkkaSpec {
class SubstreamsSupport(groupCount: Int = 2, elementCount: Int = 6) {
val source = Source(1 to elementCount).runWith(Sink.publisher)
val groupStream = Source(source).groupBy(_ % groupCount).runWith(Sink.publisher)
val masterSubscriber = StreamTestKit.SubscriberProbe[(Int, Source[Int, _])]()
val masterSubscriber = TestSubscriber.manualProbe[(Int, Source[Int, _])]()
groupStream.subscribe(masterSubscriber)
val masterSubscription = masterSubscriber.expectSubscription()
@ -113,9 +112,9 @@ class FlowGroupBySpec extends AkkaSpec {
}
"accept cancellation of master stream when not consumed anything" in assertAllStagesStopped {
val publisherProbeProbe = StreamTestKit.PublisherProbe[Int]()
val publisherProbeProbe = TestPublisher.manualProbe[Int]()
val publisher = Source(publisherProbeProbe).groupBy(_ % 2).runWith(Sink.publisher)
val subscriber = StreamTestKit.SubscriberProbe[(Int, Source[Int, _])]()
val subscriber = TestSubscriber.manualProbe[(Int, Source[Int, _])]()
publisher.subscribe(subscriber)
val upstreamSubscription = publisherProbeProbe.expectSubscription()
@ -146,16 +145,16 @@ class FlowGroupBySpec extends AkkaSpec {
"work with empty input stream" in assertAllStagesStopped {
val publisher = Source(List.empty[Int]).groupBy(_ % 2).runWith(Sink.publisher)
val subscriber = StreamTestKit.SubscriberProbe[(Int, Source[Int, _])]()
val subscriber = TestSubscriber.manualProbe[(Int, Source[Int, _])]()
publisher.subscribe(subscriber)
subscriber.expectSubscriptionAndComplete()
}
"abort on onError from upstream" in assertAllStagesStopped {
val publisherProbeProbe = StreamTestKit.PublisherProbe[Int]()
val publisherProbeProbe = TestPublisher.manualProbe[Int]()
val publisher = Source(publisherProbeProbe).groupBy(_ % 2).runWith(Sink.publisher)
val subscriber = StreamTestKit.SubscriberProbe[(Int, Source[Int, _])]()
val subscriber = TestSubscriber.manualProbe[(Int, Source[Int, _])]()
publisher.subscribe(subscriber)
val upstreamSubscription = publisherProbeProbe.expectSubscription()
@ -170,9 +169,9 @@ class FlowGroupBySpec extends AkkaSpec {
}
"abort on onError from upstream when substreams are running" in assertAllStagesStopped {
val publisherProbeProbe = StreamTestKit.PublisherProbe[Int]()
val publisherProbeProbe = TestPublisher.manualProbe[Int]()
val publisher = Source(publisherProbeProbe).groupBy(_ % 2).runWith(Sink.publisher)
val subscriber = StreamTestKit.SubscriberProbe[(Int, Source[Int, _])]()
val subscriber = TestSubscriber.manualProbe[(Int, Source[Int, _])]()
publisher.subscribe(subscriber)
val upstreamSubscription = publisherProbeProbe.expectSubscription()
@ -197,12 +196,12 @@ class FlowGroupBySpec extends AkkaSpec {
}
"fail stream when groupBy function throws" in assertAllStagesStopped {
val publisherProbeProbe = StreamTestKit.PublisherProbe[Int]()
val publisherProbeProbe = TestPublisher.manualProbe[Int]()
val exc = TE("test")
val publisher = Source(publisherProbeProbe)
.groupBy(elem if (elem == 2) throw exc else elem % 2)
.runWith(Sink.publisher)
val subscriber = StreamTestKit.SubscriberProbe[(Int, Source[Int, Unit])]()
val subscriber = TestSubscriber.manualProbe[(Int, Source[Int, Unit])]()
publisher.subscribe(subscriber)
val upstreamSubscription = publisherProbeProbe.expectSubscription()
@ -226,13 +225,13 @@ class FlowGroupBySpec extends AkkaSpec {
}
"resume stream when groupBy function throws" in {
val publisherProbeProbe = StreamTestKit.PublisherProbe[Int]()
val publisherProbeProbe = TestPublisher.manualProbe[Int]()
val exc = TE("test")
val publisher = Source(publisherProbeProbe)
.groupBy(elem if (elem == 2) throw exc else elem % 2)
.withAttributes(ActorOperationAttributes.supervisionStrategy(resumingDecider))
.runWith(Sink.publisher)
val subscriber = StreamTestKit.SubscriberProbe[(Int, Source[Int, Unit])]()
val subscriber = TestSubscriber.manualProbe[(Int, Source[Int, Unit])]()
publisher.subscribe(subscriber)
val upstreamSubscription = publisherProbeProbe.expectSubscription()
@ -268,8 +267,8 @@ class FlowGroupBySpec extends AkkaSpec {
}
"pass along early cancellation" in assertAllStagesStopped {
val up = StreamTestKit.PublisherProbe[Int]()
val down = StreamTestKit.SubscriberProbe[(Int, Source[Int, Unit])]()
val up = TestPublisher.manualProbe[Int]()
val down = TestSubscriber.manualProbe[(Int, Source[Int, Unit])]()
val flowSubscriber = Source.subscriber[Int].groupBy(_ % 2).to(Sink(down)).run()

View file

@ -9,10 +9,8 @@ import scala.concurrent.forkjoin.ThreadLocalRandom.{ current ⇒ random }
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.ScriptedTest
import akka.stream.testkit.StreamTestKit
import akka.stream.testkit.StreamTestKit.assertAllStagesStopped
import akka.stream.testkit._
import akka.stream.testkit.Utils._
class FlowGroupedWithinSpec extends AkkaSpec with ScriptedTest {
@ -24,8 +22,8 @@ class FlowGroupedWithinSpec extends AkkaSpec with ScriptedTest {
"group elements within the duration" in assertAllStagesStopped {
val input = Iterator.from(1)
val p = StreamTestKit.PublisherProbe[Int]()
val c = StreamTestKit.SubscriberProbe[immutable.Seq[Int]]()
val p = TestPublisher.manualProbe[Int]()
val c = TestSubscriber.manualProbe[immutable.Seq[Int]]()
Source(p).groupedWithin(1000, 1.second).to(Sink(c)).run()
val pSub = p.expectSubscription
val cSub = c.expectSubscription
@ -50,7 +48,7 @@ class FlowGroupedWithinSpec extends AkkaSpec with ScriptedTest {
}
"deliver bufferd elements onComplete before the timeout" in {
val c = StreamTestKit.SubscriberProbe[immutable.Seq[Int]]()
val c = TestSubscriber.manualProbe[immutable.Seq[Int]]()
Source(1 to 3).groupedWithin(1000, 10.second).to(Sink(c)).run()
val cSub = c.expectSubscription
cSub.request(100)
@ -61,8 +59,8 @@ class FlowGroupedWithinSpec extends AkkaSpec with ScriptedTest {
"buffer groups until requested from downstream" in {
val input = Iterator.from(1)
val p = StreamTestKit.PublisherProbe[Int]()
val c = StreamTestKit.SubscriberProbe[immutable.Seq[Int]]()
val p = TestPublisher.manualProbe[Int]()
val c = TestSubscriber.manualProbe[immutable.Seq[Int]]()
Source(p).groupedWithin(1000, 1.second).to(Sink(c)).run()
val pSub = p.expectSubscription
val cSub = c.expectSubscription
@ -81,8 +79,8 @@ class FlowGroupedWithinSpec extends AkkaSpec with ScriptedTest {
}
"drop empty groups" in {
val p = StreamTestKit.PublisherProbe[Int]()
val c = StreamTestKit.SubscriberProbe[immutable.Seq[Int]]()
val p = TestPublisher.manualProbe[Int]()
val c = TestSubscriber.manualProbe[immutable.Seq[Int]]()
Source(p).groupedWithin(1000, 500.millis).to(Sink(c)).run()
val pSub = p.expectSubscription
val cSub = c.expectSubscription
@ -103,8 +101,8 @@ class FlowGroupedWithinSpec extends AkkaSpec with ScriptedTest {
"reset time window when max elements reached" in {
val input = Iterator.from(1)
val p = StreamTestKit.PublisherProbe[Int]()
val c = StreamTestKit.SubscriberProbe[immutable.Seq[Int]]()
val p = TestPublisher.manualProbe[Int]()
val c = TestSubscriber.manualProbe[immutable.Seq[Int]]()
Source(p).groupedWithin(3, 2.second).to(Sink(c)).run()
val pSub = p.expectSubscription
val cSub = c.expectSubscription
@ -113,11 +111,11 @@ class FlowGroupedWithinSpec extends AkkaSpec with ScriptedTest {
demand1 should be(4)
c.expectNoMsg(1000.millis)
(1 to demand1) foreach { _ pSub.sendNext(input.next()) }
c.probe.within(1000.millis) {
c.within(1000.millis) {
c.expectNext((1 to 3).toVector)
}
c.expectNoMsg(1500.millis)
c.probe.within(1000.millis) {
c.within(1000.millis) {
c.expectNext(List(4))
}
pSub.sendComplete()

View file

@ -7,12 +7,8 @@ import scala.collection.immutable
import scala.concurrent.duration._
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.StreamTestKit
import akka.stream.testkit.StreamTestKit.OnComplete
import akka.stream.testkit.StreamTestKit.OnError
import akka.stream.testkit.StreamTestKit.OnNext
import akka.stream.testkit.StreamTestKit.assertAllStagesStopped
import akka.stream.testkit._
import akka.stream.testkit.Utils._
import akka.stream.impl.SynchronousIterablePublisher
import org.reactivestreams.Subscription
import akka.testkit.TestProbe
@ -37,7 +33,7 @@ class FlowIterableSpec extends AbstractFlowIteratorSpec {
(1 to 3).iterator.map(x if (x == 2) throw new IllegalStateException("not two") else x)
}
val p = Source(iterable).runWith(Sink.publisher)
val c = StreamTestKit.SubscriberProbe[Int]()
val c = TestSubscriber.manualProbe[Int]()
p.subscribe(c)
val sub = c.expectSubscription()
sub.request(1)
@ -54,7 +50,7 @@ class FlowIterableSpec extends AbstractFlowIteratorSpec {
override def iterator: Iterator[Int] = throw new IllegalStateException("no good iterator")
}
val p = Source(iterable).runWith(Sink.publisher)
val c = StreamTestKit.SubscriberProbe[Int]()
val c = TestSubscriber.manualProbe[Int]()
p.subscribe(c)
c.expectSubscriptionAndError().getMessage should be("no good iterator")
c.expectNoMsg(100.millis)
@ -68,7 +64,7 @@ class FlowIterableSpec extends AbstractFlowIteratorSpec {
}
}
val p = Source(iterable).runWith(Sink.publisher)
val c = StreamTestKit.SubscriberProbe[Int]()
val c = TestSubscriber.manualProbe[Int]()
p.subscribe(c)
c.expectSubscriptionAndError().getMessage should be("no next")
c.expectNoMsg(100.millis)
@ -109,7 +105,7 @@ class SynchronousIterableSpec extends AbstractFlowIteratorSpec {
(1 to 3).iterator.map(x if (x == 2) throw new IllegalStateException("not two") else x)
}
val p = SynchronousIterablePublisher(iterable, "iterable")
val c = StreamTestKit.SubscriberProbe[Int]()
val c = TestSubscriber.manualProbe[Int]()
p.subscribe(c)
val sub = c.expectSubscription()
sub.request(1)
@ -163,7 +159,7 @@ abstract class AbstractFlowIteratorSpec extends AkkaSpec {
testName must {
"produce elements" in assertAllStagesStopped {
val p = createSource(3).runWith(Sink.publisher)
val c = StreamTestKit.SubscriberProbe[Int]()
val c = TestSubscriber.manualProbe[Int]()
p.subscribe(c)
val sub = c.expectSubscription()
sub.request(1)
@ -177,7 +173,7 @@ abstract class AbstractFlowIteratorSpec extends AkkaSpec {
"complete empty" in assertAllStagesStopped {
val p = createSource(0).runWith(Sink.publisher)
val c = StreamTestKit.SubscriberProbe[Int]()
val c = TestSubscriber.manualProbe[Int]()
p.subscribe(c)
c.expectSubscriptionAndComplete()
c.expectNoMsg(100.millis)
@ -185,8 +181,8 @@ abstract class AbstractFlowIteratorSpec extends AkkaSpec {
"produce elements with multiple subscribers" in assertAllStagesStopped {
val p = createSource(3).runWith(Sink.fanoutPublisher(2, 4))
val c1 = StreamTestKit.SubscriberProbe[Int]()
val c2 = StreamTestKit.SubscriberProbe[Int]()
val c1 = TestSubscriber.manualProbe[Int]()
val c2 = TestSubscriber.manualProbe[Int]()
p.subscribe(c1)
p.subscribe(c2)
val sub1 = c1.expectSubscription()
@ -209,8 +205,8 @@ abstract class AbstractFlowIteratorSpec extends AkkaSpec {
"produce elements to later subscriber" in assertAllStagesStopped {
val p = createSource(3).runWith(Sink.fanoutPublisher(2, 4))
val c1 = StreamTestKit.SubscriberProbe[Int]()
val c2 = StreamTestKit.SubscriberProbe[Int]()
val c1 = TestSubscriber.manualProbe[Int]()
val c2 = TestSubscriber.manualProbe[Int]()
p.subscribe(c1)
val sub1 = c1.expectSubscription()
@ -232,7 +228,7 @@ abstract class AbstractFlowIteratorSpec extends AkkaSpec {
"produce elements with one transformation step" in assertAllStagesStopped {
val p = createSource(3).map(_ * 2).runWith(Sink.publisher)
val c = StreamTestKit.SubscriberProbe[Int]()
val c = TestSubscriber.manualProbe[Int]()
p.subscribe(c)
val sub = c.expectSubscription()
sub.request(10)
@ -244,7 +240,7 @@ abstract class AbstractFlowIteratorSpec extends AkkaSpec {
"produce elements with two transformation steps" in assertAllStagesStopped {
val p = createSource(4).filter(_ % 2 == 0).map(_ * 2).runWith(Sink.publisher)
val c = StreamTestKit.SubscriberProbe[Int]()
val c = TestSubscriber.manualProbe[Int]()
p.subscribe(c)
val sub = c.expectSubscription()
sub.request(10)
@ -255,7 +251,7 @@ abstract class AbstractFlowIteratorSpec extends AkkaSpec {
"not produce after cancel" in assertAllStagesStopped {
val p = createSource(3).runWith(Sink.publisher)
val c = StreamTestKit.SubscriberProbe[Int]()
val c = TestSubscriber.manualProbe[Int]()
p.subscribe(c)
val sub = c.expectSubscription()
sub.request(1)

View file

@ -4,7 +4,7 @@
package akka.stream.scaladsl
import akka.stream.{ ActorFlowMaterializer, ActorFlowMaterializerSettings }
import akka.stream.testkit.{ StreamTestKit, AkkaSpec }
import akka.stream.testkit._
import com.typesafe.config.ConfigFactory
import scala.concurrent.Await
import scala.concurrent.duration._
@ -22,7 +22,7 @@ class FlowJoinSpec extends AkkaSpec(ConfigFactory.parseString("akka.loglevel=INF
val (even, odd) = (0 to end).partition(_ % 2 == 0)
val result = Set() ++ even ++ odd ++ odd.map(_ * 10)
val source = Source(0 to end)
val probe = StreamTestKit.SubscriberProbe[Seq[Int]]()
val probe = TestSubscriber.manualProbe[Seq[Int]]()
val flow1 = Flow() { implicit b
import FlowGraph.Implicits._

View file

@ -10,9 +10,8 @@ import scala.concurrent.forkjoin.ThreadLocalRandom
import scala.util.control.NoStackTrace
import akka.stream.ActorFlowMaterializer
import akka.stream.stage._
import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.StreamTestKit
import akka.stream.testkit.StreamTestKit.assertAllStagesStopped
import akka.stream.testkit._
import akka.stream.testkit.Utils._
import akka.testkit.TestLatch
import akka.testkit.TestProbe
import akka.stream.ActorOperationAttributes.supervisionStrategy
@ -68,7 +67,7 @@ class FlowMapAsyncSpec extends AkkaSpec {
"A Flow with mapAsync" must {
"produce future elements" in assertAllStagesStopped {
val c = StreamTestKit.SubscriberProbe[Int]()
val c = TestSubscriber.manualProbe[Int]()
implicit val ec = system.dispatcher
val p = Source(1 to 3).mapAsync(4, n Future(n)).runWith(Sink(c))
val sub = c.expectSubscription()
@ -82,7 +81,7 @@ class FlowMapAsyncSpec extends AkkaSpec {
}
"produce future elements in order" in {
val c = StreamTestKit.SubscriberProbe[Int]()
val c = TestSubscriber.manualProbe[Int]()
implicit val ec = system.dispatcher
val p = Source(1 to 50).mapAsync(4, n Future {
Thread.sleep(ThreadLocalRandom.current().nextInt(1, 10))
@ -96,7 +95,7 @@ class FlowMapAsyncSpec extends AkkaSpec {
"not run more futures than requested parallelism" in {
val probe = TestProbe()
val c = StreamTestKit.SubscriberProbe[Int]()
val c = TestSubscriber.manualProbe[Int]()
implicit val ec = system.dispatcher
val p = Source(1 to 20).mapAsync(8, n Future {
probe.ref ! n
@ -122,7 +121,7 @@ class FlowMapAsyncSpec extends AkkaSpec {
"signal future failure" in assertAllStagesStopped {
val latch = TestLatch(1)
val c = StreamTestKit.SubscriberProbe[Int]()
val c = TestSubscriber.manualProbe[Int]()
implicit val ec = system.dispatcher
val p = Source(1 to 5).mapAsync(4, n Future {
if (n == 3) throw new RuntimeException("err1") with NoStackTrace
@ -139,7 +138,7 @@ class FlowMapAsyncSpec extends AkkaSpec {
"signal error from mapAsync" in assertAllStagesStopped {
val latch = TestLatch(1)
val c = StreamTestKit.SubscriberProbe[Int]()
val c = TestSubscriber.manualProbe[Int]()
implicit val ec = system.dispatcher
val p = Source(1 to 5).mapAsync(4, n
if (n == 3) throw new RuntimeException("err2") with NoStackTrace
@ -157,7 +156,7 @@ class FlowMapAsyncSpec extends AkkaSpec {
}
"resume after future failure" in assertAllStagesStopped {
val c = StreamTestKit.SubscriberProbe[Int]()
val c = TestSubscriber.manualProbe[Int]()
implicit val ec = system.dispatcher
val p = Source(1 to 5)
.mapAsync(4, n Future {
@ -183,7 +182,7 @@ class FlowMapAsyncSpec extends AkkaSpec {
}
"resume when mapAsync throws" in {
val c = StreamTestKit.SubscriberProbe[Int]()
val c = TestSubscriber.manualProbe[Int]()
implicit val ec = system.dispatcher
val p = Source(1 to 5)
.mapAsync(4, n
@ -198,7 +197,7 @@ class FlowMapAsyncSpec extends AkkaSpec {
}
"signal NPE when future is completed with null" in {
val c = StreamTestKit.SubscriberProbe[String]()
val c = TestSubscriber.manualProbe[String]()
val p = Source(List("a", "b")).mapAsync(4, elem Future.successful(null)).to(Sink(c)).run()
val sub = c.expectSubscription()
sub.request(10)
@ -206,7 +205,7 @@ class FlowMapAsyncSpec extends AkkaSpec {
}
"resume when future is completed with null" in {
val c = StreamTestKit.SubscriberProbe[String]()
val c = TestSubscriber.manualProbe[String]()
val p = Source(List("a", "b", "c"))
.mapAsync(4, elem if (elem == "b") Future.successful(null) else Future.successful(elem))
.withAttributes(supervisionStrategy(resumingDecider))
@ -218,8 +217,8 @@ class FlowMapAsyncSpec extends AkkaSpec {
}
"should handle cancel properly" in assertAllStagesStopped {
val pub = StreamTestKit.PublisherProbe[Int]()
val sub = StreamTestKit.SubscriberProbe[Int]()
val pub = TestPublisher.manualProbe[Int]()
val sub = TestSubscriber.manualProbe[Int]()
Source(pub).mapAsync(4, Future.successful).runWith(Sink(sub))
@ -250,7 +249,7 @@ class FlowMapAsyncSpec extends AkkaSpec {
}
"work when futures fail" in {
val probe = StreamTestKit.SubscriberProbe[Int]
val probe = TestSubscriber.manualProbe[Int]
val ex = new Exception("KABOOM")
Source.single(1)
.transform(() new MapAsyncOne(_ Future.failed(ex)))
@ -261,7 +260,7 @@ class FlowMapAsyncSpec extends AkkaSpec {
}
"work when futures fail later" in {
val probe = StreamTestKit.SubscriberProbe[Int]
val probe = TestSubscriber.manualProbe[Int]
val ex = new Exception("KABOOM")
Source(List(1, 2))
.transform(() new MapAsyncOne(x if (x == 1) Future.successful(1) else Future.failed(ex)))

View file

@ -9,15 +9,13 @@ import scala.concurrent.duration._
import scala.util.control.NoStackTrace
import akka.stream.ActorFlowMaterializer
import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.StreamTestKit
import akka.stream.testkit.StreamTestKit.assertAllStagesStopped
import akka.stream.testkit._
import akka.stream.testkit.scaladsl._
import akka.stream.testkit.Utils._
import akka.testkit.TestLatch
import akka.testkit.TestProbe
import akka.stream.ActorOperationAttributes.supervisionStrategy
import akka.stream.Supervision.resumingDecider
import akka.stream.testkit.StreamTestKit.OnNext
import akka.stream.testkit.StreamTestKit.OnComplete
import akka.stream.impl.ReactiveStreamsCompliance
class FlowMapAsyncUnorderedSpec extends AkkaSpec {
@ -27,7 +25,7 @@ class FlowMapAsyncUnorderedSpec extends AkkaSpec {
"A Flow with mapAsyncUnordered" must {
"produce future elements in the order they are ready" in assertAllStagesStopped {
val c = StreamTestKit.SubscriberProbe[Int]()
val c = TestSubscriber.manualProbe[Int]()
implicit val ec = system.dispatcher
val latch = (1 to 4).map(_ -> TestLatch(1)).toMap
val p = Source(1 to 4).mapAsyncUnordered(4, n Future {
@ -49,7 +47,7 @@ class FlowMapAsyncUnorderedSpec extends AkkaSpec {
"not run more futures than requested elements" in {
val probe = TestProbe()
val c = StreamTestKit.SubscriberProbe[Int]()
val c = TestSubscriber.manualProbe[Int]()
implicit val ec = system.dispatcher
val p = Source(1 to 20).mapAsyncUnordered(4, n Future {
probe.ref ! n
@ -66,7 +64,7 @@ class FlowMapAsyncUnorderedSpec extends AkkaSpec {
probe.expectNoMsg(500.millis)
sub.request(25)
probe.expectMsgAllOf(6 to 20: _*)
c.probe.within(3.seconds) {
c.within(3.seconds) {
for (_ 2 to 20) got += c.expectNext()
}
@ -76,7 +74,7 @@ class FlowMapAsyncUnorderedSpec extends AkkaSpec {
"signal future failure" in assertAllStagesStopped {
val latch = TestLatch(1)
val c = StreamTestKit.SubscriberProbe[Int]()
val c = TestSubscriber.manualProbe[Int]()
implicit val ec = system.dispatcher
val p = Source(1 to 5).mapAsyncUnordered(4, n Future {
if (n == 3) throw new RuntimeException("err1") with NoStackTrace
@ -93,7 +91,7 @@ class FlowMapAsyncUnorderedSpec extends AkkaSpec {
"signal error from mapAsyncUnordered" in assertAllStagesStopped {
val latch = TestLatch(1)
val c = StreamTestKit.SubscriberProbe[Int]()
val c = TestSubscriber.manualProbe[Int]()
implicit val ec = system.dispatcher
val p = Source(1 to 5).mapAsyncUnordered(4, n
if (n == 3) throw new RuntimeException("err2") with NoStackTrace
@ -111,19 +109,17 @@ class FlowMapAsyncUnorderedSpec extends AkkaSpec {
}
"resume after future failure" in {
val c = StreamTestKit.SubscriberProbe[Int]()
implicit val ec = system.dispatcher
val p = Source(1 to 5)
Source(1 to 5)
.mapAsyncUnordered(4, n Future {
if (n == 3) throw new RuntimeException("err3") with NoStackTrace
else n
})
.withAttributes(supervisionStrategy(resumingDecider))
.to(Sink(c)).run()
val sub = c.expectSubscription()
sub.request(10)
val expected = (OnComplete :: List(1, 2, 4, 5).map(OnNext.apply)).toSet
c.probe.receiveWhile(2.seconds, messages = 5) { case x x }.toSet should be(expected)
.runWith(TestSink.probe[Int])
.request(10)
.expectNextUnordered(1, 2, 4, 5)
.expectComplete()
}
"finish after future failure" in assertAllStagesStopped {
@ -137,22 +133,20 @@ class FlowMapAsyncUnorderedSpec extends AkkaSpec {
}
"resume when mapAsyncUnordered throws" in {
val c = StreamTestKit.SubscriberProbe[Int]()
implicit val ec = system.dispatcher
val p = Source(1 to 5)
Source(1 to 5)
.mapAsyncUnordered(4, n
if (n == 3) throw new RuntimeException("err4") with NoStackTrace
else Future(n))
.withAttributes(supervisionStrategy(resumingDecider))
.to(Sink(c)).run()
val sub = c.expectSubscription()
sub.request(10)
val expected = (OnComplete :: List(1, 2, 4, 5).map(OnNext.apply)).toSet
c.probe.receiveWhile(3.seconds, messages = 5) { case x x }.toSet should be(expected)
.runWith(TestSink.probe[Int])
.request(10)
.expectNext(1, 2, 4, 5)
.expectComplete()
}
"signal NPE when future is completed with null" in {
val c = StreamTestKit.SubscriberProbe[String]()
val c = TestSubscriber.manualProbe[String]()
val p = Source(List("a", "b")).mapAsyncUnordered(4, elem Future.successful(null)).to(Sink(c)).run()
val sub = c.expectSubscription()
sub.request(10)
@ -160,7 +154,7 @@ class FlowMapAsyncUnorderedSpec extends AkkaSpec {
}
"resume when future is completed with null" in {
val c = StreamTestKit.SubscriberProbe[String]()
val c = TestSubscriber.manualProbe[String]()
val p = Source(List("a", "b", "c"))
.mapAsyncUnordered(4, elem if (elem == "b") Future.successful(null) else Future.successful(elem))
.withAttributes(supervisionStrategy(resumingDecider))
@ -172,8 +166,8 @@ class FlowMapAsyncUnorderedSpec extends AkkaSpec {
}
"should handle cancel properly" in assertAllStagesStopped {
val pub = StreamTestKit.PublisherProbe[Int]()
val sub = StreamTestKit.SubscriberProbe[Int]()
val pub = TestPublisher.manualProbe[Int]()
val sub = TestSubscriber.manualProbe[Int]()
Source(pub).mapAsyncUnordered(4, Future.successful).runWith(Sink(sub))

View file

@ -5,10 +5,8 @@ package akka.stream.scaladsl
import scala.concurrent.duration._
import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.ScriptedTest
import akka.stream.testkit.StreamTestKit.SubscriberProbe
import akka.stream.testkit.StreamTestKit.assertAllStagesStopped
import akka.stream.testkit._
import akka.stream.testkit.Utils._
import akka.stream.ActorFlowMaterializer
class FlowMapConcatSpec extends AkkaSpec with ScriptedTest {
@ -34,7 +32,7 @@ class FlowMapConcatSpec extends AkkaSpec with ScriptedTest {
.withInputBuffer(initialSize = 2, maxSize = 2)
implicit val materializer = ActorFlowMaterializer(settings)
assertAllStagesStopped {
val s = SubscriberProbe[Int]
val s = TestSubscriber.manualProbe[Int]
val input = (1 to 20).grouped(5).toList
Source(input).mapConcat(identity).map(x { Thread.sleep(10); x }).runWith(Sink(s))
val sub = s.expectSubscription()

View file

@ -7,8 +7,7 @@ import scala.concurrent.forkjoin.ThreadLocalRandom.{ current ⇒ random }
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
import akka.stream.testkit.ScriptedTest
import akka.stream.testkit._
class FlowMapSpec extends AkkaSpec with ScriptedTest {
@ -25,7 +24,7 @@ class FlowMapSpec extends AkkaSpec with ScriptedTest {
}
"not blow up with high request counts" in {
val probe = StreamTestKit.SubscriberProbe[Int]()
val probe = TestSubscriber.manualProbe[Int]()
Source(List(1)).
map(_ + 1).map(_ + 1).map(_ + 1).map(_ + 1).map(_ + 1).
runWith(Sink.publisher).subscribe(probe)

View file

@ -9,10 +9,8 @@ import scala.util.control.NoStackTrace
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
import akka.stream.testkit.StreamTestKit.assertAllStagesStopped
import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.ScriptedTest
import akka.stream.testkit._
import akka.stream.testkit.Utils._
import akka.testkit.TestProbe
class FlowOnCompleteSpec extends AkkaSpec with ScriptedTest {
@ -26,7 +24,7 @@ class FlowOnCompleteSpec extends AkkaSpec with ScriptedTest {
"invoke callback on normal completion" in assertAllStagesStopped {
val onCompleteProbe = TestProbe()
val p = StreamTestKit.PublisherProbe[Int]()
val p = TestPublisher.manualProbe[Int]()
Source(p).to(Sink.onComplete[Int](onCompleteProbe.ref ! _)).run()
val proc = p.expectSubscription
proc.expectRequest()
@ -38,7 +36,7 @@ class FlowOnCompleteSpec extends AkkaSpec with ScriptedTest {
"yield the first error" in assertAllStagesStopped {
val onCompleteProbe = TestProbe()
val p = StreamTestKit.PublisherProbe[Int]()
val p = TestPublisher.manualProbe[Int]()
Source(p).to(Sink.onComplete[Int](onCompleteProbe.ref ! _)).run()
val proc = p.expectSubscription
proc.expectRequest()
@ -50,7 +48,7 @@ class FlowOnCompleteSpec extends AkkaSpec with ScriptedTest {
"invoke callback for an empty stream" in assertAllStagesStopped {
val onCompleteProbe = TestProbe()
val p = StreamTestKit.PublisherProbe[Int]()
val p = TestPublisher.manualProbe[Int]()
Source(p).to(Sink.onComplete[Int](onCompleteProbe.ref ! _)).run()
val proc = p.expectSubscription
proc.expectRequest()
@ -61,7 +59,7 @@ class FlowOnCompleteSpec extends AkkaSpec with ScriptedTest {
"invoke callback after transform and foreach steps " in assertAllStagesStopped {
val onCompleteProbe = TestProbe()
val p = StreamTestKit.PublisherProbe[Int]()
val p = TestPublisher.manualProbe[Int]()
import system.dispatcher // for the Future.onComplete
val foreachSink = Sink.foreach[Int] {
x onCompleteProbe.ref ! ("foreach-" + x)

View file

@ -9,11 +9,9 @@ import scala.concurrent.duration._
import scala.util.control.NoStackTrace
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
import akka.stream.testkit.StreamTestKit.SubscriberProbe
import akka.stream.testkit.StreamTestKit.PublisherProbe
import akka.stream.testkit.StreamTestKit.assertAllStagesStopped
import org.reactivestreams.Subscriber
import akka.stream.testkit._
import akka.stream.testkit.Utils._
class FlowPrefixAndTailSpec extends AkkaSpec {
@ -33,7 +31,7 @@ class FlowPrefixAndTailSpec extends AkkaSpec {
val fut = Source.empty.prefixAndTail(10).runWith(futureSink)
val (prefix, tailFlow) = Await.result(fut, 3.seconds)
prefix should be(Nil)
val tailSubscriber = SubscriberProbe[Int]
val tailSubscriber = TestSubscriber.manualProbe[Int]
tailFlow.to(Sink(tailSubscriber)).run()
tailSubscriber.expectSubscriptionAndComplete()
}
@ -43,7 +41,7 @@ class FlowPrefixAndTailSpec extends AkkaSpec {
val fut = Source(List(1, 2, 3)).prefixAndTail(10).runWith(futureSink)
val (prefix, tailFlow) = Await.result(fut, 3.seconds)
prefix should be(List(1, 2, 3))
val tailSubscriber = SubscriberProbe[Int]
val tailSubscriber = TestSubscriber.manualProbe[Int]
tailFlow.to(Sink(tailSubscriber)).run()
tailSubscriber.expectSubscriptionAndComplete()
}
@ -87,14 +85,14 @@ class FlowPrefixAndTailSpec extends AkkaSpec {
val (takes, tail) = Await.result(fut, 3.seconds)
takes should be(1 to 10)
val subscriber = StreamTestKit.SubscriberProbe[Int]()
val subscriber = TestSubscriber.manualProbe[Int]()
tail.to(Sink(subscriber)).run()
subscriber.expectSubscriptionAndComplete()
}
"handle onError when no substream open" in assertAllStagesStopped {
val publisher = StreamTestKit.PublisherProbe[Int]()
val subscriber = StreamTestKit.SubscriberProbe[(immutable.Seq[Int], Source[Int, _])]()
val publisher = TestPublisher.manualProbe[Int]()
val subscriber = TestSubscriber.manualProbe[(immutable.Seq[Int], Source[Int, _])]()
Source(publisher).prefixAndTail(3).to(Sink(subscriber)).run()
@ -111,8 +109,8 @@ class FlowPrefixAndTailSpec extends AkkaSpec {
}
"handle onError when substream is open" in assertAllStagesStopped {
val publisher = StreamTestKit.PublisherProbe[Int]()
val subscriber = StreamTestKit.SubscriberProbe[(immutable.Seq[Int], Source[Int, _])]()
val publisher = TestPublisher.manualProbe[Int]()
val subscriber = TestSubscriber.manualProbe[(immutable.Seq[Int], Source[Int, _])]()
Source(publisher).prefixAndTail(1).to(Sink(subscriber)).run()
@ -128,7 +126,7 @@ class FlowPrefixAndTailSpec extends AkkaSpec {
head should be(List(1))
subscriber.expectComplete()
val substreamSubscriber = StreamTestKit.SubscriberProbe[Int]()
val substreamSubscriber = TestSubscriber.manualProbe[Int]()
tail.to(Sink(substreamSubscriber)).run()
substreamSubscriber.expectSubscription()
@ -138,8 +136,8 @@ class FlowPrefixAndTailSpec extends AkkaSpec {
}
"handle master stream cancellation" in assertAllStagesStopped {
val publisher = StreamTestKit.PublisherProbe[Int]()
val subscriber = StreamTestKit.SubscriberProbe[(immutable.Seq[Int], Source[Int, _])]()
val publisher = TestPublisher.manualProbe[Int]()
val subscriber = TestSubscriber.manualProbe[(immutable.Seq[Int], Source[Int, _])]()
Source(publisher).prefixAndTail(3).to(Sink(subscriber)).run()
@ -156,8 +154,8 @@ class FlowPrefixAndTailSpec extends AkkaSpec {
}
"handle substream cancellation" in assertAllStagesStopped {
val publisher = StreamTestKit.PublisherProbe[Int]()
val subscriber = StreamTestKit.SubscriberProbe[(immutable.Seq[Int], Source[Int, _])]()
val publisher = TestPublisher.manualProbe[Int]()
val subscriber = TestSubscriber.manualProbe[(immutable.Seq[Int], Source[Int, _])]()
Source(publisher).prefixAndTail(1).to(Sink(subscriber)).run()
@ -173,7 +171,7 @@ class FlowPrefixAndTailSpec extends AkkaSpec {
head should be(List(1))
subscriber.expectComplete()
val substreamSubscriber = StreamTestKit.SubscriberProbe[Int]()
val substreamSubscriber = TestSubscriber.manualProbe[Int]()
tail.to(Sink(substreamSubscriber)).run()
substreamSubscriber.expectSubscription().cancel()
@ -182,8 +180,8 @@ class FlowPrefixAndTailSpec extends AkkaSpec {
}
"pass along early cancellation" in assertAllStagesStopped {
val up = StreamTestKit.PublisherProbe[Int]()
val down = StreamTestKit.SubscriberProbe[(immutable.Seq[Int], Source[Int, _])]()
val up = TestPublisher.manualProbe[Int]()
val down = TestSubscriber.manualProbe[(immutable.Seq[Int], Source[Int, _])]()
val flowSubscriber = Source.subscriber[Int].prefixAndTail(1).to(Sink(down)).run()

View file

@ -10,9 +10,9 @@ import scala.collection.immutable
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.Utils._
import akka.stream.ActorOperationAttributes
import akka.stream.Supervision
import akka.stream.testkit.StreamTestKit.assertAllStagesStopped
class FlowScanSpec extends AkkaSpec {

View file

@ -14,9 +14,8 @@ import akka.actor._
import akka.stream.ActorFlowMaterializerSettings
import akka.stream.ActorFlowMaterializer
import akka.stream.impl._
import akka.stream.testkit.{ StreamTestKit, AkkaSpec }
import akka.stream.testkit.ChainSetup
import akka.stream.testkit.StreamTestKit.assertAllStagesStopped
import akka.stream.testkit._
import akka.stream.testkit.Utils._
import akka.testkit._
import akka.testkit.TestEvent.{ UnMute, Mute }
import com.typesafe.config.ConfigFactory
@ -155,7 +154,7 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
val flow = Flow[String]
val (flowIn: Subscriber[String], flowOut: Publisher[String]) = materializeIntoSubscriberAndPublisher(flow)
val c1 = StreamTestKit.SubscriberProbe[String]()
val c1 = TestSubscriber.manualProbe[String]()
flowOut.subscribe(c1)
val source: Publisher[String] = Source(List("1", "2", "3")).runWith(Sink.publisher)
@ -173,7 +172,7 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
val flow = Flow[Int].map((i: Int) i.toString)
val (flowIn: Subscriber[Int], flowOut: Publisher[String]) = materializeIntoSubscriberAndPublisher(flow)
val c1 = StreamTestKit.SubscriberProbe[String]()
val c1 = TestSubscriber.manualProbe[String]()
flowOut.subscribe(c1)
val sub1 = c1.expectSubscription
sub1.request(3)
@ -192,7 +191,7 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
val flow = Flow[Int].map(_.toString).map("elem-" + _)
val (flowIn, flowOut) = materializeIntoSubscriberAndPublisher(flow)
val c1 = StreamTestKit.SubscriberProbe[String]()
val c1 = TestSubscriber.manualProbe[String]()
flowOut.subscribe(c1)
val sub1 = c1.expectSubscription
sub1.request(3)
@ -209,7 +208,7 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
"subscribe Subscriber" in {
val flow: Flow[String, String, _] = Flow[String]
val c1 = StreamTestKit.SubscriberProbe[String]()
val c1 = TestSubscriber.manualProbe[String]()
val sink: Sink[String, _] = flow.to(Sink(c1))
val publisher: Publisher[String] = Source(List("1", "2", "3")).runWith(Sink.publisher)
Source(publisher).to(sink).run()
@ -235,7 +234,7 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
"perform transformation operation and subscribe Subscriber" in {
val flow = Flow[Int].map(_.toString)
val c1 = StreamTestKit.SubscriberProbe[String]()
val c1 = TestSubscriber.manualProbe[String]()
val sink: Sink[Int, _] = flow.to(Sink(c1))
val publisher: Publisher[Int] = Source(List(1, 2, 3)).runWith(Sink.publisher)
Source(publisher).to(sink).run()
@ -252,9 +251,9 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
val flow = Source(List(1, 2, 3)).map(_.toString)
val p1 = flow.runWith(Sink.fanoutPublisher(2, 2))
val p2 = flow.runWith(Sink.fanoutPublisher(2, 2))
val s1 = StreamTestKit.SubscriberProbe[String]()
val s2 = StreamTestKit.SubscriberProbe[String]()
val s3 = StreamTestKit.SubscriberProbe[String]()
val s1 = TestSubscriber.manualProbe[String]()
val s2 = TestSubscriber.manualProbe[String]()
val s3 = TestSubscriber.manualProbe[String]()
p1.subscribe(s1)
p2.subscribe(s2)
p2.subscribe(s3)
@ -297,7 +296,7 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
val s1: Source[Int, _] = Source(List(1, 2, 3))
val s2: Source[String, _] = Source(List(4, 5, 6)).map(_.toString + "-s")
val subs = StreamTestKit.SubscriberProbe[Any]()
val subs = TestSubscriber.manualProbe[Any]()
val subSink = Sink.publisher[Any]
val (_, res) = f1.concat(s2).runWith(s1, subSink)
@ -319,7 +318,7 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
"adapt speed to the currently slowest subscriber" in {
new ChainSetup(identity, settings.copy(initialInputBufferSize = 1),
toFanoutPublisher(initialBufferSize = 1, maximumBufferSize = 1)) {
val downstream2 = StreamTestKit.SubscriberProbe[Any]()
val downstream2 = TestSubscriber.manualProbe[Any]()
publisher.subscribe(downstream2)
val downstream2Subscription = downstream2.expectSubscription()
@ -346,7 +345,7 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
"support slow subscriber with fan-out 2" in {
new ChainSetup(identity, settings.copy(initialInputBufferSize = 1),
toFanoutPublisher(initialBufferSize = 2, maximumBufferSize = 2)) {
val downstream2 = StreamTestKit.SubscriberProbe[Any]()
val downstream2 = TestSubscriber.manualProbe[Any]()
publisher.subscribe(downstream2)
val downstream2Subscription = downstream2.expectSubscription()
@ -398,7 +397,7 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
upstream.expectRequest(upstreamSubscription, 1)
// link now while an upstream element is already requested
val downstream2 = StreamTestKit.SubscriberProbe[Any]()
val downstream2 = TestSubscriber.manualProbe[Any]()
publisher.subscribe(downstream2)
val downstream2Subscription = downstream2.expectSubscription()
@ -424,7 +423,7 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
"be unblocked when blocking subscriber cancels subscription" in {
new ChainSetup(identity, settings.copy(initialInputBufferSize = 1),
toFanoutPublisher(initialBufferSize = 1, maximumBufferSize = 1)) {
val downstream2 = StreamTestKit.SubscriberProbe[Any]()
val downstream2 = TestSubscriber.manualProbe[Any]()
publisher.subscribe(downstream2)
val downstream2Subscription = downstream2.expectSubscription()
@ -461,7 +460,7 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
"call future subscribers' onError after onSubscribe if initial upstream was completed" in {
new ChainSetup(identity, settings.copy(initialInputBufferSize = 1),
toFanoutPublisher(initialBufferSize = 1, maximumBufferSize = 1)) {
val downstream2 = StreamTestKit.SubscriberProbe[Any]()
val downstream2 = TestSubscriber.manualProbe[Any]()
// don't link it just yet
downstreamSubscription.request(5)
@ -490,7 +489,7 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
downstream2.expectNext("a3")
downstream2.expectComplete()
val downstream3 = StreamTestKit.SubscriberProbe[Any]()
val downstream3 = TestSubscriber.manualProbe[Any]()
publisher.subscribe(downstream3)
downstream3.expectSubscription()
downstream3.expectError() should ===(ActorPublisher.NormalShutdownReason)
@ -508,7 +507,7 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
upstreamSubscription.expectCancellation()
downstream.expectError(TestException)
val downstream2 = StreamTestKit.SubscriberProbe[String]()
val downstream2 = TestSubscriber.manualProbe[String]()
publisher.subscribe(downstream2)
downstream2.expectSubscriptionAndError() should be(TestException)
}
@ -521,7 +520,7 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
downstreamSubscription.cancel()
upstreamSubscription.expectCancellation()
val downstream2 = StreamTestKit.SubscriberProbe[Any]()
val downstream2 = TestSubscriber.manualProbe[Any]()
publisher.subscribe(downstream2)
// IllegalStateException shut down
downstream2.expectSubscriptionAndError().isInstanceOf[IllegalStateException] should be(true)
@ -533,13 +532,13 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
"cancel upstream and call onError on current and future downstream subscribers if an internal error occurs" in {
new ChainSetup(faultyFlow, settings.copy(initialInputBufferSize = 1), toFanoutPublisher(initialBufferSize = 1, maximumBufferSize = 16)) {
def checkError(sprobe: StreamTestKit.SubscriberProbe[Any]): Unit = {
def checkError(sprobe: TestSubscriber.ManualProbe[Any]): Unit = {
val error = sprobe.expectError()
error.isInstanceOf[IllegalStateException] should be(true)
error.getMessage should be("Processor actor terminated abruptly")
}
val downstream2 = StreamTestKit.SubscriberProbe[Any]()
val downstream2 = TestSubscriber.manualProbe[Any]()
publisher.subscribe(downstream2)
val downstream2Subscription = downstream2.expectSubscription()
@ -570,7 +569,7 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
checkError(downstream)
checkError(downstream2)
val downstream3 = StreamTestKit.SubscriberProbe[Any]()
val downstream3 = TestSubscriber.manualProbe[Any]()
publisher.subscribe(downstream3)
downstream3.expectSubscription()
// IllegalStateException terminated abruptly

View file

@ -7,10 +7,8 @@ import scala.concurrent.duration._
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorFlowMaterializerSettings
import akka.stream.Supervision.resumingDecider
import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.StreamTestKit
import akka.stream.testkit.StreamTestKit.TE
import akka.stream.testkit.StreamTestKit.assertAllStagesStopped
import akka.stream.testkit._
import akka.stream.testkit.Utils._
import org.reactivestreams.Publisher
import akka.stream.ActorOperationAttributes
@ -22,7 +20,7 @@ class FlowSplitWhenSpec extends AkkaSpec {
implicit val materializer = ActorFlowMaterializer(settings)
case class StreamPuppet(p: Publisher[Int]) {
val probe = StreamTestKit.SubscriberProbe[Int]()
val probe = TestSubscriber.manualProbe[Int]()
p.subscribe(probe)
val subscription = probe.expectSubscription()
@ -37,7 +35,7 @@ class FlowSplitWhenSpec extends AkkaSpec {
class SubstreamsSupport(splitWhen: Int = 3, elementCount: Int = 6) {
val source = Source(1 to elementCount)
val groupStream = source.splitWhen(_ == splitWhen).runWith(Sink.publisher)
val masterSubscriber = StreamTestKit.SubscriberProbe[Source[Int, _]]()
val masterSubscriber = TestSubscriber.manualProbe[Source[Int, _]]()
groupStream.subscribe(masterSubscriber)
val masterSubscription = masterSubscriber.expectSubscription()
@ -116,12 +114,12 @@ class FlowSplitWhenSpec extends AkkaSpec {
}
"fail stream when splitWhen function throws" in assertAllStagesStopped {
val publisherProbeProbe = StreamTestKit.PublisherProbe[Int]()
val publisherProbeProbe = TestPublisher.manualProbe[Int]()
val exc = TE("test")
val publisher = Source(publisherProbeProbe)
.splitWhen(elem if (elem == 3) throw exc else elem % 3 == 0)
.runWith(Sink.publisher)
val subscriber = StreamTestKit.SubscriberProbe[Source[Int, Unit]]()
val subscriber = TestSubscriber.manualProbe[Source[Int, Unit]]()
publisher.subscribe(subscriber)
val upstreamSubscription = publisherProbeProbe.expectSubscription()
@ -148,13 +146,13 @@ class FlowSplitWhenSpec extends AkkaSpec {
}
"resume stream when splitWhen function throws" in {
val publisherProbeProbe = StreamTestKit.PublisherProbe[Int]()
val publisherProbeProbe = TestPublisher.manualProbe[Int]()
val exc = TE("test")
val publisher = Source(publisherProbeProbe)
.splitWhen(elem if (elem == 3) throw exc else elem % 3 == 0)
.withAttributes(ActorOperationAttributes.supervisionStrategy(resumingDecider))
.runWith(Sink.publisher)
val subscriber = StreamTestKit.SubscriberProbe[Source[Int, Unit]]()
val subscriber = TestSubscriber.manualProbe[Source[Int, Unit]]()
publisher.subscribe(subscriber)
val upstreamSubscription = publisherProbeProbe.expectSubscription()
@ -193,8 +191,8 @@ class FlowSplitWhenSpec extends AkkaSpec {
}
"pass along early cancellation" in assertAllStagesStopped {
val up = StreamTestKit.PublisherProbe[Int]()
val down = StreamTestKit.SubscriberProbe[Source[Int, Unit]]()
val up = TestPublisher.manualProbe[Int]()
val down = TestSubscriber.manualProbe[Source[Int, Unit]]()
val flowSubscriber = Source.subscriber[Int].splitWhen(_ % 3 == 0).to(Sink(down)).run()

View file

@ -8,8 +8,8 @@ import scala.concurrent.duration._
import scala.util.control.NoStackTrace
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
import akka.stream.testkit.StreamTestKit.assertAllStagesStopped
import akka.stream.testkit._
import akka.stream.testkit.Utils._
import akka.testkit.{ EventFilter, TestProbe }
import com.typesafe.config.ConfigFactory
import akka.stream.stage._
@ -33,7 +33,7 @@ class FlowStageSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug
}
}).
runWith(Sink.publisher)
val subscriber = StreamTestKit.SubscriberProbe[Int]()
val subscriber = TestSubscriber.manualProbe[Int]()
p2.subscribe(subscriber)
val subscription = subscriber.expectSubscription()
subscription.request(1)
@ -67,7 +67,7 @@ class FlowStageSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug
}).
runWith(Sink.publisher)
val subscriber = StreamTestKit.SubscriberProbe[Int]()
val subscriber = TestSubscriber.manualProbe[Int]()
p2.subscribe(subscriber)
val subscription = subscriber.expectSubscription()
subscription.request(4)
@ -105,7 +105,7 @@ class FlowStageSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug
}
}).runWith(Sink.publisher)
val subscriber = StreamTestKit.SubscriberProbe[Int]()
val subscriber = TestSubscriber.manualProbe[Int]()
p.subscribe(subscriber)
val subscription = subscriber.expectSubscription()
subscription.request(50)
@ -142,7 +142,7 @@ class FlowStageSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug
}
}).
runWith(Sink.publisher)
val subscriber = StreamTestKit.SubscriberProbe[Int]()
val subscriber = TestSubscriber.manualProbe[Int]()
p2.subscribe(subscriber)
val subscription = subscriber.expectSubscription()
subscription.request(1)
@ -172,10 +172,10 @@ class FlowStageSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug
}
}).
runWith(Sink.fanoutPublisher(2, 2))
val c1 = StreamTestKit.SubscriberProbe[Int]()
val c1 = TestSubscriber.manualProbe[Int]()
p2.subscribe(c1)
val sub1 = c1.expectSubscription()
val c2 = StreamTestKit.SubscriberProbe[Int]()
val c2 = TestSubscriber.manualProbe[Int]()
p2.subscribe(c2)
val sub2 = c2.expectSubscription()
sub1.request(1)
@ -208,7 +208,7 @@ class FlowStageSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug
terminationEmit(Iterator.single(s + "B"), ctx)
}).
runWith(Sink.publisher)
val c = StreamTestKit.SubscriberProbe[String]()
val c = TestSubscriber.manualProbe[String]()
p2.subscribe(c)
val s = c.expectSubscription()
s.request(1)
@ -217,7 +217,7 @@ class FlowStageSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug
}
"allow early finish" in assertAllStagesStopped {
val p = StreamTestKit.PublisherProbe[Int]()
val p = TestPublisher.manualProbe[Int]()
val p2 = Source(p).
transform(() new PushStage[Int, Int] {
var s = ""
@ -231,7 +231,7 @@ class FlowStageSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug
}).
runWith(Sink.publisher)
val proc = p.expectSubscription
val c = StreamTestKit.SubscriberProbe[Int]()
val c = TestSubscriber.manualProbe[Int]()
p2.subscribe(c)
val s = c.expectSubscription()
s.request(10)
@ -257,7 +257,7 @@ class FlowStageSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug
}
}).
runWith(Sink.publisher)
val subscriber = StreamTestKit.SubscriberProbe[Int]()
val subscriber = TestSubscriber.manualProbe[Int]()
p2.subscribe(subscriber)
val subscription = subscriber.expectSubscription()
EventFilter[IllegalArgumentException]("two not allowed") intercept {
@ -284,7 +284,7 @@ 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
runWith(Sink.publisher)
val subscriber = StreamTestKit.SubscriberProbe[Int]()
val subscriber = TestSubscriber.manualProbe[Int]()
p2.subscribe(subscriber)
val subscription = subscriber.expectSubscription()
EventFilter[IllegalArgumentException]("two not allowed") intercept {
@ -306,7 +306,7 @@ class FlowStageSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug
}
}).
runWith(Sink.publisher)
val subscriber = StreamTestKit.SubscriberProbe[Int]()
val subscriber = TestSubscriber.manualProbe[Int]()
p2.subscribe(subscriber)
val subscription = subscriber.expectSubscription()
subscription.request(2)
@ -329,7 +329,7 @@ class FlowStageSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug
terminationEmit(Iterator(1, 2, 3), ctx)
}).
runWith(Sink.publisher)
val subscriber = StreamTestKit.SubscriberProbe[Int]()
val subscriber = TestSubscriber.manualProbe[Int]()
p2.subscribe(subscriber)
val subscription = subscriber.expectSubscription()
subscription.request(4)
@ -341,7 +341,7 @@ class FlowStageSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug
}
"support converting onComplete into onError" in {
val subscriber = StreamTestKit.SubscriberProbe[Int]()
val subscriber = TestSubscriber.manualProbe[Int]()
Source(List(5, 1, 2, 3)).transform(() new PushStage[Int, Int] {
var expectedNumberOfElements: Option[Int] = None
var count = 0
@ -382,13 +382,13 @@ class FlowStageSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug
}
})
val s1 = StreamTestKit.SubscriberProbe[Int]()
val s1 = TestSubscriber.manualProbe[Int]()
flow.to(Sink(s1)).run()
s1.expectSubscription().request(3)
s1.expectNext(1, 2, 3)
s1.expectComplete()
val s2 = StreamTestKit.SubscriberProbe[Int]()
val s2 = TestSubscriber.manualProbe[Int]()
flow.to(Sink(s2)).run()
s2.expectSubscription().request(3)
s2.expectNext(1, 2, 3)
@ -397,7 +397,7 @@ class FlowStageSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug
"handle early cancelation" in assertAllStagesStopped {
val onDownstreamFinishProbe = TestProbe()
val down = StreamTestKit.SubscriberProbe[Int]()
val down = TestSubscriber.manualProbe[Int]()
val s = Source.subscriber[Int].
transform(() new PushStage[Int, Int] {
override def onPush(elem: Int, ctx: Context[Int]) =
@ -413,7 +413,7 @@ class FlowStageSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug
downstream.cancel()
onDownstreamFinishProbe.expectMsg("onDownstreamFinish")
val up = StreamTestKit.PublisherProbe[Int]
val up = TestPublisher.manualProbe[Int]
up.subscribe(s)
val upsub = up.expectSubscription()
upsub.expectCancellation()

View file

@ -7,10 +7,9 @@ import scala.collection.immutable
import scala.concurrent.duration._
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.AkkaSpec
import akka.stream.testkit._
import scala.util.control.NoStackTrace
import scala.concurrent.Await
import akka.stream.testkit.StreamTestKit.SubscriberProbe
import akka.stream.Supervision
import akka.stream.impl.ReactiveStreamsCompliance
import akka.stream.ActorOperationAttributes

View file

@ -10,9 +10,7 @@ import akka.stream.ActorFlowMaterializerSettings
import akka.stream.actor.ActorSubscriberMessage.OnComplete
import akka.stream.actor.ActorSubscriberMessage.OnNext
import akka.stream.impl.RequestMore
import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.ScriptedTest
import akka.stream.testkit.StreamTestKit
import akka.stream.testkit._
class FlowTakeSpec extends AkkaSpec with ScriptedTest {
@ -34,7 +32,7 @@ class FlowTakeSpec extends AkkaSpec with ScriptedTest {
}
"not take anything for negative n" in {
val probe = StreamTestKit.SubscriberProbe[Int]()
val probe = TestSubscriber.manualProbe[Int]()
Source(List(1, 2, 3)).take(-1).to(Sink(probe)).run()
probe.expectSubscription().request(10)
probe.expectComplete()

View file

@ -6,9 +6,8 @@ package akka.stream.scaladsl
import scala.concurrent.duration._
import akka.stream.ActorFlowMaterializer
import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.StreamTestKit
import akka.stream.testkit.StreamTestKit.assertAllStagesStopped
import akka.stream.testkit._
import akka.stream.testkit.Utils._
class FlowTakeWithinSpec extends AkkaSpec {
@ -18,8 +17,8 @@ class FlowTakeWithinSpec extends AkkaSpec {
"deliver elements within the duration, but not afterwards" in {
val input = Iterator.from(1)
val p = StreamTestKit.PublisherProbe[Int]()
val c = StreamTestKit.SubscriberProbe[Int]()
val p = TestPublisher.manualProbe[Int]()
val c = TestSubscriber.manualProbe[Int]()
Source(p).takeWithin(1.second).to(Sink(c)).run()
val pSub = p.expectSubscription()
val cSub = c.expectSubscription()
@ -39,7 +38,7 @@ class FlowTakeWithinSpec extends AkkaSpec {
}
"deliver bufferd elements onComplete before the timeout" in assertAllStagesStopped {
val c = StreamTestKit.SubscriberProbe[Int]()
val c = TestSubscriber.manualProbe[Int]()
Source(1 to 3).takeWithin(1.second).to(Sink(c)).run()
val cSub = c.expectSubscription()
c.expectNoMsg(200.millis)

View file

@ -9,8 +9,8 @@ import scala.util.control.NoStackTrace
import akka.stream.ActorFlowMaterializer
import akka.stream.TimerTransformer
import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
import akka.stream.testkit.StreamTestKit.assertAllStagesStopped
import akka.stream.testkit._
import akka.stream.testkit.Utils._
class FlowTimerTransformerSpec extends AkkaSpec {
@ -18,7 +18,7 @@ class FlowTimerTransformerSpec extends AkkaSpec {
"A Flow with TimerTransformer operations" must {
"produce scheduled ticks as expected" in assertAllStagesStopped {
val p = StreamTestKit.PublisherProbe[Int]()
val p = TestPublisher.manualProbe[Int]()
val p2 = Source(p).
timerTransform(() new TimerTransformer[Int, Int] {
schedulePeriodically("tick", 100.millis)
@ -32,7 +32,7 @@ class FlowTimerTransformerSpec extends AkkaSpec {
override def isComplete: Boolean = !isTimerActive("tick")
}).
runWith(Sink.publisher)
val subscriber = StreamTestKit.SubscriberProbe[Int]()
val subscriber = TestSubscriber.manualProbe[Int]()
p2.subscribe(subscriber)
val subscription = subscriber.expectSubscription()
subscription.request(5)
@ -43,7 +43,7 @@ class FlowTimerTransformerSpec extends AkkaSpec {
}
"schedule ticks when last transformation step (consume)" in {
val p = StreamTestKit.PublisherProbe[Int]()
val p = TestPublisher.manualProbe[Int]()
val p2 = Source(p).
timerTransform(() new TimerTransformer[Int, Int] {
schedulePeriodically("tick", 100.millis)
@ -67,7 +67,7 @@ class FlowTimerTransformerSpec extends AkkaSpec {
"propagate error if onTimer throws an exception" in assertAllStagesStopped {
val exception = new Exception("Expected exception to the rule") with NoStackTrace
val p = StreamTestKit.PublisherProbe[Int]()
val p = TestPublisher.manualProbe[Int]()
val p2 = Source(p).
timerTransform(() new TimerTransformer[Int, Int] {
scheduleOnce("tick", 100.millis)
@ -77,7 +77,7 @@ class FlowTimerTransformerSpec extends AkkaSpec {
throw exception
}).runWith(Sink.publisher)
val subscriber = StreamTestKit.SubscriberProbe[Int]()
val subscriber = TestSubscriber.manualProbe[Int]()
p2.subscribe(subscriber)
val subscription = subscriber.expectSubscription()
subscription.request(5)

View file

@ -5,11 +5,9 @@ package akka.stream.scaladsl
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.StreamTestKit.SubscriberProbe
import akka.stream.testkit.StreamTestKit
import org.reactivestreams.Subscriber
import akka.stream.testkit._
import akka.stream._
import org.reactivestreams.Subscriber
object GraphFlowSpec {
val source1 = Source(0 to 3)
@ -46,7 +44,7 @@ class GraphFlowSpec extends AkkaSpec {
implicit val materializer = ActorFlowMaterializer(settings)
def validateProbe(probe: SubscriberProbe[Int], requests: Int, result: Set[Int]): Unit = {
def validateProbe(probe: TestSubscriber.ManualProbe[Int], requests: Int, result: Set[Int]): Unit = {
val subscription = probe.expectSubscription()
val collected = (1 to requests).map { _
@ -62,7 +60,7 @@ class GraphFlowSpec extends AkkaSpec {
"FlowGraphs" when {
"turned into flows" should {
"work with a Source and Sink" in {
val probe = StreamTestKit.SubscriberProbe[Int]()
val probe = TestSubscriber.manualProbe[Int]()
val flow = Flow(partialGraph) { implicit b
partial
@ -77,7 +75,7 @@ class GraphFlowSpec extends AkkaSpec {
}
"be transformable with a Pipe" in {
val probe = StreamTestKit.SubscriberProbe[Int]()
val probe = TestSubscriber.manualProbe[Int]()
val flow = Flow(partialGraph) { implicit b
partial
@ -90,7 +88,7 @@ class GraphFlowSpec extends AkkaSpec {
}
"work with another GraphFlow" in {
val probe = StreamTestKit.SubscriberProbe[Int]()
val probe = TestSubscriber.manualProbe[Int]()
val flow1 = Flow(partialGraph) { implicit b
partial
@ -108,7 +106,7 @@ class GraphFlowSpec extends AkkaSpec {
}
"be reusable multiple times" in {
val probe = StreamTestKit.SubscriberProbe[Int]()
val probe = TestSubscriber.manualProbe[Int]()
val flow = Flow(Flow[Int].map(_ * 2)) { implicit b
importFlow
@ -126,7 +124,7 @@ class GraphFlowSpec extends AkkaSpec {
"turned into sources" should {
"work with a Sink" in {
val probe = StreamTestKit.SubscriberProbe[Int]()
val probe = TestSubscriber.manualProbe[Int]()
val source = Source(partialGraph) { implicit b
partial
@ -141,7 +139,7 @@ class GraphFlowSpec extends AkkaSpec {
}
"work with a Sink when having KeyedSource inside" in {
val probe = StreamTestKit.SubscriberProbe[Int]()
val probe = TestSubscriber.manualProbe[Int]()
val source = Source.apply(Source.subscriber[Int]) { implicit b
subSource
@ -156,7 +154,7 @@ class GraphFlowSpec extends AkkaSpec {
"be transformable with a Pipe" in {
val probe = StreamTestKit.SubscriberProbe[Int]()
val probe = TestSubscriber.manualProbe[Int]()
val source = Source(partialGraph) { implicit b
partial
@ -171,7 +169,7 @@ class GraphFlowSpec extends AkkaSpec {
}
"work with an GraphFlow" in {
val probe = StreamTestKit.SubscriberProbe[Int]()
val probe = TestSubscriber.manualProbe[Int]()
val source = Source(partialGraph) { implicit b
partial
@ -191,7 +189,7 @@ class GraphFlowSpec extends AkkaSpec {
}
"be reusable multiple times" in {
val probe = StreamTestKit.SubscriberProbe[Int]()
val probe = TestSubscriber.manualProbe[Int]()
val source = Source(Source(1 to 5)) { implicit b
s
@ -214,7 +212,7 @@ class GraphFlowSpec extends AkkaSpec {
"turned into sinks" should {
"work with a Source" in {
val probe = StreamTestKit.SubscriberProbe[Int]()
val probe = TestSubscriber.manualProbe[Int]()
val sink = Sink(partialGraph) { implicit b
partial
@ -229,7 +227,7 @@ class GraphFlowSpec extends AkkaSpec {
}
"work with a Source when having KeyedSink inside" in {
val probe = StreamTestKit.SubscriberProbe[Int]()
val probe = TestSubscriber.manualProbe[Int]()
val pubSink = Sink.publisher[Int]
val sink = Sink(pubSink) { implicit b
@ -244,7 +242,7 @@ class GraphFlowSpec extends AkkaSpec {
}
"be transformable with a Pipe" in {
val probe = StreamTestKit.SubscriberProbe[Int]()
val probe = TestSubscriber.manualProbe[Int]()
val sink = Sink(partialGraph, Flow[String].map(_.toInt))(Keep.both) { implicit b
(partial, flow)
@ -262,7 +260,7 @@ class GraphFlowSpec extends AkkaSpec {
"work with a GraphFlow" in {
val probe = StreamTestKit.SubscriberProbe[Int]()
val probe = TestSubscriber.manualProbe[Int]()
val flow = Flow(partialGraph) { implicit b
partial
@ -284,7 +282,7 @@ class GraphFlowSpec extends AkkaSpec {
"used together" should {
"materialize properly" in {
val probe = StreamTestKit.SubscriberProbe[Int]()
val probe = TestSubscriber.manualProbe[Int]()
val inSource = Source.subscriber[Int]
val outSink = Sink.publisher[Int]
@ -324,7 +322,7 @@ class GraphFlowSpec extends AkkaSpec {
}
"allow connecting source to sink directly" in {
val probe = StreamTestKit.SubscriberProbe[Int]()
val probe = TestSubscriber.manualProbe[Int]()
val inSource = Source.subscriber[Int]
val outSink = Sink.publisher[Int]

View file

@ -6,8 +6,8 @@ import scala.concurrent.duration._
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
import akka.stream.testkit.StreamTestKit.assertAllStagesStopped
import akka.stream.testkit._
import akka.stream.testkit.Utils._
class GraphBalanceSpec extends AkkaSpec {
@ -20,8 +20,8 @@ class GraphBalanceSpec extends AkkaSpec {
import FlowGraph.Implicits._
"balance between subscribers which signal demand" in assertAllStagesStopped {
val c1 = StreamTestKit.SubscriberProbe[Int]()
val c2 = StreamTestKit.SubscriberProbe[Int]()
val c1 = TestSubscriber.manualProbe[Int]()
val c2 = TestSubscriber.manualProbe[Int]()
FlowGraph.closed() { implicit b
val balance = b.add(Balance[Int](2))
@ -45,7 +45,7 @@ class GraphBalanceSpec extends AkkaSpec {
}
"support waiting for demand from all downstream subscriptions" in {
val s1 = StreamTestKit.SubscriberProbe[Int]()
val s1 = TestSubscriber.manualProbe[Int]()
val p2 = FlowGraph.closed(Sink.publisher[Int]) { implicit b
p2Sink
val balance = b.add(Balance[Int](2, waitForAllDownstreams = true))
@ -58,7 +58,7 @@ class GraphBalanceSpec extends AkkaSpec {
sub1.request(1)
s1.expectNoMsg(200.millis)
val s2 = StreamTestKit.SubscriberProbe[Int]()
val s2 = TestSubscriber.manualProbe[Int]()
p2.subscribe(s2)
val sub2 = s2.expectSubscription()
@ -74,7 +74,7 @@ class GraphBalanceSpec extends AkkaSpec {
}
"support waiting for demand from all non-cancelled downstream subscriptions" in assertAllStagesStopped {
val s1 = StreamTestKit.SubscriberProbe[Int]()
val s1 = TestSubscriber.manualProbe[Int]()
val (p2, p3) = FlowGraph.closed(Sink.publisher[Int], Sink.publisher[Int])(Keep.both) { implicit b
(p2Sink, p3Sink)
@ -88,11 +88,11 @@ class GraphBalanceSpec extends AkkaSpec {
val sub1 = s1.expectSubscription()
sub1.request(1)
val s2 = StreamTestKit.SubscriberProbe[Int]()
val s2 = TestSubscriber.manualProbe[Int]()
p2.subscribe(s2)
val sub2 = s2.expectSubscription()
val s3 = StreamTestKit.SubscriberProbe[Int]()
val s3 = TestSubscriber.manualProbe[Int]()
p3.subscribe(s3)
val sub3 = s3.expectSubscription()
@ -143,8 +143,8 @@ class GraphBalanceSpec extends AkkaSpec {
}
"produce to second even though first cancels" in assertAllStagesStopped {
val c1 = StreamTestKit.SubscriberProbe[Int]()
val c2 = StreamTestKit.SubscriberProbe[Int]()
val c1 = TestSubscriber.manualProbe[Int]()
val c2 = TestSubscriber.manualProbe[Int]()
FlowGraph.closed() { implicit b
val balance = b.add(Balance[Int](2))
@ -164,8 +164,8 @@ class GraphBalanceSpec extends AkkaSpec {
}
"produce to first even though second cancels" in assertAllStagesStopped {
val c1 = StreamTestKit.SubscriberProbe[Int]()
val c2 = StreamTestKit.SubscriberProbe[Int]()
val c1 = TestSubscriber.manualProbe[Int]()
val c2 = TestSubscriber.manualProbe[Int]()
FlowGraph.closed() { implicit b
val balance = b.add(Balance[Int](2))
@ -185,9 +185,9 @@ class GraphBalanceSpec extends AkkaSpec {
}
"cancel upstream when downstreams cancel" in assertAllStagesStopped {
val p1 = StreamTestKit.PublisherProbe[Int]()
val c1 = StreamTestKit.SubscriberProbe[Int]()
val c2 = StreamTestKit.SubscriberProbe[Int]()
val p1 = TestPublisher.manualProbe[Int]()
val c1 = TestSubscriber.manualProbe[Int]()
val c2 = TestSubscriber.manualProbe[Int]()
FlowGraph.closed() { implicit b
val balance = b.add(Balance[Int](2))

View file

@ -5,8 +5,8 @@ import scala.concurrent.duration._
import akka.stream.{ OverflowStrategy, ActorFlowMaterializerSettings }
import akka.stream.ActorFlowMaterializer
import akka.stream.testkit.{ StreamTestKit, AkkaSpec }
import akka.stream.testkit.StreamTestKit.assertAllStagesStopped
import akka.stream.testkit._
import akka.stream.testkit.Utils._
class GraphBroadcastSpec extends AkkaSpec {
@ -19,8 +19,8 @@ class GraphBroadcastSpec extends AkkaSpec {
import FlowGraph.Implicits._
"broadcast to other subscriber" in assertAllStagesStopped {
val c1 = StreamTestKit.SubscriberProbe[Int]()
val c2 = StreamTestKit.SubscriberProbe[Int]()
val c1 = TestSubscriber.manualProbe[Int]()
val c2 = TestSubscriber.manualProbe[Int]()
FlowGraph.closed() { implicit b
val bcast = b.add(Broadcast[Int](2))
@ -119,8 +119,8 @@ class GraphBroadcastSpec extends AkkaSpec {
}
"produce to other even though downstream cancels" in assertAllStagesStopped {
val c1 = StreamTestKit.SubscriberProbe[Int]()
val c2 = StreamTestKit.SubscriberProbe[Int]()
val c1 = TestSubscriber.manualProbe[Int]()
val c2 = TestSubscriber.manualProbe[Int]()
FlowGraph.closed() { implicit b
val bcast = b.add(Broadcast[Int](2))
@ -140,8 +140,8 @@ class GraphBroadcastSpec extends AkkaSpec {
}
"produce to downstream even though other cancels" in assertAllStagesStopped {
val c1 = StreamTestKit.SubscriberProbe[Int]()
val c2 = StreamTestKit.SubscriberProbe[Int]()
val c1 = TestSubscriber.manualProbe[Int]()
val c2 = TestSubscriber.manualProbe[Int]()
FlowGraph.closed() { implicit b
val bcast = b.add(Broadcast[Int](2))
@ -161,9 +161,9 @@ class GraphBroadcastSpec extends AkkaSpec {
}
"cancel upstream when downstreams cancel" in assertAllStagesStopped {
val p1 = StreamTestKit.PublisherProbe[Int]()
val c1 = StreamTestKit.SubscriberProbe[Int]()
val c2 = StreamTestKit.SubscriberProbe[Int]()
val p1 = TestPublisher.manualProbe[Int]()
val c1 = TestSubscriber.manualProbe[Int]()
val c2 = TestSubscriber.manualProbe[Int]()
FlowGraph.closed() { implicit b
val bcast = b.add(Broadcast[Int](2))
@ -190,8 +190,8 @@ class GraphBroadcastSpec extends AkkaSpec {
}
"pass along early cancellation" in assertAllStagesStopped {
val c1 = StreamTestKit.SubscriberProbe[Int]()
val c2 = StreamTestKit.SubscriberProbe[Int]()
val c1 = TestSubscriber.manualProbe[Int]()
val c2 = TestSubscriber.manualProbe[Int]()
val sink = Sink() { implicit b
val bcast = b.add(Broadcast[Int](2))
@ -202,7 +202,7 @@ class GraphBroadcastSpec extends AkkaSpec {
val s = Source.subscriber[Int].to(sink).run()
val up = StreamTestKit.PublisherProbe[Int]()
val up = TestPublisher.manualProbe[Int]()
val downsub1 = c1.expectSubscription()
val downsub2 = c2.expectSubscription()

View file

@ -4,13 +4,12 @@
package akka.stream.scaladsl
import scala.concurrent.{ Await, Promise }
import scala.concurrent.duration._
import akka.stream._
import akka.stream.scaladsl._
import akka.stream.testkit.StreamTestKit
import akka.stream.testkit.TwoStreamsSetup
import scala.concurrent.duration._
import akka.stream.testkit.StreamTestKit.assertAllStagesStopped
import akka.stream.testkit._
import akka.stream.testkit.Utils._
class GraphConcatSpec extends TwoStreamsSetup {
@ -29,7 +28,7 @@ class GraphConcatSpec extends TwoStreamsSetup {
import FlowGraph.Implicits._
"work in the happy case" in assertAllStagesStopped {
val probe = StreamTestKit.SubscriberProbe[Int]()
val probe = TestSubscriber.manualProbe[Int]()
FlowGraph.closed() { implicit b
@ -139,7 +138,7 @@ class GraphConcatSpec extends TwoStreamsSetup {
"correctly handle async errors in secondary upstream" in assertAllStagesStopped {
val promise = Promise[Int]()
val subscriber = StreamTestKit.SubscriberProbe[Int]()
val subscriber = TestSubscriber.manualProbe[Int]()
FlowGraph.closed() { implicit b
val concat = b add Concat[Int]()
@ -185,4 +184,3 @@ class GraphConcatSpec extends TwoStreamsSetup {
}
}
}

View file

@ -5,9 +5,9 @@ package akka.stream.scaladsl
import akka.stream.ActorFlowMaterializer
import akka.stream.scaladsl.FlexiMerge._
import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.StreamTestKit.{ PublisherProbe, AutoPublisher, OnNext, SubscriberProbe }
import akka.stream.testkit.StreamTestKit.assertAllStagesStopped
import akka.stream.testkit._
import akka.stream.testkit.scaladsl._
import akka.stream.testkit.Utils._
import org.reactivestreams.Publisher
import akka.stream._
import scala.util.control.NoStackTrace
@ -178,7 +178,7 @@ class GraphFlexiMergeSpec extends AkkaSpec {
"FlexiMerge" must {
"build simple fair merge" in assertAllStagesStopped {
val p = FlowGraph.closed(out) { implicit b
FlowGraph.closed(TestSink.probe[String]) { implicit b
o
val merge = b.add(fairString)
@ -186,18 +186,13 @@ class GraphFlexiMergeSpec extends AkkaSpec {
in2 ~> merge.in(1)
merge.out ~> o.inlet
}.run()
val s = SubscriberProbe[String]
p.subscribe(s)
val sub = s.expectSubscription()
sub.request(10)
(s.probe.receiveN(6).map { case OnNext(elem) elem }).toSet should be(
Set("a", "b", "c", "d", "e", "f"))
s.expectComplete()
.request(10)
.expectNextUnordered("a", "b", "c", "d", "e", "f")
.expectComplete()
}
"be able to have two fleximerges in a graph" in assertAllStagesStopped {
val p = FlowGraph.closed(in1, in2, out)((i1, i2, o) o) { implicit b
FlowGraph.closed(in1, in2, TestSink.probe[String])((i1, i2, o) o) { implicit b
(in1, in2, o)
val m1 = b.add(fairString)
val m2 = b.add(fairString)
@ -211,14 +206,9 @@ class GraphFlexiMergeSpec extends AkkaSpec {
m2.out ~> o.inlet
// format: ON
}.run()
val s = SubscriberProbe[String]
p.subscribe(s)
val sub = s.expectSubscription()
sub.request(20)
(s.probe.receiveN(12).map { case OnNext(elem) elem }).toSet should be(
Set("a", "b", "c", "d", "e", "f", "A", "B", "C", "D", "E", "F"))
s.expectComplete()
.request(20)
.expectNextUnordered("a", "b", "c", "d", "e", "f", "A", "B", "C", "D", "E", "F")
.expectComplete()
}
"allow reuse" in {
@ -239,11 +229,11 @@ class GraphFlexiMergeSpec extends AkkaSpec {
}
val p = g.run()
val s = SubscriberProbe[String]
val s = TestSubscriber.manualProbe[String]
p.subscribe(s)
val sub = s.expectSubscription()
sub.request(1000)
val received = s.probe.receiveN(1000).map { case OnNext(elem: String) elem }
val received = for (_ 1 to 1000) yield s.expectNext()
val first = received.map(_.charAt(1))
first.toSet should ===(Set('a', 'b', 'c', 'd', '+'))
first.filter(_ != '+') should ===(Seq('a', 'b', 'c', 'd'))
@ -262,7 +252,7 @@ class GraphFlexiMergeSpec extends AkkaSpec {
(zip.in1, zip.out)
}
val g = FlowGraph.closed(out) { implicit b
FlowGraph.closed(TestSink.probe[String]) { implicit b
o
val zip = b.add(Zip[String, String]())
@ -270,15 +260,10 @@ class GraphFlexiMergeSpec extends AkkaSpec {
in2 ~> zip.in1
zip.out.map(_.toString()) ~> o.inlet
}
val p = g.run()
val s = SubscriberProbe[String]
p.subscribe(s)
val sub = s.expectSubscription()
sub.request(100)
(s.probe.receiveN(2).map { case OnNext(elem) elem }).toSet should be(Set("((+,b),f)", "((+,a),e)"))
s.expectComplete()
}.run()
.request(100)
.expectNextUnordered("((+,b),f)", "((+,a),e)")
.expectComplete()
}
"build simple round robin merge" in {
@ -290,7 +275,7 @@ class GraphFlexiMergeSpec extends AkkaSpec {
merge.out ~> o.inlet
}.run()
val s = SubscriberProbe[String]
val s = TestSubscriber.manualProbe[String]
p.subscribe(s)
val sub = s.expectSubscription()
sub.request(10)
@ -312,7 +297,7 @@ class GraphFlexiMergeSpec extends AkkaSpec {
merge.out ~> o.inlet
}.run()
val s = SubscriberProbe[(Int, String)]
val s = TestSubscriber.manualProbe[(Int, String)]
p.subscribe(s)
val sub = s.expectSubscription()
sub.request(10)
@ -334,7 +319,7 @@ class GraphFlexiMergeSpec extends AkkaSpec {
// format: ON
}.run()
val s = SubscriberProbe[(Long, Int, String)]
val s = TestSubscriber.manualProbe[(Long, Int, String)]
p.subscribe(s)
val sub = s.expectSubscription()
@ -356,7 +341,7 @@ class GraphFlexiMergeSpec extends AkkaSpec {
// format: ON
}.run()
val s = SubscriberProbe[(Long, Int, String)]
val s = TestSubscriber.manualProbe[(Long, Int, String)]
p.subscribe(s)
val sub = s.expectSubscription()
@ -379,7 +364,7 @@ class GraphFlexiMergeSpec extends AkkaSpec {
merge.out ~> o.inlet
}.run()
val s = SubscriberProbe[Int]
val s = TestSubscriber.manualProbe[Int]
p.subscribe(s)
val sub = s.expectSubscription()
@ -408,9 +393,9 @@ class GraphFlexiMergeSpec extends AkkaSpec {
"build perferring merge, manually driven" in {
val output = Sink.publisher[Int]
val preferredDriver = PublisherProbe[Int]()
val otherDriver1 = PublisherProbe[Int]()
val otherDriver2 = PublisherProbe[Int]()
val preferredDriver = TestPublisher.manualProbe[Int]()
val otherDriver1 = TestPublisher.manualProbe[Int]()
val otherDriver2 = TestPublisher.manualProbe[Int]()
val p = FlowGraph.closed(output) { implicit b
o
@ -421,7 +406,7 @@ class GraphFlexiMergeSpec extends AkkaSpec {
merge.out ~> o.inlet
}.run()
val s = SubscriberProbe[Int]
val s = TestSubscriber.manualProbe[Int]
p.subscribe(s)
val sub = s.expectSubscription()
@ -464,21 +449,20 @@ class GraphFlexiMergeSpec extends AkkaSpec {
}
"support cancel of input" in assertAllStagesStopped {
val publisher = PublisherProbe[String]
val autoPublisher = TestPublisher.probe[String]()
val completionProbe = TestProbe()
val p = FlowGraph.closed(out) { implicit b
o
val merge = b.add(new TestMerge(completionProbe.ref))
Source(publisher) ~> merge.in(0)
Source(autoPublisher) ~> merge.in(0)
Source(List("b", "c", "d")) ~> merge.in(1)
Source(List("e", "f")) ~> merge.in(2)
merge.out ~> o.inlet
}.run()
val s = SubscriberProbe[String]
val s = TestSubscriber.manualProbe[String]
p.subscribe(s)
val autoPublisher = new AutoPublisher(publisher)
autoPublisher.sendNext("a")
autoPublisher.sendNext("cancel")
@ -503,35 +487,32 @@ class GraphFlexiMergeSpec extends AkkaSpec {
}
"finish when all inputs cancelled" in assertAllStagesStopped {
val publisher1 = PublisherProbe[String]
val publisher2 = PublisherProbe[String]
val publisher3 = PublisherProbe[String]
val autoPublisher1 = TestPublisher.probe[String]()
val autoPublisher2 = TestPublisher.probe[String]()
val autoPublisher3 = TestPublisher.probe[String]()
val completionProbe = TestProbe()
val p = FlowGraph.closed(out) { implicit b
o
val merge = b.add(new TestMerge(completionProbe.ref))
Source(publisher1) ~> merge.in(0)
Source(publisher2) ~> merge.in(1)
Source(publisher3) ~> merge.in(2)
Source(autoPublisher1) ~> merge.in(0)
Source(autoPublisher2) ~> merge.in(1)
Source(autoPublisher3) ~> merge.in(2)
merge.out ~> o.inlet
}.run()
val s = SubscriberProbe[String]
val s = TestSubscriber.manualProbe[String]
p.subscribe(s)
val sub = s.expectSubscription()
sub.request(10)
val autoPublisher1 = new AutoPublisher(publisher1)
autoPublisher1.sendNext("a")
autoPublisher1.sendNext("cancel")
s.expectNext("onInput: a")
val autoPublisher2 = new AutoPublisher(publisher2)
autoPublisher2.sendNext("b")
autoPublisher2.sendNext("cancel")
s.expectNext("onInput: b")
val autoPublisher3 = new AutoPublisher(publisher3)
autoPublisher3.sendNext("c")
autoPublisher3.sendNext("cancel")
s.expectNext("onInput: c")
@ -550,7 +531,7 @@ class GraphFlexiMergeSpec extends AkkaSpec {
merge.out ~> o.inlet
}.run()
val s = SubscriberProbe[String]
val s = TestSubscriber.manualProbe[String]
p.subscribe(s)
val sub = s.expectSubscription()
sub.request(10)
@ -570,7 +551,7 @@ class GraphFlexiMergeSpec extends AkkaSpec {
}
"propagate failure" in assertAllStagesStopped {
val publisher = PublisherProbe[String]
val publisher = TestPublisher.manualProbe[String]
val completionProbe = TestProbe()
val p = FlowGraph.closed(out) { implicit b
o
@ -581,13 +562,13 @@ class GraphFlexiMergeSpec extends AkkaSpec {
merge.out ~> o.inlet
}.run()
val s = SubscriberProbe[String]
val s = TestSubscriber.manualProbe[String]
p.subscribe(s)
s.expectSubscriptionAndError().getMessage should be("ERROR")
}
"emit failure" in assertAllStagesStopped {
val publisher = PublisherProbe[String]
val publisher = TestPublisher.manualProbe[String]
val completionProbe = TestProbe()
val p = FlowGraph.closed(out) { implicit b
o
@ -598,7 +579,7 @@ class GraphFlexiMergeSpec extends AkkaSpec {
merge.out ~> o.inlet
}.run()
val s = SubscriberProbe[String]
val s = TestSubscriber.manualProbe[String]
p.subscribe(s)
val sub = s.expectSubscription()
sub.request(10)
@ -607,7 +588,7 @@ class GraphFlexiMergeSpec extends AkkaSpec {
}
"emit failure for user thrown exception" in assertAllStagesStopped {
val publisher = PublisherProbe[String]
val publisher = TestPublisher.manualProbe[String]
val completionProbe = TestProbe()
val p = FlowGraph.closed(out) { implicit b
o
@ -618,7 +599,7 @@ class GraphFlexiMergeSpec extends AkkaSpec {
merge.out ~> o.inlet
}.run()
val s = SubscriberProbe[String]
val s = TestSubscriber.manualProbe[String]
p.subscribe(s)
val sub = s.expectSubscription()
sub.request(10)
@ -626,7 +607,7 @@ class GraphFlexiMergeSpec extends AkkaSpec {
}
"emit failure for user thrown exception in onComplete" in assertAllStagesStopped {
val publisher = PublisherProbe[String]
val publisher = TestPublisher.manualProbe[String]
val completionProbe = TestProbe()
val p = FlowGraph.closed(out) { implicit b
o
@ -637,7 +618,7 @@ class GraphFlexiMergeSpec extends AkkaSpec {
merge.out ~> o.inlet
}.run()
val s = SubscriberProbe[String]
val s = TestSubscriber.manualProbe[String]
p.subscribe(s)
val sub = s.expectSubscription()
sub.request(10)
@ -645,22 +626,21 @@ class GraphFlexiMergeSpec extends AkkaSpec {
}
"emit failure for user thrown exception in onUpstreamFinish 2" in assertAllStagesStopped {
val publisher = PublisherProbe[String]
val autoPublisher = TestPublisher.probe[String]()
val completionProbe = TestProbe()
val p = FlowGraph.closed(out) { implicit b
o
val merge = b.add(new TestMerge(completionProbe.ref))
Source.empty[String] ~> merge.in(0)
Source(publisher) ~> merge.in(1)
Source(autoPublisher) ~> merge.in(1)
Source.empty[String] ~> merge.in(2)
merge.out ~> o.inlet
}.run()
val autoPublisher = new AutoPublisher(publisher)
autoPublisher.sendNext("onUpstreamFinish-exc")
autoPublisher.sendNext("a")
val s = SubscriberProbe[String]
val s = TestSubscriber.manualProbe[String]
p.subscribe(s)
val sub = s.expectSubscription()
sub.request(1)
@ -671,7 +651,7 @@ class GraphFlexiMergeSpec extends AkkaSpec {
}
"support finish from onInput" in assertAllStagesStopped {
val publisher = PublisherProbe[String]
val publisher = TestPublisher.manualProbe[String]
val completionProbe = TestProbe()
val p = FlowGraph.closed(out) { implicit b
o
@ -682,7 +662,7 @@ class GraphFlexiMergeSpec extends AkkaSpec {
merge.out ~> o.inlet
}.run()
val s = SubscriberProbe[String]
val s = TestSubscriber.manualProbe[String]
p.subscribe(s)
val sub = s.expectSubscription()
sub.request(10)

View file

@ -4,12 +4,9 @@ import scala.concurrent.duration._
import scala.util.control.NoStackTrace
import FlowGraph.Implicits._
import akka.stream.ActorFlowMaterializer
import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.StreamTestKit.AutoPublisher
import akka.stream.testkit.StreamTestKit.OnNext
import akka.stream.testkit.StreamTestKit.PublisherProbe
import akka.stream.testkit.StreamTestKit.SubscriberProbe
import akka.stream.testkit.StreamTestKit.assertAllStagesStopped
import akka.stream.testkit._
import akka.stream.testkit.scaladsl._
import akka.stream.testkit.Utils._
import akka.actor.ActorSystem
import akka.stream._
import akka.actor.ActorRef
@ -128,18 +125,17 @@ object GraphFlexiRouteSpec {
}
class TestFixture(implicit val system: ActorSystem, implicit val materializer: ActorFlowMaterializer) {
val publisher = PublisherProbe[String]
val s1 = SubscriberProbe[String]
val s2 = SubscriberProbe[String]
val autoPublisher = TestPublisher.probe[String]()
val s1 = TestSubscriber.manualProbe[String]
val s2 = TestSubscriber.manualProbe[String]
val completionProbe = TestProbe()
FlowGraph.closed() { implicit b
val route = b.add(new TestRoute(completionProbe.ref))
Source(publisher) ~> route.in
Source(autoPublisher) ~> route.in
route.out0 ~> Sink(s1)
route.out1 ~> Sink(s2)
}.run()
val autoPublisher = new AutoPublisher(publisher)
autoPublisher.sendNext("a")
autoPublisher.sendNext("b")
@ -164,23 +160,18 @@ class GraphFlexiRouteSpec extends AkkaSpec {
"build simple fair route" in assertAllStagesStopped {
// we can't know exactly which elements that go to each output, because if subscription/request
// from one of the downstream is delayed the elements will be pushed to the other output
val s = SubscriberProbe[String]
val m = FlowGraph.closed() { implicit b
FlowGraph.closed(TestSink.probe[String]) { implicit b
out
val merge = b.add(Merge[String](2))
val route = b.add(new Fair[String])
in ~> route.in
route.out(0) ~> merge.in(0)
route.out(1) ~> merge.in(1)
merge.out ~> Sink(s)
merge.out ~> out
}.run()
val sub = s.expectSubscription()
sub.request(10)
(s.probe.receiveN(5).map { case OnNext(elem) elem }).toSet should be(
Set("a", "b", "c", "d", "e"))
s.expectComplete()
.request(10)
.expectNextUnordered("a", "b", "c", "d", "e")
.expectComplete()
}
"build simple round-robin route" in {
@ -192,10 +183,10 @@ class GraphFlexiRouteSpec extends AkkaSpec {
route.out(1) ~> o2.inlet
}.run()
val s1 = SubscriberProbe[String]
val s1 = TestSubscriber.manualProbe[String]
p1.subscribe(s1)
val sub1 = s1.expectSubscription()
val s2 = SubscriberProbe[String]
val s2 = TestSubscriber.manualProbe[String]
p2.subscribe(s2)
val sub2 = s2.expectSubscription()
@ -224,10 +215,10 @@ class GraphFlexiRouteSpec extends AkkaSpec {
route.out1 ~> ob.inlet
}.run()
val s1 = SubscriberProbe[Int]
val s1 = TestSubscriber.manualProbe[Int]
p1.subscribe(s1)
val sub1 = s1.expectSubscription()
val s2 = SubscriberProbe[String]
val s2 = TestSubscriber.manualProbe[String]
p2.subscribe(s2)
val sub2 = s2.expectSubscription()
@ -273,7 +264,7 @@ class GraphFlexiRouteSpec extends AkkaSpec {
s1.expectError().getMessage should be("err")
s2.expectError().getMessage should be("err")
autoPublisher.subscription.expectCancellation()
autoPublisher.expectCancellation()
}
"support error of a specific output" in assertAllStagesStopped {
@ -314,7 +305,7 @@ class GraphFlexiRouteSpec extends AkkaSpec {
s1.expectError().getMessage should be("exc")
s2.expectError().getMessage should be("exc")
autoPublisher.subscription.expectCancellation()
autoPublisher.expectCancellation()
}
"emit error for user thrown exception in onUpstreamFinish" in assertAllStagesStopped {
@ -412,7 +403,7 @@ class GraphFlexiRouteSpec extends AkkaSpec {
completionProbe.expectMsg("onDownstreamFinish: TestRoute.out0")
sub2.cancel()
autoPublisher.subscription.expectCancellation()
autoPublisher.expectCancellation()
}
"cancel upstream input when all outputs completed" in assertAllStagesStopped {
@ -429,9 +420,8 @@ class GraphFlexiRouteSpec extends AkkaSpec {
autoPublisher.sendNext("finish")
s1.expectComplete()
s2.expectComplete()
autoPublisher.subscription.expectCancellation()
autoPublisher.expectCancellation()
}
}
}

View file

@ -6,8 +6,7 @@ package akka.stream.scaladsl
import akka.stream.OperationAttributes._
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.StreamTestKit
import akka.stream.testkit._
import scala.concurrent.duration._
import scala.concurrent.Await

View file

@ -4,8 +4,7 @@
package akka.stream.scaladsl
import akka.stream.{ ActorFlowMaterializer, ActorFlowMaterializerSettings }
import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.StreamTestKit
import akka.stream.testkit._
import scala.concurrent.Await
import scala.concurrent.Future
@ -25,7 +24,7 @@ class GraphMatValueSpec extends AkkaSpec {
val foldSink = Sink.fold[Int, Int](0)(_ + _)
"expose the materialized value as source" in {
val sub = StreamTestKit.SubscriberProbe[Int]()
val sub = TestSubscriber.manualProbe[Int]()
val f = FlowGraph.closed(foldSink) { implicit b
fold
Source(1 to 10) ~> fold
@ -40,7 +39,7 @@ class GraphMatValueSpec extends AkkaSpec {
}
"expose the materialized value as source multiple times" in {
val sub = StreamTestKit.SubscriberProbe[Int]()
val sub = TestSubscriber.manualProbe[Int]()
val f = FlowGraph.closed(foldSink) { implicit b
fold

View file

@ -7,8 +7,8 @@ import akka.stream.{ ActorFlowMaterializer, ActorFlowMaterializerSettings, Inlet
import scala.concurrent.duration._
import akka.stream.testkit.{ TwoStreamsSetup, AkkaSpec, StreamTestKit }
import akka.stream.testkit.StreamTestKit.assertAllStagesStopped
import akka.stream.testkit._
import akka.stream.testkit.Utils._
class GraphMergeSpec extends TwoStreamsSetup {
import FlowGraph.Implicits._
@ -31,7 +31,7 @@ class GraphMergeSpec extends TwoStreamsSetup {
val source1 = Source(0 to 3)
val source2 = Source(4 to 9)
val source3 = Source(List[Int]())
val probe = StreamTestKit.SubscriberProbe[Int]()
val probe = TestSubscriber.manualProbe[Int]()
FlowGraph.closed() { implicit b
val m1 = b.add(Merge[Int](2))
@ -65,7 +65,7 @@ class GraphMergeSpec extends TwoStreamsSetup {
val source5 = Source(List(5))
val source6 = Source(List[Int]())
val probe = StreamTestKit.SubscriberProbe[Int]()
val probe = TestSubscriber.manualProbe[Int]()
FlowGraph.closed() { implicit b
val merge = b.add(Merge[Int](6))
@ -145,9 +145,9 @@ class GraphMergeSpec extends TwoStreamsSetup {
}
"pass along early cancellation" in assertAllStagesStopped {
val up1 = StreamTestKit.PublisherProbe[Int]
val up2 = StreamTestKit.PublisherProbe[Int]
val down = StreamTestKit.SubscriberProbe[Int]()
val up1 = TestPublisher.manualProbe[Int]
val up2 = TestPublisher.manualProbe[Int]
val down = TestSubscriber.manualProbe[Int]()
val src1 = Source.subscriber[Int]
val src2 = Source.subscriber[Int]

View file

@ -5,8 +5,7 @@ import scala.concurrent.{ Future, Await }
import scala.concurrent.duration._
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.StreamTestKit.{ OnNext, SubscriberProbe }
import akka.stream.testkit._
import akka.util.ByteString
import akka.stream.{ Inlet, Outlet, Shape, Graph }
import org.scalactic.ConversionCheckedTripleEquals
@ -156,7 +155,7 @@ class GraphOpsIntegrationSpec extends AkkaSpec with ConversionCheckedTripleEqual
"be able to run plain flow" in {
val p = Source(List(1, 2, 3)).runWith(Sink.publisher)
val s = SubscriberProbe[Int]
val s = TestSubscriber.manualProbe[Int]
val flow = Flow[Int].map(_ * 2)
FlowGraph.closed() { implicit builder
Source(p) ~> flow ~> Sink(s)

View file

@ -4,8 +4,8 @@ import scala.concurrent.duration._
import akka.stream.{ OverflowStrategy, ActorFlowMaterializerSettings }
import akka.stream.ActorFlowMaterializer
import akka.stream.testkit.{ StreamTestKit, AkkaSpec }
import akka.stream.testkit.StreamTestKit.assertAllStagesStopped
import akka.stream.testkit._
import akka.stream.testkit.Utils._
class GraphUnzipSpec extends AkkaSpec {
@ -18,8 +18,8 @@ class GraphUnzipSpec extends AkkaSpec {
import FlowGraph.Implicits._
"unzip to two subscribers" in assertAllStagesStopped {
val c1 = StreamTestKit.SubscriberProbe[Int]()
val c2 = StreamTestKit.SubscriberProbe[String]()
val c1 = TestSubscriber.manualProbe[Int]()
val c2 = TestSubscriber.manualProbe[String]()
FlowGraph.closed() { implicit b
val unzip = b.add(Unzip[Int, String]())
@ -47,8 +47,8 @@ class GraphUnzipSpec extends AkkaSpec {
}
"produce to right downstream even though left downstream cancels" in {
val c1 = StreamTestKit.SubscriberProbe[Int]()
val c2 = StreamTestKit.SubscriberProbe[String]()
val c1 = TestSubscriber.manualProbe[Int]()
val c2 = TestSubscriber.manualProbe[String]()
FlowGraph.closed() { implicit b
val unzip = b.add(Unzip[Int, String]())
@ -68,8 +68,8 @@ class GraphUnzipSpec extends AkkaSpec {
}
"produce to left downstream even though right downstream cancels" in {
val c1 = StreamTestKit.SubscriberProbe[Int]()
val c2 = StreamTestKit.SubscriberProbe[String]()
val c1 = TestSubscriber.manualProbe[Int]()
val c2 = TestSubscriber.manualProbe[String]()
FlowGraph.closed() { implicit b
val unzip = b.add(Unzip[Int, String]())
@ -89,9 +89,9 @@ class GraphUnzipSpec extends AkkaSpec {
}
"cancel upstream when downstreams cancel" in {
val p1 = StreamTestKit.PublisherProbe[(Int, String)]()
val c1 = StreamTestKit.SubscriberProbe[Int]()
val c2 = StreamTestKit.SubscriberProbe[String]()
val p1 = TestPublisher.manualProbe[(Int, String)]()
val c1 = TestSubscriber.manualProbe[Int]()
val c2 = TestSubscriber.manualProbe[String]()
FlowGraph.closed() { implicit b
val unzip = b.add(Unzip[Int, String]())
@ -118,7 +118,7 @@ class GraphUnzipSpec extends AkkaSpec {
}
"work with zip" in assertAllStagesStopped {
val c1 = StreamTestKit.SubscriberProbe[(Int, String)]()
val c1 = TestSubscriber.manualProbe[(Int, String)]()
FlowGraph.closed() { implicit b
val zip = b.add(Zip[Int, String]())
val unzip = b.add(Unzip[Int, String]())

View file

@ -3,9 +3,8 @@
*/
package akka.stream.scaladsl
import akka.stream.testkit.StreamTestKit
import akka.stream.testkit.TwoStreamsSetup
import akka.stream.testkit.StreamTestKit.assertAllStagesStopped
import akka.stream.testkit._
import akka.stream.testkit.Utils._
import akka.stream._
class GraphZipSpec extends TwoStreamsSetup {
@ -24,7 +23,7 @@ class GraphZipSpec extends TwoStreamsSetup {
"Zip" must {
"work in the happy case" in assertAllStagesStopped {
val probe = StreamTestKit.SubscriberProbe[(Int, String)]()
val probe = TestSubscriber.manualProbe[(Int, String)]()
FlowGraph.closed() { implicit b
val zip = b.add(Zip[Int, String]())

View file

@ -1,7 +1,6 @@
package akka.stream.scaladsl
import akka.stream.testkit.StreamTestKit
import akka.stream.testkit.TwoStreamsSetup
import akka.stream.testkit._
import scala.concurrent.duration._
import akka.stream._
@ -20,7 +19,7 @@ class GraphZipWithSpec extends TwoStreamsSetup {
"ZipWith" must {
"work in the happy case" in {
val probe = StreamTestKit.SubscriberProbe[Outputs]()
val probe = TestSubscriber.manualProbe[Outputs]()
FlowGraph.closed() { implicit b
val zip = b.add(ZipWith((_: Int) + (_: Int)))
@ -45,7 +44,7 @@ class GraphZipWithSpec extends TwoStreamsSetup {
}
"work in the sad case" in {
val probe = StreamTestKit.SubscriberProbe[Outputs]()
val probe = TestSubscriber.manualProbe[Outputs]()
FlowGraph.closed() { implicit b
val zip = b.add(ZipWith[Int, Int, Int]((_: Int) / (_: Int)))
@ -104,7 +103,7 @@ class GraphZipWithSpec extends TwoStreamsSetup {
}
"zipWith a ETA expanded Person.apply (3 inputs)" in {
val probe = StreamTestKit.SubscriberProbe[Person]()
val probe = TestSubscriber.manualProbe[Person]()
case class Person(name: String, surname: String, int: Int)
@ -127,7 +126,7 @@ class GraphZipWithSpec extends TwoStreamsSetup {
}
"work with up to 22 inputs" in {
val probe = StreamTestKit.SubscriberProbe[String]()
val probe = TestSubscriber.manualProbe[String]()
FlowGraph.closed() { implicit b

View file

@ -12,9 +12,8 @@ import scala.util.Failure
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
import akka.stream.testkit.StreamTestKit.assertAllStagesStopped
import akka.stream.testkit.ScriptedTest
import akka.stream.testkit._
import akka.stream.testkit.Utils._
class HeadSinkSpec extends AkkaSpec with ScriptedTest {
@ -26,7 +25,7 @@ class HeadSinkSpec extends AkkaSpec with ScriptedTest {
"A Flow with Sink.head" must {
"yield the first value" in assertAllStagesStopped {
val p = StreamTestKit.PublisherProbe[Int]()
val p = TestPublisher.manualProbe[Int]()
val f: Future[Int] = Source(p).map(identity).runWith(Sink.head)
val proc = p.expectSubscription
proc.expectRequest()
@ -36,7 +35,7 @@ class HeadSinkSpec extends AkkaSpec with ScriptedTest {
}
"yield the first value when actively constructing" in {
val p = StreamTestKit.PublisherProbe[Int]()
val p = TestPublisher.manualProbe[Int]()
val f = Sink.head[Int]
val s = Source.subscriber[Int]
val (subscriber, future) = s.toMat(f)(Keep.both).run()
@ -50,7 +49,7 @@ class HeadSinkSpec extends AkkaSpec with ScriptedTest {
}
"yield the first error" in assertAllStagesStopped {
val p = StreamTestKit.PublisherProbe[Int]()
val p = TestPublisher.manualProbe[Int]()
val f = Source(p).runWith(Sink.head)
val proc = p.expectSubscription
proc.expectRequest()
@ -61,7 +60,7 @@ class HeadSinkSpec extends AkkaSpec with ScriptedTest {
}
"yield NoSuchElementExcption for empty stream" in assertAllStagesStopped {
val p = StreamTestKit.PublisherProbe[Int]()
val p = TestPublisher.manualProbe[Int]()
val f = Source(p).runWith(Sink.head)
val proc = p.expectSubscription
proc.expectRequest()

View file

@ -6,7 +6,7 @@ package akka.stream.scaladsl
import akka.stream.ActorFlowMaterializer
import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.StreamTestKit.assertAllStagesStopped
import akka.stream.testkit.Utils._
import scala.concurrent.duration._
import scala.concurrent.Await

View file

@ -1,11 +1,10 @@
package akka.stream.scaladsl
import akka.stream.testkit.AkkaSpec
import akka.stream.testkit._
import akka.stream._
import scala.concurrent.Await
import scala.concurrent.duration._
import org.scalactic.ConversionCheckedTripleEquals
import akka.stream.testkit.StreamTestKit._
class ReverseArrowSpec extends AkkaSpec with ConversionCheckedTripleEquals {
import FlowGraph.Implicits._
@ -31,7 +30,7 @@ class ReverseArrowSpec extends AkkaSpec with ConversionCheckedTripleEquals {
}
"work from Sink" in {
val sub = SubscriberProbe[Int]
val sub = TestSubscriber.manualProbe[Int]
FlowGraph.closed() { implicit b
Sink(sub) <~ source
}.run()

View file

@ -3,8 +3,7 @@
*/
package akka.stream.scaladsl
import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.StreamTestKit.SubscriberProbe
import akka.stream.testkit._
import akka.stream.ActorFlowMaterializer
class SinkSpec extends AkkaSpec {
@ -15,7 +14,7 @@ class SinkSpec extends AkkaSpec {
"A Sink" must {
"be composable without importing modules" in {
val probes = Array.fill(3)(SubscriberProbe[Int])
val probes = Array.fill(3)(TestSubscriber.manualProbe[Int])
val sink = Sink() { implicit b
val bcast = b.add(Broadcast[Int](3))
for (i 0 to 2) bcast.out(i).filter(_ == i) ~> Sink(probes(i))
@ -32,7 +31,7 @@ class SinkSpec extends AkkaSpec {
}
"be composable with importing 1 module" in {
val probes = Array.fill(3)(SubscriberProbe[Int])
val probes = Array.fill(3)(TestSubscriber.manualProbe[Int])
val sink = Sink(Sink(probes(0))) { implicit b
s0
val bcast = b.add(Broadcast[Int](3))
@ -51,7 +50,7 @@ class SinkSpec extends AkkaSpec {
}
"be composable with importing 2 modules" in {
val probes = Array.fill(3)(SubscriberProbe[Int])
val probes = Array.fill(3)(TestSubscriber.manualProbe[Int])
val sink = Sink(Sink(probes(0)), Sink(probes(1)))(List(_, _)) { implicit b
(s0, s1)
val bcast = b.add(Broadcast[Int](3))
@ -71,7 +70,7 @@ class SinkSpec extends AkkaSpec {
}
"be composable with importing 3 modules" in {
val probes = Array.fill(3)(SubscriberProbe[Int])
val probes = Array.fill(3)(TestSubscriber.manualProbe[Int])
val sink = Sink(Sink(probes(0)), Sink(probes(1)), Sink(probes(2)))(List(_, _, _)) { implicit b
(s0, s1, s2)
val bcast = b.add(Broadcast[Int](3))

View file

@ -8,11 +8,8 @@ import scala.concurrent.duration._
import scala.util.{ Success, Failure }
import scala.util.control.NoStackTrace
import akka.stream.ActorFlowMaterializer
import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.StreamTestKit
import akka.stream.testkit._
import akka.stream.impl.PublisherSource
import akka.stream.testkit.StreamTestKit.PublisherProbe
import akka.stream.testkit.StreamTestKit.SubscriberProbe
import akka.stream.impl.ReactiveStreamsCompliance
class SourceSpec extends AkkaSpec {
@ -22,7 +19,7 @@ class SourceSpec extends AkkaSpec {
"Single Source" must {
"produce element" in {
val p = Source.single(1).runWith(Sink.publisher)
val c = StreamTestKit.SubscriberProbe[Int]()
val c = TestSubscriber.manualProbe[Int]()
p.subscribe(c)
val sub = c.expectSubscription()
sub.request(1)
@ -32,8 +29,8 @@ class SourceSpec extends AkkaSpec {
"reject later subscriber" in {
val p = Source.single(1).runWith(Sink.publisher)
val c1 = StreamTestKit.SubscriberProbe[Int]()
val c2 = StreamTestKit.SubscriberProbe[Int]()
val c1 = TestSubscriber.manualProbe[Int]()
val c2 = TestSubscriber.manualProbe[Int]()
p.subscribe(c1)
val sub1 = c1.expectSubscription()
@ -50,12 +47,12 @@ class SourceSpec extends AkkaSpec {
"Empty Source" must {
"complete immediately" in {
val p = Source.empty.runWith(Sink.publisher)
val c = StreamTestKit.SubscriberProbe[Int]()
val c = TestSubscriber.manualProbe[Int]()
p.subscribe(c)
c.expectSubscriptionAndComplete()
// reject additional subscriber
val c2 = StreamTestKit.SubscriberProbe[Int]()
val c2 = TestSubscriber.manualProbe[Int]()
p.subscribe(c2)
c2.expectSubscriptionAndError()
}
@ -65,12 +62,12 @@ class SourceSpec extends AkkaSpec {
"emit error immediately" in {
val ex = new RuntimeException with NoStackTrace
val p = Source.failed(ex).runWith(Sink.publisher)
val c = StreamTestKit.SubscriberProbe[Int]()
val c = TestSubscriber.manualProbe[Int]()
p.subscribe(c)
c.expectSubscriptionAndError(ex)
// reject additional subscriber
val c2 = StreamTestKit.SubscriberProbe[Int]()
val c2 = TestSubscriber.manualProbe[Int]()
p.subscribe(c2)
c2.expectSubscriptionAndError()
}
@ -83,7 +80,7 @@ class SourceSpec extends AkkaSpec {
val (f, neverPub) = neverSource.toMat(pubSink)(Keep.both).run()
val c = StreamTestKit.SubscriberProbe()
val c = TestSubscriber.manualProbe()
neverPub.subscribe(c)
val subs = c.expectSubscription()
@ -125,9 +122,9 @@ class SourceSpec extends AkkaSpec {
"Composite Source" must {
"merge from many inputs" in {
val probes = Seq.fill(5)(PublisherProbe[Int])
val probes = Seq.fill(5)(TestPublisher.manualProbe[Int])
val source = Source.subscriber[Int]
val out = SubscriberProbe[Int]
val out = TestSubscriber.manualProbe[Int]
val s = Source(source, source, source, source, source)(Seq(_, _, _, _, _)) { implicit b
(i0, i1, i2, i3, i4)

View file

@ -5,9 +5,8 @@ package akka.stream.scaladsl
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.StreamTestKit
import akka.stream.testkit.StreamTestKit.assertAllStagesStopped
import akka.stream.testkit._
import akka.stream.testkit.Utils._
class SubscriberSinkSpec extends AkkaSpec {
@ -19,7 +18,7 @@ class SubscriberSinkSpec extends AkkaSpec {
"A Flow with SubscriberSink" must {
"publish elements to the subscriber" in assertAllStagesStopped {
val c = StreamTestKit.SubscriberProbe[Int]()
val c = TestSubscriber.manualProbe[Int]()
Source(List(1, 2, 3)).to(Sink(c)).run()
val s = c.expectSubscription()
s.request(3)

View file

@ -7,7 +7,7 @@ import akka.actor.{ ExtendedActorSystem, ActorIdentity, ActorRef, Identify }
import akka.stream.{ ActorFlowMaterializer, ActorFlowMaterializerSettings }
import akka.stream.impl.SubscriptionTimeoutException
import akka.stream.testkit._
import akka.stream.testkit.StreamTestKit.assertAllStagesStopped
import akka.stream.testkit.Utils._
import akka.util.Timeout
import scala.concurrent.Await
@ -40,9 +40,9 @@ class SubstreamSubscriptionTimeoutSpec(conf: String) extends AkkaSpec(conf) {
"groupBy" must {
"timeout and cancel substream publishers when no-one subscribes to them after some time (time them out)" in assertAllStagesStopped {
val publisherProbe = StreamTestKit.PublisherProbe[Int]()
val publisherProbe = TestPublisher.manualProbe[Int]()
val publisher = Source(publisherProbe).groupBy(_ % 3).runWith(Sink.publisher)
val subscriber = StreamTestKit.SubscriberProbe[(Int, Source[Int, _])]()
val subscriber = TestSubscriber.manualProbe[(Int, Source[Int, _])]()
publisher.subscribe(subscriber)
val upstreamSubscription = publisherProbe.expectSubscription()
@ -56,7 +56,7 @@ class SubstreamSubscriptionTimeoutSpec(conf: String) extends AkkaSpec(conf) {
val (_, s1) = subscriber.expectNext()
// should not break normal usage
val s1SubscriberProbe = StreamTestKit.SubscriberProbe[Int]()
val s1SubscriberProbe = TestSubscriber.manualProbe[Int]()
s1.runWith(Sink.publisher).subscribe(s1SubscriberProbe)
val s1Subscription = s1SubscriberProbe.expectSubscription()
s1Subscription.request(100)
@ -64,7 +64,7 @@ class SubstreamSubscriptionTimeoutSpec(conf: String) extends AkkaSpec(conf) {
val (_, s2) = subscriber.expectNext()
// should not break normal usage
val s2SubscriberProbe = StreamTestKit.SubscriberProbe[Int]()
val s2SubscriberProbe = TestSubscriber.manualProbe[Int]()
s2.runWith(Sink.publisher).subscribe(s2SubscriberProbe)
val s2Subscription = s2SubscriberProbe.expectSubscription()
s2Subscription.request(100)
@ -82,9 +82,9 @@ class SubstreamSubscriptionTimeoutSpec(conf: String) extends AkkaSpec(conf) {
}
"timeout and stop groupBy parent actor if none of the substreams are actually consumed" in assertAllStagesStopped {
val publisherProbe = StreamTestKit.PublisherProbe[Int]()
val publisherProbe = TestPublisher.manualProbe[Int]()
val publisher = Source(publisherProbe).groupBy(_ % 2).runWith(Sink.publisher)
val subscriber = StreamTestKit.SubscriberProbe[(Int, Source[Int, _])]()
val subscriber = TestSubscriber.manualProbe[(Int, Source[Int, _])]()
publisher.subscribe(subscriber)
val upstreamSubscription = publisherProbe.expectSubscription()
@ -107,9 +107,9 @@ class SubstreamSubscriptionTimeoutSpec(conf: String) extends AkkaSpec(conf) {
}
"not timeout and cancel substream publishers when they have been subscribed to" in {
val publisherProbe = StreamTestKit.PublisherProbe[Int]()
val publisherProbe = TestPublisher.manualProbe[Int]()
val publisher = Source(publisherProbe).groupBy(_ % 2).runWith(Sink.publisher)
val subscriber = StreamTestKit.SubscriberProbe[(Int, Source[Int, _])]()
val subscriber = TestSubscriber.manualProbe[(Int, Source[Int, _])]()
publisher.subscribe(subscriber)
val upstreamSubscription = publisherProbe.expectSubscription()
@ -122,7 +122,7 @@ class SubstreamSubscriptionTimeoutSpec(conf: String) extends AkkaSpec(conf) {
val (_, s1) = subscriber.expectNext()
// should not break normal usage
val s1SubscriberProbe = StreamTestKit.SubscriberProbe[Int]()
val s1SubscriberProbe = TestSubscriber.manualProbe[Int]()
s1.runWith(Sink.publisher).subscribe(s1SubscriberProbe)
val s1Sub = s1SubscriberProbe.expectSubscription()
s1Sub.request(1)
@ -130,7 +130,7 @@ class SubstreamSubscriptionTimeoutSpec(conf: String) extends AkkaSpec(conf) {
val (_, s2) = subscriber.expectNext()
// should not break normal usage
val s2SubscriberProbe = StreamTestKit.SubscriberProbe[Int]()
val s2SubscriberProbe = TestSubscriber.manualProbe[Int]()
s2.runWith(Sink.publisher).subscribe(s2SubscriberProbe)
val s2Sub = s2SubscriberProbe.expectSubscription()

View file

@ -8,9 +8,8 @@ import akka.actor.Cancellable
import scala.concurrent.duration._
import scala.util.control.NoStackTrace
import akka.stream.ActorFlowMaterializer
import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.StreamTestKit
import akka.stream.testkit.StreamTestKit.assertAllStagesStopped
import akka.stream.testkit._
import akka.stream.testkit.Utils._
import akka.stream.ActorFlowMaterializerSettings
class TickSourceSpec extends AkkaSpec {
@ -19,7 +18,7 @@ class TickSourceSpec extends AkkaSpec {
"A Flow based on tick publisher" must {
"produce ticks" in assertAllStagesStopped {
val c = StreamTestKit.SubscriberProbe[String]()
val c = TestSubscriber.manualProbe[String]()
Source(1.second, 500.millis, "tick").to(Sink(c)).run()
val sub = c.expectSubscription()
sub.request(3)
@ -34,7 +33,7 @@ class TickSourceSpec extends AkkaSpec {
}
"drop ticks when not requested" in {
val c = StreamTestKit.SubscriberProbe[String]()
val c = TestSubscriber.manualProbe[String]()
Source(1.second, 1.second, "tick").to(Sink(c)).run()
val sub = c.expectSubscription()
sub.request(2)
@ -52,8 +51,8 @@ class TickSourceSpec extends AkkaSpec {
"reject multiple subscribers, but keep the first" in {
val p = Source(1.second, 1.second, "tick").runWith(Sink.publisher)
val c1 = StreamTestKit.SubscriberProbe[String]()
val c2 = StreamTestKit.SubscriberProbe[String]()
val c1 = TestSubscriber.manualProbe[String]()
val c2 = TestSubscriber.manualProbe[String]()
p.subscribe(c1)
p.subscribe(c2)
val sub1 = c1.expectSubscription()
@ -67,7 +66,7 @@ class TickSourceSpec extends AkkaSpec {
}
"be usable with zip for a simple form of rate limiting" in {
val c = StreamTestKit.SubscriberProbe[Int]()
val c = TestSubscriber.manualProbe[Int]()
FlowGraph.closed() { implicit b
import FlowGraph.Implicits._
@ -87,7 +86,7 @@ class TickSourceSpec extends AkkaSpec {
}
"be possible to cancel" in assertAllStagesStopped {
val c = StreamTestKit.SubscriberProbe[String]()
val c = TestSubscriber.manualProbe[String]()
val tickSource = Source(1.second, 500.millis, "tick")
val cancellable = tickSource.to(Sink(c)).run()
val sub = c.expectSubscription()