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
|
||||
========================================================
|
||||
|
|
|
|||
|
|
@ -108,7 +108,7 @@ 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
|
||||
^^^^^^^^^^^^^^^
|
||||
|
|
@ -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
|
||||
========================================================
|
||||
|
||||
|
|
|
|||
|
|
@ -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?
|
||||
-----------------------------------------------
|
||||
|
|
|
|||
|
|
@ -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(())
|
||||
|
|
|
|||
|
|
@ -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))
|
||||
|
|
|
|||
|
|
@ -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,7 +67,7 @@ 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>>() {
|
||||
|
|
@ -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));
|
||||
|
|
|
|||
|
|
@ -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));
|
||||
|
|
@ -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);
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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