diff --git a/akka-bench-jmh-dev/src/main/scala/akka/stream/MaterializationBenchmark.scala b/akka-bench-jmh-dev/src/main/scala/akka/stream/MaterializationBenchmark.scala index b06d726031..415debc9c1 100644 --- a/akka-bench-jmh-dev/src/main/scala/akka/stream/MaterializationBenchmark.scala +++ b/akka-bench-jmh-dev/src/main/scala/akka/stream/MaterializationBenchmark.scala @@ -20,7 +20,7 @@ object MaterializationBenchmark { } val graphWithJunctionsBuilder = (numOfJunctions: Int) => - FlowGraph.closed() { implicit b => + RunnableGraph.fromGraph(FlowGraph.create() { implicit b => import FlowGraph.Implicits._ val broadcast = b.add(Broadcast[Unit](numOfJunctions)) @@ -34,39 +34,39 @@ object MaterializationBenchmark { Source.single(()) ~> broadcast outlet ~> Sink.ignore - } + ClosedShape + }) val graphWithNestedImportsBuilder = (numOfNestedGraphs: Int) => { var flow: Graph[FlowShape[Unit, Unit], Unit] = Flow[Unit].map(identity) for (_ <- 1 to numOfNestedGraphs) { - flow = FlowGraph.partial(flow) { b ⇒ + flow = FlowGraph.create(flow) { b ⇒ flow ⇒ FlowShape(flow.inlet, flow.outlet) } } - FlowGraph.closed(flow) { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create(flow) { implicit b ⇒ flow ⇒ import FlowGraph.Implicits._ Source.single(()) ~> flow ~> Sink.ignore - } + ClosedShape + }) } - val graphWithImportedFlowBuilder = (numOfFlows: Int) => { - val flow = Flow[Unit].map(identity) - FlowGraph.closed() { b ⇒ - val source = b.add(Source.single(())) - var outlet = source + val graphWithImportedFlowBuilder = (numOfFlows: Int) => + RunnableGraph.fromGraph(FlowGraph.create(Source.single(())) { implicit b ⇒ source ⇒ + import FlowGraph.Implicits._ + val flow = Flow[Unit].map(identity) + var outlet: Outlet[Unit] = source.outlet for (i <- 0 until numOfFlows) { val flowShape = b.add(flow) - b.addEdge(outlet, flowShape.inlet) + outlet ~> flowShape outlet = flowShape.outlet } - - val sink = b.add(Sink.ignore) - b.addEdge(outlet, sink) - } - } + outlet ~> Sink.ignore + ClosedShape + }) } @State(Scope.Benchmark) diff --git a/akka-docs-dev/rst/java/stream-composition.rst b/akka-docs-dev/rst/java/stream-composition.rst index afe0b91cf2..419d829f71 100644 --- a/akka-docs-dev/rst/java/stream-composition.rst +++ b/akka-docs-dev/rst/java/stream-composition.rst @@ -126,8 +126,8 @@ As a first example, let's look at a more complex layout: The diagram shows a :class:`RunnableGraph` (remember, if there are no unwired ports, the graph is closed, and therefore can be materialized) that encapsulates a non-trivial stream processing network. It contains fan-in, fan-out stages, -directed and non-directed cycles. The ``closed()`` method of the :class:`FlowGraph` factory object allows the creation of a -general closed graph. For example the network on the diagram can be realized like this: +directed and non-directed cycles. The ``runnable()`` method of the :class:`FlowGraph` factory object allows the creation of a +general, closed, and runnable graph. For example the network on the diagram can be realized like this: .. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/CompositionDocTest.java#complex-graph @@ -140,7 +140,7 @@ It is possible to refer to the ports, so another version might look like this: Similar to the case in the first section, so far we have not considered modularity. We created a complex graph, but the layout is flat, not modularized. We will modify our example, and create a reusable component with the graph DSL. -The way to do it is to use the ``partial()`` method on :class:`FlowGraph` factory. If we remove the sources and sinks +The way to do it is to use the ``create()`` method on :class:`FlowGraph` factory. If we remove the sources and sinks from the previous example, what remains is a partial graph: | @@ -283,7 +283,7 @@ Attributes ---------- We have seen that we can use ``named()`` to introduce a nesting level in the fluid DSL (and also explicit nesting by using -``partial()`` from :class:`FlowGraph`). Apart from having the effect of adding a nesting level, ``named()`` is actually +``create()`` from :class:`FlowGraph`). Apart from having the effect of adding a nesting level, ``named()`` is actually a shorthand for calling ``withAttributes(Attributes.name("someName"))``. Attributes provide a way to fine-tune certain aspects of the materialized running entity. For example buffer sizes can be controlled via attributes (see :ref:`stream-buffers-scala`). When it comes to hierarchic composition, attributes are inherited by nested modules, diff --git a/akka-docs-dev/rst/java/stream-graphs.rst b/akka-docs-dev/rst/java/stream-graphs.rst index cbb0ad57b7..31b0f09e0c 100644 --- a/akka-docs-dev/rst/java/stream-graphs.rst +++ b/akka-docs-dev/rst/java/stream-graphs.rst @@ -85,8 +85,8 @@ Constructing and combining Partial Flow Graphs Sometimes it is not possible (or needed) to construct the entire computation graph in one place, but instead construct all of its different phases in different places and in the end connect them all into a complete graph and run it. -This can be achieved using ``FlowGraph.factory().partial()`` instead of -``FlowGraph.factory().closed()``, which will return a ``Graph`` instead of a +This can be achieved using ``FlowGraph.create()`` instead of +``FlowGraph.runnable()``, which will return a ``Graph`` instead of a ``RunnableGraph``. The reason of representing it as a different type is that a :class:`RunnableGraph` requires all ports to be connected, and if they are not it will throw an exception at construction time, which helps to avoid simple diff --git a/akka-docs-dev/rst/java/stream-quickstart.rst b/akka-docs-dev/rst/java/stream-quickstart.rst index 4080ad51c4..764569f024 100644 --- a/akka-docs-dev/rst/java/stream-quickstart.rst +++ b/akka-docs-dev/rst/java/stream-quickstart.rst @@ -119,7 +119,7 @@ detail in :ref:`constructing-sources-sinks-flows-from-partial-graphs-java`. Flow .. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/TwitterStreamQuickstartDocTest.java#flow-graph-broadcast As you can see, we use graph builder ``b`` to construct the graph using ``UniformFanOutShape`` and ``Flow``s. Once we have the -FlowGraph as a result of ``closed()`` method *it is immutable, thread-safe, and freely shareable*. A graph can be ``run()`` directly - +FlowGraph as a result of ``runnable()`` method *it is immutable, thread-safe, and freely shareable*. A graph can be ``run()`` directly - assuming all ports (sinks/sources) within a flow have been connected properly. It is possible also to construct several :class:`PartialFlowGraph`s and and then combine them into one fully connected graph. This will be covered in detail in :ref:`partial-flow-graph-java`. diff --git a/akka-docs-dev/rst/scala/code/docs/stream/BidiFlowDocSpec.scala b/akka-docs-dev/rst/scala/code/docs/stream/BidiFlowDocSpec.scala index 9a10eddb48..47995bf3d2 100644 --- a/akka-docs-dev/rst/scala/code/docs/stream/BidiFlowDocSpec.scala +++ b/akka-docs-dev/rst/scala/code/docs/stream/BidiFlowDocSpec.scala @@ -44,21 +44,21 @@ object BidiFlowDocSpec { } //#codec-impl - val codecVerbose = BidiFlow() { b => + val codecVerbose = BidiFlow.fromGraph(FlowGraph.create() { b => // construct and add the top flow, going outbound val outbound = b.add(Flow[Message].map(toBytes)) // construct and add the bottom flow, going inbound val inbound = b.add(Flow[ByteString].map(fromBytes)) // fuse them together into a BidiShape BidiShape.fromFlows(outbound, inbound) - } + }) // this is the same as the above - val codec = BidiFlow(toBytes _, fromBytes _) + val codec = BidiFlow.fromFunctions(toBytes _, fromBytes _) //#codec //#framing - val framing = BidiFlow() { b => + val framing = BidiFlow.fromGraph(FlowGraph.create() { b => implicit val order = ByteOrder.LITTLE_ENDIAN def addLengthHeader(bytes: ByteString) = { @@ -113,18 +113,18 @@ object BidiFlowDocSpec { val outbound = b.add(Flow[ByteString].map(addLengthHeader)) val inbound = b.add(Flow[ByteString].transform(() => new FrameParser)) BidiShape.fromFlows(outbound, inbound) - } + }) //#framing - val chopUp = BidiFlow() { b => + val chopUp = BidiFlow.fromGraph(FlowGraph.create() { b => val f = Flow[ByteString].mapConcat(_.map(ByteString(_))) BidiShape.fromFlows(b.add(f), b.add(f)) - } + }) - val accumulate = BidiFlow() { b => + val accumulate = BidiFlow.fromGraph(FlowGraph.create() { b => val f = Flow[ByteString].grouped(1000).map(_.fold(ByteString.empty)(_ ++ _)) BidiShape.fromFlows(b.add(f), b.add(f)) - } + }) } class BidiFlowDocSpec extends AkkaSpec with ConversionCheckedTripleEquals { diff --git a/akka-docs-dev/rst/scala/code/docs/stream/CompositionDocSpec.scala b/akka-docs-dev/rst/scala/code/docs/stream/CompositionDocSpec.scala index 92043e2a8b..61ae6fedeb 100644 --- a/akka-docs-dev/rst/scala/code/docs/stream/CompositionDocSpec.scala +++ b/akka-docs-dev/rst/scala/code/docs/stream/CompositionDocSpec.scala @@ -77,7 +77,7 @@ class CompositionDocSpec extends AkkaSpec { // format: OFF //#complex-graph import FlowGraph.Implicits._ - FlowGraph.closed() { implicit builder => + RunnableGraph.fromGraph(FlowGraph.create() { implicit builder => val A: Outlet[Int] = builder.add(Source.single(0)) val B: UniformFanOutShape[Int, Int] = builder.add(Broadcast[Int](2)) val C: UniformFanInShape[Int, Int] = builder.add(Merge[Int](2)) @@ -90,12 +90,14 @@ class CompositionDocSpec extends AkkaSpec { A ~> B ~> C ~> F B ~> D ~> E ~> F E ~> G - } + + ClosedShape + }) //#complex-graph //#complex-graph-alt import FlowGraph.Implicits._ - FlowGraph.closed() { implicit builder => + RunnableGraph.fromGraph(FlowGraph.create() { implicit builder => val B = builder.add(Broadcast[Int](2)) val C = builder.add(Merge[Int](2)) val E = builder.add(Balance[Int](2)) @@ -106,7 +108,8 @@ class CompositionDocSpec extends AkkaSpec { B.out(1).map(_ + 1) ~> E.in; E.out(0) ~> F.in(1) E.out(1) ~> Sink.foreach(println) - } + ClosedShape + }) //#complex-graph-alt // format: ON } @@ -115,7 +118,7 @@ class CompositionDocSpec extends AkkaSpec { // format: OFF //#partial-graph import FlowGraph.Implicits._ - val partial = FlowGraph.partial() { implicit builder => + val partial = FlowGraph.create() { implicit builder => val B = builder.add(Broadcast[Int](2)) val C = builder.add(Merge[Int](2)) val E = builder.add(Balance[Int](2)) @@ -137,17 +140,17 @@ class CompositionDocSpec extends AkkaSpec { //#partial-flow-dsl // Convert the partial graph of FlowShape to a Flow to get // access to the fluid DSL (for example to be able to call .filter()) - val flow = Flow.wrap(partial) + val flow = Flow.fromGraph(partial) // Simple way to create a graph backed Source - val source = Source() { implicit builder => + val source = Source.fromGraph( FlowGraph.create() { implicit builder => val merge = builder.add(Merge[Int](2)) Source.single(0) ~> merge Source(List(2, 3, 4)) ~> merge // Exposing exactly one output port - merge.out - } + SourceShape(merge.out) + }) // Building a Sink with a nested Flow, using the fluid DSL val sink = { @@ -164,22 +167,24 @@ class CompositionDocSpec extends AkkaSpec { "closed graph" in { //#embed-closed val closed1 = Source.single(0).to(Sink.foreach(println)) - val closed2 = FlowGraph.closed() { implicit builder => + val closed2 = RunnableGraph.fromGraph(FlowGraph.create() { implicit builder => val embeddedClosed: ClosedShape = builder.add(closed1) - } + // … + embeddedClosed + }) //#embed-closed } "materialized values" in { //#mat-combine-1 - // Materializes to Promise[Unit] (red) - val source: Source[Int, Promise[Unit]] = Source.lazyEmpty[Int] + // Materializes to Promise[Option[Int]] (red) + val source: Source[Int, Promise[Option[Int]]] = Source.maybe[Int] // Materializes to Unit (black) val flow1: Flow[Int, Int, Unit] = Flow[Int].take(100) - // Materializes to Promise[Unit] (red) - val nestedSource: Source[Int, Promise[Unit]] = + // Materializes to Promise[Int] (red) + val nestedSource: Source[Int, Promise[Option[Int]]] = source.viaMat(flow1)(Keep.left).named("nestedSource") //#mat-combine-1 @@ -206,11 +211,11 @@ class CompositionDocSpec extends AkkaSpec { //#mat-combine-3 //#mat-combine-4 - case class MyClass(private val p: Promise[Unit], conn: OutgoingConnection) { - def close() = p.success(()) + case class MyClass(private val p: Promise[Option[Int]], conn: OutgoingConnection) { + def close() = p.trySuccess(None) } - def f(p: Promise[Unit], + def f(p: Promise[Option[Int]], rest: (Future[OutgoingConnection], Future[String])): Future[MyClass] = { val connFuture = rest._1 diff --git a/akka-docs-dev/rst/scala/code/docs/stream/FlowDocSpec.scala b/akka-docs-dev/rst/scala/code/docs/stream/FlowDocSpec.scala index daeb0b39bf..3da3a92cf4 100644 --- a/akka-docs-dev/rst/scala/code/docs/stream/FlowDocSpec.scala +++ b/akka-docs-dev/rst/scala/code/docs/stream/FlowDocSpec.scala @@ -4,6 +4,7 @@ package docs.stream import akka.actor.Cancellable +import akka.stream.{ ClosedShape, FlowShape } import akka.stream.scaladsl._ import akka.stream.testkit.AkkaSpec @@ -147,17 +148,17 @@ class FlowDocSpec extends AkkaSpec { "various ways of transforming materialized values" in { import scala.concurrent.duration._ - val throttler = Flow(Source(1.second, 1.second, "test")) { implicit builder => + val throttler = Flow.fromGraph(FlowGraph.create(Source(1.second, 1.second, "test")) { implicit builder => tickSource => import FlowGraph.Implicits._ val zip = builder.add(ZipWith[String, Int, Int](Keep.right)) tickSource ~> zip.in0 - (zip.in1, zip.out) - } + FlowShape(zip.in1, zip.out) + }) //#flow-mat-combine - // An empty source that can be shut down explicitly from the outside - val source: Source[Int, Promise[Unit]] = Source.lazyEmpty[Int] + // An source that can be signalled explicitly from the outside + val source: Source[Int, Promise[Option[Int]]] = Source.maybe[Int] // A flow that internally throttles elements to 1/second, and returns a Cancellable // which can be used to shut down the stream @@ -167,7 +168,7 @@ class FlowDocSpec extends AkkaSpec { val sink: Sink[Int, Future[Int]] = Sink.head[Int] // By default, the materialized value of the leftmost stage is preserved - val r1: RunnableGraph[Promise[Unit]] = source.via(flow).to(sink) + val r1: RunnableGraph[Promise[Option[Int]]] = source.via(flow).to(sink) // Simple selection of materialized values by using Keep.right val r2: RunnableGraph[Cancellable] = source.viaMat(flow)(Keep.right).to(sink) @@ -176,17 +177,17 @@ class FlowDocSpec extends AkkaSpec { // Using runWith will always give the materialized values of the stages added // by runWith() itself val r4: Future[Int] = source.via(flow).runWith(sink) - val r5: Promise[Unit] = flow.to(sink).runWith(source) - val r6: (Promise[Unit], Future[Int]) = flow.runWith(source, sink) + val r5: Promise[Option[Int]] = flow.to(sink).runWith(source) + val r6: (Promise[Option[Int]], Future[Int]) = flow.runWith(source, sink) // Using more complext combinations - val r7: RunnableGraph[(Promise[Unit], Cancellable)] = + val r7: RunnableGraph[(Promise[Option[Int]], Cancellable)] = source.viaMat(flow)(Keep.both).to(sink) - val r8: RunnableGraph[(Promise[Unit], Future[Int])] = + val r8: RunnableGraph[(Promise[Option[Int]], Future[Int])] = source.via(flow).toMat(sink)(Keep.both) - val r9: RunnableGraph[((Promise[Unit], Cancellable), Future[Int])] = + val r9: RunnableGraph[((Promise[Option[Int]], Cancellable), Future[Int])] = source.viaMat(flow)(Keep.both).toMat(sink)(Keep.both) val r10: RunnableGraph[(Cancellable, Future[Int])] = @@ -194,7 +195,7 @@ class FlowDocSpec extends AkkaSpec { // It is also possible to map over the materialized values. In r9 we had a // doubly nested pair, but we want to flatten it out - val r11: RunnableGraph[(Promise[Unit], Cancellable, Future[Int])] = + val r11: RunnableGraph[(Promise[Option[Int]], Cancellable, Future[Int])] = r9.mapMaterializedValue { case ((promise, cancellable), future) => (promise, cancellable, future) @@ -204,17 +205,18 @@ class FlowDocSpec extends AkkaSpec { val (promise, cancellable, future) = r11.run() // Type inference works as expected - promise.success(()) + promise.success(None) cancellable.cancel() future.map(_ + 3) // The result of r11 can be also achieved by using the Graph API - val r12: RunnableGraph[(Promise[Unit], Cancellable, Future[Int])] = - FlowGraph.closed(source, flow, sink)((_, _, _)) { implicit builder => + val r12: RunnableGraph[(Promise[Option[Int]], Cancellable, Future[Int])] = + RunnableGraph.fromGraph(FlowGraph.create(source, flow, sink)((_, _, _)) { implicit builder => (src, f, dst) => import FlowGraph.Implicits._ src ~> f ~> dst - } + ClosedShape + }) //#flow-mat-combine } diff --git a/akka-docs-dev/rst/scala/code/docs/stream/FlowGraphDocSpec.scala b/akka-docs-dev/rst/scala/code/docs/stream/FlowGraphDocSpec.scala index 0ea408237d..3e30e79bd1 100644 --- a/akka-docs-dev/rst/scala/code/docs/stream/FlowGraphDocSpec.scala +++ b/akka-docs-dev/rst/scala/code/docs/stream/FlowGraphDocSpec.scala @@ -21,7 +21,7 @@ class FlowGraphDocSpec extends AkkaSpec { "build simple graph" in { //format: OFF //#simple-flow-graph - val g = FlowGraph.closed() { implicit builder: FlowGraph.Builder[Unit] => + val g = RunnableGraph.fromGraph(FlowGraph.create() { implicit builder: FlowGraph.Builder[Unit] => import FlowGraph.Implicits._ val in = Source(1 to 10) val out = Sink.ignore @@ -32,8 +32,9 @@ class FlowGraphDocSpec extends AkkaSpec { val f1, f2, f3, f4 = Flow[Int].map(_ + 10) in ~> f1 ~> bcast ~> f2 ~> merge ~> f3 ~> out - bcast ~> f4 ~> merge - } + bcast ~> f4 ~> merge + ClosedShape + }) //#simple-flow-graph //format: ON @@ -42,33 +43,10 @@ class FlowGraphDocSpec extends AkkaSpec { //#simple-graph-run } - "build simple graph without implicits" in { - //#simple-flow-graph-no-implicits - val g = FlowGraph.closed() { builder: FlowGraph.Builder[Unit] => - val in = Source(1 to 10) - val out = Sink.ignore - - val broadcast = builder.add(Broadcast[Int](2)) - val merge = builder.add(Merge[Int](2)) - - val f1 = Flow[Int].map(_ + 10) - val f3 = Flow[Int].map(_.toString) - val f2 = Flow[Int].map(_ + 20) - - builder.addEdge(builder.add(in), broadcast.in) - builder.addEdge(broadcast.out(0), f1, merge.in(0)) - builder.addEdge(broadcast.out(1), f2, merge.in(1)) - builder.addEdge(merge.out, f3, builder.add(out)) - } - //#simple-flow-graph-no-implicits - - g.run() - } - "flow connection errors" in { intercept[IllegalArgumentException] { //#simple-graph - FlowGraph.closed() { implicit builder => + RunnableGraph.fromGraph(FlowGraph.create() { implicit builder => import FlowGraph.Implicits._ val source1 = Source(1 to 10) val source2 = Source(1 to 10) @@ -78,9 +56,10 @@ class FlowGraphDocSpec extends AkkaSpec { source1 ~> zip.in0 source2 ~> zip.in1 // unconnected zip.out (!) => "must have at least 1 outgoing edge" - } + ClosedShape + }) //#simple-graph - }.getMessage should include("unconnected ports: ZipWith2.out") + }.getMessage should include("ZipWith2.out") } "reusing a flow in a graph" in { @@ -95,7 +74,7 @@ class FlowGraphDocSpec extends AkkaSpec { // format: OFF val g = //#flow-graph-reusing-a-flow - FlowGraph.closed(topHeadSink, bottomHeadSink)((_, _)) { implicit builder => + RunnableGraph.fromGraph(FlowGraph.create(topHeadSink, bottomHeadSink)((_, _)) { implicit builder => (topHS, bottomHS) => import FlowGraph.Implicits._ val broadcast = builder.add(Broadcast[Int](2)) @@ -103,7 +82,8 @@ class FlowGraphDocSpec extends AkkaSpec { broadcast.out(0) ~> sharedDoubler ~> topHS.inlet broadcast.out(1) ~> sharedDoubler ~> bottomHS.inlet - } + ClosedShape + }) //#flow-graph-reusing-a-flow // format: ON val (topFuture, bottomFuture) = g.run() @@ -153,7 +133,7 @@ class FlowGraphDocSpec extends AkkaSpec { worker: Flow[In, Out, Any], workerCount: Int): Graph[PriorityWorkerPoolShape[In, Out], Unit] = { - FlowGraph.partial() { implicit b ⇒ + FlowGraph.create() { implicit b ⇒ import FlowGraph.Implicits._ val priorityMerge = b.add(MergePreferred[In](1)) @@ -188,7 +168,7 @@ class FlowGraphDocSpec extends AkkaSpec { val worker1 = Flow[String].map("step 1 " + _) val worker2 = Flow[String].map("step 2 " + _) - FlowGraph.closed() { implicit b => + RunnableGraph.fromGraph(FlowGraph.create() { implicit b => import FlowGraph.Implicits._ val priorityPool1 = b.add(PriorityWorkerPool(worker1, 4)) @@ -201,7 +181,8 @@ class FlowGraphDocSpec extends AkkaSpec { Source(1 to 100).map("one-step, priority " + _) ~> priorityPool2.priorityJobsIn priorityPool2.resultsOut ~> Sink.foreach(println) - }.run() + ClosedShape + }).run() //#flow-graph-components-use //#flow-graph-components-shape2 @@ -223,11 +204,11 @@ class FlowGraphDocSpec extends AkkaSpec { "access to materialized value" in { //#flow-graph-matvalue import FlowGraph.Implicits._ - val foldFlow: Flow[Int, Int, Future[Int]] = Flow(Sink.fold[Int, Int](0)(_ + _)) { + val foldFlow: Flow[Int, Int, Future[Int]] = Flow.fromGraph(FlowGraph.create(Sink.fold[Int, Int](0)(_ + _)) { implicit builder ⇒ fold ⇒ - (fold.inlet, builder.materializedValue.mapAsync(4)(identity).outlet) - } + FlowShape(fold.inlet, builder.materializedValue.mapAsync(4)(identity).outlet) + }) //#flow-graph-matvalue Await.result(Source(1 to 10).via(foldFlow).runWith(Sink.head), 3.seconds) should ===(55) @@ -235,7 +216,7 @@ class FlowGraphDocSpec extends AkkaSpec { //#flow-graph-matvalue-cycle import FlowGraph.Implicits._ // This cannot produce any value: - val cyclicFold: Source[Int, Future[Int]] = Source(Sink.fold[Int, Int](0)(_ + _)) { + val cyclicFold: Source[Int, Future[Int]] = Source.fromGraph(FlowGraph.create(Sink.fold[Int, Int](0)(_ + _)) { implicit builder => fold => // - Fold cannot complete until its upstream mapAsync completes @@ -244,8 +225,8 @@ class FlowGraphDocSpec extends AkkaSpec { // As a result this Source will never emit anything, and its materialited // Future will never complete builder.materializedValue.mapAsync(4)(identity) ~> fold - builder.materializedValue.mapAsync(4)(identity).outlet - } + SourceShape(builder.materializedValue.mapAsync(4)(identity).outlet) + }) //#flow-graph-matvalue-cycle } diff --git a/akka-docs-dev/rst/scala/code/docs/stream/FlowParallelismDocSpec.scala b/akka-docs-dev/rst/scala/code/docs/stream/FlowParallelismDocSpec.scala index a19e939994..4a28737f85 100644 --- a/akka-docs-dev/rst/scala/code/docs/stream/FlowParallelismDocSpec.scala +++ b/akka-docs-dev/rst/scala/code/docs/stream/FlowParallelismDocSpec.scala @@ -1,5 +1,6 @@ package docs.stream +import akka.stream.FlowShape import akka.stream.scaladsl.{ FlowGraph, Merge, Balance, Source, Flow } import akka.stream.testkit.AkkaSpec @@ -37,7 +38,7 @@ class FlowParallelismDocSpec extends AkkaSpec { val fryingPan: Flow[ScoopOfBatter, Pancake, Unit] = Flow[ScoopOfBatter].map { batter => Pancake() } - val pancakeChef: Flow[ScoopOfBatter, Pancake, Unit] = Flow() { implicit builder => + val pancakeChef: Flow[ScoopOfBatter, Pancake, Unit] = Flow.fromGraph(FlowGraph.create() { implicit builder => val dispatchBatter = builder.add(Balance[ScoopOfBatter](2)) val mergePancakes = builder.add(Merge[Pancake](2)) @@ -49,54 +50,57 @@ class FlowParallelismDocSpec extends AkkaSpec { // uses of "fryingPan" mean actually different stages in the graph. dispatchBatter.out(1) ~> fryingPan ~> mergePancakes.in(1) - (dispatchBatter.in, mergePancakes.out) - } + FlowShape(dispatchBatter.in, mergePancakes.out) + }) //#parallelism } "Demonstrate parallelized pipelines" in { //#parallel-pipeline - val pancakeChef: Flow[ScoopOfBatter, Pancake, Unit] = Flow() { implicit builder => + val pancakeChef: Flow[ScoopOfBatter, Pancake, Unit] = + Flow.fromGraph(FlowGraph.create() { implicit builder => - val dispatchBatter = builder.add(Balance[ScoopOfBatter](2)) - val mergePancakes = builder.add(Merge[Pancake](2)) + val dispatchBatter = builder.add(Balance[ScoopOfBatter](2)) + val mergePancakes = builder.add(Merge[Pancake](2)) - // Using two pipelines, having two frying pans each, in total using - // four frying pans - dispatchBatter.out(0) ~> fryingPan1 ~> fryingPan2 ~> mergePancakes.in(0) - dispatchBatter.out(1) ~> fryingPan1 ~> fryingPan2 ~> mergePancakes.in(1) + // Using two pipelines, having two frying pans each, in total using + // four frying pans + dispatchBatter.out(0) ~> fryingPan1 ~> fryingPan2 ~> mergePancakes.in(0) + dispatchBatter.out(1) ~> fryingPan1 ~> fryingPan2 ~> mergePancakes.in(1) - (dispatchBatter.in, mergePancakes.out) - } + FlowShape(dispatchBatter.in, mergePancakes.out) + }) //#parallel-pipeline } "Demonstrate pipelined parallel processing" in { //#pipelined-parallel - val pancakeChefs1: Flow[ScoopOfBatter, HalfCookedPancake, Unit] = Flow() { implicit builder => - val dispatchBatter = builder.add(Balance[ScoopOfBatter](2)) - val mergeHalfPancakes = builder.add(Merge[HalfCookedPancake](2)) + val pancakeChefs1: Flow[ScoopOfBatter, HalfCookedPancake, Unit] = + Flow.fromGraph(FlowGraph.create() { implicit builder => + val dispatchBatter = builder.add(Balance[ScoopOfBatter](2)) + val mergeHalfPancakes = builder.add(Merge[HalfCookedPancake](2)) - // Two chefs work with one frying pan for each, half-frying the pancakes then putting - // them into a common pool - dispatchBatter.out(0) ~> fryingPan1 ~> mergeHalfPancakes.in(0) - dispatchBatter.out(1) ~> fryingPan1 ~> mergeHalfPancakes.in(1) + // Two chefs work with one frying pan for each, half-frying the pancakes then putting + // them into a common pool + dispatchBatter.out(0) ~> fryingPan1 ~> mergeHalfPancakes.in(0) + dispatchBatter.out(1) ~> fryingPan1 ~> mergeHalfPancakes.in(1) - (dispatchBatter.in, mergeHalfPancakes.out) - } + FlowShape(dispatchBatter.in, mergeHalfPancakes.out) + }) - val pancakeChefs2: Flow[HalfCookedPancake, Pancake, Unit] = Flow() { implicit builder => - val dispatchHalfPancakes = builder.add(Balance[HalfCookedPancake](2)) - val mergePancakes = builder.add(Merge[Pancake](2)) + val pancakeChefs2: Flow[HalfCookedPancake, Pancake, Unit] = + Flow.fromGraph(FlowGraph.create() { implicit builder => + val dispatchHalfPancakes = builder.add(Balance[HalfCookedPancake](2)) + val mergePancakes = builder.add(Merge[Pancake](2)) - // Two chefs work with one frying pan for each, finishing the pancakes then putting - // them into a common pool - dispatchHalfPancakes.out(0) ~> fryingPan2 ~> mergePancakes.in(0) - dispatchHalfPancakes.out(1) ~> fryingPan2 ~> mergePancakes.in(1) + // Two chefs work with one frying pan for each, finishing the pancakes then putting + // them into a common pool + dispatchHalfPancakes.out(0) ~> fryingPan2 ~> mergePancakes.in(0) + dispatchHalfPancakes.out(1) ~> fryingPan2 ~> mergePancakes.in(1) - (dispatchHalfPancakes.in, mergePancakes.out) - } + FlowShape(dispatchHalfPancakes.in, mergePancakes.out) + }) val kitchen: Flow[ScoopOfBatter, Pancake, Unit] = pancakeChefs1.via(pancakeChefs2) //#pipelined-parallel diff --git a/akka-docs-dev/rst/scala/code/docs/stream/GraphCyclesSpec.scala b/akka-docs-dev/rst/scala/code/docs/stream/GraphCyclesSpec.scala index c9a307f74a..099c948ac5 100644 --- a/akka-docs-dev/rst/scala/code/docs/stream/GraphCyclesSpec.scala +++ b/akka-docs-dev/rst/scala/code/docs/stream/GraphCyclesSpec.scala @@ -1,6 +1,6 @@ package docs.stream -import akka.stream.{ OverflowStrategy, ActorMaterializer } +import akka.stream.{ ClosedShape, OverflowStrategy, ActorMaterializer } import akka.stream.scaladsl._ import akka.stream.testkit.AkkaSpec @@ -16,7 +16,7 @@ class GraphCyclesSpec extends AkkaSpec { // format: OFF //#deadlocked // WARNING! The graph below deadlocks! - FlowGraph.closed() { implicit b => + RunnableGraph.fromGraph(FlowGraph.create() { implicit b => import FlowGraph.Implicits._ val merge = b.add(Merge[Int](2)) @@ -24,7 +24,8 @@ class GraphCyclesSpec extends AkkaSpec { source ~> merge ~> Flow[Int].map { s => println(s); s } ~> bcast ~> Sink.ignore merge <~ bcast - } + ClosedShape + }) //#deadlocked // format: ON } @@ -33,7 +34,7 @@ class GraphCyclesSpec extends AkkaSpec { // format: OFF //#unfair // WARNING! The graph below stops consuming from "source" after a few steps - FlowGraph.closed() { implicit b => + RunnableGraph.fromGraph(FlowGraph.create() { implicit b => import FlowGraph.Implicits._ val merge = b.add(MergePreferred[Int](1)) @@ -41,7 +42,8 @@ class GraphCyclesSpec extends AkkaSpec { source ~> merge ~> Flow[Int].map { s => println(s); s } ~> bcast ~> Sink.ignore merge.preferred <~ bcast - } + ClosedShape + }) //#unfair // format: ON } @@ -49,7 +51,7 @@ class GraphCyclesSpec extends AkkaSpec { "include a dropping cycle" in { // format: OFF //#dropping - FlowGraph.closed() { implicit b => + RunnableGraph.fromGraph(FlowGraph.create() { implicit b => import FlowGraph.Implicits._ val merge = b.add(Merge[Int](2)) @@ -57,7 +59,8 @@ class GraphCyclesSpec extends AkkaSpec { source ~> merge ~> Flow[Int].map { s => println(s); s } ~> bcast ~> Sink.ignore merge <~ Flow[Int].buffer(10, OverflowStrategy.dropHead) <~ bcast - } + ClosedShape + }) //#dropping // format: ON } @@ -66,7 +69,7 @@ class GraphCyclesSpec extends AkkaSpec { // format: OFF //#zipping-dead // WARNING! The graph below never processes any elements - FlowGraph.closed() { implicit b => + RunnableGraph.fromGraph(FlowGraph.create() { implicit b => import FlowGraph.Implicits._ val zip = b.add(ZipWith[Int, Int, Int]((left, right) => right)) @@ -75,7 +78,8 @@ class GraphCyclesSpec extends AkkaSpec { source ~> zip.in0 zip.out.map { s => println(s); s } ~> bcast ~> Sink.ignore zip.in1 <~ bcast - } + ClosedShape + }) //#zipping-dead // format: ON } @@ -83,7 +87,7 @@ class GraphCyclesSpec extends AkkaSpec { "include a live zipping cycle" in { // format: OFF //#zipping-live - FlowGraph.closed() { implicit b => + RunnableGraph.fromGraph(FlowGraph.create() { implicit b => import FlowGraph.Implicits._ val zip = b.add(ZipWith((left: Int, right: Int) => left)) @@ -95,7 +99,8 @@ class GraphCyclesSpec extends AkkaSpec { zip.out.map { s => println(s); s } ~> bcast ~> Sink.ignore zip.in1 <~ concat <~ start concat <~ bcast - } + ClosedShape + }) //#zipping-live // format: ON } diff --git a/akka-docs-dev/rst/scala/code/docs/stream/StreamBuffersRateSpec.scala b/akka-docs-dev/rst/scala/code/docs/stream/StreamBuffersRateSpec.scala index c6a467f607..b9979f880f 100644 --- a/akka-docs-dev/rst/scala/code/docs/stream/StreamBuffersRateSpec.scala +++ b/akka-docs-dev/rst/scala/code/docs/stream/StreamBuffersRateSpec.scala @@ -1,9 +1,8 @@ package docs.stream -import akka.stream.{ OverflowStrategy, ActorMaterializerSettings, ActorMaterializer } +import akka.stream._ import akka.stream.scaladsl._ import akka.stream.testkit.AkkaSpec -import akka.stream.Attributes class StreamBuffersRateSpec extends AkkaSpec { implicit val mat = ActorMaterializer() @@ -40,7 +39,7 @@ class StreamBuffersRateSpec extends AkkaSpec { import scala.concurrent.duration._ case class Tick() - FlowGraph.closed() { implicit b => + RunnableGraph.fromGraph(FlowGraph.create() { implicit b => import FlowGraph.Implicits._ val zipper = b.add(ZipWith[Tick, Int, Int]((tick, count) => count)) @@ -51,7 +50,8 @@ class StreamBuffersRateSpec extends AkkaSpec { .conflate(seed = (_) => 1)((count, _) => count + 1) ~> zipper.in1 zipper.out ~> Sink.foreach(println) - } + ClosedShape + }) //#buffering-abstraction-leak } diff --git a/akka-docs-dev/rst/scala/code/docs/stream/StreamPartialFlowGraphDocSpec.scala b/akka-docs-dev/rst/scala/code/docs/stream/StreamPartialFlowGraphDocSpec.scala index 5e228210d2..f85562234f 100644 --- a/akka-docs-dev/rst/scala/code/docs/stream/StreamPartialFlowGraphDocSpec.scala +++ b/akka-docs-dev/rst/scala/code/docs/stream/StreamPartialFlowGraphDocSpec.scala @@ -19,7 +19,7 @@ class StreamPartialFlowGraphDocSpec extends AkkaSpec { "build with open ports" in { //#simple-partial-flow-graph - val pickMaxOfThree = FlowGraph.partial() { implicit b => + val pickMaxOfThree = FlowGraph.create() { implicit b => import FlowGraph.Implicits._ val zip1 = b.add(ZipWith[Int, Int, Int](math.max _)) @@ -31,7 +31,7 @@ class StreamPartialFlowGraphDocSpec extends AkkaSpec { val resultSink = Sink.head[Int] - val g = FlowGraph.closed(resultSink) { implicit b => + val g = RunnableGraph.fromGraph(FlowGraph.create(resultSink) { implicit b => sink => import FlowGraph.Implicits._ @@ -42,7 +42,8 @@ class StreamPartialFlowGraphDocSpec extends AkkaSpec { Source.single(2) ~> pm3.in(1) Source.single(3) ~> pm3.in(2) pm3.out ~> sink.inlet - } + ClosedShape + }) val max: Future[Int] = g.run() Await.result(max, 300.millis) should equal(3) @@ -51,7 +52,7 @@ class StreamPartialFlowGraphDocSpec extends AkkaSpec { "build source from partial flow graph" in { //#source-from-partial-flow-graph - val pairs = Source() { implicit b => + val pairs = Source.fromGraph(FlowGraph.create() { implicit b => import FlowGraph.Implicits._ // prepare graph elements @@ -63,8 +64,8 @@ class StreamPartialFlowGraphDocSpec extends AkkaSpec { ints.filter(_ % 2 == 0) ~> zip.in1 // expose port - zip.out - } + SourceShape(zip.out) + }) val firstPair: Future[(Int, Int)] = pairs.runWith(Sink.head) //#source-from-partial-flow-graph @@ -73,20 +74,21 @@ class StreamPartialFlowGraphDocSpec extends AkkaSpec { "build flow from partial flow graph" in { //#flow-from-partial-flow-graph - val pairUpWithToString = Flow() { implicit b => - import FlowGraph.Implicits._ + val pairUpWithToString = + Flow.fromGraph(FlowGraph.create() { implicit b => + import FlowGraph.Implicits._ - // prepare graph elements - val broadcast = b.add(Broadcast[Int](2)) - val zip = b.add(Zip[Int, String]()) + // prepare graph elements + val broadcast = b.add(Broadcast[Int](2)) + val zip = b.add(Zip[Int, String]()) - // connect the graph - broadcast.out(0).map(identity) ~> zip.in0 - broadcast.out(1).map(_.toString) ~> zip.in1 + // connect the graph + broadcast.out(0).map(identity) ~> zip.in0 + broadcast.out(1).map(_.toString) ~> zip.in1 - // expose ports - (broadcast.in, zip.out) - } + // expose ports + FlowShape(broadcast.in, zip.out) + }) //#flow-from-partial-flow-graph diff --git a/akka-docs-dev/rst/scala/code/docs/stream/TwitterStreamQuickstartDocSpec.scala b/akka-docs-dev/rst/scala/code/docs/stream/TwitterStreamQuickstartDocSpec.scala index b69ebd93fa..495b5711f9 100644 --- a/akka-docs-dev/rst/scala/code/docs/stream/TwitterStreamQuickstartDocSpec.scala +++ b/akka-docs-dev/rst/scala/code/docs/stream/TwitterStreamQuickstartDocSpec.scala @@ -6,8 +6,7 @@ package docs.stream //#imports import akka.actor.ActorSystem -import akka.stream.ActorMaterializer -import akka.stream.OverflowStrategy +import akka.stream.{ ClosedShape, ActorMaterializer, OverflowStrategy } import akka.stream.scaladsl._ import scala.concurrent.Await @@ -119,14 +118,15 @@ class TwitterStreamQuickstartDocSpec extends AkkaSpec { // format: OFF //#flow-graph-broadcast - val g = FlowGraph.closed() { implicit b => + val g = RunnableGraph.fromGraph(FlowGraph.create() { implicit b => import FlowGraph.Implicits._ val bcast = b.add(Broadcast[Tweet](2)) tweets ~> bcast.in bcast.out(0) ~> Flow[Tweet].map(_.author) ~> writeAuthors bcast.out(1) ~> Flow[Tweet].mapConcat(_.hashtags.toList) ~> writeHashtags - } + ClosedShape + }) g.run() //#flow-graph-broadcast // format: ON diff --git a/akka-docs-dev/rst/scala/code/docs/stream/cookbook/RecipeDroppyBroadcast.scala b/akka-docs-dev/rst/scala/code/docs/stream/cookbook/RecipeDroppyBroadcast.scala index bf64e862e9..434edb08ee 100644 --- a/akka-docs-dev/rst/scala/code/docs/stream/cookbook/RecipeDroppyBroadcast.scala +++ b/akka-docs-dev/rst/scala/code/docs/stream/cookbook/RecipeDroppyBroadcast.scala @@ -1,6 +1,6 @@ package docs.stream.cookbook -import akka.stream.OverflowStrategy +import akka.stream.{ ClosedShape, OverflowStrategy } import akka.stream.scaladsl._ import akka.stream.testkit._ @@ -24,7 +24,7 @@ class RecipeDroppyBroadcast extends RecipeSpec { val mySink3 = Sink(sub3) //#droppy-bcast - val graph = FlowGraph.closed(mySink1, mySink2, mySink3)((_, _, _)) { implicit b => + val graph = RunnableGraph.fromGraph(FlowGraph.create(mySink1, mySink2, mySink3)((_, _, _)) { implicit b => (sink1, sink2, sink3) => import FlowGraph.Implicits._ @@ -34,7 +34,8 @@ class RecipeDroppyBroadcast extends RecipeSpec { bcast.buffer(10, OverflowStrategy.dropHead) ~> sink1 bcast.buffer(10, OverflowStrategy.dropHead) ~> sink2 bcast.buffer(10, OverflowStrategy.dropHead) ~> sink3 - } + ClosedShape + }) //#droppy-bcast graph.run() diff --git a/akka-docs-dev/rst/scala/code/docs/stream/cookbook/RecipeGlobalRateLimit.scala b/akka-docs-dev/rst/scala/code/docs/stream/cookbook/RecipeGlobalRateLimit.scala index b6d85ae738..fe4b3dba92 100644 --- a/akka-docs-dev/rst/scala/code/docs/stream/cookbook/RecipeGlobalRateLimit.scala +++ b/akka-docs-dev/rst/scala/code/docs/stream/cookbook/RecipeGlobalRateLimit.scala @@ -2,6 +2,7 @@ package docs.stream.cookbook import akka.actor.{ Props, ActorRef, Actor } import akka.actor.Actor.Receive +import akka.stream.ClosedShape import akka.stream.scaladsl._ import akka.stream.testkit._ @@ -98,12 +99,13 @@ class RecipeGlobalRateLimit extends RecipeSpec { val probe = TestSubscriber.manualProbe[String]() - FlowGraph.closed() { implicit b => + RunnableGraph.fromGraph(FlowGraph.create() { implicit b => import FlowGraph.Implicits._ val merge = b.add(Merge[String](2)) source1 ~> merge ~> Sink(probe) source2 ~> merge - }.run() + ClosedShape + }).run() probe.expectSubscription().request(1000) diff --git a/akka-docs-dev/rst/scala/code/docs/stream/cookbook/RecipeKeepAlive.scala b/akka-docs-dev/rst/scala/code/docs/stream/cookbook/RecipeKeepAlive.scala index 43391883c1..6fe5589576 100644 --- a/akka-docs-dev/rst/scala/code/docs/stream/cookbook/RecipeKeepAlive.scala +++ b/akka-docs-dev/rst/scala/code/docs/stream/cookbook/RecipeKeepAlive.scala @@ -1,5 +1,6 @@ package docs.stream.cookbook +import akka.stream.ClosedShape import akka.stream.scaladsl._ import akka.stream.testkit._ import akka.util.ByteString @@ -25,14 +26,15 @@ class RecipeKeepAlive extends RecipeSpec { val tickToKeepAlivePacket: Flow[Tick, ByteString, Unit] = Flow[Tick] .conflate(seed = (tick) => keepaliveMessage)((msg, newTick) => msg) - val graph = FlowGraph.closed() { implicit builder => + val graph = RunnableGraph.fromGraph(FlowGraph.create() { implicit builder => import FlowGraph.Implicits._ val unfairMerge = builder.add(MergePreferred[ByteString](1)) // If data is available then no keepalive is injected dataStream ~> unfairMerge.preferred ticks ~> tickToKeepAlivePacket ~> unfairMerge ~> sink - } + ClosedShape + }) //#inject-keepalive graph.run() diff --git a/akka-docs-dev/rst/scala/code/docs/stream/cookbook/RecipeManualTrigger.scala b/akka-docs-dev/rst/scala/code/docs/stream/cookbook/RecipeManualTrigger.scala index e3733f8e5d..33771f9a38 100644 --- a/akka-docs-dev/rst/scala/code/docs/stream/cookbook/RecipeManualTrigger.scala +++ b/akka-docs-dev/rst/scala/code/docs/stream/cookbook/RecipeManualTrigger.scala @@ -1,5 +1,6 @@ package docs.stream.cookbook +import akka.stream.ClosedShape import akka.stream.scaladsl._ import akka.stream.testkit._ import scala.concurrent.duration._ @@ -17,13 +18,14 @@ class RecipeManualTrigger extends RecipeSpec { val sink = Sink(sub) //#manually-triggered-stream - val graph = FlowGraph.closed() { implicit builder => + val graph = RunnableGraph.fromGraph(FlowGraph.create() { implicit builder => import FlowGraph.Implicits._ val zip = builder.add(Zip[Message, Trigger]()) elements ~> zip.in0 triggerSource ~> zip.in1 zip.out ~> Flow[(Message, Trigger)].map { case (msg, trigger) => msg } ~> sink - } + ClosedShape + }) //#manually-triggered-stream graph.run() @@ -55,14 +57,15 @@ class RecipeManualTrigger extends RecipeSpec { val sink = Sink(sub) //#manually-triggered-stream-zipwith - val graph = FlowGraph.closed() { implicit builder => + val graph = RunnableGraph.fromGraph(FlowGraph.create() { implicit builder => import FlowGraph.Implicits._ val zip = builder.add(ZipWith((msg: Message, trigger: Trigger) => msg)) elements ~> zip.in0 triggerSource ~> zip.in1 zip.out ~> sink - } + ClosedShape + }) //#manually-triggered-stream-zipwith graph.run() diff --git a/akka-docs-dev/rst/scala/code/docs/stream/cookbook/RecipeWorkerPool.scala b/akka-docs-dev/rst/scala/code/docs/stream/cookbook/RecipeWorkerPool.scala index 529e1ddb4b..7aaf2a663f 100644 --- a/akka-docs-dev/rst/scala/code/docs/stream/cookbook/RecipeWorkerPool.scala +++ b/akka-docs-dev/rst/scala/code/docs/stream/cookbook/RecipeWorkerPool.scala @@ -1,5 +1,6 @@ package docs.stream.cookbook +import akka.stream.FlowShape import akka.stream.scaladsl._ import akka.testkit.TestProbe @@ -20,7 +21,7 @@ class RecipeWorkerPool extends RecipeSpec { def balancer[In, Out](worker: Flow[In, Out, Any], workerCount: Int): Flow[In, Out, Unit] = { import FlowGraph.Implicits._ - Flow() { implicit b => + Flow.fromGraph(FlowGraph.create() { implicit b => val balancer = b.add(Balance[In](workerCount, waitForAllDownstreams = true)) val merge = b.add(Merge[Out](workerCount)) @@ -30,8 +31,8 @@ class RecipeWorkerPool extends RecipeSpec { balancer ~> worker ~> merge } - (balancer.in, merge.out) - } + FlowShape(balancer.in, merge.out) + }) } val processedJobs: Source[Result, Unit] = myJobs.via(balancer(worker, 3)) diff --git a/akka-docs-dev/rst/scala/code/docs/stream/io/StreamTcpDocSpec.scala b/akka-docs-dev/rst/scala/code/docs/stream/io/StreamTcpDocSpec.scala index b9bd4ee11e..cc77e7631a 100644 --- a/akka-docs-dev/rst/scala/code/docs/stream/io/StreamTcpDocSpec.scala +++ b/akka-docs-dev/rst/scala/code/docs/stream/io/StreamTcpDocSpec.scala @@ -73,7 +73,7 @@ class StreamTcpDocSpec extends AkkaSpec { connections runForeach { connection => - val serverLogic = Flow() { implicit b => + val serverLogic = Flow.fromGraph(FlowGraph.create() { implicit b => import FlowGraph.Implicits._ // server logic, parses incoming commands @@ -109,8 +109,8 @@ class StreamTcpDocSpec extends AkkaSpec { // then we continue using the echo-logic Flow echo.outlet ~> concat.in(1) - (echo.inlet, concat.out) - } + FlowShape(echo.inlet, concat.out) + }) connection.handleWith(serverLogic) } diff --git a/akka-docs-dev/rst/scala/stream-composition.rst b/akka-docs-dev/rst/scala/stream-composition.rst index 4172e4878a..cf84512f6d 100644 --- a/akka-docs-dev/rst/scala/stream-composition.rst +++ b/akka-docs-dev/rst/scala/stream-composition.rst @@ -126,8 +126,8 @@ As a first example, let's look at a more complex layout: The diagram shows a :class:`RunnableGraph` (remember, if there are no unwired ports, the graph is closed, and therefore can be materialized) that encapsulates a non-trivial stream processing network. It contains fan-in, fan-out stages, -directed and non-directed cycles. The ``closed()`` method of the :class:`FlowGraph` object allows the creation of a -general closed graph. For example the network on the diagram can be realized like this: +directed and non-directed cycles. The ``runnable()`` method of the :class:`FlowGraph` object allows the creation of a +general, closed, and runnable graph. For example the network on the diagram can be realized like this: .. includecode:: code/docs/stream/CompositionDocSpec.scala#complex-graph @@ -141,7 +141,7 @@ explicitly, and it is not necessary to import our linear stages via ``add()``, s Similar to the case in the first section, so far we have not considered modularity. We created a complex graph, but the layout is flat, not modularized. We will modify our example, and create a reusable component with the graph DSL. -The way to do it is to use the ``partial()`` factory method on :class:`FlowGraph`. If we remove the sources and sinks +The way to do it is to use the ``create()`` factory method on :class:`FlowGraph`. If we remove the sources and sinks from the previous example, what remains is a partial graph: | @@ -284,7 +284,7 @@ Attributes ---------- We have seen that we can use ``named()`` to introduce a nesting level in the fluid DSL (and also explicit nesting by using -``partial()`` from :class:`FlowGraph`). Apart from having the effect of adding a nesting level, ``named()`` is actually +``create()`` from :class:`FlowGraph`). Apart from having the effect of adding a nesting level, ``named()`` is actually a shorthand for calling ``withAttributes(Attributes.name("someName"))``. Attributes provide a way to fine-tune certain aspects of the materialized running entity. For example buffer sizes can be controlled via attributes (see :ref:`stream-buffers-scala`). When it comes to hierarchic composition, attributes are inherited by nested modules, diff --git a/akka-docs-dev/rst/scala/stream-graphs.rst b/akka-docs-dev/rst/scala/stream-graphs.rst index 3f91ef183a..95e665e53d 100644 --- a/akka-docs-dev/rst/scala/stream-graphs.rst +++ b/akka-docs-dev/rst/scala/stream-graphs.rst @@ -60,9 +60,6 @@ will be inferred. Notice the ``import FlowGraph.Implicits._`` which brings into scope the ``~>`` operator (read as "edge", "via" or "to") and its inverted counterpart ``<~`` (for noting down flows in the opposite direction where appropriate). -It is also possible to construct graphs without the ``~>`` operator in case you prefer to use the graph builder explicitly: - -.. includecode:: code/docs/stream/FlowGraphDocSpec.scala#simple-flow-graph-no-implicits By looking at the snippets above, it should be apparent that the :class:`FlowGraph.Builder` object is *mutable*. It is used (implicitly) by the ``~>`` operator, also making it a mutable operation as well. diff --git a/akka-http-core/src/main/scala/akka/http/impl/engine/client/OutgoingConnectionBlueprint.scala b/akka-http-core/src/main/scala/akka/http/impl/engine/client/OutgoingConnectionBlueprint.scala index 40d2b6c51b..8abbe7d9b2 100644 --- a/akka-http-core/src/main/scala/akka/http/impl/engine/client/OutgoingConnectionBlueprint.scala +++ b/akka-http-core/src/main/scala/akka/http/impl/engine/client/OutgoingConnectionBlueprint.scala @@ -76,7 +76,7 @@ private[http] object OutgoingConnectionBlueprint { case (MessageStartError(_, info), _) ⇒ throw IllegalResponseException(info) } - val core = BidiFlow() { implicit b ⇒ + val core = BidiFlow.fromGraph(FlowGraph.create() { implicit b ⇒ import FlowGraph.Implicits._ val methodBypassFanout = b.add(Broadcast[HttpRequest](2, eagerCancel = true)) val responseParsingMerge = b.add(new ResponseParsingMerge(rootParser)) @@ -103,7 +103,7 @@ private[http] object OutgoingConnectionBlueprint { wrapTls.outlet, unwrapTls.inlet, terminationFanout.out(1)) - } + }) One2OneBidiFlow[HttpRequest, HttpResponse](-1) atop core } diff --git a/akka-http-core/src/main/scala/akka/http/impl/engine/client/PoolConductor.scala b/akka-http-core/src/main/scala/akka/http/impl/engine/client/PoolConductor.scala index e33697022a..a5d5a9dba8 100644 --- a/akka-http-core/src/main/scala/akka/http/impl/engine/client/PoolConductor.scala +++ b/akka-http-core/src/main/scala/akka/http/impl/engine/client/PoolConductor.scala @@ -65,7 +65,7 @@ private object PoolConductor { */ def apply(slotCount: Int, maxRetries: Int, pipeliningLimit: Int, log: LoggingAdapter): Graph[Ports, Any] = - FlowGraph.partial() { implicit b ⇒ + FlowGraph.create() { implicit b ⇒ import FlowGraph.Implicits._ val retryMerge = b.add(MergePreferred[RequestContext](1, eagerClose = true)) diff --git a/akka-http-core/src/main/scala/akka/http/impl/engine/client/PoolFlow.scala b/akka-http-core/src/main/scala/akka/http/impl/engine/client/PoolFlow.scala index 9e77ffc604..a6d00908a7 100644 --- a/akka-http-core/src/main/scala/akka/http/impl/engine/client/PoolFlow.scala +++ b/akka-http-core/src/main/scala/akka/http/impl/engine/client/PoolFlow.scala @@ -11,7 +11,7 @@ import scala.concurrent.{ Promise, Future } import scala.util.Try import akka.event.LoggingAdapter import akka.actor._ -import akka.stream.Materializer +import akka.stream.{ FlowShape, Materializer } import akka.stream.scaladsl._ import akka.http.scaladsl.model._ import akka.http.scaladsl.Http @@ -70,7 +70,7 @@ private object PoolFlow { def apply(connectionFlow: Flow[HttpRequest, HttpResponse, Future[Http.OutgoingConnection]], remoteAddress: InetSocketAddress, settings: ConnectionPoolSettings, log: LoggingAdapter)( implicit system: ActorSystem, fm: Materializer): Flow[RequestContext, ResponseContext, Unit] = - Flow() { implicit b ⇒ + Flow.fromGraph(FlowGraph.create[FlowShape[RequestContext, ResponseContext]]() { implicit b ⇒ import settings._ import FlowGraph.Implicits._ @@ -87,6 +87,6 @@ private object PoolFlow { slot.out0 ~> responseMerge.in(ix) slot.out1 ~> slotEventMerge.in(ix) } - (conductor.requestIn, responseMerge.out) - } + FlowShape(conductor.requestIn, responseMerge.out) + }) } diff --git a/akka-http-core/src/main/scala/akka/http/impl/engine/client/PoolSlot.scala b/akka-http-core/src/main/scala/akka/http/impl/engine/client/PoolSlot.scala index 36affd6afa..c28331fd5c 100644 --- a/akka-http-core/src/main/scala/akka/http/impl/engine/client/PoolSlot.scala +++ b/akka-http-core/src/main/scala/akka/http/impl/engine/client/PoolSlot.scala @@ -53,7 +53,7 @@ private object PoolSlot { remoteAddress: InetSocketAddress, // TODO: remove after #16168 is cleared settings: ConnectionPoolSettings)(implicit system: ActorSystem, fm: Materializer): Graph[FanOutShape2[RequestContext, ResponseContext, RawSlotEvent], Any] = - FlowGraph.partial() { implicit b ⇒ + FlowGraph.create() { implicit b ⇒ import FlowGraph.Implicits._ val slotProcessor = b.add { diff --git a/akka-http-core/src/main/scala/akka/http/impl/engine/rendering/RenderSupport.scala b/akka-http-core/src/main/scala/akka/http/impl/engine/rendering/RenderSupport.scala index e8730cc0d4..5b1b3c1dc2 100644 --- a/akka-http-core/src/main/scala/akka/http/impl/engine/rendering/RenderSupport.scala +++ b/akka-http-core/src/main/scala/akka/http/impl/engine/rendering/RenderSupport.scala @@ -5,6 +5,7 @@ package akka.http.impl.engine.rendering import akka.parboiled2.CharUtils +import akka.stream.SourceShape import akka.util.ByteString import akka.event.LoggingAdapter import akka.stream.scaladsl._ @@ -31,12 +32,12 @@ private object RenderSupport { val defaultLastChunkBytes: ByteString = renderChunk(HttpEntity.LastChunk) def CancelSecond[T, Mat](first: Source[T, Mat], second: Source[T, Any]): Source[T, Mat] = { - Source(first) { implicit b ⇒ + Source.fromGraph(FlowGraph.create(first) { implicit b ⇒ frst ⇒ import FlowGraph.Implicits._ second ~> Sink.cancelled - frst.outlet - } + SourceShape(frst.outlet) + }) } def renderEntityContentType(r: Rendering, entity: HttpEntity) = diff --git a/akka-http-core/src/main/scala/akka/http/impl/engine/server/HttpServerBluePrint.scala b/akka-http-core/src/main/scala/akka/http/impl/engine/server/HttpServerBluePrint.scala index c35e60317d..efad72e9e7 100644 --- a/akka-http-core/src/main/scala/akka/http/impl/engine/server/HttpServerBluePrint.scala +++ b/akka-http-core/src/main/scala/akka/http/impl/engine/server/HttpServerBluePrint.scala @@ -116,7 +116,7 @@ private[http] object HttpServerBluePrint { .flatten(FlattenStrategy.concat) .via(Flow[ResponseRenderingOutput].transform(() ⇒ errorLogger(log, "Outgoing response stream error")).named("errorLogger")) - BidiFlow(requestParsingFlow, rendererPipeline, oneHundredContinueSource)((_, _, _) ⇒ ()) { implicit b ⇒ + BidiFlow.fromGraph(FlowGraph.create(requestParsingFlow, rendererPipeline, oneHundredContinueSource)((_, _, _) ⇒ ()) { implicit b ⇒ (requestParsing, renderer, oneHundreds) ⇒ import FlowGraph.Implicits._ @@ -172,7 +172,7 @@ private[http] object HttpServerBluePrint { wrapTls.outlet, unwrapTls.inlet, requestsIn) - } + }) } class BypassMerge(settings: ServerSettings, log: LoggingAdapter) @@ -319,7 +319,7 @@ private[http] object HttpServerBluePrint { val sink = StreamUtils.oneTimePublisherSink[FrameEvent](sinkCell, "frameHandler.in") val source = StreamUtils.oneTimeSubscriberSource[FrameEvent](sourceCell, "frameHandler.out") - val flow = Websocket.framing.join(Flow.wrap(sink, source)(Keep.none)) + val flow = Websocket.framing.join(Flow.fromSinkAndSourceMat(sink, source)(Keep.none)) new WebsocketSetup { def websocketFlow: Flow[ByteString, ByteString, Any] = flow diff --git a/akka-http-core/src/main/scala/akka/http/impl/engine/ws/Masking.scala b/akka-http-core/src/main/scala/akka/http/impl/engine/ws/Masking.scala index 084d538771..9b618b5b10 100644 --- a/akka-http-core/src/main/scala/akka/http/impl/engine/ws/Masking.scala +++ b/akka-http-core/src/main/scala/akka/http/impl/engine/ws/Masking.scala @@ -16,7 +16,7 @@ import akka.stream.stage.{ SyncDirective, Context, StatefulStage } */ private[http] object Masking { def apply(serverSide: Boolean, maskRandom: () ⇒ Random): BidiFlow[ /* net in */ FrameEvent, /* app out */ FrameEvent, /* app in */ FrameEvent, /* net out */ FrameEvent, Unit] = - BidiFlow.wrap(unmaskIf(serverSide), maskIf(!serverSide, maskRandom))(Keep.none) + BidiFlow.fromFlowsMat(unmaskIf(serverSide), maskIf(!serverSide, maskRandom))(Keep.none) def maskIf(condition: Boolean, maskRandom: () ⇒ Random): Flow[FrameEvent, FrameEvent, Unit] = if (condition) Flow[FrameEvent].transform(() ⇒ new Masking(maskRandom())) // new random per materialization diff --git a/akka-http-core/src/main/scala/akka/http/impl/engine/ws/Websocket.scala b/akka-http-core/src/main/scala/akka/http/impl/engine/ws/Websocket.scala index 3f74bf029f..ca313bfbb8 100644 --- a/akka-http-core/src/main/scala/akka/http/impl/engine/ws/Websocket.scala +++ b/akka-http-core/src/main/scala/akka/http/impl/engine/ws/Websocket.scala @@ -39,7 +39,7 @@ private[http] object Websocket { /** The lowest layer that implements the binary protocol */ def framing: BidiFlow[ByteString, FrameEvent, FrameEvent, ByteString, Unit] = - BidiFlow.wrap( + BidiFlow.fromFlowsMat( Flow[ByteString].transform(() ⇒ new FrameEventParser), Flow[FrameEvent].transform(() ⇒ new FrameEventRenderer))(Keep.none) .named("ws-framing") @@ -56,7 +56,7 @@ private[http] object Websocket { def frameHandling(serverSide: Boolean = true, closeTimeout: FiniteDuration, log: LoggingAdapter): BidiFlow[FrameEvent, FrameHandler.Output, FrameOutHandler.Input, FrameStart, Unit] = - BidiFlow.wrap( + BidiFlow.fromFlowsMat( FrameHandler.create(server = serverSide), FrameOutHandler.create(serverSide, closeTimeout, log))(Keep.none) .named("ws-frame-handling") @@ -121,7 +121,7 @@ private[http] object Websocket { MessageToFrameRenderer.create(serverSide) .named("ws-render-messages") - BidiFlow() { implicit b ⇒ + BidiFlow.fromGraph(FlowGraph.create() { implicit b ⇒ import FlowGraph.Implicits._ val split = b.add(BypassRouter) @@ -146,7 +146,7 @@ private[http] object Websocket { messagePreparation.outlet, messageRendering.inlet, merge.out) - }.named("ws-message-api") + }.named("ws-message-api")) } private object BypassRouter extends GraphStage[FanOutShape2[Output, BypassEvent, MessagePart]] { diff --git a/akka-http-core/src/main/scala/akka/http/impl/engine/ws/WebsocketClientBlueprint.scala b/akka-http-core/src/main/scala/akka/http/impl/engine/ws/WebsocketClientBlueprint.scala index b694829f15..02f7e34fd4 100644 --- a/akka-http-core/src/main/scala/akka/http/impl/engine/ws/WebsocketClientBlueprint.scala +++ b/akka-http-core/src/main/scala/akka/http/impl/engine/ws/WebsocketClientBlueprint.scala @@ -109,7 +109,7 @@ object WebsocketClientBlueprint { } } - BidiFlow() { implicit b ⇒ + BidiFlow.fromGraph(FlowGraph.create() { implicit b ⇒ import FlowGraph.Implicits._ val networkIn = b.add(Flow[ByteString].transform(() ⇒ new UpgradeStage)) @@ -126,11 +126,11 @@ object WebsocketClientBlueprint { networkIn.outlet, wsIn.inlet, httpRequestBytesAndThenWSBytes.out) - } mapMaterializedValue (_ ⇒ result.future) + }) mapMaterializedValue (_ ⇒ result.future) } def simpleTls: BidiFlow[SslTlsInbound, ByteString, ByteString, SendBytes, Unit] = - BidiFlow.wrap( + BidiFlow.fromFlowsMat( Flow[SslTlsInbound].collect { case SessionBytes(_, bytes) ⇒ bytes }, Flow[ByteString].map(SendBytes))(Keep.none) } diff --git a/akka-http-core/src/main/scala/akka/http/impl/util/JavaMapping.scala b/akka-http-core/src/main/scala/akka/http/impl/util/JavaMapping.scala index cf28c4e357..8e0d4cc60d 100644 --- a/akka-http-core/src/main/scala/akka/http/impl/util/JavaMapping.scala +++ b/akka-http-core/src/main/scala/akka/http/impl/util/JavaMapping.scala @@ -113,7 +113,7 @@ private[http] object JavaMapping { def toScala(javaObject: javadsl.Flow[JIn, JOut, M]): S = scaladsl.Flow[SIn].map(inMapping.toJava(_)).viaMat(javaObject)(scaladsl.Keep.right).map(outMapping.toScala(_)) def toJava(scalaObject: scaladsl.Flow[SIn, SOut, M]): J = - javadsl.Flow.wrap { + javadsl.Flow.fromGraph { scaladsl.Flow[JIn].map(inMapping.toScala(_)).viaMat(scalaObject)(scaladsl.Keep.right).map(outMapping.toJava(_)) } } @@ -123,7 +123,7 @@ private[http] object JavaMapping { def javaToScalaAdapterFlow[J, S](implicit mapping: JavaMapping[J, S]): scaladsl.Flow[J, S, Unit] = scaladsl.Flow[J].map(mapping.toScala(_)) def adapterBidiFlow[JIn, SIn, SOut, JOut](implicit inMapping: JavaMapping[JIn, SIn], outMapping: JavaMapping[JOut, SOut]): scaladsl.BidiFlow[JIn, SIn, SOut, JOut, Unit] = - scaladsl.BidiFlow.wrap(javaToScalaAdapterFlow(inMapping), scalaToJavaAdapterFlow(outMapping))(scaladsl.Keep.none) + scaladsl.BidiFlow.fromFlowsMat(javaToScalaAdapterFlow(inMapping), scalaToJavaAdapterFlow(outMapping))(scaladsl.Keep.none) implicit def pairMapping[J1, J2, S1, S2](implicit _1Mapping: JavaMapping[J1, S1], _2Mapping: JavaMapping[J2, S2]): JavaMapping[Pair[J1, J2], (S1, S2)] = new JavaMapping[Pair[J1, J2], (S1, S2)] { diff --git a/akka-http-core/src/main/scala/akka/http/javadsl/Http.scala b/akka-http-core/src/main/scala/akka/http/javadsl/Http.scala index f21a9c9b95..1fc0c96d69 100644 --- a/akka-http-core/src/main/scala/akka/http/javadsl/Http.scala +++ b/akka-http-core/src/main/scala/akka/http/javadsl/Http.scala @@ -86,7 +86,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension { * [[ServerBinding]]. */ def bind(interface: String, port: Int, materializer: Materializer): Source[IncomingConnection, Future[ServerBinding]] = - Source.adapt(delegate.bind(interface, port)(materializer) + new Source(delegate.bind(interface, port)(materializer) .map(new IncomingConnection(_)) .mapMaterializedValue(_.map(new ServerBinding(_))(ec))) @@ -105,7 +105,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension { httpsContext: Option[HttpsContext], log: LoggingAdapter, materializer: Materializer): Source[IncomingConnection, Future[ServerBinding]] = - Source.adapt(delegate.bind(interface, port, settings, httpsContext, log)(materializer) + new Source(delegate.bind(interface, port, settings, httpsContext, log)(materializer) .map(new IncomingConnection(_)) .mapMaterializedValue(_.map(new ServerBinding(_))(ec))) @@ -238,7 +238,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension { * Every materialization of the produced flow will attempt to establish a new outgoing connection. */ def outgoingConnection(host: String, port: Int): Flow[HttpRequest, HttpResponse, Future[OutgoingConnection]] = - Flow.wrap { + Flow.fromGraph { akka.stream.scaladsl.Flow[HttpRequest].map(_.asScala) .viaMat(delegate.outgoingConnection(host, port))(Keep.right) .mapMaterializedValue(_.map(new OutgoingConnection(_))(ec)) @@ -248,7 +248,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension { * Same as [[outgoingConnection]] but with HTTPS encryption. */ def outgoingConnectionTls(host: String, port: Int): Flow[HttpRequest, HttpResponse, Future[OutgoingConnection]] = - Flow.wrap { + Flow.fromGraph { akka.stream.scaladsl.Flow[HttpRequest].map(_.asScala) .viaMat(delegate.outgoingConnectionTls(host, port))(Keep.right) .mapMaterializedValue(_.map(new OutgoingConnection(_))(ec)) @@ -262,7 +262,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension { localAddress: Option[InetSocketAddress], settings: ClientConnectionSettings, log: LoggingAdapter): Flow[HttpRequest, HttpResponse, Future[OutgoingConnection]] = - Flow.wrap { + Flow.fromGraph { akka.stream.scaladsl.Flow[HttpRequest].map(_.asScala) .viaMat(delegate.outgoingConnection(host, port, localAddress.asScala, settings, log))(Keep.right) .mapMaterializedValue(_.map(new OutgoingConnection(_))(ec)) @@ -279,7 +279,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension { settings: ClientConnectionSettings, httpsContext: Option[HttpsContext], log: LoggingAdapter): Flow[HttpRequest, HttpResponse, Future[OutgoingConnection]] = - Flow.wrap { + Flow.fromGraph { akka.stream.scaladsl.Flow[HttpRequest].map(_.asScala) .viaMat(delegate.outgoingConnectionTls(host, port, localAddress.asScala, settings, httpsContext.map(_.asInstanceOf[akka.http.scaladsl.HttpsContext]), log))(Keep.right) @@ -629,7 +629,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension { .atopMat(wsLayer)((_, s) ⇒ adaptWsUpgradeResponse(s))) private def adaptWsFlow(wsLayer: stream.scaladsl.Flow[sm.ws.Message, sm.ws.Message, Future[scaladsl.model.ws.WebsocketUpgradeResponse]]): Flow[Message, Message, Future[WebsocketUpgradeResponse]] = - Flow.adapt(JavaMapping.adapterBidiFlow[Message, sm.ws.Message, sm.ws.Message, Message].joinMat(wsLayer)(Keep.right).mapMaterializedValue(adaptWsUpgradeResponse _)) + Flow.fromGraph(JavaMapping.adapterBidiFlow[Message, sm.ws.Message, sm.ws.Message, Message].joinMat(wsLayer)(Keep.right).mapMaterializedValue(adaptWsUpgradeResponse _)) private def adaptWsFlow[Mat](javaFlow: Flow[Message, Message, Mat]): stream.scaladsl.Flow[scaladsl.model.ws.Message, scaladsl.model.ws.Message, Mat] = stream.scaladsl.Flow[scaladsl.model.ws.Message] diff --git a/akka-http-core/src/main/scala/akka/http/javadsl/IncomingConnection.scala b/akka-http-core/src/main/scala/akka/http/javadsl/IncomingConnection.scala index 19f9066457..1b0f833b86 100644 --- a/akka-http-core/src/main/scala/akka/http/javadsl/IncomingConnection.scala +++ b/akka-http-core/src/main/scala/akka/http/javadsl/IncomingConnection.scala @@ -31,7 +31,7 @@ class IncomingConnection private[http] (delegate: akka.http.scaladsl.Http.Incomi * * Use `Flow.join` or one of the handleXXX methods to consume handle requests on this connection. */ - def flow: Flow[HttpResponse, HttpRequest, Unit] = Flow.adapt(delegate.flow).asInstanceOf[Flow[HttpResponse, HttpRequest, Unit]] + def flow: Flow[HttpResponse, HttpRequest, Unit] = Flow.fromGraph(delegate.flow).asInstanceOf[Flow[HttpResponse, HttpRequest, Unit]] /** * Handles the connection with the given flow, which is materialized exactly once diff --git a/akka-http-core/src/main/scala/akka/http/javadsl/model/ws/WebsocketUpgradeResponse.scala b/akka-http-core/src/main/scala/akka/http/javadsl/model/ws/WebsocketUpgradeResponse.scala index 7aa27356b2..fc19fb7bcc 100644 --- a/akka-http-core/src/main/scala/akka/http/javadsl/model/ws/WebsocketUpgradeResponse.scala +++ b/akka-http-core/src/main/scala/akka/http/javadsl/model/ws/WebsocketUpgradeResponse.scala @@ -38,18 +38,18 @@ object WebsocketUpgradeResponse { import akka.http.impl.util.JavaMapping.Implicits._ def adapt(scalaResponse: scaladsl.model.ws.WebsocketUpgradeResponse): WebsocketUpgradeResponse = scalaResponse match { - case ValidUpgrade(response, chosen) ⇒ + case ValidUpgrade(resp, chosen) ⇒ new WebsocketUpgradeResponse { def isValid: Boolean = true - def response: HttpResponse = scalaResponse.response + def response: HttpResponse = resp def chosenSubprotocol: Option[String] = chosen.asJava def invalidationReason: String = throw new UnsupportedOperationException("invalidationReason must not be called for valid response") } - case InvalidUpgradeResponse(response, cause) ⇒ + case InvalidUpgradeResponse(resp, cause) ⇒ new WebsocketUpgradeResponse { def isValid: Boolean = false - def response: HttpResponse = scalaResponse.response + def response: HttpResponse = resp def chosenSubprotocol: Option[String] = throw new UnsupportedOperationException("chosenSubprotocol must not be called for valid response") def invalidationReason: String = cause } diff --git a/akka-http-core/src/main/scala/akka/http/scaladsl/model/HttpEntity.scala b/akka-http-core/src/main/scala/akka/http/scaladsl/model/HttpEntity.scala index 537d0868d7..f29829b103 100644 --- a/akka-http-core/src/main/scala/akka/http/scaladsl/model/HttpEntity.scala +++ b/akka-http-core/src/main/scala/akka/http/scaladsl/model/HttpEntity.scala @@ -75,7 +75,7 @@ sealed trait HttpEntity extends jm.HttpEntity { def withContentType(contentType: ContentType): HttpEntity /** Java API */ - def getDataBytes: stream.javadsl.Source[ByteString, _] = stream.javadsl.Source.adapt(dataBytes) + def getDataBytes: stream.javadsl.Source[ByteString, _] = stream.javadsl.Source.fromGraph(dataBytes) /** Java API */ def getContentLengthOption: japi.Option[JLong] = @@ -296,7 +296,7 @@ object HttpEntity { override def productPrefix = "HttpEntity.Chunked" /** Java API */ - def getChunks: stream.javadsl.Source[jm.ChunkStreamPart, Any] = stream.javadsl.Source.adapt(chunks) + def getChunks: stream.javadsl.Source[jm.ChunkStreamPart, Any] = stream.javadsl.Source.fromGraph(chunks) } object Chunked { /** diff --git a/akka-http-core/src/main/scala/akka/http/scaladsl/model/ws/UpgradeToWebsocket.scala b/akka-http-core/src/main/scala/akka/http/scaladsl/model/ws/UpgradeToWebsocket.scala index 62c65ff10a..5508161834 100644 --- a/akka-http-core/src/main/scala/akka/http/scaladsl/model/ws/UpgradeToWebsocket.scala +++ b/akka-http-core/src/main/scala/akka/http/scaladsl/model/ws/UpgradeToWebsocket.scala @@ -52,7 +52,7 @@ trait UpgradeToWebsocket extends jm.ws.UpgradeToWebsocket { def handleMessagesWithSinkSource(inSink: Sink[Message, Any], outSource: Source[Message, Any], subprotocol: Option[String] = None): HttpResponse = - handleMessages(Flow.wrap(inSink, outSource)((_, _) ⇒ ()), subprotocol) + handleMessages(Flow.fromSinkAndSource(inSink, outSource), subprotocol) import scala.collection.JavaConverters._ @@ -88,5 +88,5 @@ trait UpgradeToWebsocket extends jm.ws.UpgradeToWebsocket { handleMessages(createScalaFlow(inSink, outSource), subprotocol = Some(subprotocol)) private[this] def createScalaFlow(inSink: stream.javadsl.Sink[jm.ws.Message, _], outSource: stream.javadsl.Source[jm.ws.Message, _]): Flow[Message, Message, Any] = - JavaMapping.toScala(Flow.wrap(inSink.asScala, outSource.asScala)((_, _) ⇒ ()).asJava) + JavaMapping.toScala(Flow.fromSinkAndSourceMat(inSink.asScala, outSource.asScala)((_, _) ⇒ ()).asJava) } diff --git a/akka-http-core/src/test/java/akka/http/javadsl/WSEchoTestClientApp.java b/akka-http-core/src/test/java/akka/http/javadsl/WSEchoTestClientApp.java index 78d0e2cd30..5a58759de6 100644 --- a/akka-http-core/src/test/java/akka/http/javadsl/WSEchoTestClientApp.java +++ b/akka-http-core/src/test/java/akka/http/javadsl/WSEchoTestClientApp.java @@ -63,7 +63,7 @@ public class WSEchoTestClientApp { .toMat(Sink.>head(), Keep.>>right()); Flow>> echoClient = - Flow.wrap(echoSink, echoSource, Keep.>, BoxedUnit>left()); + Flow.fromSinkAndSourceMat(echoSink, echoSource, Keep.>, BoxedUnit>left()); Future> result = Http.get(system).singleWebsocketRequest( diff --git a/akka-http-core/src/test/scala/akka/http/impl/engine/client/ConnectionPoolSpec.scala b/akka-http-core/src/test/scala/akka/http/impl/engine/client/ConnectionPoolSpec.scala index a3551e7606..8a76f3fb63 100644 --- a/akka-http-core/src/test/scala/akka/http/impl/engine/client/ConnectionPoolSpec.scala +++ b/akka-http-core/src/test/scala/akka/http/impl/engine/client/ConnectionPoolSpec.scala @@ -298,12 +298,10 @@ class ConnectionPoolSpec extends AkkaSpec(""" val incomingConnectionCounter = new AtomicInteger val incomingConnections = TestSubscriber.manualProbe[Http.IncomingConnection] val incomingConnectionsSub = { - val rawBytesInjection = BidiFlow() { b ⇒ - val top = b.add(Flow[SslTlsOutbound].collect[ByteString] { case SendBytes(x) ⇒ mapServerSideOutboundRawBytes(x) } - .transform(StreamUtils.recover { case NoErrorComplete ⇒ ByteString.empty })) - val bottom = b.add(Flow[ByteString].map(SessionBytes(null, _))) - BidiShape(top.inlet, top.outlet, bottom.inlet, bottom.outlet) - } + val rawBytesInjection = BidiFlow.fromFlows( + 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) // TODO getHostString in Java7 Tcp().bind(serverEndpoint.getHostName, serverEndpoint.getPort, idleTimeout = serverSettings.timeouts.idleTimeout) diff --git a/akka-http-core/src/test/scala/akka/http/impl/engine/client/HighLevelOutgoingConnectionSpec.scala b/akka-http-core/src/test/scala/akka/http/impl/engine/client/HighLevelOutgoingConnectionSpec.scala index 8e0630f2ea..2e91cb0f17 100644 --- a/akka-http-core/src/test/scala/akka/http/impl/engine/client/HighLevelOutgoingConnectionSpec.scala +++ b/akka-http-core/src/test/scala/akka/http/impl/engine/client/HighLevelOutgoingConnectionSpec.scala @@ -6,7 +6,7 @@ package akka.http.impl.engine.client import scala.concurrent.Await import scala.concurrent.duration._ -import akka.stream.ActorMaterializer +import akka.stream.{ FlowShape, ActorMaterializer } import akka.stream.scaladsl._ import akka.stream.testkit.AkkaSpec import akka.http.scaladsl.{ Http, TestUtils } @@ -44,7 +44,7 @@ class HighLevelOutgoingConnectionSpec extends AkkaSpec { val connFlow = Http().outgoingConnection(serverHostName, serverPort) val C = 4 - val doubleConnection = Flow() { implicit b ⇒ + val doubleConnection = Flow.fromGraph(FlowGraph.create() { implicit b ⇒ import FlowGraph.Implicits._ val bcast = b.add(Broadcast[HttpRequest](C)) @@ -52,8 +52,8 @@ class HighLevelOutgoingConnectionSpec extends AkkaSpec { for (i ← 0 until C) bcast.out(i) ~> connFlow ~> merge.in(i) - (bcast.in, merge.out) - } + FlowShape(bcast.in, merge.out) + }) val N = 100 val result = Source(() ⇒ Iterator.from(1)) diff --git a/akka-http-core/src/test/scala/akka/http/impl/engine/client/LowLevelOutgoingConnectionSpec.scala b/akka-http-core/src/test/scala/akka/http/impl/engine/client/LowLevelOutgoingConnectionSpec.scala index 53df22424d..4d56fcca37 100644 --- a/akka-http-core/src/test/scala/akka/http/impl/engine/client/LowLevelOutgoingConnectionSpec.scala +++ b/akka-http-core/src/test/scala/akka/http/impl/engine/client/LowLevelOutgoingConnectionSpec.scala @@ -9,7 +9,7 @@ import akka.stream.io.{ SessionBytes, SslTlsOutbound, SendBytes } import org.scalatest.Inside import akka.util.ByteString import akka.event.NoLogging -import akka.stream.ActorMaterializer +import akka.stream.{ ClosedShape, ActorMaterializer } import akka.stream.testkit._ import akka.stream.scaladsl._ import akka.http.scaladsl.model.HttpEntity._ @@ -359,14 +359,15 @@ class LowLevelOutgoingConnectionSpec extends AkkaSpec("akka.loggers = []\n akka. val netOut = TestSubscriber.manualProbe[ByteString] val netIn = TestPublisher.manualProbe[ByteString]() - FlowGraph.closed(OutgoingConnectionBlueprint(Host("example.com"), settings, NoLogging)) { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create(OutgoingConnectionBlueprint(Host("example.com"), settings, NoLogging)) { implicit b ⇒ client ⇒ import FlowGraph.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) - }.run() + ClosedShape + }).run() netOut -> netIn } diff --git a/akka-http-core/src/test/scala/akka/http/impl/engine/server/HttpServerTestSetupBase.scala b/akka-http-core/src/test/scala/akka/http/impl/engine/server/HttpServerTestSetupBase.scala index 55e646d646..37bef87e1e 100644 --- a/akka-http-core/src/test/scala/akka/http/impl/engine/server/HttpServerTestSetupBase.scala +++ b/akka-http-core/src/test/scala/akka/http/impl/engine/server/HttpServerTestSetupBase.scala @@ -15,8 +15,8 @@ import akka.actor.ActorSystem import akka.event.NoLogging import akka.util.ByteString -import akka.stream.Materializer -import akka.stream.scaladsl.{ Flow, Sink, Source, FlowGraph } +import akka.stream.{ ClosedShape, Materializer } +import akka.stream.scaladsl._ import akka.stream.testkit.{ TestPublisher, TestSubscriber } import akka.http.impl.util._ @@ -39,14 +39,15 @@ abstract class HttpServerTestSetupBase { val netIn = TestPublisher.probe[ByteString]() val netOut = ByteStringSinkProbe() - FlowGraph.closed(HttpServerBluePrint(settings, remoteAddress = remoteAddress, log = NoLogging)) { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create(HttpServerBluePrint(settings, remoteAddress = remoteAddress, log = NoLogging)) { implicit b ⇒ server ⇒ import FlowGraph.Implicits._ Source(netIn) ~> Flow[ByteString].map(SessionBytes(null, _)) ~> server.in2 server.out1 ~> Flow[SslTlsOutbound].collect { case SendBytes(x) ⇒ x } ~> netOut.sink server.out2 ~> Sink(requests) Source(responses) ~> server.in1 - }.run() + ClosedShape + }).run() netIn -> netOut } diff --git a/akka-http-core/src/test/scala/akka/http/impl/engine/ws/EchoTestClientApp.scala b/akka-http-core/src/test/scala/akka/http/impl/engine/ws/EchoTestClientApp.scala index 900b25b0bd..bd9d0d200b 100644 --- a/akka-http-core/src/test/scala/akka/http/impl/engine/ws/EchoTestClientApp.scala +++ b/akka-http-core/src/test/scala/akka/http/impl/engine/ws/EchoTestClientApp.scala @@ -50,7 +50,7 @@ object EchoTestClientApp extends App { .grouped(10000) .toMat(Sink.head)(Keep.right) - def echoClient = Flow.wrap(sink, source)(Keep.left) + def echoClient = Flow.fromSinkAndSourceMat(sink, source)(Keep.left) val (upgrade, res) = Http().singleWebsocketRequest("wss://echo.websocket.org", echoClient) res onComplete { diff --git a/akka-http-core/src/test/scala/akka/http/impl/engine/ws/MessageSpec.scala b/akka-http-core/src/test/scala/akka/http/impl/engine/ws/MessageSpec.scala index 7e8b4b3ff9..468465bfd1 100644 --- a/akka-http-core/src/test/scala/akka/http/impl/engine/ws/MessageSpec.scala +++ b/akka-http-core/src/test/scala/akka/http/impl/engine/ws/MessageSpec.scala @@ -817,8 +817,8 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec { val messageOut = TestPublisher.probe[Message]() val messageHandler: Flow[Message, Message, Unit] = - Flow.wrap { - FlowGraph.partial() { implicit b ⇒ + Flow.fromGraph { + FlowGraph.create() { implicit b ⇒ val in = b.add(Sink(messageIn)) val out = b.add(Source(messageOut)) diff --git a/akka-http-core/src/test/scala/akka/http/impl/engine/ws/WSClientAutobahnTest.scala b/akka-http-core/src/test/scala/akka/http/impl/engine/ws/WSClientAutobahnTest.scala index 4c05b8f46b..d4969eb21d 100644 --- a/akka-http-core/src/test/scala/akka/http/impl/engine/ws/WSClientAutobahnTest.scala +++ b/akka-http-core/src/test/scala/akka/http/impl/engine/ws/WSClientAutobahnTest.scala @@ -199,7 +199,7 @@ object WSClientAutobahnTest extends App { */ def runToSingleText(uri: Uri): Future[String] = { val sink = Sink.head[Message] - runWs(uri, Flow.wrap(sink, Source.lazyEmpty[Message])(Keep.left)).flatMap { + runWs(uri, Flow.fromSinkAndSourceMat(sink, Source.maybe[Message])(Keep.left)).flatMap { case tm: TextMessage ⇒ tm.textStream.runWith(Sink.fold("")(_ + _)) } } diff --git a/akka-http-core/src/test/scala/akka/http/impl/engine/ws/WebsocketClientSpec.scala b/akka-http-core/src/test/scala/akka/http/impl/engine/ws/WebsocketClientSpec.scala index a487f8340d..e5fc2bcd4b 100644 --- a/akka-http-core/src/test/scala/akka/http/impl/engine/ws/WebsocketClientSpec.scala +++ b/akka-http-core/src/test/scala/akka/http/impl/engine/ws/WebsocketClientSpec.scala @@ -7,6 +7,7 @@ package akka.http.impl.engine.ws import java.util.Random import akka.http.scaladsl.model.ws.{ InvalidUpgradeResponse, WebsocketUpgradeResponse } +import akka.stream.ClosedShape import scala.concurrent.duration._ @@ -119,7 +120,7 @@ class WebsocketClientSpec extends FreeSpec with Matchers with WithMaterializerSp "don't send out frames before handshake was finished successfully" in new TestSetup { def clientImplementation: Flow[Message, Message, Unit] = - Flow.wrap(Sink.ignore, Source.single(TextMessage("fast message")))(Keep.none) + Flow.fromSinkAndSourceMat(Sink.ignore, Source.single(TextMessage("fast message")))(Keep.none) expectWireData(UpgradeRequestBytes) expectNoWireData() @@ -311,13 +312,14 @@ class WebsocketClientSpec extends FreeSpec with Matchers with WithMaterializerSp val netIn = TestPublisher.probe[ByteString]() val graph = - FlowGraph.closed(clientLayer) { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create(clientLayer) { implicit b ⇒ client ⇒ import FlowGraph.Implicits._ Source(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 + }) val response = graph.run() @@ -365,6 +367,6 @@ class WebsocketClientSpec extends FreeSpec with Matchers with WithMaterializerSp lazy val messagesIn = TestSubscriber.probe[Message]() override def clientImplementation: Flow[Message, Message, Unit] = - Flow.wrap(Sink(messagesIn), Source(messagesOut))(Keep.none) + Flow.fromSinkAndSourceMat(Sink(messagesIn), Source(messagesOut))(Keep.none) } } diff --git a/akka-http-core/src/test/scala/akka/http/impl/engine/ws/WebsocketServerSpec.scala b/akka-http-core/src/test/scala/akka/http/impl/engine/ws/WebsocketServerSpec.scala index e8b8755d72..458f72e75f 100644 --- a/akka-http-core/src/test/scala/akka/http/impl/engine/ws/WebsocketServerSpec.scala +++ b/akka-http-core/src/test/scala/akka/http/impl/engine/ws/WebsocketServerSpec.scala @@ -38,7 +38,7 @@ class WebsocketServerSpec extends FreeSpec with Matchers with WithMaterializerSp val source = Source(List(1, 2, 3, 4, 5)).map(num ⇒ TextMessage.Strict(s"Message $num")) - val handler = Flow.wrap(Sink.ignore, source)(Keep.none) + val handler = Flow.fromSinkAndSourceMat(Sink.ignore, source)(Keep.none) val response = upgrade.get.handleMessages(handler) responses.sendNext(response) diff --git a/akka-http-core/src/test/scala/akka/http/scaladsl/ClientServerSpec.scala b/akka-http-core/src/test/scala/akka/http/scaladsl/ClientServerSpec.scala index 98167bb198..3f6e69ce1c 100644 --- a/akka-http-core/src/test/scala/akka/http/scaladsl/ClientServerSpec.scala +++ b/akka-http-core/src/test/scala/akka/http/scaladsl/ClientServerSpec.scala @@ -330,10 +330,7 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll { val sink = Sink.publisher[HttpRequest] val source = Source.subscriber[HttpResponse] - val handler = Flow(sink, source)(Keep.both) { implicit b ⇒ - (snk, src) ⇒ - (snk.inlet, src.outlet) - } + val handler = Flow.fromSinkAndSourceMat(sink, source)(Keep.both) val (pub, sub) = incomingConnection.handleWith(handler) val requestSubscriberProbe = TestSubscriber.manualProbe[HttpRequest]() diff --git a/akka-http-testkit/src/main/scala/akka/http/scaladsl/testkit/WSProbe.scala b/akka-http-testkit/src/main/scala/akka/http/scaladsl/testkit/WSProbe.scala index 92ebd379b3..dc93cbe2cb 100644 --- a/akka-http-testkit/src/main/scala/akka/http/scaladsl/testkit/WSProbe.scala +++ b/akka-http-testkit/src/main/scala/akka/http/scaladsl/testkit/WSProbe.scala @@ -104,7 +104,7 @@ object WSProbe { val subscriber = TestSubscriber.probe[Message]() val publisher = TestPublisher.probe[Message]() - def flow: Flow[Message, Message, Unit] = Flow.wrap(Sink(subscriber), Source(publisher))(Keep.none) + def flow: Flow[Message, Message, Unit] = Flow.fromSinkAndSourceMat(Sink(subscriber), Source(publisher))(Keep.none) def sendMessage(message: Message): Unit = publisher.sendNext(message) def sendMessage(text: String): Unit = sendMessage(TextMessage(text)) diff --git a/akka-stream-tck/src/test/scala/akka/stream/tck/LazyEmptySourceTest.scala b/akka-stream-tck/src/test/scala/akka/stream/tck/LazyEmptySourceTest.scala deleted file mode 100644 index 91bb1f8ad0..0000000000 --- a/akka-stream-tck/src/test/scala/akka/stream/tck/LazyEmptySourceTest.scala +++ /dev/null @@ -1,18 +0,0 @@ -/** - * Copyright (C) 2014 Typesafe Inc. - */ -package akka.stream.tck - -import org.reactivestreams.Publisher -import org.reactivestreams.Subscriber -import akka.stream.scaladsl.Source -import akka.stream.scaladsl.Sink - -class LazyEmptySourceTest extends AkkaPublisherVerification[Int] { - - def createPublisher(elements: Long): Publisher[Int] = - Source.lazyEmpty[Int].runWith(Sink.publisher) - - override def maxElementsFromPublisher(): Long = 0 -} - diff --git a/akka-stream-tck/src/test/scala/akka/stream/tck/MaybeSourceTest.scala b/akka-stream-tck/src/test/scala/akka/stream/tck/MaybeSourceTest.scala new file mode 100644 index 0000000000..c05dc2a912 --- /dev/null +++ b/akka-stream-tck/src/test/scala/akka/stream/tck/MaybeSourceTest.scala @@ -0,0 +1,20 @@ +/** + * Copyright (C) 2014 Typesafe Inc. + */ +package akka.stream.tck + +import org.reactivestreams.Publisher +import org.reactivestreams.Subscriber +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)(Keep.both).run() + p success Some(1) + pub + } + + override def maxElementsFromPublisher(): Long = 1 +} + diff --git a/akka-stream-testkit/src/test/scala/akka/stream/testkit/TwoStreamsSetup.scala b/akka-stream-testkit/src/test/scala/akka/stream/testkit/TwoStreamsSetup.scala index 8cc084ac9f..326d87a759 100644 --- a/akka-stream-testkit/src/test/scala/akka/stream/testkit/TwoStreamsSetup.scala +++ b/akka-stream-testkit/src/test/scala/akka/stream/testkit/TwoStreamsSetup.scala @@ -1,6 +1,6 @@ package akka.stream.testkit -import akka.stream.{ ActorMaterializer, ActorMaterializerSettings, Inlet, Outlet } +import akka.stream._ import akka.stream.scaladsl._ import org.reactivestreams.Publisher import scala.collection.immutable @@ -19,15 +19,15 @@ abstract class TwoStreamsSetup extends BaseTwoStreamsSetup { override def setup(p1: Publisher[Int], p2: Publisher[Int]) = { val subscriber = TestSubscriber.probe[Outputs]() - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ import FlowGraph.Implicits._ val f = fixture(b) Source(p1) ~> f.left Source(p2) ~> f.right f.out ~> Sink(subscriber) - - }.run() + ClosedShape + }).run() subscriber } diff --git a/akka-stream-tests/src/test/java/akka/stream/javadsl/BidiFlowTest.java b/akka-stream-tests/src/test/java/akka/stream/javadsl/BidiFlowTest.java index ed105ff4e8..94eae7469a 100644 --- a/akka-stream-tests/src/test/java/akka/stream/javadsl/BidiFlowTest.java +++ b/akka-stream-tests/src/test/java/akka/stream/javadsl/BidiFlowTest.java @@ -35,84 +35,83 @@ public class BidiFlowTest extends StreamTest { "FlowTest", AkkaSpec.testConf()); private final BidiFlow bidi = BidiFlow - .factory() - .create( - new Function, BidiShape>() { - @Override - public BidiShape apply(Builder b) - throws Exception { - final FlowShape top = b.graph(Flow - . empty().map(new Function() { - @Override - public Long apply(Integer arg) { - return (long) ((int) arg) + 2; - } - })); - final FlowShape bottom = b.graph(Flow - . empty().map(new Function() { - @Override - public String apply(ByteString arg) { - return arg.decodeString("UTF-8"); - } - })); - return new BidiShape(top - .inlet(), top.outlet(), bottom.inlet(), bottom.outlet()); - } - }); + .fromGraph(FlowGraph.create( + new Function, BidiShape>() { + @Override + public BidiShape apply(Builder b) + throws Exception { + final FlowShape top = b.add(Flow + .of(Integer.class).map(new Function() { + @Override + public Long apply(Integer arg) { + return (long) ((int) arg) + 2; + } + })); + final FlowShape bottom = b.add(Flow + .of(ByteString.class).map(new Function() { + @Override + public String apply(ByteString arg) { + return arg.decodeString("UTF-8"); + } + })); + return new BidiShape(top + .inlet(), top.outlet(), bottom.inlet(), bottom.outlet()); + } + })); private final BidiFlow inverse = BidiFlow - .factory() - .create( - new Function, BidiShape>() { - @Override - public BidiShape apply(Builder b) - throws Exception { - final FlowShape top = b.graph(Flow. empty() - .map(new Function() { - @Override - public Integer apply(Long arg) { - return (int) ((long) arg) + 2; - } - })); - final FlowShape bottom = b.graph(Flow - . empty().map(new Function() { - @Override - public ByteString apply(String arg) { - return ByteString.fromString(arg); - } - })); - return new BidiShape(top - .inlet(), top.outlet(), bottom.inlet(), bottom.outlet()); - } - }); + .fromGraph( + FlowGraph.create( + new Function, BidiShape>() { + @Override + public BidiShape apply(Builder b) + throws Exception { + final FlowShape top = b.add(Flow.of(Long.class) + .map(new Function() { + @Override + public Integer apply(Long arg) { + return (int) ((long) arg) + 2; + } + })); + final FlowShape bottom = b.add(Flow + .of(String.class).map(new Function() { + @Override + public ByteString apply(String arg) { + return ByteString.fromString(arg); + } + })); + return new BidiShape(top + .inlet(), top.outlet(), bottom.inlet(), bottom.outlet()); + } + })); - private final BidiFlow> bidiMat = BidiFlow - .factory() - .create( - Sink. head(), - new Function2>, SinkShape, BidiShape>() { - @Override - public BidiShape apply(Builder> b, SinkShape sink) - throws Exception { - b.from(b.graph(Source.single(42))).to(sink); - final FlowShape top = b.graph(Flow - . empty().map(new Function() { - @Override - public Long apply(Integer arg) { - return (long) ((int) arg) + 2; - } - })); - final FlowShape bottom = b.graph(Flow - . empty().map(new Function() { - @Override - public String apply(ByteString arg) { - return arg.decodeString("UTF-8"); - } - })); - return new BidiShape(top - .inlet(), top.outlet(), bottom.inlet(), bottom.outlet()); - } - }); + private final BidiFlow> bidiMat = + BidiFlow.fromGraph( + FlowGraph.create( + Sink.head(), + new Function2>, SinkShape, BidiShape>() { + @Override + public BidiShape apply(Builder> b, SinkShape sink) + throws Exception { + b.from(b.add(Source.single(42))).to(sink); + final FlowShape top = b.add(Flow + .of(Integer.class).map(new Function() { + @Override + public Long apply(Integer arg) { + return (long) ((int) arg) + 2; + } + })); + final FlowShape bottom = b.add(Flow + .of(ByteString.class).map(new Function() { + @Override + public String apply(ByteString arg) { + return arg.decodeString("UTF-8"); + } + })); + return new BidiShape(top + .inlet(), top.outlet(), bottom.inlet(), bottom.outlet()); + } + })); private final String str = "Hello World"; private final ByteString bytes = ByteString.fromString(str); @@ -126,22 +125,23 @@ public class BidiFlowTest extends StreamTest { @Test public void mustWorkInIsolation() throws Exception { - final Pair, Future> p = FlowGraph - .factory() - .closed(Sink. head(), Sink. head(), - Keep., Future> both(), - new Procedure3, Future>>, SinkShape, SinkShape>() { - @Override - public void apply(Builder, Future>> b, SinkShape st, - SinkShape sb) throws Exception { - final BidiShape s = b - .graph(bidi); - b.from(b.graph(Source.single(1))).toInlet(s.in1()); - b.from(s.out1()).to(st); - b.from(b.graph(Source.single(bytes))).toInlet(s.in2()); - b.from(s.out2()).to(sb); - } - }).run(materializer); + final Pair, Future> p = + RunnableGraph.fromGraph(FlowGraph + .create(Sink. head(), Sink. head(), + Keep., Future> both(), + new Function3, Future>>, SinkShape, SinkShape, ClosedShape>() { + @Override + public ClosedShape apply(Builder, Future>> b, SinkShape st, + SinkShape sb) throws Exception { + final BidiShape s = + b.add(bidi); + b.from(b.add(Source.single(1))).toInlet(s.in1()); + b.from(s.out1()).to(st); + b.from(b.add(Source.single(bytes))).toInlet(s.in2()); + b.from(s.out2()).to(sb); + return ClosedShape.getInstance(); + } + })).run(materializer); final Long rt = Await.result(p.first(), oneSec); final String rb = Await.result(p.second(), oneSec); @@ -152,7 +152,7 @@ public class BidiFlowTest extends StreamTest { @Test public void mustWorkAsAFlowThatIsOpenOnTheLeft() throws Exception { - final Flow f = bidi.join(Flow. empty().map( + final Flow f = bidi.join(Flow.of(Long.class).map( new Function() { @Override public ByteString apply(Long arg) { return ByteString.fromString("Hello " + arg); @@ -164,7 +164,7 @@ public class BidiFlowTest extends StreamTest { @Test public void mustWorkAsAFlowThatIsOpenOnTheRight() throws Exception { - final Flow f = Flow. empty().map( + final Flow f = Flow.of(String.class).map( new Function() { @Override public Integer apply(String arg) { return Integer.valueOf(arg); @@ -177,7 +177,7 @@ public class BidiFlowTest extends StreamTest { @Test public void mustWorkWhenAtopItsInverse() throws Exception { - final Flow f = bidi.atop(inverse).join(Flow. empty().map( + final Flow f = bidi.atop(inverse).join(Flow.of(Integer.class).map( new Function() { @Override public String apply(Integer arg) { return arg.toString(); @@ -189,7 +189,7 @@ public class BidiFlowTest extends StreamTest { @Test public void mustWorkWhenReversed() throws Exception { - final Flow f = Flow. empty().map( + final Flow f = Flow.of(Integer.class).map( new Function() { @Override public String apply(Integer arg) { return arg.toString(); @@ -201,61 +201,66 @@ public class BidiFlowTest extends StreamTest { @Test public void mustMaterializeToItsValue() throws Exception { - final Future f = FlowGraph.factory().closed(bidiMat, new Procedure2 >, BidiShape>() { + final Future f = RunnableGraph.fromGraph( + FlowGraph.create(bidiMat, + new Function2 >, BidiShape, ClosedShape>() { @Override - public void apply(Builder> b, + public ClosedShape apply(Builder> b, BidiShape shape) throws Exception { - final FlowShape left = b.graph(Flow. empty().map( - new Function() { - @Override public Integer apply(String arg) { - return Integer.valueOf(arg); - } - })); - final FlowShape right = b.graph(Flow. empty().map( - new Function() { - @Override public ByteString apply(Long arg) { - return ByteString.fromString("Hello " + arg); - } - })); + final FlowShape left = b.add(Flow.of(String.class).map( + new Function() { + @Override + public Integer apply(String arg) { + return Integer.valueOf(arg); + } + })); + final FlowShape right = b.add(Flow.of(Long.class).map( + new Function() { + @Override + public ByteString apply(Long arg) { + return ByteString.fromString("Hello " + arg); + } + })); b.from(shape.out2()).via(left).toInlet(shape.in1()) .from(shape.out1()).via(right).toInlet(shape.in2()); + return ClosedShape.getInstance(); } - }).run(materializer); + })).run(materializer); assertEquals((Integer) 42, Await.result(f, oneSec)); } @Test public void mustCombineMaterializationValues() throws Exception { - final Flow> left = Flow.factory().create( - Sink. head(), new Function2 >, SinkShape, Pair, Outlet>>() { - @Override - public Pair, Outlet> apply(Builder> b, - SinkShape sink) throws Exception { - final UniformFanOutShape bcast = b.graph(Broadcast. create(2)); - final UniformFanInShape merge = b.graph(Merge. create(2)); - final FlowShape flow = b.graph(Flow. empty().map( - new Function() { - @Override - public Integer apply(String arg) { - return Integer.valueOf(arg); - } - })); - b.from(bcast).to(sink) - .from(b.graph(Source.single(1))).viaFanOut(bcast).toFanIn(merge) - .from(flow).toFanIn(merge); - return new Pair, Outlet>(flow.inlet(), merge.out()); - } - }); - final Flow>> right = Flow.factory().create( - Sink.> head(), new Function2>>, SinkShape>, Pair, Outlet>>() { - @Override - public Pair, Outlet> apply(Builder>> b, - SinkShape> sink) throws Exception { - final FlowShape> flow = b.graph(Flow. empty().grouped(10)); - b.from(flow).to(sink); - return new Pair, Outlet>(flow.inlet(), b.source(Source.single(ByteString.fromString("10")))); - } - }); + final Flow> left = Flow.fromGraph(FlowGraph.create( + Sink.head(), new Function2>, SinkShape, FlowShape>() { + @Override + public FlowShape apply(Builder> b, + SinkShape sink) throws Exception { + final UniformFanOutShape bcast = b.add(Broadcast.create(2)); + final UniformFanInShape merge = b.add(Merge.create(2)); + final FlowShape flow = b.add(Flow.of(String.class).map( + new Function() { + @Override + public Integer apply(String arg) { + return Integer.valueOf(arg); + } + })); + b.from(bcast).to(sink) + .from(b.add(Source.single(1))).viaFanOut(bcast).toFanIn(merge) + .from(flow).toFanIn(merge); + return new FlowShape(flow.inlet(), merge.out()); + } + })); + final Flow>> right = Flow.fromGraph(FlowGraph.create( + Sink.>head(), new Function2>>, SinkShape>, FlowShape>() { + @Override + public FlowShape apply(Builder>> b, + SinkShape> sink) throws Exception { + final FlowShape> flow = b.add(Flow.of(Long.class).grouped(10)); + b.from(flow).to(sink); + return new FlowShape(flow.inlet(), b.add(Source.single(ByteString.fromString("10"))).outlet()); + } + })); final Pair, Future>, Future>> result = left.joinMat(bidiMat, Keep., Future> both()).joinMat(right, Keep., Future>, Future>> both()).run(materializer); final Future l = result.first().first(); diff --git a/akka-stream-tests/src/test/java/akka/stream/javadsl/FlowGraphTest.java b/akka-stream-tests/src/test/java/akka/stream/javadsl/FlowGraphTest.java index 94b792b27a..7f98ebb3b9 100644 --- a/akka-stream-tests/src/test/java/akka/stream/javadsl/FlowGraphTest.java +++ b/akka-stream-tests/src/test/java/akka/stream/javadsl/FlowGraphTest.java @@ -69,15 +69,16 @@ public class FlowGraphTest extends StreamTest { final Sink> publisher = Sink.publisher(); - final Source source = Source.factory().create(new Function, Outlet>() { - @Override - public Outlet apply(Builder b) throws Exception { - final UniformFanInShape merge = b.graph(Merge. create(2)); - b.flow(b.source(in1), f1, merge.in(0)); - b.flow(b.source(in2), f2, merge.in(1)); - return merge.out(); - } - }); + final Source source = Source.fromGraph( + FlowGraph.create(new Function, SourceShape>() { + @Override + public SourceShape apply(Builder b) throws Exception { + final UniformFanInShape merge = b.add(Merge.create(2)); + b.from(b.add(in1)).via(b.add(f1)).toInlet(merge.in(0)); + b.from(b.add(in2)).via(b.add(f2)).toInlet(merge.in(1)); + return new SourceShape(merge.out()); + } + })); // collecting final Publisher pub = source.runWith(publisher, materializer); @@ -93,16 +94,21 @@ public class FlowGraphTest extends StreamTest { final Iterable input1 = Arrays.asList("A", "B", "C"); final Iterable input2 = Arrays.asList(1, 2, 3); - final Builder b = FlowGraph.builder(); - final Source in1 = Source.from(input1); - final Source in2 = Source.from(input2); - final FanInShape2> zip = b.graph(Zip.create()); - final Sink, BoxedUnit> out = createSink(probe); + RunnableGraph.fromGraph( FlowGraph.create( + new Function,ClosedShape>() { + @Override + public ClosedShape apply(final Builder b) throws Exception { + final Source in1 = Source.from(input1); + final Source in2 = Source.from(input2); + final FanInShape2> zip = b.add(Zip.create()); + final Sink, BoxedUnit> out = createSink(probe); - b.edge(b.source(in1), zip.in0()); - b.edge(b.source(in2), zip.in1()); - b.edge(zip.out(), b.sink(out)); - b.run(materializer); + b.from(b.add(in1)).toInlet(zip.in0()); + b.from(b.add(in2)).toInlet(zip.in1()); + b.from(zip.out()).to(b.add(out)); + return ClosedShape.getInstance(); + } + })).run(materializer); List output = Arrays.asList(probe.receiveN(3)); @SuppressWarnings("unchecked") @@ -123,17 +129,22 @@ public class FlowGraphTest extends StreamTest { final Iterable expected1 = Arrays.asList("A", "B", "C"); final Iterable expected2 = Arrays.asList(1, 2, 3); - final Builder b = FlowGraph.builder(); - final Outlet> in = b.source(Source.from(input)); - final FanOutShape2, String, Integer> unzip = b.graph(Unzip.create()); + RunnableGraph.fromGraph(FlowGraph.create( + new Function, ClosedShape>() { + @Override + public ClosedShape apply(final Builder b) throws Exception { + final SourceShape> in = b.add(Source.from(input)); + final FanOutShape2, String, Integer> unzip = b.add(Unzip.create()); - final Sink out1 = createSink(probe1); - final Sink out2 = createSink(probe2); + final SinkShape out1 = b.add(FlowGraphTest.createSink(probe1)); + final SinkShape out2 = b.add(FlowGraphTest.createSink(probe2)); - b.edge(in, unzip.in()); - b.edge(unzip.out0(), b.sink(out1)); - b.edge(unzip.out1(), b.sink(out2)); - b.run(materializer); + b.from(in).toInlet(unzip.in()); + b.from(unzip.out0()).to(out1); + b.from(unzip.out1()).to(out2); + return ClosedShape.getInstance(); + } + })).run(materializer); List output1 = Arrays.asList(probe1.receiveN(3)); List output2 = Arrays.asList(probe2.receiveN(3)); @@ -150,24 +161,31 @@ public class FlowGraphTest extends StreamTest { final JavaTestKit probe1 = new JavaTestKit(system); final JavaTestKit probe2 = new JavaTestKit(system); - final Builder b = FlowGraph.builder(); - final Source in = Source.single(1); + RunnableGraph.fromGraph(FlowGraph.create( + new Function, ClosedShape>() { + @Override + public ClosedShape apply(final Builder b) throws Exception { + final Source in = Source.single(1); - final FanOutShape2 unzip = b.graph(UnzipWith.create( - new Function>() { - @Override public Pair apply(Integer l) throws Exception { - return new Pair(l + "!", l); + final FanOutShape2 unzip = b.add(UnzipWith.create( + new Function>() { + @Override + public Pair apply(Integer l) throws Exception { + return new Pair(l + "!", l); + } + }) + ); + + final SinkShape out1 = b.add(FlowGraphTest.createSink(probe1)); + final SinkShape out2 = b.add(FlowGraphTest.createSink(probe2)); + + b.from(b.add(in)).toInlet(unzip.in()); + b.from(unzip.out0()).to(out1); + b.from(unzip.out1()).to(out2); + return ClosedShape.getInstance(); } - }) - ); - - final Sink out1 = createSink(probe1); - final Sink out2 = createSink(probe2); - - b.edge(b.source(in), unzip.in()); - b.edge(unzip.out0(), b.sink(out1)); - b.edge(unzip.out1(), b.sink(out2)); - b.run(materializer); + } + )).run(materializer); Duration d = Duration.create(300, TimeUnit.MILLISECONDS); @@ -186,28 +204,34 @@ public class FlowGraphTest extends StreamTest { final JavaTestKit probe3 = new JavaTestKit(system); final JavaTestKit probe4 = new JavaTestKit(system); - final Builder b = FlowGraph.builder(); - final Source in = Source.single(1); + RunnableGraph.fromGraph(FlowGraph.create( + new Function, ClosedShape>() { + @Override + public ClosedShape apply(final Builder b) throws Exception { + final Source in = Source.single(1); - final FanOutShape4 unzip = b.graph(UnzipWith.create4( - new Function>() { - @Override public Tuple4 apply(Integer l) throws Exception { - return new Tuple4(l.toString(), l, l + "+" + l, l + l); + final FanOutShape4 unzip = b.add(UnzipWith.create4( + new Function>() { + @Override + public Tuple4 apply(Integer l) throws Exception { + return new Tuple4(l.toString(), l, l + "+" + l, l + l); + } + }) + ); + + final SinkShape out1 = b.add(FlowGraphTest.createSink(probe1)); + final SinkShape out2 = b.add(FlowGraphTest.createSink(probe2)); + final SinkShape out3 = b.add(FlowGraphTest.createSink(probe3)); + final SinkShape out4 = b.add(FlowGraphTest.createSink(probe4)); + + b.from(b.add(in)).toInlet(unzip.in()); + b.from(unzip.out0()).to(out1); + b.from(unzip.out1()).to(out2); + b.from(unzip.out2()).to(out3); + b.from(unzip.out3()).to(out4); + return ClosedShape.getInstance(); } - }) - ); - - final Sink out1 = createSink(probe1); - final Sink out2 = createSink(probe2); - final Sink out3 = createSink(probe3); - final Sink out4 = createSink(probe4); - - b.edge(b.source(in), unzip.in()); - b.edge(unzip.out0(), b.sink(out1)); - b.edge(unzip.out1(), b.sink(out2)); - b.edge(unzip.out2(), b.sink(out3)); - b.edge(unzip.out3(), b.sink(out4)); - b.run(materializer); + })).run(materializer); Duration d = Duration.create(300, TimeUnit.MILLISECONDS); @@ -234,15 +258,17 @@ public class FlowGraphTest extends StreamTest { } }); - final Future future = FlowGraph.factory().closed(Sink. head(), new Procedure2 >, SinkShape>() { + final Future future = RunnableGraph.fromGraph(FlowGraph.create(Sink.head(), + new Function2>, SinkShape, ClosedShape>() { @Override - public void apply(Builder > b, SinkShape out) throws Exception { - final FanInShape2 zip = b.graph(sumZip); - b.edge(b.source(in1), zip.in0()); - b.edge(b.source(in2), zip.in1()); - b.edge(zip.out(), out.inlet()); + public ClosedShape apply(Builder> b, SinkShape out) throws Exception { + final FanInShape2 zip = b.add(sumZip); + b.from(b.add(in1)).toInlet(zip.in0()); + b.from(b.add(in2)).toInlet(zip.in1()); + b.from(zip.out()).to(out); + return ClosedShape.getInstance(); } - }).run(materializer); + })).run(materializer); final Integer result = Await.result(future, Duration.create(300, TimeUnit.MILLISECONDS)); assertEquals(11, (int) result); @@ -262,17 +288,20 @@ public class FlowGraphTest extends StreamTest { } }); - final Future future = FlowGraph.factory().closed(Sink. head(), new Procedure2>, SinkShape>() { + final Future future = RunnableGraph.fromGraph( + FlowGraph.create(Sink.head(), + new Function2>, SinkShape, ClosedShape>() { @Override - public void apply(Builder> b, SinkShape out) throws Exception { - final FanInShape4 zip = b.graph(sumZip); - b.edge(b.source(in1), zip.in0()); - b.edge(b.source(in2), zip.in1()); - b.edge(b.source(in3), zip.in2()); - b.edge(b.source(in4), zip.in3()); - b.edge(zip.out(), out.inlet()); + public ClosedShape apply(Builder> b, SinkShape out) throws Exception { + final FanInShape4 zip = b.add(sumZip); + b.from(b.add(in1)).toInlet(zip.in0()); + b.from(b.add(in2)).toInlet(zip.in1()); + b.from(b.add(in3)).toInlet(zip.in2()); + b.from(b.add(in4)).toInlet(zip.in3()); + b.from(zip.out()).to(out); + return ClosedShape.getInstance(); } - }).run(materializer); + })).run(materializer); final Integer result = Await.result(future, Duration.create(300, TimeUnit.MILLISECONDS)); assertEquals(1111, (int) result); @@ -284,17 +313,19 @@ public class FlowGraphTest extends StreamTest { final Source in1 = Source.single(1); final TestProbe probe = TestProbe.apply(system); - final Future future = FlowGraph.factory().closed(Sink. head(), new Procedure2>, SinkShape>() { + final Future future = RunnableGraph.fromGraph( + FlowGraph.create(Sink. head(), new Function2>, SinkShape, ClosedShape>() { @Override - public void apply(Builder> b, SinkShape out) throws Exception { - b.from(b.graph(Source.single(1))).to(out); - b.from(b.materializedValue()).to(b.graph(Sink.foreach(new Procedure>(){ + public ClosedShape apply(Builder> b, SinkShape out) throws Exception { + b.from(b.add(Source.single(1))).to(out); + b.from(b.materializedValue()).to(b.add(Sink.foreach(new Procedure>(){ public void apply(Future mat) throws Exception { Patterns.pipe(mat, system.dispatcher()).to(probe.ref()); } }))); + return ClosedShape.getInstance(); } - }).run(materializer); + })).run(materializer); final Integer result = Await.result(future, Duration.create(300, TimeUnit.MILLISECONDS)); assertEquals(1, (int) result); diff --git a/akka-stream-tests/src/test/java/akka/stream/javadsl/FlowTest.java b/akka-stream-tests/src/test/java/akka/stream/javadsl/FlowTest.java index d2a5879fa0..ebaa33bbba 100644 --- a/akka-stream-tests/src/test/java/akka/stream/javadsl/FlowTest.java +++ b/akka-stream-tests/src/test/java/akka/stream/javadsl/FlowTest.java @@ -385,15 +385,16 @@ public class FlowTest extends StreamTest { final Sink> publisher = Sink.publisher(); - final Source source = Source.factory().create(new Function, Outlet>() { - @Override - public Outlet apply(Builder b) throws Exception { - final UniformFanInShape merge = b.graph(Merge. create(2)); - b.flow(b.source(in1), f1, merge.in(0)); - b.flow(b.source(in2), f2, merge.in(1)); - return merge.out(); - } - }); + final Source source = Source.fromGraph( + FlowGraph.create(new Function, SourceShape>() { + @Override + public SourceShape apply(Builder b) throws Exception { + final UniformFanInShape merge = b.add(Merge.create(2)); + b.from(b.add(in1)).via(b.add(f1)).toInlet(merge.in(0)); + b.from(b.add(in2)).via(b.add(f2)).toInlet(merge.in(1)); + return new SourceShape(merge.out()); + } + })); // collecting final Publisher pub = source.runWith(publisher, materializer); @@ -409,23 +410,25 @@ public class FlowTest extends StreamTest { final Iterable input1 = Arrays.asList("A", "B", "C"); final Iterable input2 = Arrays.asList(1, 2, 3); - final Builder b = FlowGraph.builder(); - final Outlet in1 = b.source(Source.from(input1)); - final Outlet in2 = b.source(Source.from(input2)); - final FanInShape2> zip = b.graph(Zip. create()); - final Inlet> out = b.sink(Sink - .foreach(new Procedure>() { - @Override - public void apply(Pair param) throws Exception { - probe.getRef().tell(param, ActorRef.noSender()); - } - })); - - b.edge(in1, zip.in0()); - b.edge(in2, zip.in1()); - b.edge(zip.out(), out); - - b.run(materializer); + RunnableGraph.fromGraph(FlowGraph.create(new Function, ClosedShape>(){ + public ClosedShape apply(Builder b) { + final Outlet in1 = b.add(Source.from(input1)).outlet(); + final Outlet in2 = b.add(Source.from(input2)).outlet(); + final FanInShape2> zip = b.add(Zip.create()); + final SinkShape> out = + b.add(Sink.foreach(new Procedure>() { + @Override + public void apply(Pair param) throws Exception { + probe.getRef().tell(param, ActorRef.noSender()); + } + })); + + b.from(in1).toInlet(zip.in0()); + b.from(in2).toInlet(zip.in1()); + b.from(zip.out()).to(out); + return ClosedShape.getInstance(); + } + })).run(materializer); List output = Arrays.asList(probe.receiveN(3)); @SuppressWarnings("unchecked") @@ -638,19 +641,18 @@ public class FlowTest extends StreamTest { @Test public void mustBeAbleToBroadcastEagerCancel() throws Exception { - final Sink out1 = Sink.cancelled(); - final Sink out2 = Sink.ignore(); - - final Sink sink = Sink.factory().create(new Function, Inlet>() { - @Override - public Inlet apply(Builder b) throws Exception { - final UniformFanOutShape broadcast = b.graph(Broadcast.create(2, true)); - - b.from(broadcast.out(0)).to(b.graph(out1)); - b.from(broadcast.out(1)).to(b.graph(out2)); - return broadcast.in(); - } - }); + final Sink sink = Sink.fromGraph( + FlowGraph.create(new Function, SinkShape>() { + @Override + public SinkShape apply(Builder b) throws Exception { + final UniformFanOutShape broadcast = b.add(Broadcast.create(2, true)); + final SinkShape out1 = b.add(Sink.cancelled()); + final SinkShape out2 = b.add(Sink.ignore()); + b.from(broadcast.out(0)).to(out1); + b.from(broadcast.out(1)).to(out2); + return new SinkShape(broadcast.in()); + } + })); final JavaTestKit probe = new JavaTestKit(system); Source source = Source.actorRef(1, OverflowStrategy.dropNew()); diff --git a/akka-stream-tests/src/test/java/akka/stream/javadsl/TcpTest.java b/akka-stream-tests/src/test/java/akka/stream/javadsl/TcpTest.java index cc226ebe77..d1fe0bf758 100644 --- a/akka-stream-tests/src/test/java/akka/stream/javadsl/TcpTest.java +++ b/akka-stream-tests/src/test/java/akka/stream/javadsl/TcpTest.java @@ -35,7 +35,7 @@ public class TcpTest extends StreamTest { final Sink> echoHandler = Sink.foreach(new Procedure() { public void apply(IncomingConnection conn) { - conn.handleWith(Flow.empty(), materializer); + conn.handleWith(Flow.of(ByteString.class), materializer); } }); diff --git a/akka-stream-tests/src/test/scala/akka/stream/ActorMaterializerSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/ActorMaterializerSpec.scala index b006971a77..57959b8600 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/ActorMaterializerSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/ActorMaterializerSpec.scala @@ -26,7 +26,7 @@ class ActorMaterializerSpec extends AkkaSpec with ImplicitSender { val m = ActorMaterializer.create(system) - val f = Source.lazyEmpty[Int].runFold(0)(_ + _)(m) + val f = Source.maybe[Int].runFold(0)(_ + _)(m) m.shutdown() @@ -43,7 +43,7 @@ class ActorMaterializerSpec extends AkkaSpec with ImplicitSender { "shut down the supervisor actor it encapsulates" in { val m = ActorMaterializer.create(system).asInstanceOf[ActorMaterializerImpl] - Source.lazyEmpty[Any].to(Sink.ignore).run()(m) + Source.maybe[Any].to(Sink.ignore).run()(m) m.supervisor ! StreamSupervisor.GetChildren expectMsgType[StreamSupervisor.Children] m.shutdown() diff --git a/akka-stream-tests/src/test/scala/akka/stream/DslFactoriesConsistencySpec.scala b/akka-stream-tests/src/test/scala/akka/stream/DslFactoriesConsistencySpec.scala index ce9400a9d0..cff1129312 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/DslFactoriesConsistencySpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/DslFactoriesConsistencySpec.scala @@ -66,11 +66,11 @@ class DslFactoriesConsistencySpec extends WordSpec with Matchers { } val testCases = Seq( - TestCase("Source", scaladsl.Source.getClass, javadsl.Source.getClass, classOf[javadsl.SourceCreate]), - TestCase("Flow", scaladsl.Flow.getClass, javadsl.Flow.getClass, classOf[javadsl.FlowCreate]), - TestCase("Sink", scaladsl.Sink.getClass, javadsl.Sink.getClass, classOf[javadsl.SinkCreate]), + TestCase("Source", scaladsl.Source.getClass, javadsl.Source.getClass), + TestCase("Flow", scaladsl.Flow.getClass, javadsl.Flow.getClass), + TestCase("Sink", scaladsl.Sink.getClass, javadsl.Sink.getClass), + TestCase("BidiFlow", scaladsl.BidiFlow.getClass, javadsl.BidiFlow.getClass), TestCase("FlowGraph", scaladsl.FlowGraph.getClass, javadsl.FlowGraph.getClass, classOf[javadsl.GraphCreate]), - TestCase("BidiFlow", scaladsl.BidiFlow.getClass, javadsl.BidiFlow.getClass, classOf[javadsl.BidiFlowCreate]), TestCase("ZipWith", Some(scaladsl.ZipWith.getClass), None, Some(javadsl.ZipWith.getClass)), TestCase("Merge", scaladsl.Merge.getClass, javadsl.Merge.getClass), TestCase("MergePreferred", scaladsl.MergePreferred.getClass, javadsl.MergePreferred.getClass), @@ -113,8 +113,8 @@ class DslFactoriesConsistencySpec extends WordSpec with Matchers { Ignore(_ == akka.stream.scaladsl.Flow.getClass, _ == "apply", _ == 24, _ ⇒ true), Ignore(_ == akka.stream.scaladsl.Sink.getClass, _ == "apply", _ == 24, _ ⇒ true), Ignore(_ == akka.stream.scaladsl.BidiFlow.getClass, _ == "apply", _ == 24, _ ⇒ true), - Ignore(_ == akka.stream.scaladsl.FlowGraph.getClass, _ == "closed", _ == 24, _ ⇒ true), - Ignore(_ == akka.stream.scaladsl.FlowGraph.getClass, _ == "partial", _ == 24, _ ⇒ true), + Ignore(_ == akka.stream.scaladsl.FlowGraph.getClass, _ == "runnable", _ == 24, _ ⇒ true), + Ignore(_ == akka.stream.scaladsl.FlowGraph.getClass, _ == "create", _ == 24, _ ⇒ true), // all generated methods like scaladsl.Sink$.akka$stream$scaladsl$Sink$$newOnCompleteStage$1 Ignore(_ ⇒ true, _.contains("$"), _ ⇒ true, _ ⇒ true)) @@ -127,8 +127,8 @@ class DslFactoriesConsistencySpec extends WordSpec with Matchers { /** * Rename * createN => create - * closedN => closed - * partialN => partial + * runnableN => runnable + * createN => create */ private val unspecializeName: PartialFunction[Method, Method] = { case m ⇒ m.copy(name = m.name.filter(Character.isLetter)) diff --git a/akka-stream-tests/src/test/scala/akka/stream/actor/ActorPublisherSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/actor/ActorPublisherSpec.scala index 4343b1f061..f4e516a8c8 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/actor/ActorPublisherSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/actor/ActorPublisherSpec.scala @@ -4,7 +4,7 @@ package akka.stream.actor import akka.actor.{ ActorRef, PoisonPill, Props } -import akka.stream.{ ActorMaterializer, ActorMaterializerSettings, ActorAttributes } +import akka.stream.{ ClosedShape, ActorMaterializer, ActorMaterializerSettings, ActorAttributes } import akka.stream.scaladsl._ import akka.stream.testkit._ import akka.stream.testkit.Utils._ @@ -350,7 +350,7 @@ class ActorPublisherSpec extends AkkaSpec(ActorPublisherSpec.config) with Implic val sink1 = Sink(ActorSubscriber[String](system.actorOf(receiverProps(probe1.ref)))) val sink2: Sink[String, ActorRef] = Sink.actorSubscriber(receiverProps(probe2.ref)) - val senderRef2 = FlowGraph.closed(Source.actorPublisher[Int](senderProps)) { implicit b ⇒ + val senderRef2 = RunnableGraph.fromGraph(FlowGraph.create(Source.actorPublisher[Int](senderProps)) { implicit b ⇒ source2 ⇒ import FlowGraph.Implicits._ @@ -364,7 +364,8 @@ class ActorPublisherSpec extends AkkaSpec(ActorPublisherSpec.config) with Implic bcast.out(0).map(_ + "mark") ~> sink1 bcast.out(1) ~> sink2 - }.run() + ClosedShape + }).run() (0 to 10).foreach { senderRef1 ! _ diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/ActorGraphInterpreterSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/ActorGraphInterpreterSpec.scala index 3a761595aa..cc4ba30309 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/ActorGraphInterpreterSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/ActorGraphInterpreterSpec.scala @@ -72,7 +72,7 @@ class ActorGraphInterpreterSpec extends AkkaSpec { override def toString = "IdentityBidi" } - val identity = BidiFlow.wrap(identityBidi).join(Flow[Int].map { x ⇒ x }) + val identity = BidiFlow.fromGraph(identityBidi).join(Flow[Int].map { x ⇒ x }) Await.result( Source(1 to 10).via(identity).grouped(100).runWith(Sink.head), @@ -117,7 +117,7 @@ class ActorGraphInterpreterSpec extends AkkaSpec { override def toString = "IdentityBidi" } - val identityBidiF = BidiFlow.wrap(identityBidi) + val identityBidiF = BidiFlow.fromGraph(identityBidi) val identity = (identityBidiF atop identityBidiF atop identityBidiF).join(Flow[Int].map { x ⇒ x }) Await.result( @@ -163,7 +163,7 @@ class ActorGraphInterpreterSpec extends AkkaSpec { override def toString = "IdentityBidi" } - val identityBidiF = BidiFlow.wrap(identityBidi) + val identityBidiF = BidiFlow.fromGraph(identityBidi) val identity = (identityBidiF atop identityBidiF atop identityBidiF).join(Flow[Int].map { x ⇒ x }) Await.result( @@ -214,7 +214,7 @@ class ActorGraphInterpreterSpec extends AkkaSpec { val takeAll = Flow[Int].grouped(200).toMat(Sink.head)(Keep.right) - val (f1, f2) = FlowGraph.closed(takeAll, takeAll)(Keep.both) { implicit b ⇒ + val (f1, f2) = RunnableGraph.fromGraph(FlowGraph.create(takeAll, takeAll)(Keep.both) { implicit b ⇒ (out1, out2) ⇒ import FlowGraph.Implicits._ val bidi = b.add(rotatedBidi) @@ -224,7 +224,8 @@ class ActorGraphInterpreterSpec extends AkkaSpec { bidi.in2 <~ Source(1 to 100) bidi.out1 ~> out1 - }.run() + ClosedShape + }).run() Await.result(f1, 3.seconds) should ===(1 to 100) Await.result(f2, 3.seconds) should ===(1 to 10) diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/FramingSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/io/FramingSpec.scala index 34ca9fe8ae..0e43042b90 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/io/FramingSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/io/FramingSpec.scala @@ -216,7 +216,7 @@ class FramingSpec extends AkkaSpec { } "support simple framing adapter" in { - val rechunkBidi = BidiFlow.wrap(rechunk, rechunk)(Keep.left) + val rechunkBidi = BidiFlow.fromFlowsMat(rechunk, rechunk)(Keep.left) val codecFlow = Framing.simpleFramingProtocol(1024) .atop(rechunkBidi) diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/TcpSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/io/TcpSpec.scala index 63793cbd21..e3503a7012 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/io/TcpSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/io/TcpSpec.scala @@ -342,7 +342,7 @@ class TcpSpec extends AkkaSpec("akka.io.tcp.windows-connection-abort-workaround- "properly full-close if requested" in assertAllStagesStopped { val serverAddress = temporaryServerAddress() val writeButIgnoreRead: Flow[ByteString, ByteString, Unit] = - Flow.wrap(Sink.ignore, Source.single(ByteString("Early response")))(Keep.right) + Flow.fromSinkAndSourceMat(Sink.ignore, Source.single(ByteString("Early response")))(Keep.right) val binding = Await.result( @@ -350,7 +350,7 @@ class TcpSpec extends AkkaSpec("akka.io.tcp.windows-connection-abort-workaround- conn.flow.join(writeButIgnoreRead).run() })(Keep.left).run(), 3.seconds) - val result = Source.lazyEmpty[ByteString] + val result = Source.maybe[ByteString] .via(Tcp().outgoingConnection(serverAddress.getHostName, serverAddress.getPort)) .runFold(ByteString.empty)(_ ++ _) @@ -385,7 +385,7 @@ class TcpSpec extends AkkaSpec("akka.io.tcp.windows-connection-abort-workaround- val serverAddress = temporaryServerAddress() val binding = Tcp(system2).bindAndHandle(Flow[ByteString], serverAddress.getHostName, serverAddress.getPort)(mat2) - val result = Source.lazyEmpty[ByteString].via(Tcp(system2).outgoingConnection(serverAddress)).runFold(0)(_ + _.size)(mat2) + val result = Source.maybe[ByteString].via(Tcp(system2).outgoingConnection(serverAddress)).runFold(0)(_ + _.size)(mat2) // Getting rid of existing connection actors by using a blunt instrument system2.actorSelection(akka.io.Tcp(system2).getManager.path / "selectors" / "$a" / "*") ! Kill diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/TimeoutsSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/io/TimeoutsSpec.scala index 1a980e2e3e..38b6637686 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/io/TimeoutsSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/io/TimeoutsSpec.scala @@ -6,7 +6,7 @@ package akka.stream.io import java.util.concurrent.TimeoutException -import akka.stream.ActorMaterializer +import akka.stream.{ ClosedShape, ActorMaterializer } import akka.stream.scaladsl._ import akka.stream.testkit.{ AkkaSpec, TestSubscriber, TestPublisher } import scala.concurrent.{ Future, Await } @@ -37,7 +37,7 @@ class TimeoutsSpec extends AkkaSpec { "fail if no initial element passes until timeout" in assertAllStagesStopped { val downstreamProbe = TestSubscriber.probe[Int]() - Source.lazyEmpty[Int] + Source.maybe[Int] .via(Timeouts.initalTimeout(1.seconds)) .runWith(Sink(downstreamProbe)) @@ -142,8 +142,8 @@ class TimeoutsSpec extends AkkaSpec { val upstreamWriter = TestPublisher.probe[Int]() val downstreamWriter = TestPublisher.probe[String]() - val upstream = Flow.wrap(Sink.ignore, Source(upstreamWriter))(Keep.left) - val downstream = Flow.wrap(Sink.ignore, Source(downstreamWriter))(Keep.left) + val upstream = Flow.fromSinkAndSourceMat(Sink.ignore, Source(upstreamWriter))(Keep.left) + val downstream = Flow.fromSinkAndSourceMat(Sink.ignore, Source(downstreamWriter))(Keep.left) val assembly: RunnableGraph[(Future[Unit], Future[Unit])] = upstream .joinMat(Timeouts.idleTimeoutBidi[Int, String](2.seconds))(Keep.left) @@ -172,14 +172,15 @@ class TimeoutsSpec extends AkkaSpec { val downWrite = TestPublisher.probe[Int]() val downRead = TestSubscriber.probe[String]() - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ import FlowGraph.Implicits._ val timeoutStage = b.add(Timeouts.idleTimeoutBidi[String, Int](2.seconds)) Source(upWrite) ~> timeoutStage.in1; timeoutStage.out1 ~> Sink(downRead) Sink(upRead) <~ timeoutStage.out2; timeoutStage.in2 <~ Source(downWrite) - }.run() + ClosedShape + }).run() // Request enough for the whole test upRead.request(100) @@ -219,14 +220,15 @@ class TimeoutsSpec extends AkkaSpec { val downWrite = TestPublisher.probe[Int]() val downRead = TestSubscriber.probe[String]() - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ import FlowGraph.Implicits._ val timeoutStage = b.add(Timeouts.idleTimeoutBidi[String, Int](2.seconds)) Source(upWrite) ~> timeoutStage.in1; timeoutStage.out1 ~> Sink(downRead) Sink(upRead) <~ timeoutStage.out2; timeoutStage.in2 <~ Source(downWrite) - }.run() + ClosedShape + }).run() val te = TE("test") diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/TlsSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/io/TlsSpec.scala index 85cdc41f0d..85289179c6 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/io/TlsSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/io/TlsSpec.scala @@ -12,7 +12,7 @@ import scala.util.Random import akka.actor.ActorSystem import akka.pattern.{ after ⇒ later } -import akka.stream.ActorMaterializer +import akka.stream.{ ClosedShape, ActorMaterializer } import akka.stream.scaladsl._ import akka.stream.stage._ import akka.stream.testkit._ @@ -385,7 +385,7 @@ class TlsSpec extends AkkaSpec("akka.loglevel=INFO\nakka.actor.debug.receive=off .recover { case e: SSLException ⇒ Right(e) } .collect { case Right(e) ⇒ e }.toMat(Sink.head)(Keep.right) - val simple = Flow.wrap(getError, Source.lazyEmpty[SslTlsOutbound])(Keep.left) + val simple = Flow.fromSinkAndSourceMat(getError, Source.maybe[SslTlsOutbound])(Keep.left) // The creation of actual TCP connections is necessary. It is the easiest way to decouple the client and server // under error conditions, and has the bonus of matching most actual SSL deployments. @@ -406,12 +406,13 @@ 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) = - FlowGraph.closed(Source.subscriber[SslTlsOutbound], Sink.head[ByteString], Sink.head[SslTlsInbound])((_, _, _)) { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create(Source.subscriber[SslTlsOutbound], Sink.head[ByteString], Sink.head[SslTlsInbound])((_, _, _)) { implicit b ⇒ (s, o1, o2) ⇒ val tls = b.add(clientTls(EagerClose)) s ~> tls.in1; tls.out1 ~> o1 o2 <~ tls.out2; tls.in2 <~ Source.failed(ex) - }.run() + ClosedShape + }).run() the[Exception] thrownBy Await.result(out1, 1.second) should be(ex) the[Exception] thrownBy Await.result(out2, 1.second) should be(ex) Thread.sleep(500) @@ -423,12 +424,13 @@ 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) = - FlowGraph.closed(Source.subscriber[ByteString], Sink.head[ByteString], Sink.head[SslTlsInbound])((_, _, _)) { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create(Source.subscriber[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 o2 <~ tls.out2; tls.in2 <~ s - }.run() + ClosedShape + }).run() the[Exception] thrownBy Await.result(out1, 1.second) should be(ex) the[Exception] thrownBy Await.result(out2, 1.second) should be(ex) Thread.sleep(500) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/BidiFlowSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/BidiFlowSpec.scala index 5b546cce24..b41196843d 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/BidiFlowSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/BidiFlowSpec.scala @@ -7,12 +7,10 @@ import akka.stream.testkit.AkkaSpec import akka.stream.testkit.Utils._ import org.scalactic.ConversionCheckedTripleEquals import akka.util.ByteString -import akka.stream.BidiShape -import akka.stream.ActorMaterializer +import akka.stream._ import scala.concurrent.Await import scala.concurrent.duration._ import scala.collection.immutable -import akka.stream.Attributes class BidiFlowSpec extends AkkaSpec with ConversionCheckedTripleEquals { import Attributes._ @@ -20,26 +18,22 @@ class BidiFlowSpec extends AkkaSpec with ConversionCheckedTripleEquals { implicit val mat = ActorMaterializer() - val bidi = BidiFlow() { b ⇒ - val top = b.add(Flow[Int].map(x ⇒ x.toLong + 2).withAttributes(name("top"))) - val bottom = b.add(Flow[ByteString].map(_.decodeString("UTF-8")).withAttributes(name("bottom"))) - BidiShape(top.inlet, top.outlet, bottom.inlet, bottom.outlet) - } + val bidi = BidiFlow.fromFlows( + Flow[Int].map(x ⇒ x.toLong + 2).withAttributes(name("top")), + Flow[ByteString].map(_.decodeString("UTF-8")).withAttributes(name("bottom"))) - val inverse = BidiFlow() { b ⇒ - val top = b.add(Flow[Long].map(x ⇒ x.toInt + 2).withAttributes(name("top"))) - val bottom = b.add(Flow[String].map(ByteString(_)).withAttributes(name("bottom"))) - BidiShape(top.inlet, top.outlet, bottom.inlet, bottom.outlet) - } + val inverse = BidiFlow.fromFlows( + Flow[Long].map(x ⇒ x.toInt + 2).withAttributes(name("top")), + Flow[String].map(ByteString(_)).withAttributes(name("bottom"))) - val bidiMat = BidiFlow(Sink.head[Int]) { implicit b ⇒ + val bidiMat = BidiFlow.fromGraph(FlowGraph.create(Sink.head[Int]) { implicit b ⇒ s ⇒ Source.single(42) ~> s val top = b.add(Flow[Int].map(x ⇒ x.toLong + 2)) val bottom = b.add(Flow[ByteString].map(_.decodeString("UTF-8"))) BidiShape(top.inlet, top.outlet, bottom.inlet, bottom.outlet) - } + }) val str = "Hello World" val bytes = ByteString(str) @@ -47,13 +41,14 @@ class BidiFlowSpec extends AkkaSpec with ConversionCheckedTripleEquals { "A BidiFlow" must { "work top/bottom in isolation" in { - val (top, bottom) = FlowGraph.closed(Sink.head[Long], Sink.head[String])(Keep.both) { implicit b ⇒ + val (top, bottom) = RunnableGraph.fromGraph(FlowGraph.create(Sink.head[Long], Sink.head[String])(Keep.both) { implicit b ⇒ (st, sb) ⇒ val s = b.add(bidi) Source.single(1) ~> s.in1; s.out1 ~> st sb <~ s.out2; s.in2 <~ Source.single(bytes) - }.run() + ClosedShape + }).run() Await.result(top, 1.second) should ===(3) Await.result(bottom, 1.second) should ===(str) @@ -85,15 +80,16 @@ class BidiFlowSpec extends AkkaSpec with ConversionCheckedTripleEquals { } "materialize to its value" in { - val f = FlowGraph.closed(bidiMat) { implicit b ⇒ + val f = RunnableGraph.fromGraph(FlowGraph.create(bidiMat) { implicit b ⇒ bidi ⇒ Flow[String].map(Integer.valueOf(_).toInt) <~> bidi <~> Flow[Long].map(x ⇒ ByteString(s"Hello $x")) - }.run() + ClosedShape + }).run() Await.result(f, 1.second) should ===(42) } "combine materialization values" in assertAllStagesStopped { - val left = Flow(Sink.head[Int]) { implicit b ⇒ + val left = Flow.fromGraph(FlowGraph.create(Sink.head[Int]) { implicit b ⇒ sink ⇒ val bcast = b.add(Broadcast[Int](2)) val merge = b.add(Merge[Int](2)) @@ -101,14 +97,14 @@ class BidiFlowSpec extends AkkaSpec with ConversionCheckedTripleEquals { bcast ~> sink Source.single(1) ~> bcast ~> merge flow ~> merge - (flow.inlet, merge.out) - } - val right = Flow(Sink.head[immutable.Seq[Long]]) { implicit b ⇒ + FlowShape(flow.inlet, merge.out) + }) + val right = Flow.fromGraph(FlowGraph.create(Sink.head[immutable.Seq[Long]]) { implicit b ⇒ sink ⇒ val flow = b.add(Flow[Long].grouped(10)) flow ~> sink - (flow.inlet, b.add(Source.single(ByteString("10")))) - } + FlowShape(flow.inlet, b.add(Source.single(ByteString("10")))) + }) val ((l, m), r) = left.joinMat(bidiMat)(Keep.both).joinMat(right)(Keep.both).run() Await.result(l, 1.second) should ===(1) Await.result(m, 1.second) should ===(42) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGraphCompileSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGraphCompileSpec.scala index 55a53a62fd..88faebfdef 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGraphCompileSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGraphCompileSpec.scala @@ -4,8 +4,7 @@ package akka.stream.scaladsl import akka.stream.Attributes._ -import akka.stream.ActorMaterializer -import akka.stream.OverflowStrategy +import akka.stream.{ ClosedShape, ActorMaterializer, OverflowStrategy } import akka.stream.testkit._ import akka.stream.stage._ @@ -41,47 +40,52 @@ class FlowGraphCompileSpec extends AkkaSpec { val out2 = Sink.head[String] "A Graph" should { + import FlowGraph.Implicits._ "build simple merge" in { - FlowGraph.closed() { b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val merge = b.add(Merge[String](2)) - b.addEdge(b.add(in1), f1, merge.in(0)) - b.addEdge(b.add(in2), f2, merge.in(1)) - b.addEdge(merge.out, f3, b.add(out1)) - }.run() + in1 ~> f1 ~> merge.in(0) + in2 ~> f2 ~> merge.in(1) + merge.out ~> f3 ~> out1 + ClosedShape + }).run() } "build simple broadcast" in { - FlowGraph.closed() { b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val bcast = b.add(Broadcast[String](2)) - b.addEdge(b.add(in1), f1, bcast.in) - b.addEdge(bcast.out(0), f2, b.add(out1)) - b.addEdge(bcast.out(1), f3, b.add(out2)) - }.run() + in1 ~> f1 ~> bcast.in + bcast.out(0) ~> f2 ~> out1 + bcast.out(1) ~> f3 ~> out2 + ClosedShape + }).run() } "build simple balance" in { - FlowGraph.closed() { b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val balance = b.add(Balance[String](2)) - b.addEdge(b.add(in1), f1, balance.in) - b.addEdge(balance.out(0), f2, b.add(out1)) - b.addEdge(balance.out(1), f3, b.add(out2)) - } + in1 ~> f1 ~> balance.in + balance.out(0) ~> f2 ~> out1 + balance.out(1) ~> f3 ~> out2 + ClosedShape + }) } "build simple merge - broadcast" in { - FlowGraph.closed() { b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val merge = b.add(Merge[String](2)) val bcast = b.add(Broadcast[String](2)) - b.addEdge(b.add(in1), f1, merge.in(0)) - b.addEdge(b.add(in2), f2, merge.in(1)) - b.addEdge(merge.out, f3, bcast.in) - b.addEdge(bcast.out(0), f4, b.add(out1)) - b.addEdge(bcast.out(1), f5, b.add(out2)) - }.run() + in1 ~> f1 ~> merge.in(0) + in2 ~> f2 ~> merge.in(1) + merge ~> f3 ~> bcast + bcast.out(0) ~> f4 ~> out1 + bcast.out(1) ~> f5 ~> out2 + ClosedShape + }).run() } "build simple merge - broadcast with implicits" in { - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ import FlowGraph.Implicits._ val merge = b.add(Merge[String](2)) val bcast = b.add(Broadcast[String](2)) @@ -90,7 +94,8 @@ class FlowGraphCompileSpec extends AkkaSpec { bcast.out(0) ~> f3 ~> b.add(out1) b.add(in2) ~> f4 ~> merge.in(1) bcast.out(1) ~> f5 ~> b.add(out2) - }.run() + ClosedShape + }).run() } /* @@ -105,24 +110,25 @@ class FlowGraphCompileSpec extends AkkaSpec { "detect cycle in " in { pending // FIXME needs cycle detection capability intercept[IllegalArgumentException] { - FlowGraph.closed() { b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val merge = b.add(Merge[String](2)) val bcast1 = b.add(Broadcast[String](2)) val bcast2 = b.add(Broadcast[String](2)) val feedbackLoopBuffer = Flow[String].buffer(10, OverflowStrategy.dropBuffer) - b.addEdge(b.add(in1), f1, merge.in(0)) - b.addEdge(merge.out, f2, bcast1.in) - b.addEdge(bcast1.out(0), f3, b.add(out1)) - b.addEdge(bcast1.out(1), feedbackLoopBuffer, bcast2.in) - b.addEdge(bcast2.out(0), f5, merge.in(1)) // cycle - b.addEdge(bcast2.out(1), f6, b.add(out2)) - } + in1 ~> f1 ~> merge.in(0) + merge ~> f2 ~> bcast1 + bcast1.out(0) ~> f3 ~> out1 + bcast1.out(1) ~> feedbackLoopBuffer ~> bcast2 + bcast2.out(0) ~> f5 ~> merge.in(1) // cycle + bcast2.out(1) ~> f6 ~> out2 + ClosedShape + }) }.getMessage.toLowerCase should include("cycle") } "express complex topologies in a readable way" in { - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val merge = b.add(Merge[String](2)) val bcast1 = b.add(Broadcast[String](2)) val bcast2 = b.add(Broadcast[String](2)) @@ -131,22 +137,24 @@ class FlowGraphCompileSpec extends AkkaSpec { b.add(in1) ~> f1 ~> merge ~> f2 ~> bcast1 ~> f3 ~> b.add(out1) bcast1 ~> feedbackLoopBuffer ~> bcast2 ~> f5 ~> merge bcast2 ~> f6 ~> b.add(out2) - }.run() + ClosedShape + }).run() } "build broadcast - merge" in { - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val bcast = b.add(Broadcast[String](2)) val merge = b.add(Merge[String](2)) import FlowGraph.Implicits._ in1 ~> f1 ~> bcast ~> f2 ~> merge ~> f3 ~> out1 bcast ~> f4 ~> merge - }.run() + ClosedShape + }).run() } "build wikipedia Topological_sorting" in { // see https://en.wikipedia.org/wiki/Topological_sorting#mediaviewer/File:Directed_acyclic_graph.png - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val b3 = b.add(Broadcast[String](2)) val b7 = b.add(Broadcast[String](2)) val b11 = b.add(Broadcast[String](3)) @@ -170,22 +178,24 @@ class FlowGraphCompileSpec extends AkkaSpec { in5 ~> f("k") ~> m11 in3 ~> f("l") ~> b3 ~> f("m") ~> m8 b3 ~> f("n") ~> m10 - }.run() + ClosedShape + }).run() } "make it optional to specify flows" in { - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val merge = b.add(Merge[String](2)) val bcast = b.add(Broadcast[String](2)) import FlowGraph.Implicits._ in1 ~> merge ~> bcast ~> out1 in2 ~> merge bcast ~> out2 - }.run() + ClosedShape + }).run() } "build unzip - zip" in { - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val zip = b.add(Zip[Int, String]()) val unzip = b.add(Unzip[Int, String]()) val out = Sink.publisher[(Int, String)] @@ -194,12 +204,13 @@ class FlowGraphCompileSpec extends AkkaSpec { unzip.out0 ~> Flow[Int].map(_ * 2) ~> zip.in0 unzip.out1 ~> zip.in1 zip.out ~> out - }.run() + ClosedShape + }).run() } "distinguish between input and output ports" in { intercept[IllegalArgumentException] { - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val zip = b.add(Zip[Int, String]()) val unzip = b.add(Unzip[Int, String]()) val wrongOut = Sink.publisher[(Int, Int)] @@ -210,22 +221,24 @@ class FlowGraphCompileSpec extends AkkaSpec { "zip.left ~> zip.right" shouldNot compile "Flow(List(1, 2, 3)) ~> zip.left ~> wrongOut" shouldNot compile """Flow(List(1 -> "a", 2 -> "b", 3 -> "c")) ~> unzip.in ~> whatever""" shouldNot compile - } - }.getMessage should include("unconnected") + ClosedShape + }) + }.getMessage should include("must correspond to") } "build with variance" in { - val out = Sink(TestSubscriber.manualProbe[Fruit]()) - FlowGraph.closed() { b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ + import FlowGraph.Implicits._ val merge = b.add(Merge[Fruit](2)) - b.addEdge(b add Source[Fruit](apples), Flow[Fruit], merge.in(0)) - b.addEdge(b add Source[Apple](apples), Flow[Apple], merge.in(1)) - b.addEdge(merge.out, Flow[Fruit].map(identity), b add out) - } + 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]()) + ClosedShape + }) } "build with variance when indices are not specified" in { - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ import FlowGraph.Implicits._ val fruitMerge = b.add(Merge[Fruit](2)) Source[Fruit](apples) ~> fruitMerge @@ -253,11 +266,12 @@ class FlowGraphCompileSpec extends AkkaSpec { Source[Apple](apples) ~> appleBcast appleBcast ~> Sink.head[Fruit] appleBcast ~> Sink.head[Apple] - } + ClosedShape + }) } "build with implicits and variance" in { - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.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]()) @@ -289,40 +303,40 @@ class FlowGraphCompileSpec extends AkkaSpec { "merge.out ~> b.add(Broadcast[Apple](2))" shouldNot compile "merge.out ~> Flow[Fruit].map(identity) ~> b.add(Broadcast[Apple](2))" shouldNot compile "fruitSource ~> merge ~> b.add(Broadcast[Apple](2))" shouldNot compile - } + ClosedShape + }) } "build with plain flow without junctions" in { - FlowGraph.closed() { b ⇒ - b.addEdge(b.add(in1), f1, b.add(out1)) - }.run() - FlowGraph.closed() { b ⇒ - b.addEdge(b.add(in1), f1, b.add(f2.to(out1))) - }.run() - FlowGraph.closed() { b ⇒ - b.addEdge(b.add(in1 via f1), f2, b.add(out1)) - }.run() - FlowGraph.closed() { implicit b ⇒ - import FlowGraph.Implicits._ - b.add(in1) ~> f1 ~> b.add(out1) - }.run() - FlowGraph.closed() { implicit b ⇒ - import FlowGraph.Implicits._ - b.add(in1) ~> b.add(out1) - }.run() - FlowGraph.closed() { implicit b ⇒ - import FlowGraph.Implicits._ - b.add(in1) ~> b.add(f1 to out1) - }.run() - FlowGraph.closed() { implicit b ⇒ - import FlowGraph.Implicits._ - b.add(in1 via f1) ~> b.add(out1) - }.run() - FlowGraph.closed() { implicit b ⇒ - import FlowGraph.Implicits._ - b.add(in1 via f1) ~> b.add(f2 to out1) - }.run() + import FlowGraph.Implicits._ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ + in1 ~> f1 ~> out1 + ClosedShape + }).run() + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ + in1 ~> f1 ~> f2.to(out1) + ClosedShape + }).run() + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ + (in1 via f1) ~> f2 ~> out1 + ClosedShape + }).run() + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ + in1 ~> out1 + ClosedShape + }).run() + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ + in1 ~> (f1 to out1) + ClosedShape + }).run() + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ + (in1 via f1) ~> out1 + ClosedShape + }).run() + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ + (in1 via f1) ~> (f2 to out1) + ClosedShape + }).run() } - } } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowJoinSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowJoinSpec.scala index 42ec6ea4de..39c6c6e7f8 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowJoinSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowJoinSpec.scala @@ -3,7 +3,7 @@ */ package akka.stream.scaladsl -import akka.stream.{ ActorMaterializer, ActorMaterializerSettings } +import akka.stream.{ FlowShape, ActorMaterializer, ActorMaterializerSettings } import akka.stream.testkit._ import com.typesafe.config.ConfigFactory import scala.concurrent.Await @@ -24,7 +24,7 @@ class FlowJoinSpec extends AkkaSpec(ConfigFactory.parseString("akka.loglevel=INF val source = Source(0 to end) val probe = TestSubscriber.manualProbe[Seq[Int]]() - val flow1 = Flow() { implicit b ⇒ + val flow1 = Flow.fromGraph(FlowGraph.create() { implicit b ⇒ import FlowGraph.Implicits._ val merge = b.add(Merge[Int](2)) val broadcast = b.add(Broadcast[Int](2)) @@ -32,8 +32,8 @@ class FlowJoinSpec extends AkkaSpec(ConfigFactory.parseString("akka.loglevel=INF merge.out ~> broadcast.in broadcast.out(0).grouped(1000) ~> Sink(probe) - (merge.in(1), broadcast.out(1)) - } + FlowShape(merge.in(1), broadcast.out(1)) + }) val flow2 = Flow[Int].filter(_ % 2 == 1).map(_ * 10).take((end + 1) / 2) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowScanSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowScanSpec.scala index e482d80c04..59229a821c 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowScanSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowScanSpec.scala @@ -41,7 +41,7 @@ class FlowScanSpec extends AkkaSpec { } "emit values promptly" in { - val f = Source.single(1).concat(Source.lazyEmpty).scan(0)(_ + _).grouped(2).runWith(Sink.head) + val f = Source.single(1).concat(Source.maybe[Int]).scan(0)(_ + _).grouped(2).runWith(Sink.head) Await.result(f, 1.second) should ===(Seq(0, 1)) } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSectionSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSectionSpec.scala index c12b6f19b3..551f6bb3a9 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSectionSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSectionSpec.scala @@ -5,7 +5,7 @@ package akka.stream.scaladsl import akka.stream.Attributes._ import akka.stream.ActorAttributes._ -import akka.stream.ActorMaterializer +import akka.stream.{ FlowShape, ActorMaterializer } import akka.stream.testkit.AkkaSpec import akka.actor.ActorRef import akka.testkit.TestProbe @@ -33,13 +33,13 @@ class FlowSectionSpec extends AkkaSpec(FlowSectionSpec.config) { } "have a nested flow with a different dispatcher" in { - val flow = Flow() { implicit b ⇒ + val flow = Flow.fromGraph(FlowGraph.create() { implicit b ⇒ import FlowGraph.Implicits._ val bcast1 = b.add(Broadcast[Int](1)) val bcast2 = b.add(Broadcast[Int](1)) bcast1 ~> Flow[Int].map(sendThreadNameTo(testActor)) ~> bcast2.in - (bcast1.in, bcast2.out(0)) - }.withAttributes(dispatcher("my-dispatcher1")) + FlowShape(bcast1.in, bcast2.out(0)) + }).withAttributes(dispatcher("my-dispatcher1")) Source.single(1).via(flow).to(Sink.ignore).run() @@ -51,21 +51,21 @@ class FlowSectionSpec extends AkkaSpec(FlowSectionSpec.config) { val probe1 = TestProbe() val probe2 = TestProbe() - val flow1 = Flow() { implicit b ⇒ + val flow1 = Flow.fromGraph(FlowGraph.create() { implicit b ⇒ import FlowGraph.Implicits._ val bcast1 = b.add(Broadcast[Int](1)) val bcast2 = b.add(Broadcast[Int](1)) bcast1 ~> Flow[Int].map(sendThreadNameTo(probe1.ref)) ~> bcast2.in - (bcast1.in, bcast2.out(0)) - }.withAttributes(dispatcher("my-dispatcher1")) + FlowShape(bcast1.in, bcast2.out(0)) + }).withAttributes(dispatcher("my-dispatcher1")) - val flow2 = Flow() { implicit b ⇒ + val flow2 = Flow.fromGraph(FlowGraph.create() { implicit b ⇒ import FlowGraph.Implicits._ val bcast1 = b.add(Broadcast[Int](1)) val bcast2 = b.add(Broadcast[Int](1)) bcast1 ~> flow1.via(Flow[Int].map(sendThreadNameTo(probe2.ref))) ~> bcast2.in - (bcast1.in, bcast2.out(0)) - }.withAttributes(dispatcher("my-dispatcher2")) + FlowShape(bcast1.in, bcast2.out(0)) + }).withAttributes(dispatcher("my-dispatcher2")) Source.single(1).via(flow2).to(Sink.ignore).run() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowTakeSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowTakeSpec.scala index d5e293ec74..866a7f9b48 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowTakeSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowTakeSpec.scala @@ -41,8 +41,8 @@ class FlowTakeSpec extends AkkaSpec with ScriptedTest { } "complete eagerly when zero or less is taken independently of upstream completion" in { - Await.result(Source.lazyEmpty.take(0).runWith(Sink.ignore), 3.second) - Await.result(Source.lazyEmpty.take(-1).runWith(Sink.ignore), 3.second) + Await.result(Source.maybe[Int].take(0).runWith(Sink.ignore), 3.second) + Await.result(Source.maybe[Int].take(-1).runWith(Sink.ignore), 3.second) } } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBackedFlowSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBackedFlowSpec.scala index 7c869790b2..311a8320b0 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBackedFlowSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBackedFlowSpec.scala @@ -12,7 +12,7 @@ import org.reactivestreams.Subscriber object GraphFlowSpec { val source1 = Source(0 to 3) - val partialGraph = FlowGraph.partial() { implicit b ⇒ + val partialGraph = FlowGraph.create() { implicit b ⇒ import FlowGraph.Implicits._ val source2 = Source(4 to 9) val source3 = Source.empty[Int] @@ -62,12 +62,11 @@ class GraphFlowSpec extends AkkaSpec { "work with a Source and Sink" in { val probe = TestSubscriber.manualProbe[Int]() - val flow = Flow(partialGraph) { implicit b ⇒ + val flow = Flow.fromGraph(FlowGraph.create(partialGraph) { implicit b ⇒ partial ⇒ import FlowGraph.Implicits._ - - (partial.inlet, partial.outlet.map(_.toInt).outlet) - } + FlowShape(partial.inlet, partial.outlet.map(_.toInt).outlet) + }) source1.via(flow).to(Sink(probe)).run() @@ -77,10 +76,9 @@ class GraphFlowSpec extends AkkaSpec { "be transformable with a Pipe" in { val probe = TestSubscriber.manualProbe[Int]() - val flow = Flow(partialGraph) { implicit b ⇒ - partial ⇒ - (partial.inlet, partial.outlet) - } + val flow = Flow.fromGraph(FlowGraph.create(partialGraph) { implicit b ⇒ + partial ⇒ FlowShape(partial.inlet, partial.outlet) + }) source1.via(flow).map(_.toInt).to(Sink(probe)).run() @@ -90,15 +88,15 @@ class GraphFlowSpec extends AkkaSpec { "work with another GraphFlow" in { val probe = TestSubscriber.manualProbe[Int]() - val flow1 = Flow(partialGraph) { implicit b ⇒ + val flow1 = Flow.fromGraph(FlowGraph.create(partialGraph) { implicit b ⇒ partial ⇒ - (partial.inlet, partial.outlet) - } + FlowShape(partial.inlet, partial.outlet) + }) - val flow2 = Flow(Flow[String].map(_.toInt)) { implicit b ⇒ + val flow2 = Flow.fromGraph(FlowGraph.create(Flow[String].map(_.toInt)) { implicit b ⇒ importFlow ⇒ - (importFlow.inlet, importFlow.outlet) - } + FlowShape(importFlow.inlet, importFlow.outlet) + }) source1.via(flow1).via(flow2).to(Sink(probe)).run() @@ -108,15 +106,15 @@ class GraphFlowSpec extends AkkaSpec { "be reusable multiple times" in { val probe = TestSubscriber.manualProbe[Int]() - val flow = Flow(Flow[Int].map(_ * 2)) { implicit b ⇒ - importFlow ⇒ - (importFlow.inlet, importFlow.outlet) - } + val flow = Flow.fromGraph(FlowGraph.create(Flow[Int].map(_ * 2)) { implicit b ⇒ + importFlow ⇒ FlowShape(importFlow.inlet, importFlow.outlet) + }) - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ import FlowGraph.Implicits._ Source(1 to 5) ~> flow ~> flow ~> Sink(probe) - }.run() + ClosedShape + }).run() validateProbe(probe, 5, Set(4, 8, 12, 16, 20)) } @@ -126,12 +124,12 @@ class GraphFlowSpec extends AkkaSpec { "work with a Sink" in { val probe = TestSubscriber.manualProbe[Int]() - val source = Source(partialGraph) { implicit b ⇒ + val source = Source.fromGraph(FlowGraph.create(partialGraph) { implicit b ⇒ partial ⇒ import FlowGraph.Implicits._ source1 ~> partial.inlet - partial.outlet.map(_.toInt).outlet - } + SourceShape(partial.outlet.map(_.toInt).outlet) + }) source.to(Sink(probe)).run() @@ -141,10 +139,10 @@ class GraphFlowSpec extends AkkaSpec { "work with a Sink when having KeyedSource inside" in { val probe = TestSubscriber.manualProbe[Int]() - val source = Source.apply(Source.subscriber[Int]) { implicit b ⇒ + val source = Source.fromGraph(FlowGraph.create(Source.subscriber[Int]) { implicit b ⇒ subSource ⇒ - subSource.outlet - } + SourceShape(subSource.outlet) + }) val mm: Subscriber[Int] = source.to(Sink(probe)).run() source1.to(Sink(mm)).run() @@ -156,12 +154,12 @@ class GraphFlowSpec extends AkkaSpec { val probe = TestSubscriber.manualProbe[Int]() - val source = Source(partialGraph) { implicit b ⇒ + val source = Source.fromGraph(FlowGraph.create(partialGraph) { implicit b ⇒ partial ⇒ import FlowGraph.Implicits._ source1 ~> partial.inlet - partial.outlet - } + SourceShape(partial.outlet) + }) source.map(_.toInt).to(Sink(probe)).run() @@ -171,17 +169,17 @@ class GraphFlowSpec extends AkkaSpec { "work with an GraphFlow" in { val probe = TestSubscriber.manualProbe[Int]() - val source = Source(partialGraph) { implicit b ⇒ + val source = Source.fromGraph(FlowGraph.create(partialGraph) { implicit b ⇒ partial ⇒ import FlowGraph.Implicits._ source1 ~> partial.inlet - partial.outlet - } + SourceShape(partial.outlet) + }) - val flow = Flow(Flow[String].map(_.toInt)) { implicit b ⇒ + val flow = Flow.fromGraph(FlowGraph.create(Flow[String].map(_.toInt)) { implicit b ⇒ importFlow ⇒ - (importFlow.inlet, importFlow.outlet) - } + FlowShape(importFlow.inlet, importFlow.outlet) + }) source.via(flow).to(Sink(probe)).run() @@ -191,20 +189,21 @@ class GraphFlowSpec extends AkkaSpec { "be reusable multiple times" in { val probe = TestSubscriber.manualProbe[Int]() - val source = Source(Source(1 to 5)) { implicit b ⇒ + val source = Source.fromGraph(FlowGraph.create(Source(1 to 5)) { implicit b ⇒ s ⇒ import FlowGraph.Implicits._ - s.outlet.map(_ * 2).outlet - } + SourceShape(s.outlet.map(_ * 2).outlet) + }) - FlowGraph.closed(source, source)(Keep.both) { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create(source, source)(Keep.both) { implicit b ⇒ (s1, s2) ⇒ import FlowGraph.Implicits._ val merge = b.add(Merge[Int](2)) s1.outlet ~> merge.in(0) merge.out ~> Sink(probe) s2.outlet.map(_ * 10) ~> merge.in(1) - }.run() + ClosedShape + }).run() validateProbe(probe, 10, Set(2, 4, 6, 8, 10, 20, 40, 60, 80, 100)) } @@ -214,12 +213,12 @@ class GraphFlowSpec extends AkkaSpec { "work with a Source" in { val probe = TestSubscriber.manualProbe[Int]() - val sink = Sink(partialGraph) { implicit b ⇒ + val sink = Sink.fromGraph(FlowGraph.create(partialGraph) { implicit b ⇒ partial ⇒ import FlowGraph.Implicits._ partial.outlet.map(_.toInt) ~> Sink(probe) - partial.inlet - } + SinkShape(partial.inlet) + }) source1.to(sink).run() @@ -230,10 +229,9 @@ class GraphFlowSpec extends AkkaSpec { val probe = TestSubscriber.manualProbe[Int]() val pubSink = Sink.publisher[Int] - val sink = Sink(pubSink) { implicit b ⇒ - p ⇒ - p.inlet - } + val sink = Sink.fromGraph(FlowGraph.create(pubSink) { implicit b ⇒ + p ⇒ SinkShape(p.inlet) + }) val mm = source1.runWith(sink) Source(mm).to(Sink(probe)).run() @@ -244,13 +242,13 @@ class GraphFlowSpec extends AkkaSpec { "be transformable with a Pipe" in { val probe = TestSubscriber.manualProbe[Int]() - val sink = Sink(partialGraph, Flow[String].map(_.toInt))(Keep.both) { implicit b ⇒ + val sink = Sink.fromGraph(FlowGraph.create(partialGraph, Flow[String].map(_.toInt))(Keep.both) { implicit b ⇒ (partial, flow) ⇒ import FlowGraph.Implicits._ flow.outlet ~> partial.inlet partial.outlet.map(_.toInt) ~> Sink(probe) - flow.inlet - } + SinkShape(flow.inlet) + }) val iSink = Flow[Int].map(_.toString).to(sink) source1.to(iSink).run() @@ -262,17 +260,17 @@ class GraphFlowSpec extends AkkaSpec { val probe = TestSubscriber.manualProbe[Int]() - val flow = Flow(partialGraph) { implicit b ⇒ + val flow = Flow.fromGraph(FlowGraph.create(partialGraph) { implicit b ⇒ partial ⇒ - (partial.inlet, partial.outlet) - } + FlowShape(partial.inlet, partial.outlet) + }) - val sink = Sink(Flow[String].map(_.toInt)) { implicit b ⇒ + val sink = Sink.fromGraph(FlowGraph.create(Flow[String].map(_.toInt)) { implicit b ⇒ flow ⇒ import FlowGraph.Implicits._ flow.outlet ~> Sink(probe) - flow.inlet - } + SinkShape(flow.inlet) + }) source1.via(flow).to(sink).run() @@ -286,32 +284,33 @@ class GraphFlowSpec extends AkkaSpec { val inSource = Source.subscriber[Int] val outSink = Sink.publisher[Int] - val flow = Flow(partialGraph) { implicit b ⇒ + val flow = Flow.fromGraph(FlowGraph.create(partialGraph) { implicit b ⇒ partial ⇒ import FlowGraph.Implicits._ - (partial.inlet, partial.outlet.map(_.toInt).outlet) - } + FlowShape(partial.inlet, partial.outlet.map(_.toInt).outlet) + }) - val source = Source(Flow[Int].map(_.toString), inSource)(Keep.right) { implicit b ⇒ + val source = Source.fromGraph(FlowGraph.create(Flow[Int].map(_.toString), inSource)(Keep.right) { implicit b ⇒ (flow, src) ⇒ import FlowGraph.Implicits._ src.outlet ~> flow.inlet - flow.outlet - } + SourceShape(flow.outlet) + }) - val sink = Sink(Flow[String].map(_.toInt), outSink)(Keep.right) { implicit b ⇒ + val sink = Sink.fromGraph(FlowGraph.create(Flow[String].map(_.toInt), outSink)(Keep.right) { implicit b ⇒ (flow, snk) ⇒ import FlowGraph.Implicits._ flow.outlet ~> snk.inlet - flow.inlet - } + SinkShape(flow.inlet) + }) - val (m1, m2, m3) = FlowGraph.closed(source, flow, sink)(Tuple3.apply) { implicit b ⇒ + val (m1, m2, m3) = RunnableGraph.fromGraph(FlowGraph.create(source, flow, sink)(Tuple3.apply) { implicit b ⇒ (src, f, snk) ⇒ import FlowGraph.Implicits._ src.outlet.map(_.toInt) ~> f.inlet f.outlet.map(_.toString) ~> snk.inlet - }.run() + ClosedShape + }).run() val subscriber = m1 val publisher = m3 @@ -326,21 +325,22 @@ class GraphFlowSpec extends AkkaSpec { val inSource = Source.subscriber[Int] val outSink = Sink.publisher[Int] - val source = Source(inSource) { implicit b ⇒ + val source = Source.fromGraph(FlowGraph.create(inSource) { implicit b ⇒ src ⇒ - src.outlet - } + SourceShape(src.outlet) + }) - val sink = Sink(outSink) { implicit b ⇒ + val sink = Sink.fromGraph(FlowGraph.create(outSink) { implicit b ⇒ snk ⇒ - snk.inlet - } + SinkShape(snk.inlet) + }) - val (m1, m2) = FlowGraph.closed(source, sink)(Keep.both) { implicit b ⇒ + val (m1, m2) = RunnableGraph.fromGraph(FlowGraph.create(source, sink)(Keep.both) { implicit b ⇒ (src, snk) ⇒ import FlowGraph.Implicits._ src.outlet ~> snk.inlet - }.run() + ClosedShape + }).run() val subscriber = m1 val publisher = m2 diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBalanceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBalanceSpec.scala index 848a6d7f05..5237b7ae05 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBalanceSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBalanceSpec.scala @@ -4,9 +4,7 @@ import scala.concurrent.Await import scala.concurrent.duration._ import scala.concurrent.Future -import akka.stream.ActorMaterializer - -import akka.stream.ActorMaterializerSettings +import akka.stream.{ ClosedShape, ActorMaterializer, ActorMaterializerSettings } import akka.stream.testkit._ import akka.stream.testkit.scaladsl._ import akka.stream.testkit.Utils._ @@ -25,12 +23,13 @@ class GraphBalanceSpec extends AkkaSpec { val c1 = TestSubscriber.manualProbe[Int]() val c2 = TestSubscriber.manualProbe[Int]() - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val balance = b.add(Balance[Int](2)) Source(List(1, 2, 3)) ~> balance.in balance.out(0) ~> Sink(c1) balance.out(1) ~> Sink(c2) - }.run() + ClosedShape + }).run() val sub1 = c1.expectSubscription() val sub2 = c2.expectSubscription() @@ -48,13 +47,14 @@ class GraphBalanceSpec extends AkkaSpec { "support waiting for demand from all downstream subscriptions" in { val s1 = TestSubscriber.manualProbe[Int]() - val p2 = FlowGraph.closed(Sink.publisher[Int]) { implicit b ⇒ + val p2 = RunnableGraph.fromGraph(FlowGraph.create(Sink.publisher[Int]) { implicit b ⇒ p2Sink ⇒ val balance = b.add(Balance[Int](2, waitForAllDownstreams = true)) Source(List(1, 2, 3)) ~> balance.in balance.out(0) ~> Sink(s1) balance.out(1) ~> p2Sink - }.run() + ClosedShape + }).run() val sub1 = s1.expectSubscription() sub1.request(1) @@ -78,14 +78,15 @@ class GraphBalanceSpec extends AkkaSpec { "support waiting for demand from all non-cancelled downstream subscriptions" in assertAllStagesStopped { val s1 = TestSubscriber.manualProbe[Int]() - val (p2, p3) = FlowGraph.closed(Sink.publisher[Int], Sink.publisher[Int])(Keep.both) { implicit b ⇒ + val (p2, p3) = RunnableGraph.fromGraph(FlowGraph.create(Sink.publisher[Int], Sink.publisher[Int])(Keep.both) { implicit b ⇒ (p2Sink, p3Sink) ⇒ val balance = b.add(Balance[Int](3, waitForAllDownstreams = true)) Source(List(1, 2, 3)) ~> balance.in balance.out(0) ~> Sink(s1) balance.out(1) ~> p2Sink balance.out(2) ~> p3Sink - }.run() + ClosedShape + }).run() val sub1 = s1.expectSubscription() sub1.request(1) @@ -112,7 +113,7 @@ class GraphBalanceSpec extends AkkaSpec { "work with 5-way balance" in { val sink = Sink.head[Seq[Int]] - val (s1, s2, s3, s4, s5) = FlowGraph.closed(sink, sink, sink, sink, sink)(Tuple5.apply) { + val (s1, s2, s3, s4, s5) = RunnableGraph.fromGraph(FlowGraph.create(sink, sink, sink, sink, sink)(Tuple5.apply) { implicit b ⇒ (f1, f2, f3, f4, f5) ⇒ val balance = b.add(Balance[Int](5, waitForAllDownstreams = true)) @@ -122,7 +123,8 @@ class GraphBalanceSpec extends AkkaSpec { balance.out(2).grouped(15) ~> f3 balance.out(3).grouped(15) ~> f4 balance.out(4).grouped(15) ~> f5 - }.run() + ClosedShape + }).run() Set(s1, s2, s3, s4, s5) flatMap (Await.result(_, 3.seconds)) should be((0 to 14).toSet) } @@ -131,14 +133,15 @@ class GraphBalanceSpec extends AkkaSpec { val numElementsForSink = 10000 val outputs = Sink.fold[Int, Int](0)(_ + _) - val results = FlowGraph.closed(outputs, outputs, outputs)(List(_, _, _)) { implicit b ⇒ + val results = RunnableGraph.fromGraph(FlowGraph.create(outputs, outputs, outputs)(List(_, _, _)) { implicit b ⇒ (o1, o2, o3) ⇒ val balance = b.add(Balance[Int](3, waitForAllDownstreams = true)) Source.repeat(1).take(numElementsForSink * 3) ~> balance.in balance.out(0) ~> o1 balance.out(1) ~> o2 balance.out(2) ~> o3 - }.run() + ClosedShape + }).run() import system.dispatcher val sum = Future.sequence(results).map { res ⇒ @@ -150,14 +153,15 @@ class GraphBalanceSpec extends AkkaSpec { "fairly balance between three outputs" in { val probe = TestSink.probe[Int] - val (p1, p2, p3) = FlowGraph.closed(probe, probe, probe)(Tuple3.apply) { implicit b ⇒ + val (p1, p2, p3) = RunnableGraph.fromGraph(FlowGraph.create(probe, probe, probe)(Tuple3.apply) { implicit b ⇒ (o1, o2, o3) ⇒ val balance = b.add(Balance[Int](3)) Source(1 to 7) ~> balance.in balance.out(0) ~> o1 balance.out(1) ~> o2 balance.out(2) ~> o3 - }.run() + ClosedShape + }).run() p1.requestNext(1) p2.requestNext(2) @@ -176,12 +180,13 @@ class GraphBalanceSpec extends AkkaSpec { val c1 = TestSubscriber.manualProbe[Int]() val c2 = TestSubscriber.manualProbe[Int]() - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val balance = b.add(Balance[Int](2)) Source(List(1, 2, 3)) ~> balance.in balance.out(0) ~> Sink(c1) balance.out(1) ~> Sink(c2) - }.run() + ClosedShape + }).run() val sub1 = c1.expectSubscription() sub1.cancel() @@ -197,12 +202,13 @@ class GraphBalanceSpec extends AkkaSpec { val c1 = TestSubscriber.manualProbe[Int]() val c2 = TestSubscriber.manualProbe[Int]() - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val balance = b.add(Balance[Int](2)) Source(List(1, 2, 3)) ~> balance.in balance.out(0) ~> Sink(c1) balance.out(1) ~> Sink(c2) - }.run() + ClosedShape + }).run() val sub1 = c1.expectSubscription() val sub2 = c2.expectSubscription() @@ -219,12 +225,13 @@ class GraphBalanceSpec extends AkkaSpec { val c1 = TestSubscriber.manualProbe[Int]() val c2 = TestSubscriber.manualProbe[Int]() - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val balance = b.add(Balance[Int](2)) Source(p1.getPublisher) ~> balance.in balance.out(0) ~> Sink(c1) balance.out(1) ~> Sink(c2) - }.run() + ClosedShape + }).run() val bsub = p1.expectSubscription() val sub1 = c1.expectSubscription() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBroadcastSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBroadcastSpec.scala index 6c8d642ccf..9b130d5e94 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBroadcastSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBroadcastSpec.scala @@ -5,8 +5,7 @@ import akka.stream.testkit.scaladsl.TestSink import scala.concurrent.{ Future, Await } import scala.concurrent.duration._ -import akka.stream.{ OverflowStrategy, ActorMaterializerSettings } -import akka.stream.ActorMaterializer +import akka.stream._ import akka.stream.testkit._ import akka.stream.testkit.Utils._ @@ -24,12 +23,13 @@ class GraphBroadcastSpec extends AkkaSpec { val c1 = TestSubscriber.manualProbe[Int]() val c2 = TestSubscriber.manualProbe[Int]() - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val bcast = b.add(Broadcast[Int](2)) Source(List(1, 2, 3)) ~> bcast.in bcast.out(0) ~> Flow[Int].buffer(16, OverflowStrategy.backpressure) ~> Sink(c1) bcast.out(1) ~> Flow[Int].buffer(16, OverflowStrategy.backpressure) ~> Sink(c2) - }.run() + ClosedShape + }).run() val sub1 = c1.expectSubscription() val sub2 = c2.expectSubscription() @@ -53,7 +53,7 @@ class GraphBroadcastSpec extends AkkaSpec { val headSink = Sink.head[Seq[Int]] import system.dispatcher - val result = FlowGraph.closed( + val result = RunnableGraph.fromGraph(FlowGraph.create( headSink, headSink, headSink, @@ -68,7 +68,8 @@ class GraphBroadcastSpec extends AkkaSpec { bcast.out(2).grouped(5) ~> p3.inlet bcast.out(3).grouped(5) ~> p4.inlet bcast.out(4).grouped(5) ~> p5.inlet - }.run() + ClosedShape + }).run() Await.result(result, 3.seconds) should be(List.fill(5)(List(1, 2, 3))) } @@ -83,7 +84,7 @@ class GraphBroadcastSpec extends AkkaSpec { (f1, f2, f3, f4, f5, f6, f7, f8, f9, f10, f11, f12, f13, f14, f15, f16, f17, f18, f19, f20, f21, f22) ⇒ Future.sequence(List(f1, f2, f3, f4, f5, f6, f7, f8, f9, f10, f11, f12, f13, f14, f15, f16, f17, f18, f19, f20, f21, f22)) - val result = FlowGraph.closed( + val result = RunnableGraph.fromGraph(FlowGraph.create( headSink, headSink, headSink, headSink, headSink, headSink, headSink, headSink, headSink, headSink, headSink, headSink, headSink, headSink, headSink, @@ -115,7 +116,8 @@ class GraphBroadcastSpec extends AkkaSpec { bcast.out(19).grouped(5) ~> p20.inlet bcast.out(20).grouped(5) ~> p21.inlet bcast.out(21).grouped(5) ~> p22.inlet - }.run() + ClosedShape + }).run() Await.result(result, 3.seconds) should be(List.fill(22)(List(1, 2, 3))) } @@ -124,12 +126,13 @@ class GraphBroadcastSpec extends AkkaSpec { val c1 = TestSubscriber.manualProbe[Int]() val c2 = TestSubscriber.manualProbe[Int]() - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val bcast = b.add(Broadcast[Int](2)) Source(List(1, 2, 3)) ~> bcast.in bcast.out(0) ~> Flow[Int] ~> Sink(c1) bcast.out(1) ~> Flow[Int] ~> Sink(c2) - }.run() + ClosedShape + }).run() val sub1 = c1.expectSubscription() sub1.cancel() @@ -145,12 +148,13 @@ class GraphBroadcastSpec extends AkkaSpec { val c1 = TestSubscriber.manualProbe[Int]() val c2 = TestSubscriber.manualProbe[Int]() - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val bcast = b.add(Broadcast[Int](2)) Source(List(1, 2, 3)) ~> bcast.in bcast.out(0) ~> Flow[Int].named("identity-a") ~> Sink(c1) bcast.out(1) ~> Flow[Int].named("identity-b") ~> Sink(c2) - }.run() + ClosedShape + }).run() val sub1 = c1.expectSubscription() val sub2 = c2.expectSubscription() @@ -167,12 +171,13 @@ class GraphBroadcastSpec extends AkkaSpec { val c1 = TestSubscriber.manualProbe[Int]() val c2 = TestSubscriber.manualProbe[Int]() - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val bcast = b.add(Broadcast[Int](2)) Source(p1.getPublisher) ~> bcast.in bcast.out(0) ~> Flow[Int] ~> Sink(c1) bcast.out(1) ~> Flow[Int] ~> Sink(c2) - }.run() + ClosedShape + }).run() val bsub = p1.expectSubscription() val sub1 = c1.expectSubscription() @@ -195,12 +200,12 @@ class GraphBroadcastSpec extends AkkaSpec { val c1 = TestSubscriber.manualProbe[Int]() val c2 = TestSubscriber.manualProbe[Int]() - val sink = Sink() { implicit b ⇒ + val sink = Sink.fromGraph(FlowGraph.create() { implicit b ⇒ val bcast = b.add(Broadcast[Int](2)) bcast.out(0) ~> Sink(c1) bcast.out(1) ~> Sink(c2) - bcast.in - } + SinkShape(bcast.in) + }) val s = Source.subscriber[Int].to(sink).run() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphConcatSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphConcatSpec.scala index 77c615fdac..6742f7a778 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphConcatSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphConcatSpec.scala @@ -30,7 +30,7 @@ class GraphConcatSpec extends TwoStreamsSetup { "work in the happy case" in assertAllStagesStopped { val probe = TestSubscriber.manualProbe[Int]() - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val concat1 = b add Concat[Int]() val concat2 = b add Concat[Int]() @@ -42,7 +42,8 @@ class GraphConcatSpec extends TwoStreamsSetup { Source(5 to 10) ~> concat2.in(1) concat2.out ~> Sink(probe) - }.run() + ClosedShape + }).run() val subscription = probe.expectSubscription() @@ -140,12 +141,13 @@ class GraphConcatSpec extends TwoStreamsSetup { val promise = Promise[Int]() val subscriber = TestSubscriber.manualProbe[Int]() - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val concat = b add Concat[Int]() Source(List(1, 2, 3)) ~> concat.in(0) Source(promise.future) ~> concat.in(1) concat.out ~> Sink(subscriber) - }.run() + ClosedShape + }).run() val subscription = subscriber.expectSubscription() subscription.request(4) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMatValueSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMatValueSpec.scala index 423daf92ae..3431f7b257 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMatValueSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMatValueSpec.scala @@ -3,7 +3,7 @@ */ package akka.stream.scaladsl -import akka.stream.{ ActorMaterializer, ActorMaterializerSettings } +import akka.stream.{ ClosedShape, SourceShape, ActorMaterializer, ActorMaterializerSettings } import akka.stream.testkit._ import scala.concurrent.Await @@ -25,11 +25,12 @@ class GraphMatValueSpec extends AkkaSpec { "expose the materialized value as source" in { val sub = TestSubscriber.manualProbe[Int]() - val f = FlowGraph.closed(foldSink) { implicit b ⇒ + val f = RunnableGraph.fromGraph(FlowGraph.create(foldSink) { implicit b ⇒ fold ⇒ Source(1 to 10) ~> fold b.materializedValue.mapAsync(4)(identity) ~> Sink(sub) - }.run() + ClosedShape + }).run() val r1 = Await.result(f, 3.seconds) sub.expectSubscription().request(1) @@ -41,7 +42,7 @@ class GraphMatValueSpec extends AkkaSpec { "expose the materialized value as source multiple times" in { val sub = TestSubscriber.manualProbe[Int]() - val f = FlowGraph.closed(foldSink) { implicit b ⇒ + val f = RunnableGraph.fromGraph(FlowGraph.create(foldSink) { implicit b ⇒ fold ⇒ val zip = b.add(ZipWith[Int, Int, Int](_ + _)) Source(1 to 10) ~> fold @@ -49,7 +50,8 @@ class GraphMatValueSpec extends AkkaSpec { b.materializedValue.mapAsync(4)(identity) ~> zip.in1 zip.out ~> Sink(sub) - }.run() + ClosedShape + }).run() val r1 = Await.result(f, 3.seconds) sub.expectSubscription().request(1) @@ -59,11 +61,11 @@ class GraphMatValueSpec extends AkkaSpec { } // Exposes the materialized value as a stream value - val foldFeedbackSource: Source[Future[Int], Future[Int]] = Source(foldSink) { implicit b ⇒ + val foldFeedbackSource: Source[Future[Int], Future[Int]] = Source.fromGraph(FlowGraph.create(foldSink) { implicit b ⇒ fold ⇒ Source(1 to 10) ~> fold - b.materializedValue - } + SourceShape(b.materializedValue) + }) "allow exposing the materialized value as port" in { val (f1, f2) = foldFeedbackSource.mapAsync(4)(identity).map(_ + 100).toMat(Sink.head)(Keep.both).run() @@ -77,23 +79,22 @@ class GraphMatValueSpec extends AkkaSpec { } "work properly with nesting and reusing" in { - val compositeSource1 = Source(foldFeedbackSource, foldFeedbackSource)(Keep.both) { implicit b ⇒ + val compositeSource1 = Source.fromGraph(FlowGraph.create(foldFeedbackSource, foldFeedbackSource)(Keep.both) { implicit b ⇒ (s1, s2) ⇒ val zip = b.add(ZipWith[Int, Int, Int](_ + _)) s1.outlet.mapAsync(4)(identity) ~> zip.in0 s2.outlet.mapAsync(4)(identity).map(_ * 100) ~> zip.in1 - zip.out - } + SourceShape(zip.out) + }) - val compositeSource2 = Source(compositeSource1, compositeSource1)(Keep.both) { implicit b ⇒ + val compositeSource2 = Source.fromGraph(FlowGraph.create(compositeSource1, compositeSource1)(Keep.both) { implicit b ⇒ (s1, s2) ⇒ val zip = b.add(ZipWith[Int, Int, Int](_ + _)) - s1.outlet ~> zip.in0 s2.outlet.map(_ * 10000) ~> zip.in1 - zip.out - } + SourceShape(zip.out) + }) val (((f1, f2), (f3, f4)), result) = compositeSource2.toMat(Sink.head)(Keep.both).run() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMergeSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMergeSpec.scala index ffc98ab09d..ecd67dd86f 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMergeSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMergeSpec.scala @@ -3,7 +3,7 @@ */ package akka.stream.scaladsl -import akka.stream.{ ActorMaterializer, ActorMaterializerSettings, Inlet, Outlet } +import akka.stream._ import scala.concurrent.duration._ @@ -33,7 +33,7 @@ class GraphMergeSpec extends TwoStreamsSetup { val source3 = Source(List[Int]()) val probe = TestSubscriber.manualProbe[Int]() - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val m1 = b.add(Merge[Int](2)) val m2 = b.add(Merge[Int](2)) @@ -43,7 +43,8 @@ class GraphMergeSpec extends TwoStreamsSetup { source2 ~> m1.in(1) source3 ~> m2.in(1) - }.run() + ClosedShape + }).run() val subscription = probe.expectSubscription() @@ -67,7 +68,7 @@ class GraphMergeSpec extends TwoStreamsSetup { val probe = TestSubscriber.manualProbe[Int]() - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val merge = b.add(Merge[Int](6)) source1 ~> merge.in(0) @@ -78,7 +79,8 @@ class GraphMergeSpec extends TwoStreamsSetup { source6 ~> merge.in(5) merge.out ~> Sink(probe) - }.run() + ClosedShape + }).run() val subscription = probe.expectSubscription() @@ -152,13 +154,14 @@ class GraphMergeSpec extends TwoStreamsSetup { val src1 = Source.subscriber[Int] val src2 = Source.subscriber[Int] - val (graphSubscriber1, graphSubscriber2) = FlowGraph.closed(src1, src2)((_, _)) { implicit b ⇒ + val (graphSubscriber1, graphSubscriber2) = RunnableGraph.fromGraph(FlowGraph.create(src1, src2)((_, _)) { implicit b ⇒ (s1, s2) ⇒ val merge = b.add(Merge[Int](2)) s1.outlet ~> merge.in(0) s2.outlet ~> merge.in(1) merge.out ~> Sink(down) - }.run() + ClosedShape + }).run() val downstream = down.expectSubscription() downstream.cancel() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphOpsIntegrationSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphOpsIntegrationSpec.scala index dea4520e96..cdc234610c 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphOpsIntegrationSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphOpsIntegrationSpec.scala @@ -3,13 +3,10 @@ package akka.stream.scaladsl import scala.collection.immutable import scala.concurrent.{ Future, Await } import scala.concurrent.duration._ -import akka.stream.ActorMaterializer -import akka.stream.ActorMaterializerSettings +import akka.stream._ import akka.stream.testkit._ import akka.util.ByteString -import akka.stream.{ Inlet, Outlet, Shape, Graph } import org.scalactic.ConversionCheckedTripleEquals -import akka.stream.Attributes object GraphOpsIntegrationSpec { import FlowGraph.Implicits._ @@ -33,7 +30,7 @@ object GraphOpsIntegrationSpec { } def apply[In, Out](pipeline: Flow[In, Out, _]): Graph[ShufflePorts[In, Out], Unit] = { - FlowGraph.partial() { implicit b ⇒ + FlowGraph.create() { implicit b ⇒ val merge = b.add(Merge[In](2)) val balance = b.add(Balance[Out](2)) merge.out ~> pipeline ~> balance.in @@ -57,7 +54,7 @@ class GraphOpsIntegrationSpec extends AkkaSpec with ConversionCheckedTripleEqual "FlowGraphs" must { "support broadcast - merge layouts" in { - val resultFuture = FlowGraph.closed(Sink.head[Seq[Int]]) { implicit b ⇒ + val resultFuture = RunnableGraph.fromGraph(FlowGraph.create(Sink.head[Seq[Int]]) { implicit b ⇒ (sink) ⇒ val bcast = b.add(Broadcast[Int](2)) val merge = b.add(Merge[Int](2)) @@ -66,14 +63,15 @@ class GraphOpsIntegrationSpec extends AkkaSpec with ConversionCheckedTripleEqual bcast.out(0) ~> merge.in(0) bcast.out(1).map(_ + 3) ~> merge.in(1) merge.out.grouped(10) ~> sink.inlet - }.run() + ClosedShape + }).run() Await.result(resultFuture, 3.seconds).sorted should be(List(1, 2, 3, 4, 5, 6)) } "support balance - merge (parallelization) layouts" in { val elements = 0 to 10 - val out = FlowGraph.closed(Sink.head[Seq[Int]]) { implicit b ⇒ + val out = RunnableGraph.fromGraph(FlowGraph.create(Sink.head[Seq[Int]]) { implicit b ⇒ (sink) ⇒ val balance = b.add(Balance[Int](5)) val merge = b.add(Merge[Int](5)) @@ -83,7 +81,8 @@ class GraphOpsIntegrationSpec extends AkkaSpec with ConversionCheckedTripleEqual for (i ← 0 until 5) balance.out(i) ~> merge.in(i) merge.out.grouped(elements.size * 2) ~> sink.inlet - }.run() + ClosedShape + }).run() Await.result(out, 3.seconds).sorted should be(elements) } @@ -93,7 +92,7 @@ class GraphOpsIntegrationSpec extends AkkaSpec with ConversionCheckedTripleEqual // see https://en.wikipedia.org/wiki/Topological_sorting#mediaviewer/File:Directed_acyclic_graph.png val seqSink = Sink.head[Seq[Int]] - val (resultFuture2, resultFuture9, resultFuture10) = FlowGraph.closed(seqSink, seqSink, seqSink)(Tuple3.apply) { implicit b ⇒ + val (resultFuture2, resultFuture9, resultFuture10) = RunnableGraph.fromGraph(FlowGraph.create(seqSink, seqSink, seqSink)(Tuple3.apply) { implicit b ⇒ (sink2, sink9, sink10) ⇒ val b3 = b.add(Broadcast[Int](2)) val b7 = b.add(Broadcast[Int](2)) @@ -129,7 +128,8 @@ class GraphOpsIntegrationSpec extends AkkaSpec with ConversionCheckedTripleEqual m9.out.grouped(1000) ~> sink9.inlet m10.out.grouped(1000) ~> sink10.inlet - }.run() + ClosedShape + }).run() Await.result(resultFuture2, 3.seconds).sorted should be(List(5, 7)) Await.result(resultFuture9, 3.seconds).sorted should be(List(3, 5, 7, 7)) @@ -139,7 +139,7 @@ class GraphOpsIntegrationSpec extends AkkaSpec with ConversionCheckedTripleEqual "allow adding of flows to sources and sinks to flows" in { - val resultFuture = FlowGraph.closed(Sink.head[Seq[Int]]) { implicit b ⇒ + val resultFuture = RunnableGraph.fromGraph(FlowGraph.create(Sink.head[Seq[Int]]) { implicit b ⇒ (sink) ⇒ val bcast = b.add(Broadcast[Int](2)) val merge = b.add(Merge[Int](2)) @@ -148,7 +148,8 @@ class GraphOpsIntegrationSpec extends AkkaSpec with ConversionCheckedTripleEqual bcast.out(0) ~> merge.in(0) bcast.out(1).map(_ + 3) ~> merge.in(1) merge.out.grouped(10) ~> sink.inlet - }.run() + ClosedShape + }).run() Await.result(resultFuture, 3.seconds) should contain theSameElementsAs (Seq(2, 4, 6, 5, 7, 9)) } @@ -157,9 +158,10 @@ class GraphOpsIntegrationSpec extends AkkaSpec with ConversionCheckedTripleEqual val p = Source(List(1, 2, 3)).runWith(Sink.publisher) val s = TestSubscriber.manualProbe[Int] val flow = Flow[Int].map(_ * 2) - FlowGraph.closed() { implicit builder ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit builder ⇒ Source(p) ~> flow ~> Sink(s) - }.run() + ClosedShape + }).run() val sub = s.expectSubscription() sub.request(10) s.expectNext(1 * 2) @@ -171,7 +173,7 @@ class GraphOpsIntegrationSpec extends AkkaSpec with ConversionCheckedTripleEqual "be possible to use as lego bricks" in { val shuffler = Shuffle(Flow[Int].map(_ + 1)) - val f: Future[Seq[Int]] = FlowGraph.closed(shuffler, shuffler, shuffler, Sink.head[Seq[Int]])((_, _, _, fut) ⇒ fut) { implicit b ⇒ + val f: Future[Seq[Int]] = RunnableGraph.fromGraph(FlowGraph.create(shuffler, shuffler, shuffler, Sink.head[Seq[Int]])((_, _, _, fut) ⇒ fut) { implicit b ⇒ (s1, s2, s3, sink) ⇒ val merge = b.add(Merge[Int](2)) @@ -188,7 +190,8 @@ class GraphOpsIntegrationSpec extends AkkaSpec with ConversionCheckedTripleEqual s3.out2 ~> merge.in(1) merge.out.grouped(1000) ~> sink - }.run() + ClosedShape + }).run() val result = Await.result(f, 3.seconds) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphPartialSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphPartialSpec.scala index b9fe1391b9..fb71aa5480 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphPartialSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphPartialSpec.scala @@ -1,6 +1,6 @@ package akka.stream.scaladsl -import akka.stream.{ ActorMaterializer, ActorMaterializerSettings, FlowShape } +import akka.stream.{ ClosedShape, ActorMaterializer, ActorMaterializerSettings, FlowShape } import akka.stream.testkit.AkkaSpec import scala.concurrent.{ Await, Future } @@ -18,7 +18,7 @@ class GraphPartialSpec extends AkkaSpec { import FlowGraph.Implicits._ "be able to build and reuse simple partial graphs" in { - val doubler = FlowGraph.partial() { implicit b ⇒ + val doubler = FlowGraph.create() { implicit b ⇒ val bcast = b.add(Broadcast[Int](2)) val zip = b.add(ZipWith((a: Int, b: Int) ⇒ a + b)) @@ -27,18 +27,19 @@ class GraphPartialSpec extends AkkaSpec { FlowShape(bcast.in, zip.out) } - val (_, _, result) = FlowGraph.closed(doubler, doubler, Sink.head[Seq[Int]])(Tuple3.apply) { implicit b ⇒ + val (_, _, result) = RunnableGraph.fromGraph(FlowGraph.create(doubler, doubler, Sink.head[Seq[Int]])(Tuple3.apply) { implicit b ⇒ (d1, d2, sink) ⇒ Source(List(1, 2, 3)) ~> d1.inlet d1.outlet ~> d2.inlet d2.outlet.grouped(100) ~> sink.inlet - }.run() + ClosedShape + }).run() Await.result(result, 3.seconds) should be(List(4, 8, 12)) } "be able to build and reuse simple materializing partial graphs" in { - val doubler = FlowGraph.partial(Sink.head[Seq[Int]]) { implicit b ⇒ + val doubler = FlowGraph.create(Sink.head[Seq[Int]]) { implicit b ⇒ sink ⇒ val bcast = b.add(Broadcast[Int](3)) val zip = b.add(ZipWith((a: Int, b: Int) ⇒ a + b)) @@ -49,12 +50,13 @@ class GraphPartialSpec extends AkkaSpec { FlowShape(bcast.in, zip.out) } - val (sub1, sub2, result) = FlowGraph.closed(doubler, doubler, Sink.head[Seq[Int]])(Tuple3.apply) { implicit b ⇒ + val (sub1, sub2, result) = RunnableGraph.fromGraph(FlowGraph.create(doubler, doubler, Sink.head[Seq[Int]])(Tuple3.apply) { implicit b ⇒ (d1, d2, sink) ⇒ Source(List(1, 2, 3)) ~> d1.inlet d1.outlet ~> d2.inlet d2.outlet.grouped(100) ~> sink.inlet - }.run() + ClosedShape + }).run() Await.result(result, 3.seconds) should be(List(4, 8, 12)) Await.result(sub1, 3.seconds) should be(List(1, 2, 3)) @@ -64,7 +66,7 @@ class GraphPartialSpec extends AkkaSpec { "be able to build and reuse complex materializing partial graphs" in { val summer = Sink.fold[Int, Int](0)(_ + _) - val doubler = FlowGraph.partial(summer, summer)(Tuple2.apply) { implicit b ⇒ + val doubler = FlowGraph.create(summer, summer)(Tuple2.apply) { implicit b ⇒ (s1, s2) ⇒ val bcast = b.add(Broadcast[Int](3)) val bcast2 = b.add(Broadcast[Int](2)) @@ -80,12 +82,13 @@ class GraphPartialSpec extends AkkaSpec { FlowShape(bcast.in, bcast2.out(1)) } - val (sub1, sub2, result) = FlowGraph.closed(doubler, doubler, Sink.head[Seq[Int]])(Tuple3.apply) { implicit b ⇒ + val (sub1, sub2, result) = RunnableGraph.fromGraph(FlowGraph.create(doubler, doubler, Sink.head[Seq[Int]])(Tuple3.apply) { implicit b ⇒ (d1, d2, sink) ⇒ Source(List(1, 2, 3)) ~> d1.inlet d1.outlet ~> d2.inlet d2.outlet.grouped(100) ~> sink.inlet - }.run() + ClosedShape + }).run() Await.result(result, 3.seconds) should be(List(4, 8, 12)) Await.result(sub1._1, 3.seconds) should be(6) @@ -95,17 +98,18 @@ class GraphPartialSpec extends AkkaSpec { } "be able to expose the ports of imported graphs" in { - val p = FlowGraph.partial(Flow[Int].map(_ + 1)) { implicit b ⇒ + val p = FlowGraph.create(Flow[Int].map(_ + 1)) { implicit b ⇒ flow ⇒ FlowShape(flow.inlet, flow.outlet) } - val fut = FlowGraph.closed(Sink.head[Int], p)(Keep.left) { implicit b ⇒ + val fut = RunnableGraph.fromGraph(FlowGraph.create(Sink.head[Int], p)(Keep.left) { implicit b ⇒ (sink, flow) ⇒ import FlowGraph.Implicits._ Source.single(0) ~> flow.inlet flow.outlet ~> sink.inlet - }.run() + ClosedShape + }).run() Await.result(fut, 3.seconds) should be(1) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphPreferredMergeSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphPreferredMergeSpec.scala index 502468c0e0..992648d5f7 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphPreferredMergeSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphPreferredMergeSpec.scala @@ -32,7 +32,7 @@ class GraphPreferredMergeSpec extends TwoStreamsSetup { val preferred = Source(Stream.fill(numElements)(1)) val aux = Source(Stream.fill(numElements)(2)) - val result = FlowGraph.closed(Sink.head[Seq[Int]]) { implicit b ⇒ + val result = RunnableGraph.fromGraph(FlowGraph.create(Sink.head[Seq[Int]]) { implicit b ⇒ sink ⇒ val merge = b.add(MergePreferred[Int](3)) preferred ~> merge.preferred @@ -41,13 +41,14 @@ class GraphPreferredMergeSpec extends TwoStreamsSetup { aux ~> merge.in(0) aux ~> merge.in(1) aux ~> merge.in(2) - }.run() + ClosedShape + }).run() Await.result(result, 3.seconds).filter(_ == 1).size should be(numElements) } "eventually pass through all elements" in { - val result = FlowGraph.closed(Sink.head[Seq[Int]]) { implicit b ⇒ + val result = RunnableGraph.fromGraph(FlowGraph.create(Sink.head[Seq[Int]]) { implicit b ⇒ sink ⇒ val merge = b.add(MergePreferred[Int](3)) Source(1 to 100) ~> merge.preferred @@ -56,7 +57,8 @@ class GraphPreferredMergeSpec extends TwoStreamsSetup { Source(101 to 200) ~> merge.in(0) Source(201 to 300) ~> merge.in(1) Source(301 to 400) ~> merge.in(2) - }.run() + ClosedShape + }).run() Await.result(result, 3.seconds).toSet should ===((1 to 400).toSet) } @@ -65,7 +67,7 @@ class GraphPreferredMergeSpec extends TwoStreamsSetup { val s = Source(0 to 3) (the[IllegalArgumentException] thrownBy { - val g = FlowGraph.closed() { implicit b ⇒ + val g = RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val merge = b.add(MergePreferred[Int](1)) s ~> merge.preferred @@ -73,7 +75,8 @@ class GraphPreferredMergeSpec extends TwoStreamsSetup { s ~> merge.in(0) merge.out ~> Sink.head[Int] - } + ClosedShape + }) }).getMessage should include("[MergePreferred.preferred] is already connected") } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphStageTimersSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphStageTimersSpec.scala index 524f1c52c9..1f1343389b 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphStageTimersSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphStageTimersSpec.scala @@ -23,12 +23,12 @@ object GraphStageTimersSpec { class SideChannel { @volatile var asyncCallback: AsyncCallback[Any] = _ - @volatile var stopPromise: Promise[Unit] = _ + @volatile var stopPromise: Promise[Option[Nothing]] = _ def isReady: Boolean = asyncCallback ne null def !(msg: Any) = asyncCallback.invoke(msg) - def stopStage(): Unit = stopPromise.trySuccess(()) + def stopStage(): Unit = stopPromise.trySuccess(None) } } @@ -81,7 +81,7 @@ class GraphStageTimersSpec extends AkkaSpec { def setupIsolatedStage: SideChannel = { val channel = new SideChannel - val stopPromise = Source.lazyEmpty[Int].via(new TestStage(testActor, channel)).to(Sink.ignore).run() + val stopPromise = Source.maybe[Nothing].via(new TestStage(testActor, channel)).to(Sink.ignore).run() channel.stopPromise = stopPromise awaitCond(channel.isReady) channel diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphUnzipSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphUnzipSpec.scala index 2657b3b2f5..677494573a 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphUnzipSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphUnzipSpec.scala @@ -5,8 +5,7 @@ package akka.stream.scaladsl import scala.concurrent.duration._ -import akka.stream.{ OverflowStrategy, ActorMaterializerSettings } -import akka.stream.ActorMaterializer +import akka.stream.{ ClosedShape, OverflowStrategy, ActorMaterializerSettings, ActorMaterializer } import akka.stream.testkit._ import akka.stream.testkit.Utils._ @@ -24,12 +23,13 @@ class GraphUnzipSpec extends AkkaSpec { val c1 = TestSubscriber.manualProbe[Int]() val c2 = TestSubscriber.manualProbe[String]() - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val unzip = b.add(Unzip[Int, String]()) Source(List(1 -> "a", 2 -> "b", 3 -> "c")) ~> unzip.in unzip.out1 ~> Flow[String].buffer(16, OverflowStrategy.backpressure) ~> Sink(c2) unzip.out0 ~> Flow[Int].buffer(16, OverflowStrategy.backpressure).map(_ * 2) ~> Sink(c1) - }.run() + ClosedShape + }).run() val sub1 = c1.expectSubscription() val sub2 = c2.expectSubscription() @@ -53,12 +53,13 @@ class GraphUnzipSpec extends AkkaSpec { val c1 = TestSubscriber.manualProbe[Int]() val c2 = TestSubscriber.manualProbe[String]() - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val unzip = b.add(Unzip[Int, String]()) Source(List(1 -> "a", 2 -> "b", 3 -> "c")) ~> unzip.in unzip.out0 ~> Sink(c1) unzip.out1 ~> Sink(c2) - }.run() + ClosedShape + }).run() val sub1 = c1.expectSubscription() val sub2 = c2.expectSubscription() @@ -74,12 +75,13 @@ class GraphUnzipSpec extends AkkaSpec { val c1 = TestSubscriber.manualProbe[Int]() val c2 = TestSubscriber.manualProbe[String]() - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val unzip = b.add(Unzip[Int, String]()) Source(List(1 -> "a", 2 -> "b", 3 -> "c")) ~> unzip.in unzip.out0 ~> Sink(c1) unzip.out1 ~> Sink(c2) - }.run() + ClosedShape + }).run() val sub1 = c1.expectSubscription() val sub2 = c2.expectSubscription() @@ -96,12 +98,13 @@ class GraphUnzipSpec extends AkkaSpec { val c1 = TestSubscriber.manualProbe[Int]() val c2 = TestSubscriber.manualProbe[String]() - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val unzip = b.add(Unzip[Int, String]()) Source(p1.getPublisher) ~> unzip.in unzip.out0 ~> Sink(c1) unzip.out1 ~> Sink(c2) - }.run() + ClosedShape + }).run() val p1Sub = p1.expectSubscription() val sub1 = c1.expectSubscription() @@ -122,14 +125,15 @@ class GraphUnzipSpec extends AkkaSpec { "work with zip" in assertAllStagesStopped { val c1 = TestSubscriber.manualProbe[(Int, String)]() - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val zip = b.add(Zip[Int, String]()) val unzip = b.add(Unzip[Int, String]()) Source(List(1 -> "a", 2 -> "b", 3 -> "c")) ~> unzip.in unzip.out0 ~> zip.in0 unzip.out1 ~> zip.in1 zip.out ~> Sink(c1) - }.run() + ClosedShape + }).run() val sub1 = c1.expectSubscription() sub1.request(5) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphUnzipWithSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphUnzipWithSpec.scala index 6f9f0094af..5dac2ad328 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphUnzipWithSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphUnzipWithSpec.scala @@ -48,14 +48,15 @@ class GraphUnzipWithSpec extends AkkaSpec { val leftSubscriber = TestSubscriber.probe[LeftOutput]() val rightSubscriber = TestSubscriber.probe[RightOutput]() - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val f = fixture(b) Source(p) ~> f.in f.left ~> Sink(leftSubscriber) f.right ~> Sink(rightSubscriber) - }.run() + ClosedShape + }).run() (leftSubscriber, rightSubscriber) } @@ -96,13 +97,15 @@ class GraphUnzipWithSpec extends AkkaSpec { val leftProbe = TestSubscriber.manualProbe[LeftOutput]() val rightProbe = TestSubscriber.manualProbe[RightOutput]() - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val unzip = b.add(UnzipWith(f)) Source(1 to 4) ~> unzip.in unzip.out0 ~> Flow[LeftOutput].buffer(4, OverflowStrategy.backpressure) ~> Sink(leftProbe) unzip.out1 ~> Flow[RightOutput].buffer(4, OverflowStrategy.backpressure) ~> Sink(rightProbe) - }.run() + + ClosedShape + }).run() val leftSubscription = leftProbe.expectSubscription() val rightSubscription = rightProbe.expectSubscription() @@ -144,14 +147,16 @@ class GraphUnzipWithSpec extends AkkaSpec { val leftProbe = TestSubscriber.manualProbe[LeftOutput]() val rightProbe = TestSubscriber.manualProbe[RightOutput]() - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val unzip = b.add(UnzipWith[Int, Int, String]((b: Int) ⇒ (1 / b, 1 + "/" + b))) Source(-2 to 2) ~> unzip.in unzip.out0 ~> Sink(leftProbe) unzip.out1 ~> Sink(rightProbe) - }.run() + + ClosedShape + }).run() val leftSubscription = leftProbe.expectSubscription() val rightSubscription = rightProbe.expectSubscription() @@ -187,7 +192,7 @@ class GraphUnzipWithSpec extends AkkaSpec { case class Person(name: String, surname: String, int: Int) - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val unzip = b.add(UnzipWith((a: Person) ⇒ Person.unapply(a).get)) Source.single(Person("Caplin", "Capybara", 3)) ~> unzip.in @@ -195,7 +200,9 @@ class GraphUnzipWithSpec extends AkkaSpec { unzip.out0 ~> Sink(probe0) unzip.out1 ~> Sink(probe1) unzip.out2 ~> Sink(probe2) - }.run() + + ClosedShape + }).run() val subscription0 = probe0.expectSubscription() val subscription1 = probe1.expectSubscription() @@ -221,7 +228,7 @@ class GraphUnzipWithSpec extends AkkaSpec { val probe15 = TestSubscriber.manualProbe[String]() val probe19 = TestSubscriber.manualProbe[String]() - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val split20 = (a: (List[Int])) ⇒ (a(0), a(0).toString, @@ -268,7 +275,8 @@ class GraphUnzipWithSpec extends AkkaSpec { unzip.out19 ~> Sink(probe19) - }.run() + ClosedShape + }).run() probe0.expectSubscription().request(1) probe5.expectSubscription().request(1) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipSpec.scala index 735f15468f..3e5443b37f 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipSpec.scala @@ -25,14 +25,16 @@ class GraphZipSpec extends TwoStreamsSetup { "work in the happy case" in assertAllStagesStopped { val probe = TestSubscriber.manualProbe[(Int, String)]() - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val zip = b.add(Zip[Int, String]()) Source(1 to 4) ~> zip.in0 Source(List("A", "B", "C", "D", "E", "F")) ~> zip.in1 zip.out ~> Sink(probe) - }.run() + + ClosedShape + }).run() val subscription = probe.expectSubscription() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipWithSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipWithSpec.scala index be4b8f213e..a693feed40 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipWithSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphZipWithSpec.scala @@ -21,13 +21,15 @@ class GraphZipWithSpec extends TwoStreamsSetup { "work in the happy case" in { val probe = TestSubscriber.manualProbe[Outputs]() - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val zip = b.add(ZipWith((_: Int) + (_: Int))) Source(1 to 4) ~> zip.in0 Source(10 to 40 by 10) ~> zip.in1 zip.out ~> Sink(probe) - }.run() + + ClosedShape + }).run() val subscription = probe.expectSubscription() @@ -46,14 +48,16 @@ class GraphZipWithSpec extends TwoStreamsSetup { "work in the sad case" in { val probe = TestSubscriber.manualProbe[Outputs]() - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val zip = b.add(ZipWith[Int, Int, Int]((_: Int) / (_: Int))) Source(1 to 4) ~> zip.in0 Source(-2 to 2) ~> zip.in1 zip.out ~> Sink(probe) - }.run() + + ClosedShape + }).run() val subscription = probe.expectSubscription() @@ -107,7 +111,7 @@ class GraphZipWithSpec extends TwoStreamsSetup { case class Person(name: String, surname: String, int: Int) - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val zip = b.add(ZipWith(Person.apply _)) Source.single("Caplin") ~> zip.in0 @@ -115,7 +119,9 @@ class GraphZipWithSpec extends TwoStreamsSetup { Source.single(3) ~> zip.in2 zip.out ~> Sink(probe) - }.run() + + ClosedShape + }).run() val subscription = probe.expectSubscription() @@ -128,7 +134,7 @@ class GraphZipWithSpec extends TwoStreamsSetup { "work with up to 22 inputs" in { val probe = TestSubscriber.manualProbe[String]() - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val sum19 = (v1: Int, v2: String, v3: Int, v4: String, v5: Int, v6: String, v7: Int, v8: String, v9: Int, v10: String, v11: Int, v12: String, v13: Int, v14: String, v15: Int, v16: String, v17: Int, v18: String, v19: Int) ⇒ @@ -159,7 +165,9 @@ class GraphZipWithSpec extends TwoStreamsSetup { Source.single(19) ~> zip.in18 zip.out ~> Sink(probe) - }.run() + + ClosedShape + }).run() val subscription = probe.expectSubscription() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/One2OneBidiFlowSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/One2OneBidiFlowSpec.scala index c09c6ab1c1..17557683c5 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/One2OneBidiFlowSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/One2OneBidiFlowSpec.scala @@ -65,7 +65,7 @@ class One2OneBidiFlowSpec extends AkkaSpec with ConversionCheckedTripleEquals { Source(1 to 1000) .log("", seen.set) - .via(One2OneBidiFlow[Int, Int](MAX_PENDING) join Flow.wrap(Sink.ignore, Source(out))(Keep.left)) + .via(One2OneBidiFlow[Int, Int](MAX_PENDING) join Flow.fromSinkAndSourceMat(Sink.ignore, Source(out))(Keep.left)) .runWith(Sink.ignore) Thread.sleep(50) @@ -82,6 +82,6 @@ class One2OneBidiFlowSpec extends AkkaSpec with ConversionCheckedTripleEquals { val outIn = TestPublisher.probe[Int]() val outOut = TestSubscriber.probe[Int]() - Source(inIn).via(One2OneBidiFlow[Int, Int](maxPending) join Flow.wrap(Sink(inOut), Source(outIn))(Keep.left)).runWith(Sink(outOut)) + Source(inIn).via(One2OneBidiFlow[Int, Int](maxPending) join Flow.fromSinkAndSourceMat(Sink(inOut), Source(outIn))(Keep.left)).runWith(Sink(outOut)) } } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/PublisherSinkSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/PublisherSinkSpec.scala index ffbdc92b39..12ca4add12 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/PublisherSinkSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/PublisherSinkSpec.scala @@ -3,7 +3,7 @@ */ package akka.stream.scaladsl -import akka.stream.ActorMaterializer +import akka.stream.{ ClosedShape, ActorMaterializer } import akka.stream.testkit.AkkaSpec import akka.stream.testkit.Utils._ @@ -19,7 +19,7 @@ class PublisherSinkSpec extends AkkaSpec { "be unique when created twice" in assertAllStagesStopped { - val (pub1, pub2) = FlowGraph.closed(Sink.publisher[Int], Sink.publisher[Int])(Keep.both) { implicit b ⇒ + val (pub1, pub2) = RunnableGraph.fromGraph(FlowGraph.create(Sink.publisher[Int], Sink.publisher[Int])(Keep.both) { implicit b ⇒ (p1, p2) ⇒ import FlowGraph.Implicits._ @@ -28,7 +28,8 @@ class PublisherSinkSpec extends AkkaSpec { Source(0 to 5) ~> bcast.in bcast.out(0).map(_ * 2) ~> p1.inlet bcast.out(1) ~> p2.inlet - }.run() + ClosedShape + }).run() val f1 = Source(pub1).map(identity).runFold(0)(_ + _) val f2 = Source(pub2).map(identity).runFold(0)(_ + _) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ReverseArrowSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ReverseArrowSpec.scala index 624bac044a..276236701f 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ReverseArrowSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ReverseArrowSpec.scala @@ -16,43 +16,48 @@ class ReverseArrowSpec extends AkkaSpec with ConversionCheckedTripleEquals { "Reverse Arrows in the Graph DSL" must { "work from Inlets" in { - Await.result(FlowGraph.closed(sink) { implicit b ⇒ + Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b ⇒ s ⇒ s.inlet <~ source - }.run(), 1.second) should ===(Seq(1, 2, 3)) + ClosedShape + }).run(), 1.second) should ===(Seq(1, 2, 3)) } "work from SinkShape" in { - Await.result(FlowGraph.closed(sink) { implicit b ⇒ + Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b ⇒ s ⇒ s <~ source - }.run(), 1.second) should ===(Seq(1, 2, 3)) + ClosedShape + }).run(), 1.second) should ===(Seq(1, 2, 3)) } "work from Sink" in { val sub = TestSubscriber.manualProbe[Int] - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ Sink(sub) <~ source - }.run() + ClosedShape + }).run() sub.expectSubscription().request(10) sub.expectNext(1, 2, 3) sub.expectComplete() } "not work from Outlets" in { - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val o: Outlet[Int] = b.add(source) "o <~ source" shouldNot compile sink <~ o - } + ClosedShape + }) } "not work from SourceShape" in { - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ val o: SourceShape[Int] = b.add(source) "o <~ source" shouldNot compile sink <~ o - } + ClosedShape + }) } "not work from Source" in { @@ -60,114 +65,127 @@ class ReverseArrowSpec extends AkkaSpec with ConversionCheckedTripleEquals { } "work from FlowShape" in { - Await.result(FlowGraph.closed(sink) { implicit b ⇒ + Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b ⇒ s ⇒ val f: FlowShape[Int, Int] = b.add(Flow[Int]) f <~ source f ~> s - }.run(), 1.second) should ===(Seq(1, 2, 3)) + ClosedShape + }).run(), 1.second) should ===(Seq(1, 2, 3)) } "work from UniformFanInShape" in { - Await.result(FlowGraph.closed(sink) { implicit b ⇒ + Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b ⇒ s ⇒ val f: UniformFanInShape[Int, Int] = b.add(Merge[Int](1)) f <~ source f ~> s - }.run(), 1.second) should ===(Seq(1, 2, 3)) + ClosedShape + }).run(), 1.second) should ===(Seq(1, 2, 3)) } "work from UniformFanOutShape" in { - Await.result(FlowGraph.closed(sink) { implicit b ⇒ + Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b ⇒ s ⇒ val f: UniformFanOutShape[Int, Int] = b.add(Broadcast[Int](1)) f <~ source f ~> s - }.run(), 1.second) should ===(Seq(1, 2, 3)) + ClosedShape + }).run(), 1.second) should ===(Seq(1, 2, 3)) } "work towards Outlets" in { - Await.result(FlowGraph.closed(sink) { implicit b ⇒ + Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b ⇒ s ⇒ val o: Outlet[Int] = b.add(source) s <~ o - }.run(), 1.second) should ===(Seq(1, 2, 3)) + ClosedShape + }).run(), 1.second) should ===(Seq(1, 2, 3)) } "work towards SourceShape" in { - Await.result(FlowGraph.closed(sink) { implicit b ⇒ + Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b ⇒ s ⇒ val o: SourceShape[Int] = b.add(source) s <~ o - }.run(), 1.second) should ===(Seq(1, 2, 3)) + ClosedShape + }).run(), 1.second) should ===(Seq(1, 2, 3)) } "work towards Source" in { - Await.result(FlowGraph.closed(sink) { implicit b ⇒ + Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b ⇒ s ⇒ s <~ source - }.run(), 1.second) should ===(Seq(1, 2, 3)) + ClosedShape + }).run(), 1.second) should ===(Seq(1, 2, 3)) } "work towards FlowShape" in { - Await.result(FlowGraph.closed(sink) { implicit b ⇒ + Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b ⇒ s ⇒ val f: FlowShape[Int, Int] = b.add(Flow[Int]) s <~ f source ~> f - }.run(), 1.second) should ===(Seq(1, 2, 3)) + ClosedShape + }).run(), 1.second) should ===(Seq(1, 2, 3)) } "work towards UniformFanInShape" in { - Await.result(FlowGraph.closed(sink) { implicit b ⇒ + Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b ⇒ s ⇒ val f: UniformFanInShape[Int, Int] = b.add(Merge[Int](1)) s <~ f source ~> f - }.run(), 1.second) should ===(Seq(1, 2, 3)) + ClosedShape + }).run(), 1.second) should ===(Seq(1, 2, 3)) } "fail towards already full UniformFanInShape" in { - Await.result(FlowGraph.closed(sink) { implicit b ⇒ + Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b ⇒ s ⇒ val f: UniformFanInShape[Int, Int] = b.add(Merge[Int](1)) val src = b.add(source) src ~> f (the[IllegalArgumentException] thrownBy (s <~ f <~ src)).getMessage should include("no more inlets free") - }.run(), 1.second) should ===(Seq(1, 2, 3)) + ClosedShape + }).run(), 1.second) should ===(Seq(1, 2, 3)) } "work towards UniformFanOutShape" in { - Await.result(FlowGraph.closed(sink) { implicit b ⇒ + Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b ⇒ s ⇒ val f: UniformFanOutShape[Int, Int] = b.add(Broadcast[Int](1)) s <~ f source ~> f - }.run(), 1.second) should ===(Seq(1, 2, 3)) + ClosedShape + }).run(), 1.second) should ===(Seq(1, 2, 3)) } "fail towards already full UniformFanOutShape" in { - Await.result(FlowGraph.closed(sink) { implicit b ⇒ + Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b ⇒ s ⇒ val f: UniformFanOutShape[Int, Int] = b.add(Broadcast[Int](1)) val src = b.add(source) src ~> f (the[IllegalArgumentException] thrownBy (s <~ f <~ src)).getMessage should include("already connected") - }.run(), 1.second) should ===(Seq(1, 2, 3)) + ClosedShape + }).run(), 1.second) should ===(Seq(1, 2, 3)) } "work across a Flow" in { - Await.result(FlowGraph.closed(sink) { implicit b ⇒ + Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b ⇒ s ⇒ s <~ Flow[Int] <~ source - }.run(), 1.second) should ===(Seq(1, 2, 3)) + ClosedShape + }).run(), 1.second) should ===(Seq(1, 2, 3)) } "work across a FlowShape" in { - Await.result(FlowGraph.closed(sink) { implicit b ⇒ + Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b ⇒ s ⇒ s <~ b.add(Flow[Int]) <~ source - }.run(), 1.second) should ===(Seq(1, 2, 3)) + ClosedShape + }).run(), 1.second) should ===(Seq(1, 2, 3)) } } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkSpec.scala index 922ed5f262..b978a95265 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkSpec.scala @@ -3,7 +3,7 @@ */ package akka.stream.scaladsl -import akka.stream.ActorMaterializer +import akka.stream.{ SinkShape, ActorMaterializer } import akka.stream.testkit.TestPublisher.ManualProbe import akka.stream.testkit._ @@ -17,11 +17,11 @@ class SinkSpec extends AkkaSpec { "be composable without importing modules" in { val probes = Array.fill(3)(TestSubscriber.manualProbe[Int]) - val sink = Sink() { implicit b ⇒ + val sink = Sink.fromGraph(FlowGraph.create() { implicit b ⇒ val bcast = b.add(Broadcast[Int](3)) for (i ← 0 to 2) bcast.out(i).filter(_ == i) ~> Sink(probes(i)) - bcast.in - } + SinkShape(bcast.in) + }) Source(List(0, 1, 2)).runWith(sink) for (i ← 0 to 2) { val p = probes(i) @@ -34,13 +34,13 @@ class SinkSpec extends AkkaSpec { "be composable with importing 1 module" in { val probes = Array.fill(3)(TestSubscriber.manualProbe[Int]) - val sink = Sink(Sink(probes(0))) { implicit b ⇒ + val sink = Sink.fromGraph(FlowGraph.create(Sink(probes(0))) { implicit b ⇒ s0 ⇒ val bcast = b.add(Broadcast[Int](3)) bcast.out(0) ~> Flow[Int].filter(_ == 0) ~> s0.inlet for (i ← 1 to 2) bcast.out(i).filter(_ == i) ~> Sink(probes(i)) - bcast.in - } + SinkShape(bcast.in) + }) Source(List(0, 1, 2)).runWith(sink) for (i ← 0 to 2) { val p = probes(i) @@ -53,14 +53,14 @@ class SinkSpec extends AkkaSpec { "be composable with importing 2 modules" in { val probes = Array.fill(3)(TestSubscriber.manualProbe[Int]) - val sink = Sink(Sink(probes(0)), Sink(probes(1)))(List(_, _)) { implicit b ⇒ + val sink = Sink.fromGraph(FlowGraph.create(Sink(probes(0)), Sink(probes(1)))(List(_, _)) { implicit b ⇒ (s0, s1) ⇒ val bcast = b.add(Broadcast[Int](3)) bcast.out(0).filter(_ == 0) ~> s0.inlet bcast.out(1).filter(_ == 1) ~> s1.inlet bcast.out(2).filter(_ == 2) ~> Sink(probes(2)) - bcast.in - } + SinkShape(bcast.in) + }) Source(List(0, 1, 2)).runWith(sink) for (i ← 0 to 2) { val p = probes(i) @@ -73,14 +73,14 @@ class SinkSpec extends AkkaSpec { "be composable with importing 3 modules" in { val probes = Array.fill(3)(TestSubscriber.manualProbe[Int]) - val sink = Sink(Sink(probes(0)), Sink(probes(1)), Sink(probes(2)))(List(_, _, _)) { implicit b ⇒ + val sink = Sink.fromGraph(FlowGraph.create(Sink(probes(0)), Sink(probes(1)), Sink(probes(2)))(List(_, _, _)) { implicit b ⇒ (s0, s1, s2) ⇒ val bcast = b.add(Broadcast[Int](3)) bcast.out(0).filter(_ == 0) ~> s0.inlet bcast.out(1).filter(_ == 1) ~> s1.inlet bcast.out(2).filter(_ == 2) ~> s2.inlet - bcast.in - } + SinkShape(bcast.in) + }) Source(List(0, 1, 2)).runWith(sink) for (i ← 0 to 2) { val p = probes(i) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SourceSpec.scala index fb5dc1ce9d..26730799c3 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SourceSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SourceSpec.scala @@ -7,10 +7,9 @@ import scala.concurrent.Await import scala.concurrent.duration._ import scala.util.{ Success, Failure } import scala.util.control.NoStackTrace -import akka.stream.ActorMaterializer +import akka.stream.{ SourceShape, ActorMaterializer } import akka.stream.testkit._ -import akka.stream.impl.PublisherSource -import akka.stream.impl.ReactiveStreamsCompliance +import akka.stream.impl.{ PublisherSource, ReactiveStreamsCompliance } class SourceSpec extends AkkaSpec { @@ -73,14 +72,14 @@ class SourceSpec extends AkkaSpec { } } - "Lazy Empty Source" must { - "complete materialized future when stream cancels" in { - val neverSource = Source.lazyEmpty - val pubSink = Sink.publisher + "Maybe Source" must { + "complete materialized future with None when stream cancels" in { + val neverSource = Source.maybe[Int] + val pubSink = Sink.publisher[Int] val (f, neverPub) = neverSource.toMat(pubSink)(Keep.both).run() - val c = TestSubscriber.manualProbe() + val c = TestSubscriber.manualProbe[Int]() neverPub.subscribe(c) val subs = c.expectSubscription() @@ -88,24 +87,35 @@ class SourceSpec extends AkkaSpec { c.expectNoMsg(300.millis) subs.cancel() - Await.result(f.future, 500.millis) + Await.result(f.future, 500.millis) shouldEqual None } - "allow external triggering of completion" in { - val neverSource = Source.lazyEmpty[Int] + "allow external triggering of empty completion" in { + val neverSource = Source.maybe[Int].filter(_ ⇒ false) val counterSink = Sink.fold[Int, Int](0) { (acc, _) ⇒ acc + 1 } val (neverPromise, counterFuture) = neverSource.toMat(counterSink)(Keep.both).run() // external cancellation - neverPromise.success(()) + neverPromise.trySuccess(None) shouldEqual true - val ready = Await.ready(counterFuture, 500.millis) - val Success(0) = ready.value.get + Await.result(counterFuture, 500.millis) shouldEqual 0 + } + + "allow external triggering of non-empty completion" in { + val neverSource = Source.maybe[Int] + val counterSink = Sink.head[Int] + + val (neverPromise, counterFuture) = neverSource.toMat(counterSink)(Keep.both).run() + + // external cancellation + neverPromise.trySuccess(Some(6)) shouldEqual true + + Await.result(counterFuture, 500.millis) shouldEqual 6 } "allow external triggering of onError" in { - val neverSource = Source.lazyEmpty + val neverSource = Source.maybe[Int] val counterSink = Sink.fold[Int, Int](0) { (acc, _) ⇒ acc + 1 } val (neverPromise, counterFuture) = neverSource.toMat(counterSink)(Keep.both).run() @@ -126,7 +136,7 @@ class SourceSpec extends AkkaSpec { val source = Source.subscriber[Int] val out = TestSubscriber.manualProbe[Int] - val s = Source(source, source, source, source, source)(Seq(_, _, _, _, _)) { implicit b ⇒ + val s = Source.fromGraph(FlowGraph.create(source, source, source, source, source)(Seq(_, _, _, _, _)) { implicit b ⇒ (i0, i1, i2, i3, i4) ⇒ import FlowGraph.Implicits._ val m = b.add(Merge[Int](5)) @@ -135,8 +145,8 @@ class SourceSpec extends AkkaSpec { i2.outlet ~> m.in(2) i3.outlet ~> m.in(3) i4.outlet ~> m.in(4) - m.out - }.to(Sink(out)).run() + SourceShape(m.out) + }).to(Sink(out)).run() for (i ← 0 to 4) probes(i).subscribe(s(i)) val sub = out.expectSubscription() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/TickSourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/TickSourceSpec.scala index 12e7b81212..de6442369d 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/TickSourceSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/TickSourceSpec.scala @@ -7,10 +7,9 @@ import akka.actor.Cancellable import scala.concurrent.duration._ import scala.util.control.NoStackTrace -import akka.stream.ActorMaterializer +import akka.stream.{ ClosedShape, ActorMaterializer, ActorMaterializerSettings } import akka.stream.testkit._ import akka.stream.testkit.Utils._ -import akka.stream.ActorMaterializerSettings class TickSourceSpec extends AkkaSpec { @@ -68,13 +67,14 @@ class TickSourceSpec extends AkkaSpec { "be usable with zip for a simple form of rate limiting" in { val c = TestSubscriber.manualProbe[Int]() - FlowGraph.closed() { implicit b ⇒ + RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒ import FlowGraph.Implicits._ val zip = b.add(Zip[Int, String]()) Source(1 to 100) ~> zip.in0 Source(1.second, 1.second, "tick") ~> zip.in1 zip.out ~> Flow[(Int, String)].map { case (n, _) ⇒ n } ~> Sink(c) - }.run() + ClosedShape + }).run() val sub = c.expectSubscription() sub.request(1000) diff --git a/akka-stream/src/main/boilerplate/akka/stream/javadsl/BidiFlowCreate.scala.template b/akka-stream/src/main/boilerplate/akka/stream/javadsl/BidiFlowCreate.scala.template deleted file mode 100644 index 541f3c068b..0000000000 --- a/akka-stream/src/main/boilerplate/akka/stream/javadsl/BidiFlowCreate.scala.template +++ /dev/null @@ -1,48 +0,0 @@ -/** - * Copyright (C) 2015 Typesafe Inc. - */ -package akka.stream.javadsl - -import akka.stream.scaladsl -import akka.stream.{ Inlet, Outlet, Shape, Graph, BidiShape } -import akka.japi.Pair -import akka.japi.function - -trait BidiFlowCreate { - - import language.implicitConversions - private implicit def p[A, B](pair: Pair[A, B]): (A, B) = pair.first -> pair.second - - /** - * Creates a BidiFlow by applying a [[FlowGraph.Builder]] to the given create function. - */ - def create[I1, O1, I2, O2](block: function.Function[FlowGraph.Builder[Unit], BidiShape[I1, O1, I2, O2]]): BidiFlow[I1, O1, I2, O2, Unit] = - new BidiFlow(scaladsl.BidiFlow() { b ⇒ block.apply(b.asJava) }) - - /** - * Creates a BidiFlow by applying a [[FlowGraph.Builder]] to the given create function. - * The given graph will be imported (using `builder.graph()`) and the resulting [[Shape]] will be passed to the create function along with the builder. - */ - def create[I1, O1, I2, O2, S <: Shape, M](g1: Graph[S, M], - block: function.Function2[FlowGraph.Builder[M], S, BidiShape[I1, O1, I2, O2]]): BidiFlow[I1, O1, I2, O2, M] = - new BidiFlow(scaladsl.BidiFlow(g1) { b ⇒ s => block.apply(b.asJava, s) }) - - /** - * Creates a BidiFlow by applying a [[FlowGraph.Builder]] to the given create function. - * The given graphs will be imported (using `builder.graph()`) and the resulting [[Shape]]s will be passed to the create function along with the builder. - */ - def create[I1, O1, I2, O2, S1 <: Shape, S2 <: Shape, M1, M2, M](g1: Graph[S1, M1], g2: Graph[S2, M2], combineMat: function.Function2[M1, M2, M], - block: function.Function3[FlowGraph.Builder[M], S1, S2, BidiShape[I1, O1, I2, O2]]): BidiFlow[I1, O1, I2, O2, M] = - new BidiFlow(scaladsl.BidiFlow(g1, g2)(combineMat.apply _) { b => (s1, s2) => block.apply(b.asJava, s1, s2) }) - - [3..21#/** - * Creates a BidiFlow by applying a [[FlowGraph.Builder]] to the given create function. - * The given graphs will be imported (using `builder.graph()`) and the resulting [[Shape]]s will be passed to the create function along with the builder. - */ - def create1[I##1, O##1, I##2, O##2, [#S1 <: Shape#], [#M1#], M]([#g1: Graph[S1, M1]#], combineMat: function.Function1[[#M1#], M], - block: function.Function2[FlowGraph.Builder[M], [#S1#], BidiShape[I##1, O##1, I##2, O##2]]): BidiFlow[I##1, O##1, I##2, O##2, M] = - new BidiFlow(scaladsl.BidiFlow([#g1#])(combineMat.apply _) { b => ([#s1#]) => block.apply(b.asJava, [#s1#]) })# - - ] - -} diff --git a/akka-stream/src/main/boilerplate/akka/stream/javadsl/FlowCreate.scala.template b/akka-stream/src/main/boilerplate/akka/stream/javadsl/FlowCreate.scala.template deleted file mode 100644 index fc821d0ee0..0000000000 --- a/akka-stream/src/main/boilerplate/akka/stream/javadsl/FlowCreate.scala.template +++ /dev/null @@ -1,55 +0,0 @@ -/** - * Copyright (C) 2015 Typesafe Inc. - */ -package akka.stream.javadsl - -import akka.stream.scaladsl -import akka.stream.{ Inlet, Outlet, Shape, Graph } -import akka.japi.Pair -import akka.japi.function - -trait FlowCreate { - - import language.implicitConversions - private implicit def p[A, B](pair: Pair[A, B]): (A, B) = pair.first -> pair.second - - /** - * Creates a Flow by passing a [[FlowGraph.Builder]] to the given create function. - * - * The create function is expected to return a pair of [[Inlet]] and [[Outlet]] which correspond to the created Flows input and output ports. - */ - def create[I, O](block: function.Function[FlowGraph.Builder[Unit], Inlet[I] Pair Outlet[O]]): Flow[I, O, Unit] = - new Flow(scaladsl.Flow() { b ⇒ block.apply(b.asJava) }) - - /** - * Creates a Flow by passing a [[FlowGraph.Builder]] to the given create function. - * The given graph will be imported (using `builder.graph()`) and the resulting shape will be passed to the create function along with the builder. - * - * The create function is expected to return a pair of [[Inlet]] and [[Outlet]] which correspond to the created Flows input and output ports. - */ - def create[I, O, S <: Shape, M](g1: Graph[S, M], block: function.Function2[FlowGraph.Builder[M], S, Inlet[I] Pair Outlet[O]]): Flow[I, O, M] = - new Flow(scaladsl.Flow(g1) { b ⇒ s => block.apply(b.asJava, s) }) - - /** - * Creates a Flow by passing a [[FlowGraph.Builder]] to the given create function. - * The given graph will be imported (using `builder.graph()`) and the resulting shape will be passed to the create function along with the builder. - * - * The create function is expected to return a pair of [[Inlet]] and [[Outlet]] which correspond to the created Flows input and output ports. - */ - def create[I, O, S1 <: Shape, S2 <: Shape, M1, M2, M](g1: Graph[S1, M1], g2: Graph[S2, M2], combineMat: function.Function2[M1, M2, M], - block: function.Function3[FlowGraph.Builder[M], S1, S2, Inlet[I] Pair Outlet[O]]): Flow[I, O, M] = - new Flow(scaladsl.Flow(g1, g2)(combineMat.apply _) { b => (s1, s2) => block.apply(b.asJava, s1, s2) }) - - [3..21#/** - * Creates a Flow by passing a [[FlowGraph.Builder]] to the given create function. - * The given graph will be imported (using `builder.graph()`) and the resulting shape will be passed to the create function along with the builder. - * - * The create function is expected to return a pair of [[Inlet]] and [[Outlet]] which correspond to the created Flows input and output ports. - */ - def create1[I, O, [#S1 <: Shape#], [#M1#], M]([#g1: Graph[S1, M1]#], combineMat: function.Function1[[#M1#], M], - block: function.Function2[FlowGraph.Builder[M], [#S1#], Inlet[I] Pair Outlet[O]]): Flow[I, O, M] = - new Flow(scaladsl.Flow([#g1#])(combineMat.apply _) { b => ([#s1#]) => block.apply(b.asJava, [#s1#]) })# - - ] - -} diff --git a/akka-stream/src/main/boilerplate/akka/stream/javadsl/GraphCreate.scala.template b/akka-stream/src/main/boilerplate/akka/stream/javadsl/GraphCreate.scala.template index adf9c5f697..edfb89ee35 100644 --- a/akka-stream/src/main/boilerplate/akka/stream/javadsl/GraphCreate.scala.template +++ b/akka-stream/src/main/boilerplate/akka/stream/javadsl/GraphCreate.scala.template @@ -7,91 +7,36 @@ import akka.stream.scaladsl import akka.stream.{ Inlet, Shape, Graph } import akka.japi.function -trait GraphCreate { - - import language.implicitConversions - private implicit def r[M](run: scaladsl.RunnableGraph[M]): RunnableGraph[M] = new RunnableGraphAdapter(run) +private[stream] abstract class GraphCreate { + /** + * Creates a new [[Graph]] of the given [[Shape]] by passing a [[FlowGraph.Builder]] to the given create function. + */ + def create[S <: Shape](block: function.Function[FlowGraph.Builder[Unit], S]): Graph[S, Unit] = + scaladsl.FlowGraph.create() { b ⇒ block.apply(b.asJava) } /** - * Creates a new fully connected graph by passing a [[FlowGraph.Builder]] to the given create function. - * - * The created graph must have all ports connected, otherwise an [[IllegalArgumentException]] is thrown. + * Creates a new [[Graph]] by importing the given graph `g1` and its [[Shape]] + * along with the [[FlowGraph.Builder]] to the given create function. */ - @throws(classOf[IllegalArgumentException]) - def closed(block: function.Procedure[FlowGraph.Builder[Unit]]): RunnableGraph[Unit] = - scaladsl.FlowGraph.closed() { b ⇒ block.apply(b.asJava) } - - /** - * Creates a new [[Graph]] by passing a [[FlowGraph.Builder]] to the given create function. - * - * Partial graphs are allowed to have unconnected ports. - */ - def partial[S <: Shape](block: function.Function[FlowGraph.Builder[Unit], S]): Graph[S, Unit] = - scaladsl.FlowGraph.partial() { b ⇒ block.apply(b.asJava) } - - /** - * Creates a new fully connected graph by importing the given graph `g1` (using `builder.graph()`) and passing its resulting [[Shape]] - * along with the[[FlowGraph.Builder]] to the given create function. - * - * The created graph must have all ports connected, otherwise an [[IllegalArgumentException]] is thrown. - */ - @throws(classOf[IllegalArgumentException]) - def closed[S1 <: Shape, M](g1: Graph[S1, M], - block: function.Procedure2[FlowGraph.Builder[M], S1]): RunnableGraph[M] = - scaladsl.FlowGraph.closed(g1) { b ⇒ s => block.apply(b.asJava, s) } - - /** - * Creates a new [[Graph]] by importing the given graph `g1` (using `builder.graph()`) and passing its resulting [[Shape]] - * along with the[[FlowGraph.Builder]] to the given create function. - * - * Partial graphs are allowed to have unconnected ports. - */ - def partial[S1 <: Shape, S <: Shape, M](g1: Graph[S1, M], + def create[S1 <: Shape, S <: Shape, M](g1: Graph[S1, M], block: function.Function2[FlowGraph.Builder[M], S1, S]): Graph[S, M] = - scaladsl.FlowGraph.partial(g1) { b ⇒ s => block.apply(b.asJava, s) } + scaladsl.FlowGraph.create(g1) { b ⇒ s => block.apply(b.asJava, s) } /** - * Creates a new fully connected graph by importing the given graphs (using `builder.graph()`) and passing their resulting [[Shape]]s - * along with the[[FlowGraph.Builder]] to the given create function. - * - * The created graph must have all ports connected, otherwise an [[IllegalArgumentException]] is thrown. + * Creates a new [[Graph]] by importing the given graphs and passing their [[Shape]]s + * along with the [[FlowGraph.Builder]] to the given create function. */ - @throws(classOf[IllegalArgumentException]) - def closed[S1 <: Shape, S2 <: Shape, M1, M2, M](g1: Graph[S1, M1], g2: Graph[S2, M2], combineMat: function.Function2[M1, M2, M], - block: function.Procedure3[FlowGraph.Builder[M], S1, S2]): RunnableGraph[M] = - scaladsl.FlowGraph.closed(g1, g2)(combineMat.apply _) { b => (s1, s2) => block.apply(b.asJava, s1, s2) } - - /** - * Creates a new [[Graph]] by importing the given graphs (using `builder.graph()`) and passing their resulting [[Shape]]s - * along with the[[FlowGraph.Builder]] to the given create function. - * - * Partial graphs are allowed to have unconnected ports. - */ - def partial[S1 <: Shape, S2 <: Shape, S <: Shape, M1, M2, M](g1: Graph[S1, M1], g2: Graph[S2, M2], combineMat: function.Function2[M1, M2, M], + def create[S1 <: Shape, S2 <: Shape, S <: Shape, M1, M2, M](g1: Graph[S1, M1], g2: Graph[S2, M2], combineMat: function.Function2[M1, M2, M], block: function.Function3[FlowGraph.Builder[M], S1, S2, S]): Graph[S, M] = - scaladsl.FlowGraph.partial(g1, g2)(combineMat.apply _) { b => (s1, s2) => block.apply(b.asJava, s1, s2) } + scaladsl.FlowGraph.create(g1, g2)(combineMat.apply) { b => (s1, s2) => block.apply(b.asJava, s1, s2) } [3..21#/** - * Creates a new fully connected graph by importing the given graphs (using `builder.graph()`) and passing their resulting [[Shape]]s - * along with the[[FlowGraph.Builder]] to the given create function. - * - * The created graph must have all ports connected, otherwise an [[IllegalArgumentException]] is thrown. + * Creates a new [[Graph]] by importing the given graphs and passing their [[Shape]]s + * along with the [[FlowGraph.Builder]] to the given create function. */ - @throws(classOf[IllegalArgumentException]) - def closed1[[#S1 <: Shape#], [#M1#], M]([#g1: Graph[S1, M1]#], combineMat: function.Function1[[#M1#], M], - block: function.Procedure2[FlowGraph.Builder[M], [#S1#]]): RunnableGraph[M] = - scaladsl.FlowGraph.closed([#g1#])(combineMat.apply _) { b => ([#s1#]) => block.apply(b.asJava, [#s1#]) } - - /** - * Creates a new [[Graph]] by importing the given graphs (using `builder.graph()`) and passing their resulting [[Shape]]s - * along with the[[FlowGraph.Builder]] to the given create function. - * - * Partial graphs are allowed to have unconnected ports. - */ - def partial1[[#S1 <: Shape#], S <: Shape, [#M1#], M]([#g1: Graph[S1, M1]#], combineMat: function.Function1[[#M1#], M], + def create1[[#S1 <: Shape#], S <: Shape, [#M1#], M]([#g1: Graph[S1, M1]#], combineMat: function.Function1[[#M1#], M], block: function.Function2[FlowGraph.Builder[M], [#S1#], S]): Graph[S, M] = - scaladsl.FlowGraph.partial([#g1#])(combineMat.apply _) { b => ([#s1#]) => block.apply(b.asJava, [#s1#]) }# + scaladsl.FlowGraph.create([#g1#])(combineMat.apply) { b => ([#s1#]) => block.apply(b.asJava, [#s1#]) }# ] - } diff --git a/akka-stream/src/main/boilerplate/akka/stream/javadsl/SinkCreate.scala.template b/akka-stream/src/main/boilerplate/akka/stream/javadsl/SinkCreate.scala.template deleted file mode 100644 index 7cca393db6..0000000000 --- a/akka-stream/src/main/boilerplate/akka/stream/javadsl/SinkCreate.scala.template +++ /dev/null @@ -1,48 +0,0 @@ -/** - * Copyright (C) 2015 Typesafe Inc. - */ -package akka.stream.javadsl - -import akka.stream.scaladsl -import akka.stream.{ Inlet, Shape, Graph } -import akka.japi.function - -trait SinkCreate { - - /** - * Creates a `Sink` by using a `FlowGraph.Builder[Unit]` on a block that expects - * a [[FlowGraph.Builder]] and returns an [[Inlet]]. - */ - def create[T](block: function.Function[FlowGraph.Builder[Unit], Inlet[T]]): Sink[T, Unit] = - new Sink(scaladsl.Sink() { b ⇒ block.apply(b.asJava) }) - - /** - * Creates a `Sink` by importing the given graph (using `builder.graph()`) and calling the provided create function - * with the `FlowGraph.Builder[M]` and the [[Shape]] resulting from importing the graph. - * The create function is expected to return the created Sink's [[Inlet]]. - */ - def create[T, S <: Shape, M](g1: Graph[S, M], - block: function.Function2[FlowGraph.Builder[M], S, Inlet[T]]): Sink[T, M] = - new Sink(scaladsl.Sink(g1) { b ⇒ s => block.apply(b.asJava, s) }) - - /** - * Creates a `Sink` by importing the given graphs (using `builder.graph()`) and calling the provided create function - * with the `FlowGraph.Builder[M]` and the [[Shape]]s resulting from importing the graphs. - * The create function is expected to return the created Sink's [[Inlet]]. - */ - def create[T, S1 <: Shape, S2 <: Shape, M1, M2, M](g1: Graph[S1, M1], g2: Graph[S2, M2], combineMat: function.Function2[M1, M2, M], - block: function.Function3[FlowGraph.Builder[M], S1, S2, Inlet[T]]): Sink[T, M] = - new Sink(scaladsl.Sink(g1, g2)(combineMat.apply _) { b => (s1, s2) => block.apply(b.asJava, s1, s2) }) - - [3..21#/** - * Creates a `Sink` by importing the given graphs (using `builder.graph()`) and calling the provided create function - * with the `FlowGraph.Builder[M]` and the [[Shape]]s resulting from importing the graphs. - * The create function is expected to return the created Sink's [[Inlet]]. - */ - def create1[T, [#S1 <: Shape#], [#M1#], M]([#g1: Graph[S1, M1]#], combineMat: function.Function1[[#M1#], M], - block: function.Function2[FlowGraph.Builder[M], [#S1#], Inlet[T]]): Sink[T, M] = - new Sink(scaladsl.Sink([#g1#])(combineMat.apply _) { b => ([#s1#]) => block.apply(b.asJava, [#s1#]) })# - - ] - -} diff --git a/akka-stream/src/main/boilerplate/akka/stream/javadsl/SourceCreate.scala.template b/akka-stream/src/main/boilerplate/akka/stream/javadsl/SourceCreate.scala.template deleted file mode 100644 index 59e33c784c..0000000000 --- a/akka-stream/src/main/boilerplate/akka/stream/javadsl/SourceCreate.scala.template +++ /dev/null @@ -1,51 +0,0 @@ -/** - * Copyright (C) 2015 Typesafe Inc. - */ -package akka.stream.javadsl - -import akka.stream.scaladsl -import akka.stream.{ Outlet, Shape, Graph } -import akka.japi.function - -trait SourceCreate { - - /** - * Creates a `Source` by using a `FlowGraph.Builder[Unit]` on a block that expects - * a [[FlowGraph.Builder]] and returns an [[Outlet]]. - */ - def create[T](block: function.Function[FlowGraph.Builder[Unit], Outlet[T]]): Source[T, Unit] = - new Source(scaladsl.Source() { b ⇒ block.apply(b.asJava) }) - - /** - * Creates a `Source` by using a `FlowGraph.Builder[M]` on a block that expects - * a [[FlowGraph.Builder]] and 1 graph and then returns an [[Outlet]]. - * The graph will be imported (using `Builder.graph()`) and the resulting shape - * will be passed into the create block. - */ - def create[T, S <: Shape, M](g1: Graph[S, M], - block: function.Function2[FlowGraph.Builder[M], S, Outlet[T]]): Source[T, M] = - new Source(scaladsl.Source(g1) { b ⇒ s => block.apply(b.asJava, s) }) - - /** - * Creates a `Source` by using a `FlowGraph.Builder[M]` on a block that expects - * a [[FlowGraph.Builder]] and 2 graphs and then returns an [[Outlet]]. - * The graphs will be imported (using `Builder.graph()`) and the resulting shapes - * will be passed into the create block. - */ - def create[T, S1 <: Shape, S2 <: Shape, M1, M2, M](g1: Graph[S1, M1], g2: Graph[S2, M2], combineMat: function.Function2[M1, M2, M], - block: function.Function3[FlowGraph.Builder[M], S1, S2, Outlet[T]]): Source[T, M] = - new Source(scaladsl.Source(g1, g2)(combineMat.apply _) { b => (s1, s2) => block.apply(b.asJava, s1, s2) }) - - [3..21#/** - * Creates a `Source` by using a `FlowGraph.Builder[M]` on a block that expects - * a [[FlowGraph.Builder]] and 1 graphs and then returns an [[Outlet]]. - * The graphs will be imported (using `Builder.graph()`) and the resulting shapes - * will be passed into the create block. - */ - def create1[T, [#S1 <: Shape#], [#M1#], M]([#g1: Graph[S1, M1]#], combineMat: function.Function1[[#M1#], M], - block: function.Function2[FlowGraph.Builder[M], [#S1#], Outlet[T]]): Source[T, M] = - new Source(scaladsl.Source([#g1#])(combineMat.apply _) { b => ([#s1#]) => block.apply(b.asJava, [#s1#]) })# - - ] - -} diff --git a/akka-stream/src/main/boilerplate/akka/stream/scaladsl/BidiFlowApply.scala.template b/akka-stream/src/main/boilerplate/akka/stream/scaladsl/BidiFlowApply.scala.template deleted file mode 100644 index f34cecdfdc..0000000000 --- a/akka-stream/src/main/boilerplate/akka/stream/scaladsl/BidiFlowApply.scala.template +++ /dev/null @@ -1,47 +0,0 @@ -/** - * Copyright (C) 2014-2015 Typesafe Inc. - */ -package akka.stream.scaladsl - -import akka.stream.{ Shape, Inlet, Outlet, Graph, BidiShape } - -trait BidiFlowApply { - - /** - * Creates a BidiFlow by applying a [[FlowGraph.Builder]] to the given create block. - */ - def apply[I1, O1, I2, O2]()(block: FlowGraph.Builder[Unit] ⇒ BidiShape[I1, O1, I2, O2]): BidiFlow[I1, O1, I2, O2, Unit] = { - val builder = new FlowGraph.Builder - val shape = block(builder) - builder.buildBidiFlow(shape) - } - - /** - * Creates a BidiFlow by applying a [[FlowGraph.Builder]] to the given create block. - * The given graph will be imported (using `builder.graph()`) and the resulting [[Shape]] will be passed to the create block along with the builder. - */ - def apply[I1, O1, I2, O2, Mat](g1: Graph[Shape, Mat])(buildBlock: FlowGraph.Builder[Mat] => (g1.Shape) ⇒ BidiShape[I1, O1, I2, O2]): BidiFlow[I1, O1, I2, O2, Mat] = { - val builder = new FlowGraph.Builder - val p = builder.add(g1) - val shape = buildBlock(builder)(p) - builder.buildBidiFlow(shape) - } - - [2..#/** - * Creates a BidiFlow by applying a [[FlowGraph.Builder]] to the given create block. - * The given graph will be imported (using `builder.graph()`) and the resulting [[Shape]] will be passed to the create block along with the builder. - */ - def apply[I##1, O##1, I##2, O##2, [#M1#], Mat]([#g1: Graph[Shape, M1]#])(combineMat: ([#M1#]) => Mat)( - buildBlock: FlowGraph.Builder[Mat] => ([#g1.Shape#]) ⇒ BidiShape[I##1, O##1, I##2, O##2]): BidiFlow[I##1, O##1, I##2, O##2, Mat] = { - val builder = new FlowGraph.Builder - val curried = combineMat.curried - val p##1 = builder.add(g##1, (m##1: M##1) ⇒ curried(m##1)) - [2..#val p1 = builder.add(g1, (f: M1 ⇒ Any, m1: M1) ⇒ f(m1))# - ] - val shape = buildBlock(builder)([#p1#]) - builder.buildBidiFlow(shape) - }# - - ] - -} diff --git a/akka-stream/src/main/boilerplate/akka/stream/scaladsl/FlowApply.scala.template b/akka-stream/src/main/boilerplate/akka/stream/scaladsl/FlowApply.scala.template deleted file mode 100644 index 236f2ca5ff..0000000000 --- a/akka-stream/src/main/boilerplate/akka/stream/scaladsl/FlowApply.scala.template +++ /dev/null @@ -1,53 +0,0 @@ -/** - * Copyright (C) 2014-2015 Typesafe Inc. - */ -package akka.stream.scaladsl - -import akka.stream.{ Shape, Inlet, Outlet, Graph } - -trait FlowApply { - - /** - * Creates a Flow by passing a [[FlowGraph.Builder]] to the given create function. - * - * The create function is expected to return a pair of [[Inlet]] and [[Outlet]] which correspond to the created Flows input and output ports. - */ - def apply[I, O]()(block: FlowGraph.Builder[Unit] ⇒ (Inlet[I], Outlet[O])): Flow[I, O, Unit] = { - val builder = new FlowGraph.Builder - val (inlet, outlet) = block(builder) - builder.buildFlow(inlet, outlet) - } - - /** - * Creates a Flow by passing a [[FlowGraph.Builder]] to the given create function. - * The given graph will be imported (using `builder.graph()`) and the resulting shape will be passed to the create function along with the builder. - * - * The create function is expected to return a pair of [[Inlet]] and [[Outlet]] which correspond to the created Flows input and output ports. - */ - def apply[I, O, Mat](g1: Graph[Shape, Mat])(buildBlock: FlowGraph.Builder[Mat] => (g1.Shape) ⇒ (Inlet[I], Outlet[O])): Flow[I, O, Mat] = { - val builder = new FlowGraph.Builder - val p = builder.add(g1) - val (inlet, outlet) = buildBlock(builder)(p) - builder.buildFlow(inlet, outlet) - } - - [2..#/** - * Creates a Flow by passing a [[FlowGraph.Builder]] to the given create function. - * The given graph will be imported (using `builder.graph()`) and the resulting shape will be passed to the create function along with the builder. - * - * The create function is expected to return a pair of [[Inlet]] and [[Outlet]] which correspond to the created Flows input and output ports. - */ - def apply[I, O, [#M1#], Mat]([#g1: Graph[Shape, M1]#])(combineMat: ([#M1#]) => Mat)( - buildBlock: FlowGraph.Builder[Mat] => ([#g1.Shape#]) ⇒ (Inlet[I], Outlet[O])): Flow[I, O, Mat] = { - val builder = new FlowGraph.Builder - val curried = combineMat.curried - val p##1 = builder.add(g##1, (m##1: M##1) ⇒ curried(m##1)) - [2..#val p1 = builder.add(g1, (f: M1 ⇒ Any, m1: M1) ⇒ f(m1))# - ] - val (inlet, outlet) = buildBlock(builder)([#p1#]) - builder.buildFlow(inlet, outlet) - }# - - ] - -} diff --git a/akka-stream/src/main/boilerplate/akka/stream/scaladsl/GraphApply.scala.template b/akka-stream/src/main/boilerplate/akka/stream/scaladsl/GraphApply.scala.template index a703e5ed4b..a2c2e48ec2 100644 --- a/akka-stream/src/main/boilerplate/akka/stream/scaladsl/GraphApply.scala.template +++ b/akka-stream/src/main/boilerplate/akka/stream/scaladsl/GraphApply.scala.template @@ -8,24 +8,10 @@ import akka.stream.impl.StreamLayout.Module import akka.stream.{ Graph, Attributes, Shape } trait GraphApply { - - /** - * Creates a new fully connected graph by passing a [[FlowGraph.Builder]] to the given create function. - * - * The created graph must have all ports connected, otherwise an [[IllegalArgumentException]] is thrown. - */ - def closed()(buildBlock: (FlowGraph.Builder[Unit]) ⇒ Unit): RunnableGraph[Unit] = { - val builder = new FlowGraph.Builder - buildBlock(builder) - builder.buildRunnable() - } - /** * Creates a new [[Graph]] by passing a [[FlowGraph.Builder]] to the given create function. - * - * Partial graphs are allowed to have unconnected ports. */ - def partial[S <: Shape]()(buildBlock: FlowGraph.Builder[Unit] ⇒ S): Graph[S, Unit] = { + def create[S <: Shape]()(buildBlock: FlowGraph.Builder[Unit] ⇒ S): Graph[S, Unit] = { val builder = new FlowGraph.Builder val s = buildBlock(builder) val mod = builder.module.nest().replaceShape(s) @@ -34,25 +20,10 @@ trait GraphApply { } /** - * Creates a new fully connected graph by importing the given graph `g1` (using `builder.graph()`) and passing its resulting [[Shape]] - * along with the[[FlowGraph.Builder]] to the given create function. - * - * The created graph must have all ports connected, otherwise an [[IllegalArgumentException]] is thrown. + * Creates a new [[Graph]] by importing the given graph `g1` and passing its [[Shape]] + * along with the [[FlowGraph.Builder]] to the given create function. */ - def closed[Mat](g1: Graph[Shape, Mat])(buildBlock: FlowGraph.Builder[Mat] ⇒ (g1.Shape) ⇒ Unit): RunnableGraph[Mat] = { - val builder = new FlowGraph.Builder - val p1 = builder.add(g1) - buildBlock(builder)(p1) - builder.buildRunnable() - } - - /** - * Creates a new [[Graph]] by importing the given graph `g1` (using `builder.graph()`) and passing its resulting [[Shape]] - * along with the[[FlowGraph.Builder]] to the given create function. - * - * Partial graphs are allowed to have unconnected ports. - */ - def partial[S <: Shape, Mat](g1: Graph[Shape, Mat])(buildBlock: FlowGraph.Builder[Mat] ⇒ (g1.Shape) ⇒ S): Graph[S, Mat] = { + def create[S <: Shape, Mat](g1: Graph[Shape, Mat])(buildBlock: FlowGraph.Builder[Mat] ⇒ (g1.Shape) ⇒ S): Graph[S, Mat] = { val builder = new FlowGraph.Builder val s1 = builder.add(g1) val s = buildBlock(builder)(s1) @@ -62,29 +33,13 @@ trait GraphApply { } - [2..#/** - * Creates a new fully connected graph by importing the given graphs (using `builder.graph()`) and passing their resulting [[Shape]]s - * along with the[[FlowGraph.Builder]] to the given create function. - * - * The created graph must have all ports connected, otherwise an [[IllegalArgumentException]] is thrown. - */ - def closed[Mat, [#M1#]]([#g1: Graph[Shape, M1]#])(combineMat: ([#M1#]) ⇒ Mat)(buildBlock: FlowGraph.Builder[Mat] ⇒ ([#g1.Shape#]) ⇒ Unit): RunnableGraph[Mat] = { - val builder = new FlowGraph.Builder - val curried = combineMat.curried - val s##1 = builder.add(g##1, (m##1: M##1) ⇒ curried(m##1)) - [2..#val s1 = builder.add(g1, (f: M1 ⇒ Any, m1: M1) ⇒ f(m1))# - ] - buildBlock(builder)([#s1#]) - builder.buildRunnable() - } + [2..# /** - * Creates a new [[Graph]] by importing the given graphs (using `builder.graph()`) and passing their resulting [[Shape]]s - * along with the[[FlowGraph.Builder]] to the given create function. - * - * Partial graphs are allowed to have unconnected ports. + * Creates a new [[Graph]] by importing the given graphs and passing their [[Shape]]s + * along with the [[FlowGraph.Builder]] to the given create function. */ - def partial[S <: Shape, Mat, [#M1#]]([#g1: Graph[Shape, M1]#])(combineMat: ([#M1#]) ⇒ Mat)(buildBlock: FlowGraph.Builder[Mat] ⇒ ([#g1.Shape#]) ⇒ S): Graph[S, Mat] = { + def create[S <: Shape, Mat, [#M1#]]([#g1: Graph[Shape, M1]#])(combineMat: ([#M1#]) ⇒ Mat)(buildBlock: FlowGraph.Builder[Mat] ⇒ ([#g1.Shape#]) ⇒ S): Graph[S, Mat] = { val builder = new FlowGraph.Builder val curried = combineMat.curried val s##1 = builder.add(g##1, (m##1: M##1) ⇒ curried(m##1)) @@ -97,15 +52,13 @@ trait GraphApply { }# ] - - } /** * INTERNAL API */ private[stream] object GraphApply { - class GraphImpl[S <: Shape, Mat](override val shape: S, private[stream] override val module: StreamLayout.Module) + final class GraphImpl[S <: Shape, Mat](override val shape: S, private[stream] override val module: StreamLayout.Module) extends Graph[S, Mat] { override def withAttributes(attr: Attributes): Graph[S, Mat] = @@ -113,5 +66,4 @@ private[stream] object GraphApply { override def named(name: String): Graph[S, Mat] = withAttributes(Attributes.name(name)) } - } diff --git a/akka-stream/src/main/boilerplate/akka/stream/scaladsl/SinkApply.scala.template b/akka-stream/src/main/boilerplate/akka/stream/scaladsl/SinkApply.scala.template deleted file mode 100644 index 889c6488a7..0000000000 --- a/akka-stream/src/main/boilerplate/akka/stream/scaladsl/SinkApply.scala.template +++ /dev/null @@ -1,50 +0,0 @@ -/** - * Copyright (C) 2014-2015 Typesafe Inc. - */ -package akka.stream.scaladsl - -import akka.stream.{ Inlet, Graph, Shape } - -trait SinkApply { - - /** - * Creates a `Sink` by using a `FlowGraph.Builder[Unit]` on a block that expects - * a [[FlowGraph.Builder]] and returns an [[Inlet]]. - */ - def apply[In]()(buildBlock: FlowGraph.Builder[Unit] => Inlet[In]): Sink[In, Unit] = { - val builder = new FlowGraph.Builder - val inlet = buildBlock(builder) - builder.buildSink(inlet) - } - - /** - * Creates a `Sink` by importing the given graph (using `builder.graph()`) and calling the provided create function - * with the `FlowGraph.Builder[M]` and the [[Shape]] resulting from importing the graph. - * The create function is expected to return the created Sink's [[Inlet]]. - */ - def apply[In, Mat](g1: Graph[Shape, Mat])(buildBlock: FlowGraph.Builder[Mat] => (g1.Shape) => Inlet[In]): Sink[In, Mat] = { - val builder = new FlowGraph.Builder - val s = builder.add(g1) - val inlet = buildBlock(builder)(s) - builder.buildSink(inlet) - } - - [2..#/** - * Creates a `Sink` by importing the given graphs (using `builder.graph()`) and calling the provided create function - * with the `FlowGraph.Builder[M]` and the [[Shape]]s resulting from importing the graphs. - * The create function is expected to return the created Sink's [[Inlet]]. - */ - def apply[In, [#M1#], Mat]([#g1: Graph[Shape, M1]#])(combineMat: ([#M1#]) ⇒ Mat)( - buildBlock: FlowGraph.Builder[Mat] ⇒ ([#g1.Shape#]) ⇒ Inlet[In]): Sink[In, Mat] = { - val builder = new FlowGraph.Builder - val curried = combineMat.curried - val s##1 = builder.add(g##1, (m##1: M##1) ⇒ curried(m##1)) - [2..#val s1 = builder.add(g1, (f: M1 ⇒ Any, m1: M1) ⇒ f(m1))# - ] - val inlet = buildBlock(builder)([#s1#]) - builder.buildSink(inlet) - }# - - ] - -} diff --git a/akka-stream/src/main/boilerplate/akka/stream/scaladsl/SourceApply.scala.template b/akka-stream/src/main/boilerplate/akka/stream/scaladsl/SourceApply.scala.template deleted file mode 100644 index b4af84cfca..0000000000 --- a/akka-stream/src/main/boilerplate/akka/stream/scaladsl/SourceApply.scala.template +++ /dev/null @@ -1,52 +0,0 @@ -/** - * Copyright (C) 2014-2015 Typesafe Inc. - */ -package akka.stream.scaladsl - -import akka.stream.{ Outlet, Shape, Graph } - -trait SourceApply { - - /** - * Creates a `Source` by using a `FlowGraph.Builder[Unit]` on a block that expects - * a [[FlowGraph.Builder]] and returns an [[Outlet]]. - */ - def apply[Out]()(buildBlock: FlowGraph.Builder[Unit] => Outlet[Out]): Source[Out, Unit] = { - val builder = new FlowGraph.Builder - val port = buildBlock(builder) - builder.buildSource(port) - } - - /** - * Creates a `Source` by using a `FlowGraph.Builder[M]` on a block that expects - * a [[FlowGraph.Builder]] and 1 graph and then returns an [[Outlet]]. - * The graph will be imported (using `Builder.graph()`) and the resulting shape - * will be passed into the create block. - */ - def apply[Out, Mat](g1: Graph[Shape, Mat])(buildBlock: FlowGraph.Builder[Mat] => (g1.Shape) => Outlet[Out]): Source[Out, Mat] = { - val builder = new FlowGraph.Builder - val p = builder.add(g1) - val port = buildBlock(builder)(p) - builder.buildSource(port) - } - - [2..#/** - * Creates a `Source` by using a `FlowGraph.Builder[M]` on a block that expects - * a [[FlowGraph.Builder]] and 2 graphs and then returns an [[Outlet]]. - * The graphs will be imported (using `Builder.graph()`) and the resulting shapes - * will be passed into the create block. - */ - def apply[Out, [#M1#], Mat]([#g1: Graph[Shape, M1]#])(combineMat: ([#M1#]) ⇒ Mat)( - buildBlock: FlowGraph.Builder[Mat] ⇒ ([#g1.Shape#]) ⇒ Outlet[Out]): Source[Out, Mat] = { - val builder = new FlowGraph.Builder - val curried = combineMat.curried - val p##1 = builder.add(g##1, (m##1: M##1) ⇒ curried(m##1)) - [2..#val p1 = builder.add(g1, (f: M1 ⇒ Any, m1: M1) ⇒ f(m1))# - ] - val port = buildBlock(builder)([#p1#]) - builder.buildSource(port) - }# - - ] - -} diff --git a/akka-stream/src/main/scala/akka/stream/Shape.scala b/akka-stream/src/main/scala/akka/stream/Shape.scala index 336b14a423..b47774cfa1 100644 --- a/akka-stream/src/main/scala/akka/stream/Shape.scala +++ b/akka-stream/src/main/scala/akka/stream/Shape.scala @@ -177,7 +177,7 @@ object ClosedShape extends ClosedShape { /** * Java API: obtain ClosedShape instance */ - def getInstance: Shape = this + def getInstance: ClosedShape = this } /** diff --git a/akka-stream/src/main/scala/akka/stream/impl/CompletedPublishers.scala b/akka-stream/src/main/scala/akka/stream/impl/CompletedPublishers.scala index f9638cfd6c..89d51285e8 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/CompletedPublishers.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/CompletedPublishers.scala @@ -4,6 +4,7 @@ package akka.stream.impl import org.reactivestreams.{ Subscriber, Publisher, Subscription } +import scala.concurrent.{ ExecutionContext, Promise } /** * INTERNAL API @@ -41,6 +42,9 @@ private[akka] final case class ErrorPublisher(t: Throwable, name: String) extend override def toString: String = name } +/** + * INTERNAL API + */ private[akka] final case class SingleElementPublisher[T](value: T, name: String) extends Publisher[T] { import ReactiveStreamsCompliance._ @@ -69,7 +73,52 @@ private[akka] final case class SingleElementPublisher[T](value: T, name: String) } catch { case _: SpecViolation ⇒ // nothing we can do } - def apply[T]: Publisher[T] = this.asInstanceOf[Publisher[T]] + + override def toString: String = name +} + +/** + * INTERNAL API + */ +private[akka] final case class MaybePublisher[T]( + promise: Promise[Option[T]], + name: String)(implicit ec: ExecutionContext) extends Publisher[T] { + import ReactiveStreamsCompliance._ + + private[this] class MaybeSubscription(subscriber: Subscriber[_ >: T]) extends Subscription { + private[this] var done: Boolean = false + override def cancel(): Unit = { + done = true + promise.trySuccess(None) + } + + override def request(elements: Long): Unit = { + if (elements < 1) rejectDueToNonPositiveDemand(subscriber) + if (!done) { + done = true + promise.future foreach { + // We consciously do not catch SpecViolation here, it will be reported to the ExecutionContext + case Some(v) ⇒ + tryOnNext(subscriber, v) + tryOnComplete(subscriber) + case None ⇒ + tryOnComplete(subscriber) + } + } + } + } + + override def subscribe(subscriber: Subscriber[_ >: T]): Unit = + try { + requireNonNullSubscriber(subscriber) + tryOnSubscribe(subscriber, new MaybeSubscription(subscriber)) + promise.future onFailure { + case error ⇒ tryOnError(subscriber, error) + } + } catch { + case sv: SpecViolation ⇒ ec.reportFailure(sv) + } + override def toString: String = name } diff --git a/akka-stream/src/main/scala/akka/stream/impl/ConstantFun.scala b/akka-stream/src/main/scala/akka/stream/impl/ConstantFun.scala new file mode 100644 index 0000000000..f65bec90b6 --- /dev/null +++ b/akka-stream/src/main/scala/akka/stream/impl/ConstantFun.scala @@ -0,0 +1,25 @@ +/** + * Copyright (C) 2015 Typesafe Inc. + */ +package akka.stream.impl + +import akka.japi.function.{ Function ⇒ JFun, Function2 ⇒ JFun2 } +import akka.japi.{ Pair ⇒ JPair } + +private[stream] object ConstantFun { + private[this] val JavaIdentityFunction = new JFun[Any, Any] { + @throws(classOf[Exception]) override def apply(param: Any): Any = param + } + + val JavaPairFunction = new JFun2[AnyRef, AnyRef, AnyRef JPair AnyRef] { + def apply(p1: AnyRef, p2: AnyRef): AnyRef JPair AnyRef = JPair(p1, p2) + } + + private[this] val ScalaIdentityFunction = (a: Any) ⇒ a + + def javaCreatePairFunction[A, B]: JFun2[A, B, JPair[A, B]] = JavaPairFunction.asInstanceOf[JFun2[A, B, JPair[A, B]]] + + def javaIdentityFunction[T]: JFun[T, T] = JavaIdentityFunction.asInstanceOf[JFun[T, T]] + + def scalaIdentityFunction[T]: T ⇒ T = ScalaIdentityFunction.asInstanceOf[T ⇒ T] +} diff --git a/akka-stream/src/main/scala/akka/stream/impl/Modules.scala b/akka-stream/src/main/scala/akka/stream/impl/Modules.scala index a1c25c33ad..b9320af0b3 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/Modules.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/Modules.scala @@ -3,8 +3,6 @@ */ package akka.stream.impl -import java.util.concurrent.atomic.AtomicBoolean - import akka.actor._ import akka.stream._ import akka.stream.impl.AcknowledgePublisher.{ Ok, Rejected } @@ -16,7 +14,6 @@ import scala.annotation.unchecked.uncheckedVariance import scala.concurrent.duration.{ FiniteDuration, _ } import scala.concurrent.{ Future, Promise } import scala.language.postfixOps -import scala.util.{ Failure, Success } /** * INTERNAL API @@ -78,31 +75,13 @@ private[akka] final class PublisherSource[Out](p: Publisher[Out], val attributes /** * INTERNAL API */ -private[akka] final class LazyEmptySource[Out](val attributes: Attributes, shape: SourceShape[Out]) extends SourceModule[Out, Promise[Unit]](shape) { - import ReactiveStreamsCompliance._ - +private[akka] final class MaybeSource[Out](val attributes: Attributes, shape: SourceShape[Out]) extends SourceModule[Out, Promise[Option[Out]]](shape) { override def create(context: MaterializationContext) = { - val p = Promise[Unit]() - - val pub = new Publisher[Unit] { - override def subscribe(s: Subscriber[_ >: Unit]) = { - requireNonNullSubscriber(s) - tryOnSubscribe(s, new Subscription { - override def request(n: Long): Unit = () - override def cancel(): Unit = p.trySuccess(()) - }) - p.future.onComplete { - case Success(_) ⇒ tryOnComplete(s) - case Failure(ex) ⇒ tryOnError(s, ex) // due to external signal - }(context.materializer.executionContext) - } - } - - pub.asInstanceOf[Publisher[Out]] → p + val p = Promise[Option[Out]]() + new MaybePublisher[Out](p, attributes.nameOrDefault("MaybeSource"))(context.materializer.executionContext) → p } - - override protected def newInstance(shape: SourceShape[Out]): SourceModule[Out, Promise[Unit]] = new LazyEmptySource[Out](attributes, shape) - override def withAttributes(attr: Attributes): Module = new LazyEmptySource(attr, amendShape(attr)) + override protected def newInstance(shape: SourceShape[Out]): SourceModule[Out, Promise[Option[Out]]] = new MaybeSource[Out](attributes, shape) + override def withAttributes(attr: Attributes): Module = new MaybeSource(attr, amendShape(attr)) } /** diff --git a/akka-stream/src/main/scala/akka/stream/impl/Stages.scala b/akka-stream/src/main/scala/akka/stream/impl/Stages.scala index 041100040d..bab3036f1a 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/Stages.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/Stages.scala @@ -67,7 +67,7 @@ private[stream] object Stages { val tickSource = name("tickSource") val singleSource = name("singleSource") val emptySource = name("emptySource") - val lazyEmptySource = name("lazyEmptySource") + val maybeSource = name("MaybeSource") val failedSource = name("failedSource") val concatSource = name("concatSource") val concatMatSource = name("concatMatSource") diff --git a/akka-stream/src/main/scala/akka/stream/io/Framing.scala b/akka-stream/src/main/scala/akka/stream/io/Framing.scala index be4fb4570a..b8d171358d 100644 --- a/akka-stream/src/main/scala/akka/stream/io/Framing.scala +++ b/akka-stream/src/main/scala/akka/stream/io/Framing.scala @@ -97,7 +97,7 @@ object Framing { }) - BidiFlow.wrap(encoder, decoder)(Keep.left) + BidiFlow.fromFlowsMat(encoder, decoder)(Keep.left) } private trait IntDecoder { diff --git a/akka-stream/src/main/scala/akka/stream/io/SslTls.scala b/akka-stream/src/main/scala/akka/stream/io/SslTls.scala index 37d9c9a097..7eb39d28a5 100644 --- a/akka-stream/src/main/scala/akka/stream/io/SslTls.scala +++ b/akka-stream/src/main/scala/akka/stream/io/SslTls.scala @@ -155,13 +155,13 @@ object SslTls { */ object SslTlsPlacebo { val forScala: scaladsl.BidiFlow[SslTlsOutbound, ByteString, ByteString, SessionBytes, Unit] = - scaladsl.BidiFlow() { implicit b ⇒ + scaladsl.BidiFlow.fromGraph(scaladsl.FlowGraph.create() { implicit b ⇒ // this constructs a session for (invalid) protocol SSL_NULL_WITH_NULL_NULL val session = SSLContext.getDefault.createSSLEngine.getSession val top = b.add(scaladsl.Flow[SslTlsOutbound].collect { case SendBytes(bytes) ⇒ bytes }) val bottom = b.add(scaladsl.Flow[ByteString].map(SessionBytes(session, _))) BidiShape.fromFlows(top, bottom) - } + }) val forJava: javadsl.BidiFlow[SslTlsOutbound, ByteString, ByteString, SessionBytes, Unit] = new javadsl.BidiFlow(forScala) } diff --git a/akka-stream/src/main/scala/akka/stream/io/Timeouts.scala b/akka-stream/src/main/scala/akka/stream/io/Timeouts.scala index e6c1715ab2..aed5dc375b 100644 --- a/akka-stream/src/main/scala/akka/stream/io/Timeouts.scala +++ b/akka-stream/src/main/scala/akka/stream/io/Timeouts.scala @@ -26,21 +26,21 @@ object Timeouts { * with a [[TimeoutException]]. */ def initalTimeout[T](timeout: FiniteDuration): Flow[T, T, Unit] = - Flow.wrap(new InitialTimeout[T](timeout)) + Flow.fromGraph(new InitialTimeout[T](timeout)) /** * If the completion of the stream does not happen until the provided timeout, the stream is failed * with a [[TimeoutException]]. */ def completionTimeout[T](timeout: FiniteDuration): Flow[T, T, Unit] = - Flow.wrap(new CompletionTimeout[T](timeout)) + Flow.fromGraph(new CompletionTimeout[T](timeout)) /** * If the time between two processed elements exceed the provided timeout, the stream is failed * with a [[TimeoutException]]. */ def idleTimeout[T](timeout: FiniteDuration): Flow[T, T, Unit] = - Flow.wrap(new IdleTimeout[T](timeout)) + Flow.fromGraph(new IdleTimeout[T](timeout)) /** * If the time between two processed elements *in any direction* exceed the provided timeout, the stream is failed @@ -52,7 +52,7 @@ object Timeouts { * the *joint* frequencies of the elements in both directions. */ def idleTimeoutBidi[A, B](timeout: FiniteDuration): BidiFlow[A, A, B, B, Unit] = - BidiFlow.wrap(new IdleTimeoutBidi[A, B](timeout)) + BidiFlow.fromGraph(new IdleTimeoutBidi[A, B](timeout)) private def idleTimeoutCheckInterval(timeout: FiniteDuration): FiniteDuration = { import scala.concurrent.duration._ diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/BidiFlow.scala b/akka-stream/src/main/scala/akka/stream/javadsl/BidiFlow.scala index 15d1283916..ceb811904f 100644 --- a/akka-stream/src/main/scala/akka/stream/javadsl/BidiFlow.scala +++ b/akka-stream/src/main/scala/akka/stream/javadsl/BidiFlow.scala @@ -7,17 +7,14 @@ import akka.japi.function import akka.stream._ object BidiFlow { - - val factory: BidiFlowCreate = new BidiFlowCreate {} - /** * A graph with the shape of a BidiFlow logically is a BidiFlow, this method makes * it so also in type. */ - def wrap[I1, O1, I2, O2, M](g: Graph[BidiShape[I1, O1, I2, O2], M]): BidiFlow[I1, O1, I2, O2, M] = + def fromGraph[I1, O1, I2, O2, M](g: Graph[BidiShape[I1, O1, I2, O2], M]): BidiFlow[I1, O1, I2, O2, M] = g match { case bidi: BidiFlow[I1, O1, I2, O2, M] ⇒ bidi - case other ⇒ new BidiFlow(scaladsl.BidiFlow.wrap(other)) + case other ⇒ new BidiFlow(scaladsl.BidiFlow.fromGraph(other)) } /** @@ -39,19 +36,42 @@ object BidiFlow { * }}} * */ - def wrap[I1, O1, I2, O2, M1, M2, M]( + def fromFlowsMat[I1, O1, I2, O2, M1, M2, M]( flow1: Graph[FlowShape[I1, O1], M1], flow2: Graph[FlowShape[I2, O2], M2], combine: function.Function2[M1, M2, M]): BidiFlow[I1, O1, I2, O2, M] = { - new BidiFlow(scaladsl.BidiFlow.wrap(flow1, flow2)(combinerToScala(combine))) + new BidiFlow(scaladsl.BidiFlow.fromFlowsMat(flow1, flow2)(combinerToScala(combine))) } + /** + * Wraps two Flows to create a ''BidiFlow''. The materialized value of the resulting BidiFlow is Unit. + * + * {{{ + * +----------------------------+ + * | Resulting BidiFlow | + * | | + * | +----------------------+ | + * I1 ~~> | Flow1 | ~~> O1 + * | +----------------------+ | + * | | + * | +----------------------+ | + * O2 <~~ | Flow2 | <~~ I2 + * | +----------------------+ | + * +----------------------------+ + * }}} + * + */ + def fromFlows[I1, O1, I2, O2, M1, M2]( + flow1: Graph[FlowShape[I1, O1], M1], + flow2: Graph[FlowShape[I2, O2], M2]): BidiFlow[I1, O1, I2, O2, Unit] = + new BidiFlow(scaladsl.BidiFlow.fromFlows(flow1, flow2)) + /** * Create a BidiFlow where the top and bottom flows are just one simple mapping * stage each, expressed by the two functions. */ def fromFunctions[I1, O1, I2, O2](top: function.Function[I1, O1], bottom: function.Function[I2, O2]): BidiFlow[I1, O1, I2, O2, Unit] = - new BidiFlow(scaladsl.BidiFlow(top.apply _, bottom.apply _)) + new BidiFlow(scaladsl.BidiFlow.fromFunctions(top.apply _, bottom.apply _)) } @@ -84,7 +104,8 @@ class BidiFlow[-I1, +O1, -I2, +O2, +Mat](delegate: scaladsl.BidiFlow[I1, O1, I2, new BidiFlow(delegate.atop(bidi.asScala)) /** - * Add the given BidiFlow as the next step in a bidirectional transformation + * Add the given BidiFlow as the next step in a bidirectional transformation 161 + * * pipeline. By convention protocol stacks are growing to the left: the right most is the bottom * layer, the closest to the metal. * {{{ diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/Flow.scala b/akka-stream/src/main/scala/akka/stream/javadsl/Flow.scala index fe92630eaf..51eb310596 100644 --- a/akka-stream/src/main/scala/akka/stream/javadsl/Flow.scala +++ b/akka-stream/src/main/scala/akka/stream/javadsl/Flow.scala @@ -4,9 +4,8 @@ package akka.stream.javadsl import akka.event.LoggingAdapter -import akka.japi.{ Pair, function } -import akka.stream.impl.Stages.Intersperse -import akka.stream.impl.{ ReactiveStreamsCompliance, StreamLayout } +import akka.japi.{ function, Pair } +import akka.stream.impl.{ ConstantFun, StreamLayout } import akka.stream.{ scaladsl, _ } import akka.stream.stage.Stage import org.reactivestreams.Processor @@ -18,48 +17,44 @@ import scala.concurrent.duration.FiniteDuration object Flow { - val factory: FlowCreate = new FlowCreate {} - - /** Adapt [[scaladsl.Flow]] for use within Java DSL */ - def adapt[I, O, M](flow: scaladsl.Flow[I, O, M]): javadsl.Flow[I, O, M] = - new Flow(flow) + private[this] val _identity = new javadsl.Flow(scaladsl.Flow[Any]) /** Create a `Flow` which can process elements of type `T`. */ - def empty[T](): javadsl.Flow[T, T, Unit] = - Flow.create() - - /** Create a `Flow` which can process elements of type `T`. */ - def create[T](): javadsl.Flow[T, T, Unit] = - adapt(scaladsl.Flow[T]) + def create[T](): javadsl.Flow[T, T, Unit] = fromGraph(scaladsl.Flow[T]) def create[I, O](processorFactory: function.Creator[Processor[I, O]]): javadsl.Flow[I, O, Unit] = - adapt(scaladsl.Flow(() ⇒ processorFactory.create())) + new Flow(scaladsl.Flow(() ⇒ processorFactory.create())) /** Create a `Flow` which can process elements of type `T`. */ - def of[T](clazz: Class[T]): javadsl.Flow[T, T, Unit] = - create[T]() + def of[T](clazz: Class[T]): javadsl.Flow[T, T, Unit] = create[T]() /** - * A graph with the shape of a flow logically is a flow, this method makes - * it so also in type. + * A graph with the shape of a flow logically is a flow, this method makes it so also in type. */ - def wrap[I, O, M](g: Graph[FlowShape[I, O], M]): Flow[I, O, M] = + def fromGraph[I, O, M](g: Graph[FlowShape[I, O], M]): Flow[I, O, M] = g match { - case f: Flow[I, O, M] ⇒ f - case other ⇒ new Flow(scaladsl.Flow.wrap(other)) + case f: Flow[I, O, M] ⇒ f + case f: scaladsl.Flow[I, O, M] if f.isIdentity ⇒ _identity.asInstanceOf[Flow[I, O, M]] + case other ⇒ new Flow(scaladsl.Flow.fromGraph(other)) } /** - * Helper to create `Flow` from a pair of sink and source. + * Helper to create `Flow` from a `Sink`and a `Source`. */ - def wrap[I, O, M1, M2, M]( - sink: Graph[SinkShape[I], M1], - source: Graph[SourceShape[O], M2], - combine: function.Function2[M1, M2, M]): Flow[I, O, M] = new Flow(scaladsl.Flow.wrap(sink, source)(combine.apply _)) + def fromSinkAndSource[I, O](sink: Graph[SinkShape[I], _], source: Graph[SourceShape[O], _]): Flow[I, O, Unit] = + new Flow(scaladsl.Flow.fromSinkAndSourceMat(sink, source)(scaladsl.Keep.none)) + + /** + * Helper to create `Flow` from a `Sink`and a `Source`. + */ + def fromSinkAndSourceMat[I, O, M1, M2, M]( + sink: Graph[SinkShape[I], M1], source: Graph[SourceShape[O], M2], + combine: function.Function2[M1, M2, M]): Flow[I, O, M] = + new Flow(scaladsl.Flow.fromSinkAndSourceMat(sink, source)(combinerToScala(combine))) } /** Create a `Flow` which can process elements of type `T`. */ -class Flow[-In, +Out, +Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends Graph[FlowShape[In, Out], Mat] { +final class Flow[-In, +Out, +Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends Graph[FlowShape[In, Out], Mat] { import scala.collection.JavaConverters._ override def shape: FlowShape[In, Out] = delegate.shape @@ -131,13 +126,13 @@ class Flow[-In, +Out, +Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends Graph * Join this [[Flow]] to another [[Flow]], by cross connecting the inputs and outputs, creating a [[RunnableGraph]] */ def join[M](flow: Graph[FlowShape[Out, In], M]): javadsl.RunnableGraph[Mat] = - new RunnableGraphAdapter(delegate.join(flow)) + RunnableGraph.fromGraph(delegate.join(flow)) /** * Join this [[Flow]] to another [[Flow]], by cross connecting the inputs and outputs, creating a [[RunnableGraph]] */ def joinMat[M, M2](flow: Graph[FlowShape[Out, In], M], combine: function.Function2[Mat, M, M2]): javadsl.RunnableGraph[M2] = - new RunnableGraphAdapter(delegate.joinMat(flow)(combinerToScala(combine))) + RunnableGraph.fromGraph(delegate.joinMat(flow)(combinerToScala(combine))) /** * Join this [[Flow]] to a [[BidiFlow]] to close off the “top” of the protocol stack: @@ -944,16 +939,15 @@ class Flow[-In, +Out, +Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends Graph * @see [[#zip]] */ def zipMat[T, M, M2](that: Graph[SourceShape[T], M], - matF: function.Function2[Mat, M, M2]): javadsl.Flow[In, Out @uncheckedVariance Pair T, M2] = { - val f = new function.Function2[FlowGraph.Builder[M], SourceShape[T], Inlet[Out]Pair Outlet[Out Pair T]] { - override def apply(b: FlowGraph.Builder[M], s: SourceShape[T]): Inlet[Out] Pair Outlet[Out Pair T] = { - val zip = b.graph(Zip.create[Out, T]) - b.from(s).toInlet(zip.in1) - new Pair(zip.in0, zip.out) - } - } - this.viaMat(Flow.factory.create(that, f), matF) - } + matF: function.Function2[Mat, M, M2]): javadsl.Flow[In, Out @uncheckedVariance Pair T, M2] = + this.viaMat(Flow.fromGraph(FlowGraph.create(that, + new function.Function2[FlowGraph.Builder[M], SourceShape[T], FlowShape[Out, Out @ uncheckedVariance Pair T]] { + def apply(b: FlowGraph.Builder[M], s: SourceShape[T]): FlowShape[Out, Out @uncheckedVariance Pair T] = { + val zip: FanInShape2[Out, T, Out Pair T] = b.add(Zip.create[Out, T]) + b.from(s).toInlet(zip.in1) + FlowShape(zip.in0, zip.out) + } + })), matF) /** * Put together the elements of current [[Flow]] and the given [[Source]] @@ -1049,7 +1043,7 @@ class Flow[-In, +Out, +Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends Graph * '''Cancels when''' downstream cancels */ def log(name: String, log: LoggingAdapter): javadsl.Flow[In, Out, Mat] = - this.log(name, javaIdentityFunction[Out], log) + this.log(name, ConstantFun.javaIdentityFunction[Out], log) /** * Logs elements flowing through the stream as well as completion and erroring. @@ -1060,7 +1054,7 @@ class Flow[-In, +Out, +Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends Graph * Uses an internally created [[LoggingAdapter]] which uses `akka.stream.Log` as it's source (use this class to configure slf4j loggers). */ def log(name: String): javadsl.Flow[In, Out, Mat] = - this.log(name, javaIdentityFunction[Out], null) + this.log(name, ConstantFun.javaIdentityFunction[Out], null) /** * Converts this Flow to a [[RunnableGraph]] that materializes to a Reactive Streams [[org.reactivestreams.Processor]] @@ -1070,16 +1064,50 @@ class Flow[-In, +Out, +Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends Graph * @return A [[RunnableGraph]] that materializes to a Processor when run() is called on it. */ def toProcessor: RunnableGraph[Processor[In @uncheckedVariance, Out @uncheckedVariance]] = { - new RunnableGraphAdapter(delegate.toProcessor) + RunnableGraph.fromGraph(delegate.toProcessor) } } +object RunnableGraph { + /** + * A graph with a closed shape is logically a runnable graph, this method makes + * it so also in type. + */ + def fromGraph[Mat](graph: Graph[ClosedShape, Mat]): RunnableGraph[Mat] = + graph match { + case r: RunnableGraph[Mat] ⇒ r + case other ⇒ new RunnableGraphAdapter[Mat](scaladsl.RunnableGraph.fromGraph(graph)) + } + + /** INTERNAL API */ + private final class RunnableGraphAdapter[Mat](runnable: scaladsl.RunnableGraph[Mat]) extends RunnableGraph[Mat] { + def shape = ClosedShape + def module = runnable.module + override def mapMaterializedValue[Mat2](f: function.Function[Mat, Mat2]): RunnableGraphAdapter[Mat2] = + new RunnableGraphAdapter(runnable.mapMaterializedValue(f.apply _)) + + override def run(materializer: Materializer): Mat = runnable.run()(materializer) + + override def withAttributes(attr: Attributes): RunnableGraphAdapter[Mat] = { + val newRunnable = runnable.withAttributes(attr) + if (newRunnable eq runnable) this + else new RunnableGraphAdapter(newRunnable) + } + + override def named(name: String): RunnableGraphAdapter[Mat] = { + val newRunnable = runnable.named(name) + if (newRunnable eq runnable) this + else new RunnableGraphAdapter(newRunnable) + } + } +} /** * Java API * * Flow with attached input and output, can be executed. */ -trait RunnableGraph[+Mat] extends Graph[ClosedShape, Mat] { +abstract class RunnableGraph[+Mat] extends Graph[ClosedShape, Mat] { + /** * Run this flow and return the materialized values of the flow. */ @@ -1089,18 +1117,3 @@ trait RunnableGraph[+Mat] extends Graph[ClosedShape, Mat] { */ def mapMaterializedValue[Mat2](f: function.Function[Mat, Mat2]): RunnableGraph[Mat2] } - -/** INTERNAL API */ -private[akka] class RunnableGraphAdapter[Mat](runnable: scaladsl.RunnableGraph[Mat]) extends RunnableGraph[Mat] { - def shape = ClosedShape - def module = runnable.module - override def mapMaterializedValue[Mat2](f: function.Function[Mat, Mat2]): RunnableGraph[Mat2] = - new RunnableGraphAdapter(runnable.mapMaterializedValue(f.apply _)) - override def run(materializer: Materializer): Mat = runnable.run()(materializer) - - override def withAttributes(attr: Attributes): RunnableGraph[Mat] = - new RunnableGraphAdapter(runnable.withAttributes(attr)) - - override def named(name: String): RunnableGraph[Mat] = - new RunnableGraphAdapter(runnable.named(name)) -} diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/Graph.scala b/akka-stream/src/main/scala/akka/stream/javadsl/Graph.scala index 89f87c97e5..844505ca92 100644 --- a/akka-stream/src/main/scala/akka/stream/javadsl/Graph.scala +++ b/akka-stream/src/main/scala/akka/stream/javadsl/Graph.scala @@ -6,6 +6,7 @@ package akka.stream.javadsl import akka.stream._ import akka.japi.Pair import scala.annotation.unchecked.uncheckedVariance +import akka.stream.impl.ConstantFun /** * Merge several streams, taking elements as they arrive from input streams @@ -227,7 +228,7 @@ object Unzip { * Creates a new `Unzip` stage with the specified output types. */ def create[A, B](): Graph[FanOutShape2[A Pair B, A, B], Unit] = - UnzipWith.create(JavaIdentityFunction.asInstanceOf[Function[Pair[A, B], Pair[A, B]]]) + UnzipWith.create(ConstantFun.javaIdentityFunction[Pair[A, B]]) /** * Creates a new `Unzip` stage with the specified output types. @@ -269,9 +270,7 @@ object Concat { // flow graph // -object FlowGraph { - - val factory: GraphCreate = new GraphCreate {} +object FlowGraph extends GraphCreate { /** * Start building a [[FlowGraph]]. @@ -284,20 +283,12 @@ object FlowGraph { final class Builder[+Mat]()(private implicit val delegate: scaladsl.FlowGraph.Builder[Mat]) { self ⇒ import akka.stream.scaladsl.FlowGraph.Implicits._ - def flow[A, B, M](from: Outlet[A], via: Graph[FlowShape[A, B], M], to: Inlet[B]): Unit = delegate.addEdge(from, via, to) - - def edge[T](from: Outlet[T], to: Inlet[T]): Unit = delegate.addEdge(from, to) - /** * Import a graph into this module, performing a deep copy, discarding its * materialized value and returning the copied Ports that are now to be * connected. */ - def graph[S <: Shape](graph: Graph[S, _]): S = delegate.add(graph) - - def source[T](source: Graph[SourceShape[T], _]): Outlet[T] = delegate.add(source).outlet - - def sink[T](sink: Graph[SinkShape[T], _]): Inlet[T] = delegate.add(sink).inlet + def add[S <: Shape](graph: Graph[S, _]): S = delegate.add(graph) /** * Returns an [[Outlet]] that gives access to the materialized value of this graph. Once the graph is materialized @@ -315,8 +306,6 @@ object FlowGraph { */ def materializedValue: Outlet[Mat @uncheckedVariance] = delegate.materializedValue - def run(mat: Materializer): Unit = delegate.buildRunnable().run()(mat) - def from[T](out: Outlet[T]): ForwardOps[T] = new ForwardOps(out) def from[T](src: SourceShape[T]): ForwardOps[T] = new ForwardOps(src.outlet) def from[I, O](f: FlowShape[I, O]): ForwardOps[O] = new ForwardOps(f.outlet) @@ -334,7 +323,7 @@ object FlowGraph { def to(dst: SinkShape[_ >: T]): Builder[Mat] = { out ~> dst; self } def toFanIn[U](j: UniformFanInShape[_ >: T, U]): Builder[Mat] = { out ~> j; self } def toFanOut[U](j: UniformFanOutShape[_ >: T, U]): Builder[Mat] = { out ~> j; self } - def via[U, M](f: FlowShape[_ >: T, U]): ForwardOps[U] = from((out ~> f).outlet) + def via[U](f: FlowShape[_ >: T, U]): ForwardOps[U] = from((out ~> f).outlet) def viaFanIn[U](j: UniformFanInShape[_ >: T, U]): ForwardOps[U] = from((out ~> j).outlet) def viaFanOut[U](j: UniformFanOutShape[_ >: T, U]): ForwardOps[U] = from((out ~> j).outlet) def out(): Outlet[T] = out diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/Sink.scala b/akka-stream/src/main/scala/akka/stream/javadsl/Sink.scala index 0d64eeb996..b53128bb01 100644 --- a/akka-stream/src/main/scala/akka/stream/javadsl/Sink.scala +++ b/akka-stream/src/main/scala/akka/stream/javadsl/Sink.scala @@ -15,14 +15,6 @@ import scala.util.Try /** Java API */ object Sink { - - val factory: SinkCreate = new SinkCreate {} - - /** Adapt [[scaladsl.Sink]] for use within Java DSL */ - //FIXME: Is this needed now? - def adapt[O, M](sink: scaladsl.Sink[O, M]): javadsl.Sink[O, M] = - new Sink(sink) - /** * A `Sink` that will invoke the given function for every received element, giving it its previous * output (or the given `zero` value) and the element as input. @@ -133,10 +125,10 @@ object Sink { * A graph with the shape of a sink logically is a sink, this method makes * it so also in type. */ - def wrap[T, M](g: Graph[SinkShape[T], M]): Sink[T, M] = + def fromGraph[T, M](g: Graph[SinkShape[T], M]): Sink[T, M] = g match { case s: Sink[T, M] ⇒ s - case other ⇒ new Sink(scaladsl.Sink.wrap(other)) + case other ⇒ new Sink(scaladsl.Sink.fromGraph(other)) } /** diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/Source.scala b/akka-stream/src/main/scala/akka/stream/javadsl/Source.scala index b88134c7aa..e7e036e4fc 100644 --- a/akka-stream/src/main/scala/akka/stream/javadsl/Source.scala +++ b/akka-stream/src/main/scala/akka/stream/javadsl/Source.scala @@ -7,7 +7,7 @@ import akka.actor.{ ActorRef, Cancellable, Props } import akka.event.LoggingAdapter import akka.japi.{ Pair, Util, function } import akka.stream._ -import akka.stream.impl.StreamLayout +import akka.stream.impl.{ ConstantFun, StreamLayout } import akka.stream.stage.Stage import org.reactivestreams.{ Publisher, Subscriber } @@ -20,32 +20,27 @@ import scala.language.{ higherKinds, implicitConversions } /** Java API */ object Source { - - val factory: SourceCreate = new SourceCreate {} - - /** Adapt [[scaladsl.Source]] for use within JavaDSL */ - // FIXME: is this needed now? - def adapt[O, M](source: scaladsl.Source[O, M]): Source[O, M] = - new Source(source) + private[this] val _empty = new Source[Any, Unit](scaladsl.Source.empty) /** * Create a `Source` with no elements, i.e. an empty stream that is completed immediately * for every connected `Sink`. */ - def empty[O](): Source[O, Unit] = - new Source(scaladsl.Source.empty) + def empty[O](): Source[O, Unit] = _empty.asInstanceOf[Source[O, Unit]] /** - * Create a `Source` with no elements, which does not complete its downstream, - * until externally triggered to do so. - * - * It materializes a [[scala.concurrent.Promise]] which will be completed - * when the downstream stage of this source cancels. This promise can also - * be used to externally trigger completion, which the source then signals - * to its downstream. + * Create a `Source` which materializes a [[scala.concurrent.Promise]] which controls what element + * will be emitted by the Source. + * If the materialized promise is completed with a Some, that value will be produced downstream, + * followed by completion. + * If the materialized promise is completed with a None, no value will be produced downstream and completion will + * be signalled immediately. + * If the materialized promise is completed with a failure, then the returned source will terminate with that error. + * If the downstream of this source cancels before the promise has been completed, then the promise will be completed + * with None. */ - def lazyEmpty[T](): Source[T, Promise[Unit]] = - new Source[T, Promise[Unit]](scaladsl.Source.lazyEmpty) + def maybe[T]: Source[T, Promise[Option[T]]] = + new Source(scaladsl.Source.maybe[T]) /** * Helper to create [[Source]] from `Publisher`. @@ -198,10 +193,11 @@ object Source { * A graph with the shape of a source logically is a source, this method makes * it so also in type. */ - def wrap[T, M](g: Graph[SourceShape[T], M]): Source[T, M] = + def fromGraph[T, M](g: Graph[SourceShape[T], M]): Source[T, M] = g match { - case s: Source[T, M] ⇒ s - case other ⇒ new Source(scaladsl.Source.wrap(other)) + case s: Source[T, M] ⇒ s + case s if s eq scaladsl.Source.empty ⇒ empty().asInstanceOf[Source[T, M]] + case other ⇒ new Source(scaladsl.Source.fromGraph(other)) } /** @@ -247,9 +243,11 @@ object Source { * Can be used as a `Publisher` */ class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Graph[SourceShape[Out], Mat] { + import scala.collection.JavaConverters._ override def shape: SourceShape[Out] = delegate.shape + private[stream] def module: StreamLayout.Module = delegate.module /** Converts this Java DSL element to its Scala DSL counterpart. */ @@ -277,13 +275,13 @@ class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Graph[Sour * Connect this [[Source]] to a [[Sink]], concatenating the processing steps of both. */ def to[M](sink: Graph[SinkShape[Out], M]): javadsl.RunnableGraph[Mat] = - new RunnableGraphAdapter(delegate.to(sink)) + RunnableGraph.fromGraph(delegate.to(sink)) /** * Connect this [[Source]] to a [[Sink]], concatenating the processing steps of both. */ def toMat[M, M2](sink: Graph[SinkShape[Out], M], combine: function.Function2[Mat, M, M2]): javadsl.RunnableGraph[M2] = - new RunnableGraphAdapter(delegate.toMat(sink)(combinerToScala(combine))) + RunnableGraph.fromGraph(delegate.toMat(sink)(combinerToScala(combine))) /** * Connect this `Source` to a `Sink` and run it. The returned value is the materialized value @@ -411,7 +409,7 @@ class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Graph[Sour */ def zipMat[T, M, M2](that: Graph[SourceShape[T], M], matF: function.Function2[Mat, M, M2]): javadsl.Source[Out @uncheckedVariance Pair T, M2] = - this.viaMat(Flow.empty[Out].zipMat(that, Keep.right[Unit, M]), matF) + this.viaMat(Flow.create[Out].zipMat(that, Keep.right[Unit, M]), matF) /** * Put together the elements of current [[Source]] and the given one @@ -909,7 +907,7 @@ class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Graph[Sour * '''Cancels when''' downstream cancels */ def log(name: String, log: LoggingAdapter): javadsl.Source[Out, Mat] = - this.log(name, javaIdentityFunction[Out], log) + this.log(name, ConstantFun.javaIdentityFunction[Out], log) /** * Logs elements flowing through the stream as well as completion and erroring. @@ -928,6 +926,6 @@ class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Graph[Sour * '''Cancels when''' downstream cancels */ def log(name: String): javadsl.Source[Out, Mat] = - this.log(name, javaIdentityFunction[Out], null) + this.log(name, ConstantFun.javaIdentityFunction[Out], null) } diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/Tcp.scala b/akka-stream/src/main/scala/akka/stream/javadsl/Tcp.scala index 69de538ab5..d7aa15935e 100644 --- a/akka-stream/src/main/scala/akka/stream/javadsl/Tcp.scala +++ b/akka-stream/src/main/scala/akka/stream/javadsl/Tcp.scala @@ -66,7 +66,7 @@ object Tcp extends ExtensionId[Tcp] with ExtensionIdProvider { * A flow representing the client on the other side of the connection. * This flow can be materialized only once. */ - def flow: Flow[ByteString, ByteString, Unit] = Flow.adapt(delegate.flow) + def flow: Flow[ByteString, ByteString, Unit] = new Flow(delegate.flow) } /** @@ -124,7 +124,7 @@ class Tcp(system: ExtendedActorSystem) extends akka.actor.Extension { options: JIterable[SocketOption], halfClose: Boolean, idleTimeout: Duration): Source[IncomingConnection, Future[ServerBinding]] = - Source.adapt(delegate.bind(interface, port, backlog, immutableSeq(options), halfClose, idleTimeout) + Source.fromGraph(delegate.bind(interface, port, backlog, immutableSeq(options), halfClose, idleTimeout) .map(new IncomingConnection(_)) .mapMaterializedValue(_.map(new ServerBinding(_))(ec))) @@ -137,7 +137,7 @@ class Tcp(system: ExtendedActorSystem) extends akka.actor.Extension { * completes is the server ready to accept client connections. */ def bind(interface: String, port: Int): Source[IncomingConnection, Future[ServerBinding]] = - Source.adapt(delegate.bind(interface, port) + Source.fromGraph(delegate.bind(interface, port) .map(new IncomingConnection(_)) .mapMaterializedValue(_.map(new ServerBinding(_))(ec))) @@ -163,7 +163,7 @@ class Tcp(system: ExtendedActorSystem) extends akka.actor.Extension { halfClose: Boolean, connectTimeout: Duration, idleTimeout: Duration): Flow[ByteString, ByteString, Future[OutgoingConnection]] = - Flow.adapt(delegate.outgoingConnection(remoteAddress, localAddress, immutableSeq(options), halfClose, connectTimeout, idleTimeout) + Flow.fromGraph(delegate.outgoingConnection(remoteAddress, localAddress, immutableSeq(options), halfClose, connectTimeout, idleTimeout) .mapMaterializedValue(_.map(new OutgoingConnection(_))(ec))) /** @@ -171,7 +171,7 @@ class Tcp(system: ExtendedActorSystem) extends akka.actor.Extension { * It represents a prospective TCP client connection to the given endpoint. */ def outgoingConnection(host: String, port: Int): Flow[ByteString, ByteString, Future[OutgoingConnection]] = - Flow.adapt(delegate.outgoingConnection(new InetSocketAddress(host, port)) + Flow.fromGraph(delegate.outgoingConnection(new InetSocketAddress(host, port)) .mapMaterializedValue(_.map(new OutgoingConnection(_))(ec))) } diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/package.scala b/akka-stream/src/main/scala/akka/stream/javadsl/package.scala index 6d38905c95..6a0e1b1ad1 100644 --- a/akka-stream/src/main/scala/akka/stream/javadsl/package.scala +++ b/akka-stream/src/main/scala/akka/stream/javadsl/package.scala @@ -3,21 +3,10 @@ */ package akka.stream -import akka.japi.function.Function - package object javadsl { - - val JavaIdentityFunction = new Function[Any, Any] { - @throws(classOf[Exception]) - override def apply(param: Any): Any = param - } - - def javaIdentityFunction[T] = JavaIdentityFunction.asInstanceOf[Function[T, T]] - def combinerToScala[M1, M2, M](f: akka.japi.function.Function2[M1, M2, M]): (M1, M2) ⇒ M = f match { case s: Function2[_, _, _] ⇒ s.asInstanceOf[(M1, M2) ⇒ M] case other ⇒ other.apply _ } - } diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/BidiFlow.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/BidiFlow.scala index 8fc258cd23..4152d64fb8 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl/BidiFlow.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl/BidiFlow.scala @@ -3,11 +3,8 @@ */ package akka.stream.scaladsl -import akka.stream.Graph -import akka.stream.BidiShape +import akka.stream._ import akka.stream.impl.StreamLayout.Module -import akka.stream.FlowShape -import akka.stream.Attributes final class BidiFlow[-I1, +O1, -I2, +O2, +Mat](private[stream] override val module: Module) extends Graph[BidiShape[I1, O1, I2, O2], Mat] { override val shape = module.shape.asInstanceOf[BidiShape[I1, O1, I2, O2]] @@ -130,16 +127,16 @@ final class BidiFlow[-I1, +O1, -I2, +O2, +Mat](private[stream] override val modu withAttributes(Attributes.name(name)) } -object BidiFlow extends BidiFlowApply { - +object BidiFlow { /** * A graph with the shape of a flow logically is a flow, this method makes * it so also in type. */ - def wrap[I1, O1, I2, O2, Mat](graph: Graph[BidiShape[I1, O1, I2, O2], Mat]): BidiFlow[I1, O1, I2, O2, Mat] = + def fromGraph[I1, O1, I2, O2, Mat](graph: Graph[BidiShape[I1, O1, I2, O2], Mat]): BidiFlow[I1, O1, I2, O2, Mat] = graph match { - case bidi: BidiFlow[I1, O1, I2, O2, Mat] ⇒ bidi - case other ⇒ new BidiFlow(other.module) + case bidi: BidiFlow[I1, O1, I2, O2, Mat] ⇒ bidi + case bidi: javadsl.BidiFlow[I1, O1, I2, O2, Mat] ⇒ bidi.asScala + case other ⇒ new BidiFlow(other.module) } /** @@ -161,23 +158,39 @@ object BidiFlow extends BidiFlowApply { * }}} * */ - def wrap[I1, O1, I2, O2, M1, M2, M]( + def fromFlowsMat[I1, O1, I2, O2, M1, M2, M]( flow1: Graph[FlowShape[I1, O1], M1], - flow2: Graph[FlowShape[I2, O2], M2])(combine: (M1, M2) ⇒ M): BidiFlow[I1, O1, I2, O2, M] = { - BidiFlow(flow1, flow2)(combine) { implicit b ⇒ - (f1, f2) ⇒ - BidiShape(f1.inlet, f1.outlet, f2.inlet, f2.outlet) - } - } + flow2: Graph[FlowShape[I2, O2], M2])(combine: (M1, M2) ⇒ M): BidiFlow[I1, O1, I2, O2, M] = + fromGraph(FlowGraph.create(flow1, flow2)(combine) { + implicit b ⇒ (f1, f2) ⇒ BidiShape(f1.inlet, f1.outlet, f2.inlet, f2.outlet) + }) + + /** + * Wraps two Flows to create a ''BidiFlow''. The materialized value of the resulting BidiFlow is Unit. + * + * {{{ + * +----------------------------+ + * | Resulting BidiFlow | + * | | + * | +----------------------+ | + * I1 ~~> | Flow1 | ~~> O1 + * | +----------------------+ | + * | | + * | +----------------------+ | + * O2 <~~ | Flow2 | <~~ I2 + * | +----------------------+ | + * +----------------------------+ + * }}} + * + */ + def fromFlows[I1, O1, I2, O2, M1, M2](flow1: Graph[FlowShape[I1, O1], M1], + flow2: Graph[FlowShape[I2, O2], M2]): BidiFlow[I1, O1, I2, O2, Unit] = + fromFlowsMat(flow1, flow2)(Keep.none) /** * Create a BidiFlow where the top and bottom flows are just one simple mapping * stage each, expressed by the two functions. */ - def apply[I1, O1, I2, O2](outbound: I1 ⇒ O1, inbound: I2 ⇒ O2): BidiFlow[I1, O1, I2, O2, Unit] = - BidiFlow() { b ⇒ - val top = b.add(Flow[I1].map(outbound)) - val bottom = b.add(Flow[I2].map(inbound)) - BidiShape(top.inlet, top.outlet, bottom.inlet, bottom.outlet) - } + def fromFunctions[I1, O1, I2, O2](outbound: I1 ⇒ O1, inbound: I2 ⇒ O2): BidiFlow[I1, O1, I2, O2, Unit] = + fromFlows(Flow[I1].map(outbound), Flow[I2].map(inbound)) } diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala index 348a3aed46..04eef540ac 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala @@ -10,7 +10,7 @@ import akka.stream.impl.SplitDecision._ import akka.stream.impl.Stages.{ DirectProcessor, MaterializingStageFactory, StageModule } import akka.stream.impl.StreamLayout.{ EmptyModule, Module } import akka.stream.impl.fusing.{ DropWithin, GroupedWithin, TakeWithin } -import akka.stream.impl.{ ReactiveStreamsCompliance, Stages, StreamLayout } +import akka.stream.impl.{ ReactiveStreamsCompliance, ConstantFun, Stages, StreamLayout } import akka.stream.stage._ import org.reactivestreams.{ Processor, Publisher, Subscriber, Subscription } @@ -34,11 +34,8 @@ final class Flow[-In, +Out, +Mat](private[stream] override val module: Module) def viaMat[T, Mat2, Mat3](flow: Graph[FlowShape[Out, T], Mat2])(combine: (Mat, Mat2) ⇒ Mat3): Flow[In, T, Mat3] = { if (this.isIdentity) { - val flowInstance: Flow[In, T, Mat2] = if (flow.isInstanceOf[javadsl.Flow[In, T, Mat2]]) - flow.asInstanceOf[javadsl.Flow[In, T, Mat2]].asScala - else - flow.asInstanceOf[Flow[In, T, Mat2]] - flowInstance.mapMaterializedValue(combine(().asInstanceOf[Mat], _)) + Flow.fromGraph(flow.asInstanceOf[Graph[FlowShape[In, T], Mat2]]) + .mapMaterializedValue(combine(().asInstanceOf[Mat], _)) } else { val flowCopy = flow.module.carbonCopy new Flow( @@ -84,13 +81,10 @@ final class Flow[-In, +Out, +Mat](private[stream] override val module: Module) * Sink into the materialized value of the resulting Sink. */ def toMat[Mat2, Mat3](sink: Graph[SinkShape[Out], Mat2])(combine: (Mat, Mat2) ⇒ Mat3): Sink[In, Mat3] = { - if (isIdentity) { - val sinkInstance: Sink[In, Mat2] = if (sink.isInstanceOf[javadsl.Sink[In, Mat2]]) - sink.asInstanceOf[javadsl.Sink[In, Mat2]].asScala - else - sink.asInstanceOf[Sink[In, Mat2]] - sinkInstance.mapMaterializedValue(combine(().asInstanceOf[Mat], _)) - } else { + if (isIdentity) + Sink.fromGraph(sink.asInstanceOf[Graph[SinkShape[In], Mat2]]) + .mapMaterializedValue(combine(().asInstanceOf[Mat], _)) + else { val sinkCopy = sink.module.carbonCopy new Sink( module @@ -223,7 +217,7 @@ final class Flow[-In, +Out, +Mat](private[stream] override val module: Module) * and `Publisher` of a [[Sink#publisher]]. */ def runWith[Mat1, Mat2](source: Graph[SourceShape[In], Mat1], sink: Graph[SinkShape[Out], Mat2])(implicit materializer: Materializer): (Mat1, Mat2) = - Source.wrap(source).via(this).toMat(sink)(Keep.both).run() + Source.fromGraph(source).via(this).toMat(sink)(Keep.both).run() /** * Converts this Flow to a [[RunnableGraph]] that materializes to a Reactive Streams [[org.reactivestreams.Processor]] @@ -249,9 +243,8 @@ final class Flow[-In, +Out, +Mat](private[stream] override val module: Module) def asJava: javadsl.Flow[In, Out, Mat] = new javadsl.Flow(this) } -object Flow extends FlowApply { - - private def shape[I, O](name: String): FlowShape[I, O] = FlowShape(Inlet(name + ".in"), Outlet(name + ".out")) +object Flow { + private[this] val identity: Flow[Any, Any, Unit] = new Flow[Any, Any, Unit](Stages.Identity()) /** * Creates a Flow from a Reactive Streams [[org.reactivestreams.Processor]] @@ -265,32 +258,49 @@ object Flow extends FlowApply { * Helper to create `Flow` without a [[Source]] or a [[Sink]]. * Example usage: `Flow[Int]` */ - def apply[T]: Flow[T, T, Unit] = new Flow[Any, Any, Any](Stages.Identity()).asInstanceOf[Flow[T, T, Unit]] + def apply[T]: Flow[T, T, Unit] = identity.asInstanceOf[Flow[T, T, Unit]] /** * A graph with the shape of a flow logically is a flow, this method makes * it so also in type. */ - def wrap[I, O, M](g: Graph[FlowShape[I, O], M]): Flow[I, O, M] = + def fromGraph[I, O, M](g: Graph[FlowShape[I, O], M]): Flow[I, O, M] = g match { - case f: Flow[I, O, M] ⇒ f - case other ⇒ new Flow(other.module) + case f: Flow[I, O, M] ⇒ f + case f: javadsl.Flow[I, O, M] ⇒ f.asScala + case other ⇒ new Flow(other.module) } /** - * Helper to create `Flow` from a pair of sink and source. + * Helper to create `Flow` from a `Sink`and a `Source`. */ - def wrap[I, O, M1, M2, M](sink: Graph[SinkShape[I], M1], source: Graph[SourceShape[O], M2])(f: (M1, M2) ⇒ M): Flow[I, O, M] = - Flow(sink, source)(f) { implicit b ⇒ (in, out) ⇒ (in.inlet, out.outlet) } + def fromSinkAndSource[I, O](sink: Graph[SinkShape[I], _], source: Graph[SourceShape[O], _]): Flow[I, O, Unit] = + fromSinkAndSourceMat(sink, source)(Keep.none) + /** + * Helper to create `Flow` from a `Sink`and a `Source`. + */ + def fromSinkAndSourceMat[I, O, M1, M2, M](sink: Graph[SinkShape[I], M1], source: Graph[SourceShape[O], M2])(f: (M1, M2) ⇒ M): Flow[I, O, M] = + fromGraph(FlowGraph.create(sink, source)(f) { implicit b ⇒ (in, out) ⇒ FlowShape(in.inlet, out.outlet) }) } +object RunnableGraph { + /** + * A graph with a closed shape is logically a runnable graph, this method makes + * it so also in type. + */ + def fromGraph[Mat](g: Graph[ClosedShape, Mat]): RunnableGraph[Mat] = + g match { + case r: RunnableGraph[Mat] ⇒ r + case other ⇒ RunnableGraph(other.module) + } +} /** * Flow with attached input and output, can be executed. */ -case class RunnableGraph[+Mat](private[stream] val module: StreamLayout.Module) extends Graph[ClosedShape, Mat] { +final case class RunnableGraph[+Mat](private[stream] val module: StreamLayout.Module) extends Graph[ClosedShape, Mat] { require(module.isRunnable) - def shape = ClosedShape + override def shape = ClosedShape /** * Transform only the materialized value of this RunnableGraph, leaving all other properties as they were. @@ -307,7 +317,6 @@ case class RunnableGraph[+Mat](private[stream] val module: StreamLayout.Module) new RunnableGraph(module.withAttributes(attr).nest()) override def named(name: String): RunnableGraph[Mat] = withAttributes(Attributes.name(name)) - } /** @@ -317,8 +326,6 @@ trait FlowOps[+Out, +Mat] { import akka.stream.impl.Stages._ type Repr[+O, +M] <: FlowOps[O, M] - private final val _identity = (x: Any) ⇒ x - /** * Transform this [[Flow]] by appending the given processing steps. * {{{ @@ -1017,7 +1024,7 @@ trait FlowOps[+Out, +Mat] { * * '''Cancels when''' downstream cancels */ - def log(name: String, extract: Out ⇒ Any = _identity)(implicit log: LoggingAdapter = null): Repr[Out, Mat] = + def log(name: String, extract: Out ⇒ Any = ConstantFun.scalaIdentityFunction)(implicit log: LoggingAdapter = null): Repr[Out, Mat] = andThen(Stages.Log(name, extract.asInstanceOf[Any ⇒ Any], Option(log))) /** @@ -1039,12 +1046,12 @@ trait FlowOps[+Out, +Mat] { * @see [[#zip]]. */ def zipMat[U, Mat2, Mat3](that: Graph[SourceShape[U], Mat2])(matF: (Mat, Mat2) ⇒ Mat3): Repr[(Out, U), Mat3] = - this.viaMat(Flow(that) { implicit b ⇒ + this.viaMat(FlowGraph.create(that) { implicit b ⇒ r ⇒ import FlowGraph.Implicits._ val zip = b.add(Zip[Out, U]()) r ~> zip.in1 - (zip.in0, zip.out) + FlowShape(zip.in0, zip.out) })(matF) /** @@ -1069,12 +1076,12 @@ trait FlowOps[+Out, +Mat] { * @see [[#zipWith]]. */ def zipWithMat[Out2, Out3, Mat2, Mat3](that: Graph[SourceShape[Out2], Mat2])(combine: (Out, Out2) ⇒ Out3)(matF: (Mat, Mat2) ⇒ Mat3): Repr[Out3, Mat3] = - this.viaMat(Flow(that) { implicit b ⇒ + this.viaMat(FlowGraph.create(that) { implicit b ⇒ r ⇒ import FlowGraph.Implicits._ val zip = b.add(ZipWith[Out, Out2, Out3](combine)) r ~> zip.in1 - (zip.in0, zip.out) + FlowShape(zip.in0, zip.out) })(matF) /** @@ -1099,12 +1106,12 @@ trait FlowOps[+Out, +Mat] { * @see [[#merge]]. */ def mergeMat[U >: Out, Mat2, Mat3](that: Graph[SourceShape[U], Mat2])(matF: (Mat, Mat2) ⇒ Mat3): Repr[U, Mat3] = - this.viaMat(Flow(that) { implicit b ⇒ + this.viaMat(FlowGraph.create(that) { implicit b ⇒ r ⇒ import FlowGraph.Implicits._ val merge = b.add(Merge[U](2)) r ~> merge.in(1) - (merge.in(0), merge.out) + FlowShape(merge.in(0), merge.out) })(matF) /** @@ -1141,12 +1148,12 @@ trait FlowOps[+Out, +Mat] { * @see [[#concat]]. */ def concatMat[U >: Out, Mat2, Mat3](that: Graph[SourceShape[U], Mat2])(matF: (Mat, Mat2) ⇒ Mat3): Repr[U, Mat3] = - this.viaMat(Flow(that) { implicit b ⇒ + this.viaMat(FlowGraph.create(that) { implicit b ⇒ r ⇒ import FlowGraph.Implicits._ val merge = b.add(Concat[U]()) r ~> merge.in(1) - (merge.in(0), merge.out) + FlowShape(merge.in(0), merge.out) })(matF) /** @@ -1179,12 +1186,12 @@ trait FlowOps[+Out, +Mat] { * @see [[#alsoTo]] */ def alsoToMat[Mat2, Mat3](that: Graph[SinkShape[Out], Mat2])(matF: (Mat, Mat2) ⇒ Mat3): Repr[Out, Mat3] = - this.viaMat(Flow(that) { implicit b ⇒ + this.viaMat(FlowGraph.create(that) { implicit b ⇒ r ⇒ import FlowGraph.Implicits._ val bcast = b.add(Broadcast[Out](2)) bcast.out(1) ~> r - (bcast.in, bcast.out(0)) + FlowShape(bcast.in, bcast.out(0)) })(matF) def withAttributes(attr: Attributes): Repr[Out, Mat] diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/Graph.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/Graph.scala index 7dca54f048..21459c2347 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl/Graph.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl/Graph.scala @@ -476,9 +476,6 @@ object ZipWith extends ZipWithApply * '''Cancels when''' any downstream cancels */ object Unzip { - - private final val _identity: Any ⇒ Any = a ⇒ a - /** * Create a new `Unzip`. */ @@ -511,9 +508,7 @@ object Concat { /** * Create a new `Concat`. */ - def apply[T](inputCount: Int = 2): Concat[T] = { - new Concat(inputCount) - } + def apply[T](inputCount: Int = 2): Concat[T] = new Concat(inputCount) } /** @@ -567,7 +562,6 @@ class Concat[T](inputCount: Int) extends GraphStage[UniformFanInShape[T, T]] { setHandler(out, new OutHandler { override def onPull() = pull(in(activeStream)) }) - } } @@ -576,7 +570,7 @@ object FlowGraph extends GraphApply { class Builder[+M] private[stream] () { private var moduleInProgress: Module = EmptyModule - def addEdge[A1, A >: A1, B, B1 >: B, M2](from: Outlet[A1], via: Graph[FlowShape[A, B], M2], to: Inlet[B1]): Unit = { + private[FlowGraph] def addEdge[A1, A >: A1, B, B1 >: B, M2](from: Outlet[A1], via: Graph[FlowShape[A, B], M2], to: Inlet[B1]): Unit = { val flowCopy = via.module.carbonCopy moduleInProgress = moduleInProgress @@ -585,9 +579,8 @@ object FlowGraph extends GraphApply { .wire(flowCopy.shape.outlets.head, to) } - def addEdge[T, U >: T](from: Outlet[T], to: Inlet[U]): Unit = { + private[FlowGraph] def addEdge[T, U >: T](from: Outlet[T], to: Inlet[U]): Unit = moduleInProgress = moduleInProgress.wire(from, to) - } /** * Import a graph into this module, performing a deep copy, discarding its @@ -657,64 +650,10 @@ object FlowGraph extends GraphApply { .wire(port, op.inPort) } - private[stream] def buildRunnable[Mat](): RunnableGraph[Mat] = { - if (!moduleInProgress.isRunnable) { - throw new IllegalArgumentException( - "Cannot build the RunnableGraph because there are unconnected ports: " + - (moduleInProgress.outPorts ++ moduleInProgress.inPorts).mkString(", ")) - } - new RunnableGraph(moduleInProgress.nest()) - } - - private[stream] def buildSource[T, Mat](outlet: Outlet[T]): Source[T, Mat] = { - if (moduleInProgress.isRunnable) - throw new IllegalArgumentException("Cannot build the Source since no ports remain open") - if (!moduleInProgress.isSource) - throw new IllegalArgumentException( - s"Cannot build Source with open inputs (${moduleInProgress.inPorts.mkString(",")}) and outputs (${moduleInProgress.outPorts.mkString(",")})") - if (moduleInProgress.outPorts.head != outlet) - throw new IllegalArgumentException(s"provided Outlet $outlet does not equal the module’s open Outlet ${moduleInProgress.outPorts.head}") - new Source(moduleInProgress.replaceShape(SourceShape(outlet)).nest()) - } - - private[stream] def buildFlow[In, Out, Mat](inlet: Inlet[In], outlet: Outlet[Out]): Flow[In, Out, Mat] = { - if (!moduleInProgress.isFlow) - throw new IllegalArgumentException( - s"Cannot build Flow with open inputs (${moduleInProgress.inPorts.mkString(",")}) and outputs (${moduleInProgress.outPorts.mkString(",")})") - if (moduleInProgress.outPorts.head != outlet) - throw new IllegalArgumentException(s"provided Outlet $outlet does not equal the module’s open Outlet ${moduleInProgress.outPorts.head}") - if (moduleInProgress.inPorts.head != inlet) - throw new IllegalArgumentException(s"provided Inlet $inlet does not equal the module’s open Inlet ${moduleInProgress.inPorts.head}") - new Flow(moduleInProgress.replaceShape(FlowShape(inlet, outlet)).nest()) - } - - private[stream] def buildBidiFlow[I1, O1, I2, O2, Mat](shape: BidiShape[I1, O1, I2, O2]): BidiFlow[I1, O1, I2, O2, Mat] = { - if (!moduleInProgress.isBidiFlow) - throw new IllegalArgumentException( - s"Cannot build BidiFlow with open inputs (${moduleInProgress.inPorts.mkString(",")}) and outputs (${moduleInProgress.outPorts.mkString(",")})") - if (moduleInProgress.outPorts.toSet != shape.outlets.toSet) - throw new IllegalArgumentException(s"provided Outlets [${shape.outlets.mkString(",")}] does not equal the module’s open Outlets [${moduleInProgress.outPorts.mkString(",")}]") - if (moduleInProgress.inPorts.toSet != shape.inlets.toSet) - throw new IllegalArgumentException(s"provided Inlets [${shape.inlets.mkString(",")}] does not equal the module’s open Inlets [${moduleInProgress.inPorts.mkString(",")}]") - new BidiFlow(moduleInProgress.replaceShape(shape).nest()) - } - - private[stream] def buildSink[T, Mat](inlet: Inlet[T]): Sink[T, Mat] = { - if (moduleInProgress.isRunnable) - throw new IllegalArgumentException("Cannot build the Sink since no ports remain open") - if (!moduleInProgress.isSink) - throw new IllegalArgumentException( - s"Cannot build Sink with open inputs (${moduleInProgress.inPorts.mkString(",")}) and outputs (${moduleInProgress.outPorts.mkString(",")})") - if (moduleInProgress.inPorts.head != inlet) - throw new IllegalArgumentException(s"provided Inlet $inlet does not equal the module’s open Inlet ${moduleInProgress.inPorts.head}") - new Sink(moduleInProgress.replaceShape(SinkShape(inlet)).nest()) - } - private[stream] def module: Module = moduleInProgress /** Converts this Scala DSL element to it's Java DSL counterpart. */ def asJava: javadsl.FlowGraph.Builder[M] = new javadsl.FlowGraph.Builder()(this) - } object Implicits { @@ -735,12 +674,11 @@ object FlowGraph extends GraphApply { else junction.in(n) } - trait CombinerBase[T] extends Any { + sealed trait CombinerBase[T] extends Any { def importAndGetPort(b: Builder[_]): Outlet[T] - def ~>[U >: T](to: Inlet[U])(implicit b: Builder[_]): Unit = { + def ~>[U >: T](to: Inlet[U])(implicit b: Builder[_]): Unit = b.addEdge(importAndGetPort(b), to) - } def ~>[Out](via: Graph[FlowShape[T, Out], Any])(implicit b: Builder[_]): PortOps[Out, Unit] = { val s = b.add(via) @@ -772,21 +710,18 @@ object FlowGraph extends GraphApply { flow.outlet } - def ~>(to: Graph[SinkShape[T], _])(implicit b: Builder[_]): Unit = { + def ~>(to: Graph[SinkShape[T], _])(implicit b: Builder[_]): Unit = b.addEdge(importAndGetPort(b), b.add(to).inlet) - } - def ~>(to: SinkShape[T])(implicit b: Builder[_]): Unit = { + def ~>(to: SinkShape[T])(implicit b: Builder[_]): Unit = b.addEdge(importAndGetPort(b), to.inlet) - } } - trait ReverseCombinerBase[T] extends Any { + sealed trait ReverseCombinerBase[T] extends Any { def importAndGetPortReverse(b: Builder[_]): Inlet[T] - def <~[U <: T](from: Outlet[U])(implicit b: Builder[_]): Unit = { + def <~[U <: T](from: Outlet[U])(implicit b: Builder[_]): Unit = b.addEdge(from, importAndGetPortReverse(b)) - } def <~[In](via: Graph[FlowShape[In, T], _])(implicit b: Builder[_]): ReversePortOps[In] = { val s = b.add(via) @@ -818,13 +753,11 @@ object FlowGraph extends GraphApply { flow.inlet } - def <~(from: Graph[SourceShape[T], _])(implicit b: Builder[_]): Unit = { + def <~(from: Graph[SourceShape[T], _])(implicit b: Builder[_]): Unit = b.addEdge(b.add(from).outlet, importAndGetPortReverse(b)) - } - def <~(from: SourceShape[T])(implicit b: Builder[_]): Unit = { + def <~(from: SourceShape[T])(implicit b: Builder[_]): Unit = b.addEdge(from.outlet, importAndGetPortReverse(b)) - } } // Although Mat is always Unit, it cannot be removed as a type parameter, otherwise the "override type" @@ -855,7 +788,7 @@ object FlowGraph extends GraphApply { throw new UnsupportedOperationException("Cannot use viaMat on a port") } - class DisabledPortOps[Out, Mat](msg: String) extends PortOps[Out, Mat](null, null) { + final class DisabledPortOps[Out, Mat](msg: String) extends PortOps[Out, Mat](null, null) { override def importAndGetPort(b: Builder[_]): Outlet[Out] = throw new IllegalArgumentException(msg) override def viaMat[T, Mat2, Mat3](flow: Graph[FlowShape[Out, T], Mat2])(combine: (Mat, Mat2) ⇒ Mat3) = @@ -866,28 +799,28 @@ object FlowGraph extends GraphApply { override def importAndGetPortReverse(b: Builder[_]): Inlet[In] = inlet } - class DisabledReversePortOps[In](msg: String) extends ReversePortOps[In](null) { + final class DisabledReversePortOps[In](msg: String) extends ReversePortOps[In](null) { override def importAndGetPortReverse(b: Builder[_]): Inlet[In] = throw new IllegalArgumentException(msg) } - implicit class FanInOps[In, Out](val j: UniformFanInShape[In, Out]) extends AnyVal with CombinerBase[Out] with ReverseCombinerBase[In] { + implicit final class FanInOps[In, Out](val j: UniformFanInShape[In, Out]) extends AnyVal with CombinerBase[Out] with ReverseCombinerBase[In] { override def importAndGetPort(b: Builder[_]): Outlet[Out] = j.out override def importAndGetPortReverse(b: Builder[_]): Inlet[In] = findIn(b, j, 0) } - implicit class FanOutOps[In, Out](val j: UniformFanOutShape[In, Out]) extends AnyVal with ReverseCombinerBase[In] { + implicit final class FanOutOps[In, Out](val j: UniformFanOutShape[In, Out]) extends AnyVal with ReverseCombinerBase[In] { override def importAndGetPortReverse(b: Builder[_]): Inlet[In] = j.in } - implicit class SinkArrow[T](val s: Graph[SinkShape[T], _]) extends AnyVal with ReverseCombinerBase[T] { + implicit final class SinkArrow[T](val s: Graph[SinkShape[T], _]) extends AnyVal with ReverseCombinerBase[T] { override def importAndGetPortReverse(b: Builder[_]): Inlet[T] = b.add(s).inlet } - implicit class SinkShapeArrow[T](val s: SinkShape[T]) extends AnyVal with ReverseCombinerBase[T] { + implicit final class SinkShapeArrow[T](val s: SinkShape[T]) extends AnyVal with ReverseCombinerBase[T] { override def importAndGetPortReverse(b: Builder[_]): Inlet[T] = s.inlet } - implicit class FlowShapeArrow[I, O](val f: FlowShape[I, O]) extends AnyVal with ReverseCombinerBase[I] { + implicit final class FlowShapeArrow[I, O](val f: FlowShape[I, O]) extends AnyVal with ReverseCombinerBase[I] { override def importAndGetPortReverse(b: Builder[_]): Inlet[I] = f.inlet def <~>[I2, O2, Mat](bidi: Graph[BidiShape[O, O2, I2, I], Mat])(implicit b: Builder[_]): BidiShape[O, O2, I2, I] = { @@ -910,7 +843,7 @@ object FlowGraph extends GraphApply { } } - implicit class FlowArrow[I, O, M](val f: Graph[FlowShape[I, O], M]) extends AnyVal { + implicit final class FlowArrow[I, O, M](val f: Graph[FlowShape[I, O], M]) extends AnyVal { def <~>[I2, O2, Mat](bidi: Graph[BidiShape[O, O2, I2, I], Mat])(implicit b: Builder[_]): BidiShape[O, O2, I2, I] = { val shape = b.add(bidi) val flow = b.add(f) @@ -934,7 +867,7 @@ object FlowGraph extends GraphApply { } } - implicit class BidiFlowShapeArrow[I1, O1, I2, O2](val bidi: BidiShape[I1, O1, I2, O2]) extends AnyVal { + implicit final class BidiFlowShapeArrow[I1, O1, I2, O2](val bidi: BidiShape[I1, O1, I2, O2]) extends AnyVal { def <~>[I3, O3](other: BidiShape[O1, O3, I3, I2])(implicit b: Builder[_]): BidiShape[O1, O3, I3, I2] = { b.addEdge(bidi.out1, other.in1) b.addEdge(other.out2, bidi.in2) @@ -971,11 +904,11 @@ object FlowGraph extends GraphApply { implicit def flow2flow[I, O](f: FlowShape[I, O])(implicit b: Builder[_]): PortOps[O, Unit] = new PortOps(f.outlet, b) - implicit class SourceArrow[T](val s: Graph[SourceShape[T], _]) extends AnyVal with CombinerBase[T] { + implicit final class SourceArrow[T](val s: Graph[SourceShape[T], _]) extends AnyVal with CombinerBase[T] { override def importAndGetPort(b: Builder[_]): Outlet[T] = b.add(s).outlet } - implicit class SourceShapeArrow[T](val s: SourceShape[T]) extends AnyVal with CombinerBase[T] { + implicit final class SourceShapeArrow[T](val s: SourceShape[T]) extends AnyVal with CombinerBase[T] { override def importAndGetPort(b: Builder[_]): Outlet[T] = s.outlet } } diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/One2OneBidiFlow.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/One2OneBidiFlow.scala index 21e331c25a..c5f6ade5bc 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl/One2OneBidiFlow.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl/One2OneBidiFlow.scala @@ -28,7 +28,7 @@ object One2OneBidiFlow { * has signalled completion. Note that no error is triggered in this case! */ def apply[I, O](maxPending: Int): BidiFlow[I, I, O, O, Unit] = - BidiFlow.wrap(new One2OneBidi[I, O](maxPending)) + BidiFlow.fromGraph(new One2OneBidi[I, O](maxPending)) class One2OneBidi[I, O](maxPending: Int) extends GraphStage[BidiShape[I, I, O, O]] { val inIn = Inlet[I]("inIn") diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala index bf031bd6c4..9a9bf71795 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala @@ -31,7 +31,7 @@ final class Sink[-In, +Mat](private[stream] override val module: Module) * of the `Source`, e.g. the `Subscriber` of a [[Source#subscriber]]. */ def runWith[Mat2](source: Graph[SourceShape[In], Mat2])(implicit materializer: Materializer): Mat2 = - Source.wrap(source).to(this).run() + Source.fromGraph(source).to(this).run() def mapMaterializedValue[Mat2](f: Mat ⇒ Mat2): Sink[In, Mat2] = new Sink(module.transformMaterializedValue(f.asInstanceOf[Any ⇒ Any])) @@ -45,7 +45,7 @@ final class Sink[-In, +Mat](private[stream] override val module: Module) def asJava: javadsl.Sink[In, Mat] = new javadsl.Sink(this) } -object Sink extends SinkApply { +object Sink { /** INTERNAL API */ private[stream] def shape[T](name: String): SinkShape[T] = SinkShape(Inlet(name + ".in")) @@ -54,10 +54,11 @@ object Sink extends SinkApply { * A graph with the shape of a sink logically is a sink, this method makes * it so also in type. */ - def wrap[T, M](g: Graph[SinkShape[T], M]): Sink[T, M] = + def fromGraph[T, M](g: Graph[SinkShape[T], M]): Sink[T, M] = g match { - case s: Sink[T, M] ⇒ s - case other ⇒ new Sink(other.module) + case s: Sink[T, M] ⇒ s + case s: javadsl.Sink[T, M] ⇒ s.asScala + case other ⇒ new Sink(other.module) } /** @@ -112,7 +113,7 @@ object Sink extends SinkApply { */ def combine[T, U](first: Sink[U, _], second: Sink[U, _], rest: Sink[U, _]*)(strategy: Int ⇒ Graph[UniformFanOutShape[T, U], Unit]): Sink[T, Unit] = - Sink.wrap(FlowGraph.partial() { implicit b ⇒ + Sink.fromGraph(FlowGraph.create() { implicit b ⇒ import FlowGraph.Implicits._ val d = b.add(strategy(rest.size + 2)) d.out(0) ~> first diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala index 8b26e49254..58dba3ca1d 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala @@ -122,7 +122,7 @@ final class Source[+Out, +Mat](private[stream] override val module: Module) * Combines several sources with fun-in strategy like `Merge` or `Concat` and returns `Source`. */ def combine[T, U](first: Source[T, _], second: Source[T, _], rest: Source[T, _]*)(strategy: Int ⇒ Graph[UniformFanInShape[T, U], Unit]): Source[U, Unit] = - Source.wrap(FlowGraph.partial() { implicit b ⇒ + Source.fromGraph(FlowGraph.create() { implicit b ⇒ import FlowGraph.Implicits._ val c = b.add(strategy(rest.size + 2)) first ~> c.in(0) @@ -138,14 +138,7 @@ final class Source[+Out, +Mat](private[stream] override val module: Module) }) } -object Source extends SourceApply { - - private[this] final val _id: Any ⇒ Any = x ⇒ x - private[this] final def id[A]: A ⇒ A = _id.asInstanceOf[A ⇒ A] - - private[stream] def apply[Out, Mat](module: SourceModule[Out, Mat]): Source[Out, Mat] = - new Source(module) - +object Source { /** INTERNAL API */ private[stream] def shape[T](name: String): SourceShape[T] = SourceShape(Outlet(name + ".out")) @@ -180,9 +173,10 @@ object Source extends SourceApply { * A graph with the shape of a source logically is a source, this method makes * it so also in type. */ - def wrap[T, M](g: Graph[SourceShape[T], M]): Source[T, M] = g match { - case s: Source[T, M] ⇒ s - case other ⇒ new Source(other.module) + def fromGraph[T, M](g: Graph[SourceShape[T], M]): Source[T, M] = g match { + case s: Source[T, M] ⇒ s + case s: javadsl.Source[T, M] ⇒ s.asScala + case other ⇒ new Source(other.module) } /** @@ -195,7 +189,7 @@ object Source extends SourceApply { * beginning) regardless of when they subscribed. */ def apply[T](iterable: immutable.Iterable[T]): Source[T, Unit] = - Source.single(iterable).mapConcat(id).withAttributes(DefaultAttributes.iterableSource) + Source.single(iterable).mapConcat(ConstantFun.scalaIdentityFunction).withAttributes(DefaultAttributes.iterableSource) /** * Start a new `Source` from the given `Future`. The stream will consist of @@ -208,7 +202,7 @@ object Source extends SourceApply { new PublisherSource( SingleElementPublisher(future, "FutureSource"), DefaultAttributes.futureSource, - shape("FutureSource"))).mapAsyncUnordered(1)(id) + shape("FutureSource"))).mapAsyncUnordered(1)(ConstantFun.scalaIdentityFunction) /** * Elements are emitted periodically with the specified interval. @@ -218,7 +212,7 @@ object Source extends SourceApply { * receive new tick elements as soon as it has requested more elements. */ def apply[T](initialDelay: FiniteDuration, interval: FiniteDuration, tick: T): Source[T, Cancellable] = - wrap(new TickSource[T](initialDelay, interval, tick)) + fromGraph(new TickSource[T](initialDelay, interval, tick)) /** * Create a `Source` with one element. @@ -245,7 +239,7 @@ object Source extends SourceApply { override def toString: String = "repeat(" + element + ")" }, "RepeatSource"), DefaultAttributes.repeat, - shape("RepeatSource"))).mapConcat(id) + shape("RepeatSource"))).mapConcat(ConstantFun.scalaIdentityFunction) } /** @@ -260,16 +254,18 @@ object Source extends SourceApply { shape("EmptySource"))) /** - * Create a `Source` with no elements, which does not complete its downstream, - * until externally triggered to do so. - * - * It materializes a [[scala.concurrent.Promise]] which will be completed - * when the downstream stage of this source cancels. This promise can also - * be used to externally trigger completion, which the source then signals - * to its downstream. + * Create a `Source` which materializes a [[scala.concurrent.Promise]] which controls what element + * will be emitted by the Source. + * If the materialized promise is completed with a Some, that value will be produced downstream, + * followed by completion. + * If the materialized promise is completed with a None, no value will be produced downstream and completion will + * be signalled immediately. + * If the materialized promise is completed with a failure, then the returned source will terminate with that error. + * If the downstream of this source cancels before the promise has been completed, then the promise will be completed + * with None. */ - def lazyEmpty[T]: Source[T, Promise[Unit]] = - new Source(new LazyEmptySource[T](DefaultAttributes.lazyEmptySource, shape("LazyEmptySource"))) + def maybe[T]: Source[T, Promise[Option[T]]] = + new Source(new MaybeSource[T](DefaultAttributes.maybeSource, shape("MaybeSource"))) /** * Create a `Source` that immediately ends the stream with the `cause` error to every connected `Sink`. @@ -337,7 +333,7 @@ object Source extends SourceApply { * Combines several sources with fun-in strategy like `Merge` or `Concat` and returns `Source`. */ def combine[T, U](first: Source[T, _], second: Source[T, _], rest: Source[T, _]*)(strategy: Int ⇒ Graph[UniformFanInShape[T, U], Unit]): Source[U, Unit] = - Source.wrap(FlowGraph.partial() { implicit b ⇒ + Source.fromGraph(FlowGraph.create() { implicit b ⇒ import FlowGraph.Implicits._ val c = b.add(strategy(rest.size + 2)) first ~> c.in(0)