=str #19128 #19127 rename Source/Sink factory apply overloads

This commit is contained in:
Martynas Mickevičius 2015-12-17 11:48:30 +02:00
parent d7b45e0fc3
commit e6e476d82a
146 changed files with 910 additions and 740 deletions

View file

@ -99,7 +99,7 @@ class FlowMapBenchmark {
} }
} }
flow = mkMaps(Source(syncTestPublisher), numberOfMapOps) { flow = mkMaps(Source.fromPublisher(syncTestPublisher), numberOfMapOps) {
if (UseGraphStageIdentity) if (UseGraphStageIdentity)
GraphStages.identity[Int] GraphStages.identity[Int]
else else

View file

@ -34,7 +34,7 @@ class FileSourcesBenchmark {
val f = File.createTempFile(getClass.getName, ".bench.tmp") val f = File.createTempFile(getClass.getName, ".bench.tmp")
f.deleteOnExit() f.deleteOnExit()
val ft = Source(() Iterator.continually(line)) val ft = Source.fromIterator(() Iterator.continually(line))
.take(10 * 39062) // adjust as needed .take(10 * 39062) // adjust as needed
.runWith(Sink.file(f)) .runWith(Sink.file(f))
Await.result(ft, 30.seconds) Await.result(ft, 30.seconds)
@ -53,7 +53,7 @@ class FileSourcesBenchmark {
def setup() { def setup() {
fileChannelSource = Source.file(file, bufSize) fileChannelSource = Source.file(file, bufSize)
fileInputStreamSource = Source.inputStream(() new FileInputStream(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 @TearDown

View file

@ -1,12 +1,16 @@
package docs; package docs;
import akka.actor.ActorSystem;
import akka.actor.Cancellable; import akka.actor.Cancellable;
import akka.http.javadsl.model.Uri; import akka.http.javadsl.model.Uri;
import akka.dispatch.Futures;
import akka.japi.function.Creator; import akka.japi.function.Creator;
import akka.japi.Pair; import akka.japi.Pair;
import akka.japi.function.Function; import akka.japi.function.Function;
import akka.stream.*; import akka.stream.*;
import akka.stream.javadsl.*; import akka.stream.javadsl.*;
import akka.stream.testkit.TestPublisher;
import akka.stream.testkit.TestSubscriber;
import akka.util.ByteString; import akka.util.ByteString;
import scala.Option; import scala.Option;
import scala.concurrent.Future; import scala.concurrent.Future;
@ -14,6 +18,9 @@ import scala.concurrent.duration.FiniteDuration;
import scala.concurrent.Promise; import scala.concurrent.Promise;
import scala.runtime.BoxedUnit; import scala.runtime.BoxedUnit;
import org.reactivestreams.Publisher;
import org.reactivestreams.Subscriber;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.io.InputStream; import java.io.InputStream;
@ -25,6 +32,7 @@ public class MigrationsJava {
// This is compile-only code, no need for actually running anything. // This is compile-only code, no need for actually running anything.
public static ActorMaterializer mat = null; public static ActorMaterializer mat = null;
public static ActorSystem sys = null;
public static class SomeInputStream extends InputStream { public static class SomeInputStream extends InputStream {
public SomeInputStream() {} public SomeInputStream() {}
@ -133,12 +141,34 @@ public class MigrationsJava {
// Complete the promise with an empty option to emulate the old lazyEmpty // Complete the promise with an empty option to emulate the old lazyEmpty
promise.trySuccess(scala.Option.empty()); 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(0, TimeUnit.MILLISECONDS),
FiniteDuration.create(200, TimeUnit.MILLISECONDS), FiniteDuration.create(200, TimeUnit.MILLISECONDS),
"tick"); "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 //#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 //#empty-flow
Flow<Integer, Integer, BoxedUnit> emptyFlow = Flow.<Integer>create(); Flow<Integer, Integer, BoxedUnit> emptyFlow = Flow.<Integer>create();
// or // or

View file

@ -224,13 +224,14 @@ should be replaced by
Source constructor name changes 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 zero elements by providing an ``Option``. This is different from ``lazyEmpty`` which only allowed completion to be
sent, but no elements. sent, but no elements.
The ``from()`` overload on ``Source`` that provide a tick source (``Source.from(delay,interval,tick)``) The ``from()`` overload on ``Source`` has been refactored to separate methods to reduce the number of overloads and
is replaced by the named method ``Source.tick()`` to reduce the number of overloads and to make the function more make source creation more discoverable.
discoverable.
``Source.subscriber`` has been renamed to ``Source.asSubscriber``.
Update procedure 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 1. All uses of ``Source.lazyEmpty`` should be replaced by ``Source.maybe`` and the returned ``Promise`` completed with
a ``None`` (an empty ``Option``) a ``None`` (an empty ``Option``)
2. Replace all uses of ``Source.from(delay,interval,tick)`` with the method ``Source.tick(delay,interval,tick)`` 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 Example
^^^^^^^ ^^^^^^^
@ -250,15 +254,51 @@ Example
promise.trySuccess(BoxedUnit.UNIT); promise.trySuccess(BoxedUnit.UNIT);
// This no longer works! // 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(0, TimeUnit.MILLISECONDS),
FiniteDuration.create(200, TimeUnit.MILLISECONDS), FiniteDuration.create(200, TimeUnit.MILLISECONDS),
"tick"); "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 should be replaced by
.. includecode:: code/docs/MigrationsJava.java#source-creators .. 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 ``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 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). 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 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 Update procedure
---------------- ----------------
1. Replace all occurences of ``Sink.publisher`` with ``Sink.publisher(false)`` 1. Replace all occurences of ``Sink.publisher`` with ``Sink.asPublisher(false)``
2. Replace all occurences of ``Sink.fanoutPublisher`` with ``Sink.publisher(true)`` 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 FlexiMerge an FlexiRoute has been replaced by GraphStage
======================================================== ========================================================

View file

@ -8,7 +8,7 @@ Integrating with Actors
======================= =======================
For piping the elements of a stream as messages to an ordinary actor you can use the For piping the elements of a stream as messages to an ordinary actor you can use the
``Sink.actorRef``. Messages can be sent to a stream via the :class:`ActorRef` that is ``Sink.actorRef``. Messages can be sent to a stream via the :class:`ActorRef` that is
materialized by ``Source.actorRef``. materialized by ``Source.actorRef``.
For more advanced use cases the :class:`ActorPublisher` and :class:`ActorSubscriber` traits are For more advanced use cases the :class:`ActorPublisher` and :class:`ActorSubscriber` traits are
@ -32,8 +32,8 @@ Akka Streams :class:`Source` or :class:`Sink`.
Source.actorRef Source.actorRef
^^^^^^^^^^^^^^^ ^^^^^^^^^^^^^^^
Messages sent to the actor that is materialized by ``Source.actorRef`` will be emitted to the Messages sent to the actor that is materialized by ``Source.actorRef`` will be emitted to the
stream if there is demand from downstream, otherwise they will be buffered until request for stream if there is demand from downstream, otherwise they will be buffered until request for
demand is received. demand is received.
Depending on the defined :class:`OverflowStrategy` it might drop elements if there is no space Depending on the defined :class:`OverflowStrategy` it might drop elements if there is no space
@ -44,7 +44,7 @@ actor interface.
The stream can be completed successfully by sending ``akka.actor.PoisonPill`` or The stream can be completed successfully by sending ``akka.actor.PoisonPill`` or
``akka.actor.Status.Success`` to the actor reference. ``akka.actor.Status.Success`` to the actor reference.
The stream can be completed with failure by sending ``akka.actor.Status.Failure`` to the The stream can be completed with failure by sending ``akka.actor.Status.Failure`` to the
actor reference. actor reference.
The actor will be stopped when the stream is completed, failed or cancelled from downstream, The actor will be stopped when the stream is completed, failed or cancelled from downstream,
@ -108,12 +108,12 @@ This is how it can be used as input :class:`Source` to a :class:`Flow`:
.. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/ActorPublisherDocTest.java#actor-publisher-usage .. 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 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 ActorSubscriber
^^^^^^^^^^^^^^^ ^^^^^^^^^^^^^^^
Extend :class:`akka.stream.actor.AbstractActorSubscriber` to make your class a stream subscriber with Extend :class:`akka.stream.actor.AbstractActorSubscriber` to make your class a stream subscriber with
full control of stream back pressure. It will receive full control of stream back pressure. It will receive
``ActorSubscriberMessage.OnNext``, ``ActorSubscriberMessage.OnComplete`` and ``ActorSubscriberMessage.OnError`` ``ActorSubscriberMessage.OnNext``, ``ActorSubscriberMessage.OnComplete`` and ``ActorSubscriberMessage.OnError``
messages from the stream. It can also receive other, non-stream messages, in the same way as any actor. messages from the stream. It can also receive other, non-stream messages, in the same way as any actor.
@ -414,7 +414,7 @@ by using the Publisher-:class:`Sink`:
.. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/ReactiveStreamsDocTest.java#source-publisher .. 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`. Additional subscription attempts will be rejected with an :class:`IllegalStateException`.
A publisher that supports multiple subscribers using fan-out/broadcasting is created as follows: A publisher that supports multiple subscribers using fan-out/broadcasting is created as follows:

View file

@ -6,11 +6,11 @@ import akka.http.scaladsl.model.Uri
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.stream._ import akka.stream._
import akka.stream.stage.{ OutHandler, InHandler, GraphStageLogic, GraphStage } 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.{ Future, ExecutionContext, Promise }
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.util.{ Failure, Success, Try } import scala.util.{ Failure, Random, Success, Try }
class MigrationsScala extends AkkaSpec { class MigrationsScala extends AkkaSpec {
@ -110,8 +110,26 @@ class MigrationsScala extends AkkaSpec {
promise.trySuccess(Some(())) promise.trySuccess(Some(()))
val ticks = Source.tick(1.second, 3.seconds, "tick") 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 //#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 //#flatMapConcat
Flow[Source[Int, Any]].flatMapConcat(identity) Flow[Source[Int, Any]].flatMapConcat(identity)
//#flatMapConcat //#flatMapConcat

View file

@ -100,7 +100,7 @@ class FlowDocSpec extends AkkaSpec {
Source(List(1, 2, 3)) Source(List(1, 2, 3))
// Create a source from a Future // Create a source from a Future
Source(Future.successful("Hello Streams!")) Source.fromFuture(Future.successful("Hello Streams!"))
// Create a source from a single element // Create a source from a single element
Source.single("only one element") Source.single("only one element")

View file

@ -9,7 +9,7 @@ class GraphCyclesSpec extends AkkaSpec {
implicit val materializer = ActorMaterializer() implicit val materializer = ActorMaterializer()
"Cycle demonstration" must { "Cycle demonstration" must {
val source = Source(() => Iterator.from(0)) val source = Source.fromIterator(() => Iterator.from(0))
"include a deadlocked cycle" in { "include a deadlocked cycle" in {

View file

@ -33,7 +33,7 @@ class RateTransformationDocSpec extends AkkaSpec {
} }
//#conflate-summarize //#conflate-summarize
val fut = Source(() => Iterator.continually(Random.nextGaussian)) val fut = Source.fromIterator(() => Iterator.continually(Random.nextGaussian))
.via(statsFlow) .via(statsFlow)
.grouped(10) .grouped(10)
.runWith(Sink.head) .runWith(Sink.head)

View file

@ -41,7 +41,7 @@ class ReactiveStreamsDocSpec extends AkkaSpec {
val impl = new Fixture { val impl = new Fixture {
override def tweets: Publisher[Tweet] = override def tweets: Publisher[Tweet] =
TwitterStreamQuickstartDocSpec.tweets.runWith(Sink.publisher(false)) TwitterStreamQuickstartDocSpec.tweets.runWith(Sink.asPublisher(false))
override def storage = TestSubscriber.manualProbe[Author] override def storage = TestSubscriber.manualProbe[Author]
@ -66,7 +66,7 @@ class ReactiveStreamsDocSpec extends AkkaSpec {
val storage = impl.storage val storage = impl.storage
//#connect-all //#connect-all
Source(tweets).via(authors).to(Sink(storage)).run() Source.fromPublisher(tweets).via(authors).to(Sink.fromSubscriber(storage)).run()
//#connect-all //#connect-all
assertResult(storage) assertResult(storage)
@ -92,7 +92,7 @@ class ReactiveStreamsDocSpec extends AkkaSpec {
//#source-publisher //#source-publisher
val authorPublisher: Publisher[Author] = 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) authorPublisher.subscribe(storage)
//#source-publisher //#source-publisher
@ -107,8 +107,8 @@ class ReactiveStreamsDocSpec extends AkkaSpec {
//#source-fanoutPublisher //#source-fanoutPublisher
val authorPublisher: Publisher[Author] = val authorPublisher: Publisher[Author] =
Source(tweets).via(authors) Source.fromPublisher(tweets).via(authors)
.runWith(Sink.publisher(fanout = true)) .runWith(Sink.asPublisher(fanout = true))
authorPublisher.subscribe(storage) authorPublisher.subscribe(storage)
authorPublisher.subscribe(alert) authorPublisher.subscribe(alert)
@ -125,7 +125,7 @@ class ReactiveStreamsDocSpec extends AkkaSpec {
//#sink-subscriber //#sink-subscriber
val tweetSubscriber: Subscriber[Tweet] = val tweetSubscriber: Subscriber[Tweet] =
authors.to(Sink(storage)).runWith(Source.subscriber[Tweet]) authors.to(Sink.fromSubscriber(storage)).runWith(Source.asSubscriber[Tweet])
tweets.subscribe(tweetSubscriber) tweets.subscribe(tweetSubscriber)
//#sink-subscriber //#sink-subscriber

View file

@ -57,7 +57,7 @@ class StreamPartialFlowGraphDocSpec extends AkkaSpec {
// prepare graph elements // prepare graph elements
val zip = b.add(Zip[Int, Int]()) val zip = b.add(Zip[Int, Int]())
def ints = Source(() => Iterator.from(1)) def ints = Source.fromIterator(() => Iterator.from(1))
// connect the graph // connect the graph
ints.filter(_ % 2 != 0) ~> zip.in0 ints.filter(_ % 2 != 0) ~> zip.in0

View file

@ -19,8 +19,8 @@ class RecipeCollectingMetrics extends RecipeSpec {
// //
// val loadPub = TestPublisher.manualProbe[Int]() // val loadPub = TestPublisher.manualProbe[Int]()
// val tickPub = TestPublisher.manualProbe[Tick]() // val tickPub = TestPublisher.manualProbe[Tick]()
// val reportTicks = Source(tickPub) // val reportTicks = Source.fromPublisher(tickPub)
// val loadUpdates = Source(loadPub) // val loadUpdates = Source.fromPublisher(loadPub)
// val futureSink = Sink.head[immutable.Seq[String]] // val futureSink = Sink.head[immutable.Seq[String]]
// val sink = Flow[String].grouped(10).to(futureSink) // val sink = Flow[String].grouped(10).to(futureSink)
// //

View file

@ -13,15 +13,15 @@ class RecipeDroppyBroadcast extends RecipeSpec {
"Recipe for a droppy broadcast" must { "Recipe for a droppy broadcast" must {
"work" in { "work" in {
val pub = TestPublisher.probe[Int]() val pub = TestPublisher.probe[Int]()
val myElements = Source(pub) val myElements = Source.fromPublisher(pub)
val sub1 = TestSubscriber.manualProbe[Int]() val sub1 = TestSubscriber.manualProbe[Int]()
val sub2 = TestSubscriber.manualProbe[Int]() val sub2 = TestSubscriber.manualProbe[Int]()
val sub3 = TestSubscriber.probe[Int]() val sub3 = TestSubscriber.probe[Int]()
val futureSink = Sink.head[Seq[Int]] val futureSink = Sink.head[Seq[Int]]
val mySink1 = Sink(sub1) val mySink1 = Sink.fromSubscriber(sub1)
val mySink2 = Sink(sub2) val mySink2 = Sink.fromSubscriber(sub2)
val mySink3 = Sink(sub3) val mySink3 = Sink.fromSubscriber(sub3)
//#droppy-bcast //#droppy-bcast
val graph = RunnableGraph.fromGraph(GraphDSL.create(mySink1, mySink2, mySink3)((_, _, _)) { implicit b => val graph = RunnableGraph.fromGraph(GraphDSL.create(mySink1, mySink2, mySink3)((_, _, _)) { implicit b =>

View file

@ -94,15 +94,15 @@ class RecipeGlobalRateLimit extends RecipeSpec {
// Use a large period and emulate the timer by hand instead // Use a large period and emulate the timer by hand instead
val limiter = system.actorOf(Limiter.props(2, 100.days, 1), "limiter") val limiter = system.actorOf(Limiter.props(2, 100.days, 1), "limiter")
val source1 = Source(() => Iterator.continually("E1")).via(limitGlobal(limiter, 2.seconds)) val source1 = Source.fromIterator(() => Iterator.continually("E1")).via(limitGlobal(limiter, 2.seconds))
val source2 = Source(() => Iterator.continually("E2")).via(limitGlobal(limiter, 2.seconds)) val source2 = Source.fromIterator(() => Iterator.continually("E2")).via(limitGlobal(limiter, 2.seconds))
val probe = TestSubscriber.manualProbe[String]() val probe = TestSubscriber.manualProbe[String]()
RunnableGraph.fromGraph(GraphDSL.create() { implicit b => RunnableGraph.fromGraph(GraphDSL.create() { implicit b =>
import GraphDSL.Implicits._ import GraphDSL.Implicits._
val merge = b.add(Merge[String](2)) val merge = b.add(Merge[String](2))
source1 ~> merge ~> Sink(probe) source1 ~> merge ~> Sink.fromSubscriber(probe)
source2 ~> merge source2 ~> merge
ClosedShape ClosedShape
}).run() }).run()

View file

@ -54,8 +54,8 @@ class RecipeHold extends RecipeSpec {
val pub = TestPublisher.probe[Int]() val pub = TestPublisher.probe[Int]()
val sub = TestSubscriber.manualProbe[Int]() val sub = TestSubscriber.manualProbe[Int]()
val source = Source(pub) val source = Source.fromPublisher(pub)
val sink = Sink(sub) val sink = Sink.fromSubscriber(sub)
source.transform(() => new HoldWithInitial(0)).to(sink).run() source.transform(() => new HoldWithInitial(0)).to(sink).run()
@ -84,8 +84,8 @@ class RecipeHold extends RecipeSpec {
val pub = TestPublisher.probe[Int]() val pub = TestPublisher.probe[Int]()
val sub = TestSubscriber.manualProbe[Int]() val sub = TestSubscriber.manualProbe[Int]()
val source = Source(pub) val source = Source.fromPublisher(pub)
val sink = Sink(sub) val sink = Sink.fromSubscriber(sub)
source.transform(() => new HoldWithWait).to(sink).run() source.transform(() => new HoldWithWait).to(sink).run()

View file

@ -14,8 +14,8 @@ class RecipeManualTrigger extends RecipeSpec {
val elements = Source(List("1", "2", "3", "4")) val elements = Source(List("1", "2", "3", "4"))
val pub = TestPublisher.probe[Trigger]() val pub = TestPublisher.probe[Trigger]()
val sub = TestSubscriber.manualProbe[Message]() val sub = TestSubscriber.manualProbe[Message]()
val triggerSource = Source(pub) val triggerSource = Source.fromPublisher(pub)
val sink = Sink(sub) val sink = Sink.fromSubscriber(sub)
//#manually-triggered-stream //#manually-triggered-stream
val graph = RunnableGraph.fromGraph(GraphDSL.create() { implicit builder => 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 elements = Source(List("1", "2", "3", "4"))
val pub = TestPublisher.probe[Trigger]() val pub = TestPublisher.probe[Trigger]()
val sub = TestSubscriber.manualProbe[Message]() val sub = TestSubscriber.manualProbe[Message]()
val triggerSource = Source(pub) val triggerSource = Source.fromPublisher(pub)
val sink = Sink(sub) val sink = Sink.fromSubscriber(sub)
//#manually-triggered-stream-zipwith //#manually-triggered-stream-zipwith
val graph = RunnableGraph.fromGraph(GraphDSL.create() { implicit builder => val graph = RunnableGraph.fromGraph(GraphDSL.create() { implicit builder =>

View file

@ -15,8 +15,8 @@ class RecipeMissedTicks extends RecipeSpec {
val pub = TestPublisher.probe[Tick]() val pub = TestPublisher.probe[Tick]()
val sub = TestSubscriber.manualProbe[Int]() val sub = TestSubscriber.manualProbe[Int]()
val tickStream = Source(pub) val tickStream = Source.fromPublisher(pub)
val sink = Sink(sub) val sink = Sink.fromSubscriber(sub)
//#missed-ticks //#missed-ticks
val missedTicks: Flow[Tick, Int, Unit] = val missedTicks: Flow[Tick, Int, Unit] =

View file

@ -22,8 +22,8 @@ class RecipeSimpleDrop extends RecipeSpec {
val pub = TestPublisher.probe[Message]() val pub = TestPublisher.probe[Message]()
val sub = TestSubscriber.manualProbe[Message]() val sub = TestSubscriber.manualProbe[Message]()
val messageSource = Source(pub) val messageSource = Source.fromPublisher(pub)
val sink = Sink(sub) val sink = Sink.fromSubscriber(sub)
messageSource.via(realDroppyStream).to(sink).run() messageSource.via(realDroppyStream).to(sink).run()

View file

@ -220,13 +220,14 @@ should be replaced by
Source constructor name changes 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 zero elements by providing an ``Option``. This is different from ``lazyEmpty`` which only allowed completion to be
sent, but no elements. sent, but no elements.
The ``apply()`` overload on ``Source`` that provide a tick source (``Source(delay,interval,tick)``) The ``apply()`` overload on ``Source`` has been refactored to separate methods to reduce the number of overloads and
is replaced by the named method ``Source.tick()`` to reduce the number of overloads and to make the function more make source creation more discoverable.
discoverable.
``Source.subscriber`` has been renamed to ``Source.asSubscriber``.
Update procedure 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 1. All uses of ``Source.lazyEmpty`` should be replaced by ``Source.maybe`` and the returned ``Promise`` completed with
a ``None`` (an empty ``Option``) a ``None`` (an empty ``Option``)
2. Replace all uses of ``Source(delay,interval,tick)`` with the method ``Source.tick(delay,interval,tick)`` 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 Example
^^^^^^^ ^^^^^^^
@ -248,10 +253,45 @@ Example
// This no longer works! // This no longer works!
val ticks = Source(1.second, 3.seconds, "tick") 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 should be replaced by
.. includecode:: code/docs/MigrationsScala.scala#source-creators .. 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 ``flatten(FlattenStrategy)`` has been replaced by named counterparts
==================================================================== ====================================================================
@ -276,6 +316,35 @@ should be replaced by
.. includecode:: code/docs/MigrationsScala.scala#flatMapConcat .. 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 FlexiMerge an FlexiRoute has been replaced by GraphStage
======================================================== ========================================================

View file

@ -8,7 +8,7 @@ Integrating with Actors
======================= =======================
For piping the elements of a stream as messages to an ordinary actor you can use the For piping the elements of a stream as messages to an ordinary actor you can use the
``Sink.actorRef``. Messages can be sent to a stream via the :class:`ActorRef` that is ``Sink.actorRef``. Messages can be sent to a stream via the :class:`ActorRef` that is
materialized by ``Source.actorRef``. materialized by ``Source.actorRef``.
For more advanced use cases the :class:`ActorPublisher` and :class:`ActorSubscriber` traits are For more advanced use cases the :class:`ActorPublisher` and :class:`ActorSubscriber` traits are
@ -27,8 +27,8 @@ Akka Streams :class:`Source` or :class:`Sink`.
Source.actorRef Source.actorRef
^^^^^^^^^^^^^^^ ^^^^^^^^^^^^^^^
Messages sent to the actor that is materialized by ``Source.actorRef`` will be emitted to the Messages sent to the actor that is materialized by ``Source.actorRef`` will be emitted to the
stream if there is demand from downstream, otherwise they will be buffered until request for stream if there is demand from downstream, otherwise they will be buffered until request for
demand is received. demand is received.
Depending on the defined :class:`OverflowStrategy` it might drop elements if there is no space Depending on the defined :class:`OverflowStrategy` it might drop elements if there is no space
@ -39,7 +39,7 @@ actor interface.
The stream can be completed successfully by sending ``akka.actor.PoisonPill`` or The stream can be completed successfully by sending ``akka.actor.PoisonPill`` or
``akka.actor.Status.Success`` to the actor reference. ``akka.actor.Status.Success`` to the actor reference.
The stream can be completed with failure by sending ``akka.actor.Status.Failure`` to the The stream can be completed with failure by sending ``akka.actor.Status.Failure`` to the
actor reference. actor reference.
The actor will be stopped when the stream is completed, failed or cancelled from downstream, The actor will be stopped when the stream is completed, failed or cancelled from downstream,
@ -102,7 +102,7 @@ This is how it can be used as input :class:`Source` to a :class:`Flow`:
.. includecode:: code/docs/stream/ActorPublisherDocSpec.scala#actor-publisher-usage .. 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`. subscription attempts will be rejected with an :class:`IllegalStateException`.
ActorSubscriber ActorSubscriber
@ -409,7 +409,7 @@ by using the Publisher-:class:`Sink`:
.. includecode:: code/docs/stream/ReactiveStreamsDocSpec.scala#source-publisher .. 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`. Additional subscription attempts will be rejected with an :class:`IllegalStateException`.
A publisher that supports multiple subscribers using fan-out/broadcasting is created as follows: 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 .. includecode:: code/docs/stream/ReactiveStreamsDocSpec.scala#use-processor
Please note that a factory is necessary to achieve reusability of the resulting :class:`Flow`. Please note that a factory is necessary to achieve reusability of the resulting :class:`Flow`.

View file

@ -32,11 +32,11 @@ The process of materialization will often create specific objects that are usefu
Interoperation with other Reactive Streams implementations 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]`. 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? What shall users of streaming libraries expect?
----------------------------------------------- -----------------------------------------------
@ -53,7 +53,7 @@ The second rule allows a library to additionally provide nice sugar for the comm
.. note:: .. note::
One important consequence of this is that a reusable flow description cannot be bound to “live” resources, any connection to or allocation of such resources must be deferred until materialization time. Examples of “live” resources are already existing TCP connections, a multicast Publisher, etc.; a TickSource does not fall into this category if its timer is created only upon materialization (as is the case for our implementation). One important consequence of this is that a reusable flow description cannot be bound to “live” resources, any connection to or allocation of such resources must be deferred until materialization time. Examples of “live” resources are already existing TCP connections, a multicast Publisher, etc.; a TickSource does not fall into this category if its timer is created only upon materialization (as is the case for our implementation).
Exceptions from this need to be well-justified and carefully documented. Exceptions from this need to be well-justified and carefully documented.
Resulting Implementation Constraints Resulting Implementation Constraints

View file

@ -73,7 +73,7 @@ private class PoolInterfaceActor(hcps: HostConnectionPoolSetup,
new InetSocketAddress(host, port), settings, setup.log) new InetSocketAddress(host, port), settings, setup.log)
.named("PoolFlow") .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() activateIdleTimeoutIfNecessary()

View file

@ -372,9 +372,9 @@ private[http] object HttpServerBluePrint {
def websocketFlow: Flow[ByteString, ByteString, Any] = flow def websocketFlow: Flow[ByteString, ByteString, Any] = flow
def installHandler(handlerFlow: Flow[FrameEvent, FrameEvent, Any])(implicit mat: Materializer): Unit = def installHandler(handlerFlow: Flow[FrameEvent, FrameEvent, Any])(implicit mat: Materializer): Unit =
Source(sinkCell.value) Source.fromPublisher(sinkCell.value)
.via(handlerFlow) .via(handlerFlow)
.to(Sink(sourceCell.value)) .to(Sink.fromSubscriber(sourceCell.value))
.run() .run()
} }
} }

View file

@ -159,14 +159,14 @@ private[http] object StreamUtils {
case Nil Nil case Nil Nil
case Seq(one) Vector(input.via(one)) case Seq(one) Vector(input.via(one))
case multiple 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 val sources = transformers.map { flow
// Doubly wrap to ensure that subscription to the running publisher happens before the final sources // Doubly wrap to ensure that subscription to the running publisher happens before the final sources
// are exposed, so there is no race // 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 // 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 sources
} }
@ -315,7 +315,7 @@ private[http] object StreamUtils {
object OneTimeValve { object OneTimeValve {
def apply(): OneTimeValve = new OneTimeValve { def apply(): OneTimeValve = new OneTimeValve {
val promise = Promise[Unit]() 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 source[T]: Source[T, Unit] = _source.asInstanceOf[Source[T, Unit]] // safe, because source won't generate any elements
def open(): Unit = promise.success(()) def open(): Unit = promise.success(())
@ -331,4 +331,4 @@ private[http] class EnhancedByteStringSource[Mat](val byteStringStream: Source[B
byteStringStream.runFold(ByteString.empty)(_ ++ _) byteStringStream.runFold(ByteString.empty)(_ ++ _)
def utf8String(implicit materializer: Materializer, ec: ExecutionContext): Future[String] = def utf8String(implicit materializer: Materializer, ec: ExecutionContext): Future[String] =
join.map(_.utf8String) join.map(_.utf8String)
} }

View file

@ -54,7 +54,7 @@ public class WSEchoTestClientApp {
TextMessage.create("abc"), TextMessage.create("abc"),
TextMessage.create("def"), TextMessage.create("def"),
TextMessage.create("ghi") TextMessage.create("ghi")
)).concat(Source.from(delayedCompletion).drop(1)); )).concat(Source.fromFuture(delayedCompletion).drop(1));
Sink<Message, Future<List<String>>> echoSink = Sink<Message, Future<List<String>>> echoSink =
Flow.of(Message.class) Flow.of(Message.class)

View file

@ -33,7 +33,7 @@ class ClientCancellationSpec extends AkkaSpec("""
def testCase(connection: Flow[HttpRequest, HttpResponse, Any]): Unit = Utils.assertAllStagesStopped { def testCase(connection: Flow[HttpRequest, HttpResponse, Any]): Unit = Utils.assertAllStagesStopped {
val requests = TestPublisher.probe[HttpRequest]() val requests = TestPublisher.probe[HttpRequest]()
val responses = TestSubscriber.probe[HttpResponse]() val responses = TestSubscriber.probe[HttpResponse]()
Source(requests).via(connection).runWith(Sink(responses)) Source.fromPublisher(requests).via(connection).runWith(Sink.fromSubscriber(responses))
responses.request(1) responses.request(1)
requests.sendNext(HttpRequest()) requests.sendNext(HttpRequest())
responses.expectNext().entity.dataBytes.runWith(Sink.cancelled) responses.expectNext().entity.dataBytes.runWith(Sink.cancelled)

View file

@ -89,7 +89,7 @@ class ConnectionPoolSpec extends AkkaSpec("""
override def testServerHandler(connNr: Int): HttpRequest HttpResponse = { override def testServerHandler(connNr: Int): HttpRequest HttpResponse = {
case request @ HttpRequest(_, Uri.Path("/a"), _, _, _) 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 super.testServerHandler(connNr)(request) withEntity entity
case x super.testServerHandler(connNr)(x) case x super.testServerHandler(connNr)(x)
} }
@ -99,7 +99,7 @@ class ConnectionPoolSpec extends AkkaSpec("""
acceptIncomingConnection() acceptIncomingConnection()
val (Success(r1), 42) = responseOut.expectNext() val (Success(r1), 42) = responseOut.expectNext()
val responseEntityProbe = TestSubscriber.probe[ByteString]() val responseEntityProbe = TestSubscriber.probe[ByteString]()
r1.entity.dataBytes.runWith(Sink(responseEntityProbe)) r1.entity.dataBytes.runWith(Sink.fromSubscriber(responseEntityProbe))
responseEntityProbe.expectSubscription().request(2) responseEntityProbe.expectSubscription().request(2)
responseEntityPub.sendNext(ByteString("YEAH")) responseEntityPub.sendNext(ByteString("YEAH"))
responseEntityProbe.expectNext(ByteString("YEAH")) responseEntityProbe.expectNext(ByteString("YEAH"))
@ -131,7 +131,7 @@ class ConnectionPoolSpec extends AkkaSpec("""
val poolFlow = Http().cachedHostConnectionPool[Int](serverHostName, serverPort, settings = settings) val poolFlow = Http().cachedHostConnectionPool[Int](serverHostName, serverPort, settings = settings)
val N = 500 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 { val idSum = requestIds.map(id HttpRequest(uri = s"/r$id") -> id).via(poolFlow).map {
case (Success(response), id) case (Success(response), id)
requestUri(response) should endWith(s"/r$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) } Flow[SslTlsOutbound].collect[ByteString] { case SendBytes(x) mapServerSideOutboundRawBytes(x) }
.transform(StreamUtils.recover { case NoErrorComplete ByteString.empty }), .transform(StreamUtils.recover { case NoErrorComplete ByteString.empty }),
Flow[ByteString].map(SessionBytes(null, _))) 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 // TODO getHostString in Java7
Tcp().bind(serverEndpoint.getHostName, serverEndpoint.getPort, idleTimeout = serverSettings.timeouts.idleTimeout) Tcp().bind(serverEndpoint.getHostName, serverEndpoint.getPort, idleTimeout = serverSettings.timeouts.idleTimeout)
.map { c .map { c
@ -345,7 +345,7 @@ class ConnectionPoolSpec extends AkkaSpec("""
def flowTestBench[T, Mat](poolFlow: Flow[(HttpRequest, T), (Try[HttpResponse], T), Mat]) = { def flowTestBench[T, Mat](poolFlow: Flow[(HttpRequest, T), (Try[HttpResponse], T), Mat]) = {
val requestIn = TestPublisher.probe[(HttpRequest, T)]() val requestIn = TestPublisher.probe[(HttpRequest, T)]()
val responseOut = TestSubscriber.manualProbe[(Try[HttpResponse], 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() val responseOutSub = responseOut.expectSubscription()
(requestIn, responseOut, responseOutSub, hcp) (requestIn, responseOut, responseOutSub, hcp)
} }

View file

@ -24,7 +24,7 @@ class HighLevelOutgoingConnectionSpec extends AkkaSpec {
serverHostName, serverPort) serverHostName, serverPort)
val N = 100 val N = 100
val result = Source(() Iterator.from(1)) val result = Source.fromIterator(() Iterator.from(1))
.take(N) .take(N)
.map(id HttpRequest(uri = s"/r$id")) .map(id HttpRequest(uri = s"/r$id"))
.via(Http().outgoingConnection(serverHostName, serverPort)) .via(Http().outgoingConnection(serverHostName, serverPort))
@ -56,7 +56,7 @@ class HighLevelOutgoingConnectionSpec extends AkkaSpec {
}) })
val N = 100 val N = 100
val result = Source(() Iterator.from(1)) val result = Source.fromIterator(() Iterator.from(1))
.take(N) .take(N)
.map(id HttpRequest(uri = s"/r$id")) .map(id HttpRequest(uri = s"/r$id"))
.via(doubleConnection) .via(doubleConnection)

View file

@ -45,7 +45,7 @@ class LowLevelOutgoingConnectionSpec extends AkkaSpec("akka.loggers = []\n akka.
"has a request with default entity" in new TestSetup { "has a request with default entity" in new TestSetup {
val probe = TestPublisher.manualProbe[ByteString]() 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( expectWireData(
"""PUT / HTTP/1.1 """PUT / HTTP/1.1
|Host: example.com |Host: example.com
@ -90,7 +90,7 @@ class LowLevelOutgoingConnectionSpec extends AkkaSpec("akka.loggers = []\n akka.
ct shouldEqual ContentTypes.`application/octet-stream` ct shouldEqual ContentTypes.`application/octet-stream`
val probe = TestSubscriber.manualProbe[ChunkStreamPart]() val probe = TestSubscriber.manualProbe[ChunkStreamPart]()
chunks.runWith(Sink(probe)) chunks.runWith(Sink.fromSubscriber(probe))
val sub = probe.expectSubscription() val sub = probe.expectSubscription()
sendWireData("3\nABC\n") sendWireData("3\nABC\n")
@ -155,7 +155,7 @@ class LowLevelOutgoingConnectionSpec extends AkkaSpec("akka.loggers = []\n akka.
val HttpResponse(_, _, HttpEntity.Chunked(ct, chunks), _) = expectResponse() val HttpResponse(_, _, HttpEntity.Chunked(ct, chunks), _) = expectResponse()
val probe = TestSubscriber.manualProbe[ChunkStreamPart]() val probe = TestSubscriber.manualProbe[ChunkStreamPart]()
chunks.runWith(Sink(probe)) chunks.runWith(Sink.fromSubscriber(probe))
val sub = probe.expectSubscription() val sub = probe.expectSubscription()
sendWireData("3\nABC\n") 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 { "catch the request entity stream being shorter than the Content-Length" in new TestSetup {
val probe = TestPublisher.manualProbe[ByteString]() 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( expectWireData(
"""PUT / HTTP/1.1 """PUT / HTTP/1.1
|Host: example.com |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 { "catch the request entity stream being longer than the Content-Length" in new TestSetup {
val probe = TestPublisher.manualProbe[ByteString]() 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( expectWireData(
"""PUT / HTTP/1.1 """PUT / HTTP/1.1
|Host: example.com |Host: example.com
@ -274,7 +274,7 @@ class LowLevelOutgoingConnectionSpec extends AkkaSpec("akka.loggers = []\n akka.
ct shouldEqual ContentTypes.`application/octet-stream` ct shouldEqual ContentTypes.`application/octet-stream`
val probe = TestSubscriber.manualProbe[ChunkStreamPart]() val probe = TestSubscriber.manualProbe[ChunkStreamPart]()
chunks.runWith(Sink(probe)) chunks.runWith(Sink.fromSubscriber(probe))
val sub = probe.expectSubscription() val sub = probe.expectSubscription()
sendWireData("3\nABC\n") 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 RunnableGraph.fromGraph(GraphDSL.create(OutgoingConnectionBlueprint(Host("example.com"), settings, NoLogging)) { implicit b
client client
import GraphDSL.Implicits._ 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 } ~> Sink(netOut) client.out1 ~> Flow[SslTlsOutbound].collect { case SendBytes(x) x } ~> Sink.fromSubscriber(netOut)
Source(requests) ~> client.in1 Source.fromPublisher(requests) ~> client.in1
client.out2 ~> Sink(responses) client.out2 ~> Sink.fromSubscriber(responses)
ClosedShape ClosedShape
}).run() }).run()

View file

@ -484,7 +484,7 @@ class RequestParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
} }
.concatSubstreams .concatSubstreams
.flatMapConcat { x .flatMapConcat { x
Source { Source.fromFuture {
x match { x match {
case Right(request) compactEntity(request.entity).fast.map(x Right(request.withEntity(x))) case Right(request) compactEntity(request.entity).fast.map(x Right(request.withEntity(x)))
case Left(error) FastFuture.successful(Left(error)) case Left(error) FastFuture.successful(Left(error))

View file

@ -46,7 +46,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
inside(expectRequest()) { inside(expectRequest()) {
case HttpRequest(POST, _, _, HttpEntity.Default(_, 12, data), _) case HttpRequest(POST, _, _, HttpEntity.Default(_, 12, data), _)
val dataProbe = TestSubscriber.manualProbe[ByteString] val dataProbe = TestSubscriber.manualProbe[ByteString]
data.to(Sink(dataProbe)).run() data.to(Sink.fromSubscriber(dataProbe)).run()
val sub = dataProbe.expectSubscription() val sub = dataProbe.expectSubscription()
sub.request(10) sub.request(10)
dataProbe.expectNoMsg(50.millis) dataProbe.expectNoMsg(50.millis)
@ -89,7 +89,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
inside(expectRequest()) { inside(expectRequest()) {
case HttpRequest(POST, _, _, HttpEntity.Chunked(_, data), _) case HttpRequest(POST, _, _, HttpEntity.Chunked(_, data), _)
val dataProbe = TestSubscriber.manualProbe[ChunkStreamPart] val dataProbe = TestSubscriber.manualProbe[ChunkStreamPart]
data.to(Sink(dataProbe)).run() data.to(Sink.fromSubscriber(dataProbe)).run()
val sub = dataProbe.expectSubscription() val sub = dataProbe.expectSubscription()
sub.request(10) sub.request(10)
dataProbe.expectNext(Chunk(ByteString("abcdef"))) dataProbe.expectNext(Chunk(ByteString("abcdef")))
@ -140,7 +140,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
inside(expectRequest()) { inside(expectRequest()) {
case HttpRequest(POST, _, _, HttpEntity.Default(_, 12, data), _) case HttpRequest(POST, _, _, HttpEntity.Default(_, 12, data), _)
val dataProbe = TestSubscriber.manualProbe[ByteString] val dataProbe = TestSubscriber.manualProbe[ByteString]
data.to(Sink(dataProbe)).run() data.to(Sink.fromSubscriber(dataProbe)).run()
val sub = dataProbe.expectSubscription() val sub = dataProbe.expectSubscription()
sub.request(10) sub.request(10)
dataProbe.expectNext(ByteString("abcdef")) dataProbe.expectNext(ByteString("abcdef"))
@ -163,7 +163,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
inside(expectRequest()) { inside(expectRequest()) {
case HttpRequest(POST, _, _, HttpEntity.Chunked(_, data), _) case HttpRequest(POST, _, _, HttpEntity.Chunked(_, data), _)
val dataProbe = TestSubscriber.manualProbe[ChunkStreamPart] val dataProbe = TestSubscriber.manualProbe[ChunkStreamPart]
data.to(Sink(dataProbe)).run() data.to(Sink.fromSubscriber(dataProbe)).run()
val sub = dataProbe.expectSubscription() val sub = dataProbe.expectSubscription()
sub.request(10) sub.request(10)
dataProbe.expectNext(Chunk(ByteString("abcdef"))) dataProbe.expectNext(Chunk(ByteString("abcdef")))
@ -212,7 +212,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
inside(expectRequest()) { inside(expectRequest()) {
case HttpRequest(POST, _, _, HttpEntity.Default(_, 12, data), _) case HttpRequest(POST, _, _, HttpEntity.Default(_, 12, data), _)
val dataProbe = TestSubscriber.manualProbe[ByteString] val dataProbe = TestSubscriber.manualProbe[ByteString]
data.to(Sink(dataProbe)).run() data.to(Sink.fromSubscriber(dataProbe)).run()
val sub = dataProbe.expectSubscription() val sub = dataProbe.expectSubscription()
sub.request(10) sub.request(10)
dataProbe.expectNext(ByteString("abcdef")) dataProbe.expectNext(ByteString("abcdef"))
@ -249,7 +249,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
inside(expectRequest()) { inside(expectRequest()) {
case HttpRequest(POST, _, _, HttpEntity.Chunked(_, data), _) case HttpRequest(POST, _, _, HttpEntity.Chunked(_, data), _)
val dataProbe = TestSubscriber.manualProbe[ChunkStreamPart] val dataProbe = TestSubscriber.manualProbe[ChunkStreamPart]
data.to(Sink(dataProbe)).run() data.to(Sink.fromSubscriber(dataProbe)).run()
val sub = dataProbe.expectSubscription() val sub = dataProbe.expectSubscription()
sub.request(10) sub.request(10)
dataProbe.expectNext(Chunk(ByteString("abcdef"))) dataProbe.expectNext(Chunk(ByteString("abcdef")))
@ -285,7 +285,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
inside(expectRequest()) { inside(expectRequest()) {
case HttpRequest(POST, _, _, HttpEntity.Default(_, 12, data), _) case HttpRequest(POST, _, _, HttpEntity.Default(_, 12, data), _)
val dataProbe = TestSubscriber.manualProbe[ByteString] val dataProbe = TestSubscriber.manualProbe[ByteString]
data.to(Sink(dataProbe)).run() data.to(Sink.fromSubscriber(dataProbe)).run()
val sub = dataProbe.expectSubscription() val sub = dataProbe.expectSubscription()
sub.request(10) sub.request(10)
dataProbe.expectNext(ByteString("abcdef")) dataProbe.expectNext(ByteString("abcdef"))
@ -308,7 +308,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
inside(expectRequest()) { inside(expectRequest()) {
case HttpRequest(POST, _, _, HttpEntity.Chunked(_, data), _) case HttpRequest(POST, _, _, HttpEntity.Chunked(_, data), _)
val dataProbe = TestSubscriber.manualProbe[ChunkStreamPart] val dataProbe = TestSubscriber.manualProbe[ChunkStreamPart]
data.to(Sink(dataProbe)).run() data.to(Sink.fromSubscriber(dataProbe)).run()
val sub = dataProbe.expectSubscription() val sub = dataProbe.expectSubscription()
sub.request(10) sub.request(10)
dataProbe.expectNext(Chunk(ByteString("abcdef"))) dataProbe.expectNext(Chunk(ByteString("abcdef")))
@ -329,7 +329,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
inside(expectRequest()) { inside(expectRequest()) {
case HttpRequest(POST, _, _, HttpEntity.Default(_, 12, data), _) case HttpRequest(POST, _, _, HttpEntity.Default(_, 12, data), _)
val dataProbe = TestSubscriber.manualProbe[ByteString] val dataProbe = TestSubscriber.manualProbe[ByteString]
data.to(Sink(dataProbe)).run() data.to(Sink.fromSubscriber(dataProbe)).run()
val sub = dataProbe.expectSubscription() val sub = dataProbe.expectSubscription()
sub.request(10) sub.request(10)
dataProbe.expectNext(ByteString("abcdef")) dataProbe.expectNext(ByteString("abcdef"))
@ -350,7 +350,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
inside(expectRequest()) { inside(expectRequest()) {
case HttpRequest(POST, _, _, HttpEntity.Chunked(_, data), _) case HttpRequest(POST, _, _, HttpEntity.Chunked(_, data), _)
val dataProbe = TestSubscriber.manualProbe[ChunkStreamPart] val dataProbe = TestSubscriber.manualProbe[ChunkStreamPart]
data.to(Sink(dataProbe)).run() data.to(Sink.fromSubscriber(dataProbe)).run()
val sub = dataProbe.expectSubscription() val sub = dataProbe.expectSubscription()
sub.request(10) sub.request(10)
dataProbe.expectNext(Chunk(ByteString("abcdef"))) dataProbe.expectNext(Chunk(ByteString("abcdef")))
@ -405,7 +405,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
val data = TestPublisher.manualProbe[ByteString]() val data = TestPublisher.manualProbe[ByteString]()
inside(expectRequest()) { inside(expectRequest()) {
case HttpRequest(GET, _, _, _, _) 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() val dataSub = data.expectSubscription()
dataSub.expectCancellation() dataSub.expectCancellation()
expectResponseWithWipedDate( expectResponseWithWipedDate(
@ -427,7 +427,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
val data = TestPublisher.manualProbe[ByteString]() val data = TestPublisher.manualProbe[ByteString]()
inside(expectRequest()) { inside(expectRequest()) {
case HttpRequest(GET, _, _, _, _) 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() val dataSub = data.expectSubscription()
dataSub.expectCancellation() dataSub.expectCancellation()
expectResponseWithWipedDate( expectResponseWithWipedDate(
@ -450,7 +450,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
val data = TestPublisher.manualProbe[ChunkStreamPart]() val data = TestPublisher.manualProbe[ChunkStreamPart]()
inside(expectRequest()) { inside(expectRequest()) {
case HttpRequest(GET, _, _, _, _) 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() val dataSub = data.expectSubscription()
dataSub.expectCancellation() dataSub.expectCancellation()
expectResponseWithWipedDate( expectResponseWithWipedDate(
@ -473,7 +473,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
val data = TestPublisher.manualProbe[ByteString]() val data = TestPublisher.manualProbe[ByteString]()
inside(expectRequest()) { inside(expectRequest()) {
case HttpRequest(GET, _, _, _, _) 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() val dataSub = data.expectSubscription()
dataSub.expectCancellation() dataSub.expectCancellation()
netOut.expectBytes(1) netOut.expectBytes(1)
@ -491,7 +491,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
inside(expectRequest()) { inside(expectRequest()) {
case HttpRequest(POST, _, _, Default(ContentType(`application/octet-stream`, None), 16, data), _) case HttpRequest(POST, _, _, Default(ContentType(`application/octet-stream`, None), 16, data), _)
val dataProbe = TestSubscriber.manualProbe[ByteString] val dataProbe = TestSubscriber.manualProbe[ByteString]
data.to(Sink(dataProbe)).run() data.to(Sink.fromSubscriber(dataProbe)).run()
val dataSub = dataProbe.expectSubscription() val dataSub = dataProbe.expectSubscription()
netOut.expectNoBytes(50.millis) netOut.expectNoBytes(50.millis)
dataSub.request(1) // triggers `100 Continue` response dataSub.request(1) // triggers `100 Continue` response
@ -527,7 +527,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
inside(expectRequest()) { inside(expectRequest()) {
case HttpRequest(POST, _, _, Chunked(ContentType(`application/octet-stream`, None), data), _) case HttpRequest(POST, _, _, Chunked(ContentType(`application/octet-stream`, None), data), _)
val dataProbe = TestSubscriber.manualProbe[ChunkStreamPart] val dataProbe = TestSubscriber.manualProbe[ChunkStreamPart]
data.to(Sink(dataProbe)).run() data.to(Sink.fromSubscriber(dataProbe)).run()
val dataSub = dataProbe.expectSubscription() val dataSub = dataProbe.expectSubscription()
netOut.expectNoBytes(50.millis) netOut.expectNoBytes(50.millis)
dataSub.request(2) // triggers `100 Continue` response dataSub.request(2) // triggers `100 Continue` response

View file

@ -38,10 +38,10 @@ abstract class HttpServerTestSetupBase {
RunnableGraph.fromGraph(GraphDSL.create(HttpServerBluePrint(settings, remoteAddress = remoteAddress, log = NoLogging)) { implicit b RunnableGraph.fromGraph(GraphDSL.create(HttpServerBluePrint(settings, remoteAddress = remoteAddress, log = NoLogging)) { implicit b
server server
import GraphDSL.Implicits._ 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.out1 ~> Flow[SslTlsOutbound].collect { case SendBytes(x) x }.buffer(1, OverflowStrategy.backpressure) ~> netOut.sink
server.out2 ~> Sink(requests) server.out2 ~> Sink.fromSubscriber(requests)
Source(responses) ~> server.in1 Source.fromPublisher(responses) ~> server.in1
ClosedShape ClosedShape
}).run() }).run()

View file

@ -35,7 +35,7 @@ object ByteStringSinkProbe {
def apply()(implicit system: ActorSystem): ByteStringSinkProbe = def apply()(implicit system: ActorSystem): ByteStringSinkProbe =
new ByteStringSinkProbe { new ByteStringSinkProbe {
val probe = TestSubscriber.probe[ByteString]() val probe = TestSubscriber.probe[ByteString]()
val sink: Sink[ByteString, Unit] = Sink(probe) val sink: Sink[ByteString, Unit] = Sink.fromSubscriber(probe)
def expectNoBytes(): Unit = { def expectNoBytes(): Unit = {
probe.ensureSubscription() probe.ensureSubscription()

View file

@ -47,7 +47,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
pushInput(header ++ data1) pushInput(header ++ data1)
val dataSource = expectBinaryMessage().dataStream val dataSource = expectBinaryMessage().dataStream
val sub = TestSubscriber.manualProbe[ByteString]() val sub = TestSubscriber.manualProbe[ByteString]()
dataSource.runWith(Sink(sub)) dataSource.runWith(Sink.fromSubscriber(sub))
val s = sub.expectSubscription() val s = sub.expectSubscription()
s.request(2) s.request(2)
sub.expectNext(data1) sub.expectNext(data1)
@ -59,7 +59,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
pushInput(header) pushInput(header)
val dataSource = expectBinaryMessage().dataStream val dataSource = expectBinaryMessage().dataStream
val sub = TestSubscriber.manualProbe[ByteString]() val sub = TestSubscriber.manualProbe[ByteString]()
dataSource.runWith(Sink(sub)) dataSource.runWith(Sink.fromSubscriber(sub))
val s = sub.expectSubscription() val s = sub.expectSubscription()
s.request(2) s.request(2)
pushInput(data1) pushInput(data1)
@ -78,7 +78,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
pushInput(header1 ++ data1) pushInput(header1 ++ data1)
val dataSource = expectBinaryMessage().dataStream val dataSource = expectBinaryMessage().dataStream
val sub = TestSubscriber.manualProbe[ByteString]() val sub = TestSubscriber.manualProbe[ByteString]()
dataSource.runWith(Sink(sub)) dataSource.runWith(Sink.fromSubscriber(sub))
val s = sub.expectSubscription() val s = sub.expectSubscription()
s.request(2) s.request(2)
sub.expectNext(data1) sub.expectNext(data1)
@ -96,7 +96,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
pushInput(header1 ++ data1) pushInput(header1 ++ data1)
val dataSource = expectBinaryMessage().dataStream val dataSource = expectBinaryMessage().dataStream
val sub = TestSubscriber.manualProbe[ByteString]() val sub = TestSubscriber.manualProbe[ByteString]()
dataSource.runWith(Sink(sub)) dataSource.runWith(Sink.fromSubscriber(sub))
val s = sub.expectSubscription() val s = sub.expectSubscription()
s.request(2) s.request(2)
sub.expectNext(data1) sub.expectNext(data1)
@ -111,7 +111,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
val dataSource2 = expectBinaryMessage().dataStream val dataSource2 = expectBinaryMessage().dataStream
val sub2 = TestSubscriber.manualProbe[ByteString]() val sub2 = TestSubscriber.manualProbe[ByteString]()
dataSource2.runWith(Sink(sub2)) dataSource2.runWith(Sink.fromSubscriber(sub2))
val s2 = sub2.expectSubscription() val s2 = sub2.expectSubscription()
s2.request(2) s2.request(2)
sub2.expectNext(data3) sub2.expectNext(data3)
@ -131,7 +131,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
pushInput(header ++ data1) pushInput(header ++ data1)
val dataSource = expectBinaryMessage().dataStream val dataSource = expectBinaryMessage().dataStream
val sub = TestSubscriber.manualProbe[ByteString]() val sub = TestSubscriber.manualProbe[ByteString]()
dataSource.runWith(Sink(sub)) dataSource.runWith(Sink.fromSubscriber(sub))
val s = sub.expectSubscription() val s = sub.expectSubscription()
s.request(2) s.request(2)
sub.expectNext(ByteString("abc", "ASCII")) sub.expectNext(ByteString("abc", "ASCII"))
@ -174,7 +174,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
pushInput(input) pushInput(input)
val parts = expectTextMessage().textStream val parts = expectTextMessage().textStream
val sub = TestSubscriber.manualProbe[String]() val sub = TestSubscriber.manualProbe[String]()
parts.runWith(Sink(sub)) parts.runWith(Sink.fromSubscriber(sub))
val s = sub.expectSubscription() val s = sub.expectSubscription()
s.request(4) s.request(4)
sub.expectNext("b") sub.expectNext("b")
@ -192,7 +192,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
pushInput(header0 ++ data0) pushInput(header0 ++ data0)
val parts = expectTextMessage().textStream val parts = expectTextMessage().textStream
val sub = TestSubscriber.manualProbe[String]() val sub = TestSubscriber.manualProbe[String]()
parts.runWith(Sink(sub)) parts.runWith(Sink.fromSubscriber(sub))
val s = sub.expectSubscription() val s = sub.expectSubscription()
s.request(4) s.request(4)
sub.expectNoMsg(100.millis) sub.expectNoMsg(100.millis)
@ -211,7 +211,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
pushInput(header ++ data1) pushInput(header ++ data1)
val dataSource = expectBinaryMessage().dataStream val dataSource = expectBinaryMessage().dataStream
val sub = TestSubscriber.manualProbe[ByteString]() val sub = TestSubscriber.manualProbe[ByteString]()
dataSource.runWith(Sink(sub)) dataSource.runWith(Sink.fromSubscriber(sub))
val s = sub.expectSubscription() val s = sub.expectSubscription()
s.request(2) s.request(2)
sub.expectNext(ByteString("äb", "UTF-8")) 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 { "for a streamed message" in new ServerTestSetup {
val data = ByteString("abcdefg", "ASCII") val data = ByteString("abcdefg", "ASCII")
val pub = TestPublisher.manualProbe[ByteString]() val pub = TestPublisher.manualProbe[ByteString]()
val msg = BinaryMessage(Source(pub)) val msg = BinaryMessage(Source.fromPublisher(pub))
pushMessage(msg) pushMessage(msg)
val sub = pub.expectSubscription() 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 { "and mask input on the client side" in new ClientTestSetup {
val data = ByteString("abcdefg", "ASCII") val data = ByteString("abcdefg", "ASCII")
val pub = TestPublisher.manualProbe[ByteString]() val pub = TestPublisher.manualProbe[ByteString]()
val msg = BinaryMessage(Source(pub)) val msg = BinaryMessage(Source.fromPublisher(pub))
pushMessage(msg) pushMessage(msg)
val sub = pub.expectSubscription() val sub = pub.expectSubscription()
@ -299,7 +299,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
"for a streamed message" in new ServerTestSetup { "for a streamed message" in new ServerTestSetup {
val text = "äbcd€fg" val text = "äbcd€fg"
val pub = TestPublisher.manualProbe[String]() val pub = TestPublisher.manualProbe[String]()
val msg = TextMessage(Source(pub)) val msg = TextMessage(Source.fromPublisher(pub))
pushMessage(msg) pushMessage(msg)
val sub = pub.expectSubscription() val sub = pub.expectSubscription()
@ -328,7 +328,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
val half2 = gclef.drop(1) val half2 = gclef.drop(1)
val pub = TestPublisher.manualProbe[String]() val pub = TestPublisher.manualProbe[String]()
val msg = TextMessage(Source(pub)) val msg = TextMessage(Source.fromPublisher(pub))
pushMessage(msg) pushMessage(msg)
val sub = pub.expectSubscription() 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 { "and mask input on the client side" in new ClientTestSetup {
val text = "abcdefg" val text = "abcdefg"
val pub = TestPublisher.manualProbe[String]() val pub = TestPublisher.manualProbe[String]()
val msg = TextMessage(Source(pub)) val msg = TextMessage(Source.fromPublisher(pub))
pushMessage(msg) pushMessage(msg)
val sub = pub.expectSubscription() val sub = pub.expectSubscription()
@ -394,13 +394,13 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
val dataSource = expectBinaryMessage().dataStream val dataSource = expectBinaryMessage().dataStream
val sub = TestSubscriber.manualProbe[ByteString]() val sub = TestSubscriber.manualProbe[ByteString]()
dataSource.runWith(Sink(sub)) dataSource.runWith(Sink.fromSubscriber(sub))
val s = sub.expectSubscription() val s = sub.expectSubscription()
s.request(2) s.request(2)
sub.expectNext(ByteString("123", "ASCII")) sub.expectNext(ByteString("123", "ASCII"))
val outPub = TestPublisher.manualProbe[ByteString]() val outPub = TestPublisher.manualProbe[ByteString]()
val msg = BinaryMessage(Source(outPub)) val msg = BinaryMessage(Source.fromPublisher(outPub))
pushMessage(msg) pushMessage(msg)
expectFrameHeaderOnNetwork(Opcode.Binary, 0, fin = false) 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) // sending another message is allowed before closing (inherently racy)
val pub = TestPublisher.manualProbe[ByteString]() val pub = TestPublisher.manualProbe[ByteString]()
val msg = BinaryMessage(Source(pub)) val msg = BinaryMessage(Source.fromPublisher(pub))
pushMessage(msg) pushMessage(msg)
expectFrameOnNetwork(Opcode.Binary, ByteString.empty, fin = false) 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()))) pushInput(frameHeader(Protocol.Opcode.Binary, 0, fin = false, mask = Some(Random.nextInt())))
val dataSource = expectBinaryMessage().dataStream val dataSource = expectBinaryMessage().dataStream
val inSubscriber = TestSubscriber.manualProbe[ByteString]() val inSubscriber = TestSubscriber.manualProbe[ByteString]()
dataSource.runWith(Sink(inSubscriber)) dataSource.runWith(Sink.fromSubscriber(inSubscriber))
val inSub = inSubscriber.expectSubscription() val inSub = inSubscriber.expectSubscription()
val outData = ByteString("def", "ASCII") 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) // sending another message is allowed before closing (inherently racy)
val pub = TestPublisher.manualProbe[ByteString]() val pub = TestPublisher.manualProbe[ByteString]()
val msg = BinaryMessage(Source(pub)) val msg = BinaryMessage(Source.fromPublisher(pub))
pushMessage(msg) pushMessage(msg)
expectFrameOnNetwork(Opcode.Binary, ByteString.empty, fin = false) 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 { "when user handler closes main stream and substream only afterwards" in new ServerTestSetup {
// send half a message // send half a message
val pub = TestPublisher.manualProbe[ByteString]() val pub = TestPublisher.manualProbe[ByteString]()
val msg = BinaryMessage(Source(pub)) val msg = BinaryMessage(Source.fromPublisher(pub))
pushMessage(msg) pushMessage(msg)
expectFrameOnNetwork(Opcode.Binary, ByteString.empty, fin = false) 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] = val messageHandler: Flow[Message, Message, Unit] =
Flow.fromSinkAndSource( Flow.fromSinkAndSource(
Flow[Message].buffer(1, OverflowStrategy.backpressure).to(Sink(messageIn)), // alternatively need to request(1) before expectComplete Flow[Message].buffer(1, OverflowStrategy.backpressure).to(Sink.fromSubscriber(messageIn)), // alternatively need to request(1) before expectComplete
Source(messageOut)) Source.fromPublisher(messageOut))
Source(netIn) Source.fromPublisher(netIn)
.via(printEvent("netIn")) .via(printEvent("netIn"))
.via(FrameEventParser) .via(FrameEventParser)
.via(Websocket .via(Websocket

View file

@ -315,7 +315,7 @@ class WebsocketClientSpec extends FreeSpec with Matchers with WithMaterializerSp
RunnableGraph.fromGraph(GraphDSL.create(clientLayer) { implicit b RunnableGraph.fromGraph(GraphDSL.create(clientLayer) { implicit b
client client
import GraphDSL.Implicits._ 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.out1 ~> Flow[SslTlsOutbound].collect { case SendBytes(x) x } ~> netOut.sink
client.out2 ~> clientImplementation ~> client.in1 client.out2 ~> clientImplementation ~> client.in1
ClosedShape ClosedShape
@ -370,6 +370,6 @@ class WebsocketClientSpec extends FreeSpec with Matchers with WithMaterializerSp
lazy val messagesIn = TestSubscriber.probe[Message]() lazy val messagesIn = TestSubscriber.probe[Message]()
override def clientImplementation: Flow[Message, Message, Unit] = 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)
} }
} }

View file

@ -49,7 +49,7 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
"properly bind a server" in { "properly bind a server" in {
val (_, hostname, port) = TestUtils.temporaryServerHostnameAndPort() val (_, hostname, port) = TestUtils.temporaryServerHostnameAndPort()
val probe = TestSubscriber.manualProbe[Http.IncomingConnection]() 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 sub = probe.expectSubscription() // if we get it we are bound
val address = Await.result(binding, 1.second).localAddress val address = Await.result(binding, 1.second).localAddress
sub.cancel() sub.cancel()
@ -60,15 +60,15 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
val binding = Http().bind(hostname, port) val binding = Http().bind(hostname, port)
val probe1 = TestSubscriber.manualProbe[Http.IncomingConnection]() val probe1 = TestSubscriber.manualProbe[Http.IncomingConnection]()
// Bind succeeded, we have a local address // 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() probe1.expectSubscription()
val probe2 = TestSubscriber.manualProbe[Http.IncomingConnection]() 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() probe2.expectSubscriptionAndError()
val probe3 = TestSubscriber.manualProbe[Http.IncomingConnection]() 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() probe3.expectSubscriptionAndError()
// Now unbind the first // Now unbind the first
@ -78,7 +78,7 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
if (!akka.util.Helpers.isWindows) { if (!akka.util.Helpers.isWindows) {
val probe4 = TestSubscriber.manualProbe[Http.IncomingConnection]() val probe4 = TestSubscriber.manualProbe[Http.IncomingConnection]()
// Bind succeeded, we have a local address // 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() probe4.expectSubscription()
// clean up // clean up
@ -448,7 +448,7 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
val settings = configOverrides.toOption.fold(ServerSettings(system))(ServerSettings(_)) val settings = configOverrides.toOption.fold(ServerSettings(system))(ServerSettings(_))
val connections = Http().bind(hostname, port, settings = settings) val connections = Http().bind(hostname, port, settings = settings)
val probe = TestSubscriber.manualProbe[Http.IncomingConnection] 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) (probe, binding)
} }
val connSourceSub = connSource.expectSubscription() val connSourceSub = connSource.expectSubscription()
@ -457,9 +457,9 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
val requestPublisherProbe = TestPublisher.manualProbe[HttpRequest]() val requestPublisherProbe = TestPublisher.manualProbe[HttpRequest]()
val responseSubscriberProbe = TestSubscriber.manualProbe[HttpResponse]() val responseSubscriberProbe = TestSubscriber.manualProbe[HttpResponse]()
val connectionFuture = Source(requestPublisherProbe) val connectionFuture = Source.fromPublisher(requestPublisherProbe)
.viaMat(Http().outgoingConnection(hostname, port, settings = settings))(Keep.right) .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) 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]) = { def acceptConnection(): (TestSubscriber.ManualProbe[HttpRequest], TestPublisher.ManualProbe[HttpResponse]) = {
connSourceSub.request(1) connSourceSub.request(1)
val incomingConnection = connSource.expectNext() val incomingConnection = connSource.expectNext()
val sink = Sink.publisher[HttpRequest](false) val sink = Sink.asPublisher[HttpRequest](false)
val source = Source.subscriber[HttpResponse] val source = Source.asSubscriber[HttpResponse]
val handler = Flow.fromSinkAndSourceMat(sink, source)(Keep.both) val handler = Flow.fromSinkAndSourceMat(sink, source)(Keep.both)

View file

@ -89,7 +89,7 @@ class HttpEntitySpec extends FreeSpec with MustMatchers with BeforeAndAfterAll {
"Infinite data stream" in { "Infinite data stream" in {
val neverCompleted = Promise[ByteString]() val neverCompleted = Promise[ByteString]()
intercept[TimeoutException] { 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") }.getMessage must be("HttpEntity.toStrict timed out after 100 milliseconds while still waiting for outstanding data")
} }
} }

View file

@ -104,7 +104,7 @@ object WSProbe {
val subscriber = TestSubscriber.probe[Message]() val subscriber = TestSubscriber.probe[Message]()
val publisher = TestPublisher.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(message: Message): Unit = publisher.sendNext(message)
def sendMessage(text: String): Unit = sendMessage(TextMessage(text)) def sendMessage(text: String): Unit = sendMessage(TextMessage(text))
@ -139,4 +139,4 @@ object WSProbe {
.awaitResult(maxChunkCollectionMills.millis) .awaitResult(maxChunkCollectionMills.millis)
.reduce(reduce) .reduce(reduce)
} }
} }

View file

@ -138,7 +138,7 @@ abstract class CoderSpec extends WordSpec with CodecSpecSupport with Inspectors
ByteString(Array.fill(size)(1.toByte)) ByteString(Array.fill(size)(1.toByte))
val sizesAfterRoundtrip = val sizesAfterRoundtrip =
Source(() sizes.toIterator.map(createByteString)) Source.fromIterator(() sizes.toIterator.map(createByteString))
.via(Coder.encoderFlow) .via(Coder.encoderFlow)
.via(Coder.decoderFlow) .via(Coder.decoderFlow)
.runFold(Seq.empty[Int])(_ :+ _.size) .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) input.via(Coder.decoderFlow).join.awaitResult(3.seconds)
def decodeFromIterator(iterator: () Iterator[ByteString]): ByteString = 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)
} }

View file

@ -201,7 +201,7 @@ class CodingDirectivesSpec extends RoutingSpec with Inside {
() text.grouped(8).map { chars () text.grouped(8).map { chars
Chunk(chars.mkString): ChunkStreamPart 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) ~> { Post() ~> `Accept-Encoding`(gzip) ~> {
encodeResponseWith(Gzip) { encodeResponseWith(Gzip) {

View file

@ -10,7 +10,7 @@ import org.reactivestreams.Publisher
class ConcatTest extends AkkaPublisherVerification[Int] { class ConcatTest extends AkkaPublisherVerification[Int] {
def createPublisher(elements: Long): Publisher[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))
} }
} }

View file

@ -15,7 +15,7 @@ class FanoutPublisherTest extends AkkaPublisherVerification[Int] {
if (elements == 0) new immutable.Iterable[Int] { override def iterator = Iterator from 0 } if (elements == 0) new immutable.Iterable[Int] { override def iterator = Iterator from 0 }
else 0 until elements.toInt else 0 until elements.toInt
Source(iterable).runWith(Sink.publisher(true)) Source(iterable).runWith(Sink.asPublisher(true))
} }
} }

View file

@ -38,7 +38,7 @@ class FilePublisherTest extends AkkaPublisherVerification[ByteString] {
def createPublisher(elements: Long): Publisher[ByteString] = def createPublisher(elements: Long): Publisher[ByteString] =
Source.file(file, chunkSize = 512) Source.file(file, chunkSize = 512)
.take(elements) .take(elements)
.runWith(Sink.publisher(false)) .runWith(Sink.asPublisher(false))
@AfterClass @AfterClass
def after = file.delete() def after = file.delete()

View file

@ -13,7 +13,7 @@ class FlattenTest extends AkkaPublisherVerification[Int] {
def createPublisher(elements: Long): Publisher[Int] = { def createPublisher(elements: Long): Publisher[Int] = {
val s1 = Source(iterable(elements / 2)) val s1 = Source(iterable(elements / 2))
val s2 = Source(iterable((elements + 1) / 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))
} }
} }

View file

@ -9,7 +9,7 @@ import org.reactivestreams.Subscriber
class FoldSinkSubscriberTest extends AkkaSubscriberBlackboxVerification[Int] { class FoldSinkSubscriberTest extends AkkaSubscriberBlackboxVerification[Int] {
override def createSubscriber(): Subscriber[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 override def createElement(element: Int): Int = element
} }

View file

@ -9,7 +9,7 @@ import org.reactivestreams.Subscriber
class ForeachSinkSubscriberTest extends AkkaSubscriberBlackboxVerification[Int] { class ForeachSinkSubscriberTest extends AkkaSubscriberBlackboxVerification[Int] {
override def createSubscriber(): Subscriber[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 override def createElement(element: Int): Int = element
} }

View file

@ -13,7 +13,7 @@ class FuturePublisherTest extends AkkaPublisherVerification[Int] {
def createPublisher(elements: Long): Publisher[Int] = { def createPublisher(elements: Long): Publisher[Int] = {
val p = Promise[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) p.success(0)
pub pub
} }

View file

@ -24,7 +24,7 @@ class GroupByTest extends AkkaPublisherVerification[Int] {
.concatSubstreams .concatSubstreams
.runWith(Sink.head) .runWith(Sink.head)
val groupSource = Await.result(futureGroupSource, 3.seconds) val groupSource = Await.result(futureGroupSource, 3.seconds)
groupSource.runWith(Sink.publisher(false)) groupSource.runWith(Sink.asPublisher(false))
} }

View file

@ -11,7 +11,7 @@ import org.reactivestreams._
class IterablePublisherTest extends AkkaPublisherVerification[Int] { class IterablePublisherTest extends AkkaPublisherVerification[Int] {
override def createPublisher(elements: Long): Publisher[Int] = { override def createPublisher(elements: Long): Publisher[Int] = {
Source(iterable(elements)).runWith(Sink.publisher(false)) Source(iterable(elements)).runWith(Sink.asPublisher(false))
} }
} }

View file

@ -10,7 +10,7 @@ import akka.stream.scaladsl.{ Keep, Source, Sink }
class MaybeSourceTest extends AkkaPublisherVerification[Int] { class MaybeSourceTest extends AkkaPublisherVerification[Int] {
def createPublisher(elements: Long): Publisher[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) p success Some(1)
pub pub
} }

View file

@ -15,7 +15,7 @@ class PrefixAndTailTest extends AkkaPublisherVerification[Int] {
def createPublisher(elements: Long): Publisher[Int] = { def createPublisher(elements: Long): Publisher[Int] = {
val futureTailSource = Source(iterable(elements)).prefixAndTail(0).map { case (_, tail) tail }.runWith(Sink.head) val futureTailSource = Source(iterable(elements)).prefixAndTail(0).map { case (_, tail) tail }.runWith(Sink.head)
val tailSource = Await.result(futureTailSource, 3.seconds) val tailSource = Await.result(futureTailSource, 3.seconds)
tailSource.runWith(Sink.publisher(false)) tailSource.runWith(Sink.asPublisher(false))
} }
} }

View file

@ -11,7 +11,7 @@ import org.reactivestreams.Publisher
class SingleElementSourceTest extends AkkaPublisherVerification[Int] { class SingleElementSourceTest extends AkkaPublisherVerification[Int] {
def createPublisher(elements: Long): Publisher[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 override def maxElementsFromPublisher(): Long = 1
} }

View file

@ -24,7 +24,7 @@ class SplitWhenTest extends AkkaPublisherVerification[Int] {
.concatSubstreams .concatSubstreams
.runWith(Sink.head) .runWith(Sink.head)
val source = Await.result(futureSource, 3.seconds) val source = Await.result(futureSource, 3.seconds)
source.runWith(Sink.publisher(false)) source.runWith(Sink.asPublisher(false))
} }
} }

View file

@ -27,7 +27,7 @@ abstract class BaseTwoStreamsSetup extends AkkaSpec {
def completedPublisher[T]: Publisher[T] = TestPublisher.empty[T] 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) def soonToFailPublisher[T]: Publisher[T] = TestPublisher.lazyError[T](TestException)

View file

@ -24,7 +24,7 @@ class ChainSetup[In, Out, M](
val upstream = TestPublisher.manualProbe[In]() val upstream = TestPublisher.manualProbe[In]()
val downstream = TestSubscriber.probe[Out]() 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 publisher = toPublisher(s, materializer)
val upstreamSubscription = upstream.expectSubscription() val upstreamSubscription = upstream.expectSubscription()
publisher.subscribe(downstream) publisher.subscribe(downstream)

View file

@ -20,7 +20,7 @@ trait ScriptedTest extends Matchers {
class ScriptException(msg: String) extends RuntimeException(msg) class ScriptException(msg: String) extends RuntimeException(msg)
def toPublisher[In, Out]: (Source[Out, _], ActorMaterializer) Publisher[Out] = 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 { object Script {
def apply[In, Out](phases: (Seq[In], Seq[Out])*): Script[In, Out] = { def apply[In, Out](phases: (Seq[In], Seq[Out])*): Script[In, Out] = {

View file

@ -23,7 +23,7 @@ class StreamTestKitSpec extends AkkaSpec {
"#toStrict with failing source" in { "#toStrict with failing source" in {
val msg = intercept[AssertionError] { val msg = intercept[AssertionError] {
Source(() new Iterator[Int] { Source.fromIterator(() new Iterator[Int] {
var i = 0 var i = 0
override def hasNext: Boolean = true override def hasNext: Boolean = true
override def next(): Int = { override def next(): Int = {

View file

@ -22,7 +22,7 @@ class TestPublisherSubscriberSpec extends AkkaSpec {
"have all events accessible from manual probes" in assertAllStagesStopped { "have all events accessible from manual probes" in assertAllStagesStopped {
val upstream = TestPublisher.manualProbe[Int]() val upstream = TestPublisher.manualProbe[Int]()
val downstream = TestSubscriber.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 upstreamSubscription = upstream.expectSubscription()
val downstreamSubscription: Subscription = downstream.expectEventPF { case OnSubscribe(sub) sub } 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 { "handle gracefully partial function that is not suitable" in assertAllStagesStopped {
val upstream = TestPublisher.manualProbe[Int]() val upstream = TestPublisher.manualProbe[Int]()
val downstream = TestSubscriber.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 upstreamSubscription = upstream.expectSubscription()
val downstreamSubscription: Subscription = downstream.expectEventPF { case OnSubscribe(sub) sub } val downstreamSubscription: Subscription = downstream.expectEventPF { case OnSubscribe(sub) sub }

View file

@ -23,9 +23,9 @@ abstract class TwoStreamsSetup extends BaseTwoStreamsSetup {
import GraphDSL.Implicits._ import GraphDSL.Implicits._
val f = fixture(b) val f = fixture(b)
Source(p1) ~> f.left Source.fromPublisher(p1) ~> f.left
Source(p2) ~> f.right Source.fromPublisher(p2) ~> f.right
f.out ~> Sink(subscriber) f.out ~> Sink.fromSubscriber(subscriber)
ClosedShape ClosedShape
}).run() }).run()

View file

@ -42,7 +42,7 @@ public class ActorPublisherTest extends StreamTest {
final ActorRef ref = system final ActorRef ref = system
.actorOf(Props.create(TestPublisher.class).withDispatcher("akka.test.stream-dispatcher")); .actorOf(Props.create(TestPublisher.class).withDispatcher("akka.test.stream-dispatcher"));
final Publisher<Integer> publisher = UntypedActorPublisher.create(ref); final Publisher<Integer> publisher = UntypedActorPublisher.create(ref);
Source.from(publisher) Source.fromPublisher(publisher)
.runForeach(new akka.japi.function.Procedure<Integer>() { .runForeach(new akka.japi.function.Procedure<Integer>() {
@Override @Override
public void apply(Integer elem) throws Exception { public void apply(Integer elem) throws Exception {

View file

@ -63,7 +63,7 @@ public class ActorSubscriberTest extends StreamTest {
final Subscriber<Integer> subscriber = UntypedActorSubscriber.create(ref); final Subscriber<Integer> subscriber = UntypedActorSubscriber.create(ref);
final java.lang.Iterable<Integer> input = Arrays.asList(1, 2, 3); 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); ref.tell("run", null);
probe.expectMsgEquals(1); probe.expectMsgEquals(1);

View file

@ -67,8 +67,8 @@ public class FlowGraphTest extends StreamTest {
final Source<String, BoxedUnit> in1 = Source.from(Arrays.asList("a", "b", "c")); final Source<String, BoxedUnit> in1 = Source.from(Arrays.asList("a", "b", "c"));
final Source<String, BoxedUnit> in2 = Source.from(Arrays.asList("d", "e", "f")); 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( final Source<String, BoxedUnit> source = Source.fromGraph(
GraphDSL.create(new Function<GraphDSL.Builder<BoxedUnit>, SourceShape<String>>() { GraphDSL.create(new Function<GraphDSL.Builder<BoxedUnit>, SourceShape<String>>() {
@Override @Override
@ -82,7 +82,7 @@ public class FlowGraphTest extends StreamTest {
// collecting // collecting
final Publisher<String> pub = source.runWith(publisher, materializer); 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)); 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)); assertEquals(new HashSet<Object>(Arrays.asList("a", "b", "c", "d", "e", "f")), new HashSet<String>(result));
@ -257,7 +257,7 @@ public class FlowGraphTest extends StreamTest {
return l + r; return l + r;
} }
}); });
final Future<Integer> future = RunnableGraph.fromGraph(GraphDSL.create(Sink.<Integer>head(), final Future<Integer> future = RunnableGraph.fromGraph(GraphDSL.create(Sink.<Integer>head(),
new Function2<Builder<Future<Integer>>, SinkShape<Integer>, ClosedShape>() { new Function2<Builder<Future<Integer>>, SinkShape<Integer>, ClosedShape>() {
@Override @Override

View file

@ -249,7 +249,7 @@ public class FlowTest extends StreamTest {
}) })
.grouped(10) .grouped(10)
.mergeSubstreams(); .mergeSubstreams();
final Future<List<List<String>>> future = final Future<List<List<String>>> future =
Source.from(input).via(flow).grouped(10).runWith(Sink.<List<List<String>>> head(), materializer); Source.from(input).via(flow).grouped(10).runWith(Sink.<List<List<String>>> head(), materializer);
final Object[] result = Await.result(future, Duration.create(1, TimeUnit.SECONDS)).toArray(); final Object[] result = Await.result(future, Duration.create(1, TimeUnit.SECONDS)).toArray();
@ -335,7 +335,7 @@ public class FlowTest extends StreamTest {
final Source<String, BoxedUnit> in1 = Source.from(Arrays.asList("a", "b", "c")); final Source<String, BoxedUnit> in1 = Source.from(Arrays.asList("a", "b", "c"));
final Source<String, BoxedUnit> in2 = Source.from(Arrays.asList("d", "e", "f")); 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( final Source<String, BoxedUnit> source = Source.fromGraph(
GraphDSL.create(new Function<GraphDSL.Builder<BoxedUnit>, SourceShape<String>>() { GraphDSL.create(new Function<GraphDSL.Builder<BoxedUnit>, SourceShape<String>>() {
@ -350,7 +350,7 @@ public class FlowTest extends StreamTest {
// collecting // collecting
final Publisher<String> pub = source.runWith(publisher, materializer); 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)); 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)); assertEquals(new HashSet<Object>(Arrays.asList("a", "b", "c", "d", "e", "f")), new HashSet<String>(result));
@ -443,7 +443,7 @@ public class FlowTest extends StreamTest {
assertEquals(Arrays.asList(1, 2, 3, 4, 5), result); assertEquals(Arrays.asList(1, 2, 3, 4, 5), result);
} }
@Test @Test
public void mustBeAbleToUseFlatMapMerge() throws Exception { public void mustBeAbleToUseFlatMapMerge() throws Exception {
final JavaTestKit probe = new JavaTestKit(system); final JavaTestKit probe = new JavaTestKit(system);
@ -558,7 +558,7 @@ public class FlowTest extends StreamTest {
final TestPublisher.ManualProbe<Integer> publisherProbe = TestPublisher.manualProbe(true,system); final TestPublisher.ManualProbe<Integer> publisherProbe = TestPublisher.manualProbe(true,system);
final JavaTestKit probe = new JavaTestKit(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( final Flow<Integer, Integer, ?> flow = Flow.of(Integer.class).map(
new Function<Integer, Integer>() { new Function<Integer, Integer>() {
public Integer apply(Integer elem) { public Integer apply(Integer elem) {

View file

@ -39,7 +39,7 @@ public class SinkTest extends StreamTest {
@Test @Test
public void mustBeAbleToUseFanoutPublisher() throws Exception { 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") @SuppressWarnings("unused")
final Publisher<Object> publisher = Source.from(new ArrayList<Object>()).runWith(pubSink, materializer); final Publisher<Object> publisher = Source.from(new ArrayList<Object>()).runWith(pubSink, materializer);
} }

View file

@ -176,7 +176,7 @@ public class SourceTest extends StreamTest {
}) })
.grouped(10) .grouped(10)
.mergeSubstreams(); .mergeSubstreams();
final Future<List<List<String>>> future = final Future<List<List<String>>> future =
source.grouped(10).runWith(Sink.<List<List<String>>> head(), materializer); source.grouped(10).runWith(Sink.<List<List<String>>> head(), materializer);
final Object[] result = Await.result(future, Duration.create(1, TimeUnit.SECONDS)).toArray(); final Object[] result = Await.result(future, Duration.create(1, TimeUnit.SECONDS)).toArray();
@ -478,7 +478,7 @@ public class SourceTest extends StreamTest {
final JavaTestKit probe = new JavaTestKit(system); final JavaTestKit probe = new JavaTestKit(system);
final Iterable<String> input = Arrays.asList("A", "B", "C"); final Iterable<String> input = Arrays.asList("A", "B", "C");
Future<String> future1 = Source.from(input).runWith(Sink.<String>head(), materializer); 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))); String result = Await.result(future2, probe.dilated(FiniteDuration.create(3, TimeUnit.SECONDS)));
assertEquals("A", result); assertEquals("A", result);
} }
@ -579,7 +579,7 @@ public class SourceTest extends StreamTest {
final ManualProbe<Integer> publisherProbe = TestPublisher.manualProbe(true,system); final ManualProbe<Integer> publisherProbe = TestPublisher.manualProbe(true,system);
final JavaTestKit probe = new JavaTestKit(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>() { new Function<Integer, Integer>() {
public Integer apply(Integer elem) { public Integer apply(Integer elem) {
if (elem == 1) throw new RuntimeException("ex"); if (elem == 1) throw new RuntimeException("ex");

View file

@ -345,9 +345,9 @@ class ActorPublisherSpec extends AkkaSpec(ActorPublisherSpec.config) with Implic
val probe2 = TestProbe() val probe2 = TestProbe()
val senderRef1 = system.actorOf(senderProps) 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 sink2: Sink[String, ActorRef] = Sink.actorSubscriber(receiverProps(probe2.ref))
val senderRef2 = RunnableGraph.fromGraph(GraphDSL.create(Source.actorPublisher[Int](senderProps)) { implicit b 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( implicit val materializer = ActorMaterializer(
ActorMaterializerSettings(system).withDispatcher("my-dispatcher1")) ActorMaterializerSettings(system).withDispatcher("my-dispatcher1"))
val s = TestSubscriber.manualProbe[String]() 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 ref ! ThreadName
expectMsgType[String] should include("my-dispatcher1") expectMsgType[String] should include("my-dispatcher1")
} }
@ -430,7 +430,7 @@ class ActorPublisherSpec extends AkkaSpec(ActorPublisherSpec.config) with Implic
val s = TestSubscriber.manualProbe[String]() val s = TestSubscriber.manualProbe[String]()
val ref = Source.actorPublisher(testPublisherProps(testActor, useTestDispatcher = false)) val ref = Source.actorPublisher(testPublisherProps(testActor, useTestDispatcher = false))
.withAttributes(ActorAttributes.dispatcher("my-dispatcher1")) .withAttributes(ActorAttributes.dispatcher("my-dispatcher1"))
.to(Sink(s)).run() .to(Sink.fromSubscriber(s)).run()
ref ! ThreadName ref ! ThreadName
expectMsgType[String] should include("my-dispatcher1") expectMsgType[String] should include("my-dispatcher1")
} }
@ -440,7 +440,7 @@ class ActorPublisherSpec extends AkkaSpec(ActorPublisherSpec.config) with Implic
val s = TestSubscriber.manualProbe[String]() val s = TestSubscriber.manualProbe[String]()
val ref = Source.actorPublisher(testPublisherProps(testActor, useTestDispatcher = false).withDispatcher("my-dispatcher1")) val ref = Source.actorPublisher(testPublisherProps(testActor, useTestDispatcher = false).withDispatcher("my-dispatcher1"))
.withAttributes(ActorAttributes.dispatcher("my-dispatcher2")) .withAttributes(ActorAttributes.dispatcher("my-dispatcher2"))
.to(Sink(s)).run() .to(Sink.fromSubscriber(s)).run()
ref ! ThreadName ref ! ThreadName
expectMsgType[String] should include("my-dispatcher1") expectMsgType[String] should include("my-dispatcher1")
} }

View file

@ -125,7 +125,7 @@ class ActorSubscriberSpec extends AkkaSpec with ImplicitSender {
"signal error" in { "signal error" in {
val e = new RuntimeException("simulated") with NoStackTrace 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" ref ! "ready"
expectMsg(OnError(e)) expectMsg(OnError(e))
} }
@ -133,7 +133,7 @@ class ActorSubscriberSpec extends AkkaSpec with ImplicitSender {
"remember requested after restart" in { "remember requested after restart" in {
// creating actor with default supervision, because stream supervisor default strategy is to stop // creating actor with default supervision, because stream supervisor default strategy is to stop
val ref = system.actorOf(manualSubscriberProps(testActor)) 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" ref ! "ready"
expectMsg(OnNext(1)) expectMsg(OnNext(1))
expectMsg(OnNext(2)) expectMsg(OnNext(2))

View file

@ -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 flow: Flow[Int, Long, _] = Flow[Int].map(_.toLong).timedIntervalBetween(in in % 2 == 1, d probe.ref ! d)
val c1 = TestSubscriber.manualProbe[Long]() 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() val s = c1.expectSubscription()
s.request(100) s.request(100)
@ -104,12 +104,12 @@ class FlowTimedSpec extends AkkaSpec with ScriptedTest {
map(_.toString), duration probe.ref ! duration). map(_.toString), duration probe.ref ! duration).
map { s: String s + "!" } 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 c1 = TestSubscriber.manualProbe[String]()
val c2 = flowOut.subscribe(c1) 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) p.subscribe(flowIn)
val s = c1.expectSubscription() val s = c1.expectSubscription()

View file

@ -41,7 +41,7 @@ class TimeoutsSpec extends AkkaSpec {
val downstreamProbe = TestSubscriber.probe[Int]() val downstreamProbe = TestSubscriber.probe[Int]()
Source.maybe[Int] Source.maybe[Int]
.initialTimeout(1.second) .initialTimeout(1.second)
.runWith(Sink(downstreamProbe)) .runWith(Sink.fromSubscriber(downstreamProbe))
downstreamProbe.expectSubscription() downstreamProbe.expectSubscription()
downstreamProbe.expectNoMsg(500.millis) downstreamProbe.expectNoMsg(500.millis)
@ -73,9 +73,9 @@ class TimeoutsSpec extends AkkaSpec {
"fail if not completed until timeout" in assertAllStagesStopped { "fail if not completed until timeout" in assertAllStagesStopped {
val upstreamProbe = TestPublisher.probe[Int]() val upstreamProbe = TestPublisher.probe[Int]()
val downstreamProbe = TestSubscriber.probe[Int]() val downstreamProbe = TestSubscriber.probe[Int]()
Source(upstreamProbe) Source.fromPublisher(upstreamProbe)
.completionTimeout(2.seconds) .completionTimeout(2.seconds)
.runWith(Sink(downstreamProbe)) .runWith(Sink.fromSubscriber(downstreamProbe))
upstreamProbe.sendNext(1) upstreamProbe.sendNext(1)
downstreamProbe.requestNext(1) downstreamProbe.requestNext(1)
@ -112,9 +112,9 @@ class TimeoutsSpec extends AkkaSpec {
"fail if time between elements is too large" in assertAllStagesStopped { "fail if time between elements is too large" in assertAllStagesStopped {
val upstreamProbe = TestPublisher.probe[Int]() val upstreamProbe = TestPublisher.probe[Int]()
val downstreamProbe = TestSubscriber.probe[Int]() val downstreamProbe = TestSubscriber.probe[Int]()
Source(upstreamProbe) Source.fromPublisher(upstreamProbe)
.idleTimeout(1.seconds) .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 // Two seconds in overall, but won't timeout until time between elements is large enough
// (i.e. this works differently from completionTimeout) // (i.e. this works differently from completionTimeout)
@ -144,8 +144,8 @@ class TimeoutsSpec extends AkkaSpec {
val upstreamWriter = TestPublisher.probe[Int]() val upstreamWriter = TestPublisher.probe[Int]()
val downstreamWriter = TestPublisher.probe[String]() val downstreamWriter = TestPublisher.probe[String]()
val upstream = Flow.fromSinkAndSourceMat(Sink.ignore, Source(upstreamWriter))(Keep.left) val upstream = Flow.fromSinkAndSourceMat(Sink.ignore, Source.fromPublisher(upstreamWriter))(Keep.left)
val downstream = Flow.fromSinkAndSourceMat(Sink.ignore, Source(downstreamWriter))(Keep.left) val downstream = Flow.fromSinkAndSourceMat(Sink.ignore, Source.fromPublisher(downstreamWriter))(Keep.left)
val assembly: RunnableGraph[(Future[Unit], Future[Unit])] = upstream val assembly: RunnableGraph[(Future[Unit], Future[Unit])] = upstream
.joinMat(BidiFlow.bidirectionalIdleTimeout[Int, String](2.seconds))(Keep.left) .joinMat(BidiFlow.bidirectionalIdleTimeout[Int, String](2.seconds))(Keep.left)
@ -177,10 +177,10 @@ class TimeoutsSpec extends AkkaSpec {
RunnableGraph.fromGraph(GraphDSL.create() { implicit b RunnableGraph.fromGraph(GraphDSL.create() { implicit b
import GraphDSL.Implicits._ import GraphDSL.Implicits._
val timeoutStage = b.add(BidiFlow.bidirectionalIdleTimeout[String, Int](2.seconds)) val timeoutStage = b.add(BidiFlow.bidirectionalIdleTimeout[String, Int](2.seconds))
Source(upWrite) ~> timeoutStage.in1; Source.fromPublisher(upWrite) ~> timeoutStage.in1;
timeoutStage.out1 ~> Sink(downRead) timeoutStage.out1 ~> Sink.fromSubscriber(downRead)
Sink(upRead) <~ timeoutStage.out2; Sink.fromSubscriber(upRead) <~ timeoutStage.out2;
timeoutStage.in2 <~ Source(downWrite) timeoutStage.in2 <~ Source.fromPublisher(downWrite)
ClosedShape ClosedShape
}).run() }).run()
@ -225,10 +225,10 @@ class TimeoutsSpec extends AkkaSpec {
RunnableGraph.fromGraph(GraphDSL.create() { implicit b RunnableGraph.fromGraph(GraphDSL.create() { implicit b
import GraphDSL.Implicits._ import GraphDSL.Implicits._
val timeoutStage = b.add(BidiFlow.bidirectionalIdleTimeout[String, Int](2.seconds)) val timeoutStage = b.add(BidiFlow.bidirectionalIdleTimeout[String, Int](2.seconds))
Source(upWrite) ~> timeoutStage.in1; Source.fromPublisher(upWrite) ~> timeoutStage.in1;
timeoutStage.out1 ~> Sink(downRead) timeoutStage.out1 ~> Sink.fromSubscriber(downRead)
Sink(upRead) <~ timeoutStage.out2; Sink.fromSubscriber(upRead) <~ timeoutStage.out2;
timeoutStage.in2 <~ Source(downWrite) timeoutStage.in2 <~ Source.fromPublisher(downWrite)
ClosedShape ClosedShape
}).run() }).run()

View file

@ -98,7 +98,7 @@ class FileSinkSpec extends AkkaSpec(UnboundedMailboxConfig) {
implicit val timeout = Timeout(3.seconds) implicit val timeout = Timeout(3.seconds)
try { 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) materializer.asInstanceOf[ActorMaterializerImpl].supervisor.tell(StreamSupervisor.GetChildren, testActor)
val ref = expectMsgType[Children].children.find(_.path.toString contains "fileSource").get 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) implicit val timeout = Timeout(3.seconds)
try { try {
Source(() Iterator.continually(TestByteStrings.head)) Source.fromIterator(() Iterator.continually(TestByteStrings.head))
.to(Sink.file(f)) .to(Sink.file(f))
.withAttributes(ActorAttributes.dispatcher("akka.actor.default-dispatcher")) .withAttributes(ActorAttributes.dispatcher("akka.actor.default-dispatcher"))
.run()(materializer) .run()(materializer)

View file

@ -76,7 +76,7 @@ class FileSourceSpec extends AkkaSpec(UnboundedMailboxConfig) {
val p = Source.file(testFile, chunkSize) val p = Source.file(testFile, chunkSize)
.withAttributes(bufferAttributes) .withAttributes(bufferAttributes)
.runWith(Sink.publisher(false)) .runWith(Sink.asPublisher(false))
val c = TestSubscriber.manualProbe[ByteString]() val c = TestSubscriber.manualProbe[ByteString]()
p.subscribe(c) p.subscribe(c)
val sub = c.expectSubscription() val sub = c.expectSubscription()
@ -113,7 +113,7 @@ class FileSourceSpec extends AkkaSpec(UnboundedMailboxConfig) {
val p = Source.file(testFile, chunkSize) val p = Source.file(testFile, chunkSize)
.withAttributes(bufferAttributes) .withAttributes(bufferAttributes)
.runWith(Sink.publisher(false)) .runWith(Sink.asPublisher(false))
val c = TestSubscriber.manualProbe[ByteString]() val c = TestSubscriber.manualProbe[ByteString]()
p.subscribe(c) 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 { "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]() val c = TestSubscriber.manualProbe[ByteString]()
p.subscribe(c) p.subscribe(c)

View file

@ -31,7 +31,7 @@ class TcpSpec extends AkkaSpec("akka.stream.materializer.subscription-timeout.ti
val tcpReadProbe = new TcpReadProbe() val tcpReadProbe = new TcpReadProbe()
val tcpWriteProbe = new TcpWriteProbe() 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() val serverConnection = server.waitAccept()
validateServerClientCommunication(testData, serverConnection, tcpReadProbe, tcpWriteProbe) 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 idle = new TcpWriteProbe() // Just register an idle upstream
val resultFuture = val resultFuture =
Source(idle.publisherProbe) Source.fromPublisher(idle.publisherProbe)
.via(Tcp().outgoingConnection(server.address)) .via(Tcp().outgoingConnection(server.address))
.runFold(ByteString.empty)((acc, in) acc ++ in) .runFold(ByteString.empty)((acc, in) acc ++ in)
val serverConnection = server.waitAccept() val serverConnection = server.waitAccept()
@ -81,7 +81,7 @@ class TcpSpec extends AkkaSpec("akka.stream.materializer.subscription-timeout.ti
val tcpWriteProbe = new TcpWriteProbe() val tcpWriteProbe = new TcpWriteProbe()
val tcpReadProbe = new TcpReadProbe() 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() val serverConnection = server.waitAccept()
// Client can still write // Client can still write
@ -111,7 +111,7 @@ class TcpSpec extends AkkaSpec("akka.stream.materializer.subscription-timeout.ti
val tcpWriteProbe = new TcpWriteProbe() val tcpWriteProbe = new TcpWriteProbe()
val tcpReadProbe = new TcpReadProbe() 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() val serverConnection = server.waitAccept()
// Server can still write // Server can still write
@ -139,7 +139,7 @@ class TcpSpec extends AkkaSpec("akka.stream.materializer.subscription-timeout.ti
val tcpWriteProbe = new TcpWriteProbe() val tcpWriteProbe = new TcpWriteProbe()
val tcpReadProbe = new TcpReadProbe() 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() val serverConnection = server.waitAccept()
// Server can still write // Server can still write
@ -171,7 +171,7 @@ class TcpSpec extends AkkaSpec("akka.stream.materializer.subscription-timeout.ti
val tcpWriteProbe = new TcpWriteProbe() val tcpWriteProbe = new TcpWriteProbe()
val tcpReadProbe = new TcpReadProbe() 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() val serverConnection = server.waitAccept()
// Client can still write // Client can still write
@ -204,7 +204,7 @@ class TcpSpec extends AkkaSpec("akka.stream.materializer.subscription-timeout.ti
val tcpWriteProbe = new TcpWriteProbe() val tcpWriteProbe = new TcpWriteProbe()
val tcpReadProbe = new TcpReadProbe() 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() val serverConnection = server.waitAccept()
// Server can still write // Server can still write
@ -234,7 +234,7 @@ class TcpSpec extends AkkaSpec("akka.stream.materializer.subscription-timeout.ti
val tcpWriteProbe = new TcpWriteProbe() val tcpWriteProbe = new TcpWriteProbe()
val tcpReadProbe = new TcpReadProbe() 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() val serverConnection = server.waitAccept()
// Server can still write // Server can still write
@ -261,7 +261,7 @@ class TcpSpec extends AkkaSpec("akka.stream.materializer.subscription-timeout.ti
val tcpWriteProbe = new TcpWriteProbe() val tcpWriteProbe = new TcpWriteProbe()
val tcpReadProbe = new TcpReadProbe() 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() val serverConnection = server.waitAccept()
// Server can still write // Server can still write
@ -290,7 +290,7 @@ class TcpSpec extends AkkaSpec("akka.stream.materializer.subscription-timeout.ti
val tcpWriteProbe = new TcpWriteProbe() val tcpWriteProbe = new TcpWriteProbe()
val tcpReadProbe = new TcpReadProbe() 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() val serverConnection = server.waitAccept()
serverConnection.abort() serverConnection.abort()
@ -312,14 +312,14 @@ class TcpSpec extends AkkaSpec("akka.stream.materializer.subscription-timeout.ti
val outgoingConnection = Tcp().outgoingConnection(server.address) val outgoingConnection = Tcp().outgoingConnection(server.address)
val conn1F = val conn1F =
Source(tcpWriteProbe1.publisherProbe) Source.fromPublisher(tcpWriteProbe1.publisherProbe)
.viaMat(outgoingConnection)(Keep.right) .viaMat(outgoingConnection)(Keep.right)
.to(Sink(tcpReadProbe1.subscriberProbe)).run() .to(Sink.fromSubscriber(tcpReadProbe1.subscriberProbe)).run()
val serverConnection1 = server.waitAccept() val serverConnection1 = server.waitAccept()
val conn2F = val conn2F =
Source(tcpWriteProbe2.publisherProbe) Source.fromPublisher(tcpWriteProbe2.publisherProbe)
.viaMat(outgoingConnection)(Keep.right) .viaMat(outgoingConnection)(Keep.right)
.to(Sink(tcpReadProbe2.subscriberProbe)) .to(Sink.fromSubscriber(tcpReadProbe2.subscriberProbe))
.run() .run()
val serverConnection2 = server.waitAccept() 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 probe1 = TestSubscriber.manualProbe[Tcp.IncomingConnection]()
val bind = Tcp(system).bind(address.getHostName, address.getPort) // TODO getHostString in Java7 val bind = Tcp(system).bind(address.getHostName, address.getPort) // TODO getHostString in Java7
// Bind succeeded, we have a local address // 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() probe1.expectSubscription()
val probe2 = TestSubscriber.manualProbe[Tcp.IncomingConnection]() val probe2 = TestSubscriber.manualProbe[Tcp.IncomingConnection]()
val binding2F = bind.to(Sink(probe2)).run() val binding2F = bind.to(Sink.fromSubscriber(probe2)).run()
probe2.expectSubscriptionAndError(BindFailedException) probe2.expectSubscriptionAndError(BindFailedException)
val probe3 = TestSubscriber.manualProbe[Tcp.IncomingConnection]() val probe3 = TestSubscriber.manualProbe[Tcp.IncomingConnection]()
val binding3F = bind.to(Sink(probe3)).run() val binding3F = bind.to(Sink.fromSubscriber(probe3)).run()
probe3.expectSubscriptionAndError() probe3.expectSubscriptionAndError()
a[BindFailedException] shouldBe thrownBy { Await.result(binding2F, 1.second) } 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]() val probe4 = TestSubscriber.manualProbe[Tcp.IncomingConnection]()
// Bind succeeded, we have a local address // 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() probe4.expectSubscription()
// clean up // clean up

View file

@ -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 { "reliably cancel subscriptions when TransportIn fails early" in assertAllStagesStopped {
val ex = new Exception("hello") val ex = new Exception("hello")
val (sub, out1, out2) = 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) (s, o1, o2)
val tls = b.add(clientTls(EagerClose)) val tls = b.add(clientTls(EagerClose))
s ~> tls.in1; tls.out1 ~> o1 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 { "reliably cancel subscriptions when UserIn fails early" in assertAllStagesStopped {
val ex = new Exception("hello") val ex = new Exception("hello")
val (sub, out1, out2) = 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) (s, o1, o2)
val tls = b.add(clientTls(EagerClose)) val tls = b.add(clientTls(EagerClose))
Source.failed[SslTlsOutbound](ex) ~> tls.in1; tls.out1 ~> o1 Source.failed[SslTlsOutbound](ex) ~> tls.in1; tls.out1 ~> o1

View file

@ -34,7 +34,7 @@ class AcknowledgeSinkSpec extends AkkaSpec {
"allow to have only one future waiting for result in each point of time" in assertAllStagesStopped { "allow to have only one future waiting for result in each point of time" in assertAllStagesStopped {
val probe = TestPublisher.manualProbe[Int]() 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 sub = probe.expectSubscription()
val future = queue.pull() val future = queue.pull()
val future2 = queue.pull() val future2 = queue.pull()
@ -49,7 +49,7 @@ class AcknowledgeSinkSpec extends AkkaSpec {
"wait for next element from upstream" in assertAllStagesStopped { "wait for next element from upstream" in assertAllStagesStopped {
val probe = TestPublisher.manualProbe[Int]() 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 sub = probe.expectSubscription()
queue.pull().pipeTo(testActor) queue.pull().pipeTo(testActor)
@ -62,7 +62,7 @@ class AcknowledgeSinkSpec extends AkkaSpec {
"fail future on stream failure" in assertAllStagesStopped { "fail future on stream failure" in assertAllStagesStopped {
val probe = TestPublisher.manualProbe[Int]() 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 sub = probe.expectSubscription()
queue.pull().pipeTo(testActor) queue.pull().pipeTo(testActor)
@ -74,7 +74,7 @@ class AcknowledgeSinkSpec extends AkkaSpec {
"fail future when stream failed" in assertAllStagesStopped { "fail future when stream failed" in assertAllStagesStopped {
val probe = TestPublisher.manualProbe[Int]() 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() val sub = probe.expectSubscription()
sub.sendError(ex) // potential race condition sub.sendError(ex) // potential race condition
@ -83,7 +83,7 @@ class AcknowledgeSinkSpec extends AkkaSpec {
"timeout future when stream cannot provide data" in assertAllStagesStopped { "timeout future when stream cannot provide data" in assertAllStagesStopped {
val probe = TestPublisher.manualProbe[Int]() 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 sub = probe.expectSubscription()
queue.pull().pipeTo(testActor) queue.pull().pipeTo(testActor)
@ -96,7 +96,7 @@ class AcknowledgeSinkSpec extends AkkaSpec {
"work when buffer is 0" in assertAllStagesStopped { "work when buffer is 0" in assertAllStagesStopped {
val probe = TestPublisher.manualProbe[Int]() 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() val sub = probe.expectSubscription()
sub.sendNext(1) sub.sendNext(1)

View file

@ -22,7 +22,7 @@ class AcknowledgeSourceSpec extends AkkaSpec {
"emit received messages to the stream" in { "emit received messages to the stream" in {
val s = TestSubscriber.manualProbe[Int]() 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 val sub = s.expectSubscription
sub.request(2) sub.request(2)
assertSuccess(true, queue.offer(1)) assertSuccess(true, queue.offer(1))
@ -35,7 +35,7 @@ class AcknowledgeSourceSpec extends AkkaSpec {
"buffer when needed" in { "buffer when needed" in {
val s = TestSubscriber.manualProbe[Int]() 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 val sub = s.expectSubscription
for (n 1 to 20) assertSuccess(true, queue.offer(n)) for (n 1 to 20) assertSuccess(true, queue.offer(n))
sub.request(10) sub.request(10)
@ -51,7 +51,7 @@ class AcknowledgeSourceSpec extends AkkaSpec {
"not fail when 0 buffer space and demand is signalled" in assertAllStagesStopped { "not fail when 0 buffer space and demand is signalled" in assertAllStagesStopped {
val s = TestSubscriber.manualProbe[Int]() 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 val sub = s.expectSubscription
sub.request(1) sub.request(1)
assertSuccess(true, queue.offer(1)) assertSuccess(true, queue.offer(1))
@ -61,7 +61,7 @@ class AcknowledgeSourceSpec extends AkkaSpec {
"return false when can reject element to buffer" in assertAllStagesStopped { "return false when can reject element to buffer" in assertAllStagesStopped {
val s = TestSubscriber.manualProbe[Int]() 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 val sub = s.expectSubscription
assertSuccess(true, queue.offer(1)) assertSuccess(true, queue.offer(1))
assertSuccess(false, queue.offer(2)) assertSuccess(false, queue.offer(2))
@ -72,7 +72,7 @@ class AcknowledgeSourceSpec extends AkkaSpec {
"wait when buffer is full and backpressure is on" in assertAllStagesStopped { "wait when buffer is full and backpressure is on" in assertAllStagesStopped {
val s = TestSubscriber.manualProbe[Int]() 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 val sub = s.expectSubscription
assertSuccess(true, queue.offer(1)) assertSuccess(true, queue.offer(1))

View file

@ -19,7 +19,7 @@ class ActorRefSourceSpec extends AkkaSpec {
"emit received messages to the stream" in { "emit received messages to the stream" in {
val s = TestSubscriber.manualProbe[Int]() 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 sub = s.expectSubscription
sub.request(2) sub.request(2)
ref ! 1 ref ! 1
@ -32,7 +32,7 @@ class ActorRefSourceSpec extends AkkaSpec {
"buffer when needed" in { "buffer when needed" in {
val s = TestSubscriber.manualProbe[Int]() 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 val sub = s.expectSubscription
for (n 1 to 20) ref ! n for (n 1 to 20) ref ! n
sub.request(10) sub.request(10)
@ -61,7 +61,7 @@ class ActorRefSourceSpec extends AkkaSpec {
"terminate when the stream is cancelled" in assertAllStagesStopped { "terminate when the stream is cancelled" in assertAllStagesStopped {
val s = TestSubscriber.manualProbe[Int]() 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) watch(ref)
val sub = s.expectSubscription val sub = s.expectSubscription
sub.cancel() sub.cancel()
@ -70,7 +70,7 @@ class ActorRefSourceSpec extends AkkaSpec {
"not fail when 0 buffer space and demand is signalled" in assertAllStagesStopped { "not fail when 0 buffer space and demand is signalled" in assertAllStagesStopped {
val s = TestSubscriber.manualProbe[Int]() 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) watch(ref)
val sub = s.expectSubscription val sub = s.expectSubscription
sub.request(100) sub.request(100)
@ -80,7 +80,7 @@ class ActorRefSourceSpec extends AkkaSpec {
"complete the stream immediatly when receiving PoisonPill" in assertAllStagesStopped { "complete the stream immediatly when receiving PoisonPill" in assertAllStagesStopped {
val s = TestSubscriber.manualProbe[Int]() 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 sub = s.expectSubscription
ref ! PoisonPill ref ! PoisonPill
s.expectComplete() s.expectComplete()
@ -88,7 +88,7 @@ class ActorRefSourceSpec extends AkkaSpec {
"signal buffered elements and complete the stream after receiving Status.Success" in assertAllStagesStopped { "signal buffered elements and complete the stream after receiving Status.Success" in assertAllStagesStopped {
val s = TestSubscriber.manualProbe[Int]() 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 val sub = s.expectSubscription
ref ! 1 ref ! 1
ref ! 2 ref ! 2
@ -101,7 +101,7 @@ class ActorRefSourceSpec extends AkkaSpec {
"not buffer elements after receiving Status.Success" in assertAllStagesStopped { "not buffer elements after receiving Status.Success" in assertAllStagesStopped {
val s = TestSubscriber.manualProbe[Int]() 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 val sub = s.expectSubscription
ref ! 1 ref ! 1
ref ! 2 ref ! 2
@ -117,7 +117,7 @@ class ActorRefSourceSpec extends AkkaSpec {
"after receiving Status.Success, allow for earlier completion with PoisonPill" in assertAllStagesStopped { "after receiving Status.Success, allow for earlier completion with PoisonPill" in assertAllStagesStopped {
val s = TestSubscriber.manualProbe[Int]() 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 val sub = s.expectSubscription
ref ! 1 ref ! 1
ref ! 2 ref ! 2
@ -131,7 +131,7 @@ class ActorRefSourceSpec extends AkkaSpec {
"fail the stream when receiving Status.Failure" in assertAllStagesStopped { "fail the stream when receiving Status.Failure" in assertAllStagesStopped {
val s = TestSubscriber.manualProbe[Int]() 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 sub = s.expectSubscription
val exc = TE("testfailure") val exc = TE("testfailure")
ref ! Status.Failure(exc) ref ! Status.Failure(exc)

View file

@ -18,11 +18,11 @@ class FlowAppendSpec extends AkkaSpec with River {
"Flow" should { "Flow" should {
"append Flow" in riverOf[String] { subscriber "append Flow" in riverOf[String] { subscriber
val flow = Flow[Int].via(otherFlow) 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 "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() Source(elements).to(sink).run()
} }
} }
@ -31,12 +31,12 @@ class FlowAppendSpec extends AkkaSpec with River {
"append Flow" in riverOf[String] { subscriber "append Flow" in riverOf[String] { subscriber
Source(elements) Source(elements)
.via(otherFlow) .via(otherFlow)
.to(Sink(subscriber)).run() .to(Sink.fromSubscriber(subscriber)).run()
} }
"append Sink" in riverOf[String] { subscriber "append Sink" in riverOf[String] { subscriber
Source(elements) Source(elements)
.to(otherFlow.to(Sink(subscriber))) .to(otherFlow.to(Sink.fromSubscriber(subscriber)))
.run() .run()
} }
} }

View file

@ -51,7 +51,7 @@ class FlowBufferSpec extends AkkaSpec {
val publisher = TestPublisher.probe[Int]() val publisher = TestPublisher.probe[Int]()
val subscriber = TestSubscriber.manualProbe[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() val sub = subscriber.expectSubscription()
// Fill up buffer // Fill up buffer
@ -69,7 +69,7 @@ class FlowBufferSpec extends AkkaSpec {
val publisher = TestPublisher.probe[Int]() val publisher = TestPublisher.probe[Int]()
val subscriber = TestSubscriber.manualProbe[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() val sub = subscriber.expectSubscription()
// Fill up buffer // Fill up buffer
@ -98,7 +98,7 @@ class FlowBufferSpec extends AkkaSpec {
val publisher = TestPublisher.probe[Int]() val publisher = TestPublisher.probe[Int]()
val subscriber = TestSubscriber.manualProbe[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() val sub = subscriber.expectSubscription()
// Fill up buffer // Fill up buffer
@ -130,7 +130,7 @@ class FlowBufferSpec extends AkkaSpec {
val publisher = TestPublisher.probe[Int]() val publisher = TestPublisher.probe[Int]()
val subscriber = TestSubscriber.manualProbe[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() val sub = subscriber.expectSubscription()
// Fill up buffer // Fill up buffer
@ -184,7 +184,7 @@ class FlowBufferSpec extends AkkaSpec {
val publisher = TestPublisher.probe[Int]() val publisher = TestPublisher.probe[Int]()
val subscriber = TestSubscriber.manualProbe[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() val sub = subscriber.expectSubscription()
// Fill up buffer // Fill up buffer
@ -211,7 +211,7 @@ class FlowBufferSpec extends AkkaSpec {
val publisher = TestPublisher.probe[Int]() val publisher = TestPublisher.probe[Int]()
val subscriber = TestSubscriber.manualProbe[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() val sub = subscriber.expectSubscription()
// Fill up buffer // Fill up buffer

View file

@ -18,7 +18,7 @@ class FlowCompileSpec extends AkkaSpec {
val strSeq = Source(Seq("a", "b", "c")) val strSeq = Source(Seq("a", "b", "c"))
import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.ExecutionContext.Implicits.global
val intFut = Source(Future { 3 }) val intFut = Source.fromFuture(Future { 3 })
implicit val materializer = ActorMaterializer(ActorMaterializerSettings(system)) implicit val materializer = ActorMaterializer(ActorMaterializerSettings(system))
"Flow" should { "Flow" should {
@ -41,15 +41,15 @@ class FlowCompileSpec extends AkkaSpec {
val closedSource: Source[Int, _] = intSeq.via(open3) val closedSource: Source[Int, _] = intSeq.via(open3)
"closedSource.run()" shouldNot compile "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 "closedSink.run()" shouldNot compile
closedSource.to(Sink.publisher[Int](false)).run() closedSource.to(Sink.asPublisher[Int](false)).run()
intSeq.to(closedSink).run() intSeq.to(closedSink).run()
} }
"append Sink" in { "append Sink" in {
val open: Flow[Int, String, _] = Flow[Int].map(_.toString) 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) val appended: Sink[Int, _] = open.to(closedSink)
"appended.run()" shouldNot compile "appended.run()" shouldNot compile
"appended.connect(Sink.head[Int])" shouldNot compile "appended.connect(Sink.head[Int])" shouldNot compile
@ -61,13 +61,13 @@ class FlowCompileSpec extends AkkaSpec {
val closedSource2: Source[String, _] = closedSource.via(open) val closedSource2: Source[String, _] = closedSource.via(open)
"closedSource2.run()" shouldNot compile "closedSource2.run()" shouldNot compile
"strSeq.connect(closedSource2)" shouldNot compile "strSeq.connect(closedSource2)" shouldNot compile
closedSource2.to(Sink.publisher[String](false)).run closedSource2.to(Sink.asPublisher[String](false)).run
} }
} }
"Sink" should { "Sink" should {
val openSource: Sink[Int, _] = 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 { "accept Source" in {
intSeq.to(openSource) intSeq.to(openSource)
} }
@ -83,7 +83,7 @@ class FlowCompileSpec extends AkkaSpec {
val openSource: Source[String, _] = val openSource: Source[String, _] =
Source(Seq(1, 2, 3)).map(_.toString) Source(Seq(1, 2, 3)).map(_.toString)
"accept Sink" in { "accept Sink" in {
openSource.to(Sink.publisher[String](false)) openSource.to(Sink.asPublisher[String](false))
} }
"not be accepted by Source" in { "not be accepted by Source" in {
"openSource.connect(intSeq)" shouldNot compile "openSource.connect(intSeq)" shouldNot compile
@ -96,7 +96,7 @@ class FlowCompileSpec extends AkkaSpec {
"RunnableGraph" should { "RunnableGraph" should {
Sink.head[String] Sink.head[String]
val closed: RunnableGraph[Publisher[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 { "run" in {
closed.run() closed.run()
} }

View file

@ -33,7 +33,7 @@ class FlowConcatAllSpec extends AkkaSpec {
val main = Source(List(s1, s2, s3, s4, s5)) val main = Source(List(s1, s2, s3, s4, s5))
val subscriber = TestSubscriber.manualProbe[Int]() 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() val subscription = subscriber.expectSubscription()
subscription.request(10) subscription.request(10)
for (i 1 to 10) for (i 1 to 10)
@ -50,7 +50,7 @@ class FlowConcatAllSpec extends AkkaSpec {
.map(_._2) .map(_._2)
.concatSubstreams .concatSubstreams
.flatMapConcat(ConstantFun.scalaIdentityFunction) .flatMapConcat(ConstantFun.scalaIdentityFunction)
.runWith(Sink(subscriber)) .runWith(Sink.fromSubscriber(subscriber))
val subscription = subscriber.expectSubscription() val subscription = subscriber.expectSubscription()
subscription.request(10) subscription.request(10)
for (i (1 to 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 { "on onError on master stream cancel the current open substream and signal error" in assertAllStagesStopped {
val publisher = TestPublisher.manualProbe[Source[Int, Unit]]() val publisher = TestPublisher.manualProbe[Source[Int, Unit]]()
val subscriber = TestSubscriber.manualProbe[Int]() 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 upstream = publisher.expectSubscription()
val downstream = subscriber.expectSubscription() val downstream = subscriber.expectSubscription()
downstream.request(1000) downstream.request(1000)
val substreamPublisher = TestPublisher.manualProbe[Int]() val substreamPublisher = TestPublisher.manualProbe[Int]()
val substreamFlow = Source(substreamPublisher) val substreamFlow = Source.fromPublisher(substreamPublisher)
upstream.expectRequest() upstream.expectRequest()
upstream.sendNext(substreamFlow) upstream.sendNext(substreamFlow)
val subUpstream = substreamPublisher.expectSubscription() 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 { "on onError on master stream cancel the currently opening substream and signal error" in assertAllStagesStopped {
val publisher = TestPublisher.manualProbe[Source[Int, Unit]]() val publisher = TestPublisher.manualProbe[Source[Int, Unit]]()
val subscriber = TestSubscriber.manualProbe[Int]() 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 upstream = publisher.expectSubscription()
val downstream = subscriber.expectSubscription() val downstream = subscriber.expectSubscription()
downstream.request(1000) downstream.request(1000)
val substreamPublisher = TestPublisher.manualProbe[Int](autoOnSubscribe = false) val substreamPublisher = TestPublisher.manualProbe[Int](autoOnSubscribe = false)
val substreamFlow = Source(substreamPublisher) val substreamFlow = Source.fromPublisher(substreamPublisher)
upstream.expectRequest() upstream.expectRequest()
upstream.sendNext(substreamFlow) upstream.sendNext(substreamFlow)
val subUpstream = substreamPublisher.expectSubscription() 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 { "on onError on opening substream, cancel the master stream and signal error " in assertAllStagesStopped {
val publisher = TestPublisher.manualProbe[Source[Int, _]]() val publisher = TestPublisher.manualProbe[Source[Int, _]]()
val subscriber = TestSubscriber.manualProbe[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 upstream = publisher.expectSubscription()
val downstream = subscriber.expectSubscription() val downstream = subscriber.expectSubscription()
downstream.request(1000) downstream.request(1000)
val substreamPublisher = TestPublisher.manualProbe[Int]() val substreamPublisher = TestPublisher.manualProbe[Int]()
val substreamFlow = Source(substreamPublisher) val substreamFlow = Source.fromPublisher(substreamPublisher)
upstream.expectRequest() upstream.expectRequest()
upstream.sendNext(substreamFlow) upstream.sendNext(substreamFlow)
subscriber.expectError(testException) 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 { "on onError on open substream, cancel the master stream and signal error " in assertAllStagesStopped {
val publisher = TestPublisher.manualProbe[Source[Int, Unit]]() val publisher = TestPublisher.manualProbe[Source[Int, Unit]]()
val subscriber = TestSubscriber.manualProbe[Int]() 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 upstream = publisher.expectSubscription()
val downstream = subscriber.expectSubscription() val downstream = subscriber.expectSubscription()
downstream.request(1000) downstream.request(1000)
val substreamPublisher = TestPublisher.manualProbe[Int]() val substreamPublisher = TestPublisher.manualProbe[Int]()
val substreamFlow = Source(substreamPublisher) val substreamFlow = Source.fromPublisher(substreamPublisher)
upstream.expectRequest() upstream.expectRequest()
upstream.sendNext(substreamFlow) upstream.sendNext(substreamFlow)
val subUpstream = substreamPublisher.expectSubscription() 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 { "on cancellation cancel the current open substream and the master stream" in assertAllStagesStopped {
val publisher = TestPublisher.manualProbe[Source[Int, Unit]]() val publisher = TestPublisher.manualProbe[Source[Int, Unit]]()
val subscriber = TestSubscriber.manualProbe[Int]() 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 upstream = publisher.expectSubscription()
val downstream = subscriber.expectSubscription() val downstream = subscriber.expectSubscription()
downstream.request(1000) downstream.request(1000)
val substreamPublisher = TestPublisher.manualProbe[Int]() val substreamPublisher = TestPublisher.manualProbe[Int]()
val substreamFlow = Source(substreamPublisher) val substreamFlow = Source.fromPublisher(substreamPublisher)
upstream.expectRequest() upstream.expectRequest()
upstream.sendNext(substreamFlow) upstream.sendNext(substreamFlow)
val subUpstream = substreamPublisher.expectSubscription() 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 { "on cancellation cancel the currently opening substream and the master stream" in assertAllStagesStopped {
val publisher = TestPublisher.manualProbe[Source[Int, Unit]]() val publisher = TestPublisher.manualProbe[Source[Int, Unit]]()
val subscriber = TestSubscriber.manualProbe[Int]() 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 upstream = publisher.expectSubscription()
val downstream = subscriber.expectSubscription() val downstream = subscriber.expectSubscription()
downstream.request(1000) downstream.request(1000)
val substreamPublisher = TestPublisher.manualProbe[Int](autoOnSubscribe = false) val substreamPublisher = TestPublisher.manualProbe[Int](autoOnSubscribe = false)
val substreamFlow = Source(substreamPublisher) val substreamFlow = Source.fromPublisher(substreamPublisher)
upstream.expectRequest() upstream.expectRequest()
upstream.sendNext(substreamFlow) upstream.sendNext(substreamFlow)
val subUpstream = substreamPublisher.expectSubscription() val subUpstream = substreamPublisher.expectSubscription()
@ -188,9 +188,9 @@ class FlowConcatAllSpec extends AkkaSpec {
val down = TestSubscriber.manualProbe[Int]() val down = TestSubscriber.manualProbe[Int]()
val flowSubscriber = Source val flowSubscriber = Source
.subscriber[Source[Int, Unit]] .asSubscriber[Source[Int, Unit]]
.flatMapConcat(ConstantFun.scalaIdentityFunction) .flatMapConcat(ConstantFun.scalaIdentityFunction)
.to(Sink(down)) .to(Sink.fromSubscriber(down))
.run() .run()
val downstream = down.expectSubscription() val downstream = down.expectSubscription()

View file

@ -17,7 +17,7 @@ class FlowConcatSpec extends BaseTwoStreamsSetup {
override def setup(p1: Publisher[Int], p2: Publisher[Int]) = { override def setup(p1: Publisher[Int], p2: Publisher[Int]) = {
val subscriber = TestSubscriber.probe[Outputs]() 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 subscriber
} }
@ -29,7 +29,7 @@ class FlowConcatSpec extends BaseTwoStreamsSetup {
val s2: Source[String, _] = Source(List(4, 5, 6)).map(_.toString + "-s") val s2: Source[String, _] = Source(List(4, 5, 6)).map(_.toString + "-s")
val subs = TestSubscriber.manualProbe[Any]() 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) 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 { "correctly handle async errors in secondary upstream" in assertAllStagesStopped {
val promise = Promise[Int]() val promise = Promise[Int]()
val subscriber = TestSubscriber.manualProbe[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() val subscription = subscriber.expectSubscription()
subscription.request(4) 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 { "subscribe at once to initial source and to one that it's concat to" in {
val publisher1 = TestPublisher.probe[Int]() val publisher1 = TestPublisher.probe[Int]()
val publisher2 = 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]) .runWith(TestSink.probe[Int])
val sub1 = publisher1.expectSubscription() val sub1 = publisher1.expectSubscription()

View file

@ -22,7 +22,7 @@ class FlowConflateSpec extends AkkaSpec {
val publisher = TestPublisher.probe[Int]() val publisher = TestPublisher.probe[Int]()
val subscriber = TestSubscriber.manualProbe[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() val sub = subscriber.expectSubscription()
for (i 1 to 100) { for (i 1 to 100) {
@ -38,7 +38,7 @@ class FlowConflateSpec extends AkkaSpec {
val publisher = TestPublisher.probe[Int]() val publisher = TestPublisher.probe[Int]()
val subscriber = TestSubscriber.manualProbe[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() val sub = subscriber.expectSubscription()
for (i 1 to 100) { for (i 1 to 100) {
@ -62,7 +62,7 @@ class FlowConflateSpec extends AkkaSpec {
val publisher = TestPublisher.probe[Int]() val publisher = TestPublisher.probe[Int]()
val subscriber = TestSubscriber.manualProbe[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() val sub = subscriber.expectSubscription()
sub.request(1) sub.request(1)

View file

@ -49,7 +49,7 @@ class FlowDelaySpec extends AkkaSpec {
val c = TestSubscriber.manualProbe[Int]() val c = TestSubscriber.manualProbe[Int]()
val p = TestPublisher.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 cSub = c.expectSubscription()
val pSub = p.expectSubscription() val pSub = p.expectSubscription()
cSub.request(100) cSub.request(100)
@ -111,7 +111,7 @@ class FlowDelaySpec extends AkkaSpec {
val c = TestSubscriber.manualProbe[Int]() val c = TestSubscriber.manualProbe[Int]()
val p = TestPublisher.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 cSub = c.expectSubscription()
val pSub = p.expectSubscription() val pSub = p.expectSubscription()
cSub.request(20) cSub.request(20)

View file

@ -28,7 +28,7 @@ class FlowDropSpec extends AkkaSpec with ScriptedTest {
"not drop anything for negative n" in { "not drop anything for negative n" in {
val probe = TestSubscriber.manualProbe[Int]() 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.expectSubscription().request(10)
probe.expectNext(1) probe.expectNext(1)
probe.expectNext(2) probe.expectNext(2)

View file

@ -18,7 +18,7 @@ class FlowDropWithinSpec extends AkkaSpec {
val input = Iterator.from(1) val input = Iterator.from(1)
val p = TestPublisher.manualProbe[Int]() val p = TestPublisher.manualProbe[Int]()
val c = TestSubscriber.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 pSub = p.expectSubscription
val cSub = c.expectSubscription val cSub = c.expectSubscription
cSub.request(100) cSub.request(100)
@ -39,7 +39,7 @@ class FlowDropWithinSpec extends AkkaSpec {
val upstream = TestPublisher.probe[Int]() val upstream = TestPublisher.probe[Int]()
val downstream = TestSubscriber.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() upstream.sendComplete()
downstream.expectSubscriptionAndComplete() downstream.expectSubscriptionAndComplete()

View file

@ -28,7 +28,7 @@ class FlowExpandSpec extends AkkaSpec {
val subscriber = TestSubscriber.probe[Int]() val subscriber = TestSubscriber.probe[Int]()
// Simply repeat the last element as an extrapolation step // 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) { for (i 1 to 100) {
// Order is important here: If the request comes first it will be extrapolated! // 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]() val subscriber = TestSubscriber.probe[Int]()
// Simply repeat the last element as an extrapolation step // 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) publisher.sendNext(42)
@ -66,7 +66,7 @@ class FlowExpandSpec extends AkkaSpec {
val subscriber = TestSubscriber.probe[Int]() val subscriber = TestSubscriber.probe[Int]()
// Simply repeat the last element as an extrapolation step // 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) publisher.sendNext(1)
subscriber.requestNext(1) subscriber.requestNext(1)
@ -94,7 +94,7 @@ class FlowExpandSpec extends AkkaSpec {
val publisher = TestPublisher.probe[Int]() val publisher = TestPublisher.probe[Int]()
val subscriber = TestSubscriber.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) publisher.sendNext(1)
subscriber.requestNext(1) subscriber.requestNext(1)

View file

@ -27,7 +27,7 @@ class FlowFilterSpec extends AkkaSpec with ScriptedTest {
implicit val materializer = ActorMaterializer(settings) implicit val materializer = ActorMaterializer(settings)
val probe = TestSubscriber.manualProbe[Int]() 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() val subscription = probe.expectSubscription()
for (_ 1 to 10000) { for (_ 1 to 10000) {

View file

@ -19,7 +19,7 @@ class FlowFlattenMergeSpec extends AkkaSpec with ScalaFutures with ConversionChe
import system.dispatcher import system.dispatcher
def src10(i: Int) = Source(i until (i + 10)) 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 toSeq = Flow[Int].grouped(1000).toMat(Sink.head)(Keep.right)
val toSet = toSeq.mapMaterializedValue(_.map(_.toSet)) 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 p1, p2 = TestPublisher.probe[Int]()
val ex = new Exception("buh") val ex = new Exception("buh")
val p = Promise[Source[Int, Unit]] 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) .flatMapMerge(5, identity)
.runWith(Sink.head) .runWith(Sink.head)
p1.expectRequest() p1.expectRequest()
@ -112,7 +112,7 @@ class FlowFlattenMergeSpec extends AkkaSpec with ScalaFutures with ConversionChe
val p1, p2 = TestPublisher.probe[Int]() val p1, p2 = TestPublisher.probe[Int]()
val ex = new Exception("buh") val ex = new Exception("buh")
val p = Promise[Int] 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) .flatMapMerge(5, identity)
.runWith(Sink.head) .runWith(Sink.head)
p1.expectRequest() p1.expectRequest()
@ -128,8 +128,8 @@ class FlowFlattenMergeSpec extends AkkaSpec with ScalaFutures with ConversionChe
val latch = TestLatch() val latch = TestLatch()
Source(1 to 3) Source(1 to 3)
.flatMapMerge(10, { .flatMapMerge(10, {
case 1 Source(p1) case 1 Source.fromPublisher(p1)
case 2 Source(p2) case 2 Source.fromPublisher(p2)
case 3 case 3
Await.ready(latch, 3.seconds) Await.ready(latch, 3.seconds)
throw ex throw ex
@ -145,7 +145,7 @@ class FlowFlattenMergeSpec extends AkkaSpec with ScalaFutures with ConversionChe
"cancel substreams when being cancelled" in { "cancel substreams when being cancelled" in {
val p1, p2 = TestPublisher.probe[Int]() val p1, p2 = TestPublisher.probe[Int]()
val ex = new Exception("buh") 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) .flatMapMerge(5, identity)
.runWith(TestSink.probe) .runWith(TestSink.probe)
sink.request(1) sink.request(1)

View file

@ -36,7 +36,7 @@ class FlowForeachSpec extends AkkaSpec {
"yield the first error" in assertAllStagesStopped { "yield the first error" in assertAllStagesStopped {
val p = TestPublisher.manualProbe[Int]() val p = TestPublisher.manualProbe[Int]()
Source(p).runForeach(testActor ! _) onFailure { Source.fromPublisher(p).runForeach(testActor ! _) onFailure {
case ex testActor ! ex case ex testActor ! ex
} }
val proc = p.expectSubscription() val proc = p.expectSubscription()

View file

@ -21,7 +21,7 @@ class FlowFromFutureSpec extends AkkaSpec {
"A Flow based on a Future" must { "A Flow based on a Future" must {
"produce one element from already successful Future" in assertAllStagesStopped { "produce one element from already successful Future" in assertAllStagesStopped {
val c = TestSubscriber.manualProbe[Int]() 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() val sub = c.expectSubscription()
c.expectNoMsg(100.millis) c.expectNoMsg(100.millis)
sub.request(1) sub.request(1)
@ -32,14 +32,14 @@ class FlowFromFutureSpec extends AkkaSpec {
"produce error from already failed Future" in assertAllStagesStopped { "produce error from already failed Future" in assertAllStagesStopped {
val ex = new RuntimeException("test") with NoStackTrace val ex = new RuntimeException("test") with NoStackTrace
val c = TestSubscriber.manualProbe[Int]() 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) c.expectSubscriptionAndError(ex)
} }
"produce one element when Future is completed" in assertAllStagesStopped { "produce one element when Future is completed" in assertAllStagesStopped {
val promise = Promise[Int]() val promise = Promise[Int]()
val c = TestSubscriber.manualProbe[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() val sub = c.expectSubscription()
sub.request(1) sub.request(1)
c.expectNoMsg(100.millis) c.expectNoMsg(100.millis)
@ -52,7 +52,7 @@ class FlowFromFutureSpec extends AkkaSpec {
"produce one element when Future is completed but not before request" in { "produce one element when Future is completed but not before request" in {
val promise = Promise[Int]() val promise = Promise[Int]()
val c = TestSubscriber.manualProbe[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() val sub = c.expectSubscription()
promise.success(1) promise.success(1)
c.expectNoMsg(200.millis) c.expectNoMsg(200.millis)
@ -63,7 +63,7 @@ class FlowFromFutureSpec extends AkkaSpec {
"produce elements with multiple subscribers" in assertAllStagesStopped { "produce elements with multiple subscribers" in assertAllStagesStopped {
val promise = Promise[Int]() 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 c1 = TestSubscriber.manualProbe[Int]()
val c2 = TestSubscriber.manualProbe[Int]() val c2 = TestSubscriber.manualProbe[Int]()
p.subscribe(c1) p.subscribe(c1)
@ -81,7 +81,7 @@ class FlowFromFutureSpec extends AkkaSpec {
"allow cancel before receiving element" in { "allow cancel before receiving element" in {
val promise = Promise[Int]() 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 keepAlive = TestSubscriber.manualProbe[Int]()
val c = TestSubscriber.manualProbe[Int]() val c = TestSubscriber.manualProbe[Int]()
p.subscribe(keepAlive) p.subscribe(keepAlive)

View file

@ -36,7 +36,7 @@ class FlowGraphCompileSpec extends AkkaSpec {
val in1 = Source(List("a", "b", "c")) val in1 = Source(List("a", "b", "c"))
val in2 = Source(List("d", "e", "f")) val in2 = Source(List("d", "e", "f"))
val out1 = Sink.publisher[String](false) val out1 = Sink.asPublisher[String](false)
val out2 = Sink.head[String] val out2 = Sink.head[String]
"A Graph" should { "A Graph" should {
@ -165,9 +165,9 @@ class FlowGraphCompileSpec extends AkkaSpec {
val in3 = Source(List("b")) val in3 = Source(List("b"))
val in5 = Source(List("b")) val in5 = Source(List("b"))
val in7 = Source(List("a")) val in7 = Source(List("a"))
val out2 = Sink.publisher[String](false) val out2 = Sink.asPublisher[String](false)
val out9 = Sink.publisher[String](false) val out9 = Sink.asPublisher[String](false)
val out10 = Sink.publisher[String](false) val out10 = Sink.asPublisher[String](false)
def f(s: String) = Flow[String].transform(op[String, String]).named(s) def f(s: String) = Flow[String].transform(op[String, String]).named(s)
import GraphDSL.Implicits._ import GraphDSL.Implicits._
@ -198,7 +198,7 @@ class FlowGraphCompileSpec extends AkkaSpec {
RunnableGraph.fromGraph(GraphDSL.create() { implicit b RunnableGraph.fromGraph(GraphDSL.create() { implicit b
val zip = b.add(Zip[Int, String]()) val zip = b.add(Zip[Int, String]())
val unzip = b.add(Unzip[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._ import GraphDSL.Implicits._
Source(List(1 -> "a", 2 -> "b", 3 -> "c")) ~> unzip.in Source(List(1 -> "a", 2 -> "b", 3 -> "c")) ~> unzip.in
unzip.out0 ~> Flow[Int].map(_ * 2) ~> zip.in0 unzip.out0 ~> Flow[Int].map(_ * 2) ~> zip.in0
@ -213,8 +213,8 @@ class FlowGraphCompileSpec extends AkkaSpec {
RunnableGraph.fromGraph(GraphDSL.create() { implicit b RunnableGraph.fromGraph(GraphDSL.create() { implicit b
val zip = b.add(Zip[Int, String]()) val zip = b.add(Zip[Int, String]())
val unzip = b.add(Unzip[Int, String]()) val unzip = b.add(Unzip[Int, String]())
val wrongOut = Sink.publisher[(Int, Int)](false) val wrongOut = Sink.asPublisher[(Int, Int)](false)
val whatever = Sink.publisher[Any](false) val whatever = Sink.asPublisher[Any](false)
"Flow(List(1, 2, 3)) ~> zip.left ~> wrongOut" shouldNot compile "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.left""" shouldNot compile
"""Flow(List("a", "b", "c")) ~> zip.out""" 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 RunnableGraph.fromGraph(GraphDSL.create() { implicit b
import GraphDSL.Implicits._ import GraphDSL.Implicits._
val merge = b.add(Merge[Fruit](2)) val merge = b.add(Merge[Fruit](2))
Source[Fruit](apples) ~> Flow[Fruit] ~> merge.in(0) Source.fromIterator[Fruit](apples) ~> Flow[Fruit] ~> merge.in(0)
Source[Apple](apples) ~> Flow[Apple] ~> merge.in(1) Source.fromIterator[Apple](apples) ~> Flow[Apple] ~> merge.in(1)
merge.out ~> Flow[Fruit].map(identity) ~> Sink(TestSubscriber.manualProbe[Fruit]()) merge.out ~> Flow[Fruit].map(identity) ~> Sink.fromSubscriber(TestSubscriber.manualProbe[Fruit]())
ClosedShape ClosedShape
}) })
} }
@ -241,31 +241,31 @@ class FlowGraphCompileSpec extends AkkaSpec {
RunnableGraph.fromGraph(GraphDSL.create() { implicit b RunnableGraph.fromGraph(GraphDSL.create() { implicit b
import GraphDSL.Implicits._ import GraphDSL.Implicits._
val fruitMerge = b.add(Merge[Fruit](2)) val fruitMerge = b.add(Merge[Fruit](2))
Source[Fruit](apples) ~> fruitMerge Source.fromIterator[Fruit](apples) ~> fruitMerge
Source[Apple](apples) ~> fruitMerge Source.fromIterator[Apple](apples) ~> fruitMerge
fruitMerge ~> Sink.head[Fruit] fruitMerge ~> Sink.head[Fruit]
"fruitMerge ~> Sink.head[Apple]" shouldNot compile "fruitMerge ~> Sink.head[Apple]" shouldNot compile
val appleMerge = b.add(Merge[Apple](2)) val appleMerge = b.add(Merge[Apple](2))
"Source[Fruit](apples) ~> appleMerge" shouldNot compile "Source[Fruit](apples) ~> appleMerge" shouldNot compile
Source.empty[Apple] ~> appleMerge Source.empty[Apple] ~> appleMerge
Source[Apple](apples) ~> appleMerge Source.fromIterator[Apple](apples) ~> appleMerge
appleMerge ~> Sink.head[Fruit] appleMerge ~> Sink.head[Fruit]
val appleMerge2 = b.add(Merge[Apple](2)) val appleMerge2 = b.add(Merge[Apple](2))
Source.empty[Apple] ~> appleMerge2 Source.empty[Apple] ~> appleMerge2
Source[Apple](apples) ~> appleMerge2 Source.fromIterator[Apple](apples) ~> appleMerge2
appleMerge2 ~> Sink.head[Apple] appleMerge2 ~> Sink.head[Apple]
val fruitBcast = b.add(Broadcast[Fruit](2)) val fruitBcast = b.add(Broadcast[Fruit](2))
Source[Apple](apples) ~> fruitBcast Source.fromIterator[Apple](apples) ~> fruitBcast
fruitBcast ~> Sink.head[Fruit] fruitBcast ~> Sink.head[Fruit]
fruitBcast ~> Sink.ignore fruitBcast ~> Sink.ignore
"fruitBcast ~> Sink.head[Apple]" shouldNot compile "fruitBcast ~> Sink.head[Apple]" shouldNot compile
val appleBcast = b.add(Broadcast[Apple](2)) val appleBcast = b.add(Broadcast[Apple](2))
"Source[Fruit](apples) ~> appleBcast" shouldNot compile "Source[Fruit](apples) ~> appleBcast" shouldNot compile
Source[Apple](apples) ~> appleBcast Source.fromIterator[Apple](apples) ~> appleBcast
appleBcast ~> Sink.head[Fruit] appleBcast ~> Sink.head[Fruit]
appleBcast ~> Sink.head[Apple] appleBcast ~> Sink.head[Apple]
ClosedShape ClosedShape
@ -274,33 +274,33 @@ class FlowGraphCompileSpec extends AkkaSpec {
"build with implicits and variance" in { "build with implicits and variance" in {
RunnableGraph.fromGraph(GraphDSL.create() { implicit b RunnableGraph.fromGraph(GraphDSL.create() { implicit b
def appleSource = b.add(Source(TestPublisher.manualProbe[Apple]())) def appleSource = b.add(Source.fromPublisher(TestPublisher.manualProbe[Apple]()))
def fruitSource = b.add(Source(TestPublisher.manualProbe[Fruit]())) def fruitSource = b.add(Source.fromPublisher(TestPublisher.manualProbe[Fruit]()))
val outA = b add Sink(TestSubscriber.manualProbe[Fruit]()) val outA = b add Sink.fromSubscriber(TestSubscriber.manualProbe[Fruit]())
val outB = b add Sink(TestSubscriber.manualProbe[Fruit]()) val outB = b add Sink.fromSubscriber(TestSubscriber.manualProbe[Fruit]())
val merge = b add Merge[Fruit](11) val merge = b add Merge[Fruit](11)
val unzip = b add Unzip[Int, String]() 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._ 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(1)
appleSource ~> merge.in(2) appleSource ~> merge.in(2)
fruitSource ~> merge.in(3) fruitSource ~> merge.in(3)
fruitSource ~> Flow[Fruit].map(identity) ~> merge.in(4) fruitSource ~> Flow[Fruit].map(identity) ~> merge.in(4)
appleSource ~> Flow[Apple].map(identity) ~> merge.in(5) appleSource ~> Flow[Apple].map(identity) ~> merge.in(5)
b.add(Source(apples)) ~> merge.in(6) b.add(Source.fromIterator(apples)) ~> merge.in(6)
b.add(Source(apples)) ~> Flow[Fruit].map(identity) ~> merge.in(7) b.add(Source.fromIterator(apples)) ~> Flow[Fruit].map(identity) ~> merge.in(7)
b.add(Source(apples)) ~> Flow[Apple].map(identity) ~> merge.in(8) b.add(Source.fromIterator(apples)) ~> Flow[Apple].map(identity) ~> merge.in(8)
merge.out ~> Flow[Fruit].map(identity) ~> outA merge.out ~> Flow[Fruit].map(identity) ~> outA
b.add(Source(apples)) ~> Flow[Apple] ~> merge.in(9) b.add(Source.fromIterator(apples)) ~> Flow[Apple] ~> merge.in(9)
b.add(Source(apples)) ~> Flow[Apple] ~> outB b.add(Source.fromIterator(apples)) ~> Flow[Apple] ~> outB
b.add(Source(apples)) ~> Flow[Apple] ~> b.add(Sink.publisher[Fruit](false)) b.add(Source.fromIterator(apples)) ~> Flow[Apple] ~> b.add(Sink.asPublisher[Fruit](false))
appleSource ~> Flow[Apple] ~> merge.in(10) appleSource ~> Flow[Apple] ~> merge.in(10)
Source(List(1 -> "a", 2 -> "b", 3 -> "c")) ~> unzip.in Source(List(1 -> "a", 2 -> "b", 3 -> "c")) ~> unzip.in
unzip.out1 ~> whatever 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 ~> b.add(Broadcast[Apple](2))" shouldNot compile
"merge.out ~> Flow[Fruit].map(identity) ~> b.add(Broadcast[Apple](2))" shouldNot compile "merge.out ~> Flow[Fruit].map(identity) ~> b.add(Broadcast[Apple](2))" shouldNot compile

View file

@ -50,9 +50,9 @@ class FlowGroupBySpec extends AkkaSpec with ScalaFutures with ConversionCheckedT
} }
class SubstreamsSupport(groupCount: Int = 2, elementCount: Int = 6, maxSubstreams: Int = -1) { 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 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, _])]() val masterSubscriber = TestSubscriber.manualProbe[(Int, Source[Int, _])]()
groupStream.subscribe(masterSubscriber) groupStream.subscribe(masterSubscriber)
@ -74,7 +74,7 @@ class FlowGroupBySpec extends AkkaSpec with ScalaFutures with ConversionCheckedT
"groupBy" must { "groupBy" must {
"work in the happy case" in assertAllStagesStopped { "work in the happy case" in assertAllStagesStopped {
new SubstreamsSupport(groupCount = 2) { 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) masterSubscriber.expectNoMsg(100.millis)
s1.expectNoMsg(100.millis) s1.expectNoMsg(100.millis)
@ -82,7 +82,7 @@ class FlowGroupBySpec extends AkkaSpec with ScalaFutures with ConversionCheckedT
s1.expectNext(1) s1.expectNext(1)
s1.expectNoMsg(100.millis) 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.expectNoMsg(100.millis)
s2.request(2) s2.request(2)
@ -123,9 +123,9 @@ class FlowGroupBySpec extends AkkaSpec with ScalaFutures with ConversionCheckedT
"accept cancellation of substreams" in assertAllStagesStopped { "accept cancellation of substreams" in assertAllStagesStopped {
new SubstreamsSupport(groupCount = 2) { 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.request(2)
substream.expectNext(2) substream.expectNext(2)
substream.expectNext(4) 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 { "accept cancellation of master stream when not consumed anything" in assertAllStagesStopped {
val publisherProbeProbe = TestPublisher.manualProbe[Int]() 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, _])]() val subscriber = TestSubscriber.manualProbe[(Int, Source[Int, _])]()
publisher.subscribe(subscriber) publisher.subscribe(subscriber)
@ -153,7 +153,7 @@ class FlowGroupBySpec extends AkkaSpec with ScalaFutures with ConversionCheckedT
} }
"work with empty input stream" in assertAllStagesStopped { "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, _])]() val subscriber = TestSubscriber.manualProbe[(Int, Source[Int, _])]()
publisher.subscribe(subscriber) publisher.subscribe(subscriber)
@ -162,7 +162,7 @@ class FlowGroupBySpec extends AkkaSpec with ScalaFutures with ConversionCheckedT
"abort on onError from upstream" in assertAllStagesStopped { "abort on onError from upstream" in assertAllStagesStopped {
val publisherProbeProbe = TestPublisher.manualProbe[Int]() 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, _])]() val subscriber = TestSubscriber.manualProbe[(Int, Source[Int, _])]()
publisher.subscribe(subscriber) 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 { "abort on onError from upstream when substreams are running" in assertAllStagesStopped {
val publisherProbeProbe = TestPublisher.manualProbe[Int]() 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, _])]() val subscriber = TestSubscriber.manualProbe[(Int, Source[Int, _])]()
publisher.subscribe(subscriber) publisher.subscribe(subscriber)
@ -191,7 +191,7 @@ class FlowGroupBySpec extends AkkaSpec with ScalaFutures with ConversionCheckedT
upstreamSubscription.sendNext(1) upstreamSubscription.sendNext(1)
val (_, substream) = subscriber.expectNext() val (_, substream) = subscriber.expectNext()
val substreamPuppet = StreamPuppet(substream.runWith(Sink.publisher(false))) val substreamPuppet = StreamPuppet(substream.runWith(Sink.asPublisher(false)))
substreamPuppet.request(1) substreamPuppet.request(1)
substreamPuppet.expectNext(1) substreamPuppet.expectNext(1)
@ -207,10 +207,10 @@ class FlowGroupBySpec extends AkkaSpec with ScalaFutures with ConversionCheckedT
"fail stream when groupBy function throws" in assertAllStagesStopped { "fail stream when groupBy function throws" in assertAllStagesStopped {
val publisherProbeProbe = TestPublisher.manualProbe[Int]() val publisherProbeProbe = TestPublisher.manualProbe[Int]()
val exc = TE("test") val exc = TE("test")
val publisher = Source(publisherProbeProbe) val publisher = Source.fromPublisher(publisherProbeProbe)
.groupBy(2, elem if (elem == 2) throw exc else elem % 2) .groupBy(2, elem if (elem == 2) throw exc else elem % 2)
.lift(_ % 2) .lift(_ % 2)
.runWith(Sink.publisher(false)) .runWith(Sink.asPublisher(false))
val subscriber = TestSubscriber.manualProbe[(Int, Source[Int, Unit])]() val subscriber = TestSubscriber.manualProbe[(Int, Source[Int, Unit])]()
publisher.subscribe(subscriber) publisher.subscribe(subscriber)
@ -222,7 +222,7 @@ class FlowGroupBySpec extends AkkaSpec with ScalaFutures with ConversionCheckedT
upstreamSubscription.sendNext(1) upstreamSubscription.sendNext(1)
val (_, substream) = subscriber.expectNext() val (_, substream) = subscriber.expectNext()
val substreamPuppet = StreamPuppet(substream.runWith(Sink.publisher(false))) val substreamPuppet = StreamPuppet(substream.runWith(Sink.asPublisher(false)))
substreamPuppet.request(1) substreamPuppet.request(1)
substreamPuppet.expectNext(1) substreamPuppet.expectNext(1)
@ -237,11 +237,11 @@ class FlowGroupBySpec extends AkkaSpec with ScalaFutures with ConversionCheckedT
"resume stream when groupBy function throws" in { "resume stream when groupBy function throws" in {
val publisherProbeProbe = TestPublisher.manualProbe[Int]() val publisherProbeProbe = TestPublisher.manualProbe[Int]()
val exc = TE("test") val exc = TE("test")
val publisher = Source(publisherProbeProbe) val publisher = Source.fromPublisher(publisherProbeProbe)
.groupBy(2, elem if (elem == 2) throw exc else elem % 2) .groupBy(2, elem if (elem == 2) throw exc else elem % 2)
.lift(_ % 2) .lift(_ % 2)
.withAttributes(ActorAttributes.supervisionStrategy(resumingDecider)) .withAttributes(ActorAttributes.supervisionStrategy(resumingDecider))
.runWith(Sink.publisher(false)) .runWith(Sink.asPublisher(false))
val subscriber = TestSubscriber.manualProbe[(Int, Source[Int, Unit])]() val subscriber = TestSubscriber.manualProbe[(Int, Source[Int, Unit])]()
publisher.subscribe(subscriber) publisher.subscribe(subscriber)
@ -253,7 +253,7 @@ class FlowGroupBySpec extends AkkaSpec with ScalaFutures with ConversionCheckedT
upstreamSubscription.sendNext(1) upstreamSubscription.sendNext(1)
val (_, substream1) = subscriber.expectNext() val (_, substream1) = subscriber.expectNext()
val substreamPuppet1 = StreamPuppet(substream1.runWith(Sink.publisher(false))) val substreamPuppet1 = StreamPuppet(substream1.runWith(Sink.asPublisher(false)))
substreamPuppet1.request(10) substreamPuppet1.request(10)
substreamPuppet1.expectNext(1) substreamPuppet1.expectNext(1)
@ -261,7 +261,7 @@ class FlowGroupBySpec extends AkkaSpec with ScalaFutures with ConversionCheckedT
upstreamSubscription.sendNext(4) upstreamSubscription.sendNext(4)
val (_, substream2) = subscriber.expectNext() val (_, substream2) = subscriber.expectNext()
val substreamPuppet2 = StreamPuppet(substream2.runWith(Sink.publisher(false))) val substreamPuppet2 = StreamPuppet(substream2.runWith(Sink.asPublisher(false)))
substreamPuppet2.request(10) substreamPuppet2.request(10)
substreamPuppet2.expectNext(4) // note that 2 was dropped 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 up = TestPublisher.manualProbe[Int]()
val down = TestSubscriber.manualProbe[(Int, Source[Int, Unit])]() 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() val downstream = down.expectSubscription()
downstream.cancel() downstream.cancel()
@ -299,7 +299,7 @@ class FlowGroupBySpec extends AkkaSpec with ScalaFutures with ConversionCheckedT
up.sendNext(1) up.sendNext(1)
val first = down.expectNext() 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.request(1)
s1.expectNext(1) s1.expectNext(1)

View file

@ -24,7 +24,7 @@ class FlowGroupedWithinSpec extends AkkaSpec with ScriptedTest {
val input = Iterator.from(1) val input = Iterator.from(1)
val p = TestPublisher.manualProbe[Int]() val p = TestPublisher.manualProbe[Int]()
val c = TestSubscriber.manualProbe[immutable.Seq[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 pSub = p.expectSubscription
val cSub = c.expectSubscription val cSub = c.expectSubscription
cSub.request(100) cSub.request(100)
@ -49,7 +49,7 @@ class FlowGroupedWithinSpec extends AkkaSpec with ScriptedTest {
"deliver bufferd elements onComplete before the timeout" in { "deliver bufferd elements onComplete before the timeout" in {
val c = TestSubscriber.manualProbe[immutable.Seq[Int]]() 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 val cSub = c.expectSubscription
cSub.request(100) cSub.request(100)
c.expectNext((1 to 3).toList) c.expectNext((1 to 3).toList)
@ -61,7 +61,7 @@ class FlowGroupedWithinSpec extends AkkaSpec with ScriptedTest {
val input = Iterator.from(1) val input = Iterator.from(1)
val p = TestPublisher.manualProbe[Int]() val p = TestPublisher.manualProbe[Int]()
val c = TestSubscriber.manualProbe[immutable.Seq[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 pSub = p.expectSubscription
val cSub = c.expectSubscription val cSub = c.expectSubscription
cSub.request(1) cSub.request(1)
@ -81,7 +81,7 @@ class FlowGroupedWithinSpec extends AkkaSpec with ScriptedTest {
"drop empty groups" in { "drop empty groups" in {
val p = TestPublisher.manualProbe[Int]() val p = TestPublisher.manualProbe[Int]()
val c = TestSubscriber.manualProbe[immutable.Seq[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 pSub = p.expectSubscription
val cSub = c.expectSubscription val cSub = c.expectSubscription
cSub.request(2) cSub.request(2)
@ -103,7 +103,7 @@ class FlowGroupedWithinSpec extends AkkaSpec with ScriptedTest {
val inputs = Iterator.from(1) val inputs = Iterator.from(1)
val upstream = TestPublisher.probe[Int]() val upstream = TestPublisher.probe[Int]()
val downstream = TestSubscriber.probe[immutable.Seq[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.request(2)
downstream.expectNoMsg(1000.millis) downstream.expectNoMsg(1000.millis)

View file

@ -33,7 +33,7 @@ class FlowIdleInjectSpec extends AkkaSpec {
val upstream = TestPublisher.probe[Int]() val upstream = TestPublisher.probe[Int]()
val downstream = TestSubscriber.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) downstream.request(1)
@ -48,7 +48,7 @@ class FlowIdleInjectSpec extends AkkaSpec {
val upstream = TestPublisher.probe[Int]() val upstream = TestPublisher.probe[Int]()
val downstream = TestSubscriber.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.request(10)
downstream.expectNextN(1 to 10) downstream.expectNextN(1 to 10)
@ -66,7 +66,7 @@ class FlowIdleInjectSpec extends AkkaSpec {
val upstream = TestPublisher.probe[Int]() val upstream = TestPublisher.probe[Int]()
val downstream = TestSubscriber.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.ensureSubscription()
downstream.expectNoMsg(1.5.second) downstream.expectNoMsg(1.5.second)
@ -81,7 +81,7 @@ class FlowIdleInjectSpec extends AkkaSpec {
val upstream = TestPublisher.probe[Int]() val upstream = TestPublisher.probe[Int]()
val downstream = TestSubscriber.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.request(10)
downstream.expectNextN(1 to 10) downstream.expectNextN(1 to 10)
@ -98,7 +98,7 @@ class FlowIdleInjectSpec extends AkkaSpec {
val upstream = TestPublisher.probe[Int]() val upstream = TestPublisher.probe[Int]()
val downstream = TestSubscriber.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.ensureSubscription()
downstream.expectNoMsg(1.5.second) downstream.expectNoMsg(1.5.second)
@ -115,7 +115,7 @@ class FlowIdleInjectSpec extends AkkaSpec {
val upstream = TestPublisher.probe[Int]() val upstream = TestPublisher.probe[Int]()
val downstream = TestSubscriber.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.request(10)
downstream.expectNextN(1 to 10) downstream.expectNextN(1 to 10)
@ -134,7 +134,7 @@ class FlowIdleInjectSpec extends AkkaSpec {
val upstream = TestPublisher.probe[Int]() val upstream = TestPublisher.probe[Int]()
val downstream = TestSubscriber.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.request(2)
downstream.expectNoMsg(500.millis) downstream.expectNoMsg(500.millis)

View file

@ -37,7 +37,7 @@ class FlowInitialDelaySpec extends AkkaSpec {
"properly ignore timer while backpressured" in Utils.assertAllStagesStopped { "properly ignore timer while backpressured" in Utils.assertAllStagesStopped {
val probe = TestSubscriber.probe[Int]() 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.ensureSubscription()
probe.expectNoMsg(1.5.second) probe.expectNoMsg(1.5.second)

View file

@ -13,7 +13,7 @@ class FlowInterleaveSpec extends BaseTwoStreamsSetup {
override def setup(p1: Publisher[Int], p2: Publisher[Int]) = { override def setup(p1: Publisher[Int], p2: Publisher[Int]) = {
val subscriber = TestSubscriber.probe[Outputs]() 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 subscriber
} }
@ -21,7 +21,7 @@ class FlowInterleaveSpec extends BaseTwoStreamsSetup {
"work in the happy case" in assertAllStagesStopped { "work in the happy case" in assertAllStagesStopped {
val probe = TestSubscriber.manualProbe[Int]() 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() val subscription = probe.expectSubscription()
@ -38,7 +38,7 @@ class FlowInterleaveSpec extends BaseTwoStreamsSetup {
"work when segmentSize is not equal elements in stream" in assertAllStagesStopped { "work when segmentSize is not equal elements in stream" in assertAllStagesStopped {
val probe = TestSubscriber.manualProbe[Int]() 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.expectSubscription().request(10)
probe.expectNext(0, 1, 3, 4, 2, 5) probe.expectNext(0, 1, 3, 4, 2, 5)
probe.expectComplete() probe.expectComplete()
@ -47,7 +47,7 @@ class FlowInterleaveSpec extends BaseTwoStreamsSetup {
"work with segmentSize = 1" in assertAllStagesStopped { "work with segmentSize = 1" in assertAllStagesStopped {
val probe = TestSubscriber.manualProbe[Int]() 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.expectSubscription().request(10)
probe.expectNext(0, 3, 1, 4, 2, 5) probe.expectNext(0, 3, 1, 4, 2, 5)
probe.expectComplete() probe.expectComplete()
@ -59,13 +59,13 @@ class FlowInterleaveSpec extends BaseTwoStreamsSetup {
"not work when segmentSize > than stream elements" in assertAllStagesStopped { "not work when segmentSize > than stream elements" in assertAllStagesStopped {
val probe = TestSubscriber.manualProbe[Int]() 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) probe.expectSubscription().request(25)
(0 to 15).foreach(probe.expectNext) (0 to 15).foreach(probe.expectNext)
probe.expectComplete() probe.expectComplete()
val probe2 = TestSubscriber.manualProbe[Int]() 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) probe2.expectSubscription().request(100)
(1 to 10).foreach(probe2.expectNext) (1 to 10).foreach(probe2.expectNext)
(21 to 25).foreach(probe2.expectNext) (21 to 25).foreach(probe2.expectNext)
@ -126,8 +126,8 @@ class FlowInterleaveSpec extends BaseTwoStreamsSetup {
val up2 = TestPublisher.manualProbe[Int]() val up2 = TestPublisher.manualProbe[Int]()
val down = TestSubscriber.manualProbe[Int]() val down = TestSubscriber.manualProbe[Int]()
val (graphSubscriber1, graphSubscriber2) = Source.subscriber[Int] val (graphSubscriber1, graphSubscriber2) = Source.asSubscriber[Int]
.interleaveMat(Source.subscriber[Int], 2)((_, _)).toMat(Sink(down))(Keep.left).run .interleaveMat(Source.asSubscriber[Int], 2)((_, _)).toMat(Sink.fromSubscriber(down))(Keep.left).run
val downstream = down.expectSubscription() val downstream = down.expectSubscription()
downstream.cancel() downstream.cancel()

View file

@ -16,7 +16,7 @@ import org.reactivestreams.Subscriber
class FlowIteratorSpec extends AbstractFlowIteratorSpec { class FlowIteratorSpec extends AbstractFlowIteratorSpec {
override def testName = "A Flow based on an iterator producing function" override def testName = "A Flow based on an iterator producing function"
override def createSource(elements: Int): Source[Int, Unit] = override def createSource(elements: Int): Source[Int, Unit] =
Source(() (1 to elements).iterator) Source.fromIterator(() (1 to elements).iterator)
} }
class FlowIterableSpec extends AbstractFlowIteratorSpec { class FlowIterableSpec extends AbstractFlowIteratorSpec {
@ -31,7 +31,7 @@ class FlowIterableSpec extends AbstractFlowIteratorSpec {
override def iterator: Iterator[Int] = override def iterator: Iterator[Int] =
(1 to 3).iterator.map(x if (x == 2) throw new IllegalStateException("not two") else x) (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]() val c = TestSubscriber.manualProbe[Int]()
p.subscribe(c) p.subscribe(c)
val sub = c.expectSubscription() val sub = c.expectSubscription()
@ -48,7 +48,7 @@ class FlowIterableSpec extends AbstractFlowIteratorSpec {
val iterable = new immutable.Iterable[Int] { val iterable = new immutable.Iterable[Int] {
override def iterator: Iterator[Int] = throw new IllegalStateException("no good iterator") 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]() val c = TestSubscriber.manualProbe[Int]()
p.subscribe(c) p.subscribe(c)
c.expectSubscriptionAndError().getMessage should be("no good iterator") c.expectSubscriptionAndError().getMessage should be("no good iterator")
@ -62,7 +62,7 @@ class FlowIterableSpec extends AbstractFlowIteratorSpec {
override def next(): Int = -1 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]() val c = TestSubscriber.manualProbe[Int]()
p.subscribe(c) p.subscribe(c)
c.expectSubscriptionAndError().getMessage should be("no next") c.expectSubscriptionAndError().getMessage should be("no next")
@ -84,7 +84,7 @@ abstract class AbstractFlowIteratorSpec extends AkkaSpec {
testName must { testName must {
"produce elements" in assertAllStagesStopped { "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]() val c = TestSubscriber.manualProbe[Int]()
p.subscribe(c) p.subscribe(c)
val sub = c.expectSubscription() val sub = c.expectSubscription()
@ -98,7 +98,7 @@ abstract class AbstractFlowIteratorSpec extends AkkaSpec {
} }
"complete empty" in assertAllStagesStopped { "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]() val c = TestSubscriber.manualProbe[Int]()
p.subscribe(c) p.subscribe(c)
c.expectSubscriptionAndComplete() c.expectSubscriptionAndComplete()
@ -106,7 +106,7 @@ abstract class AbstractFlowIteratorSpec extends AkkaSpec {
} }
"produce elements with multiple subscribers" in assertAllStagesStopped { "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 c1 = TestSubscriber.manualProbe[Int]()
val c2 = TestSubscriber.manualProbe[Int]() val c2 = TestSubscriber.manualProbe[Int]()
p.subscribe(c1) p.subscribe(c1)
@ -130,7 +130,7 @@ abstract class AbstractFlowIteratorSpec extends AkkaSpec {
} }
"produce elements to later subscriber" in assertAllStagesStopped { "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 c1 = TestSubscriber.manualProbe[Int]()
val c2 = TestSubscriber.manualProbe[Int]() val c2 = TestSubscriber.manualProbe[Int]()
p.subscribe(c1) p.subscribe(c1)
@ -153,7 +153,7 @@ abstract class AbstractFlowIteratorSpec extends AkkaSpec {
} }
"produce elements with one transformation step" in assertAllStagesStopped { "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]() val c = TestSubscriber.manualProbe[Int]()
p.subscribe(c) p.subscribe(c)
val sub = c.expectSubscription() val sub = c.expectSubscription()
@ -165,7 +165,7 @@ abstract class AbstractFlowIteratorSpec extends AkkaSpec {
} }
"produce elements with two transformation steps" in assertAllStagesStopped { "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]() val c = TestSubscriber.manualProbe[Int]()
p.subscribe(c) p.subscribe(c)
val sub = c.expectSubscription() val sub = c.expectSubscription()
@ -176,7 +176,7 @@ abstract class AbstractFlowIteratorSpec extends AkkaSpec {
} }
"not produce after cancel" in assertAllStagesStopped { "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]() val c = TestSubscriber.manualProbe[Int]()
p.subscribe(c) p.subscribe(c)
val sub = c.expectSubscription() val sub = c.expectSubscription()

View file

@ -32,7 +32,7 @@ class FlowJoinSpec extends AkkaSpec(ConfigFactory.parseString("akka.loglevel=INF
val broadcast = b.add(Broadcast[Int](2)) val broadcast = b.add(Broadcast[Int](2))
source ~> merge.in(0) source ~> merge.in(0)
merge.out ~> broadcast.in 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)) FlowShape(merge.in(1), broadcast.out(1))
}) })

View file

@ -36,7 +36,7 @@ class FlowMapAsyncSpec extends AkkaSpec with ScalaFutures {
"produce future elements" in assertAllStagesStopped { "produce future elements" in assertAllStagesStopped {
val c = TestSubscriber.manualProbe[Int]() val c = TestSubscriber.manualProbe[Int]()
implicit val ec = system.dispatcher 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() val sub = c.expectSubscription()
sub.request(2) sub.request(2)
c.expectNext(1) c.expectNext(1)
@ -53,7 +53,7 @@ class FlowMapAsyncSpec extends AkkaSpec with ScalaFutures {
val p = Source(1 to 50).mapAsync(4)(n Future { val p = Source(1 to 50).mapAsync(4)(n Future {
Thread.sleep(ThreadLocalRandom.current().nextInt(1, 10)) Thread.sleep(ThreadLocalRandom.current().nextInt(1, 10))
n n
}).to(Sink(c)).run() }).to(Sink.fromSubscriber(c)).run()
val sub = c.expectSubscription() val sub = c.expectSubscription()
sub.request(1000) sub.request(1000)
for (n 1 to 50) c.expectNext(n) 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 { val p = Source(1 to 20).mapAsync(8)(n Future {
probe.ref ! n probe.ref ! n
n n
}).to(Sink(c)).run() }).to(Sink.fromSubscriber(c)).run()
val sub = c.expectSubscription() val sub = c.expectSubscription()
probe.expectNoMsg(500.millis) probe.expectNoMsg(500.millis)
sub.request(1) sub.request(1)
@ -94,7 +94,7 @@ class FlowMapAsyncSpec extends AkkaSpec with ScalaFutures {
Await.ready(latch, 10.seconds) Await.ready(latch, 10.seconds)
n n
} }
}).to(Sink(c)).run() }).to(Sink.fromSubscriber(c)).run()
val sub = c.expectSubscription() val sub = c.expectSubscription()
sub.request(10) sub.request(10)
c.expectError().getMessage should be("err1") c.expectError().getMessage should be("err1")
@ -113,7 +113,7 @@ class FlowMapAsyncSpec extends AkkaSpec with ScalaFutures {
n n
} }
}). }).
to(Sink(c)).run() to(Sink.fromSubscriber(c)).run()
val sub = c.expectSubscription() val sub = c.expectSubscription()
sub.request(10) sub.request(10)
c.expectError().getMessage should be("err2") c.expectError().getMessage should be("err2")
@ -129,7 +129,7 @@ class FlowMapAsyncSpec extends AkkaSpec with ScalaFutures {
else n else n
}) })
.withAttributes(supervisionStrategy(resumingDecider)) .withAttributes(supervisionStrategy(resumingDecider))
.to(Sink(c)).run() .to(Sink.fromSubscriber(c)).run()
val sub = c.expectSubscription() val sub = c.expectSubscription()
sub.request(10) sub.request(10)
for (n List(1, 2, 4, 5)) c.expectNext(n) 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 if (n == 3) throw new RuntimeException("err4") with NoStackTrace
else Future(n)) else Future(n))
.withAttributes(supervisionStrategy(resumingDecider)) .withAttributes(supervisionStrategy(resumingDecider))
.to(Sink(c)).run() .to(Sink.fromSubscriber(c)).run()
val sub = c.expectSubscription() val sub = c.expectSubscription()
sub.request(10) sub.request(10)
for (n List(1, 2, 4, 5)) c.expectNext(n) 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 { "signal NPE when future is completed with null" in {
val c = TestSubscriber.manualProbe[String]() 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() val sub = c.expectSubscription()
sub.request(10) sub.request(10)
c.expectError().getMessage should be(ReactiveStreamsCompliance.ElementMustNotBeNullMsg) c.expectError().getMessage should be(ReactiveStreamsCompliance.ElementMustNotBeNullMsg)
@ -189,7 +189,7 @@ class FlowMapAsyncSpec extends AkkaSpec with ScalaFutures {
val p = Source(List("a", "b", "c")) val p = Source(List("a", "b", "c"))
.mapAsync(4)(elem if (elem == "b") Future.successful(null) else Future.successful(elem)) .mapAsync(4)(elem if (elem == "b") Future.successful(null) else Future.successful(elem))
.withAttributes(supervisionStrategy(resumingDecider)) .withAttributes(supervisionStrategy(resumingDecider))
.to(Sink(c)).run() .to(Sink.fromSubscriber(c)).run()
val sub = c.expectSubscription() val sub = c.expectSubscription()
sub.request(10) sub.request(10)
for (elem List("a", "c")) c.expectNext(elem) for (elem List("a", "c")) c.expectNext(elem)
@ -200,7 +200,7 @@ class FlowMapAsyncSpec extends AkkaSpec with ScalaFutures {
val pub = TestPublisher.manualProbe[Int]() val pub = TestPublisher.manualProbe[Int]()
val sub = TestSubscriber.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() val upstream = pub.expectSubscription()
upstream.expectRequest() upstream.expectRequest()

View file

@ -37,7 +37,7 @@ class FlowMapAsyncUnorderedSpec extends AkkaSpec with ScalaFutures with Conversi
val p = Source(1 to 4).mapAsyncUnordered(4)(n Future { val p = Source(1 to 4).mapAsyncUnordered(4)(n Future {
Await.ready(latch(n), 5.seconds) Await.ready(latch(n), 5.seconds)
n n
}).to(Sink(c)).run() }).to(Sink.fromSubscriber(c)).run()
val sub = c.expectSubscription() val sub = c.expectSubscription()
sub.request(5) sub.request(5)
latch(2).countDown() 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 { val p = Source(1 to 20).mapAsyncUnordered(4)(n Future {
probe.ref ! n probe.ref ! n
n n
}).to(Sink(c)).run() }).to(Sink.fromSubscriber(c)).run()
val sub = c.expectSubscription() val sub = c.expectSubscription()
c.expectNoMsg(200.millis) c.expectNoMsg(200.millis)
probe.expectNoMsg(Duration.Zero) probe.expectNoMsg(Duration.Zero)
@ -86,7 +86,7 @@ class FlowMapAsyncUnorderedSpec extends AkkaSpec with ScalaFutures with Conversi
Await.ready(latch, 10.seconds) Await.ready(latch, 10.seconds)
n n
} }
}).to(Sink(c)).run() }).to(Sink.fromSubscriber(c)).run()
val sub = c.expectSubscription() val sub = c.expectSubscription()
sub.request(10) sub.request(10)
c.expectError.getMessage should be("err1") c.expectError.getMessage should be("err1")
@ -105,7 +105,7 @@ class FlowMapAsyncUnorderedSpec extends AkkaSpec with ScalaFutures with Conversi
n n
} }
}). }).
to(Sink(c)).run() to(Sink.fromSubscriber(c)).run()
val sub = c.expectSubscription() val sub = c.expectSubscription()
sub.request(10) sub.request(10)
c.expectError.getMessage should be("err2") 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 { "signal NPE when future is completed with null" in {
val c = TestSubscriber.manualProbe[String]() 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() val sub = c.expectSubscription()
sub.request(10) sub.request(10)
c.expectError.getMessage should be(ReactiveStreamsCompliance.ElementMustNotBeNullMsg) 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")) val p = Source(List("a", "b", "c"))
.mapAsyncUnordered(4)(elem if (elem == "b") Future.successful(null) else Future.successful(elem)) .mapAsyncUnordered(4)(elem if (elem == "b") Future.successful(null) else Future.successful(elem))
.withAttributes(supervisionStrategy(resumingDecider)) .withAttributes(supervisionStrategy(resumingDecider))
.to(Sink(c)).run() .to(Sink.fromSubscriber(c)).run()
val sub = c.expectSubscription() val sub = c.expectSubscription()
sub.request(10) sub.request(10)
c.expectNextUnordered("a", "c") c.expectNextUnordered("a", "c")
@ -188,7 +188,7 @@ class FlowMapAsyncUnorderedSpec extends AkkaSpec with ScalaFutures with Conversi
val pub = TestPublisher.manualProbe[Int]() val pub = TestPublisher.manualProbe[Int]()
val sub = TestSubscriber.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() val upstream = pub.expectSubscription()
upstream.expectRequest() upstream.expectRequest()

Some files were not shown because too many files have changed in this diff Show more