parent
d7b45e0fc3
commit
e6e476d82a
146 changed files with 910 additions and 740 deletions
|
|
@ -99,7 +99,7 @@ class FlowMapBenchmark {
|
|||
}
|
||||
}
|
||||
|
||||
flow = mkMaps(Source(syncTestPublisher), numberOfMapOps) {
|
||||
flow = mkMaps(Source.fromPublisher(syncTestPublisher), numberOfMapOps) {
|
||||
if (UseGraphStageIdentity)
|
||||
GraphStages.identity[Int]
|
||||
else
|
||||
|
|
|
|||
|
|
@ -34,7 +34,7 @@ class FileSourcesBenchmark {
|
|||
val f = File.createTempFile(getClass.getName, ".bench.tmp")
|
||||
f.deleteOnExit()
|
||||
|
||||
val ft = Source(() ⇒ Iterator.continually(line))
|
||||
val ft = Source.fromIterator(() ⇒ Iterator.continually(line))
|
||||
.take(10 * 39062) // adjust as needed
|
||||
.runWith(Sink.file(f))
|
||||
Await.result(ft, 30.seconds)
|
||||
|
|
@ -53,7 +53,7 @@ class FileSourcesBenchmark {
|
|||
def setup() {
|
||||
fileChannelSource = Source.file(file, bufSize)
|
||||
fileInputStreamSource = Source.inputStream(() ⇒ new FileInputStream(file), bufSize)
|
||||
ioSourceLinesIterator = Source(() ⇒ scala.io.Source.fromFile(file).getLines()).map(ByteString(_))
|
||||
ioSourceLinesIterator = Source.fromIterator(() ⇒ scala.io.Source.fromFile(file).getLines()).map(ByteString(_))
|
||||
}
|
||||
|
||||
@TearDown
|
||||
|
|
|
|||
|
|
@ -1,12 +1,16 @@
|
|||
package docs;
|
||||
|
||||
import akka.actor.ActorSystem;
|
||||
import akka.actor.Cancellable;
|
||||
import akka.http.javadsl.model.Uri;
|
||||
import akka.dispatch.Futures;
|
||||
import akka.japi.function.Creator;
|
||||
import akka.japi.Pair;
|
||||
import akka.japi.function.Function;
|
||||
import akka.stream.*;
|
||||
import akka.stream.javadsl.*;
|
||||
import akka.stream.testkit.TestPublisher;
|
||||
import akka.stream.testkit.TestSubscriber;
|
||||
import akka.util.ByteString;
|
||||
import scala.Option;
|
||||
import scala.concurrent.Future;
|
||||
|
|
@ -14,6 +18,9 @@ import scala.concurrent.duration.FiniteDuration;
|
|||
import scala.concurrent.Promise;
|
||||
import scala.runtime.BoxedUnit;
|
||||
|
||||
import org.reactivestreams.Publisher;
|
||||
import org.reactivestreams.Subscriber;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
|
|
@ -25,6 +32,7 @@ public class MigrationsJava {
|
|||
|
||||
// This is compile-only code, no need for actually running anything.
|
||||
public static ActorMaterializer mat = null;
|
||||
public static ActorSystem sys = null;
|
||||
|
||||
public static class SomeInputStream extends InputStream {
|
||||
public SomeInputStream() {}
|
||||
|
|
@ -133,12 +141,34 @@ public class MigrationsJava {
|
|||
// Complete the promise with an empty option to emulate the old lazyEmpty
|
||||
promise.trySuccess(scala.Option.empty());
|
||||
|
||||
final Source<String, Cancellable> sourceUnderTest = Source.tick(
|
||||
final Source<String, Cancellable> ticks = Source.tick(
|
||||
FiniteDuration.create(0, TimeUnit.MILLISECONDS),
|
||||
FiniteDuration.create(200, TimeUnit.MILLISECONDS),
|
||||
"tick");
|
||||
|
||||
final Source<Integer, BoxedUnit> pubSource =
|
||||
Source.fromPublisher(TestPublisher.<Integer>manualProbe(true, sys));
|
||||
|
||||
final Source<Integer, BoxedUnit> futSource =
|
||||
Source.fromFuture(Futures.successful(42));
|
||||
|
||||
final Source<Integer, Subscriber<Integer>> subSource =
|
||||
Source.<Integer>asSubscriber();
|
||||
//#source-creators
|
||||
|
||||
//#sink-creators
|
||||
final Sink<Integer, BoxedUnit> subSink =
|
||||
Sink.fromSubscriber(TestSubscriber.<Integer>manualProbe(sys));
|
||||
//#sink-creators
|
||||
|
||||
//#sink-as-publisher
|
||||
final Sink<Integer, Publisher<Integer>> pubSink =
|
||||
Sink.<Integer>asPublisher(false);
|
||||
|
||||
final Sink<Integer, Publisher<Integer>> pubSinkFanout =
|
||||
Sink.<Integer>asPublisher(true);
|
||||
//#sink-as-publisher
|
||||
|
||||
//#empty-flow
|
||||
Flow<Integer, Integer, BoxedUnit> emptyFlow = Flow.<Integer>create();
|
||||
// or
|
||||
|
|
|
|||
|
|
@ -224,13 +224,14 @@ should be replaced by
|
|||
Source constructor name changes
|
||||
===============================
|
||||
|
||||
``Source.lazyEmpty`` have been replaced by ``Source.maybe`` which returns a ``Promise`` that can be completed by one or
|
||||
``Source.lazyEmpty`` has been replaced by ``Source.maybe`` which returns a ``Promise`` that can be completed by one or
|
||||
zero elements by providing an ``Option``. This is different from ``lazyEmpty`` which only allowed completion to be
|
||||
sent, but no elements.
|
||||
|
||||
The ``from()`` overload on ``Source`` that provide a tick source (``Source.from(delay,interval,tick)``)
|
||||
is replaced by the named method ``Source.tick()`` to reduce the number of overloads and to make the function more
|
||||
discoverable.
|
||||
The ``from()`` overload on ``Source`` has been refactored to separate methods to reduce the number of overloads and
|
||||
make source creation more discoverable.
|
||||
|
||||
``Source.subscriber`` has been renamed to ``Source.asSubscriber``.
|
||||
|
||||
Update procedure
|
||||
----------------
|
||||
|
|
@ -238,6 +239,9 @@ Update procedure
|
|||
1. All uses of ``Source.lazyEmpty`` should be replaced by ``Source.maybe`` and the returned ``Promise`` completed with
|
||||
a ``None`` (an empty ``Option``)
|
||||
2. Replace all uses of ``Source.from(delay,interval,tick)`` with the method ``Source.tick(delay,interval,tick)``
|
||||
3. Replace all uses of ``Source.from(publisher)`` with the method ``Source.fromPublisher(publisher)``
|
||||
4. Replace all uses of ``Source.from(future)`` with the method ``Source.fromFuture(future))``
|
||||
5. Replace all uses of ``Source.subscriber`` with the method ``Source.asSubscriber``
|
||||
|
||||
Example
|
||||
^^^^^^^
|
||||
|
|
@ -250,15 +254,51 @@ Example
|
|||
promise.trySuccess(BoxedUnit.UNIT);
|
||||
|
||||
// This no longer works!
|
||||
final Source<String, Cancellable> sourceUnderTest = Source.from(
|
||||
final Source<String, Cancellable> ticks = Source.from(
|
||||
FiniteDuration.create(0, TimeUnit.MILLISECONDS),
|
||||
FiniteDuration.create(200, TimeUnit.MILLISECONDS),
|
||||
"tick");
|
||||
|
||||
// This no longer works!
|
||||
final Source<Integer, BoxedUnit> pubSource =
|
||||
Source.from(TestPublisher.<Integer>manualProbe(true, sys));
|
||||
|
||||
// This no longer works!
|
||||
final Source<Integer, BoxedUnit> futSource =
|
||||
Source.from(Futures.successful(42));
|
||||
|
||||
// This no longer works!
|
||||
final Source<Integer, Subscriber<Integer>> subSource =
|
||||
Source.<Integer>subscriber();
|
||||
|
||||
should be replaced by
|
||||
|
||||
.. includecode:: code/docs/MigrationsJava.java#source-creators
|
||||
|
||||
Sink constructor name changes
|
||||
=============================
|
||||
|
||||
``Sink.create(subscriber)`` has been renamed to ``Sink.fromSubscriber(subscriber)`` to reduce the number of overloads and
|
||||
make sink creation more discoverable.
|
||||
|
||||
Update procedure
|
||||
----------------
|
||||
|
||||
1. Replace all uses of ``Sink.create(subscriber)`` with the method ``Sink.fromSubscriber(subscriber)``
|
||||
|
||||
Example
|
||||
^^^^^^^
|
||||
|
||||
::
|
||||
|
||||
// This no longer works!
|
||||
final Sink<Integer, BoxedUnit> subSink =
|
||||
Sink.create(TestSubscriber.<Integer>manualProbe(sys));
|
||||
|
||||
should be replaced by
|
||||
|
||||
.. includecode:: code/docs/MigrationsJava.java#sink-creators
|
||||
|
||||
``Flow.empty()`` have been removed
|
||||
==================================
|
||||
|
||||
|
|
@ -308,16 +348,30 @@ should be replaced by
|
|||
It was a common user mistake to use ``Sink.publisher`` and get into trouble since it would only support
|
||||
a single ``Subscriber``, and the discoverability of the apprpriate fix was non-obvious (Sink.fanoutPublisher).
|
||||
To make the decision whether to support fanout or not an active one, the aforementioned methods have been
|
||||
replaced with a single method: ``Sink.publisher(fanout: Boolean)``.
|
||||
replaced with a single method: ``Sink.asPublisher(fanout: Boolean)``.
|
||||
|
||||
Update procedure
|
||||
----------------
|
||||
|
||||
1. Replace all occurences of ``Sink.publisher`` with ``Sink.publisher(false)``
|
||||
2. Replace all occurences of ``Sink.fanoutPublisher`` with ``Sink.publisher(true)``
|
||||
1. Replace all occurences of ``Sink.publisher`` with ``Sink.asPublisher(false)``
|
||||
2. Replace all occurences of ``Sink.fanoutPublisher`` with ``Sink.asPublisher(true)``
|
||||
|
||||
TODO: code example
|
||||
Example
|
||||
^^^^^^^
|
||||
|
||||
::
|
||||
|
||||
// This no longer works!
|
||||
final Sink<Integer, Publisher<Integer>> pubSink =
|
||||
Sink.<Integer>publisher();
|
||||
|
||||
// This no longer works!
|
||||
final Sink<Integer, Publisher<Integer>> pubSink =
|
||||
Sink.<Integer>fanoutPublisher(2, 8);
|
||||
|
||||
should be replaced by
|
||||
|
||||
.. includecode:: code/docs/MigrationsJava.java#sink-as-publisher
|
||||
|
||||
FlexiMerge an FlexiRoute has been replaced by GraphStage
|
||||
========================================================
|
||||
|
|
|
|||
|
|
@ -8,7 +8,7 @@ Integrating with Actors
|
|||
=======================
|
||||
|
||||
For piping the elements of a stream as messages to an ordinary actor you can use the
|
||||
``Sink.actorRef``. Messages can be sent to a stream via the :class:`ActorRef` that is
|
||||
``Sink.actorRef``. Messages can be sent to a stream via the :class:`ActorRef` that is
|
||||
materialized by ``Source.actorRef``.
|
||||
|
||||
For more advanced use cases the :class:`ActorPublisher` and :class:`ActorSubscriber` traits are
|
||||
|
|
@ -32,8 +32,8 @@ Akka Streams :class:`Source` or :class:`Sink`.
|
|||
Source.actorRef
|
||||
^^^^^^^^^^^^^^^
|
||||
|
||||
Messages sent to the actor that is materialized by ``Source.actorRef`` will be emitted to the
|
||||
stream if there is demand from downstream, otherwise they will be buffered until request for
|
||||
Messages sent to the actor that is materialized by ``Source.actorRef`` will be emitted to the
|
||||
stream if there is demand from downstream, otherwise they will be buffered until request for
|
||||
demand is received.
|
||||
|
||||
Depending on the defined :class:`OverflowStrategy` it might drop elements if there is no space
|
||||
|
|
@ -44,7 +44,7 @@ actor interface.
|
|||
The stream can be completed successfully by sending ``akka.actor.PoisonPill`` or
|
||||
``akka.actor.Status.Success`` to the actor reference.
|
||||
|
||||
The stream can be completed with failure by sending ``akka.actor.Status.Failure`` to the
|
||||
The stream can be completed with failure by sending ``akka.actor.Status.Failure`` to the
|
||||
actor reference.
|
||||
|
||||
The actor will be stopped when the stream is completed, failed or cancelled from downstream,
|
||||
|
|
@ -108,12 +108,12 @@ This is how it can be used as input :class:`Source` to a :class:`Flow`:
|
|||
.. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/ActorPublisherDocTest.java#actor-publisher-usage
|
||||
|
||||
You can only attach one subscriber to this publisher. Use a ``Broadcast``-element or
|
||||
attach a ``Sink.publisher(true)`` to enable multiple subscribers.
|
||||
attach a ``Sink.asPublisher(true)`` to enable multiple subscribers.
|
||||
|
||||
ActorSubscriber
|
||||
^^^^^^^^^^^^^^^
|
||||
|
||||
Extend :class:`akka.stream.actor.AbstractActorSubscriber` to make your class a stream subscriber with
|
||||
Extend :class:`akka.stream.actor.AbstractActorSubscriber` to make your class a stream subscriber with
|
||||
full control of stream back pressure. It will receive
|
||||
``ActorSubscriberMessage.OnNext``, ``ActorSubscriberMessage.OnComplete`` and ``ActorSubscriberMessage.OnError``
|
||||
messages from the stream. It can also receive other, non-stream messages, in the same way as any actor.
|
||||
|
|
@ -414,7 +414,7 @@ by using the Publisher-:class:`Sink`:
|
|||
|
||||
.. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/ReactiveStreamsDocTest.java#source-publisher
|
||||
|
||||
A publisher that is created with ``Sink.publisher(false)`` supports only a single subscription.
|
||||
A publisher that is created with ``Sink.asPublisher(false)`` supports only a single subscription.
|
||||
Additional subscription attempts will be rejected with an :class:`IllegalStateException`.
|
||||
|
||||
A publisher that supports multiple subscribers using fan-out/broadcasting is created as follows:
|
||||
|
|
|
|||
|
|
@ -6,11 +6,11 @@ import akka.http.scaladsl.model.Uri
|
|||
import akka.stream.scaladsl._
|
||||
import akka.stream._
|
||||
import akka.stream.stage.{ OutHandler, InHandler, GraphStageLogic, GraphStage }
|
||||
import akka.stream.testkit.AkkaSpec
|
||||
import akka.stream.testkit.{ AkkaSpec, TestPublisher, TestSubscriber }
|
||||
|
||||
import scala.concurrent.{ Future, ExecutionContext, Promise }
|
||||
import scala.concurrent.duration._
|
||||
import scala.util.{ Failure, Success, Try }
|
||||
import scala.util.{ Failure, Random, Success, Try }
|
||||
|
||||
class MigrationsScala extends AkkaSpec {
|
||||
|
||||
|
|
@ -110,8 +110,26 @@ class MigrationsScala extends AkkaSpec {
|
|||
promise.trySuccess(Some(()))
|
||||
|
||||
val ticks = Source.tick(1.second, 3.seconds, "tick")
|
||||
|
||||
val pubSource = Source.fromPublisher(TestPublisher.manualProbe[Int]())
|
||||
|
||||
val itSource = Source.fromIterator(() => Iterator.continually(Random.nextGaussian))
|
||||
|
||||
val futSource = Source.fromFuture(Future.successful(42))
|
||||
|
||||
val subSource = Source.asSubscriber
|
||||
//#source-creators
|
||||
|
||||
//#sink-creators
|
||||
val subSink = Sink.fromSubscriber(TestSubscriber.manualProbe[Int]())
|
||||
//#sink-creators
|
||||
|
||||
//#sink-as-publisher
|
||||
val pubSink = Sink.asPublisher(fanout = false)
|
||||
|
||||
val pubSinkFanout = Sink.asPublisher(fanout = true)
|
||||
//#sink-as-publisher
|
||||
|
||||
//#flatMapConcat
|
||||
Flow[Source[Int, Any]].flatMapConcat(identity)
|
||||
//#flatMapConcat
|
||||
|
|
|
|||
|
|
@ -100,7 +100,7 @@ class FlowDocSpec extends AkkaSpec {
|
|||
Source(List(1, 2, 3))
|
||||
|
||||
// Create a source from a Future
|
||||
Source(Future.successful("Hello Streams!"))
|
||||
Source.fromFuture(Future.successful("Hello Streams!"))
|
||||
|
||||
// Create a source from a single element
|
||||
Source.single("only one element")
|
||||
|
|
|
|||
|
|
@ -9,7 +9,7 @@ class GraphCyclesSpec extends AkkaSpec {
|
|||
implicit val materializer = ActorMaterializer()
|
||||
|
||||
"Cycle demonstration" must {
|
||||
val source = Source(() => Iterator.from(0))
|
||||
val source = Source.fromIterator(() => Iterator.from(0))
|
||||
|
||||
"include a deadlocked cycle" in {
|
||||
|
||||
|
|
|
|||
|
|
@ -33,7 +33,7 @@ class RateTransformationDocSpec extends AkkaSpec {
|
|||
}
|
||||
//#conflate-summarize
|
||||
|
||||
val fut = Source(() => Iterator.continually(Random.nextGaussian))
|
||||
val fut = Source.fromIterator(() => Iterator.continually(Random.nextGaussian))
|
||||
.via(statsFlow)
|
||||
.grouped(10)
|
||||
.runWith(Sink.head)
|
||||
|
|
|
|||
|
|
@ -41,7 +41,7 @@ class ReactiveStreamsDocSpec extends AkkaSpec {
|
|||
|
||||
val impl = new Fixture {
|
||||
override def tweets: Publisher[Tweet] =
|
||||
TwitterStreamQuickstartDocSpec.tweets.runWith(Sink.publisher(false))
|
||||
TwitterStreamQuickstartDocSpec.tweets.runWith(Sink.asPublisher(false))
|
||||
|
||||
override def storage = TestSubscriber.manualProbe[Author]
|
||||
|
||||
|
|
@ -66,7 +66,7 @@ class ReactiveStreamsDocSpec extends AkkaSpec {
|
|||
val storage = impl.storage
|
||||
|
||||
//#connect-all
|
||||
Source(tweets).via(authors).to(Sink(storage)).run()
|
||||
Source.fromPublisher(tweets).via(authors).to(Sink.fromSubscriber(storage)).run()
|
||||
//#connect-all
|
||||
|
||||
assertResult(storage)
|
||||
|
|
@ -92,7 +92,7 @@ class ReactiveStreamsDocSpec extends AkkaSpec {
|
|||
|
||||
//#source-publisher
|
||||
val authorPublisher: Publisher[Author] =
|
||||
Source(tweets).via(authors).runWith(Sink.publisher(fanout = false))
|
||||
Source.fromPublisher(tweets).via(authors).runWith(Sink.asPublisher(fanout = false))
|
||||
|
||||
authorPublisher.subscribe(storage)
|
||||
//#source-publisher
|
||||
|
|
@ -107,8 +107,8 @@ class ReactiveStreamsDocSpec extends AkkaSpec {
|
|||
|
||||
//#source-fanoutPublisher
|
||||
val authorPublisher: Publisher[Author] =
|
||||
Source(tweets).via(authors)
|
||||
.runWith(Sink.publisher(fanout = true))
|
||||
Source.fromPublisher(tweets).via(authors)
|
||||
.runWith(Sink.asPublisher(fanout = true))
|
||||
|
||||
authorPublisher.subscribe(storage)
|
||||
authorPublisher.subscribe(alert)
|
||||
|
|
@ -125,7 +125,7 @@ class ReactiveStreamsDocSpec extends AkkaSpec {
|
|||
|
||||
//#sink-subscriber
|
||||
val tweetSubscriber: Subscriber[Tweet] =
|
||||
authors.to(Sink(storage)).runWith(Source.subscriber[Tweet])
|
||||
authors.to(Sink.fromSubscriber(storage)).runWith(Source.asSubscriber[Tweet])
|
||||
|
||||
tweets.subscribe(tweetSubscriber)
|
||||
//#sink-subscriber
|
||||
|
|
|
|||
|
|
@ -57,7 +57,7 @@ class StreamPartialFlowGraphDocSpec extends AkkaSpec {
|
|||
|
||||
// prepare graph elements
|
||||
val zip = b.add(Zip[Int, Int]())
|
||||
def ints = Source(() => Iterator.from(1))
|
||||
def ints = Source.fromIterator(() => Iterator.from(1))
|
||||
|
||||
// connect the graph
|
||||
ints.filter(_ % 2 != 0) ~> zip.in0
|
||||
|
|
|
|||
|
|
@ -19,8 +19,8 @@ class RecipeCollectingMetrics extends RecipeSpec {
|
|||
//
|
||||
// val loadPub = TestPublisher.manualProbe[Int]()
|
||||
// val tickPub = TestPublisher.manualProbe[Tick]()
|
||||
// val reportTicks = Source(tickPub)
|
||||
// val loadUpdates = Source(loadPub)
|
||||
// val reportTicks = Source.fromPublisher(tickPub)
|
||||
// val loadUpdates = Source.fromPublisher(loadPub)
|
||||
// val futureSink = Sink.head[immutable.Seq[String]]
|
||||
// val sink = Flow[String].grouped(10).to(futureSink)
|
||||
//
|
||||
|
|
|
|||
|
|
@ -13,15 +13,15 @@ class RecipeDroppyBroadcast extends RecipeSpec {
|
|||
"Recipe for a droppy broadcast" must {
|
||||
"work" in {
|
||||
val pub = TestPublisher.probe[Int]()
|
||||
val myElements = Source(pub)
|
||||
val myElements = Source.fromPublisher(pub)
|
||||
|
||||
val sub1 = TestSubscriber.manualProbe[Int]()
|
||||
val sub2 = TestSubscriber.manualProbe[Int]()
|
||||
val sub3 = TestSubscriber.probe[Int]()
|
||||
val futureSink = Sink.head[Seq[Int]]
|
||||
val mySink1 = Sink(sub1)
|
||||
val mySink2 = Sink(sub2)
|
||||
val mySink3 = Sink(sub3)
|
||||
val mySink1 = Sink.fromSubscriber(sub1)
|
||||
val mySink2 = Sink.fromSubscriber(sub2)
|
||||
val mySink3 = Sink.fromSubscriber(sub3)
|
||||
|
||||
//#droppy-bcast
|
||||
val graph = RunnableGraph.fromGraph(GraphDSL.create(mySink1, mySink2, mySink3)((_, _, _)) { implicit b =>
|
||||
|
|
|
|||
|
|
@ -94,15 +94,15 @@ class RecipeGlobalRateLimit extends RecipeSpec {
|
|||
// Use a large period and emulate the timer by hand instead
|
||||
val limiter = system.actorOf(Limiter.props(2, 100.days, 1), "limiter")
|
||||
|
||||
val source1 = Source(() => Iterator.continually("E1")).via(limitGlobal(limiter, 2.seconds))
|
||||
val source2 = Source(() => Iterator.continually("E2")).via(limitGlobal(limiter, 2.seconds))
|
||||
val source1 = Source.fromIterator(() => Iterator.continually("E1")).via(limitGlobal(limiter, 2.seconds))
|
||||
val source2 = Source.fromIterator(() => Iterator.continually("E2")).via(limitGlobal(limiter, 2.seconds))
|
||||
|
||||
val probe = TestSubscriber.manualProbe[String]()
|
||||
|
||||
RunnableGraph.fromGraph(GraphDSL.create() { implicit b =>
|
||||
import GraphDSL.Implicits._
|
||||
val merge = b.add(Merge[String](2))
|
||||
source1 ~> merge ~> Sink(probe)
|
||||
source1 ~> merge ~> Sink.fromSubscriber(probe)
|
||||
source2 ~> merge
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
|
|
|||
|
|
@ -54,8 +54,8 @@ class RecipeHold extends RecipeSpec {
|
|||
|
||||
val pub = TestPublisher.probe[Int]()
|
||||
val sub = TestSubscriber.manualProbe[Int]()
|
||||
val source = Source(pub)
|
||||
val sink = Sink(sub)
|
||||
val source = Source.fromPublisher(pub)
|
||||
val sink = Sink.fromSubscriber(sub)
|
||||
|
||||
source.transform(() => new HoldWithInitial(0)).to(sink).run()
|
||||
|
||||
|
|
@ -84,8 +84,8 @@ class RecipeHold extends RecipeSpec {
|
|||
|
||||
val pub = TestPublisher.probe[Int]()
|
||||
val sub = TestSubscriber.manualProbe[Int]()
|
||||
val source = Source(pub)
|
||||
val sink = Sink(sub)
|
||||
val source = Source.fromPublisher(pub)
|
||||
val sink = Sink.fromSubscriber(sub)
|
||||
|
||||
source.transform(() => new HoldWithWait).to(sink).run()
|
||||
|
||||
|
|
|
|||
|
|
@ -14,8 +14,8 @@ class RecipeManualTrigger extends RecipeSpec {
|
|||
val elements = Source(List("1", "2", "3", "4"))
|
||||
val pub = TestPublisher.probe[Trigger]()
|
||||
val sub = TestSubscriber.manualProbe[Message]()
|
||||
val triggerSource = Source(pub)
|
||||
val sink = Sink(sub)
|
||||
val triggerSource = Source.fromPublisher(pub)
|
||||
val sink = Sink.fromSubscriber(sub)
|
||||
|
||||
//#manually-triggered-stream
|
||||
val graph = RunnableGraph.fromGraph(GraphDSL.create() { implicit builder =>
|
||||
|
|
@ -53,8 +53,8 @@ class RecipeManualTrigger extends RecipeSpec {
|
|||
val elements = Source(List("1", "2", "3", "4"))
|
||||
val pub = TestPublisher.probe[Trigger]()
|
||||
val sub = TestSubscriber.manualProbe[Message]()
|
||||
val triggerSource = Source(pub)
|
||||
val sink = Sink(sub)
|
||||
val triggerSource = Source.fromPublisher(pub)
|
||||
val sink = Sink.fromSubscriber(sub)
|
||||
|
||||
//#manually-triggered-stream-zipwith
|
||||
val graph = RunnableGraph.fromGraph(GraphDSL.create() { implicit builder =>
|
||||
|
|
|
|||
|
|
@ -15,8 +15,8 @@ class RecipeMissedTicks extends RecipeSpec {
|
|||
|
||||
val pub = TestPublisher.probe[Tick]()
|
||||
val sub = TestSubscriber.manualProbe[Int]()
|
||||
val tickStream = Source(pub)
|
||||
val sink = Sink(sub)
|
||||
val tickStream = Source.fromPublisher(pub)
|
||||
val sink = Sink.fromSubscriber(sub)
|
||||
|
||||
//#missed-ticks
|
||||
val missedTicks: Flow[Tick, Int, Unit] =
|
||||
|
|
|
|||
|
|
@ -22,8 +22,8 @@ class RecipeSimpleDrop extends RecipeSpec {
|
|||
|
||||
val pub = TestPublisher.probe[Message]()
|
||||
val sub = TestSubscriber.manualProbe[Message]()
|
||||
val messageSource = Source(pub)
|
||||
val sink = Sink(sub)
|
||||
val messageSource = Source.fromPublisher(pub)
|
||||
val sink = Sink.fromSubscriber(sub)
|
||||
|
||||
messageSource.via(realDroppyStream).to(sink).run()
|
||||
|
||||
|
|
|
|||
|
|
@ -220,13 +220,14 @@ should be replaced by
|
|||
Source constructor name changes
|
||||
===============================
|
||||
|
||||
``Source.lazyEmpty`` have been replaced by ``Source.maybe`` which returns a ``Promise`` that can be completed by one or
|
||||
``Source.lazyEmpty`` has been replaced by ``Source.maybe`` which returns a ``Promise`` that can be completed by one or
|
||||
zero elements by providing an ``Option``. This is different from ``lazyEmpty`` which only allowed completion to be
|
||||
sent, but no elements.
|
||||
|
||||
The ``apply()`` overload on ``Source`` that provide a tick source (``Source(delay,interval,tick)``)
|
||||
is replaced by the named method ``Source.tick()`` to reduce the number of overloads and to make the function more
|
||||
discoverable.
|
||||
The ``apply()`` overload on ``Source`` has been refactored to separate methods to reduce the number of overloads and
|
||||
make source creation more discoverable.
|
||||
|
||||
``Source.subscriber`` has been renamed to ``Source.asSubscriber``.
|
||||
|
||||
Update procedure
|
||||
----------------
|
||||
|
|
@ -234,6 +235,10 @@ Update procedure
|
|||
1. All uses of ``Source.lazyEmpty`` should be replaced by ``Source.maybe`` and the returned ``Promise`` completed with
|
||||
a ``None`` (an empty ``Option``)
|
||||
2. Replace all uses of ``Source(delay,interval,tick)`` with the method ``Source.tick(delay,interval,tick)``
|
||||
3. Replace all uses of ``Source(publisher)`` with the method ``Source.fromPublisher(publisher)``
|
||||
4. Replace all uses of ``Source(() => iterator)`` with the method ``Source.fromIterator(() => iterator))``
|
||||
5. Replace all uses of ``Source(future)`` with the method ``Source.fromFuture(future))``
|
||||
6. Replace all uses of ``Source.subscriber`` with the method ``Source.asSubscriber``
|
||||
|
||||
Example
|
||||
^^^^^^^
|
||||
|
|
@ -248,10 +253,45 @@ Example
|
|||
// This no longer works!
|
||||
val ticks = Source(1.second, 3.seconds, "tick")
|
||||
|
||||
// This no longer works!
|
||||
val pubSource = Source(TestPublisher.manualProbe[Int]())
|
||||
|
||||
// This no longer works!
|
||||
val itSource = Source(() => Iterator.continually(Random.nextGaussian))
|
||||
|
||||
// This no longer works!
|
||||
val futSource = Source(Future.successful(42))
|
||||
|
||||
// This no longer works!
|
||||
val subSource = Source.subscriber
|
||||
|
||||
should be replaced by
|
||||
|
||||
.. includecode:: code/docs/MigrationsScala.scala#source-creators
|
||||
|
||||
Sink constructor name changes
|
||||
=============================
|
||||
|
||||
``Sink.apply(subscriber)`` has been renamed to ``Sink.fromSubscriber(subscriber)`` to reduce the number of overloads and
|
||||
make sink creation more discoverable.
|
||||
|
||||
Update procedure
|
||||
----------------
|
||||
|
||||
1. Replace all uses of ``Sink(subscriber)`` with the method ``Sink.fromSubscriber(subscriber)``
|
||||
|
||||
Example
|
||||
^^^^^^^
|
||||
|
||||
::
|
||||
|
||||
// This no longer works!
|
||||
val subSink = Sink(TestSubscriber.manualProbe[Int]())
|
||||
|
||||
should be replaced by
|
||||
|
||||
.. includecode:: code/docs/MigrationsScala.scala#sink-creators
|
||||
|
||||
``flatten(FlattenStrategy)`` has been replaced by named counterparts
|
||||
====================================================================
|
||||
|
||||
|
|
@ -276,6 +316,35 @@ should be replaced by
|
|||
|
||||
.. includecode:: code/docs/MigrationsScala.scala#flatMapConcat
|
||||
|
||||
`Sink.fanoutPublisher() and Sink.publisher() is now a single method`
|
||||
====================================================================
|
||||
|
||||
It was a common user mistake to use ``Sink.publisher`` and get into trouble since it would only support
|
||||
a single ``Subscriber``, and the discoverability of the apprpriate fix was non-obvious (Sink.fanoutPublisher).
|
||||
To make the decision whether to support fanout or not an active one, the aforementioned methods have been
|
||||
replaced with a single method: ``Sink.asPublisher(fanout: Boolean)``.
|
||||
|
||||
Update procedure
|
||||
----------------
|
||||
|
||||
1. Replace all occurences of ``Sink.publisher`` with ``Sink.asPublisher(false)``
|
||||
2. Replace all occurences of ``Sink.fanoutPublisher`` with ``Sink.asPublisher(true)``
|
||||
|
||||
Example
|
||||
^^^^^^^
|
||||
|
||||
::
|
||||
|
||||
// This no longer works!
|
||||
val subSink = Sink.publisher
|
||||
|
||||
// This no longer works!
|
||||
val subSink = Sink.fanoutPublisher(2, 8)
|
||||
|
||||
should be replaced by
|
||||
|
||||
.. includecode:: code/docs/MigrationsScala.scala#sink-as-publisher
|
||||
|
||||
FlexiMerge an FlexiRoute has been replaced by GraphStage
|
||||
========================================================
|
||||
|
||||
|
|
|
|||
|
|
@ -8,7 +8,7 @@ Integrating with Actors
|
|||
=======================
|
||||
|
||||
For piping the elements of a stream as messages to an ordinary actor you can use the
|
||||
``Sink.actorRef``. Messages can be sent to a stream via the :class:`ActorRef` that is
|
||||
``Sink.actorRef``. Messages can be sent to a stream via the :class:`ActorRef` that is
|
||||
materialized by ``Source.actorRef``.
|
||||
|
||||
For more advanced use cases the :class:`ActorPublisher` and :class:`ActorSubscriber` traits are
|
||||
|
|
@ -27,8 +27,8 @@ Akka Streams :class:`Source` or :class:`Sink`.
|
|||
Source.actorRef
|
||||
^^^^^^^^^^^^^^^
|
||||
|
||||
Messages sent to the actor that is materialized by ``Source.actorRef`` will be emitted to the
|
||||
stream if there is demand from downstream, otherwise they will be buffered until request for
|
||||
Messages sent to the actor that is materialized by ``Source.actorRef`` will be emitted to the
|
||||
stream if there is demand from downstream, otherwise they will be buffered until request for
|
||||
demand is received.
|
||||
|
||||
Depending on the defined :class:`OverflowStrategy` it might drop elements if there is no space
|
||||
|
|
@ -39,7 +39,7 @@ actor interface.
|
|||
The stream can be completed successfully by sending ``akka.actor.PoisonPill`` or
|
||||
``akka.actor.Status.Success`` to the actor reference.
|
||||
|
||||
The stream can be completed with failure by sending ``akka.actor.Status.Failure`` to the
|
||||
The stream can be completed with failure by sending ``akka.actor.Status.Failure`` to the
|
||||
actor reference.
|
||||
|
||||
The actor will be stopped when the stream is completed, failed or cancelled from downstream,
|
||||
|
|
@ -102,7 +102,7 @@ This is how it can be used as input :class:`Source` to a :class:`Flow`:
|
|||
|
||||
.. includecode:: code/docs/stream/ActorPublisherDocSpec.scala#actor-publisher-usage
|
||||
|
||||
A publisher that is created with ``Sink.publisher`` supports a specified number of subscribers. Additional
|
||||
A publisher that is created with ``Sink.asPublisher`` supports a specified number of subscribers. Additional
|
||||
subscription attempts will be rejected with an :class:`IllegalStateException`.
|
||||
|
||||
ActorSubscriber
|
||||
|
|
@ -409,7 +409,7 @@ by using the Publisher-:class:`Sink`:
|
|||
|
||||
.. includecode:: code/docs/stream/ReactiveStreamsDocSpec.scala#source-publisher
|
||||
|
||||
A publisher that is created with ``Sink.publisher(false)`` supports only a single subscription.
|
||||
A publisher that is created with ``Sink.asPublisher(false)`` supports only a single subscription.
|
||||
Additional subscription attempts will be rejected with an :class:`IllegalStateException`.
|
||||
|
||||
A publisher that supports multiple subscribers using fan-out/broadcasting is created as follows:
|
||||
|
|
@ -433,4 +433,3 @@ passing a factory function that will create the :class:`Processor` instances:
|
|||
.. includecode:: code/docs/stream/ReactiveStreamsDocSpec.scala#use-processor
|
||||
|
||||
Please note that a factory is necessary to achieve reusability of the resulting :class:`Flow`.
|
||||
|
||||
|
|
|
|||
|
|
@ -32,11 +32,11 @@ The process of materialization will often create specific objects that are usefu
|
|||
Interoperation with other Reactive Streams implementations
|
||||
----------------------------------------------------------
|
||||
|
||||
Akka Streams fully implement the Reactive Streams specification and interoperate with all other conformant implementations. We chose to completely separate the Reactive Streams interfaces from the user-level API because we regard them to be an SPI that is not targeted at endusers. In order to obtain a :class:`Publisher` or :class:`Subscriber` from an Akka Stream topology, a corresponding ``Sink.publisher`` or ``Source.subscriber`` element must be used.
|
||||
Akka Streams fully implement the Reactive Streams specification and interoperate with all other conformant implementations. We chose to completely separate the Reactive Streams interfaces from the user-level API because we regard them to be an SPI that is not targeted at endusers. In order to obtain a :class:`Publisher` or :class:`Subscriber` from an Akka Stream topology, a corresponding ``Sink.asPublisher`` or ``Source.asSubscriber`` element must be used.
|
||||
|
||||
All stream Processors produced by the default materialization of Akka Streams are restricted to having a single Subscriber, additional Subscribers will be rejected. The reason for this is that the stream topologies described using our DSL never require fan-out behavior from the Publisher sides of the elements, all fan-out is done using explicit elements like :class:`Broadcast[T]`.
|
||||
|
||||
This means that ``Sink.publisher(true)`` (for enabling fan-out support) must be used where broadcast behavior is needed for interoperation with other Reactive Streams implementations.
|
||||
This means that ``Sink.asPublisher(true)`` (for enabling fan-out support) must be used where broadcast behavior is needed for interoperation with other Reactive Streams implementations.
|
||||
|
||||
What shall users of streaming libraries expect?
|
||||
-----------------------------------------------
|
||||
|
|
@ -53,7 +53,7 @@ The second rule allows a library to additionally provide nice sugar for the comm
|
|||
.. note::
|
||||
|
||||
One important consequence of this is that a reusable flow description cannot be bound to “live” resources, any connection to or allocation of such resources must be deferred until materialization time. Examples of “live” resources are already existing TCP connections, a multicast Publisher, etc.; a TickSource does not fall into this category if its timer is created only upon materialization (as is the case for our implementation).
|
||||
|
||||
|
||||
Exceptions from this need to be well-justified and carefully documented.
|
||||
|
||||
Resulting Implementation Constraints
|
||||
|
|
|
|||
|
|
@ -73,7 +73,7 @@ private class PoolInterfaceActor(hcps: HostConnectionPoolSetup,
|
|||
new InetSocketAddress(host, port), settings, setup.log)
|
||||
.named("PoolFlow")
|
||||
|
||||
Source(ActorPublisher(self)).via(poolFlow).runWith(Sink(ActorSubscriber[ResponseContext](self)))
|
||||
Source.fromPublisher(ActorPublisher(self)).via(poolFlow).runWith(Sink.fromSubscriber(ActorSubscriber[ResponseContext](self)))
|
||||
}
|
||||
|
||||
activateIdleTimeoutIfNecessary()
|
||||
|
|
|
|||
|
|
@ -372,9 +372,9 @@ private[http] object HttpServerBluePrint {
|
|||
def websocketFlow: Flow[ByteString, ByteString, Any] = flow
|
||||
|
||||
def installHandler(handlerFlow: Flow[FrameEvent, FrameEvent, Any])(implicit mat: Materializer): Unit =
|
||||
Source(sinkCell.value)
|
||||
Source.fromPublisher(sinkCell.value)
|
||||
.via(handlerFlow)
|
||||
.to(Sink(sourceCell.value))
|
||||
.to(Sink.fromSubscriber(sourceCell.value))
|
||||
.run()
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -159,14 +159,14 @@ private[http] object StreamUtils {
|
|||
case Nil ⇒ Nil
|
||||
case Seq(one) ⇒ Vector(input.via(one))
|
||||
case multiple ⇒
|
||||
val (fanoutSub, fanoutPub) = Source.subscriber[ByteString].toMat(Sink.publisher(true))(Keep.both).run()
|
||||
val (fanoutSub, fanoutPub) = Source.asSubscriber[ByteString].toMat(Sink.asPublisher(true))(Keep.both).run()
|
||||
val sources = transformers.map { flow ⇒
|
||||
// Doubly wrap to ensure that subscription to the running publisher happens before the final sources
|
||||
// are exposed, so there is no race
|
||||
Source(Source(fanoutPub).viaMat(flow)(Keep.right).runWith(Sink.publisher(false)))
|
||||
Source.fromPublisher(Source.fromPublisher(fanoutPub).viaMat(flow)(Keep.right).runWith(Sink.asPublisher(false)))
|
||||
}
|
||||
// The fanout publisher must be wired to the original source after all fanout subscribers have been subscribed
|
||||
input.runWith(Sink(fanoutSub))
|
||||
input.runWith(Sink.fromSubscriber(fanoutSub))
|
||||
sources
|
||||
}
|
||||
|
||||
|
|
@ -315,7 +315,7 @@ private[http] object StreamUtils {
|
|||
object OneTimeValve {
|
||||
def apply(): OneTimeValve = new OneTimeValve {
|
||||
val promise = Promise[Unit]()
|
||||
val _source = Source(promise.future).drop(1) // we are only interested in the completion event
|
||||
val _source = Source.fromFuture(promise.future).drop(1) // we are only interested in the completion event
|
||||
|
||||
def source[T]: Source[T, Unit] = _source.asInstanceOf[Source[T, Unit]] // safe, because source won't generate any elements
|
||||
def open(): Unit = promise.success(())
|
||||
|
|
@ -331,4 +331,4 @@ private[http] class EnhancedByteStringSource[Mat](val byteStringStream: Source[B
|
|||
byteStringStream.runFold(ByteString.empty)(_ ++ _)
|
||||
def utf8String(implicit materializer: Materializer, ec: ExecutionContext): Future[String] =
|
||||
join.map(_.utf8String)
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -54,7 +54,7 @@ public class WSEchoTestClientApp {
|
|||
TextMessage.create("abc"),
|
||||
TextMessage.create("def"),
|
||||
TextMessage.create("ghi")
|
||||
)).concat(Source.from(delayedCompletion).drop(1));
|
||||
)).concat(Source.fromFuture(delayedCompletion).drop(1));
|
||||
|
||||
Sink<Message, Future<List<String>>> echoSink =
|
||||
Flow.of(Message.class)
|
||||
|
|
|
|||
|
|
@ -33,7 +33,7 @@ class ClientCancellationSpec extends AkkaSpec("""
|
|||
def testCase(connection: Flow[HttpRequest, HttpResponse, Any]): Unit = Utils.assertAllStagesStopped {
|
||||
val requests = TestPublisher.probe[HttpRequest]()
|
||||
val responses = TestSubscriber.probe[HttpResponse]()
|
||||
Source(requests).via(connection).runWith(Sink(responses))
|
||||
Source.fromPublisher(requests).via(connection).runWith(Sink.fromSubscriber(responses))
|
||||
responses.request(1)
|
||||
requests.sendNext(HttpRequest())
|
||||
responses.expectNext().entity.dataBytes.runWith(Sink.cancelled)
|
||||
|
|
|
|||
|
|
@ -89,7 +89,7 @@ class ConnectionPoolSpec extends AkkaSpec("""
|
|||
|
||||
override def testServerHandler(connNr: Int): HttpRequest ⇒ HttpResponse = {
|
||||
case request @ HttpRequest(_, Uri.Path("/a"), _, _, _) ⇒
|
||||
val entity = HttpEntity.Chunked.fromData(ContentTypes.`text/plain(UTF-8)`, Source(responseEntityPub))
|
||||
val entity = HttpEntity.Chunked.fromData(ContentTypes.`text/plain(UTF-8)`, Source.fromPublisher(responseEntityPub))
|
||||
super.testServerHandler(connNr)(request) withEntity entity
|
||||
case x ⇒ super.testServerHandler(connNr)(x)
|
||||
}
|
||||
|
|
@ -99,7 +99,7 @@ class ConnectionPoolSpec extends AkkaSpec("""
|
|||
acceptIncomingConnection()
|
||||
val (Success(r1), 42) = responseOut.expectNext()
|
||||
val responseEntityProbe = TestSubscriber.probe[ByteString]()
|
||||
r1.entity.dataBytes.runWith(Sink(responseEntityProbe))
|
||||
r1.entity.dataBytes.runWith(Sink.fromSubscriber(responseEntityProbe))
|
||||
responseEntityProbe.expectSubscription().request(2)
|
||||
responseEntityPub.sendNext(ByteString("YEAH"))
|
||||
responseEntityProbe.expectNext(ByteString("YEAH"))
|
||||
|
|
@ -131,7 +131,7 @@ class ConnectionPoolSpec extends AkkaSpec("""
|
|||
val poolFlow = Http().cachedHostConnectionPool[Int](serverHostName, serverPort, settings = settings)
|
||||
|
||||
val N = 500
|
||||
val requestIds = Source(() ⇒ Iterator.from(1)).take(N)
|
||||
val requestIds = Source.fromIterator(() ⇒ Iterator.from(1)).take(N)
|
||||
val idSum = requestIds.map(id ⇒ HttpRequest(uri = s"/r$id") -> id).via(poolFlow).map {
|
||||
case (Success(response), id) ⇒
|
||||
requestUri(response) should endWith(s"/r$id")
|
||||
|
|
@ -301,7 +301,7 @@ class ConnectionPoolSpec extends AkkaSpec("""
|
|||
Flow[SslTlsOutbound].collect[ByteString] { case SendBytes(x) ⇒ mapServerSideOutboundRawBytes(x) }
|
||||
.transform(StreamUtils.recover { case NoErrorComplete ⇒ ByteString.empty }),
|
||||
Flow[ByteString].map(SessionBytes(null, _)))
|
||||
val sink = if (autoAccept) Sink.foreach[Http.IncomingConnection](handleConnection) else Sink(incomingConnections)
|
||||
val sink = if (autoAccept) Sink.foreach[Http.IncomingConnection](handleConnection) else Sink.fromSubscriber(incomingConnections)
|
||||
// TODO getHostString in Java7
|
||||
Tcp().bind(serverEndpoint.getHostName, serverEndpoint.getPort, idleTimeout = serverSettings.timeouts.idleTimeout)
|
||||
.map { c ⇒
|
||||
|
|
@ -345,7 +345,7 @@ class ConnectionPoolSpec extends AkkaSpec("""
|
|||
def flowTestBench[T, Mat](poolFlow: Flow[(HttpRequest, T), (Try[HttpResponse], T), Mat]) = {
|
||||
val requestIn = TestPublisher.probe[(HttpRequest, T)]()
|
||||
val responseOut = TestSubscriber.manualProbe[(Try[HttpResponse], T)]
|
||||
val hcp = Source(requestIn).viaMat(poolFlow)(Keep.right).to(Sink(responseOut)).run()
|
||||
val hcp = Source.fromPublisher(requestIn).viaMat(poolFlow)(Keep.right).to(Sink.fromSubscriber(responseOut)).run()
|
||||
val responseOutSub = responseOut.expectSubscription()
|
||||
(requestIn, responseOut, responseOutSub, hcp)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -24,7 +24,7 @@ class HighLevelOutgoingConnectionSpec extends AkkaSpec {
|
|||
serverHostName, serverPort)
|
||||
|
||||
val N = 100
|
||||
val result = Source(() ⇒ Iterator.from(1))
|
||||
val result = Source.fromIterator(() ⇒ Iterator.from(1))
|
||||
.take(N)
|
||||
.map(id ⇒ HttpRequest(uri = s"/r$id"))
|
||||
.via(Http().outgoingConnection(serverHostName, serverPort))
|
||||
|
|
@ -56,7 +56,7 @@ class HighLevelOutgoingConnectionSpec extends AkkaSpec {
|
|||
})
|
||||
|
||||
val N = 100
|
||||
val result = Source(() ⇒ Iterator.from(1))
|
||||
val result = Source.fromIterator(() ⇒ Iterator.from(1))
|
||||
.take(N)
|
||||
.map(id ⇒ HttpRequest(uri = s"/r$id"))
|
||||
.via(doubleConnection)
|
||||
|
|
|
|||
|
|
@ -45,7 +45,7 @@ class LowLevelOutgoingConnectionSpec extends AkkaSpec("akka.loggers = []\n akka.
|
|||
|
||||
"has a request with default entity" in new TestSetup {
|
||||
val probe = TestPublisher.manualProbe[ByteString]()
|
||||
requestsSub.sendNext(HttpRequest(PUT, entity = HttpEntity(ContentTypes.`application/octet-stream`, 8, Source(probe))))
|
||||
requestsSub.sendNext(HttpRequest(PUT, entity = HttpEntity(ContentTypes.`application/octet-stream`, 8, Source.fromPublisher(probe))))
|
||||
expectWireData(
|
||||
"""PUT / HTTP/1.1
|
||||
|Host: example.com
|
||||
|
|
@ -90,7 +90,7 @@ class LowLevelOutgoingConnectionSpec extends AkkaSpec("akka.loggers = []\n akka.
|
|||
ct shouldEqual ContentTypes.`application/octet-stream`
|
||||
|
||||
val probe = TestSubscriber.manualProbe[ChunkStreamPart]()
|
||||
chunks.runWith(Sink(probe))
|
||||
chunks.runWith(Sink.fromSubscriber(probe))
|
||||
val sub = probe.expectSubscription()
|
||||
|
||||
sendWireData("3\nABC\n")
|
||||
|
|
@ -155,7 +155,7 @@ class LowLevelOutgoingConnectionSpec extends AkkaSpec("akka.loggers = []\n akka.
|
|||
val HttpResponse(_, _, HttpEntity.Chunked(ct, chunks), _) = expectResponse()
|
||||
|
||||
val probe = TestSubscriber.manualProbe[ChunkStreamPart]()
|
||||
chunks.runWith(Sink(probe))
|
||||
chunks.runWith(Sink.fromSubscriber(probe))
|
||||
val sub = probe.expectSubscription()
|
||||
|
||||
sendWireData("3\nABC\n")
|
||||
|
|
@ -196,7 +196,7 @@ class LowLevelOutgoingConnectionSpec extends AkkaSpec("akka.loggers = []\n akka.
|
|||
|
||||
"catch the request entity stream being shorter than the Content-Length" in new TestSetup {
|
||||
val probe = TestPublisher.manualProbe[ByteString]()
|
||||
requestsSub.sendNext(HttpRequest(PUT, entity = HttpEntity(ContentTypes.`application/octet-stream`, 8, Source(probe))))
|
||||
requestsSub.sendNext(HttpRequest(PUT, entity = HttpEntity(ContentTypes.`application/octet-stream`, 8, Source.fromPublisher(probe))))
|
||||
expectWireData(
|
||||
"""PUT / HTTP/1.1
|
||||
|Host: example.com
|
||||
|
|
@ -222,7 +222,7 @@ class LowLevelOutgoingConnectionSpec extends AkkaSpec("akka.loggers = []\n akka.
|
|||
|
||||
"catch the request entity stream being longer than the Content-Length" in new TestSetup {
|
||||
val probe = TestPublisher.manualProbe[ByteString]()
|
||||
requestsSub.sendNext(HttpRequest(PUT, entity = HttpEntity(ContentTypes.`application/octet-stream`, 8, Source(probe))))
|
||||
requestsSub.sendNext(HttpRequest(PUT, entity = HttpEntity(ContentTypes.`application/octet-stream`, 8, Source.fromPublisher(probe))))
|
||||
expectWireData(
|
||||
"""PUT / HTTP/1.1
|
||||
|Host: example.com
|
||||
|
|
@ -274,7 +274,7 @@ class LowLevelOutgoingConnectionSpec extends AkkaSpec("akka.loggers = []\n akka.
|
|||
ct shouldEqual ContentTypes.`application/octet-stream`
|
||||
|
||||
val probe = TestSubscriber.manualProbe[ChunkStreamPart]()
|
||||
chunks.runWith(Sink(probe))
|
||||
chunks.runWith(Sink.fromSubscriber(probe))
|
||||
val sub = probe.expectSubscription()
|
||||
|
||||
sendWireData("3\nABC\n")
|
||||
|
|
@ -531,10 +531,10 @@ class LowLevelOutgoingConnectionSpec extends AkkaSpec("akka.loggers = []\n akka.
|
|||
RunnableGraph.fromGraph(GraphDSL.create(OutgoingConnectionBlueprint(Host("example.com"), settings, NoLogging)) { implicit b ⇒
|
||||
client ⇒
|
||||
import GraphDSL.Implicits._
|
||||
Source(netIn) ~> Flow[ByteString].map(SessionBytes(null, _)) ~> client.in2
|
||||
client.out1 ~> Flow[SslTlsOutbound].collect { case SendBytes(x) ⇒ x } ~> Sink(netOut)
|
||||
Source(requests) ~> client.in1
|
||||
client.out2 ~> Sink(responses)
|
||||
Source.fromPublisher(netIn) ~> Flow[ByteString].map(SessionBytes(null, _)) ~> client.in2
|
||||
client.out1 ~> Flow[SslTlsOutbound].collect { case SendBytes(x) ⇒ x } ~> Sink.fromSubscriber(netOut)
|
||||
Source.fromPublisher(requests) ~> client.in1
|
||||
client.out2 ~> Sink.fromSubscriber(responses)
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
|
|
|
|||
|
|
@ -484,7 +484,7 @@ class RequestParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
|
|||
}
|
||||
.concatSubstreams
|
||||
.flatMapConcat { x ⇒
|
||||
Source {
|
||||
Source.fromFuture {
|
||||
x match {
|
||||
case Right(request) ⇒ compactEntity(request.entity).fast.map(x ⇒ Right(request.withEntity(x)))
|
||||
case Left(error) ⇒ FastFuture.successful(Left(error))
|
||||
|
|
|
|||
|
|
@ -46,7 +46,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
|
|||
inside(expectRequest()) {
|
||||
case HttpRequest(POST, _, _, HttpEntity.Default(_, 12, data), _) ⇒
|
||||
val dataProbe = TestSubscriber.manualProbe[ByteString]
|
||||
data.to(Sink(dataProbe)).run()
|
||||
data.to(Sink.fromSubscriber(dataProbe)).run()
|
||||
val sub = dataProbe.expectSubscription()
|
||||
sub.request(10)
|
||||
dataProbe.expectNoMsg(50.millis)
|
||||
|
|
@ -89,7 +89,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
|
|||
inside(expectRequest()) {
|
||||
case HttpRequest(POST, _, _, HttpEntity.Chunked(_, data), _) ⇒
|
||||
val dataProbe = TestSubscriber.manualProbe[ChunkStreamPart]
|
||||
data.to(Sink(dataProbe)).run()
|
||||
data.to(Sink.fromSubscriber(dataProbe)).run()
|
||||
val sub = dataProbe.expectSubscription()
|
||||
sub.request(10)
|
||||
dataProbe.expectNext(Chunk(ByteString("abcdef")))
|
||||
|
|
@ -140,7 +140,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
|
|||
inside(expectRequest()) {
|
||||
case HttpRequest(POST, _, _, HttpEntity.Default(_, 12, data), _) ⇒
|
||||
val dataProbe = TestSubscriber.manualProbe[ByteString]
|
||||
data.to(Sink(dataProbe)).run()
|
||||
data.to(Sink.fromSubscriber(dataProbe)).run()
|
||||
val sub = dataProbe.expectSubscription()
|
||||
sub.request(10)
|
||||
dataProbe.expectNext(ByteString("abcdef"))
|
||||
|
|
@ -163,7 +163,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
|
|||
inside(expectRequest()) {
|
||||
case HttpRequest(POST, _, _, HttpEntity.Chunked(_, data), _) ⇒
|
||||
val dataProbe = TestSubscriber.manualProbe[ChunkStreamPart]
|
||||
data.to(Sink(dataProbe)).run()
|
||||
data.to(Sink.fromSubscriber(dataProbe)).run()
|
||||
val sub = dataProbe.expectSubscription()
|
||||
sub.request(10)
|
||||
dataProbe.expectNext(Chunk(ByteString("abcdef")))
|
||||
|
|
@ -212,7 +212,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
|
|||
inside(expectRequest()) {
|
||||
case HttpRequest(POST, _, _, HttpEntity.Default(_, 12, data), _) ⇒
|
||||
val dataProbe = TestSubscriber.manualProbe[ByteString]
|
||||
data.to(Sink(dataProbe)).run()
|
||||
data.to(Sink.fromSubscriber(dataProbe)).run()
|
||||
val sub = dataProbe.expectSubscription()
|
||||
sub.request(10)
|
||||
dataProbe.expectNext(ByteString("abcdef"))
|
||||
|
|
@ -249,7 +249,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
|
|||
inside(expectRequest()) {
|
||||
case HttpRequest(POST, _, _, HttpEntity.Chunked(_, data), _) ⇒
|
||||
val dataProbe = TestSubscriber.manualProbe[ChunkStreamPart]
|
||||
data.to(Sink(dataProbe)).run()
|
||||
data.to(Sink.fromSubscriber(dataProbe)).run()
|
||||
val sub = dataProbe.expectSubscription()
|
||||
sub.request(10)
|
||||
dataProbe.expectNext(Chunk(ByteString("abcdef")))
|
||||
|
|
@ -285,7 +285,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
|
|||
inside(expectRequest()) {
|
||||
case HttpRequest(POST, _, _, HttpEntity.Default(_, 12, data), _) ⇒
|
||||
val dataProbe = TestSubscriber.manualProbe[ByteString]
|
||||
data.to(Sink(dataProbe)).run()
|
||||
data.to(Sink.fromSubscriber(dataProbe)).run()
|
||||
val sub = dataProbe.expectSubscription()
|
||||
sub.request(10)
|
||||
dataProbe.expectNext(ByteString("abcdef"))
|
||||
|
|
@ -308,7 +308,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
|
|||
inside(expectRequest()) {
|
||||
case HttpRequest(POST, _, _, HttpEntity.Chunked(_, data), _) ⇒
|
||||
val dataProbe = TestSubscriber.manualProbe[ChunkStreamPart]
|
||||
data.to(Sink(dataProbe)).run()
|
||||
data.to(Sink.fromSubscriber(dataProbe)).run()
|
||||
val sub = dataProbe.expectSubscription()
|
||||
sub.request(10)
|
||||
dataProbe.expectNext(Chunk(ByteString("abcdef")))
|
||||
|
|
@ -329,7 +329,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
|
|||
inside(expectRequest()) {
|
||||
case HttpRequest(POST, _, _, HttpEntity.Default(_, 12, data), _) ⇒
|
||||
val dataProbe = TestSubscriber.manualProbe[ByteString]
|
||||
data.to(Sink(dataProbe)).run()
|
||||
data.to(Sink.fromSubscriber(dataProbe)).run()
|
||||
val sub = dataProbe.expectSubscription()
|
||||
sub.request(10)
|
||||
dataProbe.expectNext(ByteString("abcdef"))
|
||||
|
|
@ -350,7 +350,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
|
|||
inside(expectRequest()) {
|
||||
case HttpRequest(POST, _, _, HttpEntity.Chunked(_, data), _) ⇒
|
||||
val dataProbe = TestSubscriber.manualProbe[ChunkStreamPart]
|
||||
data.to(Sink(dataProbe)).run()
|
||||
data.to(Sink.fromSubscriber(dataProbe)).run()
|
||||
val sub = dataProbe.expectSubscription()
|
||||
sub.request(10)
|
||||
dataProbe.expectNext(Chunk(ByteString("abcdef")))
|
||||
|
|
@ -405,7 +405,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
|
|||
val data = TestPublisher.manualProbe[ByteString]()
|
||||
inside(expectRequest()) {
|
||||
case HttpRequest(GET, _, _, _, _) ⇒
|
||||
responses.sendNext(HttpResponse(entity = HttpEntity.Default(ContentTypes.`text/plain(UTF-8)`, 4, Source(data))))
|
||||
responses.sendNext(HttpResponse(entity = HttpEntity.Default(ContentTypes.`text/plain(UTF-8)`, 4, Source.fromPublisher(data))))
|
||||
val dataSub = data.expectSubscription()
|
||||
dataSub.expectCancellation()
|
||||
expectResponseWithWipedDate(
|
||||
|
|
@ -427,7 +427,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
|
|||
val data = TestPublisher.manualProbe[ByteString]()
|
||||
inside(expectRequest()) {
|
||||
case HttpRequest(GET, _, _, _, _) ⇒
|
||||
responses.sendNext(HttpResponse(entity = HttpEntity.CloseDelimited(ContentTypes.`text/plain(UTF-8)`, Source(data))))
|
||||
responses.sendNext(HttpResponse(entity = HttpEntity.CloseDelimited(ContentTypes.`text/plain(UTF-8)`, Source.fromPublisher(data))))
|
||||
val dataSub = data.expectSubscription()
|
||||
dataSub.expectCancellation()
|
||||
expectResponseWithWipedDate(
|
||||
|
|
@ -450,7 +450,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
|
|||
val data = TestPublisher.manualProbe[ChunkStreamPart]()
|
||||
inside(expectRequest()) {
|
||||
case HttpRequest(GET, _, _, _, _) ⇒
|
||||
responses.sendNext(HttpResponse(entity = HttpEntity.Chunked(ContentTypes.`text/plain(UTF-8)`, Source(data))))
|
||||
responses.sendNext(HttpResponse(entity = HttpEntity.Chunked(ContentTypes.`text/plain(UTF-8)`, Source.fromPublisher(data))))
|
||||
val dataSub = data.expectSubscription()
|
||||
dataSub.expectCancellation()
|
||||
expectResponseWithWipedDate(
|
||||
|
|
@ -473,7 +473,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
|
|||
val data = TestPublisher.manualProbe[ByteString]()
|
||||
inside(expectRequest()) {
|
||||
case HttpRequest(GET, _, _, _, _) ⇒
|
||||
responses.sendNext(HttpResponse(entity = CloseDelimited(ContentTypes.`text/plain(UTF-8)`, Source(data))))
|
||||
responses.sendNext(HttpResponse(entity = CloseDelimited(ContentTypes.`text/plain(UTF-8)`, Source.fromPublisher(data))))
|
||||
val dataSub = data.expectSubscription()
|
||||
dataSub.expectCancellation()
|
||||
netOut.expectBytes(1)
|
||||
|
|
@ -491,7 +491,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
|
|||
inside(expectRequest()) {
|
||||
case HttpRequest(POST, _, _, Default(ContentType(`application/octet-stream`, None), 16, data), _) ⇒
|
||||
val dataProbe = TestSubscriber.manualProbe[ByteString]
|
||||
data.to(Sink(dataProbe)).run()
|
||||
data.to(Sink.fromSubscriber(dataProbe)).run()
|
||||
val dataSub = dataProbe.expectSubscription()
|
||||
netOut.expectNoBytes(50.millis)
|
||||
dataSub.request(1) // triggers `100 Continue` response
|
||||
|
|
@ -527,7 +527,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
|
|||
inside(expectRequest()) {
|
||||
case HttpRequest(POST, _, _, Chunked(ContentType(`application/octet-stream`, None), data), _) ⇒
|
||||
val dataProbe = TestSubscriber.manualProbe[ChunkStreamPart]
|
||||
data.to(Sink(dataProbe)).run()
|
||||
data.to(Sink.fromSubscriber(dataProbe)).run()
|
||||
val dataSub = dataProbe.expectSubscription()
|
||||
netOut.expectNoBytes(50.millis)
|
||||
dataSub.request(2) // triggers `100 Continue` response
|
||||
|
|
|
|||
|
|
@ -38,10 +38,10 @@ abstract class HttpServerTestSetupBase {
|
|||
RunnableGraph.fromGraph(GraphDSL.create(HttpServerBluePrint(settings, remoteAddress = remoteAddress, log = NoLogging)) { implicit b ⇒
|
||||
server ⇒
|
||||
import GraphDSL.Implicits._
|
||||
Source(netIn) ~> Flow[ByteString].map(SessionBytes(null, _)) ~> server.in2
|
||||
Source.fromPublisher(netIn) ~> Flow[ByteString].map(SessionBytes(null, _)) ~> server.in2
|
||||
server.out1 ~> Flow[SslTlsOutbound].collect { case SendBytes(x) ⇒ x }.buffer(1, OverflowStrategy.backpressure) ~> netOut.sink
|
||||
server.out2 ~> Sink(requests)
|
||||
Source(responses) ~> server.in1
|
||||
server.out2 ~> Sink.fromSubscriber(requests)
|
||||
Source.fromPublisher(responses) ~> server.in1
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
|
|
|
|||
|
|
@ -35,7 +35,7 @@ object ByteStringSinkProbe {
|
|||
def apply()(implicit system: ActorSystem): ByteStringSinkProbe =
|
||||
new ByteStringSinkProbe {
|
||||
val probe = TestSubscriber.probe[ByteString]()
|
||||
val sink: Sink[ByteString, Unit] = Sink(probe)
|
||||
val sink: Sink[ByteString, Unit] = Sink.fromSubscriber(probe)
|
||||
|
||||
def expectNoBytes(): Unit = {
|
||||
probe.ensureSubscription()
|
||||
|
|
|
|||
|
|
@ -47,7 +47,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
|||
pushInput(header ++ data1)
|
||||
val dataSource = expectBinaryMessage().dataStream
|
||||
val sub = TestSubscriber.manualProbe[ByteString]()
|
||||
dataSource.runWith(Sink(sub))
|
||||
dataSource.runWith(Sink.fromSubscriber(sub))
|
||||
val s = sub.expectSubscription()
|
||||
s.request(2)
|
||||
sub.expectNext(data1)
|
||||
|
|
@ -59,7 +59,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
|||
pushInput(header)
|
||||
val dataSource = expectBinaryMessage().dataStream
|
||||
val sub = TestSubscriber.manualProbe[ByteString]()
|
||||
dataSource.runWith(Sink(sub))
|
||||
dataSource.runWith(Sink.fromSubscriber(sub))
|
||||
val s = sub.expectSubscription()
|
||||
s.request(2)
|
||||
pushInput(data1)
|
||||
|
|
@ -78,7 +78,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
|||
pushInput(header1 ++ data1)
|
||||
val dataSource = expectBinaryMessage().dataStream
|
||||
val sub = TestSubscriber.manualProbe[ByteString]()
|
||||
dataSource.runWith(Sink(sub))
|
||||
dataSource.runWith(Sink.fromSubscriber(sub))
|
||||
val s = sub.expectSubscription()
|
||||
s.request(2)
|
||||
sub.expectNext(data1)
|
||||
|
|
@ -96,7 +96,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
|||
pushInput(header1 ++ data1)
|
||||
val dataSource = expectBinaryMessage().dataStream
|
||||
val sub = TestSubscriber.manualProbe[ByteString]()
|
||||
dataSource.runWith(Sink(sub))
|
||||
dataSource.runWith(Sink.fromSubscriber(sub))
|
||||
val s = sub.expectSubscription()
|
||||
s.request(2)
|
||||
sub.expectNext(data1)
|
||||
|
|
@ -111,7 +111,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
|||
|
||||
val dataSource2 = expectBinaryMessage().dataStream
|
||||
val sub2 = TestSubscriber.manualProbe[ByteString]()
|
||||
dataSource2.runWith(Sink(sub2))
|
||||
dataSource2.runWith(Sink.fromSubscriber(sub2))
|
||||
val s2 = sub2.expectSubscription()
|
||||
s2.request(2)
|
||||
sub2.expectNext(data3)
|
||||
|
|
@ -131,7 +131,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
|||
pushInput(header ++ data1)
|
||||
val dataSource = expectBinaryMessage().dataStream
|
||||
val sub = TestSubscriber.manualProbe[ByteString]()
|
||||
dataSource.runWith(Sink(sub))
|
||||
dataSource.runWith(Sink.fromSubscriber(sub))
|
||||
val s = sub.expectSubscription()
|
||||
s.request(2)
|
||||
sub.expectNext(ByteString("abc", "ASCII"))
|
||||
|
|
@ -174,7 +174,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
|||
pushInput(input)
|
||||
val parts = expectTextMessage().textStream
|
||||
val sub = TestSubscriber.manualProbe[String]()
|
||||
parts.runWith(Sink(sub))
|
||||
parts.runWith(Sink.fromSubscriber(sub))
|
||||
val s = sub.expectSubscription()
|
||||
s.request(4)
|
||||
sub.expectNext("b")
|
||||
|
|
@ -192,7 +192,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
|||
pushInput(header0 ++ data0)
|
||||
val parts = expectTextMessage().textStream
|
||||
val sub = TestSubscriber.manualProbe[String]()
|
||||
parts.runWith(Sink(sub))
|
||||
parts.runWith(Sink.fromSubscriber(sub))
|
||||
val s = sub.expectSubscription()
|
||||
s.request(4)
|
||||
sub.expectNoMsg(100.millis)
|
||||
|
|
@ -211,7 +211,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
|||
pushInput(header ++ data1)
|
||||
val dataSource = expectBinaryMessage().dataStream
|
||||
val sub = TestSubscriber.manualProbe[ByteString]()
|
||||
dataSource.runWith(Sink(sub))
|
||||
dataSource.runWith(Sink.fromSubscriber(sub))
|
||||
val s = sub.expectSubscription()
|
||||
s.request(2)
|
||||
sub.expectNext(ByteString("äb", "UTF-8"))
|
||||
|
|
@ -242,7 +242,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
|||
"for a streamed message" in new ServerTestSetup {
|
||||
val data = ByteString("abcdefg", "ASCII")
|
||||
val pub = TestPublisher.manualProbe[ByteString]()
|
||||
val msg = BinaryMessage(Source(pub))
|
||||
val msg = BinaryMessage(Source.fromPublisher(pub))
|
||||
pushMessage(msg)
|
||||
val sub = pub.expectSubscription()
|
||||
|
||||
|
|
@ -264,7 +264,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
|||
"and mask input on the client side" in new ClientTestSetup {
|
||||
val data = ByteString("abcdefg", "ASCII")
|
||||
val pub = TestPublisher.manualProbe[ByteString]()
|
||||
val msg = BinaryMessage(Source(pub))
|
||||
val msg = BinaryMessage(Source.fromPublisher(pub))
|
||||
pushMessage(msg)
|
||||
val sub = pub.expectSubscription()
|
||||
|
||||
|
|
@ -299,7 +299,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
|||
"for a streamed message" in new ServerTestSetup {
|
||||
val text = "äbcd€fg"
|
||||
val pub = TestPublisher.manualProbe[String]()
|
||||
val msg = TextMessage(Source(pub))
|
||||
val msg = TextMessage(Source.fromPublisher(pub))
|
||||
pushMessage(msg)
|
||||
val sub = pub.expectSubscription()
|
||||
|
||||
|
|
@ -328,7 +328,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
|||
val half2 = gclef.drop(1)
|
||||
|
||||
val pub = TestPublisher.manualProbe[String]()
|
||||
val msg = TextMessage(Source(pub))
|
||||
val msg = TextMessage(Source.fromPublisher(pub))
|
||||
|
||||
pushMessage(msg)
|
||||
val sub = pub.expectSubscription()
|
||||
|
|
@ -344,7 +344,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
|||
"and mask input on the client side" in new ClientTestSetup {
|
||||
val text = "abcdefg"
|
||||
val pub = TestPublisher.manualProbe[String]()
|
||||
val msg = TextMessage(Source(pub))
|
||||
val msg = TextMessage(Source.fromPublisher(pub))
|
||||
pushMessage(msg)
|
||||
val sub = pub.expectSubscription()
|
||||
|
||||
|
|
@ -394,13 +394,13 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
|||
|
||||
val dataSource = expectBinaryMessage().dataStream
|
||||
val sub = TestSubscriber.manualProbe[ByteString]()
|
||||
dataSource.runWith(Sink(sub))
|
||||
dataSource.runWith(Sink.fromSubscriber(sub))
|
||||
val s = sub.expectSubscription()
|
||||
s.request(2)
|
||||
sub.expectNext(ByteString("123", "ASCII"))
|
||||
|
||||
val outPub = TestPublisher.manualProbe[ByteString]()
|
||||
val msg = BinaryMessage(Source(outPub))
|
||||
val msg = BinaryMessage(Source.fromPublisher(outPub))
|
||||
pushMessage(msg)
|
||||
|
||||
expectFrameHeaderOnNetwork(Opcode.Binary, 0, fin = false)
|
||||
|
|
@ -467,7 +467,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
|||
|
||||
// sending another message is allowed before closing (inherently racy)
|
||||
val pub = TestPublisher.manualProbe[ByteString]()
|
||||
val msg = BinaryMessage(Source(pub))
|
||||
val msg = BinaryMessage(Source.fromPublisher(pub))
|
||||
pushMessage(msg)
|
||||
expectFrameOnNetwork(Opcode.Binary, ByteString.empty, fin = false)
|
||||
|
||||
|
|
@ -487,7 +487,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
|||
pushInput(frameHeader(Protocol.Opcode.Binary, 0, fin = false, mask = Some(Random.nextInt())))
|
||||
val dataSource = expectBinaryMessage().dataStream
|
||||
val inSubscriber = TestSubscriber.manualProbe[ByteString]()
|
||||
dataSource.runWith(Sink(inSubscriber))
|
||||
dataSource.runWith(Sink.fromSubscriber(inSubscriber))
|
||||
val inSub = inSubscriber.expectSubscription()
|
||||
|
||||
val outData = ByteString("def", "ASCII")
|
||||
|
|
@ -508,7 +508,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
|||
// sending another message is allowed before closing (inherently racy)
|
||||
|
||||
val pub = TestPublisher.manualProbe[ByteString]()
|
||||
val msg = BinaryMessage(Source(pub))
|
||||
val msg = BinaryMessage(Source.fromPublisher(pub))
|
||||
pushMessage(msg)
|
||||
expectFrameOnNetwork(Opcode.Binary, ByteString.empty, fin = false)
|
||||
|
||||
|
|
@ -572,7 +572,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
|||
"when user handler closes main stream and substream only afterwards" in new ServerTestSetup {
|
||||
// send half a message
|
||||
val pub = TestPublisher.manualProbe[ByteString]()
|
||||
val msg = BinaryMessage(Source(pub))
|
||||
val msg = BinaryMessage(Source.fromPublisher(pub))
|
||||
pushMessage(msg)
|
||||
expectFrameOnNetwork(Opcode.Binary, ByteString.empty, fin = false)
|
||||
|
||||
|
|
@ -823,10 +823,10 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
|||
|
||||
val messageHandler: Flow[Message, Message, Unit] =
|
||||
Flow.fromSinkAndSource(
|
||||
Flow[Message].buffer(1, OverflowStrategy.backpressure).to(Sink(messageIn)), // alternatively need to request(1) before expectComplete
|
||||
Source(messageOut))
|
||||
Flow[Message].buffer(1, OverflowStrategy.backpressure).to(Sink.fromSubscriber(messageIn)), // alternatively need to request(1) before expectComplete
|
||||
Source.fromPublisher(messageOut))
|
||||
|
||||
Source(netIn)
|
||||
Source.fromPublisher(netIn)
|
||||
.via(printEvent("netIn"))
|
||||
.via(FrameEventParser)
|
||||
.via(Websocket
|
||||
|
|
|
|||
|
|
@ -315,7 +315,7 @@ class WebsocketClientSpec extends FreeSpec with Matchers with WithMaterializerSp
|
|||
RunnableGraph.fromGraph(GraphDSL.create(clientLayer) { implicit b ⇒
|
||||
client ⇒
|
||||
import GraphDSL.Implicits._
|
||||
Source(netIn) ~> Flow[ByteString].map(SessionBytes(null, _)) ~> client.in2
|
||||
Source.fromPublisher(netIn) ~> Flow[ByteString].map(SessionBytes(null, _)) ~> client.in2
|
||||
client.out1 ~> Flow[SslTlsOutbound].collect { case SendBytes(x) ⇒ x } ~> netOut.sink
|
||||
client.out2 ~> clientImplementation ~> client.in1
|
||||
ClosedShape
|
||||
|
|
@ -370,6 +370,6 @@ class WebsocketClientSpec extends FreeSpec with Matchers with WithMaterializerSp
|
|||
lazy val messagesIn = TestSubscriber.probe[Message]()
|
||||
|
||||
override def clientImplementation: Flow[Message, Message, Unit] =
|
||||
Flow.fromSinkAndSourceMat(Sink(messagesIn), Source(messagesOut))(Keep.none)
|
||||
Flow.fromSinkAndSourceMat(Sink.fromSubscriber(messagesIn), Source.fromPublisher(messagesOut))(Keep.none)
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -49,7 +49,7 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
|
|||
"properly bind a server" in {
|
||||
val (_, hostname, port) = TestUtils.temporaryServerHostnameAndPort()
|
||||
val probe = TestSubscriber.manualProbe[Http.IncomingConnection]()
|
||||
val binding = Http().bind(hostname, port).toMat(Sink(probe))(Keep.left).run()
|
||||
val binding = Http().bind(hostname, port).toMat(Sink.fromSubscriber(probe))(Keep.left).run()
|
||||
val sub = probe.expectSubscription() // if we get it we are bound
|
||||
val address = Await.result(binding, 1.second).localAddress
|
||||
sub.cancel()
|
||||
|
|
@ -60,15 +60,15 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
|
|||
val binding = Http().bind(hostname, port)
|
||||
val probe1 = TestSubscriber.manualProbe[Http.IncomingConnection]()
|
||||
// Bind succeeded, we have a local address
|
||||
val b1 = Await.result(binding.to(Sink(probe1)).run(), 3.seconds)
|
||||
val b1 = Await.result(binding.to(Sink.fromSubscriber(probe1)).run(), 3.seconds)
|
||||
probe1.expectSubscription()
|
||||
|
||||
val probe2 = TestSubscriber.manualProbe[Http.IncomingConnection]()
|
||||
an[BindFailedException] shouldBe thrownBy { Await.result(binding.to(Sink(probe2)).run(), 3.seconds) }
|
||||
an[BindFailedException] shouldBe thrownBy { Await.result(binding.to(Sink.fromSubscriber(probe2)).run(), 3.seconds) }
|
||||
probe2.expectSubscriptionAndError()
|
||||
|
||||
val probe3 = TestSubscriber.manualProbe[Http.IncomingConnection]()
|
||||
an[BindFailedException] shouldBe thrownBy { Await.result(binding.to(Sink(probe3)).run(), 3.seconds) }
|
||||
an[BindFailedException] shouldBe thrownBy { Await.result(binding.to(Sink.fromSubscriber(probe3)).run(), 3.seconds) }
|
||||
probe3.expectSubscriptionAndError()
|
||||
|
||||
// Now unbind the first
|
||||
|
|
@ -78,7 +78,7 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
|
|||
if (!akka.util.Helpers.isWindows) {
|
||||
val probe4 = TestSubscriber.manualProbe[Http.IncomingConnection]()
|
||||
// Bind succeeded, we have a local address
|
||||
val b2 = Await.result(binding.to(Sink(probe4)).run(), 3.seconds)
|
||||
val b2 = Await.result(binding.to(Sink.fromSubscriber(probe4)).run(), 3.seconds)
|
||||
probe4.expectSubscription()
|
||||
|
||||
// clean up
|
||||
|
|
@ -448,7 +448,7 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
|
|||
val settings = configOverrides.toOption.fold(ServerSettings(system))(ServerSettings(_))
|
||||
val connections = Http().bind(hostname, port, settings = settings)
|
||||
val probe = TestSubscriber.manualProbe[Http.IncomingConnection]
|
||||
val binding = connections.toMat(Sink(probe))(Keep.left).run()
|
||||
val binding = connections.toMat(Sink.fromSubscriber(probe))(Keep.left).run()
|
||||
(probe, binding)
|
||||
}
|
||||
val connSourceSub = connSource.expectSubscription()
|
||||
|
|
@ -457,9 +457,9 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
|
|||
val requestPublisherProbe = TestPublisher.manualProbe[HttpRequest]()
|
||||
val responseSubscriberProbe = TestSubscriber.manualProbe[HttpResponse]()
|
||||
|
||||
val connectionFuture = Source(requestPublisherProbe)
|
||||
val connectionFuture = Source.fromPublisher(requestPublisherProbe)
|
||||
.viaMat(Http().outgoingConnection(hostname, port, settings = settings))(Keep.right)
|
||||
.to(Sink(responseSubscriberProbe)).run()
|
||||
.to(Sink.fromSubscriber(responseSubscriberProbe)).run()
|
||||
|
||||
val connection = Await.result(connectionFuture, 3.seconds)
|
||||
|
||||
|
|
@ -471,8 +471,8 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
|
|||
def acceptConnection(): (TestSubscriber.ManualProbe[HttpRequest], TestPublisher.ManualProbe[HttpResponse]) = {
|
||||
connSourceSub.request(1)
|
||||
val incomingConnection = connSource.expectNext()
|
||||
val sink = Sink.publisher[HttpRequest](false)
|
||||
val source = Source.subscriber[HttpResponse]
|
||||
val sink = Sink.asPublisher[HttpRequest](false)
|
||||
val source = Source.asSubscriber[HttpResponse]
|
||||
|
||||
val handler = Flow.fromSinkAndSourceMat(sink, source)(Keep.both)
|
||||
|
||||
|
|
|
|||
|
|
@ -89,7 +89,7 @@ class HttpEntitySpec extends FreeSpec with MustMatchers with BeforeAndAfterAll {
|
|||
"Infinite data stream" in {
|
||||
val neverCompleted = Promise[ByteString]()
|
||||
intercept[TimeoutException] {
|
||||
Await.result(Default(tpe, 42, Source(neverCompleted.future)).toStrict(100.millis), 150.millis)
|
||||
Await.result(Default(tpe, 42, Source.fromFuture(neverCompleted.future)).toStrict(100.millis), 150.millis)
|
||||
}.getMessage must be("HttpEntity.toStrict timed out after 100 milliseconds while still waiting for outstanding data")
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -104,7 +104,7 @@ object WSProbe {
|
|||
val subscriber = TestSubscriber.probe[Message]()
|
||||
val publisher = TestPublisher.probe[Message]()
|
||||
|
||||
def flow: Flow[Message, Message, Unit] = Flow.fromSinkAndSourceMat(Sink(subscriber), Source(publisher))(Keep.none)
|
||||
def flow: Flow[Message, Message, Unit] = Flow.fromSinkAndSourceMat(Sink.fromSubscriber(subscriber), Source.fromPublisher(publisher))(Keep.none)
|
||||
|
||||
def sendMessage(message: Message): Unit = publisher.sendNext(message)
|
||||
def sendMessage(text: String): Unit = sendMessage(TextMessage(text))
|
||||
|
|
@ -139,4 +139,4 @@ object WSProbe {
|
|||
.awaitResult(maxChunkCollectionMills.millis)
|
||||
.reduce(reduce)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -138,7 +138,7 @@ abstract class CoderSpec extends WordSpec with CodecSpecSupport with Inspectors
|
|||
ByteString(Array.fill(size)(1.toByte))
|
||||
|
||||
val sizesAfterRoundtrip =
|
||||
Source(() ⇒ sizes.toIterator.map(createByteString))
|
||||
Source.fromIterator(() ⇒ sizes.toIterator.map(createByteString))
|
||||
.via(Coder.encoderFlow)
|
||||
.via(Coder.decoderFlow)
|
||||
.runFold(Seq.empty[Int])(_ :+ _.size)
|
||||
|
|
@ -186,5 +186,5 @@ abstract class CoderSpec extends WordSpec with CodecSpecSupport with Inspectors
|
|||
input.via(Coder.decoderFlow).join.awaitResult(3.seconds)
|
||||
|
||||
def decodeFromIterator(iterator: () ⇒ Iterator[ByteString]): ByteString =
|
||||
Await.result(Source(iterator).via(Coder.decoderFlow).join, 3.seconds)
|
||||
Await.result(Source.fromIterator(iterator).via(Coder.decoderFlow).join, 3.seconds)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -201,7 +201,7 @@ class CodingDirectivesSpec extends RoutingSpec with Inside {
|
|||
() ⇒ text.grouped(8).map { chars ⇒
|
||||
Chunk(chars.mkString): ChunkStreamPart
|
||||
}
|
||||
val chunkedTextEntity = HttpEntity.Chunked(ContentTypes.`text/plain(UTF-8)`, Source(textChunks))
|
||||
val chunkedTextEntity = HttpEntity.Chunked(ContentTypes.`text/plain(UTF-8)`, Source.fromIterator(textChunks))
|
||||
|
||||
Post() ~> `Accept-Encoding`(gzip) ~> {
|
||||
encodeResponseWith(Gzip) {
|
||||
|
|
|
|||
|
|
@ -10,7 +10,7 @@ import org.reactivestreams.Publisher
|
|||
class ConcatTest extends AkkaPublisherVerification[Int] {
|
||||
|
||||
def createPublisher(elements: Long): Publisher[Int] = {
|
||||
Source(iterable(elements / 2)).concat(Source(iterable((elements + 1) / 2))).runWith(Sink.publisher(false))
|
||||
Source(iterable(elements / 2)).concat(Source(iterable((elements + 1) / 2))).runWith(Sink.asPublisher(false))
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -15,7 +15,7 @@ class FanoutPublisherTest extends AkkaPublisherVerification[Int] {
|
|||
if (elements == 0) new immutable.Iterable[Int] { override def iterator = Iterator from 0 }
|
||||
else 0 until elements.toInt
|
||||
|
||||
Source(iterable).runWith(Sink.publisher(true))
|
||||
Source(iterable).runWith(Sink.asPublisher(true))
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -38,7 +38,7 @@ class FilePublisherTest extends AkkaPublisherVerification[ByteString] {
|
|||
def createPublisher(elements: Long): Publisher[ByteString] =
|
||||
Source.file(file, chunkSize = 512)
|
||||
.take(elements)
|
||||
.runWith(Sink.publisher(false))
|
||||
.runWith(Sink.asPublisher(false))
|
||||
|
||||
@AfterClass
|
||||
def after = file.delete()
|
||||
|
|
|
|||
|
|
@ -13,7 +13,7 @@ class FlattenTest extends AkkaPublisherVerification[Int] {
|
|||
def createPublisher(elements: Long): Publisher[Int] = {
|
||||
val s1 = Source(iterable(elements / 2))
|
||||
val s2 = Source(iterable((elements + 1) / 2))
|
||||
Source(List(s1, s2)).flatMapConcat(ConstantFun.scalaIdentityFunction).runWith(Sink.publisher(false))
|
||||
Source(List(s1, s2)).flatMapConcat(ConstantFun.scalaIdentityFunction).runWith(Sink.asPublisher(false))
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -9,7 +9,7 @@ import org.reactivestreams.Subscriber
|
|||
class FoldSinkSubscriberTest extends AkkaSubscriberBlackboxVerification[Int] {
|
||||
|
||||
override def createSubscriber(): Subscriber[Int] =
|
||||
Flow[Int].to(Sink.fold(0)(_ + _)).runWith(Source.subscriber)
|
||||
Flow[Int].to(Sink.fold(0)(_ + _)).runWith(Source.asSubscriber)
|
||||
|
||||
override def createElement(element: Int): Int = element
|
||||
}
|
||||
|
|
|
|||
|
|
@ -9,7 +9,7 @@ import org.reactivestreams.Subscriber
|
|||
class ForeachSinkSubscriberTest extends AkkaSubscriberBlackboxVerification[Int] {
|
||||
|
||||
override def createSubscriber(): Subscriber[Int] =
|
||||
Flow[Int].to(Sink.foreach { _ ⇒ }).runWith(Source.subscriber)
|
||||
Flow[Int].to(Sink.foreach { _ ⇒ }).runWith(Source.asSubscriber)
|
||||
|
||||
override def createElement(element: Int): Int = element
|
||||
}
|
||||
|
|
|
|||
|
|
@ -13,7 +13,7 @@ class FuturePublisherTest extends AkkaPublisherVerification[Int] {
|
|||
|
||||
def createPublisher(elements: Long): Publisher[Int] = {
|
||||
val p = Promise[Int]()
|
||||
val pub = Source(p.future).runWith(Sink.publisher(false))
|
||||
val pub = Source.fromFuture(p.future).runWith(Sink.asPublisher(false))
|
||||
p.success(0)
|
||||
pub
|
||||
}
|
||||
|
|
|
|||
|
|
@ -24,7 +24,7 @@ class GroupByTest extends AkkaPublisherVerification[Int] {
|
|||
.concatSubstreams
|
||||
.runWith(Sink.head)
|
||||
val groupSource = Await.result(futureGroupSource, 3.seconds)
|
||||
groupSource.runWith(Sink.publisher(false))
|
||||
groupSource.runWith(Sink.asPublisher(false))
|
||||
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -11,7 +11,7 @@ import org.reactivestreams._
|
|||
class IterablePublisherTest extends AkkaPublisherVerification[Int] {
|
||||
|
||||
override def createPublisher(elements: Long): Publisher[Int] = {
|
||||
Source(iterable(elements)).runWith(Sink.publisher(false))
|
||||
Source(iterable(elements)).runWith(Sink.asPublisher(false))
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -10,7 +10,7 @@ import akka.stream.scaladsl.{ Keep, Source, Sink }
|
|||
class MaybeSourceTest extends AkkaPublisherVerification[Int] {
|
||||
|
||||
def createPublisher(elements: Long): Publisher[Int] = {
|
||||
val (p, pub) = Source.maybe[Int].toMat(Sink.publisher(false))(Keep.both).run()
|
||||
val (p, pub) = Source.maybe[Int].toMat(Sink.asPublisher(false))(Keep.both).run()
|
||||
p success Some(1)
|
||||
pub
|
||||
}
|
||||
|
|
|
|||
|
|
@ -15,7 +15,7 @@ class PrefixAndTailTest extends AkkaPublisherVerification[Int] {
|
|||
def createPublisher(elements: Long): Publisher[Int] = {
|
||||
val futureTailSource = Source(iterable(elements)).prefixAndTail(0).map { case (_, tail) ⇒ tail }.runWith(Sink.head)
|
||||
val tailSource = Await.result(futureTailSource, 3.seconds)
|
||||
tailSource.runWith(Sink.publisher(false))
|
||||
tailSource.runWith(Sink.asPublisher(false))
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -11,7 +11,7 @@ import org.reactivestreams.Publisher
|
|||
class SingleElementSourceTest extends AkkaPublisherVerification[Int] {
|
||||
|
||||
def createPublisher(elements: Long): Publisher[Int] =
|
||||
Source.single(1).runWith(Sink.publisher(false))
|
||||
Source.single(1).runWith(Sink.asPublisher(false))
|
||||
|
||||
override def maxElementsFromPublisher(): Long = 1
|
||||
}
|
||||
|
|
|
|||
|
|
@ -24,7 +24,7 @@ class SplitWhenTest extends AkkaPublisherVerification[Int] {
|
|||
.concatSubstreams
|
||||
.runWith(Sink.head)
|
||||
val source = Await.result(futureSource, 3.seconds)
|
||||
source.runWith(Sink.publisher(false))
|
||||
source.runWith(Sink.asPublisher(false))
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -27,7 +27,7 @@ abstract class BaseTwoStreamsSetup extends AkkaSpec {
|
|||
|
||||
def completedPublisher[T]: Publisher[T] = TestPublisher.empty[T]
|
||||
|
||||
def nonemptyPublisher[T](elems: immutable.Iterable[T]): Publisher[T] = Source(elems).runWith(Sink.publisher(false))
|
||||
def nonemptyPublisher[T](elems: immutable.Iterable[T]): Publisher[T] = Source(elems).runWith(Sink.asPublisher(false))
|
||||
|
||||
def soonToFailPublisher[T]: Publisher[T] = TestPublisher.lazyError[T](TestException)
|
||||
|
||||
|
|
|
|||
|
|
@ -24,7 +24,7 @@ class ChainSetup[In, Out, M](
|
|||
|
||||
val upstream = TestPublisher.manualProbe[In]()
|
||||
val downstream = TestSubscriber.probe[Out]()
|
||||
private val s = Source(upstream).via(stream(Flow[In].map(x ⇒ x).named("buh")))
|
||||
private val s = Source.fromPublisher(upstream).via(stream(Flow[In].map(x ⇒ x).named("buh")))
|
||||
val publisher = toPublisher(s, materializer)
|
||||
val upstreamSubscription = upstream.expectSubscription()
|
||||
publisher.subscribe(downstream)
|
||||
|
|
|
|||
|
|
@ -20,7 +20,7 @@ trait ScriptedTest extends Matchers {
|
|||
class ScriptException(msg: String) extends RuntimeException(msg)
|
||||
|
||||
def toPublisher[In, Out]: (Source[Out, _], ActorMaterializer) ⇒ Publisher[Out] =
|
||||
(f, m) ⇒ f.runWith(Sink.publisher(false))(m)
|
||||
(f, m) ⇒ f.runWith(Sink.asPublisher(false))(m)
|
||||
|
||||
object Script {
|
||||
def apply[In, Out](phases: (Seq[In], Seq[Out])*): Script[In, Out] = {
|
||||
|
|
|
|||
|
|
@ -23,7 +23,7 @@ class StreamTestKitSpec extends AkkaSpec {
|
|||
|
||||
"#toStrict with failing source" in {
|
||||
val msg = intercept[AssertionError] {
|
||||
Source(() ⇒ new Iterator[Int] {
|
||||
Source.fromIterator(() ⇒ new Iterator[Int] {
|
||||
var i = 0
|
||||
override def hasNext: Boolean = true
|
||||
override def next(): Int = {
|
||||
|
|
|
|||
|
|
@ -22,7 +22,7 @@ class TestPublisherSubscriberSpec extends AkkaSpec {
|
|||
"have all events accessible from manual probes" in assertAllStagesStopped {
|
||||
val upstream = TestPublisher.manualProbe[Int]()
|
||||
val downstream = TestSubscriber.manualProbe[Int]()
|
||||
Source(upstream).runWith(Sink.publisher(false))(materializer).subscribe(downstream)
|
||||
Source.fromPublisher(upstream).runWith(Sink.asPublisher(false))(materializer).subscribe(downstream)
|
||||
|
||||
val upstreamSubscription = upstream.expectSubscription()
|
||||
val downstreamSubscription: Subscription = downstream.expectEventPF { case OnSubscribe(sub) ⇒ sub }
|
||||
|
|
@ -46,7 +46,7 @@ class TestPublisherSubscriberSpec extends AkkaSpec {
|
|||
"handle gracefully partial function that is not suitable" in assertAllStagesStopped {
|
||||
val upstream = TestPublisher.manualProbe[Int]()
|
||||
val downstream = TestSubscriber.manualProbe[Int]()
|
||||
Source(upstream).runWith(Sink.publisher(false))(materializer).subscribe(downstream)
|
||||
Source.fromPublisher(upstream).runWith(Sink.asPublisher(false))(materializer).subscribe(downstream)
|
||||
val upstreamSubscription = upstream.expectSubscription()
|
||||
val downstreamSubscription: Subscription = downstream.expectEventPF { case OnSubscribe(sub) ⇒ sub }
|
||||
|
||||
|
|
|
|||
|
|
@ -23,9 +23,9 @@ abstract class TwoStreamsSetup extends BaseTwoStreamsSetup {
|
|||
import GraphDSL.Implicits._
|
||||
val f = fixture(b)
|
||||
|
||||
Source(p1) ~> f.left
|
||||
Source(p2) ~> f.right
|
||||
f.out ~> Sink(subscriber)
|
||||
Source.fromPublisher(p1) ~> f.left
|
||||
Source.fromPublisher(p2) ~> f.right
|
||||
f.out ~> Sink.fromSubscriber(subscriber)
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
|
|
|
|||
|
|
@ -42,7 +42,7 @@ public class ActorPublisherTest extends StreamTest {
|
|||
final ActorRef ref = system
|
||||
.actorOf(Props.create(TestPublisher.class).withDispatcher("akka.test.stream-dispatcher"));
|
||||
final Publisher<Integer> publisher = UntypedActorPublisher.create(ref);
|
||||
Source.from(publisher)
|
||||
Source.fromPublisher(publisher)
|
||||
.runForeach(new akka.japi.function.Procedure<Integer>() {
|
||||
@Override
|
||||
public void apply(Integer elem) throws Exception {
|
||||
|
|
|
|||
|
|
@ -63,7 +63,7 @@ public class ActorSubscriberTest extends StreamTest {
|
|||
final Subscriber<Integer> subscriber = UntypedActorSubscriber.create(ref);
|
||||
final java.lang.Iterable<Integer> input = Arrays.asList(1, 2, 3);
|
||||
|
||||
Source.from(input).runWith(Sink.create(subscriber), materializer);
|
||||
Source.from(input).runWith(Sink.fromSubscriber(subscriber), materializer);
|
||||
|
||||
ref.tell("run", null);
|
||||
probe.expectMsgEquals(1);
|
||||
|
|
|
|||
|
|
@ -67,8 +67,8 @@ public class FlowGraphTest extends StreamTest {
|
|||
final Source<String, BoxedUnit> in1 = Source.from(Arrays.asList("a", "b", "c"));
|
||||
final Source<String, BoxedUnit> in2 = Source.from(Arrays.asList("d", "e", "f"));
|
||||
|
||||
final Sink<String, Publisher<String>> publisher = Sink.publisher(false);
|
||||
|
||||
final Sink<String, Publisher<String>> publisher = Sink.asPublisher(false);
|
||||
|
||||
final Source<String, BoxedUnit> source = Source.fromGraph(
|
||||
GraphDSL.create(new Function<GraphDSL.Builder<BoxedUnit>, SourceShape<String>>() {
|
||||
@Override
|
||||
|
|
@ -82,7 +82,7 @@ public class FlowGraphTest extends StreamTest {
|
|||
|
||||
// collecting
|
||||
final Publisher<String> pub = source.runWith(publisher, materializer);
|
||||
final Future<List<String>> all = Source.from(pub).grouped(100).runWith(Sink.<List<String>>head(), materializer);
|
||||
final Future<List<String>> all = Source.fromPublisher(pub).grouped(100).runWith(Sink.<List<String>>head(), materializer);
|
||||
|
||||
final List<String> result = Await.result(all, Duration.apply(200, TimeUnit.MILLISECONDS));
|
||||
assertEquals(new HashSet<Object>(Arrays.asList("a", "b", "c", "d", "e", "f")), new HashSet<String>(result));
|
||||
|
|
@ -257,7 +257,7 @@ public class FlowGraphTest extends StreamTest {
|
|||
return l + r;
|
||||
}
|
||||
});
|
||||
|
||||
|
||||
final Future<Integer> future = RunnableGraph.fromGraph(GraphDSL.create(Sink.<Integer>head(),
|
||||
new Function2<Builder<Future<Integer>>, SinkShape<Integer>, ClosedShape>() {
|
||||
@Override
|
||||
|
|
|
|||
|
|
@ -249,7 +249,7 @@ public class FlowTest extends StreamTest {
|
|||
})
|
||||
.grouped(10)
|
||||
.mergeSubstreams();
|
||||
|
||||
|
||||
final Future<List<List<String>>> future =
|
||||
Source.from(input).via(flow).grouped(10).runWith(Sink.<List<List<String>>> head(), materializer);
|
||||
final Object[] result = Await.result(future, Duration.create(1, TimeUnit.SECONDS)).toArray();
|
||||
|
|
@ -335,7 +335,7 @@ public class FlowTest extends StreamTest {
|
|||
final Source<String, BoxedUnit> in1 = Source.from(Arrays.asList("a", "b", "c"));
|
||||
final Source<String, BoxedUnit> in2 = Source.from(Arrays.asList("d", "e", "f"));
|
||||
|
||||
final Sink<String, Publisher<String>> publisher = Sink.publisher(false);
|
||||
final Sink<String, Publisher<String>> publisher = Sink.asPublisher(false);
|
||||
|
||||
final Source<String, BoxedUnit> source = Source.fromGraph(
|
||||
GraphDSL.create(new Function<GraphDSL.Builder<BoxedUnit>, SourceShape<String>>() {
|
||||
|
|
@ -350,7 +350,7 @@ public class FlowTest extends StreamTest {
|
|||
|
||||
// collecting
|
||||
final Publisher<String> pub = source.runWith(publisher, materializer);
|
||||
final Future<List<String>> all = Source.from(pub).grouped(100).runWith(Sink.<List<String>>head(), materializer);
|
||||
final Future<List<String>> all = Source.fromPublisher(pub).grouped(100).runWith(Sink.<List<String>>head(), materializer);
|
||||
|
||||
final List<String> result = Await.result(all, Duration.apply(200, TimeUnit.MILLISECONDS));
|
||||
assertEquals(new HashSet<Object>(Arrays.asList("a", "b", "c", "d", "e", "f")), new HashSet<String>(result));
|
||||
|
|
@ -443,7 +443,7 @@ public class FlowTest extends StreamTest {
|
|||
|
||||
assertEquals(Arrays.asList(1, 2, 3, 4, 5), result);
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void mustBeAbleToUseFlatMapMerge() throws Exception {
|
||||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
|
|
@ -558,7 +558,7 @@ public class FlowTest extends StreamTest {
|
|||
final TestPublisher.ManualProbe<Integer> publisherProbe = TestPublisher.manualProbe(true,system);
|
||||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
|
||||
final Source<Integer, ?> source = Source.from(publisherProbe);
|
||||
final Source<Integer, ?> source = Source.fromPublisher(publisherProbe);
|
||||
final Flow<Integer, Integer, ?> flow = Flow.of(Integer.class).map(
|
||||
new Function<Integer, Integer>() {
|
||||
public Integer apply(Integer elem) {
|
||||
|
|
|
|||
|
|
@ -39,7 +39,7 @@ public class SinkTest extends StreamTest {
|
|||
|
||||
@Test
|
||||
public void mustBeAbleToUseFanoutPublisher() throws Exception {
|
||||
final Sink<Object, Publisher<Object>> pubSink = Sink.publisher(true);
|
||||
final Sink<Object, Publisher<Object>> pubSink = Sink.asPublisher(true);
|
||||
@SuppressWarnings("unused")
|
||||
final Publisher<Object> publisher = Source.from(new ArrayList<Object>()).runWith(pubSink, materializer);
|
||||
}
|
||||
|
|
|
|||
|
|
@ -176,7 +176,7 @@ public class SourceTest extends StreamTest {
|
|||
})
|
||||
.grouped(10)
|
||||
.mergeSubstreams();
|
||||
|
||||
|
||||
final Future<List<List<String>>> future =
|
||||
source.grouped(10).runWith(Sink.<List<List<String>>> head(), materializer);
|
||||
final Object[] result = Await.result(future, Duration.create(1, TimeUnit.SECONDS)).toArray();
|
||||
|
|
@ -478,7 +478,7 @@ public class SourceTest extends StreamTest {
|
|||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
final Iterable<String> input = Arrays.asList("A", "B", "C");
|
||||
Future<String> future1 = Source.from(input).runWith(Sink.<String>head(), materializer);
|
||||
Future<String> future2 = Source.from(future1).runWith(Sink.<String>head(), materializer);
|
||||
Future<String> future2 = Source.fromFuture(future1).runWith(Sink.<String>head(), materializer);
|
||||
String result = Await.result(future2, probe.dilated(FiniteDuration.create(3, TimeUnit.SECONDS)));
|
||||
assertEquals("A", result);
|
||||
}
|
||||
|
|
@ -579,7 +579,7 @@ public class SourceTest extends StreamTest {
|
|||
final ManualProbe<Integer> publisherProbe = TestPublisher.manualProbe(true,system);
|
||||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
|
||||
final Source<Integer, ?> source = Source.from(publisherProbe).map(
|
||||
final Source<Integer, ?> source = Source.fromPublisher(publisherProbe).map(
|
||||
new Function<Integer, Integer>() {
|
||||
public Integer apply(Integer elem) {
|
||||
if (elem == 1) throw new RuntimeException("ex");
|
||||
|
|
|
|||
|
|
@ -345,9 +345,9 @@ class ActorPublisherSpec extends AkkaSpec(ActorPublisherSpec.config) with Implic
|
|||
val probe2 = TestProbe()
|
||||
|
||||
val senderRef1 = system.actorOf(senderProps)
|
||||
val source1 = Source(ActorPublisher[Int](senderRef1))
|
||||
val source1 = Source.fromPublisher(ActorPublisher[Int](senderRef1))
|
||||
|
||||
val sink1 = Sink(ActorSubscriber[String](system.actorOf(receiverProps(probe1.ref))))
|
||||
val sink1 = Sink.fromSubscriber(ActorSubscriber[String](system.actorOf(receiverProps(probe1.ref))))
|
||||
val sink2: Sink[String, ActorRef] = Sink.actorSubscriber(receiverProps(probe2.ref))
|
||||
|
||||
val senderRef2 = RunnableGraph.fromGraph(GraphDSL.create(Source.actorPublisher[Int](senderProps)) { implicit b ⇒
|
||||
|
|
@ -420,7 +420,7 @@ class ActorPublisherSpec extends AkkaSpec(ActorPublisherSpec.config) with Implic
|
|||
implicit val materializer = ActorMaterializer(
|
||||
ActorMaterializerSettings(system).withDispatcher("my-dispatcher1"))
|
||||
val s = TestSubscriber.manualProbe[String]()
|
||||
val ref = Source.actorPublisher(testPublisherProps(testActor, useTestDispatcher = false)).to(Sink(s)).run()
|
||||
val ref = Source.actorPublisher(testPublisherProps(testActor, useTestDispatcher = false)).to(Sink.fromSubscriber(s)).run()
|
||||
ref ! ThreadName
|
||||
expectMsgType[String] should include("my-dispatcher1")
|
||||
}
|
||||
|
|
@ -430,7 +430,7 @@ class ActorPublisherSpec extends AkkaSpec(ActorPublisherSpec.config) with Implic
|
|||
val s = TestSubscriber.manualProbe[String]()
|
||||
val ref = Source.actorPublisher(testPublisherProps(testActor, useTestDispatcher = false))
|
||||
.withAttributes(ActorAttributes.dispatcher("my-dispatcher1"))
|
||||
.to(Sink(s)).run()
|
||||
.to(Sink.fromSubscriber(s)).run()
|
||||
ref ! ThreadName
|
||||
expectMsgType[String] should include("my-dispatcher1")
|
||||
}
|
||||
|
|
@ -440,7 +440,7 @@ class ActorPublisherSpec extends AkkaSpec(ActorPublisherSpec.config) with Implic
|
|||
val s = TestSubscriber.manualProbe[String]()
|
||||
val ref = Source.actorPublisher(testPublisherProps(testActor, useTestDispatcher = false).withDispatcher("my-dispatcher1"))
|
||||
.withAttributes(ActorAttributes.dispatcher("my-dispatcher2"))
|
||||
.to(Sink(s)).run()
|
||||
.to(Sink.fromSubscriber(s)).run()
|
||||
ref ! ThreadName
|
||||
expectMsgType[String] should include("my-dispatcher1")
|
||||
}
|
||||
|
|
|
|||
|
|
@ -125,7 +125,7 @@ class ActorSubscriberSpec extends AkkaSpec with ImplicitSender {
|
|||
|
||||
"signal error" in {
|
||||
val e = new RuntimeException("simulated") with NoStackTrace
|
||||
val ref = Source(() ⇒ throw e).runWith(Sink.actorSubscriber(manualSubscriberProps(testActor)))
|
||||
val ref = Source.fromIterator(() ⇒ throw e).runWith(Sink.actorSubscriber(manualSubscriberProps(testActor)))
|
||||
ref ! "ready"
|
||||
expectMsg(OnError(e))
|
||||
}
|
||||
|
|
@ -133,7 +133,7 @@ class ActorSubscriberSpec extends AkkaSpec with ImplicitSender {
|
|||
"remember requested after restart" in {
|
||||
// creating actor with default supervision, because stream supervisor default strategy is to stop
|
||||
val ref = system.actorOf(manualSubscriberProps(testActor))
|
||||
Source(1 to 7).runWith(Sink(ActorSubscriber[Int](ref)))
|
||||
Source(1 to 7).runWith(Sink.fromSubscriber(ActorSubscriber[Int](ref)))
|
||||
ref ! "ready"
|
||||
expectMsg(OnNext(1))
|
||||
expectMsg(OnNext(2))
|
||||
|
|
|
|||
|
|
@ -79,7 +79,7 @@ class FlowTimedSpec extends AkkaSpec with ScriptedTest {
|
|||
val flow: Flow[Int, Long, _] = Flow[Int].map(_.toLong).timedIntervalBetween(in ⇒ in % 2 == 1, d ⇒ probe.ref ! d)
|
||||
|
||||
val c1 = TestSubscriber.manualProbe[Long]()
|
||||
Source(List(1, 2, 3)).via(flow).runWith(Sink(c1))
|
||||
Source(List(1, 2, 3)).via(flow).runWith(Sink.fromSubscriber(c1))
|
||||
|
||||
val s = c1.expectSubscription()
|
||||
s.request(100)
|
||||
|
|
@ -104,12 +104,12 @@ class FlowTimedSpec extends AkkaSpec with ScriptedTest {
|
|||
map(_.toString), duration ⇒ probe.ref ! duration).
|
||||
map { s: String ⇒ s + "!" }
|
||||
|
||||
val (flowIn: Subscriber[Int], flowOut: Publisher[String]) = flow.runWith(Source.subscriber[Int], Sink.publisher[String](false))
|
||||
val (flowIn: Subscriber[Int], flowOut: Publisher[String]) = flow.runWith(Source.asSubscriber[Int], Sink.asPublisher[String](false))
|
||||
|
||||
val c1 = TestSubscriber.manualProbe[String]()
|
||||
val c2 = flowOut.subscribe(c1)
|
||||
|
||||
val p = Source(0 to 100).runWith(Sink.publisher(false))
|
||||
val p = Source(0 to 100).runWith(Sink.asPublisher(false))
|
||||
p.subscribe(flowIn)
|
||||
|
||||
val s = c1.expectSubscription()
|
||||
|
|
|
|||
|
|
@ -41,7 +41,7 @@ class TimeoutsSpec extends AkkaSpec {
|
|||
val downstreamProbe = TestSubscriber.probe[Int]()
|
||||
Source.maybe[Int]
|
||||
.initialTimeout(1.second)
|
||||
.runWith(Sink(downstreamProbe))
|
||||
.runWith(Sink.fromSubscriber(downstreamProbe))
|
||||
|
||||
downstreamProbe.expectSubscription()
|
||||
downstreamProbe.expectNoMsg(500.millis)
|
||||
|
|
@ -73,9 +73,9 @@ class TimeoutsSpec extends AkkaSpec {
|
|||
"fail if not completed until timeout" in assertAllStagesStopped {
|
||||
val upstreamProbe = TestPublisher.probe[Int]()
|
||||
val downstreamProbe = TestSubscriber.probe[Int]()
|
||||
Source(upstreamProbe)
|
||||
Source.fromPublisher(upstreamProbe)
|
||||
.completionTimeout(2.seconds)
|
||||
.runWith(Sink(downstreamProbe))
|
||||
.runWith(Sink.fromSubscriber(downstreamProbe))
|
||||
|
||||
upstreamProbe.sendNext(1)
|
||||
downstreamProbe.requestNext(1)
|
||||
|
|
@ -112,9 +112,9 @@ class TimeoutsSpec extends AkkaSpec {
|
|||
"fail if time between elements is too large" in assertAllStagesStopped {
|
||||
val upstreamProbe = TestPublisher.probe[Int]()
|
||||
val downstreamProbe = TestSubscriber.probe[Int]()
|
||||
Source(upstreamProbe)
|
||||
Source.fromPublisher(upstreamProbe)
|
||||
.idleTimeout(1.seconds)
|
||||
.runWith(Sink(downstreamProbe))
|
||||
.runWith(Sink.fromSubscriber(downstreamProbe))
|
||||
|
||||
// Two seconds in overall, but won't timeout until time between elements is large enough
|
||||
// (i.e. this works differently from completionTimeout)
|
||||
|
|
@ -144,8 +144,8 @@ class TimeoutsSpec extends AkkaSpec {
|
|||
val upstreamWriter = TestPublisher.probe[Int]()
|
||||
val downstreamWriter = TestPublisher.probe[String]()
|
||||
|
||||
val upstream = Flow.fromSinkAndSourceMat(Sink.ignore, Source(upstreamWriter))(Keep.left)
|
||||
val downstream = Flow.fromSinkAndSourceMat(Sink.ignore, Source(downstreamWriter))(Keep.left)
|
||||
val upstream = Flow.fromSinkAndSourceMat(Sink.ignore, Source.fromPublisher(upstreamWriter))(Keep.left)
|
||||
val downstream = Flow.fromSinkAndSourceMat(Sink.ignore, Source.fromPublisher(downstreamWriter))(Keep.left)
|
||||
|
||||
val assembly: RunnableGraph[(Future[Unit], Future[Unit])] = upstream
|
||||
.joinMat(BidiFlow.bidirectionalIdleTimeout[Int, String](2.seconds))(Keep.left)
|
||||
|
|
@ -177,10 +177,10 @@ class TimeoutsSpec extends AkkaSpec {
|
|||
RunnableGraph.fromGraph(GraphDSL.create() { implicit b ⇒
|
||||
import GraphDSL.Implicits._
|
||||
val timeoutStage = b.add(BidiFlow.bidirectionalIdleTimeout[String, Int](2.seconds))
|
||||
Source(upWrite) ~> timeoutStage.in1;
|
||||
timeoutStage.out1 ~> Sink(downRead)
|
||||
Sink(upRead) <~ timeoutStage.out2;
|
||||
timeoutStage.in2 <~ Source(downWrite)
|
||||
Source.fromPublisher(upWrite) ~> timeoutStage.in1;
|
||||
timeoutStage.out1 ~> Sink.fromSubscriber(downRead)
|
||||
Sink.fromSubscriber(upRead) <~ timeoutStage.out2;
|
||||
timeoutStage.in2 <~ Source.fromPublisher(downWrite)
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
|
|
@ -225,10 +225,10 @@ class TimeoutsSpec extends AkkaSpec {
|
|||
RunnableGraph.fromGraph(GraphDSL.create() { implicit b ⇒
|
||||
import GraphDSL.Implicits._
|
||||
val timeoutStage = b.add(BidiFlow.bidirectionalIdleTimeout[String, Int](2.seconds))
|
||||
Source(upWrite) ~> timeoutStage.in1;
|
||||
timeoutStage.out1 ~> Sink(downRead)
|
||||
Sink(upRead) <~ timeoutStage.out2;
|
||||
timeoutStage.in2 <~ Source(downWrite)
|
||||
Source.fromPublisher(upWrite) ~> timeoutStage.in1;
|
||||
timeoutStage.out1 ~> Sink.fromSubscriber(downRead)
|
||||
Sink.fromSubscriber(upRead) <~ timeoutStage.out2;
|
||||
timeoutStage.in2 <~ Source.fromPublisher(downWrite)
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
|
|
|
|||
|
|
@ -98,7 +98,7 @@ class FileSinkSpec extends AkkaSpec(UnboundedMailboxConfig) {
|
|||
implicit val timeout = Timeout(3.seconds)
|
||||
|
||||
try {
|
||||
Source(() ⇒ Iterator.continually(TestByteStrings.head)).runWith(Sink.file(f))(materializer)
|
||||
Source.fromIterator(() ⇒ Iterator.continually(TestByteStrings.head)).runWith(Sink.file(f))(materializer)
|
||||
|
||||
materializer.asInstanceOf[ActorMaterializerImpl].supervisor.tell(StreamSupervisor.GetChildren, testActor)
|
||||
val ref = expectMsgType[Children].children.find(_.path.toString contains "fileSource").get
|
||||
|
|
@ -116,7 +116,7 @@ class FileSinkSpec extends AkkaSpec(UnboundedMailboxConfig) {
|
|||
implicit val timeout = Timeout(3.seconds)
|
||||
|
||||
try {
|
||||
Source(() ⇒ Iterator.continually(TestByteStrings.head))
|
||||
Source.fromIterator(() ⇒ Iterator.continually(TestByteStrings.head))
|
||||
.to(Sink.file(f))
|
||||
.withAttributes(ActorAttributes.dispatcher("akka.actor.default-dispatcher"))
|
||||
.run()(materializer)
|
||||
|
|
|
|||
|
|
@ -76,7 +76,7 @@ class FileSourceSpec extends AkkaSpec(UnboundedMailboxConfig) {
|
|||
|
||||
val p = Source.file(testFile, chunkSize)
|
||||
.withAttributes(bufferAttributes)
|
||||
.runWith(Sink.publisher(false))
|
||||
.runWith(Sink.asPublisher(false))
|
||||
val c = TestSubscriber.manualProbe[ByteString]()
|
||||
p.subscribe(c)
|
||||
val sub = c.expectSubscription()
|
||||
|
|
@ -113,7 +113,7 @@ class FileSourceSpec extends AkkaSpec(UnboundedMailboxConfig) {
|
|||
|
||||
val p = Source.file(testFile, chunkSize)
|
||||
.withAttributes(bufferAttributes)
|
||||
.runWith(Sink.publisher(false))
|
||||
.runWith(Sink.asPublisher(false))
|
||||
|
||||
val c = TestSubscriber.manualProbe[ByteString]()
|
||||
p.subscribe(c)
|
||||
|
|
@ -140,7 +140,7 @@ class FileSourceSpec extends AkkaSpec(UnboundedMailboxConfig) {
|
|||
}
|
||||
|
||||
"onError whent trying to read from file which does not exist" in assertAllStagesStopped {
|
||||
val p = Source.file(notExistingFile).runWith(Sink.publisher(false))
|
||||
val p = Source.file(notExistingFile).runWith(Sink.asPublisher(false))
|
||||
val c = TestSubscriber.manualProbe[ByteString]()
|
||||
p.subscribe(c)
|
||||
|
||||
|
|
|
|||
|
|
@ -31,7 +31,7 @@ class TcpSpec extends AkkaSpec("akka.stream.materializer.subscription-timeout.ti
|
|||
|
||||
val tcpReadProbe = new TcpReadProbe()
|
||||
val tcpWriteProbe = new TcpWriteProbe()
|
||||
Source(tcpWriteProbe.publisherProbe).via(Tcp().outgoingConnection(server.address)).to(Sink(tcpReadProbe.subscriberProbe)).run()
|
||||
Source.fromPublisher(tcpWriteProbe.publisherProbe).via(Tcp().outgoingConnection(server.address)).to(Sink.fromSubscriber(tcpReadProbe.subscriberProbe)).run()
|
||||
val serverConnection = server.waitAccept()
|
||||
|
||||
validateServerClientCommunication(testData, serverConnection, tcpReadProbe, tcpWriteProbe)
|
||||
|
|
@ -61,7 +61,7 @@ class TcpSpec extends AkkaSpec("akka.stream.materializer.subscription-timeout.ti
|
|||
|
||||
val idle = new TcpWriteProbe() // Just register an idle upstream
|
||||
val resultFuture =
|
||||
Source(idle.publisherProbe)
|
||||
Source.fromPublisher(idle.publisherProbe)
|
||||
.via(Tcp().outgoingConnection(server.address))
|
||||
.runFold(ByteString.empty)((acc, in) ⇒ acc ++ in)
|
||||
val serverConnection = server.waitAccept()
|
||||
|
|
@ -81,7 +81,7 @@ class TcpSpec extends AkkaSpec("akka.stream.materializer.subscription-timeout.ti
|
|||
|
||||
val tcpWriteProbe = new TcpWriteProbe()
|
||||
val tcpReadProbe = new TcpReadProbe()
|
||||
Source(tcpWriteProbe.publisherProbe).via(Tcp().outgoingConnection(server.address)).to(Sink(tcpReadProbe.subscriberProbe)).run()
|
||||
Source.fromPublisher(tcpWriteProbe.publisherProbe).via(Tcp().outgoingConnection(server.address)).to(Sink.fromSubscriber(tcpReadProbe.subscriberProbe)).run()
|
||||
val serverConnection = server.waitAccept()
|
||||
|
||||
// Client can still write
|
||||
|
|
@ -111,7 +111,7 @@ class TcpSpec extends AkkaSpec("akka.stream.materializer.subscription-timeout.ti
|
|||
|
||||
val tcpWriteProbe = new TcpWriteProbe()
|
||||
val tcpReadProbe = new TcpReadProbe()
|
||||
Source(tcpWriteProbe.publisherProbe).via(Tcp().outgoingConnection(server.address)).to(Sink(tcpReadProbe.subscriberProbe)).run()
|
||||
Source.fromPublisher(tcpWriteProbe.publisherProbe).via(Tcp().outgoingConnection(server.address)).to(Sink.fromSubscriber(tcpReadProbe.subscriberProbe)).run()
|
||||
val serverConnection = server.waitAccept()
|
||||
|
||||
// Server can still write
|
||||
|
|
@ -139,7 +139,7 @@ class TcpSpec extends AkkaSpec("akka.stream.materializer.subscription-timeout.ti
|
|||
|
||||
val tcpWriteProbe = new TcpWriteProbe()
|
||||
val tcpReadProbe = new TcpReadProbe()
|
||||
Source(tcpWriteProbe.publisherProbe).via(Tcp().outgoingConnection(server.address)).to(Sink(tcpReadProbe.subscriberProbe)).run()
|
||||
Source.fromPublisher(tcpWriteProbe.publisherProbe).via(Tcp().outgoingConnection(server.address)).to(Sink.fromSubscriber(tcpReadProbe.subscriberProbe)).run()
|
||||
val serverConnection = server.waitAccept()
|
||||
|
||||
// Server can still write
|
||||
|
|
@ -171,7 +171,7 @@ class TcpSpec extends AkkaSpec("akka.stream.materializer.subscription-timeout.ti
|
|||
|
||||
val tcpWriteProbe = new TcpWriteProbe()
|
||||
val tcpReadProbe = new TcpReadProbe()
|
||||
Source(tcpWriteProbe.publisherProbe).via(Tcp().outgoingConnection(server.address)).to(Sink(tcpReadProbe.subscriberProbe)).run()
|
||||
Source.fromPublisher(tcpWriteProbe.publisherProbe).via(Tcp().outgoingConnection(server.address)).to(Sink.fromSubscriber(tcpReadProbe.subscriberProbe)).run()
|
||||
val serverConnection = server.waitAccept()
|
||||
|
||||
// Client can still write
|
||||
|
|
@ -204,7 +204,7 @@ class TcpSpec extends AkkaSpec("akka.stream.materializer.subscription-timeout.ti
|
|||
|
||||
val tcpWriteProbe = new TcpWriteProbe()
|
||||
val tcpReadProbe = new TcpReadProbe()
|
||||
Source(tcpWriteProbe.publisherProbe).via(Tcp().outgoingConnection(server.address)).to(Sink(tcpReadProbe.subscriberProbe)).run()
|
||||
Source.fromPublisher(tcpWriteProbe.publisherProbe).via(Tcp().outgoingConnection(server.address)).to(Sink.fromSubscriber(tcpReadProbe.subscriberProbe)).run()
|
||||
val serverConnection = server.waitAccept()
|
||||
|
||||
// Server can still write
|
||||
|
|
@ -234,7 +234,7 @@ class TcpSpec extends AkkaSpec("akka.stream.materializer.subscription-timeout.ti
|
|||
val tcpWriteProbe = new TcpWriteProbe()
|
||||
val tcpReadProbe = new TcpReadProbe()
|
||||
|
||||
Source(tcpWriteProbe.publisherProbe).via(Tcp().outgoingConnection(server.address)).to(Sink(tcpReadProbe.subscriberProbe)).run()
|
||||
Source.fromPublisher(tcpWriteProbe.publisherProbe).via(Tcp().outgoingConnection(server.address)).to(Sink.fromSubscriber(tcpReadProbe.subscriberProbe)).run()
|
||||
val serverConnection = server.waitAccept()
|
||||
|
||||
// Server can still write
|
||||
|
|
@ -261,7 +261,7 @@ class TcpSpec extends AkkaSpec("akka.stream.materializer.subscription-timeout.ti
|
|||
val tcpWriteProbe = new TcpWriteProbe()
|
||||
val tcpReadProbe = new TcpReadProbe()
|
||||
|
||||
Source(tcpWriteProbe.publisherProbe).via(Tcp().outgoingConnection(server.address)).to(Sink(tcpReadProbe.subscriberProbe)).run()
|
||||
Source.fromPublisher(tcpWriteProbe.publisherProbe).via(Tcp().outgoingConnection(server.address)).to(Sink.fromSubscriber(tcpReadProbe.subscriberProbe)).run()
|
||||
val serverConnection = server.waitAccept()
|
||||
|
||||
// Server can still write
|
||||
|
|
@ -290,7 +290,7 @@ class TcpSpec extends AkkaSpec("akka.stream.materializer.subscription-timeout.ti
|
|||
val tcpWriteProbe = new TcpWriteProbe()
|
||||
val tcpReadProbe = new TcpReadProbe()
|
||||
|
||||
Source(tcpWriteProbe.publisherProbe).via(Tcp().outgoingConnection(server.address)).to(Sink(tcpReadProbe.subscriberProbe)).run()
|
||||
Source.fromPublisher(tcpWriteProbe.publisherProbe).via(Tcp().outgoingConnection(server.address)).to(Sink.fromSubscriber(tcpReadProbe.subscriberProbe)).run()
|
||||
val serverConnection = server.waitAccept()
|
||||
|
||||
serverConnection.abort()
|
||||
|
|
@ -312,14 +312,14 @@ class TcpSpec extends AkkaSpec("akka.stream.materializer.subscription-timeout.ti
|
|||
val outgoingConnection = Tcp().outgoingConnection(server.address)
|
||||
|
||||
val conn1F =
|
||||
Source(tcpWriteProbe1.publisherProbe)
|
||||
Source.fromPublisher(tcpWriteProbe1.publisherProbe)
|
||||
.viaMat(outgoingConnection)(Keep.right)
|
||||
.to(Sink(tcpReadProbe1.subscriberProbe)).run()
|
||||
.to(Sink.fromSubscriber(tcpReadProbe1.subscriberProbe)).run()
|
||||
val serverConnection1 = server.waitAccept()
|
||||
val conn2F =
|
||||
Source(tcpWriteProbe2.publisherProbe)
|
||||
Source.fromPublisher(tcpWriteProbe2.publisherProbe)
|
||||
.viaMat(outgoingConnection)(Keep.right)
|
||||
.to(Sink(tcpReadProbe2.subscriberProbe))
|
||||
.to(Sink.fromSubscriber(tcpReadProbe2.subscriberProbe))
|
||||
.run()
|
||||
val serverConnection2 = server.waitAccept()
|
||||
|
||||
|
|
@ -465,16 +465,16 @@ class TcpSpec extends AkkaSpec("akka.stream.materializer.subscription-timeout.ti
|
|||
val probe1 = TestSubscriber.manualProbe[Tcp.IncomingConnection]()
|
||||
val bind = Tcp(system).bind(address.getHostName, address.getPort) // TODO getHostString in Java7
|
||||
// Bind succeeded, we have a local address
|
||||
val binding1 = Await.result(bind.to(Sink(probe1)).run(), 3.second)
|
||||
val binding1 = Await.result(bind.to(Sink.fromSubscriber(probe1)).run(), 3.second)
|
||||
|
||||
probe1.expectSubscription()
|
||||
|
||||
val probe2 = TestSubscriber.manualProbe[Tcp.IncomingConnection]()
|
||||
val binding2F = bind.to(Sink(probe2)).run()
|
||||
val binding2F = bind.to(Sink.fromSubscriber(probe2)).run()
|
||||
probe2.expectSubscriptionAndError(BindFailedException)
|
||||
|
||||
val probe3 = TestSubscriber.manualProbe[Tcp.IncomingConnection]()
|
||||
val binding3F = bind.to(Sink(probe3)).run()
|
||||
val binding3F = bind.to(Sink.fromSubscriber(probe3)).run()
|
||||
probe3.expectSubscriptionAndError()
|
||||
|
||||
a[BindFailedException] shouldBe thrownBy { Await.result(binding2F, 1.second) }
|
||||
|
|
@ -486,7 +486,7 @@ class TcpSpec extends AkkaSpec("akka.stream.materializer.subscription-timeout.ti
|
|||
|
||||
val probe4 = TestSubscriber.manualProbe[Tcp.IncomingConnection]()
|
||||
// Bind succeeded, we have a local address
|
||||
val binding4 = Await.result(bind.to(Sink(probe4)).run(), 3.second)
|
||||
val binding4 = Await.result(bind.to(Sink.fromSubscriber(probe4)).run(), 3.second)
|
||||
probe4.expectSubscription()
|
||||
|
||||
// clean up
|
||||
|
|
|
|||
|
|
@ -399,7 +399,7 @@ class TlsSpec extends AkkaSpec("akka.loglevel=INFO\nakka.actor.debug.receive=off
|
|||
"reliably cancel subscriptions when TransportIn fails early" in assertAllStagesStopped {
|
||||
val ex = new Exception("hello")
|
||||
val (sub, out1, out2) =
|
||||
RunnableGraph.fromGraph(GraphDSL.create(Source.subscriber[SslTlsOutbound], Sink.head[ByteString], Sink.head[SslTlsInbound])((_, _, _)) { implicit b ⇒
|
||||
RunnableGraph.fromGraph(GraphDSL.create(Source.asSubscriber[SslTlsOutbound], Sink.head[ByteString], Sink.head[SslTlsInbound])((_, _, _)) { implicit b ⇒
|
||||
(s, o1, o2) ⇒
|
||||
val tls = b.add(clientTls(EagerClose))
|
||||
s ~> tls.in1; tls.out1 ~> o1
|
||||
|
|
@ -417,7 +417,7 @@ class TlsSpec extends AkkaSpec("akka.loglevel=INFO\nakka.actor.debug.receive=off
|
|||
"reliably cancel subscriptions when UserIn fails early" in assertAllStagesStopped {
|
||||
val ex = new Exception("hello")
|
||||
val (sub, out1, out2) =
|
||||
RunnableGraph.fromGraph(GraphDSL.create(Source.subscriber[ByteString], Sink.head[ByteString], Sink.head[SslTlsInbound])((_, _, _)) { implicit b ⇒
|
||||
RunnableGraph.fromGraph(GraphDSL.create(Source.asSubscriber[ByteString], Sink.head[ByteString], Sink.head[SslTlsInbound])((_, _, _)) { implicit b ⇒
|
||||
(s, o1, o2) ⇒
|
||||
val tls = b.add(clientTls(EagerClose))
|
||||
Source.failed[SslTlsOutbound](ex) ~> tls.in1; tls.out1 ~> o1
|
||||
|
|
|
|||
|
|
@ -34,7 +34,7 @@ class AcknowledgeSinkSpec extends AkkaSpec {
|
|||
|
||||
"allow to have only one future waiting for result in each point of time" in assertAllStagesStopped {
|
||||
val probe = TestPublisher.manualProbe[Int]()
|
||||
val queue = Source(probe).runWith(Sink.queue(3))
|
||||
val queue = Source.fromPublisher(probe).runWith(Sink.queue(3))
|
||||
val sub = probe.expectSubscription()
|
||||
val future = queue.pull()
|
||||
val future2 = queue.pull()
|
||||
|
|
@ -49,7 +49,7 @@ class AcknowledgeSinkSpec extends AkkaSpec {
|
|||
|
||||
"wait for next element from upstream" in assertAllStagesStopped {
|
||||
val probe = TestPublisher.manualProbe[Int]()
|
||||
val queue = Source(probe).runWith(Sink.queue(3))
|
||||
val queue = Source.fromPublisher(probe).runWith(Sink.queue(3))
|
||||
val sub = probe.expectSubscription()
|
||||
|
||||
queue.pull().pipeTo(testActor)
|
||||
|
|
@ -62,7 +62,7 @@ class AcknowledgeSinkSpec extends AkkaSpec {
|
|||
|
||||
"fail future on stream failure" in assertAllStagesStopped {
|
||||
val probe = TestPublisher.manualProbe[Int]()
|
||||
val queue = Source(probe).runWith(Sink.queue(3))
|
||||
val queue = Source.fromPublisher(probe).runWith(Sink.queue(3))
|
||||
val sub = probe.expectSubscription()
|
||||
|
||||
queue.pull().pipeTo(testActor)
|
||||
|
|
@ -74,7 +74,7 @@ class AcknowledgeSinkSpec extends AkkaSpec {
|
|||
|
||||
"fail future when stream failed" in assertAllStagesStopped {
|
||||
val probe = TestPublisher.manualProbe[Int]()
|
||||
val queue = Source(probe).runWith(Sink.queue(3, 100.millis))
|
||||
val queue = Source.fromPublisher(probe).runWith(Sink.queue(3, 100.millis))
|
||||
val sub = probe.expectSubscription()
|
||||
sub.sendError(ex) // potential race condition
|
||||
|
||||
|
|
@ -83,7 +83,7 @@ class AcknowledgeSinkSpec extends AkkaSpec {
|
|||
|
||||
"timeout future when stream cannot provide data" in assertAllStagesStopped {
|
||||
val probe = TestPublisher.manualProbe[Int]()
|
||||
val queue = Source(probe).runWith(Sink.queue(3))
|
||||
val queue = Source.fromPublisher(probe).runWith(Sink.queue(3))
|
||||
val sub = probe.expectSubscription()
|
||||
|
||||
queue.pull().pipeTo(testActor)
|
||||
|
|
@ -96,7 +96,7 @@ class AcknowledgeSinkSpec extends AkkaSpec {
|
|||
|
||||
"work when buffer is 0" in assertAllStagesStopped {
|
||||
val probe = TestPublisher.manualProbe[Int]()
|
||||
val queue = Source(probe).runWith(Sink.queue(0))
|
||||
val queue = Source.fromPublisher(probe).runWith(Sink.queue(0))
|
||||
val sub = probe.expectSubscription()
|
||||
sub.sendNext(1)
|
||||
|
||||
|
|
|
|||
|
|
@ -22,7 +22,7 @@ class AcknowledgeSourceSpec extends AkkaSpec {
|
|||
|
||||
"emit received messages to the stream" in {
|
||||
val s = TestSubscriber.manualProbe[Int]()
|
||||
val queue = Source.queue(10, OverflowStrategy.fail).to(Sink(s)).run()
|
||||
val queue = Source.queue(10, OverflowStrategy.fail).to(Sink.fromSubscriber(s)).run()
|
||||
val sub = s.expectSubscription
|
||||
sub.request(2)
|
||||
assertSuccess(true, queue.offer(1))
|
||||
|
|
@ -35,7 +35,7 @@ class AcknowledgeSourceSpec extends AkkaSpec {
|
|||
|
||||
"buffer when needed" in {
|
||||
val s = TestSubscriber.manualProbe[Int]()
|
||||
val queue = Source.queue(100, OverflowStrategy.dropHead).to(Sink(s)).run()
|
||||
val queue = Source.queue(100, OverflowStrategy.dropHead).to(Sink.fromSubscriber(s)).run()
|
||||
val sub = s.expectSubscription
|
||||
for (n ← 1 to 20) assertSuccess(true, queue.offer(n))
|
||||
sub.request(10)
|
||||
|
|
@ -51,7 +51,7 @@ class AcknowledgeSourceSpec extends AkkaSpec {
|
|||
|
||||
"not fail when 0 buffer space and demand is signalled" in assertAllStagesStopped {
|
||||
val s = TestSubscriber.manualProbe[Int]()
|
||||
val queue = Source.queue(0, OverflowStrategy.dropHead).to(Sink(s)).run()
|
||||
val queue = Source.queue(0, OverflowStrategy.dropHead).to(Sink.fromSubscriber(s)).run()
|
||||
val sub = s.expectSubscription
|
||||
sub.request(1)
|
||||
assertSuccess(true, queue.offer(1))
|
||||
|
|
@ -61,7 +61,7 @@ class AcknowledgeSourceSpec extends AkkaSpec {
|
|||
|
||||
"return false when can reject element to buffer" in assertAllStagesStopped {
|
||||
val s = TestSubscriber.manualProbe[Int]()
|
||||
val queue = Source.queue(1, OverflowStrategy.dropNew).to(Sink(s)).run()
|
||||
val queue = Source.queue(1, OverflowStrategy.dropNew).to(Sink.fromSubscriber(s)).run()
|
||||
val sub = s.expectSubscription
|
||||
assertSuccess(true, queue.offer(1))
|
||||
assertSuccess(false, queue.offer(2))
|
||||
|
|
@ -72,7 +72,7 @@ class AcknowledgeSourceSpec extends AkkaSpec {
|
|||
|
||||
"wait when buffer is full and backpressure is on" in assertAllStagesStopped {
|
||||
val s = TestSubscriber.manualProbe[Int]()
|
||||
val queue = Source.queue(2, OverflowStrategy.backpressure).to(Sink(s)).run()
|
||||
val queue = Source.queue(2, OverflowStrategy.backpressure).to(Sink.fromSubscriber(s)).run()
|
||||
val sub = s.expectSubscription
|
||||
assertSuccess(true, queue.offer(1))
|
||||
|
||||
|
|
|
|||
|
|
@ -19,7 +19,7 @@ class ActorRefSourceSpec extends AkkaSpec {
|
|||
|
||||
"emit received messages to the stream" in {
|
||||
val s = TestSubscriber.manualProbe[Int]()
|
||||
val ref = Source.actorRef(10, OverflowStrategy.fail).to(Sink(s)).run()
|
||||
val ref = Source.actorRef(10, OverflowStrategy.fail).to(Sink.fromSubscriber(s)).run()
|
||||
val sub = s.expectSubscription
|
||||
sub.request(2)
|
||||
ref ! 1
|
||||
|
|
@ -32,7 +32,7 @@ class ActorRefSourceSpec extends AkkaSpec {
|
|||
|
||||
"buffer when needed" in {
|
||||
val s = TestSubscriber.manualProbe[Int]()
|
||||
val ref = Source.actorRef(100, OverflowStrategy.dropHead).to(Sink(s)).run()
|
||||
val ref = Source.actorRef(100, OverflowStrategy.dropHead).to(Sink.fromSubscriber(s)).run()
|
||||
val sub = s.expectSubscription
|
||||
for (n ← 1 to 20) ref ! n
|
||||
sub.request(10)
|
||||
|
|
@ -61,7 +61,7 @@ class ActorRefSourceSpec extends AkkaSpec {
|
|||
|
||||
"terminate when the stream is cancelled" in assertAllStagesStopped {
|
||||
val s = TestSubscriber.manualProbe[Int]()
|
||||
val ref = Source.actorRef(0, OverflowStrategy.fail).to(Sink(s)).run()
|
||||
val ref = Source.actorRef(0, OverflowStrategy.fail).to(Sink.fromSubscriber(s)).run()
|
||||
watch(ref)
|
||||
val sub = s.expectSubscription
|
||||
sub.cancel()
|
||||
|
|
@ -70,7 +70,7 @@ class ActorRefSourceSpec extends AkkaSpec {
|
|||
|
||||
"not fail when 0 buffer space and demand is signalled" in assertAllStagesStopped {
|
||||
val s = TestSubscriber.manualProbe[Int]()
|
||||
val ref = Source.actorRef(0, OverflowStrategy.dropHead).to(Sink(s)).run()
|
||||
val ref = Source.actorRef(0, OverflowStrategy.dropHead).to(Sink.fromSubscriber(s)).run()
|
||||
watch(ref)
|
||||
val sub = s.expectSubscription
|
||||
sub.request(100)
|
||||
|
|
@ -80,7 +80,7 @@ class ActorRefSourceSpec extends AkkaSpec {
|
|||
|
||||
"complete the stream immediatly when receiving PoisonPill" in assertAllStagesStopped {
|
||||
val s = TestSubscriber.manualProbe[Int]()
|
||||
val ref = Source.actorRef(10, OverflowStrategy.fail).to(Sink(s)).run()
|
||||
val ref = Source.actorRef(10, OverflowStrategy.fail).to(Sink.fromSubscriber(s)).run()
|
||||
val sub = s.expectSubscription
|
||||
ref ! PoisonPill
|
||||
s.expectComplete()
|
||||
|
|
@ -88,7 +88,7 @@ class ActorRefSourceSpec extends AkkaSpec {
|
|||
|
||||
"signal buffered elements and complete the stream after receiving Status.Success" in assertAllStagesStopped {
|
||||
val s = TestSubscriber.manualProbe[Int]()
|
||||
val ref = Source.actorRef(3, OverflowStrategy.fail).to(Sink(s)).run()
|
||||
val ref = Source.actorRef(3, OverflowStrategy.fail).to(Sink.fromSubscriber(s)).run()
|
||||
val sub = s.expectSubscription
|
||||
ref ! 1
|
||||
ref ! 2
|
||||
|
|
@ -101,7 +101,7 @@ class ActorRefSourceSpec extends AkkaSpec {
|
|||
|
||||
"not buffer elements after receiving Status.Success" in assertAllStagesStopped {
|
||||
val s = TestSubscriber.manualProbe[Int]()
|
||||
val ref = Source.actorRef(3, OverflowStrategy.dropBuffer).to(Sink(s)).run()
|
||||
val ref = Source.actorRef(3, OverflowStrategy.dropBuffer).to(Sink.fromSubscriber(s)).run()
|
||||
val sub = s.expectSubscription
|
||||
ref ! 1
|
||||
ref ! 2
|
||||
|
|
@ -117,7 +117,7 @@ class ActorRefSourceSpec extends AkkaSpec {
|
|||
|
||||
"after receiving Status.Success, allow for earlier completion with PoisonPill" in assertAllStagesStopped {
|
||||
val s = TestSubscriber.manualProbe[Int]()
|
||||
val ref = Source.actorRef(3, OverflowStrategy.dropBuffer).to(Sink(s)).run()
|
||||
val ref = Source.actorRef(3, OverflowStrategy.dropBuffer).to(Sink.fromSubscriber(s)).run()
|
||||
val sub = s.expectSubscription
|
||||
ref ! 1
|
||||
ref ! 2
|
||||
|
|
@ -131,7 +131,7 @@ class ActorRefSourceSpec extends AkkaSpec {
|
|||
|
||||
"fail the stream when receiving Status.Failure" in assertAllStagesStopped {
|
||||
val s = TestSubscriber.manualProbe[Int]()
|
||||
val ref = Source.actorRef(10, OverflowStrategy.fail).to(Sink(s)).run()
|
||||
val ref = Source.actorRef(10, OverflowStrategy.fail).to(Sink.fromSubscriber(s)).run()
|
||||
val sub = s.expectSubscription
|
||||
val exc = TE("testfailure")
|
||||
ref ! Status.Failure(exc)
|
||||
|
|
|
|||
|
|
@ -18,11 +18,11 @@ class FlowAppendSpec extends AkkaSpec with River {
|
|||
"Flow" should {
|
||||
"append Flow" in riverOf[String] { subscriber ⇒
|
||||
val flow = Flow[Int].via(otherFlow)
|
||||
Source(elements).via(flow).to(Sink(subscriber)).run()
|
||||
Source(elements).via(flow).to(Sink.fromSubscriber(subscriber)).run()
|
||||
}
|
||||
|
||||
"append Sink" in riverOf[String] { subscriber ⇒
|
||||
val sink = Flow[Int].to(otherFlow.to(Sink(subscriber)))
|
||||
val sink = Flow[Int].to(otherFlow.to(Sink.fromSubscriber(subscriber)))
|
||||
Source(elements).to(sink).run()
|
||||
}
|
||||
}
|
||||
|
|
@ -31,12 +31,12 @@ class FlowAppendSpec extends AkkaSpec with River {
|
|||
"append Flow" in riverOf[String] { subscriber ⇒
|
||||
Source(elements)
|
||||
.via(otherFlow)
|
||||
.to(Sink(subscriber)).run()
|
||||
.to(Sink.fromSubscriber(subscriber)).run()
|
||||
}
|
||||
|
||||
"append Sink" in riverOf[String] { subscriber ⇒
|
||||
Source(elements)
|
||||
.to(otherFlow.to(Sink(subscriber)))
|
||||
.to(otherFlow.to(Sink.fromSubscriber(subscriber)))
|
||||
.run()
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -51,7 +51,7 @@ class FlowBufferSpec extends AkkaSpec {
|
|||
val publisher = TestPublisher.probe[Int]()
|
||||
val subscriber = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
Source(publisher).buffer(100, overflowStrategy = OverflowStrategy.backpressure).to(Sink(subscriber)).run()
|
||||
Source.fromPublisher(publisher).buffer(100, overflowStrategy = OverflowStrategy.backpressure).to(Sink.fromSubscriber(subscriber)).run()
|
||||
val sub = subscriber.expectSubscription()
|
||||
|
||||
// Fill up buffer
|
||||
|
|
@ -69,7 +69,7 @@ class FlowBufferSpec extends AkkaSpec {
|
|||
val publisher = TestPublisher.probe[Int]()
|
||||
val subscriber = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
Source(publisher).buffer(100, overflowStrategy = OverflowStrategy.dropHead).to(Sink(subscriber)).run()
|
||||
Source.fromPublisher(publisher).buffer(100, overflowStrategy = OverflowStrategy.dropHead).to(Sink.fromSubscriber(subscriber)).run()
|
||||
val sub = subscriber.expectSubscription()
|
||||
|
||||
// Fill up buffer
|
||||
|
|
@ -98,7 +98,7 @@ class FlowBufferSpec extends AkkaSpec {
|
|||
val publisher = TestPublisher.probe[Int]()
|
||||
val subscriber = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
Source(publisher).buffer(100, overflowStrategy = OverflowStrategy.dropTail).to(Sink(subscriber)).run()
|
||||
Source.fromPublisher(publisher).buffer(100, overflowStrategy = OverflowStrategy.dropTail).to(Sink.fromSubscriber(subscriber)).run()
|
||||
val sub = subscriber.expectSubscription()
|
||||
|
||||
// Fill up buffer
|
||||
|
|
@ -130,7 +130,7 @@ class FlowBufferSpec extends AkkaSpec {
|
|||
val publisher = TestPublisher.probe[Int]()
|
||||
val subscriber = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
Source(publisher).buffer(100, overflowStrategy = OverflowStrategy.dropBuffer).to(Sink(subscriber)).run()
|
||||
Source.fromPublisher(publisher).buffer(100, overflowStrategy = OverflowStrategy.dropBuffer).to(Sink.fromSubscriber(subscriber)).run()
|
||||
val sub = subscriber.expectSubscription()
|
||||
|
||||
// Fill up buffer
|
||||
|
|
@ -184,7 +184,7 @@ class FlowBufferSpec extends AkkaSpec {
|
|||
val publisher = TestPublisher.probe[Int]()
|
||||
val subscriber = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
Source(publisher).buffer(100, overflowStrategy = OverflowStrategy.fail).to(Sink(subscriber)).run()
|
||||
Source.fromPublisher(publisher).buffer(100, overflowStrategy = OverflowStrategy.fail).to(Sink.fromSubscriber(subscriber)).run()
|
||||
val sub = subscriber.expectSubscription()
|
||||
|
||||
// Fill up buffer
|
||||
|
|
@ -211,7 +211,7 @@ class FlowBufferSpec extends AkkaSpec {
|
|||
val publisher = TestPublisher.probe[Int]()
|
||||
val subscriber = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
Source(publisher).buffer(1, overflowStrategy = strategy).to(Sink(subscriber)).run()
|
||||
Source.fromPublisher(publisher).buffer(1, overflowStrategy = strategy).to(Sink.fromSubscriber(subscriber)).run()
|
||||
val sub = subscriber.expectSubscription()
|
||||
|
||||
// Fill up buffer
|
||||
|
|
|
|||
|
|
@ -18,7 +18,7 @@ class FlowCompileSpec extends AkkaSpec {
|
|||
val strSeq = Source(Seq("a", "b", "c"))
|
||||
|
||||
import scala.concurrent.ExecutionContext.Implicits.global
|
||||
val intFut = Source(Future { 3 })
|
||||
val intFut = Source.fromFuture(Future { 3 })
|
||||
implicit val materializer = ActorMaterializer(ActorMaterializerSettings(system))
|
||||
|
||||
"Flow" should {
|
||||
|
|
@ -41,15 +41,15 @@ class FlowCompileSpec extends AkkaSpec {
|
|||
val closedSource: Source[Int, _] = intSeq.via(open3)
|
||||
"closedSource.run()" shouldNot compile
|
||||
|
||||
val closedSink: Sink[Int, _] = open3.to(Sink.publisher[Int](false))
|
||||
val closedSink: Sink[Int, _] = open3.to(Sink.asPublisher[Int](false))
|
||||
"closedSink.run()" shouldNot compile
|
||||
|
||||
closedSource.to(Sink.publisher[Int](false)).run()
|
||||
closedSource.to(Sink.asPublisher[Int](false)).run()
|
||||
intSeq.to(closedSink).run()
|
||||
}
|
||||
"append Sink" in {
|
||||
val open: Flow[Int, String, _] = Flow[Int].map(_.toString)
|
||||
val closedSink: Sink[String, _] = Flow[String].map(_.hashCode).to(Sink.publisher[Int](false))
|
||||
val closedSink: Sink[String, _] = Flow[String].map(_.hashCode).to(Sink.asPublisher[Int](false))
|
||||
val appended: Sink[Int, _] = open.to(closedSink)
|
||||
"appended.run()" shouldNot compile
|
||||
"appended.connect(Sink.head[Int])" shouldNot compile
|
||||
|
|
@ -61,13 +61,13 @@ class FlowCompileSpec extends AkkaSpec {
|
|||
val closedSource2: Source[String, _] = closedSource.via(open)
|
||||
"closedSource2.run()" shouldNot compile
|
||||
"strSeq.connect(closedSource2)" shouldNot compile
|
||||
closedSource2.to(Sink.publisher[String](false)).run
|
||||
closedSource2.to(Sink.asPublisher[String](false)).run
|
||||
}
|
||||
}
|
||||
|
||||
"Sink" should {
|
||||
val openSource: Sink[Int, _] =
|
||||
Flow[Int].map(_.toString).to(Sink.publisher[String](false))
|
||||
Flow[Int].map(_.toString).to(Sink.asPublisher[String](false))
|
||||
"accept Source" in {
|
||||
intSeq.to(openSource)
|
||||
}
|
||||
|
|
@ -83,7 +83,7 @@ class FlowCompileSpec extends AkkaSpec {
|
|||
val openSource: Source[String, _] =
|
||||
Source(Seq(1, 2, 3)).map(_.toString)
|
||||
"accept Sink" in {
|
||||
openSource.to(Sink.publisher[String](false))
|
||||
openSource.to(Sink.asPublisher[String](false))
|
||||
}
|
||||
"not be accepted by Source" in {
|
||||
"openSource.connect(intSeq)" shouldNot compile
|
||||
|
|
@ -96,7 +96,7 @@ class FlowCompileSpec extends AkkaSpec {
|
|||
"RunnableGraph" should {
|
||||
Sink.head[String]
|
||||
val closed: RunnableGraph[Publisher[String]] =
|
||||
Source(Seq(1, 2, 3)).map(_.toString).toMat(Sink.publisher[String](false))(Keep.right)
|
||||
Source(Seq(1, 2, 3)).map(_.toString).toMat(Sink.asPublisher[String](false))(Keep.right)
|
||||
"run" in {
|
||||
closed.run()
|
||||
}
|
||||
|
|
|
|||
|
|
@ -33,7 +33,7 @@ class FlowConcatAllSpec extends AkkaSpec {
|
|||
val main = Source(List(s1, s2, s3, s4, s5))
|
||||
|
||||
val subscriber = TestSubscriber.manualProbe[Int]()
|
||||
main.flatMapConcat(ConstantFun.scalaIdentityFunction).to(Sink(subscriber)).run()
|
||||
main.flatMapConcat(ConstantFun.scalaIdentityFunction).to(Sink.fromSubscriber(subscriber)).run()
|
||||
val subscription = subscriber.expectSubscription()
|
||||
subscription.request(10)
|
||||
for (i ← 1 to 10)
|
||||
|
|
@ -50,7 +50,7 @@ class FlowConcatAllSpec extends AkkaSpec {
|
|||
.map(_._2)
|
||||
.concatSubstreams
|
||||
.flatMapConcat(ConstantFun.scalaIdentityFunction)
|
||||
.runWith(Sink(subscriber))
|
||||
.runWith(Sink.fromSubscriber(subscriber))
|
||||
val subscription = subscriber.expectSubscription()
|
||||
subscription.request(10)
|
||||
for (i ← (1 to 10))
|
||||
|
|
@ -62,14 +62,14 @@ class FlowConcatAllSpec extends AkkaSpec {
|
|||
"on onError on master stream cancel the current open substream and signal error" in assertAllStagesStopped {
|
||||
val publisher = TestPublisher.manualProbe[Source[Int, Unit]]()
|
||||
val subscriber = TestSubscriber.manualProbe[Int]()
|
||||
Source(publisher).flatMapConcat(ConstantFun.scalaIdentityFunction).to(Sink(subscriber)).run()
|
||||
Source.fromPublisher(publisher).flatMapConcat(ConstantFun.scalaIdentityFunction).to(Sink.fromSubscriber(subscriber)).run()
|
||||
|
||||
val upstream = publisher.expectSubscription()
|
||||
val downstream = subscriber.expectSubscription()
|
||||
downstream.request(1000)
|
||||
|
||||
val substreamPublisher = TestPublisher.manualProbe[Int]()
|
||||
val substreamFlow = Source(substreamPublisher)
|
||||
val substreamFlow = Source.fromPublisher(substreamPublisher)
|
||||
upstream.expectRequest()
|
||||
upstream.sendNext(substreamFlow)
|
||||
val subUpstream = substreamPublisher.expectSubscription()
|
||||
|
|
@ -82,14 +82,14 @@ class FlowConcatAllSpec extends AkkaSpec {
|
|||
"on onError on master stream cancel the currently opening substream and signal error" in assertAllStagesStopped {
|
||||
val publisher = TestPublisher.manualProbe[Source[Int, Unit]]()
|
||||
val subscriber = TestSubscriber.manualProbe[Int]()
|
||||
Source(publisher).flatMapConcat(ConstantFun.scalaIdentityFunction).to(Sink(subscriber)).run()
|
||||
Source.fromPublisher(publisher).flatMapConcat(ConstantFun.scalaIdentityFunction).to(Sink.fromSubscriber(subscriber)).run()
|
||||
|
||||
val upstream = publisher.expectSubscription()
|
||||
val downstream = subscriber.expectSubscription()
|
||||
downstream.request(1000)
|
||||
|
||||
val substreamPublisher = TestPublisher.manualProbe[Int](autoOnSubscribe = false)
|
||||
val substreamFlow = Source(substreamPublisher)
|
||||
val substreamFlow = Source.fromPublisher(substreamPublisher)
|
||||
upstream.expectRequest()
|
||||
upstream.sendNext(substreamFlow)
|
||||
val subUpstream = substreamPublisher.expectSubscription()
|
||||
|
|
@ -105,14 +105,14 @@ class FlowConcatAllSpec extends AkkaSpec {
|
|||
"on onError on opening substream, cancel the master stream and signal error " in assertAllStagesStopped {
|
||||
val publisher = TestPublisher.manualProbe[Source[Int, _]]()
|
||||
val subscriber = TestSubscriber.manualProbe[Int]()
|
||||
Source(publisher).flatMapConcat(_ ⇒ throw testException).to(Sink(subscriber)).run()
|
||||
Source.fromPublisher(publisher).flatMapConcat(_ ⇒ throw testException).to(Sink.fromSubscriber(subscriber)).run()
|
||||
|
||||
val upstream = publisher.expectSubscription()
|
||||
val downstream = subscriber.expectSubscription()
|
||||
downstream.request(1000)
|
||||
|
||||
val substreamPublisher = TestPublisher.manualProbe[Int]()
|
||||
val substreamFlow = Source(substreamPublisher)
|
||||
val substreamFlow = Source.fromPublisher(substreamPublisher)
|
||||
upstream.expectRequest()
|
||||
upstream.sendNext(substreamFlow)
|
||||
subscriber.expectError(testException)
|
||||
|
|
@ -122,14 +122,14 @@ class FlowConcatAllSpec extends AkkaSpec {
|
|||
"on onError on open substream, cancel the master stream and signal error " in assertAllStagesStopped {
|
||||
val publisher = TestPublisher.manualProbe[Source[Int, Unit]]()
|
||||
val subscriber = TestSubscriber.manualProbe[Int]()
|
||||
Source(publisher).flatMapConcat(ConstantFun.scalaIdentityFunction).to(Sink(subscriber)).run()
|
||||
Source.fromPublisher(publisher).flatMapConcat(ConstantFun.scalaIdentityFunction).to(Sink.fromSubscriber(subscriber)).run()
|
||||
|
||||
val upstream = publisher.expectSubscription()
|
||||
val downstream = subscriber.expectSubscription()
|
||||
downstream.request(1000)
|
||||
|
||||
val substreamPublisher = TestPublisher.manualProbe[Int]()
|
||||
val substreamFlow = Source(substreamPublisher)
|
||||
val substreamFlow = Source.fromPublisher(substreamPublisher)
|
||||
upstream.expectRequest()
|
||||
upstream.sendNext(substreamFlow)
|
||||
val subUpstream = substreamPublisher.expectSubscription()
|
||||
|
|
@ -142,14 +142,14 @@ class FlowConcatAllSpec extends AkkaSpec {
|
|||
"on cancellation cancel the current open substream and the master stream" in assertAllStagesStopped {
|
||||
val publisher = TestPublisher.manualProbe[Source[Int, Unit]]()
|
||||
val subscriber = TestSubscriber.manualProbe[Int]()
|
||||
Source(publisher).flatMapConcat(ConstantFun.scalaIdentityFunction).to(Sink(subscriber)).run()
|
||||
Source.fromPublisher(publisher).flatMapConcat(ConstantFun.scalaIdentityFunction).to(Sink.fromSubscriber(subscriber)).run()
|
||||
|
||||
val upstream = publisher.expectSubscription()
|
||||
val downstream = subscriber.expectSubscription()
|
||||
downstream.request(1000)
|
||||
|
||||
val substreamPublisher = TestPublisher.manualProbe[Int]()
|
||||
val substreamFlow = Source(substreamPublisher)
|
||||
val substreamFlow = Source.fromPublisher(substreamPublisher)
|
||||
upstream.expectRequest()
|
||||
upstream.sendNext(substreamFlow)
|
||||
val subUpstream = substreamPublisher.expectSubscription()
|
||||
|
|
@ -163,14 +163,14 @@ class FlowConcatAllSpec extends AkkaSpec {
|
|||
"on cancellation cancel the currently opening substream and the master stream" in assertAllStagesStopped {
|
||||
val publisher = TestPublisher.manualProbe[Source[Int, Unit]]()
|
||||
val subscriber = TestSubscriber.manualProbe[Int]()
|
||||
Source(publisher).flatMapConcat(ConstantFun.scalaIdentityFunction).to(Sink(subscriber)).run()
|
||||
Source.fromPublisher(publisher).flatMapConcat(ConstantFun.scalaIdentityFunction).to(Sink.fromSubscriber(subscriber)).run()
|
||||
|
||||
val upstream = publisher.expectSubscription()
|
||||
val downstream = subscriber.expectSubscription()
|
||||
downstream.request(1000)
|
||||
|
||||
val substreamPublisher = TestPublisher.manualProbe[Int](autoOnSubscribe = false)
|
||||
val substreamFlow = Source(substreamPublisher)
|
||||
val substreamFlow = Source.fromPublisher(substreamPublisher)
|
||||
upstream.expectRequest()
|
||||
upstream.sendNext(substreamFlow)
|
||||
val subUpstream = substreamPublisher.expectSubscription()
|
||||
|
|
@ -188,9 +188,9 @@ class FlowConcatAllSpec extends AkkaSpec {
|
|||
val down = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
val flowSubscriber = Source
|
||||
.subscriber[Source[Int, Unit]]
|
||||
.asSubscriber[Source[Int, Unit]]
|
||||
.flatMapConcat(ConstantFun.scalaIdentityFunction)
|
||||
.to(Sink(down))
|
||||
.to(Sink.fromSubscriber(down))
|
||||
.run()
|
||||
|
||||
val downstream = down.expectSubscription()
|
||||
|
|
|
|||
|
|
@ -17,7 +17,7 @@ class FlowConcatSpec extends BaseTwoStreamsSetup {
|
|||
|
||||
override def setup(p1: Publisher[Int], p2: Publisher[Int]) = {
|
||||
val subscriber = TestSubscriber.probe[Outputs]()
|
||||
Source(p1).concat(Source(p2)).runWith(Sink(subscriber))
|
||||
Source.fromPublisher(p1).concat(Source.fromPublisher(p2)).runWith(Sink.fromSubscriber(subscriber))
|
||||
subscriber
|
||||
}
|
||||
|
||||
|
|
@ -29,7 +29,7 @@ class FlowConcatSpec extends BaseTwoStreamsSetup {
|
|||
val s2: Source[String, _] = Source(List(4, 5, 6)).map(_.toString + "-s")
|
||||
|
||||
val subs = TestSubscriber.manualProbe[Any]()
|
||||
val subSink = Sink.publisher[Any](false)
|
||||
val subSink = Sink.asPublisher[Any](false)
|
||||
|
||||
val (_, res) = f1.concat(s2).runWith(s1, subSink)
|
||||
|
||||
|
|
@ -101,7 +101,7 @@ class FlowConcatSpec extends BaseTwoStreamsSetup {
|
|||
"correctly handle async errors in secondary upstream" in assertAllStagesStopped {
|
||||
val promise = Promise[Int]()
|
||||
val subscriber = TestSubscriber.manualProbe[Int]()
|
||||
Source(List(1, 2, 3)).concat(Source(promise.future)).runWith(Sink(subscriber))
|
||||
Source(List(1, 2, 3)).concat(Source.fromFuture(promise.future)).runWith(Sink.fromSubscriber(subscriber))
|
||||
|
||||
val subscription = subscriber.expectSubscription()
|
||||
subscription.request(4)
|
||||
|
|
@ -152,7 +152,7 @@ class FlowConcatSpec extends BaseTwoStreamsSetup {
|
|||
"subscribe at once to initial source and to one that it's concat to" in {
|
||||
val publisher1 = TestPublisher.probe[Int]()
|
||||
val publisher2 = TestPublisher.probe[Int]()
|
||||
val probeSink = Source(publisher1).concat(Source(publisher2))
|
||||
val probeSink = Source.fromPublisher(publisher1).concat(Source.fromPublisher(publisher2))
|
||||
.runWith(TestSink.probe[Int])
|
||||
|
||||
val sub1 = publisher1.expectSubscription()
|
||||
|
|
|
|||
|
|
@ -22,7 +22,7 @@ class FlowConflateSpec extends AkkaSpec {
|
|||
val publisher = TestPublisher.probe[Int]()
|
||||
val subscriber = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
Source(publisher).conflate(seed = i ⇒ i)(aggregate = (sum, i) ⇒ sum + i).to(Sink(subscriber)).run()
|
||||
Source.fromPublisher(publisher).conflate(seed = i ⇒ i)(aggregate = (sum, i) ⇒ sum + i).to(Sink.fromSubscriber(subscriber)).run()
|
||||
val sub = subscriber.expectSubscription()
|
||||
|
||||
for (i ← 1 to 100) {
|
||||
|
|
@ -38,7 +38,7 @@ class FlowConflateSpec extends AkkaSpec {
|
|||
val publisher = TestPublisher.probe[Int]()
|
||||
val subscriber = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
Source(publisher).conflate(seed = i ⇒ i)(aggregate = (sum, i) ⇒ sum + i).to(Sink(subscriber)).run()
|
||||
Source.fromPublisher(publisher).conflate(seed = i ⇒ i)(aggregate = (sum, i) ⇒ sum + i).to(Sink.fromSubscriber(subscriber)).run()
|
||||
val sub = subscriber.expectSubscription()
|
||||
|
||||
for (i ← 1 to 100) {
|
||||
|
|
@ -62,7 +62,7 @@ class FlowConflateSpec extends AkkaSpec {
|
|||
val publisher = TestPublisher.probe[Int]()
|
||||
val subscriber = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
Source(publisher).conflate(seed = i ⇒ i)(aggregate = (sum, i) ⇒ sum + i).to(Sink(subscriber)).run()
|
||||
Source.fromPublisher(publisher).conflate(seed = i ⇒ i)(aggregate = (sum, i) ⇒ sum + i).to(Sink.fromSubscriber(subscriber)).run()
|
||||
val sub = subscriber.expectSubscription()
|
||||
|
||||
sub.request(1)
|
||||
|
|
|
|||
|
|
@ -49,7 +49,7 @@ class FlowDelaySpec extends AkkaSpec {
|
|||
val c = TestSubscriber.manualProbe[Int]()
|
||||
val p = TestPublisher.manualProbe[Int]()
|
||||
|
||||
Source(p).delay(300.millis).to(Sink(c)).run()
|
||||
Source.fromPublisher(p).delay(300.millis).to(Sink.fromSubscriber(c)).run()
|
||||
val cSub = c.expectSubscription()
|
||||
val pSub = p.expectSubscription()
|
||||
cSub.request(100)
|
||||
|
|
@ -111,7 +111,7 @@ class FlowDelaySpec extends AkkaSpec {
|
|||
val c = TestSubscriber.manualProbe[Int]()
|
||||
val p = TestPublisher.manualProbe[Int]()
|
||||
|
||||
Source(p).delay(10.seconds, DelayOverflowStrategy.emitEarly).withAttributes(inputBuffer(16, 16)).to(Sink(c)).run()
|
||||
Source.fromPublisher(p).delay(10.seconds, DelayOverflowStrategy.emitEarly).withAttributes(inputBuffer(16, 16)).to(Sink.fromSubscriber(c)).run()
|
||||
val cSub = c.expectSubscription()
|
||||
val pSub = p.expectSubscription()
|
||||
cSub.request(20)
|
||||
|
|
|
|||
|
|
@ -28,7 +28,7 @@ class FlowDropSpec extends AkkaSpec with ScriptedTest {
|
|||
|
||||
"not drop anything for negative n" in {
|
||||
val probe = TestSubscriber.manualProbe[Int]()
|
||||
Source(List(1, 2, 3)).drop(-1).to(Sink(probe)).run()
|
||||
Source(List(1, 2, 3)).drop(-1).to(Sink.fromSubscriber(probe)).run()
|
||||
probe.expectSubscription().request(10)
|
||||
probe.expectNext(1)
|
||||
probe.expectNext(2)
|
||||
|
|
|
|||
|
|
@ -18,7 +18,7 @@ class FlowDropWithinSpec extends AkkaSpec {
|
|||
val input = Iterator.from(1)
|
||||
val p = TestPublisher.manualProbe[Int]()
|
||||
val c = TestSubscriber.manualProbe[Int]()
|
||||
Source(p).dropWithin(1.second).to(Sink(c)).run()
|
||||
Source.fromPublisher(p).dropWithin(1.second).to(Sink.fromSubscriber(c)).run()
|
||||
val pSub = p.expectSubscription
|
||||
val cSub = c.expectSubscription
|
||||
cSub.request(100)
|
||||
|
|
@ -39,7 +39,7 @@ class FlowDropWithinSpec extends AkkaSpec {
|
|||
val upstream = TestPublisher.probe[Int]()
|
||||
val downstream = TestSubscriber.probe[Int]()
|
||||
|
||||
Source(upstream).dropWithin(1.day).runWith(Sink(downstream))
|
||||
Source.fromPublisher(upstream).dropWithin(1.day).runWith(Sink.fromSubscriber(downstream))
|
||||
|
||||
upstream.sendComplete()
|
||||
downstream.expectSubscriptionAndComplete()
|
||||
|
|
|
|||
|
|
@ -28,7 +28,7 @@ class FlowExpandSpec extends AkkaSpec {
|
|||
val subscriber = TestSubscriber.probe[Int]()
|
||||
|
||||
// Simply repeat the last element as an extrapolation step
|
||||
Source(publisher).expand(seed = i ⇒ i)(extrapolate = i ⇒ (i, i)).to(Sink(subscriber)).run()
|
||||
Source.fromPublisher(publisher).expand(seed = i ⇒ i)(extrapolate = i ⇒ (i, i)).to(Sink.fromSubscriber(subscriber)).run()
|
||||
|
||||
for (i ← 1 to 100) {
|
||||
// Order is important here: If the request comes first it will be extrapolated!
|
||||
|
|
@ -44,7 +44,7 @@ class FlowExpandSpec extends AkkaSpec {
|
|||
val subscriber = TestSubscriber.probe[Int]()
|
||||
|
||||
// Simply repeat the last element as an extrapolation step
|
||||
Source(publisher).expand(seed = i ⇒ i)(extrapolate = i ⇒ (i, i)).to(Sink(subscriber)).run()
|
||||
Source.fromPublisher(publisher).expand(seed = i ⇒ i)(extrapolate = i ⇒ (i, i)).to(Sink.fromSubscriber(subscriber)).run()
|
||||
|
||||
publisher.sendNext(42)
|
||||
|
||||
|
|
@ -66,7 +66,7 @@ class FlowExpandSpec extends AkkaSpec {
|
|||
val subscriber = TestSubscriber.probe[Int]()
|
||||
|
||||
// Simply repeat the last element as an extrapolation step
|
||||
Source(publisher).expand(seed = i ⇒ i)(extrapolate = i ⇒ (i, i)).to(Sink(subscriber)).run()
|
||||
Source.fromPublisher(publisher).expand(seed = i ⇒ i)(extrapolate = i ⇒ (i, i)).to(Sink.fromSubscriber(subscriber)).run()
|
||||
|
||||
publisher.sendNext(1)
|
||||
subscriber.requestNext(1)
|
||||
|
|
@ -94,7 +94,7 @@ class FlowExpandSpec extends AkkaSpec {
|
|||
val publisher = TestPublisher.probe[Int]()
|
||||
val subscriber = TestSubscriber.probe[Int]()
|
||||
|
||||
Source(publisher).expand(seed = i ⇒ i)(extrapolate = i ⇒ (i, i)).to(Sink(subscriber)).run()
|
||||
Source.fromPublisher(publisher).expand(seed = i ⇒ i)(extrapolate = i ⇒ (i, i)).to(Sink.fromSubscriber(subscriber)).run()
|
||||
|
||||
publisher.sendNext(1)
|
||||
subscriber.requestNext(1)
|
||||
|
|
|
|||
|
|
@ -27,7 +27,7 @@ class FlowFilterSpec extends AkkaSpec with ScriptedTest {
|
|||
implicit val materializer = ActorMaterializer(settings)
|
||||
|
||||
val probe = TestSubscriber.manualProbe[Int]()
|
||||
Source(List.fill(1000)(0) ::: List(1)).filter(_ != 0).runWith(Sink(probe))
|
||||
Source(List.fill(1000)(0) ::: List(1)).filter(_ != 0).runWith(Sink.fromSubscriber(probe))
|
||||
|
||||
val subscription = probe.expectSubscription()
|
||||
for (_ ← 1 to 10000) {
|
||||
|
|
|
|||
|
|
@ -19,7 +19,7 @@ class FlowFlattenMergeSpec extends AkkaSpec with ScalaFutures with ConversionChe
|
|||
import system.dispatcher
|
||||
|
||||
def src10(i: Int) = Source(i until (i + 10))
|
||||
def blocked = Source(Promise[Int].future)
|
||||
def blocked = Source.fromFuture(Promise[Int].future)
|
||||
|
||||
val toSeq = Flow[Int].grouped(1000).toMat(Sink.head)(Keep.right)
|
||||
val toSet = toSeq.mapMaterializedValue(_.map(_.toSet))
|
||||
|
|
@ -98,7 +98,7 @@ class FlowFlattenMergeSpec extends AkkaSpec with ScalaFutures with ConversionChe
|
|||
val p1, p2 = TestPublisher.probe[Int]()
|
||||
val ex = new Exception("buh")
|
||||
val p = Promise[Source[Int, Unit]]
|
||||
(Source(List(Source(p1), Source(p2))) ++ Source(p.future))
|
||||
(Source(List(Source.fromPublisher(p1), Source.fromPublisher(p2))) ++ Source.fromFuture(p.future))
|
||||
.flatMapMerge(5, identity)
|
||||
.runWith(Sink.head)
|
||||
p1.expectRequest()
|
||||
|
|
@ -112,7 +112,7 @@ class FlowFlattenMergeSpec extends AkkaSpec with ScalaFutures with ConversionChe
|
|||
val p1, p2 = TestPublisher.probe[Int]()
|
||||
val ex = new Exception("buh")
|
||||
val p = Promise[Int]
|
||||
Source(List(Source(p1), Source(p2), Source(p.future)))
|
||||
Source(List(Source.fromPublisher(p1), Source.fromPublisher(p2), Source.fromFuture(p.future)))
|
||||
.flatMapMerge(5, identity)
|
||||
.runWith(Sink.head)
|
||||
p1.expectRequest()
|
||||
|
|
@ -128,8 +128,8 @@ class FlowFlattenMergeSpec extends AkkaSpec with ScalaFutures with ConversionChe
|
|||
val latch = TestLatch()
|
||||
Source(1 to 3)
|
||||
.flatMapMerge(10, {
|
||||
case 1 ⇒ Source(p1)
|
||||
case 2 ⇒ Source(p2)
|
||||
case 1 ⇒ Source.fromPublisher(p1)
|
||||
case 2 ⇒ Source.fromPublisher(p2)
|
||||
case 3 ⇒
|
||||
Await.ready(latch, 3.seconds)
|
||||
throw ex
|
||||
|
|
@ -145,7 +145,7 @@ class FlowFlattenMergeSpec extends AkkaSpec with ScalaFutures with ConversionChe
|
|||
"cancel substreams when being cancelled" in {
|
||||
val p1, p2 = TestPublisher.probe[Int]()
|
||||
val ex = new Exception("buh")
|
||||
val sink = Source(List(Source(p1), Source(p2)))
|
||||
val sink = Source(List(Source.fromPublisher(p1), Source.fromPublisher(p2)))
|
||||
.flatMapMerge(5, identity)
|
||||
.runWith(TestSink.probe)
|
||||
sink.request(1)
|
||||
|
|
|
|||
|
|
@ -36,7 +36,7 @@ class FlowForeachSpec extends AkkaSpec {
|
|||
|
||||
"yield the first error" in assertAllStagesStopped {
|
||||
val p = TestPublisher.manualProbe[Int]()
|
||||
Source(p).runForeach(testActor ! _) onFailure {
|
||||
Source.fromPublisher(p).runForeach(testActor ! _) onFailure {
|
||||
case ex ⇒ testActor ! ex
|
||||
}
|
||||
val proc = p.expectSubscription()
|
||||
|
|
|
|||
|
|
@ -21,7 +21,7 @@ class FlowFromFutureSpec extends AkkaSpec {
|
|||
"A Flow based on a Future" must {
|
||||
"produce one element from already successful Future" in assertAllStagesStopped {
|
||||
val c = TestSubscriber.manualProbe[Int]()
|
||||
val p = Source(Future.successful(1)).runWith(Sink.publisher(true)).subscribe(c)
|
||||
val p = Source.fromFuture(Future.successful(1)).runWith(Sink.asPublisher(true)).subscribe(c)
|
||||
val sub = c.expectSubscription()
|
||||
c.expectNoMsg(100.millis)
|
||||
sub.request(1)
|
||||
|
|
@ -32,14 +32,14 @@ class FlowFromFutureSpec extends AkkaSpec {
|
|||
"produce error from already failed Future" in assertAllStagesStopped {
|
||||
val ex = new RuntimeException("test") with NoStackTrace
|
||||
val c = TestSubscriber.manualProbe[Int]()
|
||||
Source(Future.failed[Int](ex)).runWith(Sink.publisher(false)).subscribe(c)
|
||||
Source.fromFuture(Future.failed[Int](ex)).runWith(Sink.asPublisher(false)).subscribe(c)
|
||||
c.expectSubscriptionAndError(ex)
|
||||
}
|
||||
|
||||
"produce one element when Future is completed" in assertAllStagesStopped {
|
||||
val promise = Promise[Int]()
|
||||
val c = TestSubscriber.manualProbe[Int]()
|
||||
Source(promise.future).runWith(Sink.publisher(true)).subscribe(c)
|
||||
Source.fromFuture(promise.future).runWith(Sink.asPublisher(true)).subscribe(c)
|
||||
val sub = c.expectSubscription()
|
||||
sub.request(1)
|
||||
c.expectNoMsg(100.millis)
|
||||
|
|
@ -52,7 +52,7 @@ class FlowFromFutureSpec extends AkkaSpec {
|
|||
"produce one element when Future is completed but not before request" in {
|
||||
val promise = Promise[Int]()
|
||||
val c = TestSubscriber.manualProbe[Int]()
|
||||
Source(promise.future).runWith(Sink.publisher(true)).subscribe(c)
|
||||
Source.fromFuture(promise.future).runWith(Sink.asPublisher(true)).subscribe(c)
|
||||
val sub = c.expectSubscription()
|
||||
promise.success(1)
|
||||
c.expectNoMsg(200.millis)
|
||||
|
|
@ -63,7 +63,7 @@ class FlowFromFutureSpec extends AkkaSpec {
|
|||
|
||||
"produce elements with multiple subscribers" in assertAllStagesStopped {
|
||||
val promise = Promise[Int]()
|
||||
val p = Source(promise.future).runWith(Sink.publisher(true))
|
||||
val p = Source.fromFuture(promise.future).runWith(Sink.asPublisher(true))
|
||||
val c1 = TestSubscriber.manualProbe[Int]()
|
||||
val c2 = TestSubscriber.manualProbe[Int]()
|
||||
p.subscribe(c1)
|
||||
|
|
@ -81,7 +81,7 @@ class FlowFromFutureSpec extends AkkaSpec {
|
|||
|
||||
"allow cancel before receiving element" in {
|
||||
val promise = Promise[Int]()
|
||||
val p = Source(promise.future).runWith(Sink.publisher(true))
|
||||
val p = Source.fromFuture(promise.future).runWith(Sink.asPublisher(true))
|
||||
val keepAlive = TestSubscriber.manualProbe[Int]()
|
||||
val c = TestSubscriber.manualProbe[Int]()
|
||||
p.subscribe(keepAlive)
|
||||
|
|
|
|||
|
|
@ -36,7 +36,7 @@ class FlowGraphCompileSpec extends AkkaSpec {
|
|||
|
||||
val in1 = Source(List("a", "b", "c"))
|
||||
val in2 = Source(List("d", "e", "f"))
|
||||
val out1 = Sink.publisher[String](false)
|
||||
val out1 = Sink.asPublisher[String](false)
|
||||
val out2 = Sink.head[String]
|
||||
|
||||
"A Graph" should {
|
||||
|
|
@ -165,9 +165,9 @@ class FlowGraphCompileSpec extends AkkaSpec {
|
|||
val in3 = Source(List("b"))
|
||||
val in5 = Source(List("b"))
|
||||
val in7 = Source(List("a"))
|
||||
val out2 = Sink.publisher[String](false)
|
||||
val out9 = Sink.publisher[String](false)
|
||||
val out10 = Sink.publisher[String](false)
|
||||
val out2 = Sink.asPublisher[String](false)
|
||||
val out9 = Sink.asPublisher[String](false)
|
||||
val out10 = Sink.asPublisher[String](false)
|
||||
def f(s: String) = Flow[String].transform(op[String, String]).named(s)
|
||||
import GraphDSL.Implicits._
|
||||
|
||||
|
|
@ -198,7 +198,7 @@ class FlowGraphCompileSpec extends AkkaSpec {
|
|||
RunnableGraph.fromGraph(GraphDSL.create() { implicit b ⇒
|
||||
val zip = b.add(Zip[Int, String]())
|
||||
val unzip = b.add(Unzip[Int, String]())
|
||||
val out = Sink.publisher[(Int, String)](false)
|
||||
val out = Sink.asPublisher[(Int, String)](false)
|
||||
import GraphDSL.Implicits._
|
||||
Source(List(1 -> "a", 2 -> "b", 3 -> "c")) ~> unzip.in
|
||||
unzip.out0 ~> Flow[Int].map(_ * 2) ~> zip.in0
|
||||
|
|
@ -213,8 +213,8 @@ class FlowGraphCompileSpec extends AkkaSpec {
|
|||
RunnableGraph.fromGraph(GraphDSL.create() { implicit b ⇒
|
||||
val zip = b.add(Zip[Int, String]())
|
||||
val unzip = b.add(Unzip[Int, String]())
|
||||
val wrongOut = Sink.publisher[(Int, Int)](false)
|
||||
val whatever = Sink.publisher[Any](false)
|
||||
val wrongOut = Sink.asPublisher[(Int, Int)](false)
|
||||
val whatever = Sink.asPublisher[Any](false)
|
||||
"Flow(List(1, 2, 3)) ~> zip.left ~> wrongOut" shouldNot compile
|
||||
"""Flow(List("a", "b", "c")) ~> zip.left""" shouldNot compile
|
||||
"""Flow(List("a", "b", "c")) ~> zip.out""" shouldNot compile
|
||||
|
|
@ -230,9 +230,9 @@ class FlowGraphCompileSpec extends AkkaSpec {
|
|||
RunnableGraph.fromGraph(GraphDSL.create() { implicit b ⇒
|
||||
import GraphDSL.Implicits._
|
||||
val merge = b.add(Merge[Fruit](2))
|
||||
Source[Fruit](apples) ~> Flow[Fruit] ~> merge.in(0)
|
||||
Source[Apple](apples) ~> Flow[Apple] ~> merge.in(1)
|
||||
merge.out ~> Flow[Fruit].map(identity) ~> Sink(TestSubscriber.manualProbe[Fruit]())
|
||||
Source.fromIterator[Fruit](apples) ~> Flow[Fruit] ~> merge.in(0)
|
||||
Source.fromIterator[Apple](apples) ~> Flow[Apple] ~> merge.in(1)
|
||||
merge.out ~> Flow[Fruit].map(identity) ~> Sink.fromSubscriber(TestSubscriber.manualProbe[Fruit]())
|
||||
ClosedShape
|
||||
})
|
||||
}
|
||||
|
|
@ -241,31 +241,31 @@ class FlowGraphCompileSpec extends AkkaSpec {
|
|||
RunnableGraph.fromGraph(GraphDSL.create() { implicit b ⇒
|
||||
import GraphDSL.Implicits._
|
||||
val fruitMerge = b.add(Merge[Fruit](2))
|
||||
Source[Fruit](apples) ~> fruitMerge
|
||||
Source[Apple](apples) ~> fruitMerge
|
||||
Source.fromIterator[Fruit](apples) ~> fruitMerge
|
||||
Source.fromIterator[Apple](apples) ~> fruitMerge
|
||||
fruitMerge ~> Sink.head[Fruit]
|
||||
"fruitMerge ~> Sink.head[Apple]" shouldNot compile
|
||||
|
||||
val appleMerge = b.add(Merge[Apple](2))
|
||||
"Source[Fruit](apples) ~> appleMerge" shouldNot compile
|
||||
Source.empty[Apple] ~> appleMerge
|
||||
Source[Apple](apples) ~> appleMerge
|
||||
Source.fromIterator[Apple](apples) ~> appleMerge
|
||||
appleMerge ~> Sink.head[Fruit]
|
||||
|
||||
val appleMerge2 = b.add(Merge[Apple](2))
|
||||
Source.empty[Apple] ~> appleMerge2
|
||||
Source[Apple](apples) ~> appleMerge2
|
||||
Source.fromIterator[Apple](apples) ~> appleMerge2
|
||||
appleMerge2 ~> Sink.head[Apple]
|
||||
|
||||
val fruitBcast = b.add(Broadcast[Fruit](2))
|
||||
Source[Apple](apples) ~> fruitBcast
|
||||
Source.fromIterator[Apple](apples) ~> fruitBcast
|
||||
fruitBcast ~> Sink.head[Fruit]
|
||||
fruitBcast ~> Sink.ignore
|
||||
"fruitBcast ~> Sink.head[Apple]" shouldNot compile
|
||||
|
||||
val appleBcast = b.add(Broadcast[Apple](2))
|
||||
"Source[Fruit](apples) ~> appleBcast" shouldNot compile
|
||||
Source[Apple](apples) ~> appleBcast
|
||||
Source.fromIterator[Apple](apples) ~> appleBcast
|
||||
appleBcast ~> Sink.head[Fruit]
|
||||
appleBcast ~> Sink.head[Apple]
|
||||
ClosedShape
|
||||
|
|
@ -274,33 +274,33 @@ class FlowGraphCompileSpec extends AkkaSpec {
|
|||
|
||||
"build with implicits and variance" in {
|
||||
RunnableGraph.fromGraph(GraphDSL.create() { implicit b ⇒
|
||||
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]())
|
||||
def appleSource = b.add(Source.fromPublisher(TestPublisher.manualProbe[Apple]()))
|
||||
def fruitSource = b.add(Source.fromPublisher(TestPublisher.manualProbe[Fruit]()))
|
||||
val outA = b add Sink.fromSubscriber(TestSubscriber.manualProbe[Fruit]())
|
||||
val outB = b add Sink.fromSubscriber(TestSubscriber.manualProbe[Fruit]())
|
||||
val merge = b add Merge[Fruit](11)
|
||||
val unzip = b add Unzip[Int, String]()
|
||||
val whatever = b add Sink.publisher[Any](false)
|
||||
val whatever = b add Sink.asPublisher[Any](false)
|
||||
import GraphDSL.Implicits._
|
||||
b.add(Source[Fruit](apples)) ~> merge.in(0)
|
||||
b.add(Source.fromIterator[Fruit](apples)) ~> merge.in(0)
|
||||
appleSource ~> merge.in(1)
|
||||
appleSource ~> merge.in(2)
|
||||
fruitSource ~> merge.in(3)
|
||||
fruitSource ~> Flow[Fruit].map(identity) ~> merge.in(4)
|
||||
appleSource ~> Flow[Apple].map(identity) ~> merge.in(5)
|
||||
b.add(Source(apples)) ~> merge.in(6)
|
||||
b.add(Source(apples)) ~> Flow[Fruit].map(identity) ~> merge.in(7)
|
||||
b.add(Source(apples)) ~> Flow[Apple].map(identity) ~> merge.in(8)
|
||||
b.add(Source.fromIterator(apples)) ~> merge.in(6)
|
||||
b.add(Source.fromIterator(apples)) ~> Flow[Fruit].map(identity) ~> merge.in(7)
|
||||
b.add(Source.fromIterator(apples)) ~> Flow[Apple].map(identity) ~> merge.in(8)
|
||||
merge.out ~> Flow[Fruit].map(identity) ~> outA
|
||||
|
||||
b.add(Source(apples)) ~> Flow[Apple] ~> merge.in(9)
|
||||
b.add(Source(apples)) ~> Flow[Apple] ~> outB
|
||||
b.add(Source(apples)) ~> Flow[Apple] ~> b.add(Sink.publisher[Fruit](false))
|
||||
b.add(Source.fromIterator(apples)) ~> Flow[Apple] ~> merge.in(9)
|
||||
b.add(Source.fromIterator(apples)) ~> Flow[Apple] ~> outB
|
||||
b.add(Source.fromIterator(apples)) ~> Flow[Apple] ~> b.add(Sink.asPublisher[Fruit](false))
|
||||
appleSource ~> Flow[Apple] ~> merge.in(10)
|
||||
|
||||
Source(List(1 -> "a", 2 -> "b", 3 -> "c")) ~> unzip.in
|
||||
unzip.out1 ~> whatever
|
||||
unzip.out0 ~> b.add(Sink.publisher[Any](false))
|
||||
unzip.out0 ~> b.add(Sink.asPublisher[Any](false))
|
||||
|
||||
"merge.out ~> b.add(Broadcast[Apple](2))" shouldNot compile
|
||||
"merge.out ~> Flow[Fruit].map(identity) ~> b.add(Broadcast[Apple](2))" shouldNot compile
|
||||
|
|
|
|||
|
|
@ -50,9 +50,9 @@ class FlowGroupBySpec extends AkkaSpec with ScalaFutures with ConversionCheckedT
|
|||
}
|
||||
|
||||
class SubstreamsSupport(groupCount: Int = 2, elementCount: Int = 6, maxSubstreams: Int = -1) {
|
||||
val source = Source(1 to elementCount).runWith(Sink.publisher(false))
|
||||
val source = Source(1 to elementCount).runWith(Sink.asPublisher(false))
|
||||
val max = if (maxSubstreams > 0) maxSubstreams else groupCount
|
||||
val groupStream = Source(source).groupBy(max, _ % groupCount).lift(_ % groupCount).runWith(Sink.publisher(false))
|
||||
val groupStream = Source.fromPublisher(source).groupBy(max, _ % groupCount).lift(_ % groupCount).runWith(Sink.asPublisher(false))
|
||||
val masterSubscriber = TestSubscriber.manualProbe[(Int, Source[Int, _])]()
|
||||
|
||||
groupStream.subscribe(masterSubscriber)
|
||||
|
|
@ -74,7 +74,7 @@ class FlowGroupBySpec extends AkkaSpec with ScalaFutures with ConversionCheckedT
|
|||
"groupBy" must {
|
||||
"work in the happy case" in assertAllStagesStopped {
|
||||
new SubstreamsSupport(groupCount = 2) {
|
||||
val s1 = StreamPuppet(getSubFlow(1).runWith(Sink.publisher(false)))
|
||||
val s1 = StreamPuppet(getSubFlow(1).runWith(Sink.asPublisher(false)))
|
||||
masterSubscriber.expectNoMsg(100.millis)
|
||||
|
||||
s1.expectNoMsg(100.millis)
|
||||
|
|
@ -82,7 +82,7 @@ class FlowGroupBySpec extends AkkaSpec with ScalaFutures with ConversionCheckedT
|
|||
s1.expectNext(1)
|
||||
s1.expectNoMsg(100.millis)
|
||||
|
||||
val s2 = StreamPuppet(getSubFlow(0).runWith(Sink.publisher(false)))
|
||||
val s2 = StreamPuppet(getSubFlow(0).runWith(Sink.asPublisher(false)))
|
||||
|
||||
s2.expectNoMsg(100.millis)
|
||||
s2.request(2)
|
||||
|
|
@ -123,9 +123,9 @@ class FlowGroupBySpec extends AkkaSpec with ScalaFutures with ConversionCheckedT
|
|||
|
||||
"accept cancellation of substreams" in assertAllStagesStopped {
|
||||
new SubstreamsSupport(groupCount = 2) {
|
||||
StreamPuppet(getSubFlow(1).runWith(Sink.publisher(false))).cancel()
|
||||
StreamPuppet(getSubFlow(1).runWith(Sink.asPublisher(false))).cancel()
|
||||
|
||||
val substream = StreamPuppet(getSubFlow(0).runWith(Sink.publisher(false)))
|
||||
val substream = StreamPuppet(getSubFlow(0).runWith(Sink.asPublisher(false)))
|
||||
substream.request(2)
|
||||
substream.expectNext(2)
|
||||
substream.expectNext(4)
|
||||
|
|
@ -142,7 +142,7 @@ class FlowGroupBySpec extends AkkaSpec with ScalaFutures with ConversionCheckedT
|
|||
|
||||
"accept cancellation of master stream when not consumed anything" in assertAllStagesStopped {
|
||||
val publisherProbeProbe = TestPublisher.manualProbe[Int]()
|
||||
val publisher = Source(publisherProbeProbe).groupBy(2, _ % 2).lift(_ % 2).runWith(Sink.publisher(false))
|
||||
val publisher = Source.fromPublisher(publisherProbeProbe).groupBy(2, _ % 2).lift(_ % 2).runWith(Sink.asPublisher(false))
|
||||
val subscriber = TestSubscriber.manualProbe[(Int, Source[Int, _])]()
|
||||
publisher.subscribe(subscriber)
|
||||
|
||||
|
|
@ -153,7 +153,7 @@ class FlowGroupBySpec extends AkkaSpec with ScalaFutures with ConversionCheckedT
|
|||
}
|
||||
|
||||
"work with empty input stream" in assertAllStagesStopped {
|
||||
val publisher = Source(List.empty[Int]).groupBy(2, _ % 2).lift(_ % 2).runWith(Sink.publisher(false))
|
||||
val publisher = Source(List.empty[Int]).groupBy(2, _ % 2).lift(_ % 2).runWith(Sink.asPublisher(false))
|
||||
val subscriber = TestSubscriber.manualProbe[(Int, Source[Int, _])]()
|
||||
publisher.subscribe(subscriber)
|
||||
|
||||
|
|
@ -162,7 +162,7 @@ class FlowGroupBySpec extends AkkaSpec with ScalaFutures with ConversionCheckedT
|
|||
|
||||
"abort on onError from upstream" in assertAllStagesStopped {
|
||||
val publisherProbeProbe = TestPublisher.manualProbe[Int]()
|
||||
val publisher = Source(publisherProbeProbe).groupBy(2, _ % 2).lift(_ % 2).runWith(Sink.publisher(false))
|
||||
val publisher = Source.fromPublisher(publisherProbeProbe).groupBy(2, _ % 2).lift(_ % 2).runWith(Sink.asPublisher(false))
|
||||
val subscriber = TestSubscriber.manualProbe[(Int, Source[Int, _])]()
|
||||
publisher.subscribe(subscriber)
|
||||
|
||||
|
|
@ -179,7 +179,7 @@ class FlowGroupBySpec extends AkkaSpec with ScalaFutures with ConversionCheckedT
|
|||
|
||||
"abort on onError from upstream when substreams are running" in assertAllStagesStopped {
|
||||
val publisherProbeProbe = TestPublisher.manualProbe[Int]()
|
||||
val publisher = Source(publisherProbeProbe).groupBy(2, _ % 2).lift(_ % 2).runWith(Sink.publisher(false))
|
||||
val publisher = Source.fromPublisher(publisherProbeProbe).groupBy(2, _ % 2).lift(_ % 2).runWith(Sink.asPublisher(false))
|
||||
val subscriber = TestSubscriber.manualProbe[(Int, Source[Int, _])]()
|
||||
publisher.subscribe(subscriber)
|
||||
|
||||
|
|
@ -191,7 +191,7 @@ class FlowGroupBySpec extends AkkaSpec with ScalaFutures with ConversionCheckedT
|
|||
upstreamSubscription.sendNext(1)
|
||||
|
||||
val (_, substream) = subscriber.expectNext()
|
||||
val substreamPuppet = StreamPuppet(substream.runWith(Sink.publisher(false)))
|
||||
val substreamPuppet = StreamPuppet(substream.runWith(Sink.asPublisher(false)))
|
||||
|
||||
substreamPuppet.request(1)
|
||||
substreamPuppet.expectNext(1)
|
||||
|
|
@ -207,10 +207,10 @@ class FlowGroupBySpec extends AkkaSpec with ScalaFutures with ConversionCheckedT
|
|||
"fail stream when groupBy function throws" in assertAllStagesStopped {
|
||||
val publisherProbeProbe = TestPublisher.manualProbe[Int]()
|
||||
val exc = TE("test")
|
||||
val publisher = Source(publisherProbeProbe)
|
||||
val publisher = Source.fromPublisher(publisherProbeProbe)
|
||||
.groupBy(2, elem ⇒ if (elem == 2) throw exc else elem % 2)
|
||||
.lift(_ % 2)
|
||||
.runWith(Sink.publisher(false))
|
||||
.runWith(Sink.asPublisher(false))
|
||||
val subscriber = TestSubscriber.manualProbe[(Int, Source[Int, Unit])]()
|
||||
publisher.subscribe(subscriber)
|
||||
|
||||
|
|
@ -222,7 +222,7 @@ class FlowGroupBySpec extends AkkaSpec with ScalaFutures with ConversionCheckedT
|
|||
upstreamSubscription.sendNext(1)
|
||||
|
||||
val (_, substream) = subscriber.expectNext()
|
||||
val substreamPuppet = StreamPuppet(substream.runWith(Sink.publisher(false)))
|
||||
val substreamPuppet = StreamPuppet(substream.runWith(Sink.asPublisher(false)))
|
||||
|
||||
substreamPuppet.request(1)
|
||||
substreamPuppet.expectNext(1)
|
||||
|
|
@ -237,11 +237,11 @@ class FlowGroupBySpec extends AkkaSpec with ScalaFutures with ConversionCheckedT
|
|||
"resume stream when groupBy function throws" in {
|
||||
val publisherProbeProbe = TestPublisher.manualProbe[Int]()
|
||||
val exc = TE("test")
|
||||
val publisher = Source(publisherProbeProbe)
|
||||
val publisher = Source.fromPublisher(publisherProbeProbe)
|
||||
.groupBy(2, elem ⇒ if (elem == 2) throw exc else elem % 2)
|
||||
.lift(_ % 2)
|
||||
.withAttributes(ActorAttributes.supervisionStrategy(resumingDecider))
|
||||
.runWith(Sink.publisher(false))
|
||||
.runWith(Sink.asPublisher(false))
|
||||
val subscriber = TestSubscriber.manualProbe[(Int, Source[Int, Unit])]()
|
||||
publisher.subscribe(subscriber)
|
||||
|
||||
|
|
@ -253,7 +253,7 @@ class FlowGroupBySpec extends AkkaSpec with ScalaFutures with ConversionCheckedT
|
|||
upstreamSubscription.sendNext(1)
|
||||
|
||||
val (_, substream1) = subscriber.expectNext()
|
||||
val substreamPuppet1 = StreamPuppet(substream1.runWith(Sink.publisher(false)))
|
||||
val substreamPuppet1 = StreamPuppet(substream1.runWith(Sink.asPublisher(false)))
|
||||
substreamPuppet1.request(10)
|
||||
substreamPuppet1.expectNext(1)
|
||||
|
||||
|
|
@ -261,7 +261,7 @@ class FlowGroupBySpec extends AkkaSpec with ScalaFutures with ConversionCheckedT
|
|||
upstreamSubscription.sendNext(4)
|
||||
|
||||
val (_, substream2) = subscriber.expectNext()
|
||||
val substreamPuppet2 = StreamPuppet(substream2.runWith(Sink.publisher(false)))
|
||||
val substreamPuppet2 = StreamPuppet(substream2.runWith(Sink.asPublisher(false)))
|
||||
substreamPuppet2.request(10)
|
||||
substreamPuppet2.expectNext(4) // note that 2 was dropped
|
||||
|
||||
|
|
@ -281,7 +281,7 @@ class FlowGroupBySpec extends AkkaSpec with ScalaFutures with ConversionCheckedT
|
|||
val up = TestPublisher.manualProbe[Int]()
|
||||
val down = TestSubscriber.manualProbe[(Int, Source[Int, Unit])]()
|
||||
|
||||
val flowSubscriber = Source.subscriber[Int].groupBy(2, _ % 2).lift(_ % 2).to(Sink(down)).run()
|
||||
val flowSubscriber = Source.asSubscriber[Int].groupBy(2, _ % 2).lift(_ % 2).to(Sink.fromSubscriber(down)).run()
|
||||
|
||||
val downstream = down.expectSubscription()
|
||||
downstream.cancel()
|
||||
|
|
@ -299,7 +299,7 @@ class FlowGroupBySpec extends AkkaSpec with ScalaFutures with ConversionCheckedT
|
|||
|
||||
up.sendNext(1)
|
||||
val first = down.expectNext()
|
||||
val s1 = StreamPuppet(first._2.runWith(Sink.publisher(false)))
|
||||
val s1 = StreamPuppet(first._2.runWith(Sink.asPublisher(false)))
|
||||
|
||||
s1.request(1)
|
||||
s1.expectNext(1)
|
||||
|
|
|
|||
|
|
@ -24,7 +24,7 @@ class FlowGroupedWithinSpec extends AkkaSpec with ScriptedTest {
|
|||
val input = Iterator.from(1)
|
||||
val p = TestPublisher.manualProbe[Int]()
|
||||
val c = TestSubscriber.manualProbe[immutable.Seq[Int]]()
|
||||
Source(p).groupedWithin(1000, 1.second).to(Sink(c)).run()
|
||||
Source.fromPublisher(p).groupedWithin(1000, 1.second).to(Sink.fromSubscriber(c)).run()
|
||||
val pSub = p.expectSubscription
|
||||
val cSub = c.expectSubscription
|
||||
cSub.request(100)
|
||||
|
|
@ -49,7 +49,7 @@ class FlowGroupedWithinSpec extends AkkaSpec with ScriptedTest {
|
|||
|
||||
"deliver bufferd elements onComplete before the timeout" in {
|
||||
val c = TestSubscriber.manualProbe[immutable.Seq[Int]]()
|
||||
Source(1 to 3).groupedWithin(1000, 10.second).to(Sink(c)).run()
|
||||
Source(1 to 3).groupedWithin(1000, 10.second).to(Sink.fromSubscriber(c)).run()
|
||||
val cSub = c.expectSubscription
|
||||
cSub.request(100)
|
||||
c.expectNext((1 to 3).toList)
|
||||
|
|
@ -61,7 +61,7 @@ class FlowGroupedWithinSpec extends AkkaSpec with ScriptedTest {
|
|||
val input = Iterator.from(1)
|
||||
val p = TestPublisher.manualProbe[Int]()
|
||||
val c = TestSubscriber.manualProbe[immutable.Seq[Int]]()
|
||||
Source(p).groupedWithin(1000, 1.second).to(Sink(c)).run()
|
||||
Source.fromPublisher(p).groupedWithin(1000, 1.second).to(Sink.fromSubscriber(c)).run()
|
||||
val pSub = p.expectSubscription
|
||||
val cSub = c.expectSubscription
|
||||
cSub.request(1)
|
||||
|
|
@ -81,7 +81,7 @@ class FlowGroupedWithinSpec extends AkkaSpec with ScriptedTest {
|
|||
"drop empty groups" in {
|
||||
val p = TestPublisher.manualProbe[Int]()
|
||||
val c = TestSubscriber.manualProbe[immutable.Seq[Int]]()
|
||||
Source(p).groupedWithin(1000, 500.millis).to(Sink(c)).run()
|
||||
Source.fromPublisher(p).groupedWithin(1000, 500.millis).to(Sink.fromSubscriber(c)).run()
|
||||
val pSub = p.expectSubscription
|
||||
val cSub = c.expectSubscription
|
||||
cSub.request(2)
|
||||
|
|
@ -103,7 +103,7 @@ class FlowGroupedWithinSpec extends AkkaSpec with ScriptedTest {
|
|||
val inputs = Iterator.from(1)
|
||||
val upstream = TestPublisher.probe[Int]()
|
||||
val downstream = TestSubscriber.probe[immutable.Seq[Int]]()
|
||||
Source(upstream).groupedWithin(3, 2.second).to(Sink(downstream)).run()
|
||||
Source.fromPublisher(upstream).groupedWithin(3, 2.second).to(Sink.fromSubscriber(downstream)).run()
|
||||
|
||||
downstream.request(2)
|
||||
downstream.expectNoMsg(1000.millis)
|
||||
|
|
|
|||
|
|
@ -33,7 +33,7 @@ class FlowIdleInjectSpec extends AkkaSpec {
|
|||
val upstream = TestPublisher.probe[Int]()
|
||||
val downstream = TestSubscriber.probe[Int]()
|
||||
|
||||
Source(upstream).keepAlive(1.second, () ⇒ 0).runWith(Sink(downstream))
|
||||
Source.fromPublisher(upstream).keepAlive(1.second, () ⇒ 0).runWith(Sink.fromSubscriber(downstream))
|
||||
|
||||
downstream.request(1)
|
||||
|
||||
|
|
@ -48,7 +48,7 @@ class FlowIdleInjectSpec extends AkkaSpec {
|
|||
val upstream = TestPublisher.probe[Int]()
|
||||
val downstream = TestSubscriber.probe[Int]()
|
||||
|
||||
(Source(1 to 10) ++ Source(upstream)).keepAlive(1.second, () ⇒ 0).runWith(Sink(downstream))
|
||||
(Source(1 to 10) ++ Source.fromPublisher(upstream)).keepAlive(1.second, () ⇒ 0).runWith(Sink.fromSubscriber(downstream))
|
||||
|
||||
downstream.request(10)
|
||||
downstream.expectNextN(1 to 10)
|
||||
|
|
@ -66,7 +66,7 @@ class FlowIdleInjectSpec extends AkkaSpec {
|
|||
val upstream = TestPublisher.probe[Int]()
|
||||
val downstream = TestSubscriber.probe[Int]()
|
||||
|
||||
Source(upstream).keepAlive(1.second, () ⇒ 0).runWith(Sink(downstream))
|
||||
Source.fromPublisher(upstream).keepAlive(1.second, () ⇒ 0).runWith(Sink.fromSubscriber(downstream))
|
||||
|
||||
downstream.ensureSubscription()
|
||||
downstream.expectNoMsg(1.5.second)
|
||||
|
|
@ -81,7 +81,7 @@ class FlowIdleInjectSpec extends AkkaSpec {
|
|||
val upstream = TestPublisher.probe[Int]()
|
||||
val downstream = TestSubscriber.probe[Int]()
|
||||
|
||||
(Source(1 to 10) ++ Source(upstream)).keepAlive(1.second, () ⇒ 0).runWith(Sink(downstream))
|
||||
(Source(1 to 10) ++ Source.fromPublisher(upstream)).keepAlive(1.second, () ⇒ 0).runWith(Sink.fromSubscriber(downstream))
|
||||
|
||||
downstream.request(10)
|
||||
downstream.expectNextN(1 to 10)
|
||||
|
|
@ -98,7 +98,7 @@ class FlowIdleInjectSpec extends AkkaSpec {
|
|||
val upstream = TestPublisher.probe[Int]()
|
||||
val downstream = TestSubscriber.probe[Int]()
|
||||
|
||||
Source(upstream).keepAlive(1.second, () ⇒ 0).runWith(Sink(downstream))
|
||||
Source.fromPublisher(upstream).keepAlive(1.second, () ⇒ 0).runWith(Sink.fromSubscriber(downstream))
|
||||
|
||||
downstream.ensureSubscription()
|
||||
downstream.expectNoMsg(1.5.second)
|
||||
|
|
@ -115,7 +115,7 @@ class FlowIdleInjectSpec extends AkkaSpec {
|
|||
val upstream = TestPublisher.probe[Int]()
|
||||
val downstream = TestSubscriber.probe[Int]()
|
||||
|
||||
(Source(1 to 10) ++ Source(upstream)).keepAlive(1.second, () ⇒ 0).runWith(Sink(downstream))
|
||||
(Source(1 to 10) ++ Source.fromPublisher(upstream)).keepAlive(1.second, () ⇒ 0).runWith(Sink.fromSubscriber(downstream))
|
||||
|
||||
downstream.request(10)
|
||||
downstream.expectNextN(1 to 10)
|
||||
|
|
@ -134,7 +134,7 @@ class FlowIdleInjectSpec extends AkkaSpec {
|
|||
val upstream = TestPublisher.probe[Int]()
|
||||
val downstream = TestSubscriber.probe[Int]()
|
||||
|
||||
Source(upstream).keepAlive(1.second, () ⇒ 0).runWith(Sink(downstream))
|
||||
Source.fromPublisher(upstream).keepAlive(1.second, () ⇒ 0).runWith(Sink.fromSubscriber(downstream))
|
||||
|
||||
downstream.request(2)
|
||||
downstream.expectNoMsg(500.millis)
|
||||
|
|
|
|||
|
|
@ -37,7 +37,7 @@ class FlowInitialDelaySpec extends AkkaSpec {
|
|||
|
||||
"properly ignore timer while backpressured" in Utils.assertAllStagesStopped {
|
||||
val probe = TestSubscriber.probe[Int]()
|
||||
Source(1 to 10).initialDelay(0.5.second).runWith(Sink(probe))
|
||||
Source(1 to 10).initialDelay(0.5.second).runWith(Sink.fromSubscriber(probe))
|
||||
|
||||
probe.ensureSubscription()
|
||||
probe.expectNoMsg(1.5.second)
|
||||
|
|
|
|||
|
|
@ -13,7 +13,7 @@ class FlowInterleaveSpec extends BaseTwoStreamsSetup {
|
|||
|
||||
override def setup(p1: Publisher[Int], p2: Publisher[Int]) = {
|
||||
val subscriber = TestSubscriber.probe[Outputs]()
|
||||
Source(p1).interleave(Source(p2), 2).runWith(Sink(subscriber))
|
||||
Source.fromPublisher(p1).interleave(Source.fromPublisher(p2), 2).runWith(Sink.fromSubscriber(subscriber))
|
||||
subscriber
|
||||
}
|
||||
|
||||
|
|
@ -21,7 +21,7 @@ class FlowInterleaveSpec extends BaseTwoStreamsSetup {
|
|||
|
||||
"work in the happy case" in assertAllStagesStopped {
|
||||
val probe = TestSubscriber.manualProbe[Int]()
|
||||
Source(0 to 3).interleave(Source(4 to 6), 2).interleave(Source(7 to 11), 3).runWith(Sink(probe))
|
||||
Source(0 to 3).interleave(Source(4 to 6), 2).interleave(Source(7 to 11), 3).runWith(Sink.fromSubscriber(probe))
|
||||
|
||||
val subscription = probe.expectSubscription()
|
||||
|
||||
|
|
@ -38,7 +38,7 @@ class FlowInterleaveSpec extends BaseTwoStreamsSetup {
|
|||
"work when segmentSize is not equal elements in stream" in assertAllStagesStopped {
|
||||
val probe = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
Source(0 to 2).interleave(Source(3 to 5), 2).runWith(Sink(probe))
|
||||
Source(0 to 2).interleave(Source(3 to 5), 2).runWith(Sink.fromSubscriber(probe))
|
||||
probe.expectSubscription().request(10)
|
||||
probe.expectNext(0, 1, 3, 4, 2, 5)
|
||||
probe.expectComplete()
|
||||
|
|
@ -47,7 +47,7 @@ class FlowInterleaveSpec extends BaseTwoStreamsSetup {
|
|||
"work with segmentSize = 1" in assertAllStagesStopped {
|
||||
val probe = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
Source(0 to 2).interleave(Source(3 to 5), 1).runWith(Sink(probe))
|
||||
Source(0 to 2).interleave(Source(3 to 5), 1).runWith(Sink.fromSubscriber(probe))
|
||||
probe.expectSubscription().request(10)
|
||||
probe.expectNext(0, 3, 1, 4, 2, 5)
|
||||
probe.expectComplete()
|
||||
|
|
@ -59,13 +59,13 @@ class FlowInterleaveSpec extends BaseTwoStreamsSetup {
|
|||
|
||||
"not work when segmentSize > than stream elements" in assertAllStagesStopped {
|
||||
val probe = TestSubscriber.manualProbe[Int]()
|
||||
Source(0 to 2).interleave(Source(3 to 15), 10).runWith(Sink(probe))
|
||||
Source(0 to 2).interleave(Source(3 to 15), 10).runWith(Sink.fromSubscriber(probe))
|
||||
probe.expectSubscription().request(25)
|
||||
(0 to 15).foreach(probe.expectNext)
|
||||
probe.expectComplete()
|
||||
|
||||
val probe2 = TestSubscriber.manualProbe[Int]()
|
||||
Source(1 to 20).interleave(Source(21 to 25), 10).runWith(Sink(probe2))
|
||||
Source(1 to 20).interleave(Source(21 to 25), 10).runWith(Sink.fromSubscriber(probe2))
|
||||
probe2.expectSubscription().request(100)
|
||||
(1 to 10).foreach(probe2.expectNext)
|
||||
(21 to 25).foreach(probe2.expectNext)
|
||||
|
|
@ -126,8 +126,8 @@ class FlowInterleaveSpec extends BaseTwoStreamsSetup {
|
|||
val up2 = TestPublisher.manualProbe[Int]()
|
||||
val down = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
val (graphSubscriber1, graphSubscriber2) = Source.subscriber[Int]
|
||||
.interleaveMat(Source.subscriber[Int], 2)((_, _)).toMat(Sink(down))(Keep.left).run
|
||||
val (graphSubscriber1, graphSubscriber2) = Source.asSubscriber[Int]
|
||||
.interleaveMat(Source.asSubscriber[Int], 2)((_, _)).toMat(Sink.fromSubscriber(down))(Keep.left).run
|
||||
|
||||
val downstream = down.expectSubscription()
|
||||
downstream.cancel()
|
||||
|
|
|
|||
|
|
@ -16,7 +16,7 @@ import org.reactivestreams.Subscriber
|
|||
class FlowIteratorSpec extends AbstractFlowIteratorSpec {
|
||||
override def testName = "A Flow based on an iterator producing function"
|
||||
override def createSource(elements: Int): Source[Int, Unit] =
|
||||
Source(() ⇒ (1 to elements).iterator)
|
||||
Source.fromIterator(() ⇒ (1 to elements).iterator)
|
||||
}
|
||||
|
||||
class FlowIterableSpec extends AbstractFlowIteratorSpec {
|
||||
|
|
@ -31,7 +31,7 @@ class FlowIterableSpec extends AbstractFlowIteratorSpec {
|
|||
override def iterator: Iterator[Int] =
|
||||
(1 to 3).iterator.map(x ⇒ if (x == 2) throw new IllegalStateException("not two") else x)
|
||||
}
|
||||
val p = Source(iterable).runWith(Sink.publisher(false))
|
||||
val p = Source(iterable).runWith(Sink.asPublisher(false))
|
||||
val c = TestSubscriber.manualProbe[Int]()
|
||||
p.subscribe(c)
|
||||
val sub = c.expectSubscription()
|
||||
|
|
@ -48,7 +48,7 @@ class FlowIterableSpec extends AbstractFlowIteratorSpec {
|
|||
val iterable = new immutable.Iterable[Int] {
|
||||
override def iterator: Iterator[Int] = throw new IllegalStateException("no good iterator")
|
||||
}
|
||||
val p = Source(iterable).runWith(Sink.publisher(false))
|
||||
val p = Source(iterable).runWith(Sink.asPublisher(false))
|
||||
val c = TestSubscriber.manualProbe[Int]()
|
||||
p.subscribe(c)
|
||||
c.expectSubscriptionAndError().getMessage should be("no good iterator")
|
||||
|
|
@ -62,7 +62,7 @@ class FlowIterableSpec extends AbstractFlowIteratorSpec {
|
|||
override def next(): Int = -1
|
||||
}
|
||||
}
|
||||
val p = Source(iterable).runWith(Sink.publisher(false))
|
||||
val p = Source(iterable).runWith(Sink.asPublisher(false))
|
||||
val c = TestSubscriber.manualProbe[Int]()
|
||||
p.subscribe(c)
|
||||
c.expectSubscriptionAndError().getMessage should be("no next")
|
||||
|
|
@ -84,7 +84,7 @@ abstract class AbstractFlowIteratorSpec extends AkkaSpec {
|
|||
|
||||
testName must {
|
||||
"produce elements" in assertAllStagesStopped {
|
||||
val p = createSource(3).runWith(Sink.publisher(false))
|
||||
val p = createSource(3).runWith(Sink.asPublisher(false))
|
||||
val c = TestSubscriber.manualProbe[Int]()
|
||||
p.subscribe(c)
|
||||
val sub = c.expectSubscription()
|
||||
|
|
@ -98,7 +98,7 @@ abstract class AbstractFlowIteratorSpec extends AkkaSpec {
|
|||
}
|
||||
|
||||
"complete empty" in assertAllStagesStopped {
|
||||
val p = createSource(0).runWith(Sink.publisher(false))
|
||||
val p = createSource(0).runWith(Sink.asPublisher(false))
|
||||
val c = TestSubscriber.manualProbe[Int]()
|
||||
p.subscribe(c)
|
||||
c.expectSubscriptionAndComplete()
|
||||
|
|
@ -106,7 +106,7 @@ abstract class AbstractFlowIteratorSpec extends AkkaSpec {
|
|||
}
|
||||
|
||||
"produce elements with multiple subscribers" in assertAllStagesStopped {
|
||||
val p = createSource(3).runWith(Sink.publisher(true))
|
||||
val p = createSource(3).runWith(Sink.asPublisher(true))
|
||||
val c1 = TestSubscriber.manualProbe[Int]()
|
||||
val c2 = TestSubscriber.manualProbe[Int]()
|
||||
p.subscribe(c1)
|
||||
|
|
@ -130,7 +130,7 @@ abstract class AbstractFlowIteratorSpec extends AkkaSpec {
|
|||
}
|
||||
|
||||
"produce elements to later subscriber" in assertAllStagesStopped {
|
||||
val p = createSource(3).runWith(Sink.publisher(true))
|
||||
val p = createSource(3).runWith(Sink.asPublisher(true))
|
||||
val c1 = TestSubscriber.manualProbe[Int]()
|
||||
val c2 = TestSubscriber.manualProbe[Int]()
|
||||
p.subscribe(c1)
|
||||
|
|
@ -153,7 +153,7 @@ abstract class AbstractFlowIteratorSpec extends AkkaSpec {
|
|||
}
|
||||
|
||||
"produce elements with one transformation step" in assertAllStagesStopped {
|
||||
val p = createSource(3).map(_ * 2).runWith(Sink.publisher(false))
|
||||
val p = createSource(3).map(_ * 2).runWith(Sink.asPublisher(false))
|
||||
val c = TestSubscriber.manualProbe[Int]()
|
||||
p.subscribe(c)
|
||||
val sub = c.expectSubscription()
|
||||
|
|
@ -165,7 +165,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(false))
|
||||
val p = createSource(4).filter(_ % 2 == 0).map(_ * 2).runWith(Sink.asPublisher(false))
|
||||
val c = TestSubscriber.manualProbe[Int]()
|
||||
p.subscribe(c)
|
||||
val sub = c.expectSubscription()
|
||||
|
|
@ -176,7 +176,7 @@ abstract class AbstractFlowIteratorSpec extends AkkaSpec {
|
|||
}
|
||||
|
||||
"not produce after cancel" in assertAllStagesStopped {
|
||||
val p = createSource(3).runWith(Sink.publisher(false))
|
||||
val p = createSource(3).runWith(Sink.asPublisher(false))
|
||||
val c = TestSubscriber.manualProbe[Int]()
|
||||
p.subscribe(c)
|
||||
val sub = c.expectSubscription()
|
||||
|
|
|
|||
|
|
@ -32,7 +32,7 @@ class FlowJoinSpec extends AkkaSpec(ConfigFactory.parseString("akka.loglevel=INF
|
|||
val broadcast = b.add(Broadcast[Int](2))
|
||||
source ~> merge.in(0)
|
||||
merge.out ~> broadcast.in
|
||||
broadcast.out(0).grouped(1000) ~> Sink(probe)
|
||||
broadcast.out(0).grouped(1000) ~> Sink.fromSubscriber(probe)
|
||||
|
||||
FlowShape(merge.in(1), broadcast.out(1))
|
||||
})
|
||||
|
|
|
|||
|
|
@ -36,7 +36,7 @@ class FlowMapAsyncSpec extends AkkaSpec with ScalaFutures {
|
|||
"produce future elements" in assertAllStagesStopped {
|
||||
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 p = Source(1 to 3).mapAsync(4)(n ⇒ Future(n)).runWith(Sink.fromSubscriber(c))
|
||||
val sub = c.expectSubscription()
|
||||
sub.request(2)
|
||||
c.expectNext(1)
|
||||
|
|
@ -53,7 +53,7 @@ class FlowMapAsyncSpec extends AkkaSpec with ScalaFutures {
|
|||
val p = Source(1 to 50).mapAsync(4)(n ⇒ Future {
|
||||
Thread.sleep(ThreadLocalRandom.current().nextInt(1, 10))
|
||||
n
|
||||
}).to(Sink(c)).run()
|
||||
}).to(Sink.fromSubscriber(c)).run()
|
||||
val sub = c.expectSubscription()
|
||||
sub.request(1000)
|
||||
for (n ← 1 to 50) c.expectNext(n)
|
||||
|
|
@ -67,7 +67,7 @@ class FlowMapAsyncSpec extends AkkaSpec with ScalaFutures {
|
|||
val p = Source(1 to 20).mapAsync(8)(n ⇒ Future {
|
||||
probe.ref ! n
|
||||
n
|
||||
}).to(Sink(c)).run()
|
||||
}).to(Sink.fromSubscriber(c)).run()
|
||||
val sub = c.expectSubscription()
|
||||
probe.expectNoMsg(500.millis)
|
||||
sub.request(1)
|
||||
|
|
@ -94,7 +94,7 @@ class FlowMapAsyncSpec extends AkkaSpec with ScalaFutures {
|
|||
Await.ready(latch, 10.seconds)
|
||||
n
|
||||
}
|
||||
}).to(Sink(c)).run()
|
||||
}).to(Sink.fromSubscriber(c)).run()
|
||||
val sub = c.expectSubscription()
|
||||
sub.request(10)
|
||||
c.expectError().getMessage should be("err1")
|
||||
|
|
@ -113,7 +113,7 @@ class FlowMapAsyncSpec extends AkkaSpec with ScalaFutures {
|
|||
n
|
||||
}
|
||||
}).
|
||||
to(Sink(c)).run()
|
||||
to(Sink.fromSubscriber(c)).run()
|
||||
val sub = c.expectSubscription()
|
||||
sub.request(10)
|
||||
c.expectError().getMessage should be("err2")
|
||||
|
|
@ -129,7 +129,7 @@ class FlowMapAsyncSpec extends AkkaSpec with ScalaFutures {
|
|||
else n
|
||||
})
|
||||
.withAttributes(supervisionStrategy(resumingDecider))
|
||||
.to(Sink(c)).run()
|
||||
.to(Sink.fromSubscriber(c)).run()
|
||||
val sub = c.expectSubscription()
|
||||
sub.request(10)
|
||||
for (n ← List(1, 2, 4, 5)) c.expectNext(n)
|
||||
|
|
@ -169,7 +169,7 @@ class FlowMapAsyncSpec extends AkkaSpec with ScalaFutures {
|
|||
if (n == 3) throw new RuntimeException("err4") with NoStackTrace
|
||||
else Future(n))
|
||||
.withAttributes(supervisionStrategy(resumingDecider))
|
||||
.to(Sink(c)).run()
|
||||
.to(Sink.fromSubscriber(c)).run()
|
||||
val sub = c.expectSubscription()
|
||||
sub.request(10)
|
||||
for (n ← List(1, 2, 4, 5)) c.expectNext(n)
|
||||
|
|
@ -178,7 +178,7 @@ class FlowMapAsyncSpec extends AkkaSpec with ScalaFutures {
|
|||
|
||||
"signal NPE when future is completed with null" in {
|
||||
val c = TestSubscriber.manualProbe[String]()
|
||||
val p = Source(List("a", "b")).mapAsync(4)(elem ⇒ Future.successful(null)).to(Sink(c)).run()
|
||||
val p = Source(List("a", "b")).mapAsync(4)(elem ⇒ Future.successful(null)).to(Sink.fromSubscriber(c)).run()
|
||||
val sub = c.expectSubscription()
|
||||
sub.request(10)
|
||||
c.expectError().getMessage should be(ReactiveStreamsCompliance.ElementMustNotBeNullMsg)
|
||||
|
|
@ -189,7 +189,7 @@ class FlowMapAsyncSpec extends AkkaSpec with ScalaFutures {
|
|||
val p = Source(List("a", "b", "c"))
|
||||
.mapAsync(4)(elem ⇒ if (elem == "b") Future.successful(null) else Future.successful(elem))
|
||||
.withAttributes(supervisionStrategy(resumingDecider))
|
||||
.to(Sink(c)).run()
|
||||
.to(Sink.fromSubscriber(c)).run()
|
||||
val sub = c.expectSubscription()
|
||||
sub.request(10)
|
||||
for (elem ← List("a", "c")) c.expectNext(elem)
|
||||
|
|
@ -200,7 +200,7 @@ class FlowMapAsyncSpec extends AkkaSpec with ScalaFutures {
|
|||
val pub = TestPublisher.manualProbe[Int]()
|
||||
val sub = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
Source(pub).mapAsync(4)(Future.successful).runWith(Sink(sub))
|
||||
Source.fromPublisher(pub).mapAsync(4)(Future.successful).runWith(Sink.fromSubscriber(sub))
|
||||
|
||||
val upstream = pub.expectSubscription()
|
||||
upstream.expectRequest()
|
||||
|
|
|
|||
|
|
@ -37,7 +37,7 @@ class FlowMapAsyncUnorderedSpec extends AkkaSpec with ScalaFutures with Conversi
|
|||
val p = Source(1 to 4).mapAsyncUnordered(4)(n ⇒ Future {
|
||||
Await.ready(latch(n), 5.seconds)
|
||||
n
|
||||
}).to(Sink(c)).run()
|
||||
}).to(Sink.fromSubscriber(c)).run()
|
||||
val sub = c.expectSubscription()
|
||||
sub.request(5)
|
||||
latch(2).countDown()
|
||||
|
|
@ -58,7 +58,7 @@ class FlowMapAsyncUnorderedSpec extends AkkaSpec with ScalaFutures with Conversi
|
|||
val p = Source(1 to 20).mapAsyncUnordered(4)(n ⇒ Future {
|
||||
probe.ref ! n
|
||||
n
|
||||
}).to(Sink(c)).run()
|
||||
}).to(Sink.fromSubscriber(c)).run()
|
||||
val sub = c.expectSubscription()
|
||||
c.expectNoMsg(200.millis)
|
||||
probe.expectNoMsg(Duration.Zero)
|
||||
|
|
@ -86,7 +86,7 @@ class FlowMapAsyncUnorderedSpec extends AkkaSpec with ScalaFutures with Conversi
|
|||
Await.ready(latch, 10.seconds)
|
||||
n
|
||||
}
|
||||
}).to(Sink(c)).run()
|
||||
}).to(Sink.fromSubscriber(c)).run()
|
||||
val sub = c.expectSubscription()
|
||||
sub.request(10)
|
||||
c.expectError.getMessage should be("err1")
|
||||
|
|
@ -105,7 +105,7 @@ class FlowMapAsyncUnorderedSpec extends AkkaSpec with ScalaFutures with Conversi
|
|||
n
|
||||
}
|
||||
}).
|
||||
to(Sink(c)).run()
|
||||
to(Sink.fromSubscriber(c)).run()
|
||||
val sub = c.expectSubscription()
|
||||
sub.request(10)
|
||||
c.expectError.getMessage should be("err2")
|
||||
|
|
@ -166,7 +166,7 @@ class FlowMapAsyncUnorderedSpec extends AkkaSpec with ScalaFutures with Conversi
|
|||
|
||||
"signal NPE when future is completed with null" in {
|
||||
val c = TestSubscriber.manualProbe[String]()
|
||||
val p = Source(List("a", "b")).mapAsyncUnordered(4)(elem ⇒ Future.successful(null)).to(Sink(c)).run()
|
||||
val p = Source(List("a", "b")).mapAsyncUnordered(4)(elem ⇒ Future.successful(null)).to(Sink.fromSubscriber(c)).run()
|
||||
val sub = c.expectSubscription()
|
||||
sub.request(10)
|
||||
c.expectError.getMessage should be(ReactiveStreamsCompliance.ElementMustNotBeNullMsg)
|
||||
|
|
@ -177,7 +177,7 @@ class FlowMapAsyncUnorderedSpec extends AkkaSpec with ScalaFutures with Conversi
|
|||
val p = Source(List("a", "b", "c"))
|
||||
.mapAsyncUnordered(4)(elem ⇒ if (elem == "b") Future.successful(null) else Future.successful(elem))
|
||||
.withAttributes(supervisionStrategy(resumingDecider))
|
||||
.to(Sink(c)).run()
|
||||
.to(Sink.fromSubscriber(c)).run()
|
||||
val sub = c.expectSubscription()
|
||||
sub.request(10)
|
||||
c.expectNextUnordered("a", "c")
|
||||
|
|
@ -188,7 +188,7 @@ class FlowMapAsyncUnorderedSpec extends AkkaSpec with ScalaFutures with Conversi
|
|||
val pub = TestPublisher.manualProbe[Int]()
|
||||
val sub = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
Source(pub).mapAsyncUnordered(4)(Future.successful).runWith(Sink(sub))
|
||||
Source.fromPublisher(pub).mapAsyncUnordered(4)(Future.successful).runWith(Sink.fromSubscriber(sub))
|
||||
|
||||
val upstream = pub.expectSubscription()
|
||||
upstream.expectRequest()
|
||||
|
|
|
|||
Some files were not shown because too many files have changed in this diff Show more
Loading…
Add table
Add a link
Reference in a new issue