!str #18692 javadsl.FlowGraph.Builder.add()

* also make factories more consistent by only offering
  FlowGraph.create()
* also remove secondary (edge-based) FlowGraph.Builder DSL
* also improve naming for conversions from Graph to
  Source/Flow/BidiFlow/Sink
This commit is contained in:
Viktor Klang 2015-10-21 22:45:39 +02:00 committed by Roland Kuhn
parent 0f99a42df9
commit f29d7affbd
120 changed files with 1535 additions and 1897 deletions

View file

@ -20,7 +20,7 @@ object MaterializationBenchmark {
} }
val graphWithJunctionsBuilder = (numOfJunctions: Int) => val graphWithJunctionsBuilder = (numOfJunctions: Int) =>
FlowGraph.closed() { implicit b => RunnableGraph.fromGraph(FlowGraph.create() { implicit b =>
import FlowGraph.Implicits._ import FlowGraph.Implicits._
val broadcast = b.add(Broadcast[Unit](numOfJunctions)) val broadcast = b.add(Broadcast[Unit](numOfJunctions))
@ -34,39 +34,39 @@ object MaterializationBenchmark {
Source.single(()) ~> broadcast Source.single(()) ~> broadcast
outlet ~> Sink.ignore outlet ~> Sink.ignore
} ClosedShape
})
val graphWithNestedImportsBuilder = (numOfNestedGraphs: Int) => { val graphWithNestedImportsBuilder = (numOfNestedGraphs: Int) => {
var flow: Graph[FlowShape[Unit, Unit], Unit] = Flow[Unit].map(identity) var flow: Graph[FlowShape[Unit, Unit], Unit] = Flow[Unit].map(identity)
for (_ <- 1 to numOfNestedGraphs) { for (_ <- 1 to numOfNestedGraphs) {
flow = FlowGraph.partial(flow) { b flow = FlowGraph.create(flow) { b
flow flow
FlowShape(flow.inlet, flow.outlet) FlowShape(flow.inlet, flow.outlet)
} }
} }
FlowGraph.closed(flow) { implicit b RunnableGraph.fromGraph(FlowGraph.create(flow) { implicit b
flow flow
import FlowGraph.Implicits._ import FlowGraph.Implicits._
Source.single(()) ~> flow ~> Sink.ignore Source.single(()) ~> flow ~> Sink.ignore
} ClosedShape
})
} }
val graphWithImportedFlowBuilder = (numOfFlows: Int) => { val graphWithImportedFlowBuilder = (numOfFlows: Int) =>
val flow = Flow[Unit].map(identity) RunnableGraph.fromGraph(FlowGraph.create(Source.single(())) { implicit b source
FlowGraph.closed() { b import FlowGraph.Implicits._
val source = b.add(Source.single(())) val flow = Flow[Unit].map(identity)
var outlet = source var outlet: Outlet[Unit] = source.outlet
for (i <- 0 until numOfFlows) { for (i <- 0 until numOfFlows) {
val flowShape = b.add(flow) val flowShape = b.add(flow)
b.addEdge(outlet, flowShape.inlet) outlet ~> flowShape
outlet = flowShape.outlet outlet = flowShape.outlet
} }
outlet ~> Sink.ignore
val sink = b.add(Sink.ignore) ClosedShape
b.addEdge(outlet, sink) })
}
}
} }
@State(Scope.Benchmark) @State(Scope.Benchmark)

View file

@ -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 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, 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 directed and non-directed cycles. The ``runnable()`` 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: 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 .. 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 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 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: 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 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 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 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, :ref:`stream-buffers-scala`). When it comes to hierarchic composition, attributes are inherited by nested modules,

View file

@ -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 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. 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 This can be achieved using ``FlowGraph.create()`` instead of
``FlowGraph.factory().closed()``, which will return a ``Graph`` instead of a ``FlowGraph.runnable()``, which will return a ``Graph`` instead of a
``RunnableGraph``. The reason of representing it as a different type is that 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 :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 it will throw an exception at construction time, which helps to avoid simple

View file

@ -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 .. 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 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 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`. and then combine them into one fully connected graph. This will be covered in detail in :ref:`partial-flow-graph-java`.

View file

@ -44,21 +44,21 @@ object BidiFlowDocSpec {
} }
//#codec-impl //#codec-impl
val codecVerbose = BidiFlow() { b => val codecVerbose = BidiFlow.fromGraph(FlowGraph.create() { b =>
// construct and add the top flow, going outbound // construct and add the top flow, going outbound
val outbound = b.add(Flow[Message].map(toBytes)) val outbound = b.add(Flow[Message].map(toBytes))
// construct and add the bottom flow, going inbound // construct and add the bottom flow, going inbound
val inbound = b.add(Flow[ByteString].map(fromBytes)) val inbound = b.add(Flow[ByteString].map(fromBytes))
// fuse them together into a BidiShape // fuse them together into a BidiShape
BidiShape.fromFlows(outbound, inbound) BidiShape.fromFlows(outbound, inbound)
} })
// this is the same as the above // this is the same as the above
val codec = BidiFlow(toBytes _, fromBytes _) val codec = BidiFlow.fromFunctions(toBytes _, fromBytes _)
//#codec //#codec
//#framing //#framing
val framing = BidiFlow() { b => val framing = BidiFlow.fromGraph(FlowGraph.create() { b =>
implicit val order = ByteOrder.LITTLE_ENDIAN implicit val order = ByteOrder.LITTLE_ENDIAN
def addLengthHeader(bytes: ByteString) = { def addLengthHeader(bytes: ByteString) = {
@ -113,18 +113,18 @@ object BidiFlowDocSpec {
val outbound = b.add(Flow[ByteString].map(addLengthHeader)) val outbound = b.add(Flow[ByteString].map(addLengthHeader))
val inbound = b.add(Flow[ByteString].transform(() => new FrameParser)) val inbound = b.add(Flow[ByteString].transform(() => new FrameParser))
BidiShape.fromFlows(outbound, inbound) BidiShape.fromFlows(outbound, inbound)
} })
//#framing //#framing
val chopUp = BidiFlow() { b => val chopUp = BidiFlow.fromGraph(FlowGraph.create() { b =>
val f = Flow[ByteString].mapConcat(_.map(ByteString(_))) val f = Flow[ByteString].mapConcat(_.map(ByteString(_)))
BidiShape.fromFlows(b.add(f), b.add(f)) 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)(_ ++ _)) val f = Flow[ByteString].grouped(1000).map(_.fold(ByteString.empty)(_ ++ _))
BidiShape.fromFlows(b.add(f), b.add(f)) BidiShape.fromFlows(b.add(f), b.add(f))
} })
} }
class BidiFlowDocSpec extends AkkaSpec with ConversionCheckedTripleEquals { class BidiFlowDocSpec extends AkkaSpec with ConversionCheckedTripleEquals {

View file

@ -77,7 +77,7 @@ class CompositionDocSpec extends AkkaSpec {
// format: OFF // format: OFF
//#complex-graph //#complex-graph
import FlowGraph.Implicits._ import FlowGraph.Implicits._
FlowGraph.closed() { implicit builder => RunnableGraph.fromGraph(FlowGraph.create() { implicit builder =>
val A: Outlet[Int] = builder.add(Source.single(0)) val A: Outlet[Int] = builder.add(Source.single(0))
val B: UniformFanOutShape[Int, Int] = builder.add(Broadcast[Int](2)) val B: UniformFanOutShape[Int, Int] = builder.add(Broadcast[Int](2))
val C: UniformFanInShape[Int, Int] = builder.add(Merge[Int](2)) val C: UniformFanInShape[Int, Int] = builder.add(Merge[Int](2))
@ -90,12 +90,14 @@ class CompositionDocSpec extends AkkaSpec {
A ~> B ~> C ~> F A ~> B ~> C ~> F
B ~> D ~> E ~> F B ~> D ~> E ~> F
E ~> G E ~> G
}
ClosedShape
})
//#complex-graph //#complex-graph
//#complex-graph-alt //#complex-graph-alt
import FlowGraph.Implicits._ import FlowGraph.Implicits._
FlowGraph.closed() { implicit builder => RunnableGraph.fromGraph(FlowGraph.create() { implicit builder =>
val B = builder.add(Broadcast[Int](2)) val B = builder.add(Broadcast[Int](2))
val C = builder.add(Merge[Int](2)) val C = builder.add(Merge[Int](2))
val E = builder.add(Balance[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) B.out(1).map(_ + 1) ~> E.in; E.out(0) ~> F.in(1)
E.out(1) ~> Sink.foreach(println) E.out(1) ~> Sink.foreach(println)
} ClosedShape
})
//#complex-graph-alt //#complex-graph-alt
// format: ON // format: ON
} }
@ -115,7 +118,7 @@ class CompositionDocSpec extends AkkaSpec {
// format: OFF // format: OFF
//#partial-graph //#partial-graph
import FlowGraph.Implicits._ import FlowGraph.Implicits._
val partial = FlowGraph.partial() { implicit builder => val partial = FlowGraph.create() { implicit builder =>
val B = builder.add(Broadcast[Int](2)) val B = builder.add(Broadcast[Int](2))
val C = builder.add(Merge[Int](2)) val C = builder.add(Merge[Int](2))
val E = builder.add(Balance[Int](2)) val E = builder.add(Balance[Int](2))
@ -137,17 +140,17 @@ class CompositionDocSpec extends AkkaSpec {
//#partial-flow-dsl //#partial-flow-dsl
// Convert the partial graph of FlowShape to a Flow to get // Convert the partial graph of FlowShape to a Flow to get
// access to the fluid DSL (for example to be able to call .filter()) // 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 // 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)) val merge = builder.add(Merge[Int](2))
Source.single(0) ~> merge Source.single(0) ~> merge
Source(List(2, 3, 4)) ~> merge Source(List(2, 3, 4)) ~> merge
// Exposing exactly one output port // Exposing exactly one output port
merge.out SourceShape(merge.out)
} })
// Building a Sink with a nested Flow, using the fluid DSL // Building a Sink with a nested Flow, using the fluid DSL
val sink = { val sink = {
@ -164,22 +167,24 @@ class CompositionDocSpec extends AkkaSpec {
"closed graph" in { "closed graph" in {
//#embed-closed //#embed-closed
val closed1 = Source.single(0).to(Sink.foreach(println)) 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) val embeddedClosed: ClosedShape = builder.add(closed1)
} //
embeddedClosed
})
//#embed-closed //#embed-closed
} }
"materialized values" in { "materialized values" in {
//#mat-combine-1 //#mat-combine-1
// Materializes to Promise[Unit] (red) // Materializes to Promise[Option[Int]] (red)
val source: Source[Int, Promise[Unit]] = Source.lazyEmpty[Int] val source: Source[Int, Promise[Option[Int]]] = Source.maybe[Int]
// Materializes to Unit (black) // Materializes to Unit (black)
val flow1: Flow[Int, Int, Unit] = Flow[Int].take(100) val flow1: Flow[Int, Int, Unit] = Flow[Int].take(100)
// Materializes to Promise[Unit] (red) // Materializes to Promise[Int] (red)
val nestedSource: Source[Int, Promise[Unit]] = val nestedSource: Source[Int, Promise[Option[Int]]] =
source.viaMat(flow1)(Keep.left).named("nestedSource") source.viaMat(flow1)(Keep.left).named("nestedSource")
//#mat-combine-1 //#mat-combine-1
@ -206,11 +211,11 @@ class CompositionDocSpec extends AkkaSpec {
//#mat-combine-3 //#mat-combine-3
//#mat-combine-4 //#mat-combine-4
case class MyClass(private val p: Promise[Unit], conn: OutgoingConnection) { case class MyClass(private val p: Promise[Option[Int]], conn: OutgoingConnection) {
def close() = p.success(()) def close() = p.trySuccess(None)
} }
def f(p: Promise[Unit], def f(p: Promise[Option[Int]],
rest: (Future[OutgoingConnection], Future[String])): Future[MyClass] = { rest: (Future[OutgoingConnection], Future[String])): Future[MyClass] = {
val connFuture = rest._1 val connFuture = rest._1

View file

@ -4,6 +4,7 @@
package docs.stream package docs.stream
import akka.actor.Cancellable import akka.actor.Cancellable
import akka.stream.{ ClosedShape, FlowShape }
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
@ -147,17 +148,17 @@ class FlowDocSpec extends AkkaSpec {
"various ways of transforming materialized values" in { "various ways of transforming materialized values" in {
import scala.concurrent.duration._ 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 => tickSource =>
import FlowGraph.Implicits._ import FlowGraph.Implicits._
val zip = builder.add(ZipWith[String, Int, Int](Keep.right)) val zip = builder.add(ZipWith[String, Int, Int](Keep.right))
tickSource ~> zip.in0 tickSource ~> zip.in0
(zip.in1, zip.out) FlowShape(zip.in1, zip.out)
} })
//#flow-mat-combine //#flow-mat-combine
// An empty source that can be shut down explicitly from the outside // An source that can be signalled explicitly from the outside
val source: Source[Int, Promise[Unit]] = Source.lazyEmpty[Int] val source: Source[Int, Promise[Option[Int]]] = Source.maybe[Int]
// A flow that internally throttles elements to 1/second, and returns a Cancellable // A flow that internally throttles elements to 1/second, and returns a Cancellable
// which can be used to shut down the stream // 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] val sink: Sink[Int, Future[Int]] = Sink.head[Int]
// By default, the materialized value of the leftmost stage is preserved // 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 // Simple selection of materialized values by using Keep.right
val r2: RunnableGraph[Cancellable] = source.viaMat(flow)(Keep.right).to(sink) 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 // Using runWith will always give the materialized values of the stages added
// by runWith() itself // by runWith() itself
val r4: Future[Int] = source.via(flow).runWith(sink) val r4: Future[Int] = source.via(flow).runWith(sink)
val r5: Promise[Unit] = flow.to(sink).runWith(source) val r5: Promise[Option[Int]] = flow.to(sink).runWith(source)
val r6: (Promise[Unit], Future[Int]) = flow.runWith(source, sink) val r6: (Promise[Option[Int]], Future[Int]) = flow.runWith(source, sink)
// Using more complext combinations // Using more complext combinations
val r7: RunnableGraph[(Promise[Unit], Cancellable)] = val r7: RunnableGraph[(Promise[Option[Int]], Cancellable)] =
source.viaMat(flow)(Keep.both).to(sink) 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) 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) source.viaMat(flow)(Keep.both).toMat(sink)(Keep.both)
val r10: RunnableGraph[(Cancellable, Future[Int])] = 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 // 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 // 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 { r9.mapMaterializedValue {
case ((promise, cancellable), future) => case ((promise, cancellable), future) =>
(promise, cancellable, future) (promise, cancellable, future)
@ -204,17 +205,18 @@ class FlowDocSpec extends AkkaSpec {
val (promise, cancellable, future) = r11.run() val (promise, cancellable, future) = r11.run()
// Type inference works as expected // Type inference works as expected
promise.success(()) promise.success(None)
cancellable.cancel() cancellable.cancel()
future.map(_ + 3) future.map(_ + 3)
// The result of r11 can be also achieved by using the Graph API // The result of r11 can be also achieved by using the Graph API
val r12: RunnableGraph[(Promise[Unit], Cancellable, Future[Int])] = val r12: RunnableGraph[(Promise[Option[Int]], Cancellable, Future[Int])] =
FlowGraph.closed(source, flow, sink)((_, _, _)) { implicit builder => RunnableGraph.fromGraph(FlowGraph.create(source, flow, sink)((_, _, _)) { implicit builder =>
(src, f, dst) => (src, f, dst) =>
import FlowGraph.Implicits._ import FlowGraph.Implicits._
src ~> f ~> dst src ~> f ~> dst
} ClosedShape
})
//#flow-mat-combine //#flow-mat-combine
} }

View file

@ -21,7 +21,7 @@ class FlowGraphDocSpec extends AkkaSpec {
"build simple graph" in { "build simple graph" in {
//format: OFF //format: OFF
//#simple-flow-graph //#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._ import FlowGraph.Implicits._
val in = Source(1 to 10) val in = Source(1 to 10)
val out = Sink.ignore val out = Sink.ignore
@ -32,8 +32,9 @@ class FlowGraphDocSpec extends AkkaSpec {
val f1, f2, f3, f4 = Flow[Int].map(_ + 10) val f1, f2, f3, f4 = Flow[Int].map(_ + 10)
in ~> f1 ~> bcast ~> f2 ~> merge ~> f3 ~> out in ~> f1 ~> bcast ~> f2 ~> merge ~> f3 ~> out
bcast ~> f4 ~> merge bcast ~> f4 ~> merge
} ClosedShape
})
//#simple-flow-graph //#simple-flow-graph
//format: ON //format: ON
@ -42,33 +43,10 @@ class FlowGraphDocSpec extends AkkaSpec {
//#simple-graph-run //#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 { "flow connection errors" in {
intercept[IllegalArgumentException] { intercept[IllegalArgumentException] {
//#simple-graph //#simple-graph
FlowGraph.closed() { implicit builder => RunnableGraph.fromGraph(FlowGraph.create() { implicit builder =>
import FlowGraph.Implicits._ import FlowGraph.Implicits._
val source1 = Source(1 to 10) val source1 = Source(1 to 10)
val source2 = Source(1 to 10) val source2 = Source(1 to 10)
@ -78,9 +56,10 @@ class FlowGraphDocSpec extends AkkaSpec {
source1 ~> zip.in0 source1 ~> zip.in0
source2 ~> zip.in1 source2 ~> zip.in1
// unconnected zip.out (!) => "must have at least 1 outgoing edge" // unconnected zip.out (!) => "must have at least 1 outgoing edge"
} ClosedShape
})
//#simple-graph //#simple-graph
}.getMessage should include("unconnected ports: ZipWith2.out") }.getMessage should include("ZipWith2.out")
} }
"reusing a flow in a graph" in { "reusing a flow in a graph" in {
@ -95,7 +74,7 @@ class FlowGraphDocSpec extends AkkaSpec {
// format: OFF // format: OFF
val g = val g =
//#flow-graph-reusing-a-flow //#flow-graph-reusing-a-flow
FlowGraph.closed(topHeadSink, bottomHeadSink)((_, _)) { implicit builder => RunnableGraph.fromGraph(FlowGraph.create(topHeadSink, bottomHeadSink)((_, _)) { implicit builder =>
(topHS, bottomHS) => (topHS, bottomHS) =>
import FlowGraph.Implicits._ import FlowGraph.Implicits._
val broadcast = builder.add(Broadcast[Int](2)) val broadcast = builder.add(Broadcast[Int](2))
@ -103,7 +82,8 @@ class FlowGraphDocSpec extends AkkaSpec {
broadcast.out(0) ~> sharedDoubler ~> topHS.inlet broadcast.out(0) ~> sharedDoubler ~> topHS.inlet
broadcast.out(1) ~> sharedDoubler ~> bottomHS.inlet broadcast.out(1) ~> sharedDoubler ~> bottomHS.inlet
} ClosedShape
})
//#flow-graph-reusing-a-flow //#flow-graph-reusing-a-flow
// format: ON // format: ON
val (topFuture, bottomFuture) = g.run() val (topFuture, bottomFuture) = g.run()
@ -153,7 +133,7 @@ class FlowGraphDocSpec extends AkkaSpec {
worker: Flow[In, Out, Any], worker: Flow[In, Out, Any],
workerCount: Int): Graph[PriorityWorkerPoolShape[In, Out], Unit] = { workerCount: Int): Graph[PriorityWorkerPoolShape[In, Out], Unit] = {
FlowGraph.partial() { implicit b FlowGraph.create() { implicit b
import FlowGraph.Implicits._ import FlowGraph.Implicits._
val priorityMerge = b.add(MergePreferred[In](1)) val priorityMerge = b.add(MergePreferred[In](1))
@ -188,7 +168,7 @@ class FlowGraphDocSpec extends AkkaSpec {
val worker1 = Flow[String].map("step 1 " + _) val worker1 = Flow[String].map("step 1 " + _)
val worker2 = Flow[String].map("step 2 " + _) val worker2 = Flow[String].map("step 2 " + _)
FlowGraph.closed() { implicit b => RunnableGraph.fromGraph(FlowGraph.create() { implicit b =>
import FlowGraph.Implicits._ import FlowGraph.Implicits._
val priorityPool1 = b.add(PriorityWorkerPool(worker1, 4)) 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 Source(1 to 100).map("one-step, priority " + _) ~> priorityPool2.priorityJobsIn
priorityPool2.resultsOut ~> Sink.foreach(println) priorityPool2.resultsOut ~> Sink.foreach(println)
}.run() ClosedShape
}).run()
//#flow-graph-components-use //#flow-graph-components-use
//#flow-graph-components-shape2 //#flow-graph-components-shape2
@ -223,11 +204,11 @@ class FlowGraphDocSpec extends AkkaSpec {
"access to materialized value" in { "access to materialized value" in {
//#flow-graph-matvalue //#flow-graph-matvalue
import FlowGraph.Implicits._ 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 implicit builder
fold fold
(fold.inlet, builder.materializedValue.mapAsync(4)(identity).outlet) FlowShape(fold.inlet, builder.materializedValue.mapAsync(4)(identity).outlet)
} })
//#flow-graph-matvalue //#flow-graph-matvalue
Await.result(Source(1 to 10).via(foldFlow).runWith(Sink.head), 3.seconds) should ===(55) 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 //#flow-graph-matvalue-cycle
import FlowGraph.Implicits._ import FlowGraph.Implicits._
// This cannot produce any value: // 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 => implicit builder =>
fold => fold =>
// - Fold cannot complete until its upstream mapAsync completes // - 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 // As a result this Source will never emit anything, and its materialited
// Future will never complete // Future will never complete
builder.materializedValue.mapAsync(4)(identity) ~> fold builder.materializedValue.mapAsync(4)(identity) ~> fold
builder.materializedValue.mapAsync(4)(identity).outlet SourceShape(builder.materializedValue.mapAsync(4)(identity).outlet)
} })
//#flow-graph-matvalue-cycle //#flow-graph-matvalue-cycle
} }

View file

@ -1,5 +1,6 @@
package docs.stream package docs.stream
import akka.stream.FlowShape
import akka.stream.scaladsl.{ FlowGraph, Merge, Balance, Source, Flow } import akka.stream.scaladsl.{ FlowGraph, Merge, Balance, Source, Flow }
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
@ -37,7 +38,7 @@ class FlowParallelismDocSpec extends AkkaSpec {
val fryingPan: Flow[ScoopOfBatter, Pancake, Unit] = val fryingPan: Flow[ScoopOfBatter, Pancake, Unit] =
Flow[ScoopOfBatter].map { batter => Pancake() } 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 dispatchBatter = builder.add(Balance[ScoopOfBatter](2))
val mergePancakes = builder.add(Merge[Pancake](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. // uses of "fryingPan" mean actually different stages in the graph.
dispatchBatter.out(1) ~> fryingPan ~> mergePancakes.in(1) dispatchBatter.out(1) ~> fryingPan ~> mergePancakes.in(1)
(dispatchBatter.in, mergePancakes.out) FlowShape(dispatchBatter.in, mergePancakes.out)
} })
//#parallelism //#parallelism
} }
"Demonstrate parallelized pipelines" in { "Demonstrate parallelized pipelines" in {
//#parallel-pipeline //#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 dispatchBatter = builder.add(Balance[ScoopOfBatter](2))
val mergePancakes = builder.add(Merge[Pancake](2)) val mergePancakes = builder.add(Merge[Pancake](2))
// Using two pipelines, having two frying pans each, in total using // Using two pipelines, having two frying pans each, in total using
// four frying pans // four frying pans
dispatchBatter.out(0) ~> fryingPan1 ~> fryingPan2 ~> mergePancakes.in(0) dispatchBatter.out(0) ~> fryingPan1 ~> fryingPan2 ~> mergePancakes.in(0)
dispatchBatter.out(1) ~> fryingPan1 ~> fryingPan2 ~> mergePancakes.in(1) dispatchBatter.out(1) ~> fryingPan1 ~> fryingPan2 ~> mergePancakes.in(1)
(dispatchBatter.in, mergePancakes.out) FlowShape(dispatchBatter.in, mergePancakes.out)
} })
//#parallel-pipeline //#parallel-pipeline
} }
"Demonstrate pipelined parallel processing" in { "Demonstrate pipelined parallel processing" in {
//#pipelined-parallel //#pipelined-parallel
val pancakeChefs1: Flow[ScoopOfBatter, HalfCookedPancake, Unit] = Flow() { implicit builder => val pancakeChefs1: Flow[ScoopOfBatter, HalfCookedPancake, Unit] =
val dispatchBatter = builder.add(Balance[ScoopOfBatter](2)) Flow.fromGraph(FlowGraph.create() { implicit builder =>
val mergeHalfPancakes = builder.add(Merge[HalfCookedPancake](2)) 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 // Two chefs work with one frying pan for each, half-frying the pancakes then putting
// them into a common pool // them into a common pool
dispatchBatter.out(0) ~> fryingPan1 ~> mergeHalfPancakes.in(0) dispatchBatter.out(0) ~> fryingPan1 ~> mergeHalfPancakes.in(0)
dispatchBatter.out(1) ~> fryingPan1 ~> mergeHalfPancakes.in(1) 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 pancakeChefs2: Flow[HalfCookedPancake, Pancake, Unit] =
val dispatchHalfPancakes = builder.add(Balance[HalfCookedPancake](2)) Flow.fromGraph(FlowGraph.create() { implicit builder =>
val mergePancakes = builder.add(Merge[Pancake](2)) 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 // Two chefs work with one frying pan for each, finishing the pancakes then putting
// them into a common pool // them into a common pool
dispatchHalfPancakes.out(0) ~> fryingPan2 ~> mergePancakes.in(0) dispatchHalfPancakes.out(0) ~> fryingPan2 ~> mergePancakes.in(0)
dispatchHalfPancakes.out(1) ~> fryingPan2 ~> mergePancakes.in(1) 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) val kitchen: Flow[ScoopOfBatter, Pancake, Unit] = pancakeChefs1.via(pancakeChefs2)
//#pipelined-parallel //#pipelined-parallel

View file

@ -1,6 +1,6 @@
package docs.stream package docs.stream
import akka.stream.{ OverflowStrategy, ActorMaterializer } import akka.stream.{ ClosedShape, OverflowStrategy, ActorMaterializer }
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
@ -16,7 +16,7 @@ class GraphCyclesSpec extends AkkaSpec {
// format: OFF // format: OFF
//#deadlocked //#deadlocked
// WARNING! The graph below deadlocks! // WARNING! The graph below deadlocks!
FlowGraph.closed() { implicit b => RunnableGraph.fromGraph(FlowGraph.create() { implicit b =>
import FlowGraph.Implicits._ import FlowGraph.Implicits._
val merge = b.add(Merge[Int](2)) 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 source ~> merge ~> Flow[Int].map { s => println(s); s } ~> bcast ~> Sink.ignore
merge <~ bcast merge <~ bcast
} ClosedShape
})
//#deadlocked //#deadlocked
// format: ON // format: ON
} }
@ -33,7 +34,7 @@ class GraphCyclesSpec extends AkkaSpec {
// format: OFF // format: OFF
//#unfair //#unfair
// WARNING! The graph below stops consuming from "source" after a few steps // 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._ import FlowGraph.Implicits._
val merge = b.add(MergePreferred[Int](1)) 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 source ~> merge ~> Flow[Int].map { s => println(s); s } ~> bcast ~> Sink.ignore
merge.preferred <~ bcast merge.preferred <~ bcast
} ClosedShape
})
//#unfair //#unfair
// format: ON // format: ON
} }
@ -49,7 +51,7 @@ class GraphCyclesSpec extends AkkaSpec {
"include a dropping cycle" in { "include a dropping cycle" in {
// format: OFF // format: OFF
//#dropping //#dropping
FlowGraph.closed() { implicit b => RunnableGraph.fromGraph(FlowGraph.create() { implicit b =>
import FlowGraph.Implicits._ import FlowGraph.Implicits._
val merge = b.add(Merge[Int](2)) 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 source ~> merge ~> Flow[Int].map { s => println(s); s } ~> bcast ~> Sink.ignore
merge <~ Flow[Int].buffer(10, OverflowStrategy.dropHead) <~ bcast merge <~ Flow[Int].buffer(10, OverflowStrategy.dropHead) <~ bcast
} ClosedShape
})
//#dropping //#dropping
// format: ON // format: ON
} }
@ -66,7 +69,7 @@ class GraphCyclesSpec extends AkkaSpec {
// format: OFF // format: OFF
//#zipping-dead //#zipping-dead
// WARNING! The graph below never processes any elements // WARNING! The graph below never processes any elements
FlowGraph.closed() { implicit b => RunnableGraph.fromGraph(FlowGraph.create() { implicit b =>
import FlowGraph.Implicits._ import FlowGraph.Implicits._
val zip = b.add(ZipWith[Int, Int, Int]((left, right) => right)) val zip = b.add(ZipWith[Int, Int, Int]((left, right) => right))
@ -75,7 +78,8 @@ class GraphCyclesSpec extends AkkaSpec {
source ~> zip.in0 source ~> zip.in0
zip.out.map { s => println(s); s } ~> bcast ~> Sink.ignore zip.out.map { s => println(s); s } ~> bcast ~> Sink.ignore
zip.in1 <~ bcast zip.in1 <~ bcast
} ClosedShape
})
//#zipping-dead //#zipping-dead
// format: ON // format: ON
} }
@ -83,7 +87,7 @@ class GraphCyclesSpec extends AkkaSpec {
"include a live zipping cycle" in { "include a live zipping cycle" in {
// format: OFF // format: OFF
//#zipping-live //#zipping-live
FlowGraph.closed() { implicit b => RunnableGraph.fromGraph(FlowGraph.create() { implicit b =>
import FlowGraph.Implicits._ import FlowGraph.Implicits._
val zip = b.add(ZipWith((left: Int, right: Int) => left)) 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.out.map { s => println(s); s } ~> bcast ~> Sink.ignore
zip.in1 <~ concat <~ start zip.in1 <~ concat <~ start
concat <~ bcast concat <~ bcast
} ClosedShape
})
//#zipping-live //#zipping-live
// format: ON // format: ON
} }

View file

@ -1,9 +1,8 @@
package docs.stream package docs.stream
import akka.stream.{ OverflowStrategy, ActorMaterializerSettings, ActorMaterializer } import akka.stream._
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
import akka.stream.Attributes
class StreamBuffersRateSpec extends AkkaSpec { class StreamBuffersRateSpec extends AkkaSpec {
implicit val mat = ActorMaterializer() implicit val mat = ActorMaterializer()
@ -40,7 +39,7 @@ class StreamBuffersRateSpec extends AkkaSpec {
import scala.concurrent.duration._ import scala.concurrent.duration._
case class Tick() case class Tick()
FlowGraph.closed() { implicit b => RunnableGraph.fromGraph(FlowGraph.create() { implicit b =>
import FlowGraph.Implicits._ import FlowGraph.Implicits._
val zipper = b.add(ZipWith[Tick, Int, Int]((tick, count) => count)) 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 .conflate(seed = (_) => 1)((count, _) => count + 1) ~> zipper.in1
zipper.out ~> Sink.foreach(println) zipper.out ~> Sink.foreach(println)
} ClosedShape
})
//#buffering-abstraction-leak //#buffering-abstraction-leak
} }

View file

@ -19,7 +19,7 @@ class StreamPartialFlowGraphDocSpec extends AkkaSpec {
"build with open ports" in { "build with open ports" in {
//#simple-partial-flow-graph //#simple-partial-flow-graph
val pickMaxOfThree = FlowGraph.partial() { implicit b => val pickMaxOfThree = FlowGraph.create() { implicit b =>
import FlowGraph.Implicits._ import FlowGraph.Implicits._
val zip1 = b.add(ZipWith[Int, Int, Int](math.max _)) val zip1 = b.add(ZipWith[Int, Int, Int](math.max _))
@ -31,7 +31,7 @@ class StreamPartialFlowGraphDocSpec extends AkkaSpec {
val resultSink = Sink.head[Int] val resultSink = Sink.head[Int]
val g = FlowGraph.closed(resultSink) { implicit b => val g = RunnableGraph.fromGraph(FlowGraph.create(resultSink) { implicit b =>
sink => sink =>
import FlowGraph.Implicits._ import FlowGraph.Implicits._
@ -42,7 +42,8 @@ class StreamPartialFlowGraphDocSpec extends AkkaSpec {
Source.single(2) ~> pm3.in(1) Source.single(2) ~> pm3.in(1)
Source.single(3) ~> pm3.in(2) Source.single(3) ~> pm3.in(2)
pm3.out ~> sink.inlet pm3.out ~> sink.inlet
} ClosedShape
})
val max: Future[Int] = g.run() val max: Future[Int] = g.run()
Await.result(max, 300.millis) should equal(3) Await.result(max, 300.millis) should equal(3)
@ -51,7 +52,7 @@ class StreamPartialFlowGraphDocSpec extends AkkaSpec {
"build source from partial flow graph" in { "build source from partial flow graph" in {
//#source-from-partial-flow-graph //#source-from-partial-flow-graph
val pairs = Source() { implicit b => val pairs = Source.fromGraph(FlowGraph.create() { implicit b =>
import FlowGraph.Implicits._ import FlowGraph.Implicits._
// prepare graph elements // prepare graph elements
@ -63,8 +64,8 @@ class StreamPartialFlowGraphDocSpec extends AkkaSpec {
ints.filter(_ % 2 == 0) ~> zip.in1 ints.filter(_ % 2 == 0) ~> zip.in1
// expose port // expose port
zip.out SourceShape(zip.out)
} })
val firstPair: Future[(Int, Int)] = pairs.runWith(Sink.head) val firstPair: Future[(Int, Int)] = pairs.runWith(Sink.head)
//#source-from-partial-flow-graph //#source-from-partial-flow-graph
@ -73,20 +74,21 @@ class StreamPartialFlowGraphDocSpec extends AkkaSpec {
"build flow from partial flow graph" in { "build flow from partial flow graph" in {
//#flow-from-partial-flow-graph //#flow-from-partial-flow-graph
val pairUpWithToString = Flow() { implicit b => val pairUpWithToString =
import FlowGraph.Implicits._ Flow.fromGraph(FlowGraph.create() { implicit b =>
import FlowGraph.Implicits._
// prepare graph elements // prepare graph elements
val broadcast = b.add(Broadcast[Int](2)) val broadcast = b.add(Broadcast[Int](2))
val zip = b.add(Zip[Int, String]()) val zip = b.add(Zip[Int, String]())
// connect the graph // connect the graph
broadcast.out(0).map(identity) ~> zip.in0 broadcast.out(0).map(identity) ~> zip.in0
broadcast.out(1).map(_.toString) ~> zip.in1 broadcast.out(1).map(_.toString) ~> zip.in1
// expose ports // expose ports
(broadcast.in, zip.out) FlowShape(broadcast.in, zip.out)
} })
//#flow-from-partial-flow-graph //#flow-from-partial-flow-graph

View file

@ -6,8 +6,7 @@ package docs.stream
//#imports //#imports
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.stream.ActorMaterializer import akka.stream.{ ClosedShape, ActorMaterializer, OverflowStrategy }
import akka.stream.OverflowStrategy
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import scala.concurrent.Await import scala.concurrent.Await
@ -119,14 +118,15 @@ class TwitterStreamQuickstartDocSpec extends AkkaSpec {
// format: OFF // format: OFF
//#flow-graph-broadcast //#flow-graph-broadcast
val g = FlowGraph.closed() { implicit b => val g = RunnableGraph.fromGraph(FlowGraph.create() { implicit b =>
import FlowGraph.Implicits._ import FlowGraph.Implicits._
val bcast = b.add(Broadcast[Tweet](2)) val bcast = b.add(Broadcast[Tweet](2))
tweets ~> bcast.in tweets ~> bcast.in
bcast.out(0) ~> Flow[Tweet].map(_.author) ~> writeAuthors bcast.out(0) ~> Flow[Tweet].map(_.author) ~> writeAuthors
bcast.out(1) ~> Flow[Tweet].mapConcat(_.hashtags.toList) ~> writeHashtags bcast.out(1) ~> Flow[Tweet].mapConcat(_.hashtags.toList) ~> writeHashtags
} ClosedShape
})
g.run() g.run()
//#flow-graph-broadcast //#flow-graph-broadcast
// format: ON // format: ON

View file

@ -1,6 +1,6 @@
package docs.stream.cookbook package docs.stream.cookbook
import akka.stream.OverflowStrategy import akka.stream.{ ClosedShape, OverflowStrategy }
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.stream.testkit._ import akka.stream.testkit._
@ -24,7 +24,7 @@ class RecipeDroppyBroadcast extends RecipeSpec {
val mySink3 = Sink(sub3) val mySink3 = Sink(sub3)
//#droppy-bcast //#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) => (sink1, sink2, sink3) =>
import FlowGraph.Implicits._ import FlowGraph.Implicits._
@ -34,7 +34,8 @@ class RecipeDroppyBroadcast extends RecipeSpec {
bcast.buffer(10, OverflowStrategy.dropHead) ~> sink1 bcast.buffer(10, OverflowStrategy.dropHead) ~> sink1
bcast.buffer(10, OverflowStrategy.dropHead) ~> sink2 bcast.buffer(10, OverflowStrategy.dropHead) ~> sink2
bcast.buffer(10, OverflowStrategy.dropHead) ~> sink3 bcast.buffer(10, OverflowStrategy.dropHead) ~> sink3
} ClosedShape
})
//#droppy-bcast //#droppy-bcast
graph.run() graph.run()

View file

@ -2,6 +2,7 @@ package docs.stream.cookbook
import akka.actor.{ Props, ActorRef, Actor } import akka.actor.{ Props, ActorRef, Actor }
import akka.actor.Actor.Receive import akka.actor.Actor.Receive
import akka.stream.ClosedShape
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.stream.testkit._ import akka.stream.testkit._
@ -98,12 +99,13 @@ class RecipeGlobalRateLimit extends RecipeSpec {
val probe = TestSubscriber.manualProbe[String]() val probe = TestSubscriber.manualProbe[String]()
FlowGraph.closed() { implicit b => RunnableGraph.fromGraph(FlowGraph.create() { implicit b =>
import FlowGraph.Implicits._ import FlowGraph.Implicits._
val merge = b.add(Merge[String](2)) val merge = b.add(Merge[String](2))
source1 ~> merge ~> Sink(probe) source1 ~> merge ~> Sink(probe)
source2 ~> merge source2 ~> merge
}.run() ClosedShape
}).run()
probe.expectSubscription().request(1000) probe.expectSubscription().request(1000)

View file

@ -1,5 +1,6 @@
package docs.stream.cookbook package docs.stream.cookbook
import akka.stream.ClosedShape
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.stream.testkit._ import akka.stream.testkit._
import akka.util.ByteString import akka.util.ByteString
@ -25,14 +26,15 @@ class RecipeKeepAlive extends RecipeSpec {
val tickToKeepAlivePacket: Flow[Tick, ByteString, Unit] = Flow[Tick] val tickToKeepAlivePacket: Flow[Tick, ByteString, Unit] = Flow[Tick]
.conflate(seed = (tick) => keepaliveMessage)((msg, newTick) => msg) .conflate(seed = (tick) => keepaliveMessage)((msg, newTick) => msg)
val graph = FlowGraph.closed() { implicit builder => val graph = RunnableGraph.fromGraph(FlowGraph.create() { implicit builder =>
import FlowGraph.Implicits._ import FlowGraph.Implicits._
val unfairMerge = builder.add(MergePreferred[ByteString](1)) val unfairMerge = builder.add(MergePreferred[ByteString](1))
// If data is available then no keepalive is injected // If data is available then no keepalive is injected
dataStream ~> unfairMerge.preferred dataStream ~> unfairMerge.preferred
ticks ~> tickToKeepAlivePacket ~> unfairMerge ~> sink ticks ~> tickToKeepAlivePacket ~> unfairMerge ~> sink
} ClosedShape
})
//#inject-keepalive //#inject-keepalive
graph.run() graph.run()

View file

@ -1,5 +1,6 @@
package docs.stream.cookbook package docs.stream.cookbook
import akka.stream.ClosedShape
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.stream.testkit._ import akka.stream.testkit._
import scala.concurrent.duration._ import scala.concurrent.duration._
@ -17,13 +18,14 @@ class RecipeManualTrigger extends RecipeSpec {
val sink = Sink(sub) val sink = Sink(sub)
//#manually-triggered-stream //#manually-triggered-stream
val graph = FlowGraph.closed() { implicit builder => val graph = RunnableGraph.fromGraph(FlowGraph.create() { implicit builder =>
import FlowGraph.Implicits._ import FlowGraph.Implicits._
val zip = builder.add(Zip[Message, Trigger]()) val zip = builder.add(Zip[Message, Trigger]())
elements ~> zip.in0 elements ~> zip.in0
triggerSource ~> zip.in1 triggerSource ~> zip.in1
zip.out ~> Flow[(Message, Trigger)].map { case (msg, trigger) => msg } ~> sink zip.out ~> Flow[(Message, Trigger)].map { case (msg, trigger) => msg } ~> sink
} ClosedShape
})
//#manually-triggered-stream //#manually-triggered-stream
graph.run() graph.run()
@ -55,14 +57,15 @@ class RecipeManualTrigger extends RecipeSpec {
val sink = Sink(sub) val sink = Sink(sub)
//#manually-triggered-stream-zipwith //#manually-triggered-stream-zipwith
val graph = FlowGraph.closed() { implicit builder => val graph = RunnableGraph.fromGraph(FlowGraph.create() { implicit builder =>
import FlowGraph.Implicits._ import FlowGraph.Implicits._
val zip = builder.add(ZipWith((msg: Message, trigger: Trigger) => msg)) val zip = builder.add(ZipWith((msg: Message, trigger: Trigger) => msg))
elements ~> zip.in0 elements ~> zip.in0
triggerSource ~> zip.in1 triggerSource ~> zip.in1
zip.out ~> sink zip.out ~> sink
} ClosedShape
})
//#manually-triggered-stream-zipwith //#manually-triggered-stream-zipwith
graph.run() graph.run()

View file

@ -1,5 +1,6 @@
package docs.stream.cookbook package docs.stream.cookbook
import akka.stream.FlowShape
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.testkit.TestProbe 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] = { def balancer[In, Out](worker: Flow[In, Out, Any], workerCount: Int): Flow[In, Out, Unit] = {
import FlowGraph.Implicits._ import FlowGraph.Implicits._
Flow() { implicit b => Flow.fromGraph(FlowGraph.create() { implicit b =>
val balancer = b.add(Balance[In](workerCount, waitForAllDownstreams = true)) val balancer = b.add(Balance[In](workerCount, waitForAllDownstreams = true))
val merge = b.add(Merge[Out](workerCount)) val merge = b.add(Merge[Out](workerCount))
@ -30,8 +31,8 @@ class RecipeWorkerPool extends RecipeSpec {
balancer ~> worker ~> merge balancer ~> worker ~> merge
} }
(balancer.in, merge.out) FlowShape(balancer.in, merge.out)
} })
} }
val processedJobs: Source[Result, Unit] = myJobs.via(balancer(worker, 3)) val processedJobs: Source[Result, Unit] = myJobs.via(balancer(worker, 3))

View file

@ -73,7 +73,7 @@ class StreamTcpDocSpec extends AkkaSpec {
connections runForeach { connection => connections runForeach { connection =>
val serverLogic = Flow() { implicit b => val serverLogic = Flow.fromGraph(FlowGraph.create() { implicit b =>
import FlowGraph.Implicits._ import FlowGraph.Implicits._
// server logic, parses incoming commands // server logic, parses incoming commands
@ -109,8 +109,8 @@ class StreamTcpDocSpec extends AkkaSpec {
// then we continue using the echo-logic Flow // then we continue using the echo-logic Flow
echo.outlet ~> concat.in(1) echo.outlet ~> concat.in(1)
(echo.inlet, concat.out) FlowShape(echo.inlet, concat.out)
} })
connection.handleWith(serverLogic) connection.handleWith(serverLogic)
} }

View file

@ -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 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, 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 directed and non-directed cycles. The ``runnable()`` 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: 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 .. 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 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 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: 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 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 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 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, :ref:`stream-buffers-scala`). When it comes to hierarchic composition, attributes are inherited by nested modules,

View file

@ -60,9 +60,6 @@ will be inferred.
Notice the ``import FlowGraph.Implicits._`` which brings into scope the ``~>`` operator (read as "edge", "via" or "to") 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). 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*. 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. It is used (implicitly) by the ``~>`` operator, also making it a mutable operation as well.

View file

@ -76,7 +76,7 @@ private[http] object OutgoingConnectionBlueprint {
case (MessageStartError(_, info), _) throw IllegalResponseException(info) case (MessageStartError(_, info), _) throw IllegalResponseException(info)
} }
val core = BidiFlow() { implicit b val core = BidiFlow.fromGraph(FlowGraph.create() { implicit b
import FlowGraph.Implicits._ import FlowGraph.Implicits._
val methodBypassFanout = b.add(Broadcast[HttpRequest](2, eagerCancel = true)) val methodBypassFanout = b.add(Broadcast[HttpRequest](2, eagerCancel = true))
val responseParsingMerge = b.add(new ResponseParsingMerge(rootParser)) val responseParsingMerge = b.add(new ResponseParsingMerge(rootParser))
@ -103,7 +103,7 @@ private[http] object OutgoingConnectionBlueprint {
wrapTls.outlet, wrapTls.outlet,
unwrapTls.inlet, unwrapTls.inlet,
terminationFanout.out(1)) terminationFanout.out(1))
} })
One2OneBidiFlow[HttpRequest, HttpResponse](-1) atop core One2OneBidiFlow[HttpRequest, HttpResponse](-1) atop core
} }

View file

@ -65,7 +65,7 @@ private object PoolConductor {
*/ */
def apply(slotCount: Int, maxRetries: Int, pipeliningLimit: Int, log: LoggingAdapter): Graph[Ports, Any] = def apply(slotCount: Int, maxRetries: Int, pipeliningLimit: Int, log: LoggingAdapter): Graph[Ports, Any] =
FlowGraph.partial() { implicit b FlowGraph.create() { implicit b
import FlowGraph.Implicits._ import FlowGraph.Implicits._
val retryMerge = b.add(MergePreferred[RequestContext](1, eagerClose = true)) val retryMerge = b.add(MergePreferred[RequestContext](1, eagerClose = true))

View file

@ -11,7 +11,7 @@ import scala.concurrent.{ Promise, Future }
import scala.util.Try import scala.util.Try
import akka.event.LoggingAdapter import akka.event.LoggingAdapter
import akka.actor._ import akka.actor._
import akka.stream.Materializer import akka.stream.{ FlowShape, Materializer }
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.http.scaladsl.model._ import akka.http.scaladsl.model._
import akka.http.scaladsl.Http import akka.http.scaladsl.Http
@ -70,7 +70,7 @@ private object PoolFlow {
def apply(connectionFlow: Flow[HttpRequest, HttpResponse, Future[Http.OutgoingConnection]], def apply(connectionFlow: Flow[HttpRequest, HttpResponse, Future[Http.OutgoingConnection]],
remoteAddress: InetSocketAddress, settings: ConnectionPoolSettings, log: LoggingAdapter)( remoteAddress: InetSocketAddress, settings: ConnectionPoolSettings, log: LoggingAdapter)(
implicit system: ActorSystem, fm: Materializer): Flow[RequestContext, ResponseContext, Unit] = implicit system: ActorSystem, fm: Materializer): Flow[RequestContext, ResponseContext, Unit] =
Flow() { implicit b Flow.fromGraph(FlowGraph.create[FlowShape[RequestContext, ResponseContext]]() { implicit b
import settings._ import settings._
import FlowGraph.Implicits._ import FlowGraph.Implicits._
@ -87,6 +87,6 @@ private object PoolFlow {
slot.out0 ~> responseMerge.in(ix) slot.out0 ~> responseMerge.in(ix)
slot.out1 ~> slotEventMerge.in(ix) slot.out1 ~> slotEventMerge.in(ix)
} }
(conductor.requestIn, responseMerge.out) FlowShape(conductor.requestIn, responseMerge.out)
} })
} }

View file

@ -53,7 +53,7 @@ private object PoolSlot {
remoteAddress: InetSocketAddress, // TODO: remove after #16168 is cleared remoteAddress: InetSocketAddress, // TODO: remove after #16168 is cleared
settings: ConnectionPoolSettings)(implicit system: ActorSystem, settings: ConnectionPoolSettings)(implicit system: ActorSystem,
fm: Materializer): Graph[FanOutShape2[RequestContext, ResponseContext, RawSlotEvent], Any] = fm: Materializer): Graph[FanOutShape2[RequestContext, ResponseContext, RawSlotEvent], Any] =
FlowGraph.partial() { implicit b FlowGraph.create() { implicit b
import FlowGraph.Implicits._ import FlowGraph.Implicits._
val slotProcessor = b.add { val slotProcessor = b.add {

View file

@ -5,6 +5,7 @@
package akka.http.impl.engine.rendering package akka.http.impl.engine.rendering
import akka.parboiled2.CharUtils import akka.parboiled2.CharUtils
import akka.stream.SourceShape
import akka.util.ByteString import akka.util.ByteString
import akka.event.LoggingAdapter import akka.event.LoggingAdapter
import akka.stream.scaladsl._ import akka.stream.scaladsl._
@ -31,12 +32,12 @@ private object RenderSupport {
val defaultLastChunkBytes: ByteString = renderChunk(HttpEntity.LastChunk) val defaultLastChunkBytes: ByteString = renderChunk(HttpEntity.LastChunk)
def CancelSecond[T, Mat](first: Source[T, Mat], second: Source[T, Any]): Source[T, Mat] = { 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 frst
import FlowGraph.Implicits._ import FlowGraph.Implicits._
second ~> Sink.cancelled second ~> Sink.cancelled
frst.outlet SourceShape(frst.outlet)
} })
} }
def renderEntityContentType(r: Rendering, entity: HttpEntity) = def renderEntityContentType(r: Rendering, entity: HttpEntity) =

View file

@ -116,7 +116,7 @@ private[http] object HttpServerBluePrint {
.flatten(FlattenStrategy.concat) .flatten(FlattenStrategy.concat)
.via(Flow[ResponseRenderingOutput].transform(() errorLogger(log, "Outgoing response stream error")).named("errorLogger")) .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) (requestParsing, renderer, oneHundreds)
import FlowGraph.Implicits._ import FlowGraph.Implicits._
@ -172,7 +172,7 @@ private[http] object HttpServerBluePrint {
wrapTls.outlet, wrapTls.outlet,
unwrapTls.inlet, unwrapTls.inlet,
requestsIn) requestsIn)
} })
} }
class BypassMerge(settings: ServerSettings, log: LoggingAdapter) class BypassMerge(settings: ServerSettings, log: LoggingAdapter)
@ -319,7 +319,7 @@ private[http] object HttpServerBluePrint {
val sink = StreamUtils.oneTimePublisherSink[FrameEvent](sinkCell, "frameHandler.in") val sink = StreamUtils.oneTimePublisherSink[FrameEvent](sinkCell, "frameHandler.in")
val source = StreamUtils.oneTimeSubscriberSource[FrameEvent](sourceCell, "frameHandler.out") 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 { new WebsocketSetup {
def websocketFlow: Flow[ByteString, ByteString, Any] = flow def websocketFlow: Flow[ByteString, ByteString, Any] = flow

View file

@ -16,7 +16,7 @@ import akka.stream.stage.{ SyncDirective, Context, StatefulStage }
*/ */
private[http] object Masking { private[http] object Masking {
def apply(serverSide: Boolean, maskRandom: () Random): BidiFlow[ /* net in */ FrameEvent, /* app out */ FrameEvent, /* app in */ FrameEvent, /* net out */ FrameEvent, Unit] = 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] = def maskIf(condition: Boolean, maskRandom: () Random): Flow[FrameEvent, FrameEvent, Unit] =
if (condition) Flow[FrameEvent].transform(() new Masking(maskRandom())) // new random per materialization if (condition) Flow[FrameEvent].transform(() new Masking(maskRandom())) // new random per materialization

View file

@ -39,7 +39,7 @@ private[http] object Websocket {
/** The lowest layer that implements the binary protocol */ /** The lowest layer that implements the binary protocol */
def framing: BidiFlow[ByteString, FrameEvent, FrameEvent, ByteString, Unit] = def framing: BidiFlow[ByteString, FrameEvent, FrameEvent, ByteString, Unit] =
BidiFlow.wrap( BidiFlow.fromFlowsMat(
Flow[ByteString].transform(() new FrameEventParser), Flow[ByteString].transform(() new FrameEventParser),
Flow[FrameEvent].transform(() new FrameEventRenderer))(Keep.none) Flow[FrameEvent].transform(() new FrameEventRenderer))(Keep.none)
.named("ws-framing") .named("ws-framing")
@ -56,7 +56,7 @@ private[http] object Websocket {
def frameHandling(serverSide: Boolean = true, def frameHandling(serverSide: Boolean = true,
closeTimeout: FiniteDuration, closeTimeout: FiniteDuration,
log: LoggingAdapter): BidiFlow[FrameEvent, FrameHandler.Output, FrameOutHandler.Input, FrameStart, Unit] = log: LoggingAdapter): BidiFlow[FrameEvent, FrameHandler.Output, FrameOutHandler.Input, FrameStart, Unit] =
BidiFlow.wrap( BidiFlow.fromFlowsMat(
FrameHandler.create(server = serverSide), FrameHandler.create(server = serverSide),
FrameOutHandler.create(serverSide, closeTimeout, log))(Keep.none) FrameOutHandler.create(serverSide, closeTimeout, log))(Keep.none)
.named("ws-frame-handling") .named("ws-frame-handling")
@ -121,7 +121,7 @@ private[http] object Websocket {
MessageToFrameRenderer.create(serverSide) MessageToFrameRenderer.create(serverSide)
.named("ws-render-messages") .named("ws-render-messages")
BidiFlow() { implicit b BidiFlow.fromGraph(FlowGraph.create() { implicit b
import FlowGraph.Implicits._ import FlowGraph.Implicits._
val split = b.add(BypassRouter) val split = b.add(BypassRouter)
@ -146,7 +146,7 @@ private[http] object Websocket {
messagePreparation.outlet, messagePreparation.outlet,
messageRendering.inlet, messageRendering.inlet,
merge.out) merge.out)
}.named("ws-message-api") }.named("ws-message-api"))
} }
private object BypassRouter extends GraphStage[FanOutShape2[Output, BypassEvent, MessagePart]] { private object BypassRouter extends GraphStage[FanOutShape2[Output, BypassEvent, MessagePart]] {

View file

@ -109,7 +109,7 @@ object WebsocketClientBlueprint {
} }
} }
BidiFlow() { implicit b BidiFlow.fromGraph(FlowGraph.create() { implicit b
import FlowGraph.Implicits._ import FlowGraph.Implicits._
val networkIn = b.add(Flow[ByteString].transform(() new UpgradeStage)) val networkIn = b.add(Flow[ByteString].transform(() new UpgradeStage))
@ -126,11 +126,11 @@ object WebsocketClientBlueprint {
networkIn.outlet, networkIn.outlet,
wsIn.inlet, wsIn.inlet,
httpRequestBytesAndThenWSBytes.out) httpRequestBytesAndThenWSBytes.out)
} mapMaterializedValue (_ result.future) }) mapMaterializedValue (_ result.future)
} }
def simpleTls: BidiFlow[SslTlsInbound, ByteString, ByteString, SendBytes, Unit] = def simpleTls: BidiFlow[SslTlsInbound, ByteString, ByteString, SendBytes, Unit] =
BidiFlow.wrap( BidiFlow.fromFlowsMat(
Flow[SslTlsInbound].collect { case SessionBytes(_, bytes) bytes }, Flow[SslTlsInbound].collect { case SessionBytes(_, bytes) bytes },
Flow[ByteString].map(SendBytes))(Keep.none) Flow[ByteString].map(SendBytes))(Keep.none)
} }

View file

@ -113,7 +113,7 @@ private[http] object JavaMapping {
def toScala(javaObject: javadsl.Flow[JIn, JOut, M]): S = def toScala(javaObject: javadsl.Flow[JIn, JOut, M]): S =
scaladsl.Flow[SIn].map(inMapping.toJava(_)).viaMat(javaObject)(scaladsl.Keep.right).map(outMapping.toScala(_)) scaladsl.Flow[SIn].map(inMapping.toJava(_)).viaMat(javaObject)(scaladsl.Keep.right).map(outMapping.toScala(_))
def toJava(scalaObject: scaladsl.Flow[SIn, SOut, M]): J = 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(_)) 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] = def javaToScalaAdapterFlow[J, S](implicit mapping: JavaMapping[J, S]): scaladsl.Flow[J, S, Unit] =
scaladsl.Flow[J].map(mapping.toScala(_)) 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] = 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)] = 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)] { new JavaMapping[Pair[J1, J2], (S1, S2)] {

View file

@ -86,7 +86,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
* [[ServerBinding]]. * [[ServerBinding]].
*/ */
def bind(interface: String, port: Int, materializer: Materializer): Source[IncomingConnection, Future[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(_)) .map(new IncomingConnection(_))
.mapMaterializedValue(_.map(new ServerBinding(_))(ec))) .mapMaterializedValue(_.map(new ServerBinding(_))(ec)))
@ -105,7 +105,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
httpsContext: Option[HttpsContext], httpsContext: Option[HttpsContext],
log: LoggingAdapter, log: LoggingAdapter,
materializer: Materializer): Source[IncomingConnection, Future[ServerBinding]] = 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(_)) .map(new IncomingConnection(_))
.mapMaterializedValue(_.map(new ServerBinding(_))(ec))) .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. * 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]] = def outgoingConnection(host: String, port: Int): Flow[HttpRequest, HttpResponse, Future[OutgoingConnection]] =
Flow.wrap { Flow.fromGraph {
akka.stream.scaladsl.Flow[HttpRequest].map(_.asScala) akka.stream.scaladsl.Flow[HttpRequest].map(_.asScala)
.viaMat(delegate.outgoingConnection(host, port))(Keep.right) .viaMat(delegate.outgoingConnection(host, port))(Keep.right)
.mapMaterializedValue(_.map(new OutgoingConnection(_))(ec)) .mapMaterializedValue(_.map(new OutgoingConnection(_))(ec))
@ -248,7 +248,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
* Same as [[outgoingConnection]] but with HTTPS encryption. * Same as [[outgoingConnection]] but with HTTPS encryption.
*/ */
def outgoingConnectionTls(host: String, port: Int): Flow[HttpRequest, HttpResponse, Future[OutgoingConnection]] = def outgoingConnectionTls(host: String, port: Int): Flow[HttpRequest, HttpResponse, Future[OutgoingConnection]] =
Flow.wrap { Flow.fromGraph {
akka.stream.scaladsl.Flow[HttpRequest].map(_.asScala) akka.stream.scaladsl.Flow[HttpRequest].map(_.asScala)
.viaMat(delegate.outgoingConnectionTls(host, port))(Keep.right) .viaMat(delegate.outgoingConnectionTls(host, port))(Keep.right)
.mapMaterializedValue(_.map(new OutgoingConnection(_))(ec)) .mapMaterializedValue(_.map(new OutgoingConnection(_))(ec))
@ -262,7 +262,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
localAddress: Option[InetSocketAddress], localAddress: Option[InetSocketAddress],
settings: ClientConnectionSettings, settings: ClientConnectionSettings,
log: LoggingAdapter): Flow[HttpRequest, HttpResponse, Future[OutgoingConnection]] = log: LoggingAdapter): Flow[HttpRequest, HttpResponse, Future[OutgoingConnection]] =
Flow.wrap { Flow.fromGraph {
akka.stream.scaladsl.Flow[HttpRequest].map(_.asScala) akka.stream.scaladsl.Flow[HttpRequest].map(_.asScala)
.viaMat(delegate.outgoingConnection(host, port, localAddress.asScala, settings, log))(Keep.right) .viaMat(delegate.outgoingConnection(host, port, localAddress.asScala, settings, log))(Keep.right)
.mapMaterializedValue(_.map(new OutgoingConnection(_))(ec)) .mapMaterializedValue(_.map(new OutgoingConnection(_))(ec))
@ -279,7 +279,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
settings: ClientConnectionSettings, settings: ClientConnectionSettings,
httpsContext: Option[HttpsContext], httpsContext: Option[HttpsContext],
log: LoggingAdapter): Flow[HttpRequest, HttpResponse, Future[OutgoingConnection]] = log: LoggingAdapter): Flow[HttpRequest, HttpResponse, Future[OutgoingConnection]] =
Flow.wrap { Flow.fromGraph {
akka.stream.scaladsl.Flow[HttpRequest].map(_.asScala) akka.stream.scaladsl.Flow[HttpRequest].map(_.asScala)
.viaMat(delegate.outgoingConnectionTls(host, port, localAddress.asScala, settings, .viaMat(delegate.outgoingConnectionTls(host, port, localAddress.asScala, settings,
httpsContext.map(_.asInstanceOf[akka.http.scaladsl.HttpsContext]), log))(Keep.right) 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))) .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]] = 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] = 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] stream.scaladsl.Flow[scaladsl.model.ws.Message]

View file

@ -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. * 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 * Handles the connection with the given flow, which is materialized exactly once

View file

@ -38,18 +38,18 @@ object WebsocketUpgradeResponse {
import akka.http.impl.util.JavaMapping.Implicits._ import akka.http.impl.util.JavaMapping.Implicits._
def adapt(scalaResponse: scaladsl.model.ws.WebsocketUpgradeResponse): WebsocketUpgradeResponse = def adapt(scalaResponse: scaladsl.model.ws.WebsocketUpgradeResponse): WebsocketUpgradeResponse =
scalaResponse match { scalaResponse match {
case ValidUpgrade(response, chosen) case ValidUpgrade(resp, chosen)
new WebsocketUpgradeResponse { new WebsocketUpgradeResponse {
def isValid: Boolean = true def isValid: Boolean = true
def response: HttpResponse = scalaResponse.response def response: HttpResponse = resp
def chosenSubprotocol: Option[String] = chosen.asJava def chosenSubprotocol: Option[String] = chosen.asJava
def invalidationReason: String = def invalidationReason: String =
throw new UnsupportedOperationException("invalidationReason must not be called for valid response") throw new UnsupportedOperationException("invalidationReason must not be called for valid response")
} }
case InvalidUpgradeResponse(response, cause) case InvalidUpgradeResponse(resp, cause)
new WebsocketUpgradeResponse { new WebsocketUpgradeResponse {
def isValid: Boolean = false 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 chosenSubprotocol: Option[String] = throw new UnsupportedOperationException("chosenSubprotocol must not be called for valid response")
def invalidationReason: String = cause def invalidationReason: String = cause
} }

View file

@ -75,7 +75,7 @@ sealed trait HttpEntity extends jm.HttpEntity {
def withContentType(contentType: ContentType): HttpEntity def withContentType(contentType: ContentType): HttpEntity
/** Java API */ /** 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 */ /** Java API */
def getContentLengthOption: japi.Option[JLong] = def getContentLengthOption: japi.Option[JLong] =
@ -296,7 +296,7 @@ object HttpEntity {
override def productPrefix = "HttpEntity.Chunked" override def productPrefix = "HttpEntity.Chunked"
/** Java API */ /** 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 { object Chunked {
/** /**

View file

@ -52,7 +52,7 @@ trait UpgradeToWebsocket extends jm.ws.UpgradeToWebsocket {
def handleMessagesWithSinkSource(inSink: Sink[Message, Any], def handleMessagesWithSinkSource(inSink: Sink[Message, Any],
outSource: Source[Message, Any], outSource: Source[Message, Any],
subprotocol: Option[String] = None): HttpResponse = subprotocol: Option[String] = None): HttpResponse =
handleMessages(Flow.wrap(inSink, outSource)((_, _) ()), subprotocol) handleMessages(Flow.fromSinkAndSource(inSink, outSource), subprotocol)
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._
@ -88,5 +88,5 @@ trait UpgradeToWebsocket extends jm.ws.UpgradeToWebsocket {
handleMessages(createScalaFlow(inSink, outSource), subprotocol = Some(subprotocol)) 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] = 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)
} }

View file

@ -63,7 +63,7 @@ public class WSEchoTestClientApp {
.toMat(Sink.<List<String>>head(), Keep.<BoxedUnit, Future<List<String>>>right()); .toMat(Sink.<List<String>>head(), Keep.<BoxedUnit, Future<List<String>>>right());
Flow<Message, Message, Future<List<String>>> echoClient = Flow<Message, Message, Future<List<String>>> echoClient =
Flow.wrap(echoSink, echoSource, Keep.<Future<List<String>>, BoxedUnit>left()); Flow.fromSinkAndSourceMat(echoSink, echoSource, Keep.<Future<List<String>>, BoxedUnit>left());
Future<List<String>> result = Future<List<String>> result =
Http.get(system).singleWebsocketRequest( Http.get(system).singleWebsocketRequest(

View file

@ -298,12 +298,10 @@ class ConnectionPoolSpec extends AkkaSpec("""
val incomingConnectionCounter = new AtomicInteger val incomingConnectionCounter = new AtomicInteger
val incomingConnections = TestSubscriber.manualProbe[Http.IncomingConnection] val incomingConnections = TestSubscriber.manualProbe[Http.IncomingConnection]
val incomingConnectionsSub = { val incomingConnectionsSub = {
val rawBytesInjection = BidiFlow() { b val rawBytesInjection = BidiFlow.fromFlows(
val top = b.add(Flow[SslTlsOutbound].collect[ByteString] { case SendBytes(x) mapServerSideOutboundRawBytes(x) } Flow[SslTlsOutbound].collect[ByteString] { case SendBytes(x) mapServerSideOutboundRawBytes(x) }
.transform(StreamUtils.recover { case NoErrorComplete ByteString.empty })) .transform(StreamUtils.recover { case NoErrorComplete ByteString.empty }),
val bottom = b.add(Flow[ByteString].map(SessionBytes(null, _))) Flow[ByteString].map(SessionBytes(null, _)))
BidiShape(top.inlet, top.outlet, bottom.inlet, bottom.outlet)
}
val sink = if (autoAccept) Sink.foreach[Http.IncomingConnection](handleConnection) else Sink(incomingConnections) val sink = if (autoAccept) Sink.foreach[Http.IncomingConnection](handleConnection) else Sink(incomingConnections)
// TODO getHostString in Java7 // TODO getHostString in Java7
Tcp().bind(serverEndpoint.getHostName, serverEndpoint.getPort, idleTimeout = serverSettings.timeouts.idleTimeout) Tcp().bind(serverEndpoint.getHostName, serverEndpoint.getPort, idleTimeout = serverSettings.timeouts.idleTimeout)

View file

@ -6,7 +6,7 @@ package akka.http.impl.engine.client
import scala.concurrent.Await import scala.concurrent.Await
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.stream.ActorMaterializer import akka.stream.{ FlowShape, ActorMaterializer }
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
import akka.http.scaladsl.{ Http, TestUtils } import akka.http.scaladsl.{ Http, TestUtils }
@ -44,7 +44,7 @@ class HighLevelOutgoingConnectionSpec extends AkkaSpec {
val connFlow = Http().outgoingConnection(serverHostName, serverPort) val connFlow = Http().outgoingConnection(serverHostName, serverPort)
val C = 4 val C = 4
val doubleConnection = Flow() { implicit b val doubleConnection = Flow.fromGraph(FlowGraph.create() { implicit b
import FlowGraph.Implicits._ import FlowGraph.Implicits._
val bcast = b.add(Broadcast[HttpRequest](C)) val bcast = b.add(Broadcast[HttpRequest](C))
@ -52,8 +52,8 @@ class HighLevelOutgoingConnectionSpec extends AkkaSpec {
for (i 0 until C) for (i 0 until C)
bcast.out(i) ~> connFlow ~> merge.in(i) bcast.out(i) ~> connFlow ~> merge.in(i)
(bcast.in, merge.out) FlowShape(bcast.in, merge.out)
} })
val N = 100 val N = 100
val result = Source(() Iterator.from(1)) val result = Source(() Iterator.from(1))

View file

@ -9,7 +9,7 @@ import akka.stream.io.{ SessionBytes, SslTlsOutbound, SendBytes }
import org.scalatest.Inside import org.scalatest.Inside
import akka.util.ByteString import akka.util.ByteString
import akka.event.NoLogging import akka.event.NoLogging
import akka.stream.ActorMaterializer import akka.stream.{ ClosedShape, ActorMaterializer }
import akka.stream.testkit._ import akka.stream.testkit._
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.http.scaladsl.model.HttpEntity._ import akka.http.scaladsl.model.HttpEntity._
@ -359,14 +359,15 @@ class LowLevelOutgoingConnectionSpec extends AkkaSpec("akka.loggers = []\n akka.
val netOut = TestSubscriber.manualProbe[ByteString] val netOut = TestSubscriber.manualProbe[ByteString]
val netIn = TestPublisher.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 client
import FlowGraph.Implicits._ import FlowGraph.Implicits._
Source(netIn) ~> Flow[ByteString].map(SessionBytes(null, _)) ~> client.in2 Source(netIn) ~> Flow[ByteString].map(SessionBytes(null, _)) ~> client.in2
client.out1 ~> Flow[SslTlsOutbound].collect { case SendBytes(x) x } ~> Sink(netOut) client.out1 ~> Flow[SslTlsOutbound].collect { case SendBytes(x) x } ~> Sink(netOut)
Source(requests) ~> client.in1 Source(requests) ~> client.in1
client.out2 ~> Sink(responses) client.out2 ~> Sink(responses)
}.run() ClosedShape
}).run()
netOut -> netIn netOut -> netIn
} }

View file

@ -15,8 +15,8 @@ import akka.actor.ActorSystem
import akka.event.NoLogging import akka.event.NoLogging
import akka.util.ByteString import akka.util.ByteString
import akka.stream.Materializer import akka.stream.{ ClosedShape, Materializer }
import akka.stream.scaladsl.{ Flow, Sink, Source, FlowGraph } import akka.stream.scaladsl._
import akka.stream.testkit.{ TestPublisher, TestSubscriber } import akka.stream.testkit.{ TestPublisher, TestSubscriber }
import akka.http.impl.util._ import akka.http.impl.util._
@ -39,14 +39,15 @@ abstract class HttpServerTestSetupBase {
val netIn = TestPublisher.probe[ByteString]() val netIn = TestPublisher.probe[ByteString]()
val netOut = ByteStringSinkProbe() 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 server
import FlowGraph.Implicits._ import FlowGraph.Implicits._
Source(netIn) ~> Flow[ByteString].map(SessionBytes(null, _)) ~> server.in2 Source(netIn) ~> Flow[ByteString].map(SessionBytes(null, _)) ~> server.in2
server.out1 ~> Flow[SslTlsOutbound].collect { case SendBytes(x) x } ~> netOut.sink server.out1 ~> Flow[SslTlsOutbound].collect { case SendBytes(x) x } ~> netOut.sink
server.out2 ~> Sink(requests) server.out2 ~> Sink(requests)
Source(responses) ~> server.in1 Source(responses) ~> server.in1
}.run() ClosedShape
}).run()
netIn -> netOut netIn -> netOut
} }

View file

@ -50,7 +50,7 @@ object EchoTestClientApp extends App {
.grouped(10000) .grouped(10000)
.toMat(Sink.head)(Keep.right) .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) val (upgrade, res) = Http().singleWebsocketRequest("wss://echo.websocket.org", echoClient)
res onComplete { res onComplete {

View file

@ -817,8 +817,8 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
val messageOut = TestPublisher.probe[Message]() val messageOut = TestPublisher.probe[Message]()
val messageHandler: Flow[Message, Message, Unit] = val messageHandler: Flow[Message, Message, Unit] =
Flow.wrap { Flow.fromGraph {
FlowGraph.partial() { implicit b FlowGraph.create() { implicit b
val in = b.add(Sink(messageIn)) val in = b.add(Sink(messageIn))
val out = b.add(Source(messageOut)) val out = b.add(Source(messageOut))

View file

@ -199,7 +199,7 @@ object WSClientAutobahnTest extends App {
*/ */
def runToSingleText(uri: Uri): Future[String] = { def runToSingleText(uri: Uri): Future[String] = {
val sink = Sink.head[Message] 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("")(_ + _)) case tm: TextMessage tm.textStream.runWith(Sink.fold("")(_ + _))
} }
} }

View file

@ -7,6 +7,7 @@ package akka.http.impl.engine.ws
import java.util.Random import java.util.Random
import akka.http.scaladsl.model.ws.{ InvalidUpgradeResponse, WebsocketUpgradeResponse } import akka.http.scaladsl.model.ws.{ InvalidUpgradeResponse, WebsocketUpgradeResponse }
import akka.stream.ClosedShape
import scala.concurrent.duration._ 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 { "don't send out frames before handshake was finished successfully" in new TestSetup {
def clientImplementation: Flow[Message, Message, Unit] = 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) expectWireData(UpgradeRequestBytes)
expectNoWireData() expectNoWireData()
@ -311,13 +312,14 @@ class WebsocketClientSpec extends FreeSpec with Matchers with WithMaterializerSp
val netIn = TestPublisher.probe[ByteString]() val netIn = TestPublisher.probe[ByteString]()
val graph = val graph =
FlowGraph.closed(clientLayer) { implicit b RunnableGraph.fromGraph(FlowGraph.create(clientLayer) { implicit b
client client
import FlowGraph.Implicits._ import FlowGraph.Implicits._
Source(netIn) ~> Flow[ByteString].map(SessionBytes(null, _)) ~> client.in2 Source(netIn) ~> Flow[ByteString].map(SessionBytes(null, _)) ~> client.in2
client.out1 ~> Flow[SslTlsOutbound].collect { case SendBytes(x) x } ~> netOut.sink client.out1 ~> Flow[SslTlsOutbound].collect { case SendBytes(x) x } ~> netOut.sink
client.out2 ~> clientImplementation ~> client.in1 client.out2 ~> clientImplementation ~> client.in1
} ClosedShape
})
val response = graph.run() val response = graph.run()
@ -365,6 +367,6 @@ class WebsocketClientSpec extends FreeSpec with Matchers with WithMaterializerSp
lazy val messagesIn = TestSubscriber.probe[Message]() lazy val messagesIn = TestSubscriber.probe[Message]()
override def clientImplementation: Flow[Message, Message, Unit] = override def clientImplementation: Flow[Message, Message, Unit] =
Flow.wrap(Sink(messagesIn), Source(messagesOut))(Keep.none) Flow.fromSinkAndSourceMat(Sink(messagesIn), Source(messagesOut))(Keep.none)
} }
} }

View file

@ -38,7 +38,7 @@ class WebsocketServerSpec extends FreeSpec with Matchers with WithMaterializerSp
val source = val source =
Source(List(1, 2, 3, 4, 5)).map(num TextMessage.Strict(s"Message $num")) 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) val response = upgrade.get.handleMessages(handler)
responses.sendNext(response) responses.sendNext(response)

View file

@ -330,10 +330,7 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
val sink = Sink.publisher[HttpRequest] val sink = Sink.publisher[HttpRequest]
val source = Source.subscriber[HttpResponse] val source = Source.subscriber[HttpResponse]
val handler = Flow(sink, source)(Keep.both) { implicit b val handler = Flow.fromSinkAndSourceMat(sink, source)(Keep.both)
(snk, src)
(snk.inlet, src.outlet)
}
val (pub, sub) = incomingConnection.handleWith(handler) val (pub, sub) = incomingConnection.handleWith(handler)
val requestSubscriberProbe = TestSubscriber.manualProbe[HttpRequest]() val requestSubscriberProbe = TestSubscriber.manualProbe[HttpRequest]()

View file

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

View file

@ -1,18 +0,0 @@
/**
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
*/
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
}

View file

@ -0,0 +1,20 @@
/**
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
*/
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
}

View file

@ -1,6 +1,6 @@
package akka.stream.testkit package akka.stream.testkit
import akka.stream.{ ActorMaterializer, ActorMaterializerSettings, Inlet, Outlet } import akka.stream._
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import org.reactivestreams.Publisher import org.reactivestreams.Publisher
import scala.collection.immutable import scala.collection.immutable
@ -19,15 +19,15 @@ abstract class TwoStreamsSetup extends BaseTwoStreamsSetup {
override def setup(p1: Publisher[Int], p2: Publisher[Int]) = { override def setup(p1: Publisher[Int], p2: Publisher[Int]) = {
val subscriber = TestSubscriber.probe[Outputs]() val subscriber = TestSubscriber.probe[Outputs]()
FlowGraph.closed() { implicit b RunnableGraph.fromGraph(FlowGraph.create() { implicit b
import FlowGraph.Implicits._ import FlowGraph.Implicits._
val f = fixture(b) val f = fixture(b)
Source(p1) ~> f.left Source(p1) ~> f.left
Source(p2) ~> f.right Source(p2) ~> f.right
f.out ~> Sink(subscriber) f.out ~> Sink(subscriber)
ClosedShape
}.run() }).run()
subscriber subscriber
} }

View file

@ -35,84 +35,83 @@ public class BidiFlowTest extends StreamTest {
"FlowTest", AkkaSpec.testConf()); "FlowTest", AkkaSpec.testConf());
private final BidiFlow<Integer, Long, ByteString, String, BoxedUnit> bidi = BidiFlow private final BidiFlow<Integer, Long, ByteString, String, BoxedUnit> bidi = BidiFlow
.factory() .fromGraph(FlowGraph.create(
.create( new Function<FlowGraph.Builder<BoxedUnit>, BidiShape<Integer, Long, ByteString, String>>() {
new Function<FlowGraph.Builder<BoxedUnit>, BidiShape<Integer, Long, ByteString, String>>() { @Override
@Override public BidiShape<Integer, Long, ByteString, String> apply(Builder<BoxedUnit> b)
public BidiShape<Integer, Long, ByteString, String> apply(Builder<BoxedUnit> b) throws Exception {
throws Exception { final FlowShape<Integer, Long> top = b.add(Flow
final FlowShape<Integer, Long> top = b.graph(Flow .of(Integer.class).map(new Function<Integer, Long>() {
.<Integer> empty().map(new Function<Integer, Long>() { @Override
@Override public Long apply(Integer arg) {
public Long apply(Integer arg) { return (long) ((int) arg) + 2;
return (long) ((int) arg) + 2; }
} }));
})); final FlowShape<ByteString, String> bottom = b.add(Flow
final FlowShape<ByteString, String> bottom = b.graph(Flow .of(ByteString.class).map(new Function<ByteString, String>() {
.<ByteString> empty().map(new Function<ByteString, String>() { @Override
@Override public String apply(ByteString arg) {
public String apply(ByteString arg) { return arg.decodeString("UTF-8");
return arg.decodeString("UTF-8"); }
} }));
})); return new BidiShape<Integer, Long, ByteString, String>(top
return new BidiShape<Integer, Long, ByteString, String>(top .inlet(), top.outlet(), bottom.inlet(), bottom.outlet());
.inlet(), top.outlet(), bottom.inlet(), bottom.outlet()); }
} }));
});
private final BidiFlow<Long, Integer, String, ByteString, BoxedUnit> inverse = BidiFlow private final BidiFlow<Long, Integer, String, ByteString, BoxedUnit> inverse = BidiFlow
.factory() .fromGraph(
.create( FlowGraph.create(
new Function<FlowGraph.Builder<BoxedUnit>, BidiShape<Long, Integer, String, ByteString>>() { new Function<FlowGraph.Builder<BoxedUnit>, BidiShape<Long, Integer, String, ByteString>>() {
@Override @Override
public BidiShape<Long, Integer, String, ByteString> apply(Builder<BoxedUnit> b) public BidiShape<Long, Integer, String, ByteString> apply(Builder<BoxedUnit> b)
throws Exception { throws Exception {
final FlowShape<Long, Integer> top = b.graph(Flow.<Long> empty() final FlowShape<Long, Integer> top = b.add(Flow.of(Long.class)
.map(new Function<Long, Integer>() { .map(new Function<Long, Integer>() {
@Override @Override
public Integer apply(Long arg) { public Integer apply(Long arg) {
return (int) ((long) arg) + 2; return (int) ((long) arg) + 2;
} }
})); }));
final FlowShape<String, ByteString> bottom = b.graph(Flow final FlowShape<String, ByteString> bottom = b.add(Flow
.<String> empty().map(new Function<String, ByteString>() { .of(String.class).map(new Function<String, ByteString>() {
@Override @Override
public ByteString apply(String arg) { public ByteString apply(String arg) {
return ByteString.fromString(arg); return ByteString.fromString(arg);
} }
})); }));
return new BidiShape<Long, Integer, String, ByteString>(top return new BidiShape<Long, Integer, String, ByteString>(top
.inlet(), top.outlet(), bottom.inlet(), bottom.outlet()); .inlet(), top.outlet(), bottom.inlet(), bottom.outlet());
} }
}); }));
private final BidiFlow<Integer, Long, ByteString, String, Future<Integer>> bidiMat = BidiFlow private final BidiFlow<Integer, Long, ByteString, String, Future<Integer>> bidiMat =
.factory() BidiFlow.fromGraph(
.create( FlowGraph.create(
Sink.<Integer> head(), Sink.<Integer>head(),
new Function2<FlowGraph.Builder<Future<Integer>>, SinkShape<Integer>, BidiShape<Integer, Long, ByteString, String>>() { new Function2<FlowGraph.Builder<Future<Integer>>, SinkShape<Integer>, BidiShape<Integer, Long, ByteString, String>>() {
@Override @Override
public BidiShape<Integer, Long, ByteString, String> apply(Builder<Future<Integer>> b, SinkShape<Integer> sink) public BidiShape<Integer, Long, ByteString, String> apply(Builder<Future<Integer>> b, SinkShape<Integer> sink)
throws Exception { throws Exception {
b.from(b.graph(Source.single(42))).to(sink); b.from(b.add(Source.single(42))).to(sink);
final FlowShape<Integer, Long> top = b.graph(Flow final FlowShape<Integer, Long> top = b.add(Flow
.<Integer> empty().map(new Function<Integer, Long>() { .of(Integer.class).map(new Function<Integer, Long>() {
@Override @Override
public Long apply(Integer arg) { public Long apply(Integer arg) {
return (long) ((int) arg) + 2; return (long) ((int) arg) + 2;
} }
})); }));
final FlowShape<ByteString, String> bottom = b.graph(Flow final FlowShape<ByteString, String> bottom = b.add(Flow
.<ByteString> empty().map(new Function<ByteString, String>() { .of(ByteString.class).map(new Function<ByteString, String>() {
@Override @Override
public String apply(ByteString arg) { public String apply(ByteString arg) {
return arg.decodeString("UTF-8"); return arg.decodeString("UTF-8");
} }
})); }));
return new BidiShape<Integer, Long, ByteString, String>(top return new BidiShape<Integer, Long, ByteString, String>(top
.inlet(), top.outlet(), bottom.inlet(), bottom.outlet()); .inlet(), top.outlet(), bottom.inlet(), bottom.outlet());
} }
}); }));
private final String str = "Hello World"; private final String str = "Hello World";
private final ByteString bytes = ByteString.fromString(str); private final ByteString bytes = ByteString.fromString(str);
@ -126,22 +125,23 @@ public class BidiFlowTest extends StreamTest {
@Test @Test
public void mustWorkInIsolation() throws Exception { public void mustWorkInIsolation() throws Exception {
final Pair<Future<Long>, Future<String>> p = FlowGraph final Pair<Future<Long>, Future<String>> p =
.factory() RunnableGraph.fromGraph(FlowGraph
.closed(Sink.<Long> head(), Sink.<String> head(), .create(Sink.<Long> head(), Sink.<String> head(),
Keep.<Future<Long>, Future<String>> both(), Keep.<Future<Long>, Future<String>> both(),
new Procedure3<Builder<Pair<Future<Long>, Future<String>>>, SinkShape<Long>, SinkShape<String>>() { new Function3<Builder<Pair<Future<Long>, Future<String>>>, SinkShape<Long>, SinkShape<String>, ClosedShape>() {
@Override @Override
public void apply(Builder<Pair<Future<Long>, Future<String>>> b, SinkShape<Long> st, public ClosedShape apply(Builder<Pair<Future<Long>, Future<String>>> b, SinkShape<Long> st,
SinkShape<String> sb) throws Exception { SinkShape<String> sb) throws Exception {
final BidiShape<Integer, Long, ByteString, String> s = b final BidiShape<Integer, Long, ByteString, String> s =
.graph(bidi); b.add(bidi);
b.from(b.graph(Source.single(1))).toInlet(s.in1()); b.from(b.add(Source.single(1))).toInlet(s.in1());
b.from(s.out1()).to(st); b.from(s.out1()).to(st);
b.from(b.graph(Source.single(bytes))).toInlet(s.in2()); b.from(b.add(Source.single(bytes))).toInlet(s.in2());
b.from(s.out2()).to(sb); b.from(s.out2()).to(sb);
} return ClosedShape.getInstance();
}).run(materializer); }
})).run(materializer);
final Long rt = Await.result(p.first(), oneSec); final Long rt = Await.result(p.first(), oneSec);
final String rb = Await.result(p.second(), oneSec); final String rb = Await.result(p.second(), oneSec);
@ -152,7 +152,7 @@ public class BidiFlowTest extends StreamTest {
@Test @Test
public void mustWorkAsAFlowThatIsOpenOnTheLeft() throws Exception { public void mustWorkAsAFlowThatIsOpenOnTheLeft() throws Exception {
final Flow<Integer, String, BoxedUnit> f = bidi.join(Flow.<Long> empty().map( final Flow<Integer, String, BoxedUnit> f = bidi.join(Flow.of(Long.class).map(
new Function<Long, ByteString>() { new Function<Long, ByteString>() {
@Override public ByteString apply(Long arg) { @Override public ByteString apply(Long arg) {
return ByteString.fromString("Hello " + arg); return ByteString.fromString("Hello " + arg);
@ -164,7 +164,7 @@ public class BidiFlowTest extends StreamTest {
@Test @Test
public void mustWorkAsAFlowThatIsOpenOnTheRight() throws Exception { public void mustWorkAsAFlowThatIsOpenOnTheRight() throws Exception {
final Flow<ByteString, Long, BoxedUnit> f = Flow.<String> empty().map( final Flow<ByteString, Long, BoxedUnit> f = Flow.of(String.class).map(
new Function<String, Integer>() { new Function<String, Integer>() {
@Override public Integer apply(String arg) { @Override public Integer apply(String arg) {
return Integer.valueOf(arg); return Integer.valueOf(arg);
@ -177,7 +177,7 @@ public class BidiFlowTest extends StreamTest {
@Test @Test
public void mustWorkWhenAtopItsInverse() throws Exception { public void mustWorkWhenAtopItsInverse() throws Exception {
final Flow<Integer,String,BoxedUnit> f = bidi.atop(inverse).join(Flow.<Integer> empty().map( final Flow<Integer,String,BoxedUnit> f = bidi.atop(inverse).join(Flow.of(Integer.class).map(
new Function<Integer, String>() { new Function<Integer, String>() {
@Override public String apply(Integer arg) { @Override public String apply(Integer arg) {
return arg.toString(); return arg.toString();
@ -189,7 +189,7 @@ public class BidiFlowTest extends StreamTest {
@Test @Test
public void mustWorkWhenReversed() throws Exception { public void mustWorkWhenReversed() throws Exception {
final Flow<Integer,String,BoxedUnit> f = Flow.<Integer> empty().map( final Flow<Integer,String,BoxedUnit> f = Flow.of(Integer.class).map(
new Function<Integer, String>() { new Function<Integer, String>() {
@Override public String apply(Integer arg) { @Override public String apply(Integer arg) {
return arg.toString(); return arg.toString();
@ -201,61 +201,66 @@ public class BidiFlowTest extends StreamTest {
@Test @Test
public void mustMaterializeToItsValue() throws Exception { public void mustMaterializeToItsValue() throws Exception {
final Future<Integer> f = FlowGraph.factory().closed(bidiMat, new Procedure2<Builder<Future<Integer> >, BidiShape<Integer, Long, ByteString, String>>() { final Future<Integer> f = RunnableGraph.fromGraph(
FlowGraph.create(bidiMat,
new Function2<Builder<Future<Integer> >, BidiShape<Integer, Long, ByteString, String>, ClosedShape>() {
@Override @Override
public void apply(Builder<Future<Integer>> b, public ClosedShape apply(Builder<Future<Integer>> b,
BidiShape<Integer, Long, ByteString, String> shape) throws Exception { BidiShape<Integer, Long, ByteString, String> shape) throws Exception {
final FlowShape<String, Integer> left = b.graph(Flow.<String> empty().map( final FlowShape<String, Integer> left = b.add(Flow.of(String.class).map(
new Function<String, Integer>() { new Function<String, Integer>() {
@Override public Integer apply(String arg) { @Override
return Integer.valueOf(arg); public Integer apply(String arg) {
} return Integer.valueOf(arg);
})); }
final FlowShape<Long, ByteString> right = b.graph(Flow.<Long> empty().map( }));
new Function<Long, ByteString>() { final FlowShape<Long, ByteString> right = b.add(Flow.of(Long.class).map(
@Override public ByteString apply(Long arg) { new Function<Long, ByteString>() {
return ByteString.fromString("Hello " + arg); @Override
} public ByteString apply(Long arg) {
})); return ByteString.fromString("Hello " + arg);
}
}));
b.from(shape.out2()).via(left).toInlet(shape.in1()) b.from(shape.out2()).via(left).toInlet(shape.in1())
.from(shape.out1()).via(right).toInlet(shape.in2()); .from(shape.out1()).via(right).toInlet(shape.in2());
return ClosedShape.getInstance();
} }
}).run(materializer); })).run(materializer);
assertEquals((Integer) 42, Await.result(f, oneSec)); assertEquals((Integer) 42, Await.result(f, oneSec));
} }
@Test @Test
public void mustCombineMaterializationValues() throws Exception { public void mustCombineMaterializationValues() throws Exception {
final Flow<String, Integer, Future<Integer>> left = Flow.factory().create( final Flow<String, Integer, Future<Integer>> left = Flow.fromGraph(FlowGraph.create(
Sink.<Integer> head(), new Function2<Builder<Future<Integer> >, SinkShape<Integer>, Pair<Inlet<String>, Outlet<Integer>>>() { Sink.<Integer>head(), new Function2<Builder<Future<Integer>>, SinkShape<Integer>, FlowShape<String, Integer>>() {
@Override @Override
public Pair<Inlet<String>, Outlet<Integer>> apply(Builder<Future<Integer>> b, public FlowShape<String, Integer> apply(Builder<Future<Integer>> b,
SinkShape<Integer> sink) throws Exception { SinkShape<Integer> sink) throws Exception {
final UniformFanOutShape<Integer, Integer> bcast = b.graph(Broadcast.<Integer> create(2)); final UniformFanOutShape<Integer, Integer> bcast = b.add(Broadcast.<Integer>create(2));
final UniformFanInShape<Integer, Integer> merge = b.graph(Merge.<Integer> create(2)); final UniformFanInShape<Integer, Integer> merge = b.add(Merge.<Integer>create(2));
final FlowShape<String, Integer> flow = b.graph(Flow.<String> empty().map( final FlowShape<String, Integer> flow = b.add(Flow.of(String.class).map(
new Function<String, Integer>() { new Function<String, Integer>() {
@Override @Override
public Integer apply(String arg) { public Integer apply(String arg) {
return Integer.valueOf(arg); return Integer.valueOf(arg);
} }
})); }));
b.from(bcast).to(sink) b.from(bcast).to(sink)
.from(b.graph(Source.single(1))).viaFanOut(bcast).toFanIn(merge) .from(b.add(Source.single(1))).viaFanOut(bcast).toFanIn(merge)
.from(flow).toFanIn(merge); .from(flow).toFanIn(merge);
return new Pair<Inlet<String>, Outlet<Integer>>(flow.inlet(), merge.out()); return new FlowShape<String, Integer>(flow.inlet(), merge.out());
} }
}); }));
final Flow<Long, ByteString, Future<List<Long>>> right = Flow.factory().create( final Flow<Long, ByteString, Future<List<Long>>> right = Flow.fromGraph(FlowGraph.create(
Sink.<List<Long>> head(), new Function2<Builder<Future<List<Long>>>, SinkShape<List<Long>>, Pair<Inlet<Long>, Outlet<ByteString>>>() { Sink.<List<Long>>head(), new Function2<Builder<Future<List<Long>>>, SinkShape<List<Long>>, FlowShape<Long, ByteString>>() {
@Override @Override
public Pair<Inlet<Long>, Outlet<ByteString>> apply(Builder<Future<List<Long>>> b, public FlowShape<Long, ByteString> apply(Builder<Future<List<Long>>> b,
SinkShape<List<Long>> sink) throws Exception { SinkShape<List<Long>> sink) throws Exception {
final FlowShape<Long, List<Long>> flow = b.graph(Flow.<Long> empty().grouped(10)); final FlowShape<Long, List<Long>> flow = b.add(Flow.of(Long.class).grouped(10));
b.from(flow).to(sink); b.from(flow).to(sink);
return new Pair<Inlet<Long>, Outlet<ByteString>>(flow.inlet(), b.source(Source.single(ByteString.fromString("10")))); return new FlowShape<Long, ByteString>(flow.inlet(), b.add(Source.single(ByteString.fromString("10"))).outlet());
} }
}); }));
final Pair<Pair<Future<Integer>, Future<Integer>>, Future<List<Long>>> result = final Pair<Pair<Future<Integer>, Future<Integer>>, Future<List<Long>>> result =
left.joinMat(bidiMat, Keep.<Future<Integer>, Future<Integer>> both()).joinMat(right, Keep.<Pair<Future<Integer>, Future<Integer>>, Future<List<Long>>> both()).run(materializer); left.joinMat(bidiMat, Keep.<Future<Integer>, Future<Integer>> both()).joinMat(right, Keep.<Pair<Future<Integer>, Future<Integer>>, Future<List<Long>>> both()).run(materializer);
final Future<Integer> l = result.first().first(); final Future<Integer> l = result.first().first();

View file

@ -69,15 +69,16 @@ public class FlowGraphTest extends StreamTest {
final Sink<String, Publisher<String>> publisher = Sink.publisher(); final Sink<String, Publisher<String>> publisher = Sink.publisher();
final Source<String, BoxedUnit> source = Source.factory().create(new Function<FlowGraph.Builder<BoxedUnit>, Outlet<String>>() { final Source<String, BoxedUnit> source = Source.fromGraph(
@Override FlowGraph.create(new Function<FlowGraph.Builder<BoxedUnit>, SourceShape<String>>() {
public Outlet<String> apply(Builder<BoxedUnit> b) throws Exception { @Override
final UniformFanInShape<String, String> merge = b.graph(Merge.<String> create(2)); public SourceShape<String> apply(Builder<BoxedUnit> b) throws Exception {
b.flow(b.source(in1), f1, merge.in(0)); final UniformFanInShape<String, String> merge = b.add(Merge.<String>create(2));
b.flow(b.source(in2), f2, merge.in(1)); b.from(b.add(in1)).via(b.add(f1)).toInlet(merge.in(0));
return merge.out(); b.from(b.add(in2)).via(b.add(f2)).toInlet(merge.in(1));
} return new SourceShape<String>(merge.out());
}); }
}));
// collecting // collecting
final Publisher<String> pub = source.runWith(publisher, materializer); final Publisher<String> pub = source.runWith(publisher, materializer);
@ -93,16 +94,21 @@ public class FlowGraphTest extends StreamTest {
final Iterable<String> input1 = Arrays.asList("A", "B", "C"); final Iterable<String> input1 = Arrays.asList("A", "B", "C");
final Iterable<Integer> input2 = Arrays.asList(1, 2, 3); final Iterable<Integer> input2 = Arrays.asList(1, 2, 3);
final Builder<BoxedUnit> b = FlowGraph.builder(); RunnableGraph.fromGraph( FlowGraph.create(
final Source<String, BoxedUnit> in1 = Source.from(input1); new Function<Builder<BoxedUnit>,ClosedShape>() {
final Source<Integer, BoxedUnit> in2 = Source.from(input2); @Override
final FanInShape2<String, Integer, Pair<String,Integer>> zip = b.graph(Zip.<String, Integer>create()); public ClosedShape apply(final Builder<BoxedUnit> b) throws Exception {
final Sink<Pair<String, Integer>, BoxedUnit> out = createSink(probe); final Source<String, BoxedUnit> in1 = Source.from(input1);
final Source<Integer, BoxedUnit> in2 = Source.from(input2);
final FanInShape2<String, Integer, Pair<String,Integer>> zip = b.add(Zip.<String, Integer>create());
final Sink<Pair<String, Integer>, BoxedUnit> out = createSink(probe);
b.edge(b.source(in1), zip.in0()); b.from(b.add(in1)).toInlet(zip.in0());
b.edge(b.source(in2), zip.in1()); b.from(b.add(in2)).toInlet(zip.in1());
b.edge(zip.out(), b.sink(out)); b.from(zip.out()).to(b.add(out));
b.run(materializer); return ClosedShape.getInstance();
}
})).run(materializer);
List<Object> output = Arrays.asList(probe.receiveN(3)); List<Object> output = Arrays.asList(probe.receiveN(3));
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
@ -123,17 +129,22 @@ public class FlowGraphTest extends StreamTest {
final Iterable<String> expected1 = Arrays.asList("A", "B", "C"); final Iterable<String> expected1 = Arrays.asList("A", "B", "C");
final Iterable<Integer> expected2 = Arrays.asList(1, 2, 3); final Iterable<Integer> expected2 = Arrays.asList(1, 2, 3);
final Builder<BoxedUnit> b = FlowGraph.builder(); RunnableGraph.fromGraph(FlowGraph.create(
final Outlet<Pair<String, Integer>> in = b.source(Source.from(input)); new Function<Builder<BoxedUnit>, ClosedShape>() {
final FanOutShape2<Pair<String, Integer>, String, Integer> unzip = b.graph(Unzip.<String, Integer>create()); @Override
public ClosedShape apply(final Builder<BoxedUnit> b) throws Exception {
final SourceShape<Pair<String, Integer>> in = b.add(Source.from(input));
final FanOutShape2<Pair<String, Integer>, String, Integer> unzip = b.add(Unzip.<String, Integer>create());
final Sink<String, BoxedUnit> out1 = createSink(probe1); final SinkShape<String> out1 = b.add(FlowGraphTest.<String>createSink(probe1));
final Sink<Integer, BoxedUnit> out2 = createSink(probe2); final SinkShape<Integer> out2 = b.add(FlowGraphTest.<Integer>createSink(probe2));
b.edge(in, unzip.in()); b.from(in).toInlet(unzip.in());
b.edge(unzip.out0(), b.sink(out1)); b.from(unzip.out0()).to(out1);
b.edge(unzip.out1(), b.sink(out2)); b.from(unzip.out1()).to(out2);
b.run(materializer); return ClosedShape.getInstance();
}
})).run(materializer);
List<Object> output1 = Arrays.asList(probe1.receiveN(3)); List<Object> output1 = Arrays.asList(probe1.receiveN(3));
List<Object> output2 = Arrays.asList(probe2.receiveN(3)); List<Object> output2 = Arrays.asList(probe2.receiveN(3));
@ -150,24 +161,31 @@ public class FlowGraphTest extends StreamTest {
final JavaTestKit probe1 = new JavaTestKit(system); final JavaTestKit probe1 = new JavaTestKit(system);
final JavaTestKit probe2 = new JavaTestKit(system); final JavaTestKit probe2 = new JavaTestKit(system);
final Builder<BoxedUnit> b = FlowGraph.builder(); RunnableGraph.fromGraph(FlowGraph.create(
final Source<Integer, BoxedUnit> in = Source.single(1); new Function<Builder<BoxedUnit>, ClosedShape>() {
@Override
public ClosedShape apply(final Builder<BoxedUnit> b) throws Exception {
final Source<Integer, BoxedUnit> in = Source.single(1);
final FanOutShape2<Integer, String, Integer> unzip = b.graph(UnzipWith.create( final FanOutShape2<Integer, String, Integer> unzip = b.add(UnzipWith.create(
new Function<Integer, Pair<String, Integer>>() { new Function<Integer, Pair<String, Integer>>() {
@Override public Pair<String, Integer> apply(Integer l) throws Exception { @Override
return new Pair<String, Integer>(l + "!", l); public Pair<String, Integer> apply(Integer l) throws Exception {
return new Pair<String, Integer>(l + "!", l);
}
})
);
final SinkShape<String> out1 = b.add(FlowGraphTest.<String>createSink(probe1));
final SinkShape<Integer> out2 = b.add(FlowGraphTest.<Integer>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();
} }
}) }
); )).run(materializer);
final Sink<String, BoxedUnit> out1 = createSink(probe1);
final Sink<Integer, BoxedUnit> 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);
Duration d = Duration.create(300, TimeUnit.MILLISECONDS); Duration d = Duration.create(300, TimeUnit.MILLISECONDS);
@ -186,28 +204,34 @@ public class FlowGraphTest extends StreamTest {
final JavaTestKit probe3 = new JavaTestKit(system); final JavaTestKit probe3 = new JavaTestKit(system);
final JavaTestKit probe4 = new JavaTestKit(system); final JavaTestKit probe4 = new JavaTestKit(system);
final Builder<BoxedUnit> b = FlowGraph.builder(); RunnableGraph.fromGraph(FlowGraph.create(
final Source<Integer, BoxedUnit> in = Source.single(1); new Function<Builder<BoxedUnit>, ClosedShape>() {
@Override
public ClosedShape apply(final Builder<BoxedUnit> b) throws Exception {
final Source<Integer, BoxedUnit> in = Source.single(1);
final FanOutShape4<Integer, String, Integer, String, Integer> unzip = b.graph(UnzipWith.create4( final FanOutShape4<Integer, String, Integer, String, Integer> unzip = b.add(UnzipWith.create4(
new Function<Integer, Tuple4<String, Integer, String, Integer>>() { new Function<Integer, Tuple4<String, Integer, String, Integer>>() {
@Override public Tuple4<String, Integer, String, Integer> apply(Integer l) throws Exception { @Override
return new Tuple4<String, Integer, String, Integer>(l.toString(), l, l + "+" + l, l + l); public Tuple4<String, Integer, String, Integer> apply(Integer l) throws Exception {
return new Tuple4<String, Integer, String, Integer>(l.toString(), l, l + "+" + l, l + l);
}
})
);
final SinkShape<String> out1 = b.add(FlowGraphTest.<String>createSink(probe1));
final SinkShape<Integer> out2 = b.add(FlowGraphTest.<Integer>createSink(probe2));
final SinkShape<String> out3 = b.add(FlowGraphTest.<String>createSink(probe3));
final SinkShape<Integer> out4 = b.add(FlowGraphTest.<Integer>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();
} }
}) })).run(materializer);
);
final Sink<String, BoxedUnit> out1 = createSink(probe1);
final Sink<Integer, BoxedUnit> out2 = createSink(probe2);
final Sink<String, BoxedUnit> out3 = createSink(probe3);
final Sink<Integer, BoxedUnit> 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);
Duration d = Duration.create(300, TimeUnit.MILLISECONDS); Duration d = Duration.create(300, TimeUnit.MILLISECONDS);
@ -234,15 +258,17 @@ public class FlowGraphTest extends StreamTest {
} }
}); });
final Future<Integer> future = FlowGraph.factory().closed(Sink.<Integer> head(), new Procedure2<Builder<Future<Integer> >, SinkShape<Integer>>() { final Future<Integer> future = RunnableGraph.fromGraph(FlowGraph.create(Sink.<Integer>head(),
new Function2<Builder<Future<Integer>>, SinkShape<Integer>, ClosedShape>() {
@Override @Override
public void apply(Builder<Future<Integer> > b, SinkShape<Integer> out) throws Exception { public ClosedShape apply(Builder<Future<Integer>> b, SinkShape<Integer> out) throws Exception {
final FanInShape2<Integer, Integer, Integer> zip = b.graph(sumZip); final FanInShape2<Integer, Integer, Integer> zip = b.add(sumZip);
b.edge(b.source(in1), zip.in0()); b.from(b.add(in1)).toInlet(zip.in0());
b.edge(b.source(in2), zip.in1()); b.from(b.add(in2)).toInlet(zip.in1());
b.edge(zip.out(), out.inlet()); b.from(zip.out()).to(out);
return ClosedShape.getInstance();
} }
}).run(materializer); })).run(materializer);
final Integer result = Await.result(future, Duration.create(300, TimeUnit.MILLISECONDS)); final Integer result = Await.result(future, Duration.create(300, TimeUnit.MILLISECONDS));
assertEquals(11, (int) result); assertEquals(11, (int) result);
@ -262,17 +288,20 @@ public class FlowGraphTest extends StreamTest {
} }
}); });
final Future<Integer> future = FlowGraph.factory().closed(Sink.<Integer> head(), new Procedure2<Builder<Future<Integer>>, SinkShape<Integer>>() { final Future<Integer> future = RunnableGraph.fromGraph(
FlowGraph.create(Sink.<Integer>head(),
new Function2<Builder<Future<Integer>>, SinkShape<Integer>, ClosedShape>() {
@Override @Override
public void apply(Builder<Future<Integer>> b, SinkShape<Integer> out) throws Exception { public ClosedShape apply(Builder<Future<Integer>> b, SinkShape<Integer> out) throws Exception {
final FanInShape4<Integer, Integer, Integer, Integer, Integer> zip = b.graph(sumZip); final FanInShape4<Integer, Integer, Integer, Integer, Integer> zip = b.add(sumZip);
b.edge(b.source(in1), zip.in0()); b.from(b.add(in1)).toInlet(zip.in0());
b.edge(b.source(in2), zip.in1()); b.from(b.add(in2)).toInlet(zip.in1());
b.edge(b.source(in3), zip.in2()); b.from(b.add(in3)).toInlet(zip.in2());
b.edge(b.source(in4), zip.in3()); b.from(b.add(in4)).toInlet(zip.in3());
b.edge(zip.out(), out.inlet()); b.from(zip.out()).to(out);
return ClosedShape.getInstance();
} }
}).run(materializer); })).run(materializer);
final Integer result = Await.result(future, Duration.create(300, TimeUnit.MILLISECONDS)); final Integer result = Await.result(future, Duration.create(300, TimeUnit.MILLISECONDS));
assertEquals(1111, (int) result); assertEquals(1111, (int) result);
@ -284,17 +313,19 @@ public class FlowGraphTest extends StreamTest {
final Source<Integer, BoxedUnit> in1 = Source.single(1); final Source<Integer, BoxedUnit> in1 = Source.single(1);
final TestProbe probe = TestProbe.apply(system); final TestProbe probe = TestProbe.apply(system);
final Future<Integer> future = FlowGraph.factory().closed(Sink.<Integer> head(), new Procedure2<Builder<Future<Integer>>, SinkShape<Integer>>() { final Future<Integer> future = RunnableGraph.fromGraph(
FlowGraph.create(Sink.<Integer> head(), new Function2<Builder<Future<Integer>>, SinkShape<Integer>, ClosedShape>() {
@Override @Override
public void apply(Builder<Future<Integer>> b, SinkShape<Integer> out) throws Exception { public ClosedShape apply(Builder<Future<Integer>> b, SinkShape<Integer> out) throws Exception {
b.from(b.graph(Source.single(1))).to(out); b.from(b.add(Source.single(1))).to(out);
b.from(b.materializedValue()).to(b.graph(Sink.foreach(new Procedure<Future<Integer>>(){ b.from(b.materializedValue()).to(b.add(Sink.foreach(new Procedure<Future<Integer>>(){
public void apply(Future<Integer> mat) throws Exception { public void apply(Future<Integer> mat) throws Exception {
Patterns.pipe(mat, system.dispatcher()).to(probe.ref()); 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)); final Integer result = Await.result(future, Duration.create(300, TimeUnit.MILLISECONDS));
assertEquals(1, (int) result); assertEquals(1, (int) result);

View file

@ -385,15 +385,16 @@ public class FlowTest extends StreamTest {
final Sink<String, Publisher<String>> publisher = Sink.publisher(); final Sink<String, Publisher<String>> publisher = Sink.publisher();
final Source<String, BoxedUnit> source = Source.factory().create(new Function<FlowGraph.Builder<BoxedUnit>, Outlet<String>>() { final Source<String, BoxedUnit> source = Source.fromGraph(
@Override FlowGraph.create(new Function<FlowGraph.Builder<BoxedUnit>, SourceShape<String>>() {
public Outlet<String> apply(Builder<BoxedUnit> b) throws Exception { @Override
final UniformFanInShape<String, String> merge = b.graph(Merge.<String> create(2)); public SourceShape<String> apply(Builder<BoxedUnit> b) throws Exception {
b.flow(b.source(in1), f1, merge.in(0)); final UniformFanInShape<String, String> merge = b.add(Merge.<String>create(2));
b.flow(b.source(in2), f2, merge.in(1)); b.from(b.add(in1)).via(b.add(f1)).toInlet(merge.in(0));
return merge.out(); b.from(b.add(in2)).via(b.add(f2)).toInlet(merge.in(1));
} return new SourceShape<String>(merge.out());
}); }
}));
// collecting // collecting
final Publisher<String> pub = source.runWith(publisher, materializer); final Publisher<String> pub = source.runWith(publisher, materializer);
@ -409,23 +410,25 @@ public class FlowTest extends StreamTest {
final Iterable<String> input1 = Arrays.asList("A", "B", "C"); final Iterable<String> input1 = Arrays.asList("A", "B", "C");
final Iterable<Integer> input2 = Arrays.asList(1, 2, 3); final Iterable<Integer> input2 = Arrays.asList(1, 2, 3);
final Builder<BoxedUnit> b = FlowGraph.<BoxedUnit>builder(); RunnableGraph.fromGraph(FlowGraph.create(new Function<Builder<BoxedUnit>, ClosedShape>(){
final Outlet<String> in1 = b.source(Source.from(input1)); public ClosedShape apply(Builder<BoxedUnit> b) {
final Outlet<Integer> in2 = b.source(Source.from(input2)); final Outlet<String> in1 = b.add(Source.from(input1)).outlet();
final FanInShape2<String, Integer, Pair<String, Integer>> zip = b.graph(Zip.<String, Integer> create()); final Outlet<Integer> in2 = b.add(Source.from(input2)).outlet();
final Inlet<Pair<String, Integer>> out = b.sink(Sink final FanInShape2<String, Integer, Pair<String, Integer>> zip = b.add(Zip.<String, Integer>create());
.foreach(new Procedure<Pair<String, Integer>>() { final SinkShape<Pair<String, Integer>> out =
@Override b.add(Sink.foreach(new Procedure<Pair<String, Integer>>() {
public void apply(Pair<String, Integer> param) throws Exception { @Override
probe.getRef().tell(param, ActorRef.noSender()); public void apply(Pair<String, Integer> param) throws Exception {
} probe.getRef().tell(param, ActorRef.noSender());
})); }
}));
b.edge(in1, zip.in0()); b.from(in1).toInlet(zip.in0());
b.edge(in2, zip.in1()); b.from(in2).toInlet(zip.in1());
b.edge(zip.out(), out); b.from(zip.out()).to(out);
return ClosedShape.getInstance();
b.run(materializer); }
})).run(materializer);
List<Object> output = Arrays.asList(probe.receiveN(3)); List<Object> output = Arrays.asList(probe.receiveN(3));
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
@ -638,19 +641,18 @@ public class FlowTest extends StreamTest {
@Test @Test
public void mustBeAbleToBroadcastEagerCancel() throws Exception { public void mustBeAbleToBroadcastEagerCancel() throws Exception {
final Sink<String, BoxedUnit> out1 = Sink.cancelled(); final Sink<String, BoxedUnit> sink = Sink.fromGraph(
final Sink<String, ?> out2 = Sink.ignore(); FlowGraph.create(new Function<FlowGraph.Builder<BoxedUnit>, SinkShape<String>>() {
@Override
final Sink<String, BoxedUnit> sink = Sink.factory().<String>create(new Function<FlowGraph.Builder<BoxedUnit>, Inlet<String>>() { public SinkShape<String> apply(Builder<BoxedUnit> b) throws Exception {
@Override final UniformFanOutShape<String, String> broadcast = b.add(Broadcast.<String>create(2, true));
public Inlet<String> apply(Builder<BoxedUnit> b) throws Exception { final SinkShape<String> out1 = b.add(Sink.<String>cancelled());
final UniformFanOutShape<String, String> broadcast = b.graph(Broadcast.<String>create(2, true)); final SinkShape<String> out2 = b.add(Sink.<String>ignore());
b.from(broadcast.out(0)).to(out1);
b.from(broadcast.out(0)).to(b.graph(out1)); b.from(broadcast.out(1)).to(out2);
b.from(broadcast.out(1)).to(b.graph(out2)); return new SinkShape<String>(broadcast.in());
return broadcast.in(); }
} }));
});
final JavaTestKit probe = new JavaTestKit(system); final JavaTestKit probe = new JavaTestKit(system);
Source<String, ActorRef> source = Source.actorRef(1, OverflowStrategy.dropNew()); Source<String, ActorRef> source = Source.actorRef(1, OverflowStrategy.dropNew());

View file

@ -35,7 +35,7 @@ public class TcpTest extends StreamTest {
final Sink<IncomingConnection, Future<BoxedUnit>> echoHandler = final Sink<IncomingConnection, Future<BoxedUnit>> echoHandler =
Sink.foreach(new Procedure<IncomingConnection>() { Sink.foreach(new Procedure<IncomingConnection>() {
public void apply(IncomingConnection conn) { public void apply(IncomingConnection conn) {
conn.handleWith(Flow.<ByteString>empty(), materializer); conn.handleWith(Flow.of(ByteString.class), materializer);
} }
}); });

View file

@ -26,7 +26,7 @@ class ActorMaterializerSpec extends AkkaSpec with ImplicitSender {
val m = ActorMaterializer.create(system) val m = ActorMaterializer.create(system)
val f = Source.lazyEmpty[Int].runFold(0)(_ + _)(m) val f = Source.maybe[Int].runFold(0)(_ + _)(m)
m.shutdown() m.shutdown()
@ -43,7 +43,7 @@ class ActorMaterializerSpec extends AkkaSpec with ImplicitSender {
"shut down the supervisor actor it encapsulates" in { "shut down the supervisor actor it encapsulates" in {
val m = ActorMaterializer.create(system).asInstanceOf[ActorMaterializerImpl] 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 m.supervisor ! StreamSupervisor.GetChildren
expectMsgType[StreamSupervisor.Children] expectMsgType[StreamSupervisor.Children]
m.shutdown() m.shutdown()

View file

@ -66,11 +66,11 @@ class DslFactoriesConsistencySpec extends WordSpec with Matchers {
} }
val testCases = Seq( val testCases = Seq(
TestCase("Source", scaladsl.Source.getClass, javadsl.Source.getClass, classOf[javadsl.SourceCreate]), TestCase("Source", scaladsl.Source.getClass, javadsl.Source.getClass),
TestCase("Flow", scaladsl.Flow.getClass, javadsl.Flow.getClass, classOf[javadsl.FlowCreate]), TestCase("Flow", scaladsl.Flow.getClass, javadsl.Flow.getClass),
TestCase("Sink", scaladsl.Sink.getClass, javadsl.Sink.getClass, classOf[javadsl.SinkCreate]), 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("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("ZipWith", Some(scaladsl.ZipWith.getClass), None, Some(javadsl.ZipWith.getClass)),
TestCase("Merge", scaladsl.Merge.getClass, javadsl.Merge.getClass), TestCase("Merge", scaladsl.Merge.getClass, javadsl.Merge.getClass),
TestCase("MergePreferred", scaladsl.MergePreferred.getClass, javadsl.MergePreferred.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.Flow.getClass, _ == "apply", _ == 24, _ true),
Ignore(_ == akka.stream.scaladsl.Sink.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.BidiFlow.getClass, _ == "apply", _ == 24, _ true),
Ignore(_ == akka.stream.scaladsl.FlowGraph.getClass, _ == "closed", _ == 24, _ true), Ignore(_ == akka.stream.scaladsl.FlowGraph.getClass, _ == "runnable", _ == 24, _ true),
Ignore(_ == akka.stream.scaladsl.FlowGraph.getClass, _ == "partial", _ == 24, _ true), Ignore(_ == akka.stream.scaladsl.FlowGraph.getClass, _ == "create", _ == 24, _ true),
// all generated methods like scaladsl.Sink$.akka$stream$scaladsl$Sink$$newOnCompleteStage$1 // all generated methods like scaladsl.Sink$.akka$stream$scaladsl$Sink$$newOnCompleteStage$1
Ignore(_ true, _.contains("$"), _ true, _ true)) Ignore(_ true, _.contains("$"), _ true, _ true))
@ -127,8 +127,8 @@ class DslFactoriesConsistencySpec extends WordSpec with Matchers {
/** /**
* Rename * Rename
* createN => create * createN => create
* closedN => closed * runnableN => runnable
* partialN => partial * createN => create
*/ */
private val unspecializeName: PartialFunction[Method, Method] = { private val unspecializeName: PartialFunction[Method, Method] = {
case m m.copy(name = m.name.filter(Character.isLetter)) case m m.copy(name = m.name.filter(Character.isLetter))

View file

@ -4,7 +4,7 @@
package akka.stream.actor package akka.stream.actor
import akka.actor.{ ActorRef, PoisonPill, Props } 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.scaladsl._
import akka.stream.testkit._ import akka.stream.testkit._
import akka.stream.testkit.Utils._ 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 sink1 = Sink(ActorSubscriber[String](system.actorOf(receiverProps(probe1.ref))))
val sink2: Sink[String, ActorRef] = Sink.actorSubscriber(receiverProps(probe2.ref)) val sink2: Sink[String, ActorRef] = Sink.actorSubscriber(receiverProps(probe2.ref))
val senderRef2 = FlowGraph.closed(Source.actorPublisher[Int](senderProps)) { implicit b val senderRef2 = RunnableGraph.fromGraph(FlowGraph.create(Source.actorPublisher[Int](senderProps)) { implicit b
source2 source2
import FlowGraph.Implicits._ import FlowGraph.Implicits._
@ -364,7 +364,8 @@ class ActorPublisherSpec extends AkkaSpec(ActorPublisherSpec.config) with Implic
bcast.out(0).map(_ + "mark") ~> sink1 bcast.out(0).map(_ + "mark") ~> sink1
bcast.out(1) ~> sink2 bcast.out(1) ~> sink2
}.run() ClosedShape
}).run()
(0 to 10).foreach { (0 to 10).foreach {
senderRef1 ! _ senderRef1 ! _

View file

@ -72,7 +72,7 @@ class ActorGraphInterpreterSpec extends AkkaSpec {
override def toString = "IdentityBidi" 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( Await.result(
Source(1 to 10).via(identity).grouped(100).runWith(Sink.head), Source(1 to 10).via(identity).grouped(100).runWith(Sink.head),
@ -117,7 +117,7 @@ class ActorGraphInterpreterSpec extends AkkaSpec {
override def toString = "IdentityBidi" 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 }) val identity = (identityBidiF atop identityBidiF atop identityBidiF).join(Flow[Int].map { x x })
Await.result( Await.result(
@ -163,7 +163,7 @@ class ActorGraphInterpreterSpec extends AkkaSpec {
override def toString = "IdentityBidi" 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 }) val identity = (identityBidiF atop identityBidiF atop identityBidiF).join(Flow[Int].map { x x })
Await.result( Await.result(
@ -214,7 +214,7 @@ class ActorGraphInterpreterSpec extends AkkaSpec {
val takeAll = Flow[Int].grouped(200).toMat(Sink.head)(Keep.right) 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) (out1, out2)
import FlowGraph.Implicits._ import FlowGraph.Implicits._
val bidi = b.add(rotatedBidi) val bidi = b.add(rotatedBidi)
@ -224,7 +224,8 @@ class ActorGraphInterpreterSpec extends AkkaSpec {
bidi.in2 <~ Source(1 to 100) bidi.in2 <~ Source(1 to 100)
bidi.out1 ~> out1 bidi.out1 ~> out1
}.run() ClosedShape
}).run()
Await.result(f1, 3.seconds) should ===(1 to 100) Await.result(f1, 3.seconds) should ===(1 to 100)
Await.result(f2, 3.seconds) should ===(1 to 10) Await.result(f2, 3.seconds) should ===(1 to 10)

View file

@ -216,7 +216,7 @@ class FramingSpec extends AkkaSpec {
} }
"support simple framing adapter" in { "support simple framing adapter" in {
val rechunkBidi = BidiFlow.wrap(rechunk, rechunk)(Keep.left) val rechunkBidi = BidiFlow.fromFlowsMat(rechunk, rechunk)(Keep.left)
val codecFlow = val codecFlow =
Framing.simpleFramingProtocol(1024) Framing.simpleFramingProtocol(1024)
.atop(rechunkBidi) .atop(rechunkBidi)

View file

@ -342,7 +342,7 @@ class TcpSpec extends AkkaSpec("akka.io.tcp.windows-connection-abort-workaround-
"properly full-close if requested" in assertAllStagesStopped { "properly full-close if requested" in assertAllStagesStopped {
val serverAddress = temporaryServerAddress() val serverAddress = temporaryServerAddress()
val writeButIgnoreRead: Flow[ByteString, ByteString, Unit] = 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 = val binding =
Await.result( Await.result(
@ -350,7 +350,7 @@ class TcpSpec extends AkkaSpec("akka.io.tcp.windows-connection-abort-workaround-
conn.flow.join(writeButIgnoreRead).run() conn.flow.join(writeButIgnoreRead).run()
})(Keep.left).run(), 3.seconds) })(Keep.left).run(), 3.seconds)
val result = Source.lazyEmpty[ByteString] val result = Source.maybe[ByteString]
.via(Tcp().outgoingConnection(serverAddress.getHostName, serverAddress.getPort)) .via(Tcp().outgoingConnection(serverAddress.getHostName, serverAddress.getPort))
.runFold(ByteString.empty)(_ ++ _) .runFold(ByteString.empty)(_ ++ _)
@ -385,7 +385,7 @@ class TcpSpec extends AkkaSpec("akka.io.tcp.windows-connection-abort-workaround-
val serverAddress = temporaryServerAddress() val serverAddress = temporaryServerAddress()
val binding = Tcp(system2).bindAndHandle(Flow[ByteString], serverAddress.getHostName, serverAddress.getPort)(mat2) 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 // Getting rid of existing connection actors by using a blunt instrument
system2.actorSelection(akka.io.Tcp(system2).getManager.path / "selectors" / "$a" / "*") ! Kill system2.actorSelection(akka.io.Tcp(system2).getManager.path / "selectors" / "$a" / "*") ! Kill

View file

@ -6,7 +6,7 @@ package akka.stream.io
import java.util.concurrent.TimeoutException import java.util.concurrent.TimeoutException
import akka.stream.ActorMaterializer import akka.stream.{ ClosedShape, ActorMaterializer }
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.stream.testkit.{ AkkaSpec, TestSubscriber, TestPublisher } import akka.stream.testkit.{ AkkaSpec, TestSubscriber, TestPublisher }
import scala.concurrent.{ Future, Await } import scala.concurrent.{ Future, Await }
@ -37,7 +37,7 @@ class TimeoutsSpec extends AkkaSpec {
"fail if no initial element passes until timeout" in assertAllStagesStopped { "fail if no initial element passes until timeout" in assertAllStagesStopped {
val downstreamProbe = TestSubscriber.probe[Int]() val downstreamProbe = TestSubscriber.probe[Int]()
Source.lazyEmpty[Int] Source.maybe[Int]
.via(Timeouts.initalTimeout(1.seconds)) .via(Timeouts.initalTimeout(1.seconds))
.runWith(Sink(downstreamProbe)) .runWith(Sink(downstreamProbe))
@ -142,8 +142,8 @@ class TimeoutsSpec extends AkkaSpec {
val upstreamWriter = TestPublisher.probe[Int]() val upstreamWriter = TestPublisher.probe[Int]()
val downstreamWriter = TestPublisher.probe[String]() val downstreamWriter = TestPublisher.probe[String]()
val upstream = Flow.wrap(Sink.ignore, Source(upstreamWriter))(Keep.left) val upstream = Flow.fromSinkAndSourceMat(Sink.ignore, Source(upstreamWriter))(Keep.left)
val downstream = Flow.wrap(Sink.ignore, Source(downstreamWriter))(Keep.left) val downstream = Flow.fromSinkAndSourceMat(Sink.ignore, Source(downstreamWriter))(Keep.left)
val assembly: RunnableGraph[(Future[Unit], Future[Unit])] = upstream val assembly: RunnableGraph[(Future[Unit], Future[Unit])] = upstream
.joinMat(Timeouts.idleTimeoutBidi[Int, String](2.seconds))(Keep.left) .joinMat(Timeouts.idleTimeoutBidi[Int, String](2.seconds))(Keep.left)
@ -172,14 +172,15 @@ class TimeoutsSpec extends AkkaSpec {
val downWrite = TestPublisher.probe[Int]() val downWrite = TestPublisher.probe[Int]()
val downRead = TestSubscriber.probe[String]() val downRead = TestSubscriber.probe[String]()
FlowGraph.closed() { implicit b RunnableGraph.fromGraph(FlowGraph.create() { implicit b
import FlowGraph.Implicits._ import FlowGraph.Implicits._
val timeoutStage = b.add(Timeouts.idleTimeoutBidi[String, Int](2.seconds)) val timeoutStage = b.add(Timeouts.idleTimeoutBidi[String, Int](2.seconds))
Source(upWrite) ~> timeoutStage.in1; Source(upWrite) ~> timeoutStage.in1;
timeoutStage.out1 ~> Sink(downRead) timeoutStage.out1 ~> Sink(downRead)
Sink(upRead) <~ timeoutStage.out2; Sink(upRead) <~ timeoutStage.out2;
timeoutStage.in2 <~ Source(downWrite) timeoutStage.in2 <~ Source(downWrite)
}.run() ClosedShape
}).run()
// Request enough for the whole test // Request enough for the whole test
upRead.request(100) upRead.request(100)
@ -219,14 +220,15 @@ class TimeoutsSpec extends AkkaSpec {
val downWrite = TestPublisher.probe[Int]() val downWrite = TestPublisher.probe[Int]()
val downRead = TestSubscriber.probe[String]() val downRead = TestSubscriber.probe[String]()
FlowGraph.closed() { implicit b RunnableGraph.fromGraph(FlowGraph.create() { implicit b
import FlowGraph.Implicits._ import FlowGraph.Implicits._
val timeoutStage = b.add(Timeouts.idleTimeoutBidi[String, Int](2.seconds)) val timeoutStage = b.add(Timeouts.idleTimeoutBidi[String, Int](2.seconds))
Source(upWrite) ~> timeoutStage.in1; Source(upWrite) ~> timeoutStage.in1;
timeoutStage.out1 ~> Sink(downRead) timeoutStage.out1 ~> Sink(downRead)
Sink(upRead) <~ timeoutStage.out2; Sink(upRead) <~ timeoutStage.out2;
timeoutStage.in2 <~ Source(downWrite) timeoutStage.in2 <~ Source(downWrite)
}.run() ClosedShape
}).run()
val te = TE("test") val te = TE("test")

View file

@ -12,7 +12,7 @@ import scala.util.Random
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.pattern.{ after later } import akka.pattern.{ after later }
import akka.stream.ActorMaterializer import akka.stream.{ ClosedShape, ActorMaterializer }
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.stream.stage._ import akka.stream.stage._
import akka.stream.testkit._ 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) } .recover { case e: SSLException Right(e) }
.collect { case Right(e) e }.toMat(Sink.head)(Keep.right) .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 // 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. // 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 { "reliably cancel subscriptions when TransportIn fails early" in assertAllStagesStopped {
val ex = new Exception("hello") val ex = new Exception("hello")
val (sub, out1, out2) = val (sub, out1, out2) =
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) (s, o1, o2)
val tls = b.add(clientTls(EagerClose)) val tls = b.add(clientTls(EagerClose))
s ~> tls.in1; tls.out1 ~> o1 s ~> tls.in1; tls.out1 ~> o1
o2 <~ tls.out2; tls.in2 <~ Source.failed(ex) 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(out1, 1.second) should be(ex)
the[Exception] thrownBy Await.result(out2, 1.second) should be(ex) the[Exception] thrownBy Await.result(out2, 1.second) should be(ex)
Thread.sleep(500) 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 { "reliably cancel subscriptions when UserIn fails early" in assertAllStagesStopped {
val ex = new Exception("hello") val ex = new Exception("hello")
val (sub, out1, out2) = val (sub, out1, out2) =
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) (s, o1, o2)
val tls = b.add(clientTls(EagerClose)) val tls = b.add(clientTls(EagerClose))
Source.failed[SslTlsOutbound](ex) ~> tls.in1; tls.out1 ~> o1 Source.failed[SslTlsOutbound](ex) ~> tls.in1; tls.out1 ~> o1
o2 <~ tls.out2; tls.in2 <~ s 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(out1, 1.second) should be(ex)
the[Exception] thrownBy Await.result(out2, 1.second) should be(ex) the[Exception] thrownBy Await.result(out2, 1.second) should be(ex)
Thread.sleep(500) Thread.sleep(500)

View file

@ -7,12 +7,10 @@ import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.Utils._ import akka.stream.testkit.Utils._
import org.scalactic.ConversionCheckedTripleEquals import org.scalactic.ConversionCheckedTripleEquals
import akka.util.ByteString import akka.util.ByteString
import akka.stream.BidiShape import akka.stream._
import akka.stream.ActorMaterializer
import scala.concurrent.Await import scala.concurrent.Await
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.collection.immutable import scala.collection.immutable
import akka.stream.Attributes
class BidiFlowSpec extends AkkaSpec with ConversionCheckedTripleEquals { class BidiFlowSpec extends AkkaSpec with ConversionCheckedTripleEquals {
import Attributes._ import Attributes._
@ -20,26 +18,22 @@ class BidiFlowSpec extends AkkaSpec with ConversionCheckedTripleEquals {
implicit val mat = ActorMaterializer() implicit val mat = ActorMaterializer()
val bidi = BidiFlow() { b val bidi = BidiFlow.fromFlows(
val top = b.add(Flow[Int].map(x x.toLong + 2).withAttributes(name("top"))) Flow[Int].map(x x.toLong + 2).withAttributes(name("top")),
val bottom = b.add(Flow[ByteString].map(_.decodeString("UTF-8")).withAttributes(name("bottom"))) Flow[ByteString].map(_.decodeString("UTF-8")).withAttributes(name("bottom")))
BidiShape(top.inlet, top.outlet, bottom.inlet, bottom.outlet)
}
val inverse = BidiFlow() { b val inverse = BidiFlow.fromFlows(
val top = b.add(Flow[Long].map(x x.toInt + 2).withAttributes(name("top"))) Flow[Long].map(x x.toInt + 2).withAttributes(name("top")),
val bottom = b.add(Flow[String].map(ByteString(_)).withAttributes(name("bottom"))) Flow[String].map(ByteString(_)).withAttributes(name("bottom")))
BidiShape(top.inlet, top.outlet, bottom.inlet, bottom.outlet)
}
val bidiMat = BidiFlow(Sink.head[Int]) { implicit b val bidiMat = BidiFlow.fromGraph(FlowGraph.create(Sink.head[Int]) { implicit b
s s
Source.single(42) ~> s Source.single(42) ~> s
val top = b.add(Flow[Int].map(x x.toLong + 2)) val top = b.add(Flow[Int].map(x x.toLong + 2))
val bottom = b.add(Flow[ByteString].map(_.decodeString("UTF-8"))) val bottom = b.add(Flow[ByteString].map(_.decodeString("UTF-8")))
BidiShape(top.inlet, top.outlet, bottom.inlet, bottom.outlet) BidiShape(top.inlet, top.outlet, bottom.inlet, bottom.outlet)
} })
val str = "Hello World" val str = "Hello World"
val bytes = ByteString(str) val bytes = ByteString(str)
@ -47,13 +41,14 @@ class BidiFlowSpec extends AkkaSpec with ConversionCheckedTripleEquals {
"A BidiFlow" must { "A BidiFlow" must {
"work top/bottom in isolation" in { "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) (st, sb)
val s = b.add(bidi) val s = b.add(bidi)
Source.single(1) ~> s.in1; s.out1 ~> st Source.single(1) ~> s.in1; s.out1 ~> st
sb <~ s.out2; s.in2 <~ Source.single(bytes) sb <~ s.out2; s.in2 <~ Source.single(bytes)
}.run() ClosedShape
}).run()
Await.result(top, 1.second) should ===(3) Await.result(top, 1.second) should ===(3)
Await.result(bottom, 1.second) should ===(str) Await.result(bottom, 1.second) should ===(str)
@ -85,15 +80,16 @@ class BidiFlowSpec extends AkkaSpec with ConversionCheckedTripleEquals {
} }
"materialize to its value" in { "materialize to its value" in {
val f = FlowGraph.closed(bidiMat) { implicit b val f = RunnableGraph.fromGraph(FlowGraph.create(bidiMat) { implicit b
bidi bidi
Flow[String].map(Integer.valueOf(_).toInt) <~> bidi <~> Flow[Long].map(x ByteString(s"Hello $x")) 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) Await.result(f, 1.second) should ===(42)
} }
"combine materialization values" in assertAllStagesStopped { "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 sink
val bcast = b.add(Broadcast[Int](2)) val bcast = b.add(Broadcast[Int](2))
val merge = b.add(Merge[Int](2)) val merge = b.add(Merge[Int](2))
@ -101,14 +97,14 @@ class BidiFlowSpec extends AkkaSpec with ConversionCheckedTripleEquals {
bcast ~> sink bcast ~> sink
Source.single(1) ~> bcast ~> merge Source.single(1) ~> bcast ~> merge
flow ~> merge flow ~> merge
(flow.inlet, merge.out) FlowShape(flow.inlet, merge.out)
} })
val right = Flow(Sink.head[immutable.Seq[Long]]) { implicit b val right = Flow.fromGraph(FlowGraph.create(Sink.head[immutable.Seq[Long]]) { implicit b
sink sink
val flow = b.add(Flow[Long].grouped(10)) val flow = b.add(Flow[Long].grouped(10))
flow ~> sink 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() val ((l, m), r) = left.joinMat(bidiMat)(Keep.both).joinMat(right)(Keep.both).run()
Await.result(l, 1.second) should ===(1) Await.result(l, 1.second) should ===(1)
Await.result(m, 1.second) should ===(42) Await.result(m, 1.second) should ===(42)

View file

@ -4,8 +4,7 @@
package akka.stream.scaladsl package akka.stream.scaladsl
import akka.stream.Attributes._ import akka.stream.Attributes._
import akka.stream.ActorMaterializer import akka.stream.{ ClosedShape, ActorMaterializer, OverflowStrategy }
import akka.stream.OverflowStrategy
import akka.stream.testkit._ import akka.stream.testkit._
import akka.stream.stage._ import akka.stream.stage._
@ -41,47 +40,52 @@ class FlowGraphCompileSpec extends AkkaSpec {
val out2 = Sink.head[String] val out2 = Sink.head[String]
"A Graph" should { "A Graph" should {
import FlowGraph.Implicits._
"build simple merge" in { "build simple merge" in {
FlowGraph.closed() { b RunnableGraph.fromGraph(FlowGraph.create() { implicit b
val merge = b.add(Merge[String](2)) val merge = b.add(Merge[String](2))
b.addEdge(b.add(in1), f1, merge.in(0)) in1 ~> f1 ~> merge.in(0)
b.addEdge(b.add(in2), f2, merge.in(1)) in2 ~> f2 ~> merge.in(1)
b.addEdge(merge.out, f3, b.add(out1)) merge.out ~> f3 ~> out1
}.run() ClosedShape
}).run()
} }
"build simple broadcast" in { "build simple broadcast" in {
FlowGraph.closed() { b RunnableGraph.fromGraph(FlowGraph.create() { implicit b
val bcast = b.add(Broadcast[String](2)) val bcast = b.add(Broadcast[String](2))
b.addEdge(b.add(in1), f1, bcast.in) in1 ~> f1 ~> bcast.in
b.addEdge(bcast.out(0), f2, b.add(out1)) bcast.out(0) ~> f2 ~> out1
b.addEdge(bcast.out(1), f3, b.add(out2)) bcast.out(1) ~> f3 ~> out2
}.run() ClosedShape
}).run()
} }
"build simple balance" in { "build simple balance" in {
FlowGraph.closed() { b RunnableGraph.fromGraph(FlowGraph.create() { implicit b
val balance = b.add(Balance[String](2)) val balance = b.add(Balance[String](2))
b.addEdge(b.add(in1), f1, balance.in) in1 ~> f1 ~> balance.in
b.addEdge(balance.out(0), f2, b.add(out1)) balance.out(0) ~> f2 ~> out1
b.addEdge(balance.out(1), f3, b.add(out2)) balance.out(1) ~> f3 ~> out2
} ClosedShape
})
} }
"build simple merge - broadcast" in { "build simple merge - broadcast" in {
FlowGraph.closed() { b RunnableGraph.fromGraph(FlowGraph.create() { implicit b
val merge = b.add(Merge[String](2)) val merge = b.add(Merge[String](2))
val bcast = b.add(Broadcast[String](2)) val bcast = b.add(Broadcast[String](2))
b.addEdge(b.add(in1), f1, merge.in(0)) in1 ~> f1 ~> merge.in(0)
b.addEdge(b.add(in2), f2, merge.in(1)) in2 ~> f2 ~> merge.in(1)
b.addEdge(merge.out, f3, bcast.in) merge ~> f3 ~> bcast
b.addEdge(bcast.out(0), f4, b.add(out1)) bcast.out(0) ~> f4 ~> out1
b.addEdge(bcast.out(1), f5, b.add(out2)) bcast.out(1) ~> f5 ~> out2
}.run() ClosedShape
}).run()
} }
"build simple merge - broadcast with implicits" in { "build simple merge - broadcast with implicits" in {
FlowGraph.closed() { implicit b RunnableGraph.fromGraph(FlowGraph.create() { implicit b
import FlowGraph.Implicits._ import FlowGraph.Implicits._
val merge = b.add(Merge[String](2)) val merge = b.add(Merge[String](2))
val bcast = b.add(Broadcast[String](2)) val bcast = b.add(Broadcast[String](2))
@ -90,7 +94,8 @@ class FlowGraphCompileSpec extends AkkaSpec {
bcast.out(0) ~> f3 ~> b.add(out1) bcast.out(0) ~> f3 ~> b.add(out1)
b.add(in2) ~> f4 ~> merge.in(1) b.add(in2) ~> f4 ~> merge.in(1)
bcast.out(1) ~> f5 ~> b.add(out2) bcast.out(1) ~> f5 ~> b.add(out2)
}.run() ClosedShape
}).run()
} }
/* /*
@ -105,24 +110,25 @@ class FlowGraphCompileSpec extends AkkaSpec {
"detect cycle in " in { "detect cycle in " in {
pending // FIXME needs cycle detection capability pending // FIXME needs cycle detection capability
intercept[IllegalArgumentException] { intercept[IllegalArgumentException] {
FlowGraph.closed() { b RunnableGraph.fromGraph(FlowGraph.create() { implicit b
val merge = b.add(Merge[String](2)) val merge = b.add(Merge[String](2))
val bcast1 = b.add(Broadcast[String](2)) val bcast1 = b.add(Broadcast[String](2))
val bcast2 = b.add(Broadcast[String](2)) val bcast2 = b.add(Broadcast[String](2))
val feedbackLoopBuffer = Flow[String].buffer(10, OverflowStrategy.dropBuffer) val feedbackLoopBuffer = Flow[String].buffer(10, OverflowStrategy.dropBuffer)
b.addEdge(b.add(in1), f1, merge.in(0)) in1 ~> f1 ~> merge.in(0)
b.addEdge(merge.out, f2, bcast1.in) merge ~> f2 ~> bcast1
b.addEdge(bcast1.out(0), f3, b.add(out1)) bcast1.out(0) ~> f3 ~> out1
b.addEdge(bcast1.out(1), feedbackLoopBuffer, bcast2.in) bcast1.out(1) ~> feedbackLoopBuffer ~> bcast2
b.addEdge(bcast2.out(0), f5, merge.in(1)) // cycle bcast2.out(0) ~> f5 ~> merge.in(1) // cycle
b.addEdge(bcast2.out(1), f6, b.add(out2)) bcast2.out(1) ~> f6 ~> out2
} ClosedShape
})
}.getMessage.toLowerCase should include("cycle") }.getMessage.toLowerCase should include("cycle")
} }
"express complex topologies in a readable way" in { "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 merge = b.add(Merge[String](2))
val bcast1 = b.add(Broadcast[String](2)) val bcast1 = b.add(Broadcast[String](2))
val bcast2 = 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) b.add(in1) ~> f1 ~> merge ~> f2 ~> bcast1 ~> f3 ~> b.add(out1)
bcast1 ~> feedbackLoopBuffer ~> bcast2 ~> f5 ~> merge bcast1 ~> feedbackLoopBuffer ~> bcast2 ~> f5 ~> merge
bcast2 ~> f6 ~> b.add(out2) bcast2 ~> f6 ~> b.add(out2)
}.run() ClosedShape
}).run()
} }
"build broadcast - merge" in { "build broadcast - merge" in {
FlowGraph.closed() { implicit b RunnableGraph.fromGraph(FlowGraph.create() { implicit b
val bcast = b.add(Broadcast[String](2)) val bcast = b.add(Broadcast[String](2))
val merge = b.add(Merge[String](2)) val merge = b.add(Merge[String](2))
import FlowGraph.Implicits._ import FlowGraph.Implicits._
in1 ~> f1 ~> bcast ~> f2 ~> merge ~> f3 ~> out1 in1 ~> f1 ~> bcast ~> f2 ~> merge ~> f3 ~> out1
bcast ~> f4 ~> merge bcast ~> f4 ~> merge
}.run() ClosedShape
}).run()
} }
"build wikipedia Topological_sorting" in { "build wikipedia Topological_sorting" in {
// see https://en.wikipedia.org/wiki/Topological_sorting#mediaviewer/File:Directed_acyclic_graph.png // 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 b3 = b.add(Broadcast[String](2))
val b7 = b.add(Broadcast[String](2)) val b7 = b.add(Broadcast[String](2))
val b11 = b.add(Broadcast[String](3)) val b11 = b.add(Broadcast[String](3))
@ -170,22 +178,24 @@ class FlowGraphCompileSpec extends AkkaSpec {
in5 ~> f("k") ~> m11 in5 ~> f("k") ~> m11
in3 ~> f("l") ~> b3 ~> f("m") ~> m8 in3 ~> f("l") ~> b3 ~> f("m") ~> m8
b3 ~> f("n") ~> m10 b3 ~> f("n") ~> m10
}.run() ClosedShape
}).run()
} }
"make it optional to specify flows" in { "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 merge = b.add(Merge[String](2))
val bcast = b.add(Broadcast[String](2)) val bcast = b.add(Broadcast[String](2))
import FlowGraph.Implicits._ import FlowGraph.Implicits._
in1 ~> merge ~> bcast ~> out1 in1 ~> merge ~> bcast ~> out1
in2 ~> merge in2 ~> merge
bcast ~> out2 bcast ~> out2
}.run() ClosedShape
}).run()
} }
"build unzip - zip" in { "build unzip - zip" in {
FlowGraph.closed() { implicit b RunnableGraph.fromGraph(FlowGraph.create() { implicit b
val zip = b.add(Zip[Int, String]()) val zip = b.add(Zip[Int, String]())
val unzip = b.add(Unzip[Int, String]()) val unzip = b.add(Unzip[Int, String]())
val out = Sink.publisher[(Int, String)] val out = Sink.publisher[(Int, String)]
@ -194,12 +204,13 @@ class FlowGraphCompileSpec extends AkkaSpec {
unzip.out0 ~> Flow[Int].map(_ * 2) ~> zip.in0 unzip.out0 ~> Flow[Int].map(_ * 2) ~> zip.in0
unzip.out1 ~> zip.in1 unzip.out1 ~> zip.in1
zip.out ~> out zip.out ~> out
}.run() ClosedShape
}).run()
} }
"distinguish between input and output ports" in { "distinguish between input and output ports" in {
intercept[IllegalArgumentException] { intercept[IllegalArgumentException] {
FlowGraph.closed() { implicit b RunnableGraph.fromGraph(FlowGraph.create() { implicit b
val zip = b.add(Zip[Int, String]()) val zip = b.add(Zip[Int, String]())
val unzip = b.add(Unzip[Int, String]()) val unzip = b.add(Unzip[Int, String]())
val wrongOut = Sink.publisher[(Int, Int)] val wrongOut = Sink.publisher[(Int, Int)]
@ -210,22 +221,24 @@ class FlowGraphCompileSpec extends AkkaSpec {
"zip.left ~> zip.right" shouldNot compile "zip.left ~> zip.right" shouldNot compile
"Flow(List(1, 2, 3)) ~> zip.left ~> wrongOut" 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 """Flow(List(1 -> "a", 2 -> "b", 3 -> "c")) ~> unzip.in ~> whatever""" shouldNot compile
} ClosedShape
}.getMessage should include("unconnected") })
}.getMessage should include("must correspond to")
} }
"build with variance" in { "build with variance" in {
val out = Sink(TestSubscriber.manualProbe[Fruit]()) RunnableGraph.fromGraph(FlowGraph.create() { implicit b
FlowGraph.closed() { b import FlowGraph.Implicits._
val merge = b.add(Merge[Fruit](2)) val merge = b.add(Merge[Fruit](2))
b.addEdge(b add Source[Fruit](apples), Flow[Fruit], merge.in(0)) Source[Fruit](apples) ~> Flow[Fruit] ~> merge.in(0)
b.addEdge(b add Source[Apple](apples), Flow[Apple], merge.in(1)) Source[Apple](apples) ~> Flow[Apple] ~> merge.in(1)
b.addEdge(merge.out, Flow[Fruit].map(identity), b add out) merge.out ~> Flow[Fruit].map(identity) ~> Sink(TestSubscriber.manualProbe[Fruit]())
} ClosedShape
})
} }
"build with variance when indices are not specified" in { "build with variance when indices are not specified" in {
FlowGraph.closed() { implicit b RunnableGraph.fromGraph(FlowGraph.create() { implicit b
import FlowGraph.Implicits._ import FlowGraph.Implicits._
val fruitMerge = b.add(Merge[Fruit](2)) val fruitMerge = b.add(Merge[Fruit](2))
Source[Fruit](apples) ~> fruitMerge Source[Fruit](apples) ~> fruitMerge
@ -253,11 +266,12 @@ class FlowGraphCompileSpec extends AkkaSpec {
Source[Apple](apples) ~> appleBcast Source[Apple](apples) ~> appleBcast
appleBcast ~> Sink.head[Fruit] appleBcast ~> Sink.head[Fruit]
appleBcast ~> Sink.head[Apple] appleBcast ~> Sink.head[Apple]
} ClosedShape
})
} }
"build with implicits and variance" in { "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 appleSource = b.add(Source(TestPublisher.manualProbe[Apple]()))
def fruitSource = b.add(Source(TestPublisher.manualProbe[Fruit]())) def fruitSource = b.add(Source(TestPublisher.manualProbe[Fruit]()))
val outA = b add Sink(TestSubscriber.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 ~> b.add(Broadcast[Apple](2))" shouldNot compile
"merge.out ~> Flow[Fruit].map(identity) ~> b.add(Broadcast[Apple](2))" shouldNot compile "merge.out ~> Flow[Fruit].map(identity) ~> b.add(Broadcast[Apple](2))" shouldNot compile
"fruitSource ~> merge ~> b.add(Broadcast[Apple](2))" shouldNot compile "fruitSource ~> merge ~> b.add(Broadcast[Apple](2))" shouldNot compile
} ClosedShape
})
} }
"build with plain flow without junctions" in { "build with plain flow without junctions" in {
FlowGraph.closed() { b import FlowGraph.Implicits._
b.addEdge(b.add(in1), f1, b.add(out1)) RunnableGraph.fromGraph(FlowGraph.create() { implicit b
}.run() in1 ~> f1 ~> out1
FlowGraph.closed() { b ClosedShape
b.addEdge(b.add(in1), f1, b.add(f2.to(out1))) }).run()
}.run() RunnableGraph.fromGraph(FlowGraph.create() { implicit b
FlowGraph.closed() { b in1 ~> f1 ~> f2.to(out1)
b.addEdge(b.add(in1 via f1), f2, b.add(out1)) ClosedShape
}.run() }).run()
FlowGraph.closed() { implicit b RunnableGraph.fromGraph(FlowGraph.create() { implicit b
import FlowGraph.Implicits._ (in1 via f1) ~> f2 ~> out1
b.add(in1) ~> f1 ~> b.add(out1) ClosedShape
}.run() }).run()
FlowGraph.closed() { implicit b RunnableGraph.fromGraph(FlowGraph.create() { implicit b
import FlowGraph.Implicits._ in1 ~> out1
b.add(in1) ~> b.add(out1) ClosedShape
}.run() }).run()
FlowGraph.closed() { implicit b RunnableGraph.fromGraph(FlowGraph.create() { implicit b
import FlowGraph.Implicits._ in1 ~> (f1 to out1)
b.add(in1) ~> b.add(f1 to out1) ClosedShape
}.run() }).run()
FlowGraph.closed() { implicit b RunnableGraph.fromGraph(FlowGraph.create() { implicit b
import FlowGraph.Implicits._ (in1 via f1) ~> out1
b.add(in1 via f1) ~> b.add(out1) ClosedShape
}.run() }).run()
FlowGraph.closed() { implicit b RunnableGraph.fromGraph(FlowGraph.create() { implicit b
import FlowGraph.Implicits._ (in1 via f1) ~> (f2 to out1)
b.add(in1 via f1) ~> b.add(f2 to out1) ClosedShape
}.run() }).run()
} }
} }
} }

View file

@ -3,7 +3,7 @@
*/ */
package akka.stream.scaladsl package akka.stream.scaladsl
import akka.stream.{ ActorMaterializer, ActorMaterializerSettings } import akka.stream.{ FlowShape, ActorMaterializer, ActorMaterializerSettings }
import akka.stream.testkit._ import akka.stream.testkit._
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import scala.concurrent.Await import scala.concurrent.Await
@ -24,7 +24,7 @@ class FlowJoinSpec extends AkkaSpec(ConfigFactory.parseString("akka.loglevel=INF
val source = Source(0 to end) val source = Source(0 to end)
val probe = TestSubscriber.manualProbe[Seq[Int]]() val probe = TestSubscriber.manualProbe[Seq[Int]]()
val flow1 = Flow() { implicit b val flow1 = Flow.fromGraph(FlowGraph.create() { implicit b
import FlowGraph.Implicits._ import FlowGraph.Implicits._
val merge = b.add(Merge[Int](2)) val merge = b.add(Merge[Int](2))
val broadcast = b.add(Broadcast[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 merge.out ~> broadcast.in
broadcast.out(0).grouped(1000) ~> Sink(probe) 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) val flow2 = Flow[Int].filter(_ % 2 == 1).map(_ * 10).take((end + 1) / 2)

View file

@ -41,7 +41,7 @@ class FlowScanSpec extends AkkaSpec {
} }
"emit values promptly" in { "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)) Await.result(f, 1.second) should ===(Seq(0, 1))
} }

View file

@ -5,7 +5,7 @@ package akka.stream.scaladsl
import akka.stream.Attributes._ import akka.stream.Attributes._
import akka.stream.ActorAttributes._ import akka.stream.ActorAttributes._
import akka.stream.ActorMaterializer import akka.stream.{ FlowShape, ActorMaterializer }
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
import akka.actor.ActorRef import akka.actor.ActorRef
import akka.testkit.TestProbe import akka.testkit.TestProbe
@ -33,13 +33,13 @@ class FlowSectionSpec extends AkkaSpec(FlowSectionSpec.config) {
} }
"have a nested flow with a different dispatcher" in { "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._ import FlowGraph.Implicits._
val bcast1 = b.add(Broadcast[Int](1)) val bcast1 = b.add(Broadcast[Int](1))
val bcast2 = b.add(Broadcast[Int](1)) val bcast2 = b.add(Broadcast[Int](1))
bcast1 ~> Flow[Int].map(sendThreadNameTo(testActor)) ~> bcast2.in bcast1 ~> Flow[Int].map(sendThreadNameTo(testActor)) ~> bcast2.in
(bcast1.in, bcast2.out(0)) FlowShape(bcast1.in, bcast2.out(0))
}.withAttributes(dispatcher("my-dispatcher1")) }).withAttributes(dispatcher("my-dispatcher1"))
Source.single(1).via(flow).to(Sink.ignore).run() Source.single(1).via(flow).to(Sink.ignore).run()
@ -51,21 +51,21 @@ class FlowSectionSpec extends AkkaSpec(FlowSectionSpec.config) {
val probe1 = TestProbe() val probe1 = TestProbe()
val probe2 = TestProbe() val probe2 = TestProbe()
val flow1 = Flow() { implicit b val flow1 = Flow.fromGraph(FlowGraph.create() { implicit b
import FlowGraph.Implicits._ import FlowGraph.Implicits._
val bcast1 = b.add(Broadcast[Int](1)) val bcast1 = b.add(Broadcast[Int](1))
val bcast2 = b.add(Broadcast[Int](1)) val bcast2 = b.add(Broadcast[Int](1))
bcast1 ~> Flow[Int].map(sendThreadNameTo(probe1.ref)) ~> bcast2.in bcast1 ~> Flow[Int].map(sendThreadNameTo(probe1.ref)) ~> bcast2.in
(bcast1.in, bcast2.out(0)) FlowShape(bcast1.in, bcast2.out(0))
}.withAttributes(dispatcher("my-dispatcher1")) }).withAttributes(dispatcher("my-dispatcher1"))
val flow2 = Flow() { implicit b val flow2 = Flow.fromGraph(FlowGraph.create() { implicit b
import FlowGraph.Implicits._ import FlowGraph.Implicits._
val bcast1 = b.add(Broadcast[Int](1)) val bcast1 = b.add(Broadcast[Int](1))
val bcast2 = b.add(Broadcast[Int](1)) val bcast2 = b.add(Broadcast[Int](1))
bcast1 ~> flow1.via(Flow[Int].map(sendThreadNameTo(probe2.ref))) ~> bcast2.in bcast1 ~> flow1.via(Flow[Int].map(sendThreadNameTo(probe2.ref))) ~> bcast2.in
(bcast1.in, bcast2.out(0)) FlowShape(bcast1.in, bcast2.out(0))
}.withAttributes(dispatcher("my-dispatcher2")) }).withAttributes(dispatcher("my-dispatcher2"))
Source.single(1).via(flow2).to(Sink.ignore).run() Source.single(1).via(flow2).to(Sink.ignore).run()

View file

@ -41,8 +41,8 @@ class FlowTakeSpec extends AkkaSpec with ScriptedTest {
} }
"complete eagerly when zero or less is taken independently of upstream completion" in { "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.maybe[Int].take(0).runWith(Sink.ignore), 3.second)
Await.result(Source.lazyEmpty.take(-1).runWith(Sink.ignore), 3.second) Await.result(Source.maybe[Int].take(-1).runWith(Sink.ignore), 3.second)
} }
} }

View file

@ -12,7 +12,7 @@ import org.reactivestreams.Subscriber
object GraphFlowSpec { object GraphFlowSpec {
val source1 = Source(0 to 3) val source1 = Source(0 to 3)
val partialGraph = FlowGraph.partial() { implicit b val partialGraph = FlowGraph.create() { implicit b
import FlowGraph.Implicits._ import FlowGraph.Implicits._
val source2 = Source(4 to 9) val source2 = Source(4 to 9)
val source3 = Source.empty[Int] val source3 = Source.empty[Int]
@ -62,12 +62,11 @@ class GraphFlowSpec extends AkkaSpec {
"work with a Source and Sink" in { "work with a Source and Sink" in {
val probe = TestSubscriber.manualProbe[Int]() val probe = TestSubscriber.manualProbe[Int]()
val flow = Flow(partialGraph) { implicit b val flow = Flow.fromGraph(FlowGraph.create(partialGraph) { implicit b
partial partial
import FlowGraph.Implicits._ import FlowGraph.Implicits._
FlowShape(partial.inlet, partial.outlet.map(_.toInt).outlet)
(partial.inlet, partial.outlet.map(_.toInt).outlet) })
}
source1.via(flow).to(Sink(probe)).run() source1.via(flow).to(Sink(probe)).run()
@ -77,10 +76,9 @@ class GraphFlowSpec extends AkkaSpec {
"be transformable with a Pipe" in { "be transformable with a Pipe" in {
val probe = TestSubscriber.manualProbe[Int]() val probe = TestSubscriber.manualProbe[Int]()
val flow = Flow(partialGraph) { implicit b val flow = Flow.fromGraph(FlowGraph.create(partialGraph) { implicit b
partial partial FlowShape(partial.inlet, partial.outlet)
(partial.inlet, partial.outlet) })
}
source1.via(flow).map(_.toInt).to(Sink(probe)).run() source1.via(flow).map(_.toInt).to(Sink(probe)).run()
@ -90,15 +88,15 @@ class GraphFlowSpec extends AkkaSpec {
"work with another GraphFlow" in { "work with another GraphFlow" in {
val probe = TestSubscriber.manualProbe[Int]() val probe = TestSubscriber.manualProbe[Int]()
val flow1 = Flow(partialGraph) { implicit b val flow1 = Flow.fromGraph(FlowGraph.create(partialGraph) { implicit b
partial 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
(importFlow.inlet, importFlow.outlet) FlowShape(importFlow.inlet, importFlow.outlet)
} })
source1.via(flow1).via(flow2).to(Sink(probe)).run() source1.via(flow1).via(flow2).to(Sink(probe)).run()
@ -108,15 +106,15 @@ class GraphFlowSpec extends AkkaSpec {
"be reusable multiple times" in { "be reusable multiple times" in {
val probe = TestSubscriber.manualProbe[Int]() val probe = TestSubscriber.manualProbe[Int]()
val flow = Flow(Flow[Int].map(_ * 2)) { implicit b val flow = Flow.fromGraph(FlowGraph.create(Flow[Int].map(_ * 2)) { implicit b
importFlow importFlow FlowShape(importFlow.inlet, importFlow.outlet)
(importFlow.inlet, importFlow.outlet) })
}
FlowGraph.closed() { implicit b RunnableGraph.fromGraph(FlowGraph.create() { implicit b
import FlowGraph.Implicits._ import FlowGraph.Implicits._
Source(1 to 5) ~> flow ~> flow ~> Sink(probe) Source(1 to 5) ~> flow ~> flow ~> Sink(probe)
}.run() ClosedShape
}).run()
validateProbe(probe, 5, Set(4, 8, 12, 16, 20)) validateProbe(probe, 5, Set(4, 8, 12, 16, 20))
} }
@ -126,12 +124,12 @@ class GraphFlowSpec extends AkkaSpec {
"work with a Sink" in { "work with a Sink" in {
val probe = TestSubscriber.manualProbe[Int]() val probe = TestSubscriber.manualProbe[Int]()
val source = Source(partialGraph) { implicit b val source = Source.fromGraph(FlowGraph.create(partialGraph) { implicit b
partial partial
import FlowGraph.Implicits._ import FlowGraph.Implicits._
source1 ~> partial.inlet source1 ~> partial.inlet
partial.outlet.map(_.toInt).outlet SourceShape(partial.outlet.map(_.toInt).outlet)
} })
source.to(Sink(probe)).run() source.to(Sink(probe)).run()
@ -141,10 +139,10 @@ class GraphFlowSpec extends AkkaSpec {
"work with a Sink when having KeyedSource inside" in { "work with a Sink when having KeyedSource inside" in {
val probe = TestSubscriber.manualProbe[Int]() 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
subSource.outlet SourceShape(subSource.outlet)
} })
val mm: Subscriber[Int] = source.to(Sink(probe)).run() val mm: Subscriber[Int] = source.to(Sink(probe)).run()
source1.to(Sink(mm)).run() source1.to(Sink(mm)).run()
@ -156,12 +154,12 @@ class GraphFlowSpec extends AkkaSpec {
val probe = TestSubscriber.manualProbe[Int]() val probe = TestSubscriber.manualProbe[Int]()
val source = Source(partialGraph) { implicit b val source = Source.fromGraph(FlowGraph.create(partialGraph) { implicit b
partial partial
import FlowGraph.Implicits._ import FlowGraph.Implicits._
source1 ~> partial.inlet source1 ~> partial.inlet
partial.outlet SourceShape(partial.outlet)
} })
source.map(_.toInt).to(Sink(probe)).run() source.map(_.toInt).to(Sink(probe)).run()
@ -171,17 +169,17 @@ class GraphFlowSpec extends AkkaSpec {
"work with an GraphFlow" in { "work with an GraphFlow" in {
val probe = TestSubscriber.manualProbe[Int]() val probe = TestSubscriber.manualProbe[Int]()
val source = Source(partialGraph) { implicit b val source = Source.fromGraph(FlowGraph.create(partialGraph) { implicit b
partial partial
import FlowGraph.Implicits._ import FlowGraph.Implicits._
source1 ~> partial.inlet 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
(importFlow.inlet, importFlow.outlet) FlowShape(importFlow.inlet, importFlow.outlet)
} })
source.via(flow).to(Sink(probe)).run() source.via(flow).to(Sink(probe)).run()
@ -191,20 +189,21 @@ class GraphFlowSpec extends AkkaSpec {
"be reusable multiple times" in { "be reusable multiple times" in {
val probe = TestSubscriber.manualProbe[Int]() 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 s
import FlowGraph.Implicits._ 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) (s1, s2)
import FlowGraph.Implicits._ import FlowGraph.Implicits._
val merge = b.add(Merge[Int](2)) val merge = b.add(Merge[Int](2))
s1.outlet ~> merge.in(0) s1.outlet ~> merge.in(0)
merge.out ~> Sink(probe) merge.out ~> Sink(probe)
s2.outlet.map(_ * 10) ~> merge.in(1) s2.outlet.map(_ * 10) ~> merge.in(1)
}.run() ClosedShape
}).run()
validateProbe(probe, 10, Set(2, 4, 6, 8, 10, 20, 40, 60, 80, 100)) 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 { "work with a Source" in {
val probe = TestSubscriber.manualProbe[Int]() val probe = TestSubscriber.manualProbe[Int]()
val sink = Sink(partialGraph) { implicit b val sink = Sink.fromGraph(FlowGraph.create(partialGraph) { implicit b
partial partial
import FlowGraph.Implicits._ import FlowGraph.Implicits._
partial.outlet.map(_.toInt) ~> Sink(probe) partial.outlet.map(_.toInt) ~> Sink(probe)
partial.inlet SinkShape(partial.inlet)
} })
source1.to(sink).run() source1.to(sink).run()
@ -230,10 +229,9 @@ class GraphFlowSpec extends AkkaSpec {
val probe = TestSubscriber.manualProbe[Int]() val probe = TestSubscriber.manualProbe[Int]()
val pubSink = Sink.publisher[Int] val pubSink = Sink.publisher[Int]
val sink = Sink(pubSink) { implicit b val sink = Sink.fromGraph(FlowGraph.create(pubSink) { implicit b
p p SinkShape(p.inlet)
p.inlet })
}
val mm = source1.runWith(sink) val mm = source1.runWith(sink)
Source(mm).to(Sink(probe)).run() Source(mm).to(Sink(probe)).run()
@ -244,13 +242,13 @@ class GraphFlowSpec extends AkkaSpec {
"be transformable with a Pipe" in { "be transformable with a Pipe" in {
val probe = TestSubscriber.manualProbe[Int]() 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) (partial, flow)
import FlowGraph.Implicits._ import FlowGraph.Implicits._
flow.outlet ~> partial.inlet flow.outlet ~> partial.inlet
partial.outlet.map(_.toInt) ~> Sink(probe) partial.outlet.map(_.toInt) ~> Sink(probe)
flow.inlet SinkShape(flow.inlet)
} })
val iSink = Flow[Int].map(_.toString).to(sink) val iSink = Flow[Int].map(_.toString).to(sink)
source1.to(iSink).run() source1.to(iSink).run()
@ -262,17 +260,17 @@ class GraphFlowSpec extends AkkaSpec {
val probe = TestSubscriber.manualProbe[Int]() val probe = TestSubscriber.manualProbe[Int]()
val flow = Flow(partialGraph) { implicit b val flow = Flow.fromGraph(FlowGraph.create(partialGraph) { implicit b
partial 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 flow
import FlowGraph.Implicits._ import FlowGraph.Implicits._
flow.outlet ~> Sink(probe) flow.outlet ~> Sink(probe)
flow.inlet SinkShape(flow.inlet)
} })
source1.via(flow).to(sink).run() source1.via(flow).to(sink).run()
@ -286,32 +284,33 @@ class GraphFlowSpec extends AkkaSpec {
val inSource = Source.subscriber[Int] val inSource = Source.subscriber[Int]
val outSink = Sink.publisher[Int] val outSink = Sink.publisher[Int]
val flow = Flow(partialGraph) { implicit b val flow = Flow.fromGraph(FlowGraph.create(partialGraph) { implicit b
partial partial
import FlowGraph.Implicits._ 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) (flow, src)
import FlowGraph.Implicits._ import FlowGraph.Implicits._
src.outlet ~> flow.inlet 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) (flow, snk)
import FlowGraph.Implicits._ import FlowGraph.Implicits._
flow.outlet ~> snk.inlet 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) (src, f, snk)
import FlowGraph.Implicits._ import FlowGraph.Implicits._
src.outlet.map(_.toInt) ~> f.inlet src.outlet.map(_.toInt) ~> f.inlet
f.outlet.map(_.toString) ~> snk.inlet f.outlet.map(_.toString) ~> snk.inlet
}.run() ClosedShape
}).run()
val subscriber = m1 val subscriber = m1
val publisher = m3 val publisher = m3
@ -326,21 +325,22 @@ class GraphFlowSpec extends AkkaSpec {
val inSource = Source.subscriber[Int] val inSource = Source.subscriber[Int]
val outSink = Sink.publisher[Int] val outSink = Sink.publisher[Int]
val source = Source(inSource) { implicit b val source = Source.fromGraph(FlowGraph.create(inSource) { implicit b
src src
src.outlet SourceShape(src.outlet)
} })
val sink = Sink(outSink) { implicit b val sink = Sink.fromGraph(FlowGraph.create(outSink) { implicit b
snk 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) (src, snk)
import FlowGraph.Implicits._ import FlowGraph.Implicits._
src.outlet ~> snk.inlet src.outlet ~> snk.inlet
}.run() ClosedShape
}).run()
val subscriber = m1 val subscriber = m1
val publisher = m2 val publisher = m2

View file

@ -4,9 +4,7 @@ import scala.concurrent.Await
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.concurrent.Future import scala.concurrent.Future
import akka.stream.ActorMaterializer import akka.stream.{ ClosedShape, ActorMaterializer, ActorMaterializerSettings }
import akka.stream.ActorMaterializerSettings
import akka.stream.testkit._ import akka.stream.testkit._
import akka.stream.testkit.scaladsl._ import akka.stream.testkit.scaladsl._
import akka.stream.testkit.Utils._ import akka.stream.testkit.Utils._
@ -25,12 +23,13 @@ class GraphBalanceSpec extends AkkaSpec {
val c1 = TestSubscriber.manualProbe[Int]() val c1 = TestSubscriber.manualProbe[Int]()
val c2 = 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)) val balance = b.add(Balance[Int](2))
Source(List(1, 2, 3)) ~> balance.in Source(List(1, 2, 3)) ~> balance.in
balance.out(0) ~> Sink(c1) balance.out(0) ~> Sink(c1)
balance.out(1) ~> Sink(c2) balance.out(1) ~> Sink(c2)
}.run() ClosedShape
}).run()
val sub1 = c1.expectSubscription() val sub1 = c1.expectSubscription()
val sub2 = c2.expectSubscription() val sub2 = c2.expectSubscription()
@ -48,13 +47,14 @@ class GraphBalanceSpec extends AkkaSpec {
"support waiting for demand from all downstream subscriptions" in { "support waiting for demand from all downstream subscriptions" in {
val s1 = TestSubscriber.manualProbe[Int]() 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 p2Sink
val balance = b.add(Balance[Int](2, waitForAllDownstreams = true)) val balance = b.add(Balance[Int](2, waitForAllDownstreams = true))
Source(List(1, 2, 3)) ~> balance.in Source(List(1, 2, 3)) ~> balance.in
balance.out(0) ~> Sink(s1) balance.out(0) ~> Sink(s1)
balance.out(1) ~> p2Sink balance.out(1) ~> p2Sink
}.run() ClosedShape
}).run()
val sub1 = s1.expectSubscription() val sub1 = s1.expectSubscription()
sub1.request(1) sub1.request(1)
@ -78,14 +78,15 @@ class GraphBalanceSpec extends AkkaSpec {
"support waiting for demand from all non-cancelled downstream subscriptions" in assertAllStagesStopped { "support waiting for demand from all non-cancelled downstream subscriptions" in assertAllStagesStopped {
val s1 = TestSubscriber.manualProbe[Int]() 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) (p2Sink, p3Sink)
val balance = b.add(Balance[Int](3, waitForAllDownstreams = true)) val balance = b.add(Balance[Int](3, waitForAllDownstreams = true))
Source(List(1, 2, 3)) ~> balance.in Source(List(1, 2, 3)) ~> balance.in
balance.out(0) ~> Sink(s1) balance.out(0) ~> Sink(s1)
balance.out(1) ~> p2Sink balance.out(1) ~> p2Sink
balance.out(2) ~> p3Sink balance.out(2) ~> p3Sink
}.run() ClosedShape
}).run()
val sub1 = s1.expectSubscription() val sub1 = s1.expectSubscription()
sub1.request(1) sub1.request(1)
@ -112,7 +113,7 @@ class GraphBalanceSpec extends AkkaSpec {
"work with 5-way balance" in { "work with 5-way balance" in {
val sink = Sink.head[Seq[Int]] 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 implicit b
(f1, f2, f3, f4, f5) (f1, f2, f3, f4, f5)
val balance = b.add(Balance[Int](5, waitForAllDownstreams = true)) 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(2).grouped(15) ~> f3
balance.out(3).grouped(15) ~> f4 balance.out(3).grouped(15) ~> f4
balance.out(4).grouped(15) ~> f5 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) 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 numElementsForSink = 10000
val outputs = Sink.fold[Int, Int](0)(_ + _) 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) (o1, o2, o3)
val balance = b.add(Balance[Int](3, waitForAllDownstreams = true)) val balance = b.add(Balance[Int](3, waitForAllDownstreams = true))
Source.repeat(1).take(numElementsForSink * 3) ~> balance.in Source.repeat(1).take(numElementsForSink * 3) ~> balance.in
balance.out(0) ~> o1 balance.out(0) ~> o1
balance.out(1) ~> o2 balance.out(1) ~> o2
balance.out(2) ~> o3 balance.out(2) ~> o3
}.run() ClosedShape
}).run()
import system.dispatcher import system.dispatcher
val sum = Future.sequence(results).map { res val sum = Future.sequence(results).map { res
@ -150,14 +153,15 @@ class GraphBalanceSpec extends AkkaSpec {
"fairly balance between three outputs" in { "fairly balance between three outputs" in {
val probe = TestSink.probe[Int] 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) (o1, o2, o3)
val balance = b.add(Balance[Int](3)) val balance = b.add(Balance[Int](3))
Source(1 to 7) ~> balance.in Source(1 to 7) ~> balance.in
balance.out(0) ~> o1 balance.out(0) ~> o1
balance.out(1) ~> o2 balance.out(1) ~> o2
balance.out(2) ~> o3 balance.out(2) ~> o3
}.run() ClosedShape
}).run()
p1.requestNext(1) p1.requestNext(1)
p2.requestNext(2) p2.requestNext(2)
@ -176,12 +180,13 @@ class GraphBalanceSpec extends AkkaSpec {
val c1 = TestSubscriber.manualProbe[Int]() val c1 = TestSubscriber.manualProbe[Int]()
val c2 = 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)) val balance = b.add(Balance[Int](2))
Source(List(1, 2, 3)) ~> balance.in Source(List(1, 2, 3)) ~> balance.in
balance.out(0) ~> Sink(c1) balance.out(0) ~> Sink(c1)
balance.out(1) ~> Sink(c2) balance.out(1) ~> Sink(c2)
}.run() ClosedShape
}).run()
val sub1 = c1.expectSubscription() val sub1 = c1.expectSubscription()
sub1.cancel() sub1.cancel()
@ -197,12 +202,13 @@ class GraphBalanceSpec extends AkkaSpec {
val c1 = TestSubscriber.manualProbe[Int]() val c1 = TestSubscriber.manualProbe[Int]()
val c2 = 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)) val balance = b.add(Balance[Int](2))
Source(List(1, 2, 3)) ~> balance.in Source(List(1, 2, 3)) ~> balance.in
balance.out(0) ~> Sink(c1) balance.out(0) ~> Sink(c1)
balance.out(1) ~> Sink(c2) balance.out(1) ~> Sink(c2)
}.run() ClosedShape
}).run()
val sub1 = c1.expectSubscription() val sub1 = c1.expectSubscription()
val sub2 = c2.expectSubscription() val sub2 = c2.expectSubscription()
@ -219,12 +225,13 @@ class GraphBalanceSpec extends AkkaSpec {
val c1 = TestSubscriber.manualProbe[Int]() val c1 = TestSubscriber.manualProbe[Int]()
val c2 = 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)) val balance = b.add(Balance[Int](2))
Source(p1.getPublisher) ~> balance.in Source(p1.getPublisher) ~> balance.in
balance.out(0) ~> Sink(c1) balance.out(0) ~> Sink(c1)
balance.out(1) ~> Sink(c2) balance.out(1) ~> Sink(c2)
}.run() ClosedShape
}).run()
val bsub = p1.expectSubscription() val bsub = p1.expectSubscription()
val sub1 = c1.expectSubscription() val sub1 = c1.expectSubscription()

View file

@ -5,8 +5,7 @@ import akka.stream.testkit.scaladsl.TestSink
import scala.concurrent.{ Future, Await } import scala.concurrent.{ Future, Await }
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.stream.{ OverflowStrategy, ActorMaterializerSettings } import akka.stream._
import akka.stream.ActorMaterializer
import akka.stream.testkit._ import akka.stream.testkit._
import akka.stream.testkit.Utils._ import akka.stream.testkit.Utils._
@ -24,12 +23,13 @@ class GraphBroadcastSpec extends AkkaSpec {
val c1 = TestSubscriber.manualProbe[Int]() val c1 = TestSubscriber.manualProbe[Int]()
val c2 = 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)) val bcast = b.add(Broadcast[Int](2))
Source(List(1, 2, 3)) ~> bcast.in Source(List(1, 2, 3)) ~> bcast.in
bcast.out(0) ~> Flow[Int].buffer(16, OverflowStrategy.backpressure) ~> Sink(c1) bcast.out(0) ~> Flow[Int].buffer(16, OverflowStrategy.backpressure) ~> Sink(c1)
bcast.out(1) ~> Flow[Int].buffer(16, OverflowStrategy.backpressure) ~> Sink(c2) bcast.out(1) ~> Flow[Int].buffer(16, OverflowStrategy.backpressure) ~> Sink(c2)
}.run() ClosedShape
}).run()
val sub1 = c1.expectSubscription() val sub1 = c1.expectSubscription()
val sub2 = c2.expectSubscription() val sub2 = c2.expectSubscription()
@ -53,7 +53,7 @@ class GraphBroadcastSpec extends AkkaSpec {
val headSink = Sink.head[Seq[Int]] val headSink = Sink.head[Seq[Int]]
import system.dispatcher import system.dispatcher
val result = FlowGraph.closed( val result = RunnableGraph.fromGraph(FlowGraph.create(
headSink, headSink,
headSink, headSink,
headSink, headSink,
@ -68,7 +68,8 @@ class GraphBroadcastSpec extends AkkaSpec {
bcast.out(2).grouped(5) ~> p3.inlet bcast.out(2).grouped(5) ~> p3.inlet
bcast.out(3).grouped(5) ~> p4.inlet bcast.out(3).grouped(5) ~> p4.inlet
bcast.out(4).grouped(5) ~> p5.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))) 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) (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)) 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, 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(19).grouped(5) ~> p20.inlet
bcast.out(20).grouped(5) ~> p21.inlet bcast.out(20).grouped(5) ~> p21.inlet
bcast.out(21).grouped(5) ~> p22.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))) 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 c1 = TestSubscriber.manualProbe[Int]()
val c2 = 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)) val bcast = b.add(Broadcast[Int](2))
Source(List(1, 2, 3)) ~> bcast.in Source(List(1, 2, 3)) ~> bcast.in
bcast.out(0) ~> Flow[Int] ~> Sink(c1) bcast.out(0) ~> Flow[Int] ~> Sink(c1)
bcast.out(1) ~> Flow[Int] ~> Sink(c2) bcast.out(1) ~> Flow[Int] ~> Sink(c2)
}.run() ClosedShape
}).run()
val sub1 = c1.expectSubscription() val sub1 = c1.expectSubscription()
sub1.cancel() sub1.cancel()
@ -145,12 +148,13 @@ class GraphBroadcastSpec extends AkkaSpec {
val c1 = TestSubscriber.manualProbe[Int]() val c1 = TestSubscriber.manualProbe[Int]()
val c2 = 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)) val bcast = b.add(Broadcast[Int](2))
Source(List(1, 2, 3)) ~> bcast.in Source(List(1, 2, 3)) ~> bcast.in
bcast.out(0) ~> Flow[Int].named("identity-a") ~> Sink(c1) bcast.out(0) ~> Flow[Int].named("identity-a") ~> Sink(c1)
bcast.out(1) ~> Flow[Int].named("identity-b") ~> Sink(c2) bcast.out(1) ~> Flow[Int].named("identity-b") ~> Sink(c2)
}.run() ClosedShape
}).run()
val sub1 = c1.expectSubscription() val sub1 = c1.expectSubscription()
val sub2 = c2.expectSubscription() val sub2 = c2.expectSubscription()
@ -167,12 +171,13 @@ class GraphBroadcastSpec extends AkkaSpec {
val c1 = TestSubscriber.manualProbe[Int]() val c1 = TestSubscriber.manualProbe[Int]()
val c2 = 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)) val bcast = b.add(Broadcast[Int](2))
Source(p1.getPublisher) ~> bcast.in Source(p1.getPublisher) ~> bcast.in
bcast.out(0) ~> Flow[Int] ~> Sink(c1) bcast.out(0) ~> Flow[Int] ~> Sink(c1)
bcast.out(1) ~> Flow[Int] ~> Sink(c2) bcast.out(1) ~> Flow[Int] ~> Sink(c2)
}.run() ClosedShape
}).run()
val bsub = p1.expectSubscription() val bsub = p1.expectSubscription()
val sub1 = c1.expectSubscription() val sub1 = c1.expectSubscription()
@ -195,12 +200,12 @@ class GraphBroadcastSpec extends AkkaSpec {
val c1 = TestSubscriber.manualProbe[Int]() val c1 = TestSubscriber.manualProbe[Int]()
val c2 = 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)) val bcast = b.add(Broadcast[Int](2))
bcast.out(0) ~> Sink(c1) bcast.out(0) ~> Sink(c1)
bcast.out(1) ~> Sink(c2) bcast.out(1) ~> Sink(c2)
bcast.in SinkShape(bcast.in)
} })
val s = Source.subscriber[Int].to(sink).run() val s = Source.subscriber[Int].to(sink).run()

View file

@ -30,7 +30,7 @@ class GraphConcatSpec extends TwoStreamsSetup {
"work in the happy case" in assertAllStagesStopped { "work in the happy case" in assertAllStagesStopped {
val probe = TestSubscriber.manualProbe[Int]() val probe = TestSubscriber.manualProbe[Int]()
FlowGraph.closed() { implicit b RunnableGraph.fromGraph(FlowGraph.create() { implicit b
val concat1 = b add Concat[Int]() val concat1 = b add Concat[Int]()
val concat2 = 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) Source(5 to 10) ~> concat2.in(1)
concat2.out ~> Sink(probe) concat2.out ~> Sink(probe)
}.run() ClosedShape
}).run()
val subscription = probe.expectSubscription() val subscription = probe.expectSubscription()
@ -140,12 +141,13 @@ class GraphConcatSpec extends TwoStreamsSetup {
val promise = Promise[Int]() val promise = Promise[Int]()
val subscriber = TestSubscriber.manualProbe[Int]() val subscriber = TestSubscriber.manualProbe[Int]()
FlowGraph.closed() { implicit b RunnableGraph.fromGraph(FlowGraph.create() { implicit b
val concat = b add Concat[Int]() val concat = b add Concat[Int]()
Source(List(1, 2, 3)) ~> concat.in(0) Source(List(1, 2, 3)) ~> concat.in(0)
Source(promise.future) ~> concat.in(1) Source(promise.future) ~> concat.in(1)
concat.out ~> Sink(subscriber) concat.out ~> Sink(subscriber)
}.run() ClosedShape
}).run()
val subscription = subscriber.expectSubscription() val subscription = subscriber.expectSubscription()
subscription.request(4) subscription.request(4)

View file

@ -3,7 +3,7 @@
*/ */
package akka.stream.scaladsl package akka.stream.scaladsl
import akka.stream.{ ActorMaterializer, ActorMaterializerSettings } import akka.stream.{ ClosedShape, SourceShape, ActorMaterializer, ActorMaterializerSettings }
import akka.stream.testkit._ import akka.stream.testkit._
import scala.concurrent.Await import scala.concurrent.Await
@ -25,11 +25,12 @@ class GraphMatValueSpec extends AkkaSpec {
"expose the materialized value as source" in { "expose the materialized value as source" in {
val sub = TestSubscriber.manualProbe[Int]() val sub = TestSubscriber.manualProbe[Int]()
val f = FlowGraph.closed(foldSink) { implicit b val f = RunnableGraph.fromGraph(FlowGraph.create(foldSink) { implicit b
fold fold
Source(1 to 10) ~> fold Source(1 to 10) ~> fold
b.materializedValue.mapAsync(4)(identity) ~> Sink(sub) b.materializedValue.mapAsync(4)(identity) ~> Sink(sub)
}.run() ClosedShape
}).run()
val r1 = Await.result(f, 3.seconds) val r1 = Await.result(f, 3.seconds)
sub.expectSubscription().request(1) sub.expectSubscription().request(1)
@ -41,7 +42,7 @@ class GraphMatValueSpec extends AkkaSpec {
"expose the materialized value as source multiple times" in { "expose the materialized value as source multiple times" in {
val sub = TestSubscriber.manualProbe[Int]() val sub = TestSubscriber.manualProbe[Int]()
val f = FlowGraph.closed(foldSink) { implicit b val f = RunnableGraph.fromGraph(FlowGraph.create(foldSink) { implicit b
fold fold
val zip = b.add(ZipWith[Int, Int, Int](_ + _)) val zip = b.add(ZipWith[Int, Int, Int](_ + _))
Source(1 to 10) ~> fold Source(1 to 10) ~> fold
@ -49,7 +50,8 @@ class GraphMatValueSpec extends AkkaSpec {
b.materializedValue.mapAsync(4)(identity) ~> zip.in1 b.materializedValue.mapAsync(4)(identity) ~> zip.in1
zip.out ~> Sink(sub) zip.out ~> Sink(sub)
}.run() ClosedShape
}).run()
val r1 = Await.result(f, 3.seconds) val r1 = Await.result(f, 3.seconds)
sub.expectSubscription().request(1) sub.expectSubscription().request(1)
@ -59,11 +61,11 @@ class GraphMatValueSpec extends AkkaSpec {
} }
// Exposes the materialized value as a stream value // 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 fold
Source(1 to 10) ~> fold Source(1 to 10) ~> fold
b.materializedValue SourceShape(b.materializedValue)
} })
"allow exposing the materialized value as port" in { "allow exposing the materialized value as port" in {
val (f1, f2) = foldFeedbackSource.mapAsync(4)(identity).map(_ + 100).toMat(Sink.head)(Keep.both).run() 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 { "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) (s1, s2)
val zip = b.add(ZipWith[Int, Int, Int](_ + _)) val zip = b.add(ZipWith[Int, Int, Int](_ + _))
s1.outlet.mapAsync(4)(identity) ~> zip.in0 s1.outlet.mapAsync(4)(identity) ~> zip.in0
s2.outlet.mapAsync(4)(identity).map(_ * 100) ~> zip.in1 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) (s1, s2)
val zip = b.add(ZipWith[Int, Int, Int](_ + _)) val zip = b.add(ZipWith[Int, Int, Int](_ + _))
s1.outlet ~> zip.in0 s1.outlet ~> zip.in0
s2.outlet.map(_ * 10000) ~> zip.in1 s2.outlet.map(_ * 10000) ~> zip.in1
zip.out SourceShape(zip.out)
} })
val (((f1, f2), (f3, f4)), result) = compositeSource2.toMat(Sink.head)(Keep.both).run() val (((f1, f2), (f3, f4)), result) = compositeSource2.toMat(Sink.head)(Keep.both).run()

View file

@ -3,7 +3,7 @@
*/ */
package akka.stream.scaladsl package akka.stream.scaladsl
import akka.stream.{ ActorMaterializer, ActorMaterializerSettings, Inlet, Outlet } import akka.stream._
import scala.concurrent.duration._ import scala.concurrent.duration._
@ -33,7 +33,7 @@ class GraphMergeSpec extends TwoStreamsSetup {
val source3 = Source(List[Int]()) val source3 = Source(List[Int]())
val probe = TestSubscriber.manualProbe[Int]() val probe = TestSubscriber.manualProbe[Int]()
FlowGraph.closed() { implicit b RunnableGraph.fromGraph(FlowGraph.create() { implicit b
val m1 = b.add(Merge[Int](2)) val m1 = b.add(Merge[Int](2))
val m2 = b.add(Merge[Int](2)) val m2 = b.add(Merge[Int](2))
@ -43,7 +43,8 @@ class GraphMergeSpec extends TwoStreamsSetup {
source2 ~> m1.in(1) source2 ~> m1.in(1)
source3 ~> m2.in(1) source3 ~> m2.in(1)
}.run() ClosedShape
}).run()
val subscription = probe.expectSubscription() val subscription = probe.expectSubscription()
@ -67,7 +68,7 @@ class GraphMergeSpec extends TwoStreamsSetup {
val probe = TestSubscriber.manualProbe[Int]() val probe = TestSubscriber.manualProbe[Int]()
FlowGraph.closed() { implicit b RunnableGraph.fromGraph(FlowGraph.create() { implicit b
val merge = b.add(Merge[Int](6)) val merge = b.add(Merge[Int](6))
source1 ~> merge.in(0) source1 ~> merge.in(0)
@ -78,7 +79,8 @@ class GraphMergeSpec extends TwoStreamsSetup {
source6 ~> merge.in(5) source6 ~> merge.in(5)
merge.out ~> Sink(probe) merge.out ~> Sink(probe)
}.run() ClosedShape
}).run()
val subscription = probe.expectSubscription() val subscription = probe.expectSubscription()
@ -152,13 +154,14 @@ class GraphMergeSpec extends TwoStreamsSetup {
val src1 = Source.subscriber[Int] val src1 = Source.subscriber[Int]
val src2 = 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) (s1, s2)
val merge = b.add(Merge[Int](2)) val merge = b.add(Merge[Int](2))
s1.outlet ~> merge.in(0) s1.outlet ~> merge.in(0)
s2.outlet ~> merge.in(1) s2.outlet ~> merge.in(1)
merge.out ~> Sink(down) merge.out ~> Sink(down)
}.run() ClosedShape
}).run()
val downstream = down.expectSubscription() val downstream = down.expectSubscription()
downstream.cancel() downstream.cancel()

View file

@ -3,13 +3,10 @@ package akka.stream.scaladsl
import scala.collection.immutable import scala.collection.immutable
import scala.concurrent.{ Future, Await } import scala.concurrent.{ Future, Await }
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.stream.ActorMaterializer import akka.stream._
import akka.stream.ActorMaterializerSettings
import akka.stream.testkit._ import akka.stream.testkit._
import akka.util.ByteString import akka.util.ByteString
import akka.stream.{ Inlet, Outlet, Shape, Graph }
import org.scalactic.ConversionCheckedTripleEquals import org.scalactic.ConversionCheckedTripleEquals
import akka.stream.Attributes
object GraphOpsIntegrationSpec { object GraphOpsIntegrationSpec {
import FlowGraph.Implicits._ import FlowGraph.Implicits._
@ -33,7 +30,7 @@ object GraphOpsIntegrationSpec {
} }
def apply[In, Out](pipeline: Flow[In, Out, _]): Graph[ShufflePorts[In, Out], Unit] = { 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 merge = b.add(Merge[In](2))
val balance = b.add(Balance[Out](2)) val balance = b.add(Balance[Out](2))
merge.out ~> pipeline ~> balance.in merge.out ~> pipeline ~> balance.in
@ -57,7 +54,7 @@ class GraphOpsIntegrationSpec extends AkkaSpec with ConversionCheckedTripleEqual
"FlowGraphs" must { "FlowGraphs" must {
"support broadcast - merge layouts" in { "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) (sink)
val bcast = b.add(Broadcast[Int](2)) val bcast = b.add(Broadcast[Int](2))
val merge = b.add(Merge[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(0) ~> merge.in(0)
bcast.out(1).map(_ + 3) ~> merge.in(1) bcast.out(1).map(_ + 3) ~> merge.in(1)
merge.out.grouped(10) ~> sink.inlet merge.out.grouped(10) ~> sink.inlet
}.run() ClosedShape
}).run()
Await.result(resultFuture, 3.seconds).sorted should be(List(1, 2, 3, 4, 5, 6)) Await.result(resultFuture, 3.seconds).sorted should be(List(1, 2, 3, 4, 5, 6))
} }
"support balance - merge (parallelization) layouts" in { "support balance - merge (parallelization) layouts" in {
val elements = 0 to 10 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) (sink)
val balance = b.add(Balance[Int](5)) val balance = b.add(Balance[Int](5))
val merge = b.add(Merge[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) for (i 0 until 5) balance.out(i) ~> merge.in(i)
merge.out.grouped(elements.size * 2) ~> sink.inlet merge.out.grouped(elements.size * 2) ~> sink.inlet
}.run() ClosedShape
}).run()
Await.result(out, 3.seconds).sorted should be(elements) 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 // see https://en.wikipedia.org/wiki/Topological_sorting#mediaviewer/File:Directed_acyclic_graph.png
val seqSink = Sink.head[Seq[Int]] 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) (sink2, sink9, sink10)
val b3 = b.add(Broadcast[Int](2)) val b3 = b.add(Broadcast[Int](2))
val b7 = 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 m9.out.grouped(1000) ~> sink9.inlet
m10.out.grouped(1000) ~> sink10.inlet m10.out.grouped(1000) ~> sink10.inlet
}.run() ClosedShape
}).run()
Await.result(resultFuture2, 3.seconds).sorted should be(List(5, 7)) Await.result(resultFuture2, 3.seconds).sorted should be(List(5, 7))
Await.result(resultFuture9, 3.seconds).sorted should be(List(3, 5, 7, 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 { "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) (sink)
val bcast = b.add(Broadcast[Int](2)) val bcast = b.add(Broadcast[Int](2))
val merge = b.add(Merge[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(0) ~> merge.in(0)
bcast.out(1).map(_ + 3) ~> merge.in(1) bcast.out(1).map(_ + 3) ~> merge.in(1)
merge.out.grouped(10) ~> sink.inlet merge.out.grouped(10) ~> sink.inlet
}.run() ClosedShape
}).run()
Await.result(resultFuture, 3.seconds) should contain theSameElementsAs (Seq(2, 4, 6, 5, 7, 9)) 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 p = Source(List(1, 2, 3)).runWith(Sink.publisher)
val s = TestSubscriber.manualProbe[Int] val s = TestSubscriber.manualProbe[Int]
val flow = Flow[Int].map(_ * 2) val flow = Flow[Int].map(_ * 2)
FlowGraph.closed() { implicit builder RunnableGraph.fromGraph(FlowGraph.create() { implicit builder
Source(p) ~> flow ~> Sink(s) Source(p) ~> flow ~> Sink(s)
}.run() ClosedShape
}).run()
val sub = s.expectSubscription() val sub = s.expectSubscription()
sub.request(10) sub.request(10)
s.expectNext(1 * 2) s.expectNext(1 * 2)
@ -171,7 +173,7 @@ class GraphOpsIntegrationSpec extends AkkaSpec with ConversionCheckedTripleEqual
"be possible to use as lego bricks" in { "be possible to use as lego bricks" in {
val shuffler = Shuffle(Flow[Int].map(_ + 1)) 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) (s1, s2, s3, sink)
val merge = b.add(Merge[Int](2)) val merge = b.add(Merge[Int](2))
@ -188,7 +190,8 @@ class GraphOpsIntegrationSpec extends AkkaSpec with ConversionCheckedTripleEqual
s3.out2 ~> merge.in(1) s3.out2 ~> merge.in(1)
merge.out.grouped(1000) ~> sink merge.out.grouped(1000) ~> sink
}.run() ClosedShape
}).run()
val result = Await.result(f, 3.seconds) val result = Await.result(f, 3.seconds)

View file

@ -1,6 +1,6 @@
package akka.stream.scaladsl package akka.stream.scaladsl
import akka.stream.{ ActorMaterializer, ActorMaterializerSettings, FlowShape } import akka.stream.{ ClosedShape, ActorMaterializer, ActorMaterializerSettings, FlowShape }
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
import scala.concurrent.{ Await, Future } import scala.concurrent.{ Await, Future }
@ -18,7 +18,7 @@ class GraphPartialSpec extends AkkaSpec {
import FlowGraph.Implicits._ import FlowGraph.Implicits._
"be able to build and reuse simple partial graphs" in { "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 bcast = b.add(Broadcast[Int](2))
val zip = b.add(ZipWith((a: Int, b: Int) a + b)) val zip = b.add(ZipWith((a: Int, b: Int) a + b))
@ -27,18 +27,19 @@ class GraphPartialSpec extends AkkaSpec {
FlowShape(bcast.in, zip.out) 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) (d1, d2, sink)
Source(List(1, 2, 3)) ~> d1.inlet Source(List(1, 2, 3)) ~> d1.inlet
d1.outlet ~> d2.inlet d1.outlet ~> d2.inlet
d2.outlet.grouped(100) ~> sink.inlet d2.outlet.grouped(100) ~> sink.inlet
}.run() ClosedShape
}).run()
Await.result(result, 3.seconds) should be(List(4, 8, 12)) Await.result(result, 3.seconds) should be(List(4, 8, 12))
} }
"be able to build and reuse simple materializing partial graphs" in { "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 sink
val bcast = b.add(Broadcast[Int](3)) val bcast = b.add(Broadcast[Int](3))
val zip = b.add(ZipWith((a: Int, b: Int) a + b)) val zip = b.add(ZipWith((a: Int, b: Int) a + b))
@ -49,12 +50,13 @@ class GraphPartialSpec extends AkkaSpec {
FlowShape(bcast.in, zip.out) 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) (d1, d2, sink)
Source(List(1, 2, 3)) ~> d1.inlet Source(List(1, 2, 3)) ~> d1.inlet
d1.outlet ~> d2.inlet d1.outlet ~> d2.inlet
d2.outlet.grouped(100) ~> sink.inlet d2.outlet.grouped(100) ~> sink.inlet
}.run() ClosedShape
}).run()
Await.result(result, 3.seconds) should be(List(4, 8, 12)) Await.result(result, 3.seconds) should be(List(4, 8, 12))
Await.result(sub1, 3.seconds) should be(List(1, 2, 3)) 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 { "be able to build and reuse complex materializing partial graphs" in {
val summer = Sink.fold[Int, Int](0)(_ + _) 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) (s1, s2)
val bcast = b.add(Broadcast[Int](3)) val bcast = b.add(Broadcast[Int](3))
val bcast2 = b.add(Broadcast[Int](2)) val bcast2 = b.add(Broadcast[Int](2))
@ -80,12 +82,13 @@ class GraphPartialSpec extends AkkaSpec {
FlowShape(bcast.in, bcast2.out(1)) 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) (d1, d2, sink)
Source(List(1, 2, 3)) ~> d1.inlet Source(List(1, 2, 3)) ~> d1.inlet
d1.outlet ~> d2.inlet d1.outlet ~> d2.inlet
d2.outlet.grouped(100) ~> sink.inlet d2.outlet.grouped(100) ~> sink.inlet
}.run() ClosedShape
}).run()
Await.result(result, 3.seconds) should be(List(4, 8, 12)) Await.result(result, 3.seconds) should be(List(4, 8, 12))
Await.result(sub1._1, 3.seconds) should be(6) 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 { "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 flow
FlowShape(flow.inlet, flow.outlet) 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) (sink, flow)
import FlowGraph.Implicits._ import FlowGraph.Implicits._
Source.single(0) ~> flow.inlet Source.single(0) ~> flow.inlet
flow.outlet ~> sink.inlet flow.outlet ~> sink.inlet
}.run() ClosedShape
}).run()
Await.result(fut, 3.seconds) should be(1) Await.result(fut, 3.seconds) should be(1)

View file

@ -32,7 +32,7 @@ class GraphPreferredMergeSpec extends TwoStreamsSetup {
val preferred = Source(Stream.fill(numElements)(1)) val preferred = Source(Stream.fill(numElements)(1))
val aux = Source(Stream.fill(numElements)(2)) 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 sink
val merge = b.add(MergePreferred[Int](3)) val merge = b.add(MergePreferred[Int](3))
preferred ~> merge.preferred preferred ~> merge.preferred
@ -41,13 +41,14 @@ class GraphPreferredMergeSpec extends TwoStreamsSetup {
aux ~> merge.in(0) aux ~> merge.in(0)
aux ~> merge.in(1) aux ~> merge.in(1)
aux ~> merge.in(2) aux ~> merge.in(2)
}.run() ClosedShape
}).run()
Await.result(result, 3.seconds).filter(_ == 1).size should be(numElements) Await.result(result, 3.seconds).filter(_ == 1).size should be(numElements)
} }
"eventually pass through all elements" in { "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 sink
val merge = b.add(MergePreferred[Int](3)) val merge = b.add(MergePreferred[Int](3))
Source(1 to 100) ~> merge.preferred Source(1 to 100) ~> merge.preferred
@ -56,7 +57,8 @@ class GraphPreferredMergeSpec extends TwoStreamsSetup {
Source(101 to 200) ~> merge.in(0) Source(101 to 200) ~> merge.in(0)
Source(201 to 300) ~> merge.in(1) Source(201 to 300) ~> merge.in(1)
Source(301 to 400) ~> merge.in(2) Source(301 to 400) ~> merge.in(2)
}.run() ClosedShape
}).run()
Await.result(result, 3.seconds).toSet should ===((1 to 400).toSet) 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) val s = Source(0 to 3)
(the[IllegalArgumentException] thrownBy { (the[IllegalArgumentException] thrownBy {
val g = FlowGraph.closed() { implicit b val g = RunnableGraph.fromGraph(FlowGraph.create() { implicit b
val merge = b.add(MergePreferred[Int](1)) val merge = b.add(MergePreferred[Int](1))
s ~> merge.preferred s ~> merge.preferred
@ -73,7 +75,8 @@ class GraphPreferredMergeSpec extends TwoStreamsSetup {
s ~> merge.in(0) s ~> merge.in(0)
merge.out ~> Sink.head[Int] merge.out ~> Sink.head[Int]
} ClosedShape
})
}).getMessage should include("[MergePreferred.preferred] is already connected") }).getMessage should include("[MergePreferred.preferred] is already connected")
} }

View file

@ -23,12 +23,12 @@ object GraphStageTimersSpec {
class SideChannel { class SideChannel {
@volatile var asyncCallback: AsyncCallback[Any] = _ @volatile var asyncCallback: AsyncCallback[Any] = _
@volatile var stopPromise: Promise[Unit] = _ @volatile var stopPromise: Promise[Option[Nothing]] = _
def isReady: Boolean = asyncCallback ne null def isReady: Boolean = asyncCallback ne null
def !(msg: Any) = asyncCallback.invoke(msg) 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 = { def setupIsolatedStage: SideChannel = {
val channel = new 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 channel.stopPromise = stopPromise
awaitCond(channel.isReady) awaitCond(channel.isReady)
channel channel

View file

@ -5,8 +5,7 @@ package akka.stream.scaladsl
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.stream.{ OverflowStrategy, ActorMaterializerSettings } import akka.stream.{ ClosedShape, OverflowStrategy, ActorMaterializerSettings, ActorMaterializer }
import akka.stream.ActorMaterializer
import akka.stream.testkit._ import akka.stream.testkit._
import akka.stream.testkit.Utils._ import akka.stream.testkit.Utils._
@ -24,12 +23,13 @@ class GraphUnzipSpec extends AkkaSpec {
val c1 = TestSubscriber.manualProbe[Int]() val c1 = TestSubscriber.manualProbe[Int]()
val c2 = TestSubscriber.manualProbe[String]() val c2 = TestSubscriber.manualProbe[String]()
FlowGraph.closed() { implicit b RunnableGraph.fromGraph(FlowGraph.create() { implicit b
val unzip = b.add(Unzip[Int, String]()) val unzip = b.add(Unzip[Int, String]())
Source(List(1 -> "a", 2 -> "b", 3 -> "c")) ~> unzip.in Source(List(1 -> "a", 2 -> "b", 3 -> "c")) ~> unzip.in
unzip.out1 ~> Flow[String].buffer(16, OverflowStrategy.backpressure) ~> Sink(c2) unzip.out1 ~> Flow[String].buffer(16, OverflowStrategy.backpressure) ~> Sink(c2)
unzip.out0 ~> Flow[Int].buffer(16, OverflowStrategy.backpressure).map(_ * 2) ~> Sink(c1) unzip.out0 ~> Flow[Int].buffer(16, OverflowStrategy.backpressure).map(_ * 2) ~> Sink(c1)
}.run() ClosedShape
}).run()
val sub1 = c1.expectSubscription() val sub1 = c1.expectSubscription()
val sub2 = c2.expectSubscription() val sub2 = c2.expectSubscription()
@ -53,12 +53,13 @@ class GraphUnzipSpec extends AkkaSpec {
val c1 = TestSubscriber.manualProbe[Int]() val c1 = TestSubscriber.manualProbe[Int]()
val c2 = TestSubscriber.manualProbe[String]() val c2 = TestSubscriber.manualProbe[String]()
FlowGraph.closed() { implicit b RunnableGraph.fromGraph(FlowGraph.create() { implicit b
val unzip = b.add(Unzip[Int, String]()) val unzip = b.add(Unzip[Int, String]())
Source(List(1 -> "a", 2 -> "b", 3 -> "c")) ~> unzip.in Source(List(1 -> "a", 2 -> "b", 3 -> "c")) ~> unzip.in
unzip.out0 ~> Sink(c1) unzip.out0 ~> Sink(c1)
unzip.out1 ~> Sink(c2) unzip.out1 ~> Sink(c2)
}.run() ClosedShape
}).run()
val sub1 = c1.expectSubscription() val sub1 = c1.expectSubscription()
val sub2 = c2.expectSubscription() val sub2 = c2.expectSubscription()
@ -74,12 +75,13 @@ class GraphUnzipSpec extends AkkaSpec {
val c1 = TestSubscriber.manualProbe[Int]() val c1 = TestSubscriber.manualProbe[Int]()
val c2 = TestSubscriber.manualProbe[String]() val c2 = TestSubscriber.manualProbe[String]()
FlowGraph.closed() { implicit b RunnableGraph.fromGraph(FlowGraph.create() { implicit b
val unzip = b.add(Unzip[Int, String]()) val unzip = b.add(Unzip[Int, String]())
Source(List(1 -> "a", 2 -> "b", 3 -> "c")) ~> unzip.in Source(List(1 -> "a", 2 -> "b", 3 -> "c")) ~> unzip.in
unzip.out0 ~> Sink(c1) unzip.out0 ~> Sink(c1)
unzip.out1 ~> Sink(c2) unzip.out1 ~> Sink(c2)
}.run() ClosedShape
}).run()
val sub1 = c1.expectSubscription() val sub1 = c1.expectSubscription()
val sub2 = c2.expectSubscription() val sub2 = c2.expectSubscription()
@ -96,12 +98,13 @@ class GraphUnzipSpec extends AkkaSpec {
val c1 = TestSubscriber.manualProbe[Int]() val c1 = TestSubscriber.manualProbe[Int]()
val c2 = TestSubscriber.manualProbe[String]() val c2 = TestSubscriber.manualProbe[String]()
FlowGraph.closed() { implicit b RunnableGraph.fromGraph(FlowGraph.create() { implicit b
val unzip = b.add(Unzip[Int, String]()) val unzip = b.add(Unzip[Int, String]())
Source(p1.getPublisher) ~> unzip.in Source(p1.getPublisher) ~> unzip.in
unzip.out0 ~> Sink(c1) unzip.out0 ~> Sink(c1)
unzip.out1 ~> Sink(c2) unzip.out1 ~> Sink(c2)
}.run() ClosedShape
}).run()
val p1Sub = p1.expectSubscription() val p1Sub = p1.expectSubscription()
val sub1 = c1.expectSubscription() val sub1 = c1.expectSubscription()
@ -122,14 +125,15 @@ class GraphUnzipSpec extends AkkaSpec {
"work with zip" in assertAllStagesStopped { "work with zip" in assertAllStagesStopped {
val c1 = TestSubscriber.manualProbe[(Int, String)]() val c1 = TestSubscriber.manualProbe[(Int, String)]()
FlowGraph.closed() { implicit b RunnableGraph.fromGraph(FlowGraph.create() { implicit b
val zip = b.add(Zip[Int, String]()) val zip = b.add(Zip[Int, String]())
val unzip = b.add(Unzip[Int, String]()) val unzip = b.add(Unzip[Int, String]())
Source(List(1 -> "a", 2 -> "b", 3 -> "c")) ~> unzip.in Source(List(1 -> "a", 2 -> "b", 3 -> "c")) ~> unzip.in
unzip.out0 ~> zip.in0 unzip.out0 ~> zip.in0
unzip.out1 ~> zip.in1 unzip.out1 ~> zip.in1
zip.out ~> Sink(c1) zip.out ~> Sink(c1)
}.run() ClosedShape
}).run()
val sub1 = c1.expectSubscription() val sub1 = c1.expectSubscription()
sub1.request(5) sub1.request(5)

View file

@ -48,14 +48,15 @@ class GraphUnzipWithSpec extends AkkaSpec {
val leftSubscriber = TestSubscriber.probe[LeftOutput]() val leftSubscriber = TestSubscriber.probe[LeftOutput]()
val rightSubscriber = TestSubscriber.probe[RightOutput]() val rightSubscriber = TestSubscriber.probe[RightOutput]()
FlowGraph.closed() { implicit b RunnableGraph.fromGraph(FlowGraph.create() { implicit b
val f = fixture(b) val f = fixture(b)
Source(p) ~> f.in Source(p) ~> f.in
f.left ~> Sink(leftSubscriber) f.left ~> Sink(leftSubscriber)
f.right ~> Sink(rightSubscriber) f.right ~> Sink(rightSubscriber)
}.run() ClosedShape
}).run()
(leftSubscriber, rightSubscriber) (leftSubscriber, rightSubscriber)
} }
@ -96,13 +97,15 @@ class GraphUnzipWithSpec extends AkkaSpec {
val leftProbe = TestSubscriber.manualProbe[LeftOutput]() val leftProbe = TestSubscriber.manualProbe[LeftOutput]()
val rightProbe = TestSubscriber.manualProbe[RightOutput]() val rightProbe = TestSubscriber.manualProbe[RightOutput]()
FlowGraph.closed() { implicit b RunnableGraph.fromGraph(FlowGraph.create() { implicit b
val unzip = b.add(UnzipWith(f)) val unzip = b.add(UnzipWith(f))
Source(1 to 4) ~> unzip.in Source(1 to 4) ~> unzip.in
unzip.out0 ~> Flow[LeftOutput].buffer(4, OverflowStrategy.backpressure) ~> Sink(leftProbe) unzip.out0 ~> Flow[LeftOutput].buffer(4, OverflowStrategy.backpressure) ~> Sink(leftProbe)
unzip.out1 ~> Flow[RightOutput].buffer(4, OverflowStrategy.backpressure) ~> Sink(rightProbe) unzip.out1 ~> Flow[RightOutput].buffer(4, OverflowStrategy.backpressure) ~> Sink(rightProbe)
}.run()
ClosedShape
}).run()
val leftSubscription = leftProbe.expectSubscription() val leftSubscription = leftProbe.expectSubscription()
val rightSubscription = rightProbe.expectSubscription() val rightSubscription = rightProbe.expectSubscription()
@ -144,14 +147,16 @@ class GraphUnzipWithSpec extends AkkaSpec {
val leftProbe = TestSubscriber.manualProbe[LeftOutput]() val leftProbe = TestSubscriber.manualProbe[LeftOutput]()
val rightProbe = TestSubscriber.manualProbe[RightOutput]() 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))) val unzip = b.add(UnzipWith[Int, Int, String]((b: Int) (1 / b, 1 + "/" + b)))
Source(-2 to 2) ~> unzip.in Source(-2 to 2) ~> unzip.in
unzip.out0 ~> Sink(leftProbe) unzip.out0 ~> Sink(leftProbe)
unzip.out1 ~> Sink(rightProbe) unzip.out1 ~> Sink(rightProbe)
}.run()
ClosedShape
}).run()
val leftSubscription = leftProbe.expectSubscription() val leftSubscription = leftProbe.expectSubscription()
val rightSubscription = rightProbe.expectSubscription() val rightSubscription = rightProbe.expectSubscription()
@ -187,7 +192,7 @@ class GraphUnzipWithSpec extends AkkaSpec {
case class Person(name: String, surname: String, int: Int) 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)) val unzip = b.add(UnzipWith((a: Person) Person.unapply(a).get))
Source.single(Person("Caplin", "Capybara", 3)) ~> unzip.in Source.single(Person("Caplin", "Capybara", 3)) ~> unzip.in
@ -195,7 +200,9 @@ class GraphUnzipWithSpec extends AkkaSpec {
unzip.out0 ~> Sink(probe0) unzip.out0 ~> Sink(probe0)
unzip.out1 ~> Sink(probe1) unzip.out1 ~> Sink(probe1)
unzip.out2 ~> Sink(probe2) unzip.out2 ~> Sink(probe2)
}.run()
ClosedShape
}).run()
val subscription0 = probe0.expectSubscription() val subscription0 = probe0.expectSubscription()
val subscription1 = probe1.expectSubscription() val subscription1 = probe1.expectSubscription()
@ -221,7 +228,7 @@ class GraphUnzipWithSpec extends AkkaSpec {
val probe15 = TestSubscriber.manualProbe[String]() val probe15 = TestSubscriber.manualProbe[String]()
val probe19 = TestSubscriber.manualProbe[String]() val probe19 = TestSubscriber.manualProbe[String]()
FlowGraph.closed() { implicit b RunnableGraph.fromGraph(FlowGraph.create() { implicit b
val split20 = (a: (List[Int])) val split20 = (a: (List[Int]))
(a(0), a(0).toString, (a(0), a(0).toString,
@ -268,7 +275,8 @@ class GraphUnzipWithSpec extends AkkaSpec {
unzip.out19 ~> Sink(probe19) unzip.out19 ~> Sink(probe19)
}.run() ClosedShape
}).run()
probe0.expectSubscription().request(1) probe0.expectSubscription().request(1)
probe5.expectSubscription().request(1) probe5.expectSubscription().request(1)

View file

@ -25,14 +25,16 @@ class GraphZipSpec extends TwoStreamsSetup {
"work in the happy case" in assertAllStagesStopped { "work in the happy case" in assertAllStagesStopped {
val probe = TestSubscriber.manualProbe[(Int, String)]() val probe = TestSubscriber.manualProbe[(Int, String)]()
FlowGraph.closed() { implicit b RunnableGraph.fromGraph(FlowGraph.create() { implicit b
val zip = b.add(Zip[Int, String]()) val zip = b.add(Zip[Int, String]())
Source(1 to 4) ~> zip.in0 Source(1 to 4) ~> zip.in0
Source(List("A", "B", "C", "D", "E", "F")) ~> zip.in1 Source(List("A", "B", "C", "D", "E", "F")) ~> zip.in1
zip.out ~> Sink(probe) zip.out ~> Sink(probe)
}.run()
ClosedShape
}).run()
val subscription = probe.expectSubscription() val subscription = probe.expectSubscription()

View file

@ -21,13 +21,15 @@ class GraphZipWithSpec extends TwoStreamsSetup {
"work in the happy case" in { "work in the happy case" in {
val probe = TestSubscriber.manualProbe[Outputs]() val probe = TestSubscriber.manualProbe[Outputs]()
FlowGraph.closed() { implicit b RunnableGraph.fromGraph(FlowGraph.create() { implicit b
val zip = b.add(ZipWith((_: Int) + (_: Int))) val zip = b.add(ZipWith((_: Int) + (_: Int)))
Source(1 to 4) ~> zip.in0 Source(1 to 4) ~> zip.in0
Source(10 to 40 by 10) ~> zip.in1 Source(10 to 40 by 10) ~> zip.in1
zip.out ~> Sink(probe) zip.out ~> Sink(probe)
}.run()
ClosedShape
}).run()
val subscription = probe.expectSubscription() val subscription = probe.expectSubscription()
@ -46,14 +48,16 @@ class GraphZipWithSpec extends TwoStreamsSetup {
"work in the sad case" in { "work in the sad case" in {
val probe = TestSubscriber.manualProbe[Outputs]() 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))) val zip = b.add(ZipWith[Int, Int, Int]((_: Int) / (_: Int)))
Source(1 to 4) ~> zip.in0 Source(1 to 4) ~> zip.in0
Source(-2 to 2) ~> zip.in1 Source(-2 to 2) ~> zip.in1
zip.out ~> Sink(probe) zip.out ~> Sink(probe)
}.run()
ClosedShape
}).run()
val subscription = probe.expectSubscription() val subscription = probe.expectSubscription()
@ -107,7 +111,7 @@ class GraphZipWithSpec extends TwoStreamsSetup {
case class Person(name: String, surname: String, int: Int) 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 _)) val zip = b.add(ZipWith(Person.apply _))
Source.single("Caplin") ~> zip.in0 Source.single("Caplin") ~> zip.in0
@ -115,7 +119,9 @@ class GraphZipWithSpec extends TwoStreamsSetup {
Source.single(3) ~> zip.in2 Source.single(3) ~> zip.in2
zip.out ~> Sink(probe) zip.out ~> Sink(probe)
}.run()
ClosedShape
}).run()
val subscription = probe.expectSubscription() val subscription = probe.expectSubscription()
@ -128,7 +134,7 @@ class GraphZipWithSpec extends TwoStreamsSetup {
"work with up to 22 inputs" in { "work with up to 22 inputs" in {
val probe = TestSubscriber.manualProbe[String]() 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, 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) 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 Source.single(19) ~> zip.in18
zip.out ~> Sink(probe) zip.out ~> Sink(probe)
}.run()
ClosedShape
}).run()
val subscription = probe.expectSubscription() val subscription = probe.expectSubscription()

View file

@ -65,7 +65,7 @@ class One2OneBidiFlowSpec extends AkkaSpec with ConversionCheckedTripleEquals {
Source(1 to 1000) Source(1 to 1000)
.log("", seen.set) .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) .runWith(Sink.ignore)
Thread.sleep(50) Thread.sleep(50)
@ -82,6 +82,6 @@ class One2OneBidiFlowSpec extends AkkaSpec with ConversionCheckedTripleEquals {
val outIn = TestPublisher.probe[Int]() val outIn = TestPublisher.probe[Int]()
val outOut = TestSubscriber.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))
} }
} }

View file

@ -3,7 +3,7 @@
*/ */
package akka.stream.scaladsl package akka.stream.scaladsl
import akka.stream.ActorMaterializer import akka.stream.{ ClosedShape, ActorMaterializer }
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.Utils._ import akka.stream.testkit.Utils._
@ -19,7 +19,7 @@ class PublisherSinkSpec extends AkkaSpec {
"be unique when created twice" in assertAllStagesStopped { "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) (p1, p2)
import FlowGraph.Implicits._ import FlowGraph.Implicits._
@ -28,7 +28,8 @@ class PublisherSinkSpec extends AkkaSpec {
Source(0 to 5) ~> bcast.in Source(0 to 5) ~> bcast.in
bcast.out(0).map(_ * 2) ~> p1.inlet bcast.out(0).map(_ * 2) ~> p1.inlet
bcast.out(1) ~> p2.inlet bcast.out(1) ~> p2.inlet
}.run() ClosedShape
}).run()
val f1 = Source(pub1).map(identity).runFold(0)(_ + _) val f1 = Source(pub1).map(identity).runFold(0)(_ + _)
val f2 = Source(pub2).map(identity).runFold(0)(_ + _) val f2 = Source(pub2).map(identity).runFold(0)(_ + _)

View file

@ -16,43 +16,48 @@ class ReverseArrowSpec extends AkkaSpec with ConversionCheckedTripleEquals {
"Reverse Arrows in the Graph DSL" must { "Reverse Arrows in the Graph DSL" must {
"work from Inlets" in { "work from Inlets" in {
Await.result(FlowGraph.closed(sink) { implicit b Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b
s s
s.inlet <~ source s.inlet <~ source
}.run(), 1.second) should ===(Seq(1, 2, 3)) ClosedShape
}).run(), 1.second) should ===(Seq(1, 2, 3))
} }
"work from SinkShape" in { "work from SinkShape" in {
Await.result(FlowGraph.closed(sink) { implicit b Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b
s s
s <~ source s <~ source
}.run(), 1.second) should ===(Seq(1, 2, 3)) ClosedShape
}).run(), 1.second) should ===(Seq(1, 2, 3))
} }
"work from Sink" in { "work from Sink" in {
val sub = TestSubscriber.manualProbe[Int] val sub = TestSubscriber.manualProbe[Int]
FlowGraph.closed() { implicit b RunnableGraph.fromGraph(FlowGraph.create() { implicit b
Sink(sub) <~ source Sink(sub) <~ source
}.run() ClosedShape
}).run()
sub.expectSubscription().request(10) sub.expectSubscription().request(10)
sub.expectNext(1, 2, 3) sub.expectNext(1, 2, 3)
sub.expectComplete() sub.expectComplete()
} }
"not work from Outlets" in { "not work from Outlets" in {
FlowGraph.closed() { implicit b RunnableGraph.fromGraph(FlowGraph.create() { implicit b
val o: Outlet[Int] = b.add(source) val o: Outlet[Int] = b.add(source)
"o <~ source" shouldNot compile "o <~ source" shouldNot compile
sink <~ o sink <~ o
} ClosedShape
})
} }
"not work from SourceShape" in { "not work from SourceShape" in {
FlowGraph.closed() { implicit b RunnableGraph.fromGraph(FlowGraph.create() { implicit b
val o: SourceShape[Int] = b.add(source) val o: SourceShape[Int] = b.add(source)
"o <~ source" shouldNot compile "o <~ source" shouldNot compile
sink <~ o sink <~ o
} ClosedShape
})
} }
"not work from Source" in { "not work from Source" in {
@ -60,114 +65,127 @@ class ReverseArrowSpec extends AkkaSpec with ConversionCheckedTripleEquals {
} }
"work from FlowShape" in { "work from FlowShape" in {
Await.result(FlowGraph.closed(sink) { implicit b Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b
s s
val f: FlowShape[Int, Int] = b.add(Flow[Int]) val f: FlowShape[Int, Int] = b.add(Flow[Int])
f <~ source f <~ source
f ~> s f ~> s
}.run(), 1.second) should ===(Seq(1, 2, 3)) ClosedShape
}).run(), 1.second) should ===(Seq(1, 2, 3))
} }
"work from UniformFanInShape" in { "work from UniformFanInShape" in {
Await.result(FlowGraph.closed(sink) { implicit b Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b
s s
val f: UniformFanInShape[Int, Int] = b.add(Merge[Int](1)) val f: UniformFanInShape[Int, Int] = b.add(Merge[Int](1))
f <~ source f <~ source
f ~> s f ~> s
}.run(), 1.second) should ===(Seq(1, 2, 3)) ClosedShape
}).run(), 1.second) should ===(Seq(1, 2, 3))
} }
"work from UniformFanOutShape" in { "work from UniformFanOutShape" in {
Await.result(FlowGraph.closed(sink) { implicit b Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b
s s
val f: UniformFanOutShape[Int, Int] = b.add(Broadcast[Int](1)) val f: UniformFanOutShape[Int, Int] = b.add(Broadcast[Int](1))
f <~ source f <~ source
f ~> s f ~> s
}.run(), 1.second) should ===(Seq(1, 2, 3)) ClosedShape
}).run(), 1.second) should ===(Seq(1, 2, 3))
} }
"work towards Outlets" in { "work towards Outlets" in {
Await.result(FlowGraph.closed(sink) { implicit b Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b
s s
val o: Outlet[Int] = b.add(source) val o: Outlet[Int] = b.add(source)
s <~ o s <~ o
}.run(), 1.second) should ===(Seq(1, 2, 3)) ClosedShape
}).run(), 1.second) should ===(Seq(1, 2, 3))
} }
"work towards SourceShape" in { "work towards SourceShape" in {
Await.result(FlowGraph.closed(sink) { implicit b Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b
s s
val o: SourceShape[Int] = b.add(source) val o: SourceShape[Int] = b.add(source)
s <~ o s <~ o
}.run(), 1.second) should ===(Seq(1, 2, 3)) ClosedShape
}).run(), 1.second) should ===(Seq(1, 2, 3))
} }
"work towards Source" in { "work towards Source" in {
Await.result(FlowGraph.closed(sink) { implicit b Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b
s s
s <~ source s <~ source
}.run(), 1.second) should ===(Seq(1, 2, 3)) ClosedShape
}).run(), 1.second) should ===(Seq(1, 2, 3))
} }
"work towards FlowShape" in { "work towards FlowShape" in {
Await.result(FlowGraph.closed(sink) { implicit b Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b
s s
val f: FlowShape[Int, Int] = b.add(Flow[Int]) val f: FlowShape[Int, Int] = b.add(Flow[Int])
s <~ f s <~ f
source ~> f source ~> f
}.run(), 1.second) should ===(Seq(1, 2, 3)) ClosedShape
}).run(), 1.second) should ===(Seq(1, 2, 3))
} }
"work towards UniformFanInShape" in { "work towards UniformFanInShape" in {
Await.result(FlowGraph.closed(sink) { implicit b Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b
s s
val f: UniformFanInShape[Int, Int] = b.add(Merge[Int](1)) val f: UniformFanInShape[Int, Int] = b.add(Merge[Int](1))
s <~ f s <~ f
source ~> 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 { "fail towards already full UniformFanInShape" in {
Await.result(FlowGraph.closed(sink) { implicit b Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b
s s
val f: UniformFanInShape[Int, Int] = b.add(Merge[Int](1)) val f: UniformFanInShape[Int, Int] = b.add(Merge[Int](1))
val src = b.add(source) val src = b.add(source)
src ~> f src ~> f
(the[IllegalArgumentException] thrownBy (s <~ f <~ src)).getMessage should include("no more inlets free") (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 { "work towards UniformFanOutShape" in {
Await.result(FlowGraph.closed(sink) { implicit b Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b
s s
val f: UniformFanOutShape[Int, Int] = b.add(Broadcast[Int](1)) val f: UniformFanOutShape[Int, Int] = b.add(Broadcast[Int](1))
s <~ f s <~ f
source ~> 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 { "fail towards already full UniformFanOutShape" in {
Await.result(FlowGraph.closed(sink) { implicit b Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b
s s
val f: UniformFanOutShape[Int, Int] = b.add(Broadcast[Int](1)) val f: UniformFanOutShape[Int, Int] = b.add(Broadcast[Int](1))
val src = b.add(source) val src = b.add(source)
src ~> f src ~> f
(the[IllegalArgumentException] thrownBy (s <~ f <~ src)).getMessage should include("already connected") (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 { "work across a Flow" in {
Await.result(FlowGraph.closed(sink) { implicit b Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b
s s
s <~ Flow[Int] <~ source 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 { "work across a FlowShape" in {
Await.result(FlowGraph.closed(sink) { implicit b Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b
s s
s <~ b.add(Flow[Int]) <~ source s <~ b.add(Flow[Int]) <~ source
}.run(), 1.second) should ===(Seq(1, 2, 3)) ClosedShape
}).run(), 1.second) should ===(Seq(1, 2, 3))
} }
} }

View file

@ -3,7 +3,7 @@
*/ */
package akka.stream.scaladsl package akka.stream.scaladsl
import akka.stream.ActorMaterializer import akka.stream.{ SinkShape, ActorMaterializer }
import akka.stream.testkit.TestPublisher.ManualProbe import akka.stream.testkit.TestPublisher.ManualProbe
import akka.stream.testkit._ import akka.stream.testkit._
@ -17,11 +17,11 @@ class SinkSpec extends AkkaSpec {
"be composable without importing modules" in { "be composable without importing modules" in {
val probes = Array.fill(3)(TestSubscriber.manualProbe[Int]) 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)) val bcast = b.add(Broadcast[Int](3))
for (i 0 to 2) bcast.out(i).filter(_ == i) ~> Sink(probes(i)) 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) Source(List(0, 1, 2)).runWith(sink)
for (i 0 to 2) { for (i 0 to 2) {
val p = probes(i) val p = probes(i)
@ -34,13 +34,13 @@ class SinkSpec extends AkkaSpec {
"be composable with importing 1 module" in { "be composable with importing 1 module" in {
val probes = Array.fill(3)(TestSubscriber.manualProbe[Int]) 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 s0
val bcast = b.add(Broadcast[Int](3)) val bcast = b.add(Broadcast[Int](3))
bcast.out(0) ~> Flow[Int].filter(_ == 0) ~> s0.inlet bcast.out(0) ~> Flow[Int].filter(_ == 0) ~> s0.inlet
for (i 1 to 2) bcast.out(i).filter(_ == i) ~> Sink(probes(i)) 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) Source(List(0, 1, 2)).runWith(sink)
for (i 0 to 2) { for (i 0 to 2) {
val p = probes(i) val p = probes(i)
@ -53,14 +53,14 @@ class SinkSpec extends AkkaSpec {
"be composable with importing 2 modules" in { "be composable with importing 2 modules" in {
val probes = Array.fill(3)(TestSubscriber.manualProbe[Int]) 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) (s0, s1)
val bcast = b.add(Broadcast[Int](3)) val bcast = b.add(Broadcast[Int](3))
bcast.out(0).filter(_ == 0) ~> s0.inlet bcast.out(0).filter(_ == 0) ~> s0.inlet
bcast.out(1).filter(_ == 1) ~> s1.inlet bcast.out(1).filter(_ == 1) ~> s1.inlet
bcast.out(2).filter(_ == 2) ~> Sink(probes(2)) bcast.out(2).filter(_ == 2) ~> Sink(probes(2))
bcast.in SinkShape(bcast.in)
} })
Source(List(0, 1, 2)).runWith(sink) Source(List(0, 1, 2)).runWith(sink)
for (i 0 to 2) { for (i 0 to 2) {
val p = probes(i) val p = probes(i)
@ -73,14 +73,14 @@ class SinkSpec extends AkkaSpec {
"be composable with importing 3 modules" in { "be composable with importing 3 modules" in {
val probes = Array.fill(3)(TestSubscriber.manualProbe[Int]) 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) (s0, s1, s2)
val bcast = b.add(Broadcast[Int](3)) val bcast = b.add(Broadcast[Int](3))
bcast.out(0).filter(_ == 0) ~> s0.inlet bcast.out(0).filter(_ == 0) ~> s0.inlet
bcast.out(1).filter(_ == 1) ~> s1.inlet bcast.out(1).filter(_ == 1) ~> s1.inlet
bcast.out(2).filter(_ == 2) ~> s2.inlet bcast.out(2).filter(_ == 2) ~> s2.inlet
bcast.in SinkShape(bcast.in)
} })
Source(List(0, 1, 2)).runWith(sink) Source(List(0, 1, 2)).runWith(sink)
for (i 0 to 2) { for (i 0 to 2) {
val p = probes(i) val p = probes(i)

View file

@ -7,10 +7,9 @@ import scala.concurrent.Await
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.util.{ Success, Failure } import scala.util.{ Success, Failure }
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace
import akka.stream.ActorMaterializer import akka.stream.{ SourceShape, ActorMaterializer }
import akka.stream.testkit._ import akka.stream.testkit._
import akka.stream.impl.PublisherSource import akka.stream.impl.{ PublisherSource, ReactiveStreamsCompliance }
import akka.stream.impl.ReactiveStreamsCompliance
class SourceSpec extends AkkaSpec { class SourceSpec extends AkkaSpec {
@ -73,14 +72,14 @@ class SourceSpec extends AkkaSpec {
} }
} }
"Lazy Empty Source" must { "Maybe Source" must {
"complete materialized future when stream cancels" in { "complete materialized future with None when stream cancels" in {
val neverSource = Source.lazyEmpty val neverSource = Source.maybe[Int]
val pubSink = Sink.publisher val pubSink = Sink.publisher[Int]
val (f, neverPub) = neverSource.toMat(pubSink)(Keep.both).run() val (f, neverPub) = neverSource.toMat(pubSink)(Keep.both).run()
val c = TestSubscriber.manualProbe() val c = TestSubscriber.manualProbe[Int]()
neverPub.subscribe(c) neverPub.subscribe(c)
val subs = c.expectSubscription() val subs = c.expectSubscription()
@ -88,24 +87,35 @@ class SourceSpec extends AkkaSpec {
c.expectNoMsg(300.millis) c.expectNoMsg(300.millis)
subs.cancel() subs.cancel()
Await.result(f.future, 500.millis) Await.result(f.future, 500.millis) shouldEqual None
} }
"allow external triggering of completion" in { "allow external triggering of empty completion" in {
val neverSource = Source.lazyEmpty[Int] val neverSource = Source.maybe[Int].filter(_ false)
val counterSink = Sink.fold[Int, Int](0) { (acc, _) acc + 1 } val counterSink = Sink.fold[Int, Int](0) { (acc, _) acc + 1 }
val (neverPromise, counterFuture) = neverSource.toMat(counterSink)(Keep.both).run() val (neverPromise, counterFuture) = neverSource.toMat(counterSink)(Keep.both).run()
// external cancellation // external cancellation
neverPromise.success(()) neverPromise.trySuccess(None) shouldEqual true
val ready = Await.ready(counterFuture, 500.millis) Await.result(counterFuture, 500.millis) shouldEqual 0
val Success(0) = ready.value.get }
"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 { "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 counterSink = Sink.fold[Int, Int](0) { (acc, _) acc + 1 }
val (neverPromise, counterFuture) = neverSource.toMat(counterSink)(Keep.both).run() val (neverPromise, counterFuture) = neverSource.toMat(counterSink)(Keep.both).run()
@ -126,7 +136,7 @@ class SourceSpec extends AkkaSpec {
val source = Source.subscriber[Int] val source = Source.subscriber[Int]
val out = TestSubscriber.manualProbe[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) (i0, i1, i2, i3, i4)
import FlowGraph.Implicits._ import FlowGraph.Implicits._
val m = b.add(Merge[Int](5)) val m = b.add(Merge[Int](5))
@ -135,8 +145,8 @@ class SourceSpec extends AkkaSpec {
i2.outlet ~> m.in(2) i2.outlet ~> m.in(2)
i3.outlet ~> m.in(3) i3.outlet ~> m.in(3)
i4.outlet ~> m.in(4) i4.outlet ~> m.in(4)
m.out SourceShape(m.out)
}.to(Sink(out)).run() }).to(Sink(out)).run()
for (i 0 to 4) probes(i).subscribe(s(i)) for (i 0 to 4) probes(i).subscribe(s(i))
val sub = out.expectSubscription() val sub = out.expectSubscription()

View file

@ -7,10 +7,9 @@ import akka.actor.Cancellable
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace
import akka.stream.ActorMaterializer import akka.stream.{ ClosedShape, ActorMaterializer, ActorMaterializerSettings }
import akka.stream.testkit._ import akka.stream.testkit._
import akka.stream.testkit.Utils._ import akka.stream.testkit.Utils._
import akka.stream.ActorMaterializerSettings
class TickSourceSpec extends AkkaSpec { class TickSourceSpec extends AkkaSpec {
@ -68,13 +67,14 @@ class TickSourceSpec extends AkkaSpec {
"be usable with zip for a simple form of rate limiting" in { "be usable with zip for a simple form of rate limiting" in {
val c = TestSubscriber.manualProbe[Int]() val c = TestSubscriber.manualProbe[Int]()
FlowGraph.closed() { implicit b RunnableGraph.fromGraph(FlowGraph.create() { implicit b
import FlowGraph.Implicits._ import FlowGraph.Implicits._
val zip = b.add(Zip[Int, String]()) val zip = b.add(Zip[Int, String]())
Source(1 to 100) ~> zip.in0 Source(1 to 100) ~> zip.in0
Source(1.second, 1.second, "tick") ~> zip.in1 Source(1.second, 1.second, "tick") ~> zip.in1
zip.out ~> Flow[(Int, String)].map { case (n, _) n } ~> Sink(c) zip.out ~> Flow[(Int, String)].map { case (n, _) n } ~> Sink(c)
}.run() ClosedShape
}).run()
val sub = c.expectSubscription() val sub = c.expectSubscription()
sub.request(1000) sub.request(1000)

View file

@ -1,48 +0,0 @@
/**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/
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#]) })#
]
}

View file

@ -1,55 +0,0 @@
/**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/
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#]) })#
]
}

View file

@ -7,91 +7,36 @@ import akka.stream.scaladsl
import akka.stream.{ Inlet, Shape, Graph } import akka.stream.{ Inlet, Shape, Graph }
import akka.japi.function import akka.japi.function
trait GraphCreate { private[stream] abstract class GraphCreate {
/**
import language.implicitConversions * Creates a new [[Graph]] of the given [[Shape]] by passing a [[FlowGraph.Builder]] to the given create function.
private implicit def r[M](run: scaladsl.RunnableGraph[M]): RunnableGraph[M] = new RunnableGraphAdapter(run) */
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. * Creates a new [[Graph]] by importing the given graph `g1` and its [[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 create[S1 <: Shape, S <: Shape, M](g1: Graph[S1, M],
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],
block: function.Function2[FlowGraph.Builder[M], S1, S]): Graph[S, 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 * Creates a new [[Graph]] by importing the given graphs and passing their [[Shape]]s
* along with the[[FlowGraph.Builder]] to the given create function. * 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 create[S1 <: Shape, S2 <: Shape, S <: Shape, M1, M2, M](g1: Graph[S1, M1], g2: Graph[S2, M2], combineMat: function.Function2[M1, M2, M],
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],
block: function.Function3[FlowGraph.Builder[M], S1, S2, S]): Graph[S, 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#/** [3..21#/**
* Creates a new fully connected graph by importing the given graphs (using `builder.graph()`) and passing their resulting [[Shape]]s * Creates a new [[Graph]] by importing the given graphs and passing their [[Shape]]s
* along with the[[FlowGraph.Builder]] to the given create function. * 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 create1[[#S1 <: Shape#], S <: Shape, [#M1#], M]([#g1: Graph[S1, M1]#], combineMat: function.Function1[[#M1#], M],
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],
block: function.Function2[FlowGraph.Builder[M], [#S1#], S]): Graph[S, 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#]) }#
] ]
} }

View file

@ -1,48 +0,0 @@
/**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/
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#]) })#
]
}

View file

@ -1,51 +0,0 @@
/**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/
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#]) })#
]
}

View file

@ -1,47 +0,0 @@
/**
* Copyright (C) 2014-2015 Typesafe Inc. <http://www.typesafe.com>
*/
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)
}#
]
}

View file

@ -1,53 +0,0 @@
/**
* Copyright (C) 2014-2015 Typesafe Inc. <http://www.typesafe.com>
*/
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)
}#
]
}

View file

@ -8,24 +8,10 @@ import akka.stream.impl.StreamLayout.Module
import akka.stream.{ Graph, Attributes, Shape } import akka.stream.{ Graph, Attributes, Shape }
trait GraphApply { 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. * 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 builder = new FlowGraph.Builder
val s = buildBlock(builder) val s = buildBlock(builder)
val mod = builder.module.nest().replaceShape(s) 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]] * Creates a new [[Graph]] by importing the given graph `g1` and passing its [[Shape]]
* along with the[[FlowGraph.Builder]] to the given create function. * 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](g1: Graph[Shape, Mat])(buildBlock: FlowGraph.Builder[Mat] ⇒ (g1.Shape) ⇒ Unit): RunnableGraph[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 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] = {
val builder = new FlowGraph.Builder val builder = new FlowGraph.Builder
val s1 = builder.add(g1) val s1 = builder.add(g1)
val s = buildBlock(builder)(s1) val s = buildBlock(builder)(s1)
@ -62,29 +33,13 @@ trait GraphApply {
} }
[2..#/** [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()
}
/** /**
* Creates a new [[Graph]] by importing the given graphs (using `builder.graph()`) and passing their resulting [[Shape]]s * Creates a new [[Graph]] by importing the given graphs and passing their [[Shape]]s
* along with the[[FlowGraph.Builder]] to the given create function. * along with the [[FlowGraph.Builder]] to the given create function.
*
* Partial graphs are allowed to have unconnected ports.
*/ */
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 builder = new FlowGraph.Builder
val curried = combineMat.curried val curried = combineMat.curried
val s##1 = builder.add(g##1, (m##1: M##1) ⇒ curried(m##1)) val s##1 = builder.add(g##1, (m##1: M##1) ⇒ curried(m##1))
@ -97,15 +52,13 @@ trait GraphApply {
}# }#
] ]
} }
/** /**
* INTERNAL API * INTERNAL API
*/ */
private[stream] object GraphApply { 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] { extends Graph[S, Mat] {
override def withAttributes(attr: Attributes): 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)) override def named(name: String): Graph[S, Mat] = withAttributes(Attributes.name(name))
} }
} }

View file

@ -1,50 +0,0 @@
/**
* Copyright (C) 2014-2015 Typesafe Inc. <http://www.typesafe.com>
*/
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)
}#
]
}

View file

@ -1,52 +0,0 @@
/**
* Copyright (C) 2014-2015 Typesafe Inc. <http://www.typesafe.com>
*/
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)
}#
]
}

View file

@ -177,7 +177,7 @@ object ClosedShape extends ClosedShape {
/** /**
* Java API: obtain ClosedShape instance * Java API: obtain ClosedShape instance
*/ */
def getInstance: Shape = this def getInstance: ClosedShape = this
} }
/** /**

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