=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)
GraphStages.identity[Int]
else

View file

@ -34,7 +34,7 @@ class FileSourcesBenchmark {
val f = File.createTempFile(getClass.getName, ".bench.tmp")
f.deleteOnExit()
val ft = Source(() Iterator.continually(line))
val ft = Source.fromIterator(() Iterator.continually(line))
.take(10 * 39062) // adjust as needed
.runWith(Sink.file(f))
Await.result(ft, 30.seconds)
@ -53,7 +53,7 @@ class FileSourcesBenchmark {
def setup() {
fileChannelSource = Source.file(file, bufSize)
fileInputStreamSource = Source.inputStream(() new FileInputStream(file), bufSize)
ioSourceLinesIterator = Source(() scala.io.Source.fromFile(file).getLines()).map(ByteString(_))
ioSourceLinesIterator = Source.fromIterator(() scala.io.Source.fromFile(file).getLines()).map(ByteString(_))
}
@TearDown

View file

@ -1,12 +1,16 @@
package docs;
import akka.actor.ActorSystem;
import akka.actor.Cancellable;
import akka.http.javadsl.model.Uri;
import akka.dispatch.Futures;
import akka.japi.function.Creator;
import akka.japi.Pair;
import akka.japi.function.Function;
import akka.stream.*;
import akka.stream.javadsl.*;
import akka.stream.testkit.TestPublisher;
import akka.stream.testkit.TestSubscriber;
import akka.util.ByteString;
import scala.Option;
import scala.concurrent.Future;
@ -14,6 +18,9 @@ import scala.concurrent.duration.FiniteDuration;
import scala.concurrent.Promise;
import scala.runtime.BoxedUnit;
import org.reactivestreams.Publisher;
import org.reactivestreams.Subscriber;
import java.io.File;
import java.io.IOException;
import java.io.InputStream;
@ -25,6 +32,7 @@ public class MigrationsJava {
// This is compile-only code, no need for actually running anything.
public static ActorMaterializer mat = null;
public static ActorSystem sys = null;
public static class SomeInputStream extends InputStream {
public SomeInputStream() {}
@ -133,12 +141,34 @@ public class MigrationsJava {
// Complete the promise with an empty option to emulate the old lazyEmpty
promise.trySuccess(scala.Option.empty());
final Source<String, Cancellable> sourceUnderTest = Source.tick(
final Source<String, Cancellable> ticks = Source.tick(
FiniteDuration.create(0, TimeUnit.MILLISECONDS),
FiniteDuration.create(200, TimeUnit.MILLISECONDS),
"tick");
final Source<Integer, BoxedUnit> pubSource =
Source.fromPublisher(TestPublisher.<Integer>manualProbe(true, sys));
final Source<Integer, BoxedUnit> futSource =
Source.fromFuture(Futures.successful(42));
final Source<Integer, Subscriber<Integer>> subSource =
Source.<Integer>asSubscriber();
//#source-creators
//#sink-creators
final Sink<Integer, BoxedUnit> subSink =
Sink.fromSubscriber(TestSubscriber.<Integer>manualProbe(sys));
//#sink-creators
//#sink-as-publisher
final Sink<Integer, Publisher<Integer>> pubSink =
Sink.<Integer>asPublisher(false);
final Sink<Integer, Publisher<Integer>> pubSinkFanout =
Sink.<Integer>asPublisher(true);
//#sink-as-publisher
//#empty-flow
Flow<Integer, Integer, BoxedUnit> emptyFlow = Flow.<Integer>create();
// or

View file

@ -224,13 +224,14 @@ should be replaced by
Source constructor name changes
===============================
``Source.lazyEmpty`` have been replaced by ``Source.maybe`` which returns a ``Promise`` that can be completed by one or
``Source.lazyEmpty`` has been replaced by ``Source.maybe`` which returns a ``Promise`` that can be completed by one or
zero elements by providing an ``Option``. This is different from ``lazyEmpty`` which only allowed completion to be
sent, but no elements.
The ``from()`` overload on ``Source`` that provide a tick source (``Source.from(delay,interval,tick)``)
is replaced by the named method ``Source.tick()`` to reduce the number of overloads and to make the function more
discoverable.
The ``from()`` overload on ``Source`` has been refactored to separate methods to reduce the number of overloads and
make source creation more discoverable.
``Source.subscriber`` has been renamed to ``Source.asSubscriber``.
Update procedure
----------------
@ -238,6 +239,9 @@ Update procedure
1. All uses of ``Source.lazyEmpty`` should be replaced by ``Source.maybe`` and the returned ``Promise`` completed with
a ``None`` (an empty ``Option``)
2. Replace all uses of ``Source.from(delay,interval,tick)`` with the method ``Source.tick(delay,interval,tick)``
3. Replace all uses of ``Source.from(publisher)`` with the method ``Source.fromPublisher(publisher)``
4. Replace all uses of ``Source.from(future)`` with the method ``Source.fromFuture(future))``
5. Replace all uses of ``Source.subscriber`` with the method ``Source.asSubscriber``
Example
^^^^^^^
@ -250,15 +254,51 @@ Example
promise.trySuccess(BoxedUnit.UNIT);
// This no longer works!
final Source<String, Cancellable> sourceUnderTest = Source.from(
final Source<String, Cancellable> ticks = Source.from(
FiniteDuration.create(0, TimeUnit.MILLISECONDS),
FiniteDuration.create(200, TimeUnit.MILLISECONDS),
"tick");
// This no longer works!
final Source<Integer, BoxedUnit> pubSource =
Source.from(TestPublisher.<Integer>manualProbe(true, sys));
// This no longer works!
final Source<Integer, BoxedUnit> futSource =
Source.from(Futures.successful(42));
// This no longer works!
final Source<Integer, Subscriber<Integer>> subSource =
Source.<Integer>subscriber();
should be replaced by
.. includecode:: code/docs/MigrationsJava.java#source-creators
Sink constructor name changes
=============================
``Sink.create(subscriber)`` has been renamed to ``Sink.fromSubscriber(subscriber)`` to reduce the number of overloads and
make sink creation more discoverable.
Update procedure
----------------
1. Replace all uses of ``Sink.create(subscriber)`` with the method ``Sink.fromSubscriber(subscriber)``
Example
^^^^^^^
::
// This no longer works!
final Sink<Integer, BoxedUnit> subSink =
Sink.create(TestSubscriber.<Integer>manualProbe(sys));
should be replaced by
.. includecode:: code/docs/MigrationsJava.java#sink-creators
``Flow.empty()`` have been removed
==================================
@ -308,16 +348,30 @@ should be replaced by
It was a common user mistake to use ``Sink.publisher`` and get into trouble since it would only support
a single ``Subscriber``, and the discoverability of the apprpriate fix was non-obvious (Sink.fanoutPublisher).
To make the decision whether to support fanout or not an active one, the aforementioned methods have been
replaced with a single method: ``Sink.publisher(fanout: Boolean)``.
replaced with a single method: ``Sink.asPublisher(fanout: Boolean)``.
Update procedure
----------------
1. Replace all occurences of ``Sink.publisher`` with ``Sink.publisher(false)``
2. Replace all occurences of ``Sink.fanoutPublisher`` with ``Sink.publisher(true)``
1. Replace all occurences of ``Sink.publisher`` with ``Sink.asPublisher(false)``
2. Replace all occurences of ``Sink.fanoutPublisher`` with ``Sink.asPublisher(true)``
TODO: code example
Example
^^^^^^^
::
// This no longer works!
final Sink<Integer, Publisher<Integer>> pubSink =
Sink.<Integer>publisher();
// This no longer works!
final Sink<Integer, Publisher<Integer>> pubSink =
Sink.<Integer>fanoutPublisher(2, 8);
should be replaced by
.. includecode:: code/docs/MigrationsJava.java#sink-as-publisher
FlexiMerge an FlexiRoute has been replaced by GraphStage
========================================================

View file

@ -108,7 +108,7 @@ This is how it can be used as input :class:`Source` to a :class:`Flow`:
.. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/ActorPublisherDocTest.java#actor-publisher-usage
You can only attach one subscriber to this publisher. Use a ``Broadcast``-element or
attach a ``Sink.publisher(true)`` to enable multiple subscribers.
attach a ``Sink.asPublisher(true)`` to enable multiple subscribers.
ActorSubscriber
^^^^^^^^^^^^^^^
@ -414,7 +414,7 @@ by using the Publisher-:class:`Sink`:
.. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/ReactiveStreamsDocTest.java#source-publisher
A publisher that is created with ``Sink.publisher(false)`` supports only a single subscription.
A publisher that is created with ``Sink.asPublisher(false)`` supports only a single subscription.
Additional subscription attempts will be rejected with an :class:`IllegalStateException`.
A publisher that supports multiple subscribers using fan-out/broadcasting is created as follows:

View file

@ -6,11 +6,11 @@ import akka.http.scaladsl.model.Uri
import akka.stream.scaladsl._
import akka.stream._
import akka.stream.stage.{ OutHandler, InHandler, GraphStageLogic, GraphStage }
import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.{ AkkaSpec, TestPublisher, TestSubscriber }
import scala.concurrent.{ Future, ExecutionContext, Promise }
import scala.concurrent.duration._
import scala.util.{ Failure, Success, Try }
import scala.util.{ Failure, Random, Success, Try }
class MigrationsScala extends AkkaSpec {
@ -110,8 +110,26 @@ class MigrationsScala extends AkkaSpec {
promise.trySuccess(Some(()))
val ticks = Source.tick(1.second, 3.seconds, "tick")
val pubSource = Source.fromPublisher(TestPublisher.manualProbe[Int]())
val itSource = Source.fromIterator(() => Iterator.continually(Random.nextGaussian))
val futSource = Source.fromFuture(Future.successful(42))
val subSource = Source.asSubscriber
//#source-creators
//#sink-creators
val subSink = Sink.fromSubscriber(TestSubscriber.manualProbe[Int]())
//#sink-creators
//#sink-as-publisher
val pubSink = Sink.asPublisher(fanout = false)
val pubSinkFanout = Sink.asPublisher(fanout = true)
//#sink-as-publisher
//#flatMapConcat
Flow[Source[Int, Any]].flatMapConcat(identity)
//#flatMapConcat

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

@ -220,13 +220,14 @@ should be replaced by
Source constructor name changes
===============================
``Source.lazyEmpty`` have been replaced by ``Source.maybe`` which returns a ``Promise`` that can be completed by one or
``Source.lazyEmpty`` has been replaced by ``Source.maybe`` which returns a ``Promise`` that can be completed by one or
zero elements by providing an ``Option``. This is different from ``lazyEmpty`` which only allowed completion to be
sent, but no elements.
The ``apply()`` overload on ``Source`` that provide a tick source (``Source(delay,interval,tick)``)
is replaced by the named method ``Source.tick()`` to reduce the number of overloads and to make the function more
discoverable.
The ``apply()`` overload on ``Source`` has been refactored to separate methods to reduce the number of overloads and
make source creation more discoverable.
``Source.subscriber`` has been renamed to ``Source.asSubscriber``.
Update procedure
----------------
@ -234,6 +235,10 @@ Update procedure
1. All uses of ``Source.lazyEmpty`` should be replaced by ``Source.maybe`` and the returned ``Promise`` completed with
a ``None`` (an empty ``Option``)
2. Replace all uses of ``Source(delay,interval,tick)`` with the method ``Source.tick(delay,interval,tick)``
3. Replace all uses of ``Source(publisher)`` with the method ``Source.fromPublisher(publisher)``
4. Replace all uses of ``Source(() => iterator)`` with the method ``Source.fromIterator(() => iterator))``
5. Replace all uses of ``Source(future)`` with the method ``Source.fromFuture(future))``
6. Replace all uses of ``Source.subscriber`` with the method ``Source.asSubscriber``
Example
^^^^^^^
@ -248,10 +253,45 @@ Example
// This no longer works!
val ticks = Source(1.second, 3.seconds, "tick")
// This no longer works!
val pubSource = Source(TestPublisher.manualProbe[Int]())
// This no longer works!
val itSource = Source(() => Iterator.continually(Random.nextGaussian))
// This no longer works!
val futSource = Source(Future.successful(42))
// This no longer works!
val subSource = Source.subscriber
should be replaced by
.. includecode:: code/docs/MigrationsScala.scala#source-creators
Sink constructor name changes
=============================
``Sink.apply(subscriber)`` has been renamed to ``Sink.fromSubscriber(subscriber)`` to reduce the number of overloads and
make sink creation more discoverable.
Update procedure
----------------
1. Replace all uses of ``Sink(subscriber)`` with the method ``Sink.fromSubscriber(subscriber)``
Example
^^^^^^^
::
// This no longer works!
val subSink = Sink(TestSubscriber.manualProbe[Int]())
should be replaced by
.. includecode:: code/docs/MigrationsScala.scala#sink-creators
``flatten(FlattenStrategy)`` has been replaced by named counterparts
====================================================================
@ -276,6 +316,35 @@ should be replaced by
.. includecode:: code/docs/MigrationsScala.scala#flatMapConcat
`Sink.fanoutPublisher() and Sink.publisher() is now a single method`
====================================================================
It was a common user mistake to use ``Sink.publisher`` and get into trouble since it would only support
a single ``Subscriber``, and the discoverability of the apprpriate fix was non-obvious (Sink.fanoutPublisher).
To make the decision whether to support fanout or not an active one, the aforementioned methods have been
replaced with a single method: ``Sink.asPublisher(fanout: Boolean)``.
Update procedure
----------------
1. Replace all occurences of ``Sink.publisher`` with ``Sink.asPublisher(false)``
2. Replace all occurences of ``Sink.fanoutPublisher`` with ``Sink.asPublisher(true)``
Example
^^^^^^^
::
// This no longer works!
val subSink = Sink.publisher
// This no longer works!
val subSink = Sink.fanoutPublisher(2, 8)
should be replaced by
.. includecode:: code/docs/MigrationsScala.scala#sink-as-publisher
FlexiMerge an FlexiRoute has been replaced by GraphStage
========================================================

View file

@ -102,7 +102,7 @@ This is how it can be used as input :class:`Source` to a :class:`Flow`:
.. includecode:: code/docs/stream/ActorPublisherDocSpec.scala#actor-publisher-usage
A publisher that is created with ``Sink.publisher`` supports a specified number of subscribers. Additional
A publisher that is created with ``Sink.asPublisher`` supports a specified number of subscribers. Additional
subscription attempts will be rejected with an :class:`IllegalStateException`.
ActorSubscriber
@ -409,7 +409,7 @@ by using the Publisher-:class:`Sink`:
.. includecode:: code/docs/stream/ReactiveStreamsDocSpec.scala#source-publisher
A publisher that is created with ``Sink.publisher(false)`` supports only a single subscription.
A publisher that is created with ``Sink.asPublisher(false)`` supports only a single subscription.
Additional subscription attempts will be rejected with an :class:`IllegalStateException`.
A publisher that supports multiple subscribers using fan-out/broadcasting is created as follows:
@ -433,4 +433,3 @@ passing a factory function that will create the :class:`Processor` instances:
.. includecode:: code/docs/stream/ReactiveStreamsDocSpec.scala#use-processor
Please note that a factory is necessary to achieve reusability of the resulting :class:`Flow`.

View file

@ -32,11 +32,11 @@ The process of materialization will often create specific objects that are usefu
Interoperation with other Reactive Streams implementations
----------------------------------------------------------
Akka Streams fully implement the Reactive Streams specification and interoperate with all other conformant implementations. We chose to completely separate the Reactive Streams interfaces from the user-level API because we regard them to be an SPI that is not targeted at endusers. In order to obtain a :class:`Publisher` or :class:`Subscriber` from an Akka Stream topology, a corresponding ``Sink.publisher`` or ``Source.subscriber`` element must be used.
Akka Streams fully implement the Reactive Streams specification and interoperate with all other conformant implementations. We chose to completely separate the Reactive Streams interfaces from the user-level API because we regard them to be an SPI that is not targeted at endusers. In order to obtain a :class:`Publisher` or :class:`Subscriber` from an Akka Stream topology, a corresponding ``Sink.asPublisher`` or ``Source.asSubscriber`` element must be used.
All stream Processors produced by the default materialization of Akka Streams are restricted to having a single Subscriber, additional Subscribers will be rejected. The reason for this is that the stream topologies described using our DSL never require fan-out behavior from the Publisher sides of the elements, all fan-out is done using explicit elements like :class:`Broadcast[T]`.
This means that ``Sink.publisher(true)`` (for enabling fan-out support) must be used where broadcast behavior is needed for interoperation with other Reactive Streams implementations.
This means that ``Sink.asPublisher(true)`` (for enabling fan-out support) must be used where broadcast behavior is needed for interoperation with other Reactive Streams implementations.
What shall users of streaming libraries expect?
-----------------------------------------------

View file

@ -73,7 +73,7 @@ private class PoolInterfaceActor(hcps: HostConnectionPoolSetup,
new InetSocketAddress(host, port), settings, setup.log)
.named("PoolFlow")
Source(ActorPublisher(self)).via(poolFlow).runWith(Sink(ActorSubscriber[ResponseContext](self)))
Source.fromPublisher(ActorPublisher(self)).via(poolFlow).runWith(Sink.fromSubscriber(ActorSubscriber[ResponseContext](self)))
}
activateIdleTimeoutIfNecessary()

View file

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

View file

@ -159,14 +159,14 @@ private[http] object StreamUtils {
case Nil Nil
case Seq(one) Vector(input.via(one))
case multiple
val (fanoutSub, fanoutPub) = Source.subscriber[ByteString].toMat(Sink.publisher(true))(Keep.both).run()
val (fanoutSub, fanoutPub) = Source.asSubscriber[ByteString].toMat(Sink.asPublisher(true))(Keep.both).run()
val sources = transformers.map { flow
// Doubly wrap to ensure that subscription to the running publisher happens before the final sources
// are exposed, so there is no race
Source(Source(fanoutPub).viaMat(flow)(Keep.right).runWith(Sink.publisher(false)))
Source.fromPublisher(Source.fromPublisher(fanoutPub).viaMat(flow)(Keep.right).runWith(Sink.asPublisher(false)))
}
// The fanout publisher must be wired to the original source after all fanout subscribers have been subscribed
input.runWith(Sink(fanoutSub))
input.runWith(Sink.fromSubscriber(fanoutSub))
sources
}
@ -315,7 +315,7 @@ private[http] object StreamUtils {
object OneTimeValve {
def apply(): OneTimeValve = new OneTimeValve {
val promise = Promise[Unit]()
val _source = Source(promise.future).drop(1) // we are only interested in the completion event
val _source = Source.fromFuture(promise.future).drop(1) // we are only interested in the completion event
def source[T]: Source[T, Unit] = _source.asInstanceOf[Source[T, Unit]] // safe, because source won't generate any elements
def open(): Unit = promise.success(())

View file

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

View file

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

View file

@ -89,7 +89,7 @@ class ConnectionPoolSpec extends AkkaSpec("""
override def testServerHandler(connNr: Int): HttpRequest HttpResponse = {
case request @ HttpRequest(_, Uri.Path("/a"), _, _, _)
val entity = HttpEntity.Chunked.fromData(ContentTypes.`text/plain(UTF-8)`, Source(responseEntityPub))
val entity = HttpEntity.Chunked.fromData(ContentTypes.`text/plain(UTF-8)`, Source.fromPublisher(responseEntityPub))
super.testServerHandler(connNr)(request) withEntity entity
case x super.testServerHandler(connNr)(x)
}
@ -99,7 +99,7 @@ class ConnectionPoolSpec extends AkkaSpec("""
acceptIncomingConnection()
val (Success(r1), 42) = responseOut.expectNext()
val responseEntityProbe = TestSubscriber.probe[ByteString]()
r1.entity.dataBytes.runWith(Sink(responseEntityProbe))
r1.entity.dataBytes.runWith(Sink.fromSubscriber(responseEntityProbe))
responseEntityProbe.expectSubscription().request(2)
responseEntityPub.sendNext(ByteString("YEAH"))
responseEntityProbe.expectNext(ByteString("YEAH"))
@ -131,7 +131,7 @@ class ConnectionPoolSpec extends AkkaSpec("""
val poolFlow = Http().cachedHostConnectionPool[Int](serverHostName, serverPort, settings = settings)
val N = 500
val requestIds = Source(() Iterator.from(1)).take(N)
val requestIds = Source.fromIterator(() Iterator.from(1)).take(N)
val idSum = requestIds.map(id HttpRequest(uri = s"/r$id") -> id).via(poolFlow).map {
case (Success(response), id)
requestUri(response) should endWith(s"/r$id")
@ -301,7 +301,7 @@ class ConnectionPoolSpec extends AkkaSpec("""
Flow[SslTlsOutbound].collect[ByteString] { case SendBytes(x) mapServerSideOutboundRawBytes(x) }
.transform(StreamUtils.recover { case NoErrorComplete ByteString.empty }),
Flow[ByteString].map(SessionBytes(null, _)))
val sink = if (autoAccept) Sink.foreach[Http.IncomingConnection](handleConnection) else Sink(incomingConnections)
val sink = if (autoAccept) Sink.foreach[Http.IncomingConnection](handleConnection) else Sink.fromSubscriber(incomingConnections)
// TODO getHostString in Java7
Tcp().bind(serverEndpoint.getHostName, serverEndpoint.getPort, idleTimeout = serverSettings.timeouts.idleTimeout)
.map { c
@ -345,7 +345,7 @@ class ConnectionPoolSpec extends AkkaSpec("""
def flowTestBench[T, Mat](poolFlow: Flow[(HttpRequest, T), (Try[HttpResponse], T), Mat]) = {
val requestIn = TestPublisher.probe[(HttpRequest, T)]()
val responseOut = TestSubscriber.manualProbe[(Try[HttpResponse], T)]
val hcp = Source(requestIn).viaMat(poolFlow)(Keep.right).to(Sink(responseOut)).run()
val hcp = Source.fromPublisher(requestIn).viaMat(poolFlow)(Keep.right).to(Sink.fromSubscriber(responseOut)).run()
val responseOutSub = responseOut.expectSubscription()
(requestIn, responseOut, responseOutSub, hcp)
}

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

@ -315,7 +315,7 @@ class WebsocketClientSpec extends FreeSpec with Matchers with WithMaterializerSp
RunnableGraph.fromGraph(GraphDSL.create(clientLayer) { implicit b
client
import GraphDSL.Implicits._
Source(netIn) ~> Flow[ByteString].map(SessionBytes(null, _)) ~> client.in2
Source.fromPublisher(netIn) ~> Flow[ByteString].map(SessionBytes(null, _)) ~> client.in2
client.out1 ~> Flow[SslTlsOutbound].collect { case SendBytes(x) x } ~> netOut.sink
client.out2 ~> clientImplementation ~> client.in1
ClosedShape
@ -370,6 +370,6 @@ class WebsocketClientSpec extends FreeSpec with Matchers with WithMaterializerSp
lazy val messagesIn = TestSubscriber.probe[Message]()
override def clientImplementation: Flow[Message, Message, Unit] =
Flow.fromSinkAndSourceMat(Sink(messagesIn), Source(messagesOut))(Keep.none)
Flow.fromSinkAndSourceMat(Sink.fromSubscriber(messagesIn), Source.fromPublisher(messagesOut))(Keep.none)
}
}

View file

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

View file

@ -89,7 +89,7 @@ class HttpEntitySpec extends FreeSpec with MustMatchers with BeforeAndAfterAll {
"Infinite data stream" in {
val neverCompleted = Promise[ByteString]()
intercept[TimeoutException] {
Await.result(Default(tpe, 42, Source(neverCompleted.future)).toStrict(100.millis), 150.millis)
Await.result(Default(tpe, 42, Source.fromFuture(neverCompleted.future)).toStrict(100.millis), 150.millis)
}.getMessage must be("HttpEntity.toStrict timed out after 100 milliseconds while still waiting for outstanding data")
}
}

View file

@ -104,7 +104,7 @@ object WSProbe {
val subscriber = TestSubscriber.probe[Message]()
val publisher = TestPublisher.probe[Message]()
def flow: Flow[Message, Message, Unit] = Flow.fromSinkAndSourceMat(Sink(subscriber), Source(publisher))(Keep.none)
def flow: Flow[Message, Message, Unit] = Flow.fromSinkAndSourceMat(Sink.fromSubscriber(subscriber), Source.fromPublisher(publisher))(Keep.none)
def sendMessage(message: Message): Unit = publisher.sendNext(message)
def sendMessage(text: String): Unit = sendMessage(TextMessage(text))

View file

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

View file

@ -201,7 +201,7 @@ class CodingDirectivesSpec extends RoutingSpec with Inside {
() text.grouped(8).map { chars
Chunk(chars.mkString): ChunkStreamPart
}
val chunkedTextEntity = HttpEntity.Chunked(ContentTypes.`text/plain(UTF-8)`, Source(textChunks))
val chunkedTextEntity = HttpEntity.Chunked(ContentTypes.`text/plain(UTF-8)`, Source.fromIterator(textChunks))
Post() ~> `Accept-Encoding`(gzip) ~> {
encodeResponseWith(Gzip) {

View file

@ -10,7 +10,7 @@ import org.reactivestreams.Publisher
class ConcatTest extends AkkaPublisherVerification[Int] {
def createPublisher(elements: Long): Publisher[Int] = {
Source(iterable(elements / 2)).concat(Source(iterable((elements + 1) / 2))).runWith(Sink.publisher(false))
Source(iterable(elements / 2)).concat(Source(iterable((elements + 1) / 2))).runWith(Sink.asPublisher(false))
}
}

View file

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

View file

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

View file

@ -13,7 +13,7 @@ class FlattenTest extends AkkaPublisherVerification[Int] {
def createPublisher(elements: Long): Publisher[Int] = {
val s1 = Source(iterable(elements / 2))
val s2 = Source(iterable((elements + 1) / 2))
Source(List(s1, s2)).flatMapConcat(ConstantFun.scalaIdentityFunction).runWith(Sink.publisher(false))
Source(List(s1, s2)).flatMapConcat(ConstantFun.scalaIdentityFunction).runWith(Sink.asPublisher(false))
}
}

View file

@ -9,7 +9,7 @@ import org.reactivestreams.Subscriber
class FoldSinkSubscriberTest extends AkkaSubscriberBlackboxVerification[Int] {
override def createSubscriber(): Subscriber[Int] =
Flow[Int].to(Sink.fold(0)(_ + _)).runWith(Source.subscriber)
Flow[Int].to(Sink.fold(0)(_ + _)).runWith(Source.asSubscriber)
override def createElement(element: Int): Int = element
}

View file

@ -9,7 +9,7 @@ import org.reactivestreams.Subscriber
class ForeachSinkSubscriberTest extends AkkaSubscriberBlackboxVerification[Int] {
override def createSubscriber(): Subscriber[Int] =
Flow[Int].to(Sink.foreach { _ }).runWith(Source.subscriber)
Flow[Int].to(Sink.foreach { _ }).runWith(Source.asSubscriber)
override def createElement(element: Int): Int = element
}

View file

@ -13,7 +13,7 @@ class FuturePublisherTest extends AkkaPublisherVerification[Int] {
def createPublisher(elements: Long): Publisher[Int] = {
val p = Promise[Int]()
val pub = Source(p.future).runWith(Sink.publisher(false))
val pub = Source.fromFuture(p.future).runWith(Sink.asPublisher(false))
p.success(0)
pub
}

View file

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

View file

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

View file

@ -10,7 +10,7 @@ import akka.stream.scaladsl.{ Keep, Source, Sink }
class MaybeSourceTest extends AkkaPublisherVerification[Int] {
def createPublisher(elements: Long): Publisher[Int] = {
val (p, pub) = Source.maybe[Int].toMat(Sink.publisher(false))(Keep.both).run()
val (p, pub) = Source.maybe[Int].toMat(Sink.asPublisher(false))(Keep.both).run()
p success Some(1)
pub
}

View file

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

View file

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

View file

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

View file

@ -27,7 +27,7 @@ abstract class BaseTwoStreamsSetup extends AkkaSpec {
def completedPublisher[T]: Publisher[T] = TestPublisher.empty[T]
def nonemptyPublisher[T](elems: immutable.Iterable[T]): Publisher[T] = Source(elems).runWith(Sink.publisher(false))
def nonemptyPublisher[T](elems: immutable.Iterable[T]): Publisher[T] = Source(elems).runWith(Sink.asPublisher(false))
def soonToFailPublisher[T]: Publisher[T] = TestPublisher.lazyError[T](TestException)

View file

@ -24,7 +24,7 @@ class ChainSetup[In, Out, M](
val upstream = TestPublisher.manualProbe[In]()
val downstream = TestSubscriber.probe[Out]()
private val s = Source(upstream).via(stream(Flow[In].map(x x).named("buh")))
private val s = Source.fromPublisher(upstream).via(stream(Flow[In].map(x x).named("buh")))
val publisher = toPublisher(s, materializer)
val upstreamSubscription = upstream.expectSubscription()
publisher.subscribe(downstream)

View file

@ -20,7 +20,7 @@ trait ScriptedTest extends Matchers {
class ScriptException(msg: String) extends RuntimeException(msg)
def toPublisher[In, Out]: (Source[Out, _], ActorMaterializer) Publisher[Out] =
(f, m) f.runWith(Sink.publisher(false))(m)
(f, m) f.runWith(Sink.asPublisher(false))(m)
object Script {
def apply[In, Out](phases: (Seq[In], Seq[Out])*): Script[In, Out] = {

View file

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

View file

@ -22,7 +22,7 @@ class TestPublisherSubscriberSpec extends AkkaSpec {
"have all events accessible from manual probes" in assertAllStagesStopped {
val upstream = TestPublisher.manualProbe[Int]()
val downstream = TestSubscriber.manualProbe[Int]()
Source(upstream).runWith(Sink.publisher(false))(materializer).subscribe(downstream)
Source.fromPublisher(upstream).runWith(Sink.asPublisher(false))(materializer).subscribe(downstream)
val upstreamSubscription = upstream.expectSubscription()
val downstreamSubscription: Subscription = downstream.expectEventPF { case OnSubscribe(sub) sub }
@ -46,7 +46,7 @@ class TestPublisherSubscriberSpec extends AkkaSpec {
"handle gracefully partial function that is not suitable" in assertAllStagesStopped {
val upstream = TestPublisher.manualProbe[Int]()
val downstream = TestSubscriber.manualProbe[Int]()
Source(upstream).runWith(Sink.publisher(false))(materializer).subscribe(downstream)
Source.fromPublisher(upstream).runWith(Sink.asPublisher(false))(materializer).subscribe(downstream)
val upstreamSubscription = upstream.expectSubscription()
val downstreamSubscription: Subscription = downstream.expectEventPF { case OnSubscribe(sub) sub }

View file

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

View file

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

View file

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

View file

@ -67,7 +67,7 @@ public class FlowGraphTest extends StreamTest {
final Source<String, BoxedUnit> in1 = Source.from(Arrays.asList("a", "b", "c"));
final Source<String, BoxedUnit> in2 = Source.from(Arrays.asList("d", "e", "f"));
final Sink<String, Publisher<String>> publisher = Sink.publisher(false);
final Sink<String, Publisher<String>> publisher = Sink.asPublisher(false);
final Source<String, BoxedUnit> source = Source.fromGraph(
GraphDSL.create(new Function<GraphDSL.Builder<BoxedUnit>, SourceShape<String>>() {
@ -82,7 +82,7 @@ public class FlowGraphTest extends StreamTest {
// collecting
final Publisher<String> pub = source.runWith(publisher, materializer);
final Future<List<String>> all = Source.from(pub).grouped(100).runWith(Sink.<List<String>>head(), materializer);
final Future<List<String>> all = Source.fromPublisher(pub).grouped(100).runWith(Sink.<List<String>>head(), materializer);
final List<String> result = Await.result(all, Duration.apply(200, TimeUnit.MILLISECONDS));
assertEquals(new HashSet<Object>(Arrays.asList("a", "b", "c", "d", "e", "f")), new HashSet<String>(result));

View file

@ -335,7 +335,7 @@ public class FlowTest extends StreamTest {
final Source<String, BoxedUnit> in1 = Source.from(Arrays.asList("a", "b", "c"));
final Source<String, BoxedUnit> in2 = Source.from(Arrays.asList("d", "e", "f"));
final Sink<String, Publisher<String>> publisher = Sink.publisher(false);
final Sink<String, Publisher<String>> publisher = Sink.asPublisher(false);
final Source<String, BoxedUnit> source = Source.fromGraph(
GraphDSL.create(new Function<GraphDSL.Builder<BoxedUnit>, SourceShape<String>>() {
@ -350,7 +350,7 @@ public class FlowTest extends StreamTest {
// collecting
final Publisher<String> pub = source.runWith(publisher, materializer);
final Future<List<String>> all = Source.from(pub).grouped(100).runWith(Sink.<List<String>>head(), materializer);
final Future<List<String>> all = Source.fromPublisher(pub).grouped(100).runWith(Sink.<List<String>>head(), materializer);
final List<String> result = Await.result(all, Duration.apply(200, TimeUnit.MILLISECONDS));
assertEquals(new HashSet<Object>(Arrays.asList("a", "b", "c", "d", "e", "f")), new HashSet<String>(result));
@ -558,7 +558,7 @@ public class FlowTest extends StreamTest {
final TestPublisher.ManualProbe<Integer> publisherProbe = TestPublisher.manualProbe(true,system);
final JavaTestKit probe = new JavaTestKit(system);
final Source<Integer, ?> source = Source.from(publisherProbe);
final Source<Integer, ?> source = Source.fromPublisher(publisherProbe);
final Flow<Integer, Integer, ?> flow = Flow.of(Integer.class).map(
new Function<Integer, Integer>() {
public Integer apply(Integer elem) {

View file

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

View file

@ -478,7 +478,7 @@ public class SourceTest extends StreamTest {
final JavaTestKit probe = new JavaTestKit(system);
final Iterable<String> input = Arrays.asList("A", "B", "C");
Future<String> future1 = Source.from(input).runWith(Sink.<String>head(), materializer);
Future<String> future2 = Source.from(future1).runWith(Sink.<String>head(), materializer);
Future<String> future2 = Source.fromFuture(future1).runWith(Sink.<String>head(), materializer);
String result = Await.result(future2, probe.dilated(FiniteDuration.create(3, TimeUnit.SECONDS)));
assertEquals("A", result);
}
@ -579,7 +579,7 @@ public class SourceTest extends StreamTest {
final ManualProbe<Integer> publisherProbe = TestPublisher.manualProbe(true,system);
final JavaTestKit probe = new JavaTestKit(system);
final Source<Integer, ?> source = Source.from(publisherProbe).map(
final Source<Integer, ?> source = Source.fromPublisher(publisherProbe).map(
new Function<Integer, Integer>() {
public Integer apply(Integer elem) {
if (elem == 1) throw new RuntimeException("ex");

View file

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

View file

@ -125,7 +125,7 @@ class ActorSubscriberSpec extends AkkaSpec with ImplicitSender {
"signal error" in {
val e = new RuntimeException("simulated") with NoStackTrace
val ref = Source(() throw e).runWith(Sink.actorSubscriber(manualSubscriberProps(testActor)))
val ref = Source.fromIterator(() throw e).runWith(Sink.actorSubscriber(manualSubscriberProps(testActor)))
ref ! "ready"
expectMsg(OnError(e))
}
@ -133,7 +133,7 @@ class ActorSubscriberSpec extends AkkaSpec with ImplicitSender {
"remember requested after restart" in {
// creating actor with default supervision, because stream supervisor default strategy is to stop
val ref = system.actorOf(manualSubscriberProps(testActor))
Source(1 to 7).runWith(Sink(ActorSubscriber[Int](ref)))
Source(1 to 7).runWith(Sink.fromSubscriber(ActorSubscriber[Int](ref)))
ref ! "ready"
expectMsg(OnNext(1))
expectMsg(OnNext(2))

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 c1 = TestSubscriber.manualProbe[Long]()
Source(List(1, 2, 3)).via(flow).runWith(Sink(c1))
Source(List(1, 2, 3)).via(flow).runWith(Sink.fromSubscriber(c1))
val s = c1.expectSubscription()
s.request(100)
@ -104,12 +104,12 @@ class FlowTimedSpec extends AkkaSpec with ScriptedTest {
map(_.toString), duration probe.ref ! duration).
map { s: String s + "!" }
val (flowIn: Subscriber[Int], flowOut: Publisher[String]) = flow.runWith(Source.subscriber[Int], Sink.publisher[String](false))
val (flowIn: Subscriber[Int], flowOut: Publisher[String]) = flow.runWith(Source.asSubscriber[Int], Sink.asPublisher[String](false))
val c1 = TestSubscriber.manualProbe[String]()
val c2 = flowOut.subscribe(c1)
val p = Source(0 to 100).runWith(Sink.publisher(false))
val p = Source(0 to 100).runWith(Sink.asPublisher(false))
p.subscribe(flowIn)
val s = c1.expectSubscription()

View file

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

View file

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

View file

@ -76,7 +76,7 @@ class FileSourceSpec extends AkkaSpec(UnboundedMailboxConfig) {
val p = Source.file(testFile, chunkSize)
.withAttributes(bufferAttributes)
.runWith(Sink.publisher(false))
.runWith(Sink.asPublisher(false))
val c = TestSubscriber.manualProbe[ByteString]()
p.subscribe(c)
val sub = c.expectSubscription()
@ -113,7 +113,7 @@ class FileSourceSpec extends AkkaSpec(UnboundedMailboxConfig) {
val p = Source.file(testFile, chunkSize)
.withAttributes(bufferAttributes)
.runWith(Sink.publisher(false))
.runWith(Sink.asPublisher(false))
val c = TestSubscriber.manualProbe[ByteString]()
p.subscribe(c)
@ -140,7 +140,7 @@ class FileSourceSpec extends AkkaSpec(UnboundedMailboxConfig) {
}
"onError whent trying to read from file which does not exist" in assertAllStagesStopped {
val p = Source.file(notExistingFile).runWith(Sink.publisher(false))
val p = Source.file(notExistingFile).runWith(Sink.asPublisher(false))
val c = TestSubscriber.manualProbe[ByteString]()
p.subscribe(c)

View file

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

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 {
val ex = new Exception("hello")
val (sub, out1, out2) =
RunnableGraph.fromGraph(GraphDSL.create(Source.subscriber[SslTlsOutbound], Sink.head[ByteString], Sink.head[SslTlsInbound])((_, _, _)) { implicit b
RunnableGraph.fromGraph(GraphDSL.create(Source.asSubscriber[SslTlsOutbound], Sink.head[ByteString], Sink.head[SslTlsInbound])((_, _, _)) { implicit b
(s, o1, o2)
val tls = b.add(clientTls(EagerClose))
s ~> tls.in1; tls.out1 ~> o1
@ -417,7 +417,7 @@ class TlsSpec extends AkkaSpec("akka.loglevel=INFO\nakka.actor.debug.receive=off
"reliably cancel subscriptions when UserIn fails early" in assertAllStagesStopped {
val ex = new Exception("hello")
val (sub, out1, out2) =
RunnableGraph.fromGraph(GraphDSL.create(Source.subscriber[ByteString], Sink.head[ByteString], Sink.head[SslTlsInbound])((_, _, _)) { implicit b
RunnableGraph.fromGraph(GraphDSL.create(Source.asSubscriber[ByteString], Sink.head[ByteString], Sink.head[SslTlsInbound])((_, _, _)) { implicit b
(s, o1, o2)
val tls = b.add(clientTls(EagerClose))
Source.failed[SslTlsOutbound](ex) ~> tls.in1; tls.out1 ~> o1

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 {
val probe = TestPublisher.manualProbe[Int]()
val queue = Source(probe).runWith(Sink.queue(3))
val queue = Source.fromPublisher(probe).runWith(Sink.queue(3))
val sub = probe.expectSubscription()
val future = queue.pull()
val future2 = queue.pull()
@ -49,7 +49,7 @@ class AcknowledgeSinkSpec extends AkkaSpec {
"wait for next element from upstream" in assertAllStagesStopped {
val probe = TestPublisher.manualProbe[Int]()
val queue = Source(probe).runWith(Sink.queue(3))
val queue = Source.fromPublisher(probe).runWith(Sink.queue(3))
val sub = probe.expectSubscription()
queue.pull().pipeTo(testActor)
@ -62,7 +62,7 @@ class AcknowledgeSinkSpec extends AkkaSpec {
"fail future on stream failure" in assertAllStagesStopped {
val probe = TestPublisher.manualProbe[Int]()
val queue = Source(probe).runWith(Sink.queue(3))
val queue = Source.fromPublisher(probe).runWith(Sink.queue(3))
val sub = probe.expectSubscription()
queue.pull().pipeTo(testActor)
@ -74,7 +74,7 @@ class AcknowledgeSinkSpec extends AkkaSpec {
"fail future when stream failed" in assertAllStagesStopped {
val probe = TestPublisher.manualProbe[Int]()
val queue = Source(probe).runWith(Sink.queue(3, 100.millis))
val queue = Source.fromPublisher(probe).runWith(Sink.queue(3, 100.millis))
val sub = probe.expectSubscription()
sub.sendError(ex) // potential race condition
@ -83,7 +83,7 @@ class AcknowledgeSinkSpec extends AkkaSpec {
"timeout future when stream cannot provide data" in assertAllStagesStopped {
val probe = TestPublisher.manualProbe[Int]()
val queue = Source(probe).runWith(Sink.queue(3))
val queue = Source.fromPublisher(probe).runWith(Sink.queue(3))
val sub = probe.expectSubscription()
queue.pull().pipeTo(testActor)
@ -96,7 +96,7 @@ class AcknowledgeSinkSpec extends AkkaSpec {
"work when buffer is 0" in assertAllStagesStopped {
val probe = TestPublisher.manualProbe[Int]()
val queue = Source(probe).runWith(Sink.queue(0))
val queue = Source.fromPublisher(probe).runWith(Sink.queue(0))
val sub = probe.expectSubscription()
sub.sendNext(1)

View file

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

View file

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

View file

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

View file

@ -51,7 +51,7 @@ class FlowBufferSpec extends AkkaSpec {
val publisher = TestPublisher.probe[Int]()
val subscriber = TestSubscriber.manualProbe[Int]()
Source(publisher).buffer(100, overflowStrategy = OverflowStrategy.backpressure).to(Sink(subscriber)).run()
Source.fromPublisher(publisher).buffer(100, overflowStrategy = OverflowStrategy.backpressure).to(Sink.fromSubscriber(subscriber)).run()
val sub = subscriber.expectSubscription()
// Fill up buffer
@ -69,7 +69,7 @@ class FlowBufferSpec extends AkkaSpec {
val publisher = TestPublisher.probe[Int]()
val subscriber = TestSubscriber.manualProbe[Int]()
Source(publisher).buffer(100, overflowStrategy = OverflowStrategy.dropHead).to(Sink(subscriber)).run()
Source.fromPublisher(publisher).buffer(100, overflowStrategy = OverflowStrategy.dropHead).to(Sink.fromSubscriber(subscriber)).run()
val sub = subscriber.expectSubscription()
// Fill up buffer
@ -98,7 +98,7 @@ class FlowBufferSpec extends AkkaSpec {
val publisher = TestPublisher.probe[Int]()
val subscriber = TestSubscriber.manualProbe[Int]()
Source(publisher).buffer(100, overflowStrategy = OverflowStrategy.dropTail).to(Sink(subscriber)).run()
Source.fromPublisher(publisher).buffer(100, overflowStrategy = OverflowStrategy.dropTail).to(Sink.fromSubscriber(subscriber)).run()
val sub = subscriber.expectSubscription()
// Fill up buffer
@ -130,7 +130,7 @@ class FlowBufferSpec extends AkkaSpec {
val publisher = TestPublisher.probe[Int]()
val subscriber = TestSubscriber.manualProbe[Int]()
Source(publisher).buffer(100, overflowStrategy = OverflowStrategy.dropBuffer).to(Sink(subscriber)).run()
Source.fromPublisher(publisher).buffer(100, overflowStrategy = OverflowStrategy.dropBuffer).to(Sink.fromSubscriber(subscriber)).run()
val sub = subscriber.expectSubscription()
// Fill up buffer
@ -184,7 +184,7 @@ class FlowBufferSpec extends AkkaSpec {
val publisher = TestPublisher.probe[Int]()
val subscriber = TestSubscriber.manualProbe[Int]()
Source(publisher).buffer(100, overflowStrategy = OverflowStrategy.fail).to(Sink(subscriber)).run()
Source.fromPublisher(publisher).buffer(100, overflowStrategy = OverflowStrategy.fail).to(Sink.fromSubscriber(subscriber)).run()
val sub = subscriber.expectSubscription()
// Fill up buffer
@ -211,7 +211,7 @@ class FlowBufferSpec extends AkkaSpec {
val publisher = TestPublisher.probe[Int]()
val subscriber = TestSubscriber.manualProbe[Int]()
Source(publisher).buffer(1, overflowStrategy = strategy).to(Sink(subscriber)).run()
Source.fromPublisher(publisher).buffer(1, overflowStrategy = strategy).to(Sink.fromSubscriber(subscriber)).run()
val sub = subscriber.expectSubscription()
// Fill up buffer

View file

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

View file

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

View file

@ -17,7 +17,7 @@ class FlowConcatSpec extends BaseTwoStreamsSetup {
override def setup(p1: Publisher[Int], p2: Publisher[Int]) = {
val subscriber = TestSubscriber.probe[Outputs]()
Source(p1).concat(Source(p2)).runWith(Sink(subscriber))
Source.fromPublisher(p1).concat(Source.fromPublisher(p2)).runWith(Sink.fromSubscriber(subscriber))
subscriber
}
@ -29,7 +29,7 @@ class FlowConcatSpec extends BaseTwoStreamsSetup {
val s2: Source[String, _] = Source(List(4, 5, 6)).map(_.toString + "-s")
val subs = TestSubscriber.manualProbe[Any]()
val subSink = Sink.publisher[Any](false)
val subSink = Sink.asPublisher[Any](false)
val (_, res) = f1.concat(s2).runWith(s1, subSink)
@ -101,7 +101,7 @@ class FlowConcatSpec extends BaseTwoStreamsSetup {
"correctly handle async errors in secondary upstream" in assertAllStagesStopped {
val promise = Promise[Int]()
val subscriber = TestSubscriber.manualProbe[Int]()
Source(List(1, 2, 3)).concat(Source(promise.future)).runWith(Sink(subscriber))
Source(List(1, 2, 3)).concat(Source.fromFuture(promise.future)).runWith(Sink.fromSubscriber(subscriber))
val subscription = subscriber.expectSubscription()
subscription.request(4)
@ -152,7 +152,7 @@ class FlowConcatSpec extends BaseTwoStreamsSetup {
"subscribe at once to initial source and to one that it's concat to" in {
val publisher1 = TestPublisher.probe[Int]()
val publisher2 = TestPublisher.probe[Int]()
val probeSink = Source(publisher1).concat(Source(publisher2))
val probeSink = Source.fromPublisher(publisher1).concat(Source.fromPublisher(publisher2))
.runWith(TestSink.probe[Int])
val sub1 = publisher1.expectSubscription()

View file

@ -22,7 +22,7 @@ class FlowConflateSpec extends AkkaSpec {
val publisher = TestPublisher.probe[Int]()
val subscriber = TestSubscriber.manualProbe[Int]()
Source(publisher).conflate(seed = i i)(aggregate = (sum, i) sum + i).to(Sink(subscriber)).run()
Source.fromPublisher(publisher).conflate(seed = i i)(aggregate = (sum, i) sum + i).to(Sink.fromSubscriber(subscriber)).run()
val sub = subscriber.expectSubscription()
for (i 1 to 100) {
@ -38,7 +38,7 @@ class FlowConflateSpec extends AkkaSpec {
val publisher = TestPublisher.probe[Int]()
val subscriber = TestSubscriber.manualProbe[Int]()
Source(publisher).conflate(seed = i i)(aggregate = (sum, i) sum + i).to(Sink(subscriber)).run()
Source.fromPublisher(publisher).conflate(seed = i i)(aggregate = (sum, i) sum + i).to(Sink.fromSubscriber(subscriber)).run()
val sub = subscriber.expectSubscription()
for (i 1 to 100) {
@ -62,7 +62,7 @@ class FlowConflateSpec extends AkkaSpec {
val publisher = TestPublisher.probe[Int]()
val subscriber = TestSubscriber.manualProbe[Int]()
Source(publisher).conflate(seed = i i)(aggregate = (sum, i) sum + i).to(Sink(subscriber)).run()
Source.fromPublisher(publisher).conflate(seed = i i)(aggregate = (sum, i) sum + i).to(Sink.fromSubscriber(subscriber)).run()
val sub = subscriber.expectSubscription()
sub.request(1)

View file

@ -49,7 +49,7 @@ class FlowDelaySpec extends AkkaSpec {
val c = TestSubscriber.manualProbe[Int]()
val p = TestPublisher.manualProbe[Int]()
Source(p).delay(300.millis).to(Sink(c)).run()
Source.fromPublisher(p).delay(300.millis).to(Sink.fromSubscriber(c)).run()
val cSub = c.expectSubscription()
val pSub = p.expectSubscription()
cSub.request(100)
@ -111,7 +111,7 @@ class FlowDelaySpec extends AkkaSpec {
val c = TestSubscriber.manualProbe[Int]()
val p = TestPublisher.manualProbe[Int]()
Source(p).delay(10.seconds, DelayOverflowStrategy.emitEarly).withAttributes(inputBuffer(16, 16)).to(Sink(c)).run()
Source.fromPublisher(p).delay(10.seconds, DelayOverflowStrategy.emitEarly).withAttributes(inputBuffer(16, 16)).to(Sink.fromSubscriber(c)).run()
val cSub = c.expectSubscription()
val pSub = p.expectSubscription()
cSub.request(20)

View file

@ -28,7 +28,7 @@ class FlowDropSpec extends AkkaSpec with ScriptedTest {
"not drop anything for negative n" in {
val probe = TestSubscriber.manualProbe[Int]()
Source(List(1, 2, 3)).drop(-1).to(Sink(probe)).run()
Source(List(1, 2, 3)).drop(-1).to(Sink.fromSubscriber(probe)).run()
probe.expectSubscription().request(10)
probe.expectNext(1)
probe.expectNext(2)

View file

@ -18,7 +18,7 @@ class FlowDropWithinSpec extends AkkaSpec {
val input = Iterator.from(1)
val p = TestPublisher.manualProbe[Int]()
val c = TestSubscriber.manualProbe[Int]()
Source(p).dropWithin(1.second).to(Sink(c)).run()
Source.fromPublisher(p).dropWithin(1.second).to(Sink.fromSubscriber(c)).run()
val pSub = p.expectSubscription
val cSub = c.expectSubscription
cSub.request(100)
@ -39,7 +39,7 @@ class FlowDropWithinSpec extends AkkaSpec {
val upstream = TestPublisher.probe[Int]()
val downstream = TestSubscriber.probe[Int]()
Source(upstream).dropWithin(1.day).runWith(Sink(downstream))
Source.fromPublisher(upstream).dropWithin(1.day).runWith(Sink.fromSubscriber(downstream))
upstream.sendComplete()
downstream.expectSubscriptionAndComplete()

View file

@ -28,7 +28,7 @@ class FlowExpandSpec extends AkkaSpec {
val subscriber = TestSubscriber.probe[Int]()
// Simply repeat the last element as an extrapolation step
Source(publisher).expand(seed = i i)(extrapolate = i (i, i)).to(Sink(subscriber)).run()
Source.fromPublisher(publisher).expand(seed = i i)(extrapolate = i (i, i)).to(Sink.fromSubscriber(subscriber)).run()
for (i 1 to 100) {
// Order is important here: If the request comes first it will be extrapolated!
@ -44,7 +44,7 @@ class FlowExpandSpec extends AkkaSpec {
val subscriber = TestSubscriber.probe[Int]()
// Simply repeat the last element as an extrapolation step
Source(publisher).expand(seed = i i)(extrapolate = i (i, i)).to(Sink(subscriber)).run()
Source.fromPublisher(publisher).expand(seed = i i)(extrapolate = i (i, i)).to(Sink.fromSubscriber(subscriber)).run()
publisher.sendNext(42)
@ -66,7 +66,7 @@ class FlowExpandSpec extends AkkaSpec {
val subscriber = TestSubscriber.probe[Int]()
// Simply repeat the last element as an extrapolation step
Source(publisher).expand(seed = i i)(extrapolate = i (i, i)).to(Sink(subscriber)).run()
Source.fromPublisher(publisher).expand(seed = i i)(extrapolate = i (i, i)).to(Sink.fromSubscriber(subscriber)).run()
publisher.sendNext(1)
subscriber.requestNext(1)
@ -94,7 +94,7 @@ class FlowExpandSpec extends AkkaSpec {
val publisher = TestPublisher.probe[Int]()
val subscriber = TestSubscriber.probe[Int]()
Source(publisher).expand(seed = i i)(extrapolate = i (i, i)).to(Sink(subscriber)).run()
Source.fromPublisher(publisher).expand(seed = i i)(extrapolate = i (i, i)).to(Sink.fromSubscriber(subscriber)).run()
publisher.sendNext(1)
subscriber.requestNext(1)

View file

@ -27,7 +27,7 @@ class FlowFilterSpec extends AkkaSpec with ScriptedTest {
implicit val materializer = ActorMaterializer(settings)
val probe = TestSubscriber.manualProbe[Int]()
Source(List.fill(1000)(0) ::: List(1)).filter(_ != 0).runWith(Sink(probe))
Source(List.fill(1000)(0) ::: List(1)).filter(_ != 0).runWith(Sink.fromSubscriber(probe))
val subscription = probe.expectSubscription()
for (_ 1 to 10000) {

View file

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

View file

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

View file

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

View file

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

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

View file

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

View file

@ -33,7 +33,7 @@ class FlowIdleInjectSpec extends AkkaSpec {
val upstream = TestPublisher.probe[Int]()
val downstream = TestSubscriber.probe[Int]()
Source(upstream).keepAlive(1.second, () 0).runWith(Sink(downstream))
Source.fromPublisher(upstream).keepAlive(1.second, () 0).runWith(Sink.fromSubscriber(downstream))
downstream.request(1)
@ -48,7 +48,7 @@ class FlowIdleInjectSpec extends AkkaSpec {
val upstream = TestPublisher.probe[Int]()
val downstream = TestSubscriber.probe[Int]()
(Source(1 to 10) ++ Source(upstream)).keepAlive(1.second, () 0).runWith(Sink(downstream))
(Source(1 to 10) ++ Source.fromPublisher(upstream)).keepAlive(1.second, () 0).runWith(Sink.fromSubscriber(downstream))
downstream.request(10)
downstream.expectNextN(1 to 10)
@ -66,7 +66,7 @@ class FlowIdleInjectSpec extends AkkaSpec {
val upstream = TestPublisher.probe[Int]()
val downstream = TestSubscriber.probe[Int]()
Source(upstream).keepAlive(1.second, () 0).runWith(Sink(downstream))
Source.fromPublisher(upstream).keepAlive(1.second, () 0).runWith(Sink.fromSubscriber(downstream))
downstream.ensureSubscription()
downstream.expectNoMsg(1.5.second)
@ -81,7 +81,7 @@ class FlowIdleInjectSpec extends AkkaSpec {
val upstream = TestPublisher.probe[Int]()
val downstream = TestSubscriber.probe[Int]()
(Source(1 to 10) ++ Source(upstream)).keepAlive(1.second, () 0).runWith(Sink(downstream))
(Source(1 to 10) ++ Source.fromPublisher(upstream)).keepAlive(1.second, () 0).runWith(Sink.fromSubscriber(downstream))
downstream.request(10)
downstream.expectNextN(1 to 10)
@ -98,7 +98,7 @@ class FlowIdleInjectSpec extends AkkaSpec {
val upstream = TestPublisher.probe[Int]()
val downstream = TestSubscriber.probe[Int]()
Source(upstream).keepAlive(1.second, () 0).runWith(Sink(downstream))
Source.fromPublisher(upstream).keepAlive(1.second, () 0).runWith(Sink.fromSubscriber(downstream))
downstream.ensureSubscription()
downstream.expectNoMsg(1.5.second)
@ -115,7 +115,7 @@ class FlowIdleInjectSpec extends AkkaSpec {
val upstream = TestPublisher.probe[Int]()
val downstream = TestSubscriber.probe[Int]()
(Source(1 to 10) ++ Source(upstream)).keepAlive(1.second, () 0).runWith(Sink(downstream))
(Source(1 to 10) ++ Source.fromPublisher(upstream)).keepAlive(1.second, () 0).runWith(Sink.fromSubscriber(downstream))
downstream.request(10)
downstream.expectNextN(1 to 10)
@ -134,7 +134,7 @@ class FlowIdleInjectSpec extends AkkaSpec {
val upstream = TestPublisher.probe[Int]()
val downstream = TestSubscriber.probe[Int]()
Source(upstream).keepAlive(1.second, () 0).runWith(Sink(downstream))
Source.fromPublisher(upstream).keepAlive(1.second, () 0).runWith(Sink.fromSubscriber(downstream))
downstream.request(2)
downstream.expectNoMsg(500.millis)

View file

@ -37,7 +37,7 @@ class FlowInitialDelaySpec extends AkkaSpec {
"properly ignore timer while backpressured" in Utils.assertAllStagesStopped {
val probe = TestSubscriber.probe[Int]()
Source(1 to 10).initialDelay(0.5.second).runWith(Sink(probe))
Source(1 to 10).initialDelay(0.5.second).runWith(Sink.fromSubscriber(probe))
probe.ensureSubscription()
probe.expectNoMsg(1.5.second)

View file

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

View file

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

View file

@ -32,7 +32,7 @@ class FlowJoinSpec extends AkkaSpec(ConfigFactory.parseString("akka.loglevel=INF
val broadcast = b.add(Broadcast[Int](2))
source ~> merge.in(0)
merge.out ~> broadcast.in
broadcast.out(0).grouped(1000) ~> Sink(probe)
broadcast.out(0).grouped(1000) ~> Sink.fromSubscriber(probe)
FlowShape(merge.in(1), broadcast.out(1))
})

View file

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

View file

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

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