!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:
parent
0f99a42df9
commit
f29d7affbd
120 changed files with 1535 additions and 1897 deletions
|
|
@ -20,7 +20,7 @@ object MaterializationBenchmark {
|
|||
}
|
||||
|
||||
val graphWithJunctionsBuilder = (numOfJunctions: Int) =>
|
||||
FlowGraph.closed() { implicit b =>
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b =>
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
val broadcast = b.add(Broadcast[Unit](numOfJunctions))
|
||||
|
|
@ -34,39 +34,39 @@ object MaterializationBenchmark {
|
|||
|
||||
Source.single(()) ~> broadcast
|
||||
outlet ~> Sink.ignore
|
||||
}
|
||||
ClosedShape
|
||||
})
|
||||
|
||||
val graphWithNestedImportsBuilder = (numOfNestedGraphs: Int) => {
|
||||
var flow: Graph[FlowShape[Unit, Unit], Unit] = Flow[Unit].map(identity)
|
||||
for (_ <- 1 to numOfNestedGraphs) {
|
||||
flow = FlowGraph.partial(flow) { b ⇒
|
||||
flow = FlowGraph.create(flow) { b ⇒
|
||||
flow ⇒
|
||||
FlowShape(flow.inlet, flow.outlet)
|
||||
}
|
||||
}
|
||||
|
||||
FlowGraph.closed(flow) { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create(flow) { implicit b ⇒
|
||||
flow ⇒
|
||||
import FlowGraph.Implicits._
|
||||
Source.single(()) ~> flow ~> Sink.ignore
|
||||
}
|
||||
ClosedShape
|
||||
})
|
||||
}
|
||||
|
||||
val graphWithImportedFlowBuilder = (numOfFlows: Int) => {
|
||||
val graphWithImportedFlowBuilder = (numOfFlows: Int) =>
|
||||
RunnableGraph.fromGraph(FlowGraph.create(Source.single(())) { implicit b ⇒ source ⇒
|
||||
import FlowGraph.Implicits._
|
||||
val flow = Flow[Unit].map(identity)
|
||||
FlowGraph.closed() { b ⇒
|
||||
val source = b.add(Source.single(()))
|
||||
var outlet = source
|
||||
var outlet: Outlet[Unit] = source.outlet
|
||||
for (i <- 0 until numOfFlows) {
|
||||
val flowShape = b.add(flow)
|
||||
b.addEdge(outlet, flowShape.inlet)
|
||||
outlet ~> flowShape
|
||||
outlet = flowShape.outlet
|
||||
}
|
||||
|
||||
val sink = b.add(Sink.ignore)
|
||||
b.addEdge(outlet, sink)
|
||||
}
|
||||
}
|
||||
outlet ~> Sink.ignore
|
||||
ClosedShape
|
||||
})
|
||||
}
|
||||
|
||||
@State(Scope.Benchmark)
|
||||
|
|
|
|||
|
|
@ -126,8 +126,8 @@ As a first example, let's look at a more complex layout:
|
|||
|
||||
The diagram shows a :class:`RunnableGraph` (remember, if there are no unwired ports, the graph is closed, and therefore
|
||||
can be materialized) that encapsulates a non-trivial stream processing network. It contains fan-in, fan-out stages,
|
||||
directed and non-directed cycles. The ``closed()`` method of the :class:`FlowGraph` factory object allows the creation of a
|
||||
general closed graph. For example the network on the diagram can be realized like this:
|
||||
directed and non-directed cycles. The ``runnable()`` method of the :class:`FlowGraph` factory object allows the creation of a
|
||||
general, closed, and runnable graph. For example the network on the diagram can be realized like this:
|
||||
|
||||
.. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/CompositionDocTest.java#complex-graph
|
||||
|
||||
|
|
@ -140,7 +140,7 @@ It is possible to refer to the ports, so another version might look like this:
|
|||
|
||||
Similar to the case in the first section, so far we have not considered modularity. We created a complex graph, but
|
||||
the layout is flat, not modularized. We will modify our example, and create a reusable component with the graph DSL.
|
||||
The way to do it is to use the ``partial()`` method on :class:`FlowGraph` factory. If we remove the sources and sinks
|
||||
The way to do it is to use the ``create()`` method on :class:`FlowGraph` factory. If we remove the sources and sinks
|
||||
from the previous example, what remains is a partial graph:
|
||||
|
||||
|
|
||||
|
|
@ -283,7 +283,7 @@ Attributes
|
|||
----------
|
||||
|
||||
We have seen that we can use ``named()`` to introduce a nesting level in the fluid DSL (and also explicit nesting by using
|
||||
``partial()`` from :class:`FlowGraph`). Apart from having the effect of adding a nesting level, ``named()`` is actually
|
||||
``create()`` from :class:`FlowGraph`). Apart from having the effect of adding a nesting level, ``named()`` is actually
|
||||
a shorthand for calling ``withAttributes(Attributes.name("someName"))``. Attributes provide a way to fine-tune certain
|
||||
aspects of the materialized running entity. For example buffer sizes can be controlled via attributes (see
|
||||
:ref:`stream-buffers-scala`). When it comes to hierarchic composition, attributes are inherited by nested modules,
|
||||
|
|
|
|||
|
|
@ -85,8 +85,8 @@ Constructing and combining Partial Flow Graphs
|
|||
Sometimes it is not possible (or needed) to construct the entire computation graph in one place, but instead construct
|
||||
all of its different phases in different places and in the end connect them all into a complete graph and run it.
|
||||
|
||||
This can be achieved using ``FlowGraph.factory().partial()`` instead of
|
||||
``FlowGraph.factory().closed()``, which will return a ``Graph`` instead of a
|
||||
This can be achieved using ``FlowGraph.create()`` instead of
|
||||
``FlowGraph.runnable()``, which will return a ``Graph`` instead of a
|
||||
``RunnableGraph``. The reason of representing it as a different type is that a
|
||||
:class:`RunnableGraph` requires all ports to be connected, and if they are not
|
||||
it will throw an exception at construction time, which helps to avoid simple
|
||||
|
|
|
|||
|
|
@ -119,7 +119,7 @@ detail in :ref:`constructing-sources-sinks-flows-from-partial-graphs-java`. Flow
|
|||
.. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/TwitterStreamQuickstartDocTest.java#flow-graph-broadcast
|
||||
|
||||
As you can see, we use graph builder ``b`` to construct the graph using ``UniformFanOutShape`` and ``Flow``s. Once we have the
|
||||
FlowGraph as a result of ``closed()`` method *it is immutable, thread-safe, and freely shareable*. A graph can be ``run()`` directly -
|
||||
FlowGraph as a result of ``runnable()`` method *it is immutable, thread-safe, and freely shareable*. A graph can be ``run()`` directly -
|
||||
assuming all ports (sinks/sources) within a flow have been connected properly. It is possible also to construct several :class:`PartialFlowGraph`s and
|
||||
and then combine them into one fully connected graph. This will be covered in detail in :ref:`partial-flow-graph-java`.
|
||||
|
||||
|
|
|
|||
|
|
@ -44,21 +44,21 @@ object BidiFlowDocSpec {
|
|||
}
|
||||
//#codec-impl
|
||||
|
||||
val codecVerbose = BidiFlow() { b =>
|
||||
val codecVerbose = BidiFlow.fromGraph(FlowGraph.create() { b =>
|
||||
// construct and add the top flow, going outbound
|
||||
val outbound = b.add(Flow[Message].map(toBytes))
|
||||
// construct and add the bottom flow, going inbound
|
||||
val inbound = b.add(Flow[ByteString].map(fromBytes))
|
||||
// fuse them together into a BidiShape
|
||||
BidiShape.fromFlows(outbound, inbound)
|
||||
}
|
||||
})
|
||||
|
||||
// this is the same as the above
|
||||
val codec = BidiFlow(toBytes _, fromBytes _)
|
||||
val codec = BidiFlow.fromFunctions(toBytes _, fromBytes _)
|
||||
//#codec
|
||||
|
||||
//#framing
|
||||
val framing = BidiFlow() { b =>
|
||||
val framing = BidiFlow.fromGraph(FlowGraph.create() { b =>
|
||||
implicit val order = ByteOrder.LITTLE_ENDIAN
|
||||
|
||||
def addLengthHeader(bytes: ByteString) = {
|
||||
|
|
@ -113,18 +113,18 @@ object BidiFlowDocSpec {
|
|||
val outbound = b.add(Flow[ByteString].map(addLengthHeader))
|
||||
val inbound = b.add(Flow[ByteString].transform(() => new FrameParser))
|
||||
BidiShape.fromFlows(outbound, inbound)
|
||||
}
|
||||
})
|
||||
//#framing
|
||||
|
||||
val chopUp = BidiFlow() { b =>
|
||||
val chopUp = BidiFlow.fromGraph(FlowGraph.create() { b =>
|
||||
val f = Flow[ByteString].mapConcat(_.map(ByteString(_)))
|
||||
BidiShape.fromFlows(b.add(f), b.add(f))
|
||||
}
|
||||
})
|
||||
|
||||
val accumulate = BidiFlow() { b =>
|
||||
val accumulate = BidiFlow.fromGraph(FlowGraph.create() { b =>
|
||||
val f = Flow[ByteString].grouped(1000).map(_.fold(ByteString.empty)(_ ++ _))
|
||||
BidiShape.fromFlows(b.add(f), b.add(f))
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
class BidiFlowDocSpec extends AkkaSpec with ConversionCheckedTripleEquals {
|
||||
|
|
|
|||
|
|
@ -77,7 +77,7 @@ class CompositionDocSpec extends AkkaSpec {
|
|||
// format: OFF
|
||||
//#complex-graph
|
||||
import FlowGraph.Implicits._
|
||||
FlowGraph.closed() { implicit builder =>
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit builder =>
|
||||
val A: Outlet[Int] = builder.add(Source.single(0))
|
||||
val B: UniformFanOutShape[Int, Int] = builder.add(Broadcast[Int](2))
|
||||
val C: UniformFanInShape[Int, Int] = builder.add(Merge[Int](2))
|
||||
|
|
@ -90,12 +90,14 @@ class CompositionDocSpec extends AkkaSpec {
|
|||
A ~> B ~> C ~> F
|
||||
B ~> D ~> E ~> F
|
||||
E ~> G
|
||||
}
|
||||
|
||||
ClosedShape
|
||||
})
|
||||
//#complex-graph
|
||||
|
||||
//#complex-graph-alt
|
||||
import FlowGraph.Implicits._
|
||||
FlowGraph.closed() { implicit builder =>
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit builder =>
|
||||
val B = builder.add(Broadcast[Int](2))
|
||||
val C = builder.add(Merge[Int](2))
|
||||
val E = builder.add(Balance[Int](2))
|
||||
|
|
@ -106,7 +108,8 @@ class CompositionDocSpec extends AkkaSpec {
|
|||
|
||||
B.out(1).map(_ + 1) ~> E.in; E.out(0) ~> F.in(1)
|
||||
E.out(1) ~> Sink.foreach(println)
|
||||
}
|
||||
ClosedShape
|
||||
})
|
||||
//#complex-graph-alt
|
||||
// format: ON
|
||||
}
|
||||
|
|
@ -115,7 +118,7 @@ class CompositionDocSpec extends AkkaSpec {
|
|||
// format: OFF
|
||||
//#partial-graph
|
||||
import FlowGraph.Implicits._
|
||||
val partial = FlowGraph.partial() { implicit builder =>
|
||||
val partial = FlowGraph.create() { implicit builder =>
|
||||
val B = builder.add(Broadcast[Int](2))
|
||||
val C = builder.add(Merge[Int](2))
|
||||
val E = builder.add(Balance[Int](2))
|
||||
|
|
@ -137,17 +140,17 @@ class CompositionDocSpec extends AkkaSpec {
|
|||
//#partial-flow-dsl
|
||||
// Convert the partial graph of FlowShape to a Flow to get
|
||||
// access to the fluid DSL (for example to be able to call .filter())
|
||||
val flow = Flow.wrap(partial)
|
||||
val flow = Flow.fromGraph(partial)
|
||||
|
||||
// Simple way to create a graph backed Source
|
||||
val source = Source() { implicit builder =>
|
||||
val source = Source.fromGraph( FlowGraph.create() { implicit builder =>
|
||||
val merge = builder.add(Merge[Int](2))
|
||||
Source.single(0) ~> merge
|
||||
Source(List(2, 3, 4)) ~> merge
|
||||
|
||||
// Exposing exactly one output port
|
||||
merge.out
|
||||
}
|
||||
SourceShape(merge.out)
|
||||
})
|
||||
|
||||
// Building a Sink with a nested Flow, using the fluid DSL
|
||||
val sink = {
|
||||
|
|
@ -164,22 +167,24 @@ class CompositionDocSpec extends AkkaSpec {
|
|||
"closed graph" in {
|
||||
//#embed-closed
|
||||
val closed1 = Source.single(0).to(Sink.foreach(println))
|
||||
val closed2 = FlowGraph.closed() { implicit builder =>
|
||||
val closed2 = RunnableGraph.fromGraph(FlowGraph.create() { implicit builder =>
|
||||
val embeddedClosed: ClosedShape = builder.add(closed1)
|
||||
}
|
||||
// …
|
||||
embeddedClosed
|
||||
})
|
||||
//#embed-closed
|
||||
}
|
||||
|
||||
"materialized values" in {
|
||||
//#mat-combine-1
|
||||
// Materializes to Promise[Unit] (red)
|
||||
val source: Source[Int, Promise[Unit]] = Source.lazyEmpty[Int]
|
||||
// Materializes to Promise[Option[Int]] (red)
|
||||
val source: Source[Int, Promise[Option[Int]]] = Source.maybe[Int]
|
||||
|
||||
// Materializes to Unit (black)
|
||||
val flow1: Flow[Int, Int, Unit] = Flow[Int].take(100)
|
||||
|
||||
// Materializes to Promise[Unit] (red)
|
||||
val nestedSource: Source[Int, Promise[Unit]] =
|
||||
// Materializes to Promise[Int] (red)
|
||||
val nestedSource: Source[Int, Promise[Option[Int]]] =
|
||||
source.viaMat(flow1)(Keep.left).named("nestedSource")
|
||||
//#mat-combine-1
|
||||
|
||||
|
|
@ -206,11 +211,11 @@ class CompositionDocSpec extends AkkaSpec {
|
|||
//#mat-combine-3
|
||||
|
||||
//#mat-combine-4
|
||||
case class MyClass(private val p: Promise[Unit], conn: OutgoingConnection) {
|
||||
def close() = p.success(())
|
||||
case class MyClass(private val p: Promise[Option[Int]], conn: OutgoingConnection) {
|
||||
def close() = p.trySuccess(None)
|
||||
}
|
||||
|
||||
def f(p: Promise[Unit],
|
||||
def f(p: Promise[Option[Int]],
|
||||
rest: (Future[OutgoingConnection], Future[String])): Future[MyClass] = {
|
||||
|
||||
val connFuture = rest._1
|
||||
|
|
|
|||
|
|
@ -4,6 +4,7 @@
|
|||
package docs.stream
|
||||
|
||||
import akka.actor.Cancellable
|
||||
import akka.stream.{ ClosedShape, FlowShape }
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.testkit.AkkaSpec
|
||||
|
||||
|
|
@ -147,17 +148,17 @@ class FlowDocSpec extends AkkaSpec {
|
|||
"various ways of transforming materialized values" in {
|
||||
import scala.concurrent.duration._
|
||||
|
||||
val throttler = Flow(Source(1.second, 1.second, "test")) { implicit builder =>
|
||||
val throttler = Flow.fromGraph(FlowGraph.create(Source(1.second, 1.second, "test")) { implicit builder =>
|
||||
tickSource =>
|
||||
import FlowGraph.Implicits._
|
||||
val zip = builder.add(ZipWith[String, Int, Int](Keep.right))
|
||||
tickSource ~> zip.in0
|
||||
(zip.in1, zip.out)
|
||||
}
|
||||
FlowShape(zip.in1, zip.out)
|
||||
})
|
||||
|
||||
//#flow-mat-combine
|
||||
// An empty source that can be shut down explicitly from the outside
|
||||
val source: Source[Int, Promise[Unit]] = Source.lazyEmpty[Int]
|
||||
// An source that can be signalled explicitly from the outside
|
||||
val source: Source[Int, Promise[Option[Int]]] = Source.maybe[Int]
|
||||
|
||||
// A flow that internally throttles elements to 1/second, and returns a Cancellable
|
||||
// which can be used to shut down the stream
|
||||
|
|
@ -167,7 +168,7 @@ class FlowDocSpec extends AkkaSpec {
|
|||
val sink: Sink[Int, Future[Int]] = Sink.head[Int]
|
||||
|
||||
// By default, the materialized value of the leftmost stage is preserved
|
||||
val r1: RunnableGraph[Promise[Unit]] = source.via(flow).to(sink)
|
||||
val r1: RunnableGraph[Promise[Option[Int]]] = source.via(flow).to(sink)
|
||||
|
||||
// Simple selection of materialized values by using Keep.right
|
||||
val r2: RunnableGraph[Cancellable] = source.viaMat(flow)(Keep.right).to(sink)
|
||||
|
|
@ -176,17 +177,17 @@ class FlowDocSpec extends AkkaSpec {
|
|||
// Using runWith will always give the materialized values of the stages added
|
||||
// by runWith() itself
|
||||
val r4: Future[Int] = source.via(flow).runWith(sink)
|
||||
val r5: Promise[Unit] = flow.to(sink).runWith(source)
|
||||
val r6: (Promise[Unit], Future[Int]) = flow.runWith(source, sink)
|
||||
val r5: Promise[Option[Int]] = flow.to(sink).runWith(source)
|
||||
val r6: (Promise[Option[Int]], Future[Int]) = flow.runWith(source, sink)
|
||||
|
||||
// Using more complext combinations
|
||||
val r7: RunnableGraph[(Promise[Unit], Cancellable)] =
|
||||
val r7: RunnableGraph[(Promise[Option[Int]], Cancellable)] =
|
||||
source.viaMat(flow)(Keep.both).to(sink)
|
||||
|
||||
val r8: RunnableGraph[(Promise[Unit], Future[Int])] =
|
||||
val r8: RunnableGraph[(Promise[Option[Int]], Future[Int])] =
|
||||
source.via(flow).toMat(sink)(Keep.both)
|
||||
|
||||
val r9: RunnableGraph[((Promise[Unit], Cancellable), Future[Int])] =
|
||||
val r9: RunnableGraph[((Promise[Option[Int]], Cancellable), Future[Int])] =
|
||||
source.viaMat(flow)(Keep.both).toMat(sink)(Keep.both)
|
||||
|
||||
val r10: RunnableGraph[(Cancellable, Future[Int])] =
|
||||
|
|
@ -194,7 +195,7 @@ class FlowDocSpec extends AkkaSpec {
|
|||
|
||||
// It is also possible to map over the materialized values. In r9 we had a
|
||||
// doubly nested pair, but we want to flatten it out
|
||||
val r11: RunnableGraph[(Promise[Unit], Cancellable, Future[Int])] =
|
||||
val r11: RunnableGraph[(Promise[Option[Int]], Cancellable, Future[Int])] =
|
||||
r9.mapMaterializedValue {
|
||||
case ((promise, cancellable), future) =>
|
||||
(promise, cancellable, future)
|
||||
|
|
@ -204,17 +205,18 @@ class FlowDocSpec extends AkkaSpec {
|
|||
val (promise, cancellable, future) = r11.run()
|
||||
|
||||
// Type inference works as expected
|
||||
promise.success(())
|
||||
promise.success(None)
|
||||
cancellable.cancel()
|
||||
future.map(_ + 3)
|
||||
|
||||
// The result of r11 can be also achieved by using the Graph API
|
||||
val r12: RunnableGraph[(Promise[Unit], Cancellable, Future[Int])] =
|
||||
FlowGraph.closed(source, flow, sink)((_, _, _)) { implicit builder =>
|
||||
val r12: RunnableGraph[(Promise[Option[Int]], Cancellable, Future[Int])] =
|
||||
RunnableGraph.fromGraph(FlowGraph.create(source, flow, sink)((_, _, _)) { implicit builder =>
|
||||
(src, f, dst) =>
|
||||
import FlowGraph.Implicits._
|
||||
src ~> f ~> dst
|
||||
}
|
||||
ClosedShape
|
||||
})
|
||||
|
||||
//#flow-mat-combine
|
||||
}
|
||||
|
|
|
|||
|
|
@ -21,7 +21,7 @@ class FlowGraphDocSpec extends AkkaSpec {
|
|||
"build simple graph" in {
|
||||
//format: OFF
|
||||
//#simple-flow-graph
|
||||
val g = FlowGraph.closed() { implicit builder: FlowGraph.Builder[Unit] =>
|
||||
val g = RunnableGraph.fromGraph(FlowGraph.create() { implicit builder: FlowGraph.Builder[Unit] =>
|
||||
import FlowGraph.Implicits._
|
||||
val in = Source(1 to 10)
|
||||
val out = Sink.ignore
|
||||
|
|
@ -33,7 +33,8 @@ class FlowGraphDocSpec extends AkkaSpec {
|
|||
|
||||
in ~> f1 ~> bcast ~> f2 ~> merge ~> f3 ~> out
|
||||
bcast ~> f4 ~> merge
|
||||
}
|
||||
ClosedShape
|
||||
})
|
||||
//#simple-flow-graph
|
||||
//format: ON
|
||||
|
||||
|
|
@ -42,33 +43,10 @@ class FlowGraphDocSpec extends AkkaSpec {
|
|||
//#simple-graph-run
|
||||
}
|
||||
|
||||
"build simple graph without implicits" in {
|
||||
//#simple-flow-graph-no-implicits
|
||||
val g = FlowGraph.closed() { builder: FlowGraph.Builder[Unit] =>
|
||||
val in = Source(1 to 10)
|
||||
val out = Sink.ignore
|
||||
|
||||
val broadcast = builder.add(Broadcast[Int](2))
|
||||
val merge = builder.add(Merge[Int](2))
|
||||
|
||||
val f1 = Flow[Int].map(_ + 10)
|
||||
val f3 = Flow[Int].map(_.toString)
|
||||
val f2 = Flow[Int].map(_ + 20)
|
||||
|
||||
builder.addEdge(builder.add(in), broadcast.in)
|
||||
builder.addEdge(broadcast.out(0), f1, merge.in(0))
|
||||
builder.addEdge(broadcast.out(1), f2, merge.in(1))
|
||||
builder.addEdge(merge.out, f3, builder.add(out))
|
||||
}
|
||||
//#simple-flow-graph-no-implicits
|
||||
|
||||
g.run()
|
||||
}
|
||||
|
||||
"flow connection errors" in {
|
||||
intercept[IllegalArgumentException] {
|
||||
//#simple-graph
|
||||
FlowGraph.closed() { implicit builder =>
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit builder =>
|
||||
import FlowGraph.Implicits._
|
||||
val source1 = Source(1 to 10)
|
||||
val source2 = Source(1 to 10)
|
||||
|
|
@ -78,9 +56,10 @@ class FlowGraphDocSpec extends AkkaSpec {
|
|||
source1 ~> zip.in0
|
||||
source2 ~> zip.in1
|
||||
// unconnected zip.out (!) => "must have at least 1 outgoing edge"
|
||||
}
|
||||
ClosedShape
|
||||
})
|
||||
//#simple-graph
|
||||
}.getMessage should include("unconnected ports: ZipWith2.out")
|
||||
}.getMessage should include("ZipWith2.out")
|
||||
}
|
||||
|
||||
"reusing a flow in a graph" in {
|
||||
|
|
@ -95,7 +74,7 @@ class FlowGraphDocSpec extends AkkaSpec {
|
|||
// format: OFF
|
||||
val g =
|
||||
//#flow-graph-reusing-a-flow
|
||||
FlowGraph.closed(topHeadSink, bottomHeadSink)((_, _)) { implicit builder =>
|
||||
RunnableGraph.fromGraph(FlowGraph.create(topHeadSink, bottomHeadSink)((_, _)) { implicit builder =>
|
||||
(topHS, bottomHS) =>
|
||||
import FlowGraph.Implicits._
|
||||
val broadcast = builder.add(Broadcast[Int](2))
|
||||
|
|
@ -103,7 +82,8 @@ class FlowGraphDocSpec extends AkkaSpec {
|
|||
|
||||
broadcast.out(0) ~> sharedDoubler ~> topHS.inlet
|
||||
broadcast.out(1) ~> sharedDoubler ~> bottomHS.inlet
|
||||
}
|
||||
ClosedShape
|
||||
})
|
||||
//#flow-graph-reusing-a-flow
|
||||
// format: ON
|
||||
val (topFuture, bottomFuture) = g.run()
|
||||
|
|
@ -153,7 +133,7 @@ class FlowGraphDocSpec extends AkkaSpec {
|
|||
worker: Flow[In, Out, Any],
|
||||
workerCount: Int): Graph[PriorityWorkerPoolShape[In, Out], Unit] = {
|
||||
|
||||
FlowGraph.partial() { implicit b ⇒
|
||||
FlowGraph.create() { implicit b ⇒
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
val priorityMerge = b.add(MergePreferred[In](1))
|
||||
|
|
@ -188,7 +168,7 @@ class FlowGraphDocSpec extends AkkaSpec {
|
|||
val worker1 = Flow[String].map("step 1 " + _)
|
||||
val worker2 = Flow[String].map("step 2 " + _)
|
||||
|
||||
FlowGraph.closed() { implicit b =>
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b =>
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
val priorityPool1 = b.add(PriorityWorkerPool(worker1, 4))
|
||||
|
|
@ -201,7 +181,8 @@ class FlowGraphDocSpec extends AkkaSpec {
|
|||
Source(1 to 100).map("one-step, priority " + _) ~> priorityPool2.priorityJobsIn
|
||||
|
||||
priorityPool2.resultsOut ~> Sink.foreach(println)
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
//#flow-graph-components-use
|
||||
|
||||
//#flow-graph-components-shape2
|
||||
|
|
@ -223,11 +204,11 @@ class FlowGraphDocSpec extends AkkaSpec {
|
|||
"access to materialized value" in {
|
||||
//#flow-graph-matvalue
|
||||
import FlowGraph.Implicits._
|
||||
val foldFlow: Flow[Int, Int, Future[Int]] = Flow(Sink.fold[Int, Int](0)(_ + _)) {
|
||||
val foldFlow: Flow[Int, Int, Future[Int]] = Flow.fromGraph(FlowGraph.create(Sink.fold[Int, Int](0)(_ + _)) {
|
||||
implicit builder ⇒
|
||||
fold ⇒
|
||||
(fold.inlet, builder.materializedValue.mapAsync(4)(identity).outlet)
|
||||
}
|
||||
FlowShape(fold.inlet, builder.materializedValue.mapAsync(4)(identity).outlet)
|
||||
})
|
||||
//#flow-graph-matvalue
|
||||
|
||||
Await.result(Source(1 to 10).via(foldFlow).runWith(Sink.head), 3.seconds) should ===(55)
|
||||
|
|
@ -235,7 +216,7 @@ class FlowGraphDocSpec extends AkkaSpec {
|
|||
//#flow-graph-matvalue-cycle
|
||||
import FlowGraph.Implicits._
|
||||
// This cannot produce any value:
|
||||
val cyclicFold: Source[Int, Future[Int]] = Source(Sink.fold[Int, Int](0)(_ + _)) {
|
||||
val cyclicFold: Source[Int, Future[Int]] = Source.fromGraph(FlowGraph.create(Sink.fold[Int, Int](0)(_ + _)) {
|
||||
implicit builder =>
|
||||
fold =>
|
||||
// - Fold cannot complete until its upstream mapAsync completes
|
||||
|
|
@ -244,8 +225,8 @@ class FlowGraphDocSpec extends AkkaSpec {
|
|||
// As a result this Source will never emit anything, and its materialited
|
||||
// Future will never complete
|
||||
builder.materializedValue.mapAsync(4)(identity) ~> fold
|
||||
builder.materializedValue.mapAsync(4)(identity).outlet
|
||||
}
|
||||
SourceShape(builder.materializedValue.mapAsync(4)(identity).outlet)
|
||||
})
|
||||
//#flow-graph-matvalue-cycle
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -1,5 +1,6 @@
|
|||
package docs.stream
|
||||
|
||||
import akka.stream.FlowShape
|
||||
import akka.stream.scaladsl.{ FlowGraph, Merge, Balance, Source, Flow }
|
||||
import akka.stream.testkit.AkkaSpec
|
||||
|
||||
|
|
@ -37,7 +38,7 @@ class FlowParallelismDocSpec extends AkkaSpec {
|
|||
val fryingPan: Flow[ScoopOfBatter, Pancake, Unit] =
|
||||
Flow[ScoopOfBatter].map { batter => Pancake() }
|
||||
|
||||
val pancakeChef: Flow[ScoopOfBatter, Pancake, Unit] = Flow() { implicit builder =>
|
||||
val pancakeChef: Flow[ScoopOfBatter, Pancake, Unit] = Flow.fromGraph(FlowGraph.create() { implicit builder =>
|
||||
val dispatchBatter = builder.add(Balance[ScoopOfBatter](2))
|
||||
val mergePancakes = builder.add(Merge[Pancake](2))
|
||||
|
||||
|
|
@ -49,15 +50,16 @@ class FlowParallelismDocSpec extends AkkaSpec {
|
|||
// uses of "fryingPan" mean actually different stages in the graph.
|
||||
dispatchBatter.out(1) ~> fryingPan ~> mergePancakes.in(1)
|
||||
|
||||
(dispatchBatter.in, mergePancakes.out)
|
||||
}
|
||||
FlowShape(dispatchBatter.in, mergePancakes.out)
|
||||
})
|
||||
|
||||
//#parallelism
|
||||
}
|
||||
|
||||
"Demonstrate parallelized pipelines" in {
|
||||
//#parallel-pipeline
|
||||
val pancakeChef: Flow[ScoopOfBatter, Pancake, Unit] = Flow() { implicit builder =>
|
||||
val pancakeChef: Flow[ScoopOfBatter, Pancake, Unit] =
|
||||
Flow.fromGraph(FlowGraph.create() { implicit builder =>
|
||||
|
||||
val dispatchBatter = builder.add(Balance[ScoopOfBatter](2))
|
||||
val mergePancakes = builder.add(Merge[Pancake](2))
|
||||
|
|
@ -67,14 +69,15 @@ class FlowParallelismDocSpec extends AkkaSpec {
|
|||
dispatchBatter.out(0) ~> fryingPan1 ~> fryingPan2 ~> mergePancakes.in(0)
|
||||
dispatchBatter.out(1) ~> fryingPan1 ~> fryingPan2 ~> mergePancakes.in(1)
|
||||
|
||||
(dispatchBatter.in, mergePancakes.out)
|
||||
}
|
||||
FlowShape(dispatchBatter.in, mergePancakes.out)
|
||||
})
|
||||
//#parallel-pipeline
|
||||
}
|
||||
|
||||
"Demonstrate pipelined parallel processing" in {
|
||||
//#pipelined-parallel
|
||||
val pancakeChefs1: Flow[ScoopOfBatter, HalfCookedPancake, Unit] = Flow() { implicit builder =>
|
||||
val pancakeChefs1: Flow[ScoopOfBatter, HalfCookedPancake, Unit] =
|
||||
Flow.fromGraph(FlowGraph.create() { implicit builder =>
|
||||
val dispatchBatter = builder.add(Balance[ScoopOfBatter](2))
|
||||
val mergeHalfPancakes = builder.add(Merge[HalfCookedPancake](2))
|
||||
|
||||
|
|
@ -83,10 +86,11 @@ class FlowParallelismDocSpec extends AkkaSpec {
|
|||
dispatchBatter.out(0) ~> fryingPan1 ~> mergeHalfPancakes.in(0)
|
||||
dispatchBatter.out(1) ~> fryingPan1 ~> mergeHalfPancakes.in(1)
|
||||
|
||||
(dispatchBatter.in, mergeHalfPancakes.out)
|
||||
}
|
||||
FlowShape(dispatchBatter.in, mergeHalfPancakes.out)
|
||||
})
|
||||
|
||||
val pancakeChefs2: Flow[HalfCookedPancake, Pancake, Unit] = Flow() { implicit builder =>
|
||||
val pancakeChefs2: Flow[HalfCookedPancake, Pancake, Unit] =
|
||||
Flow.fromGraph(FlowGraph.create() { implicit builder =>
|
||||
val dispatchHalfPancakes = builder.add(Balance[HalfCookedPancake](2))
|
||||
val mergePancakes = builder.add(Merge[Pancake](2))
|
||||
|
||||
|
|
@ -95,8 +99,8 @@ class FlowParallelismDocSpec extends AkkaSpec {
|
|||
dispatchHalfPancakes.out(0) ~> fryingPan2 ~> mergePancakes.in(0)
|
||||
dispatchHalfPancakes.out(1) ~> fryingPan2 ~> mergePancakes.in(1)
|
||||
|
||||
(dispatchHalfPancakes.in, mergePancakes.out)
|
||||
}
|
||||
FlowShape(dispatchHalfPancakes.in, mergePancakes.out)
|
||||
})
|
||||
|
||||
val kitchen: Flow[ScoopOfBatter, Pancake, Unit] = pancakeChefs1.via(pancakeChefs2)
|
||||
//#pipelined-parallel
|
||||
|
|
|
|||
|
|
@ -1,6 +1,6 @@
|
|||
package docs.stream
|
||||
|
||||
import akka.stream.{ OverflowStrategy, ActorMaterializer }
|
||||
import akka.stream.{ ClosedShape, OverflowStrategy, ActorMaterializer }
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.testkit.AkkaSpec
|
||||
|
||||
|
|
@ -16,7 +16,7 @@ class GraphCyclesSpec extends AkkaSpec {
|
|||
// format: OFF
|
||||
//#deadlocked
|
||||
// WARNING! The graph below deadlocks!
|
||||
FlowGraph.closed() { implicit b =>
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b =>
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
val merge = b.add(Merge[Int](2))
|
||||
|
|
@ -24,7 +24,8 @@ class GraphCyclesSpec extends AkkaSpec {
|
|||
|
||||
source ~> merge ~> Flow[Int].map { s => println(s); s } ~> bcast ~> Sink.ignore
|
||||
merge <~ bcast
|
||||
}
|
||||
ClosedShape
|
||||
})
|
||||
//#deadlocked
|
||||
// format: ON
|
||||
}
|
||||
|
|
@ -33,7 +34,7 @@ class GraphCyclesSpec extends AkkaSpec {
|
|||
// format: OFF
|
||||
//#unfair
|
||||
// WARNING! The graph below stops consuming from "source" after a few steps
|
||||
FlowGraph.closed() { implicit b =>
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b =>
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
val merge = b.add(MergePreferred[Int](1))
|
||||
|
|
@ -41,7 +42,8 @@ class GraphCyclesSpec extends AkkaSpec {
|
|||
|
||||
source ~> merge ~> Flow[Int].map { s => println(s); s } ~> bcast ~> Sink.ignore
|
||||
merge.preferred <~ bcast
|
||||
}
|
||||
ClosedShape
|
||||
})
|
||||
//#unfair
|
||||
// format: ON
|
||||
}
|
||||
|
|
@ -49,7 +51,7 @@ class GraphCyclesSpec extends AkkaSpec {
|
|||
"include a dropping cycle" in {
|
||||
// format: OFF
|
||||
//#dropping
|
||||
FlowGraph.closed() { implicit b =>
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b =>
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
val merge = b.add(Merge[Int](2))
|
||||
|
|
@ -57,7 +59,8 @@ class GraphCyclesSpec extends AkkaSpec {
|
|||
|
||||
source ~> merge ~> Flow[Int].map { s => println(s); s } ~> bcast ~> Sink.ignore
|
||||
merge <~ Flow[Int].buffer(10, OverflowStrategy.dropHead) <~ bcast
|
||||
}
|
||||
ClosedShape
|
||||
})
|
||||
//#dropping
|
||||
// format: ON
|
||||
}
|
||||
|
|
@ -66,7 +69,7 @@ class GraphCyclesSpec extends AkkaSpec {
|
|||
// format: OFF
|
||||
//#zipping-dead
|
||||
// WARNING! The graph below never processes any elements
|
||||
FlowGraph.closed() { implicit b =>
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b =>
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
val zip = b.add(ZipWith[Int, Int, Int]((left, right) => right))
|
||||
|
|
@ -75,7 +78,8 @@ class GraphCyclesSpec extends AkkaSpec {
|
|||
source ~> zip.in0
|
||||
zip.out.map { s => println(s); s } ~> bcast ~> Sink.ignore
|
||||
zip.in1 <~ bcast
|
||||
}
|
||||
ClosedShape
|
||||
})
|
||||
//#zipping-dead
|
||||
// format: ON
|
||||
}
|
||||
|
|
@ -83,7 +87,7 @@ class GraphCyclesSpec extends AkkaSpec {
|
|||
"include a live zipping cycle" in {
|
||||
// format: OFF
|
||||
//#zipping-live
|
||||
FlowGraph.closed() { implicit b =>
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b =>
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
val zip = b.add(ZipWith((left: Int, right: Int) => left))
|
||||
|
|
@ -95,7 +99,8 @@ class GraphCyclesSpec extends AkkaSpec {
|
|||
zip.out.map { s => println(s); s } ~> bcast ~> Sink.ignore
|
||||
zip.in1 <~ concat <~ start
|
||||
concat <~ bcast
|
||||
}
|
||||
ClosedShape
|
||||
})
|
||||
//#zipping-live
|
||||
// format: ON
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,9 +1,8 @@
|
|||
package docs.stream
|
||||
|
||||
import akka.stream.{ OverflowStrategy, ActorMaterializerSettings, ActorMaterializer }
|
||||
import akka.stream._
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.testkit.AkkaSpec
|
||||
import akka.stream.Attributes
|
||||
|
||||
class StreamBuffersRateSpec extends AkkaSpec {
|
||||
implicit val mat = ActorMaterializer()
|
||||
|
|
@ -40,7 +39,7 @@ class StreamBuffersRateSpec extends AkkaSpec {
|
|||
import scala.concurrent.duration._
|
||||
case class Tick()
|
||||
|
||||
FlowGraph.closed() { implicit b =>
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b =>
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
val zipper = b.add(ZipWith[Tick, Int, Int]((tick, count) => count))
|
||||
|
|
@ -51,7 +50,8 @@ class StreamBuffersRateSpec extends AkkaSpec {
|
|||
.conflate(seed = (_) => 1)((count, _) => count + 1) ~> zipper.in1
|
||||
|
||||
zipper.out ~> Sink.foreach(println)
|
||||
}
|
||||
ClosedShape
|
||||
})
|
||||
//#buffering-abstraction-leak
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -19,7 +19,7 @@ class StreamPartialFlowGraphDocSpec extends AkkaSpec {
|
|||
|
||||
"build with open ports" in {
|
||||
//#simple-partial-flow-graph
|
||||
val pickMaxOfThree = FlowGraph.partial() { implicit b =>
|
||||
val pickMaxOfThree = FlowGraph.create() { implicit b =>
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
val zip1 = b.add(ZipWith[Int, Int, Int](math.max _))
|
||||
|
|
@ -31,7 +31,7 @@ class StreamPartialFlowGraphDocSpec extends AkkaSpec {
|
|||
|
||||
val resultSink = Sink.head[Int]
|
||||
|
||||
val g = FlowGraph.closed(resultSink) { implicit b =>
|
||||
val g = RunnableGraph.fromGraph(FlowGraph.create(resultSink) { implicit b =>
|
||||
sink =>
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
|
|
@ -42,7 +42,8 @@ class StreamPartialFlowGraphDocSpec extends AkkaSpec {
|
|||
Source.single(2) ~> pm3.in(1)
|
||||
Source.single(3) ~> pm3.in(2)
|
||||
pm3.out ~> sink.inlet
|
||||
}
|
||||
ClosedShape
|
||||
})
|
||||
|
||||
val max: Future[Int] = g.run()
|
||||
Await.result(max, 300.millis) should equal(3)
|
||||
|
|
@ -51,7 +52,7 @@ class StreamPartialFlowGraphDocSpec extends AkkaSpec {
|
|||
|
||||
"build source from partial flow graph" in {
|
||||
//#source-from-partial-flow-graph
|
||||
val pairs = Source() { implicit b =>
|
||||
val pairs = Source.fromGraph(FlowGraph.create() { implicit b =>
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
// prepare graph elements
|
||||
|
|
@ -63,8 +64,8 @@ class StreamPartialFlowGraphDocSpec extends AkkaSpec {
|
|||
ints.filter(_ % 2 == 0) ~> zip.in1
|
||||
|
||||
// expose port
|
||||
zip.out
|
||||
}
|
||||
SourceShape(zip.out)
|
||||
})
|
||||
|
||||
val firstPair: Future[(Int, Int)] = pairs.runWith(Sink.head)
|
||||
//#source-from-partial-flow-graph
|
||||
|
|
@ -73,7 +74,8 @@ class StreamPartialFlowGraphDocSpec extends AkkaSpec {
|
|||
|
||||
"build flow from partial flow graph" in {
|
||||
//#flow-from-partial-flow-graph
|
||||
val pairUpWithToString = Flow() { implicit b =>
|
||||
val pairUpWithToString =
|
||||
Flow.fromGraph(FlowGraph.create() { implicit b =>
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
// prepare graph elements
|
||||
|
|
@ -85,8 +87,8 @@ class StreamPartialFlowGraphDocSpec extends AkkaSpec {
|
|||
broadcast.out(1).map(_.toString) ~> zip.in1
|
||||
|
||||
// expose ports
|
||||
(broadcast.in, zip.out)
|
||||
}
|
||||
FlowShape(broadcast.in, zip.out)
|
||||
})
|
||||
|
||||
//#flow-from-partial-flow-graph
|
||||
|
||||
|
|
|
|||
|
|
@ -6,8 +6,7 @@ package docs.stream
|
|||
//#imports
|
||||
|
||||
import akka.actor.ActorSystem
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.stream.OverflowStrategy
|
||||
import akka.stream.{ ClosedShape, ActorMaterializer, OverflowStrategy }
|
||||
import akka.stream.scaladsl._
|
||||
|
||||
import scala.concurrent.Await
|
||||
|
|
@ -119,14 +118,15 @@ class TwitterStreamQuickstartDocSpec extends AkkaSpec {
|
|||
|
||||
// format: OFF
|
||||
//#flow-graph-broadcast
|
||||
val g = FlowGraph.closed() { implicit b =>
|
||||
val g = RunnableGraph.fromGraph(FlowGraph.create() { implicit b =>
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
val bcast = b.add(Broadcast[Tweet](2))
|
||||
tweets ~> bcast.in
|
||||
bcast.out(0) ~> Flow[Tweet].map(_.author) ~> writeAuthors
|
||||
bcast.out(1) ~> Flow[Tweet].mapConcat(_.hashtags.toList) ~> writeHashtags
|
||||
}
|
||||
ClosedShape
|
||||
})
|
||||
g.run()
|
||||
//#flow-graph-broadcast
|
||||
// format: ON
|
||||
|
|
|
|||
|
|
@ -1,6 +1,6 @@
|
|||
package docs.stream.cookbook
|
||||
|
||||
import akka.stream.OverflowStrategy
|
||||
import akka.stream.{ ClosedShape, OverflowStrategy }
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.testkit._
|
||||
|
||||
|
|
@ -24,7 +24,7 @@ class RecipeDroppyBroadcast extends RecipeSpec {
|
|||
val mySink3 = Sink(sub3)
|
||||
|
||||
//#droppy-bcast
|
||||
val graph = FlowGraph.closed(mySink1, mySink2, mySink3)((_, _, _)) { implicit b =>
|
||||
val graph = RunnableGraph.fromGraph(FlowGraph.create(mySink1, mySink2, mySink3)((_, _, _)) { implicit b =>
|
||||
(sink1, sink2, sink3) =>
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
|
|
@ -34,7 +34,8 @@ class RecipeDroppyBroadcast extends RecipeSpec {
|
|||
bcast.buffer(10, OverflowStrategy.dropHead) ~> sink1
|
||||
bcast.buffer(10, OverflowStrategy.dropHead) ~> sink2
|
||||
bcast.buffer(10, OverflowStrategy.dropHead) ~> sink3
|
||||
}
|
||||
ClosedShape
|
||||
})
|
||||
//#droppy-bcast
|
||||
|
||||
graph.run()
|
||||
|
|
|
|||
|
|
@ -2,6 +2,7 @@ package docs.stream.cookbook
|
|||
|
||||
import akka.actor.{ Props, ActorRef, Actor }
|
||||
import akka.actor.Actor.Receive
|
||||
import akka.stream.ClosedShape
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.testkit._
|
||||
|
||||
|
|
@ -98,12 +99,13 @@ class RecipeGlobalRateLimit extends RecipeSpec {
|
|||
|
||||
val probe = TestSubscriber.manualProbe[String]()
|
||||
|
||||
FlowGraph.closed() { implicit b =>
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b =>
|
||||
import FlowGraph.Implicits._
|
||||
val merge = b.add(Merge[String](2))
|
||||
source1 ~> merge ~> Sink(probe)
|
||||
source2 ~> merge
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
probe.expectSubscription().request(1000)
|
||||
|
||||
|
|
|
|||
|
|
@ -1,5 +1,6 @@
|
|||
package docs.stream.cookbook
|
||||
|
||||
import akka.stream.ClosedShape
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.testkit._
|
||||
import akka.util.ByteString
|
||||
|
|
@ -25,14 +26,15 @@ class RecipeKeepAlive extends RecipeSpec {
|
|||
val tickToKeepAlivePacket: Flow[Tick, ByteString, Unit] = Flow[Tick]
|
||||
.conflate(seed = (tick) => keepaliveMessage)((msg, newTick) => msg)
|
||||
|
||||
val graph = FlowGraph.closed() { implicit builder =>
|
||||
val graph = RunnableGraph.fromGraph(FlowGraph.create() { implicit builder =>
|
||||
import FlowGraph.Implicits._
|
||||
val unfairMerge = builder.add(MergePreferred[ByteString](1))
|
||||
|
||||
// If data is available then no keepalive is injected
|
||||
dataStream ~> unfairMerge.preferred
|
||||
ticks ~> tickToKeepAlivePacket ~> unfairMerge ~> sink
|
||||
}
|
||||
ClosedShape
|
||||
})
|
||||
//#inject-keepalive
|
||||
|
||||
graph.run()
|
||||
|
|
|
|||
|
|
@ -1,5 +1,6 @@
|
|||
package docs.stream.cookbook
|
||||
|
||||
import akka.stream.ClosedShape
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.testkit._
|
||||
import scala.concurrent.duration._
|
||||
|
|
@ -17,13 +18,14 @@ class RecipeManualTrigger extends RecipeSpec {
|
|||
val sink = Sink(sub)
|
||||
|
||||
//#manually-triggered-stream
|
||||
val graph = FlowGraph.closed() { implicit builder =>
|
||||
val graph = RunnableGraph.fromGraph(FlowGraph.create() { implicit builder =>
|
||||
import FlowGraph.Implicits._
|
||||
val zip = builder.add(Zip[Message, Trigger]())
|
||||
elements ~> zip.in0
|
||||
triggerSource ~> zip.in1
|
||||
zip.out ~> Flow[(Message, Trigger)].map { case (msg, trigger) => msg } ~> sink
|
||||
}
|
||||
ClosedShape
|
||||
})
|
||||
//#manually-triggered-stream
|
||||
|
||||
graph.run()
|
||||
|
|
@ -55,14 +57,15 @@ class RecipeManualTrigger extends RecipeSpec {
|
|||
val sink = Sink(sub)
|
||||
|
||||
//#manually-triggered-stream-zipwith
|
||||
val graph = FlowGraph.closed() { implicit builder =>
|
||||
val graph = RunnableGraph.fromGraph(FlowGraph.create() { implicit builder =>
|
||||
import FlowGraph.Implicits._
|
||||
val zip = builder.add(ZipWith((msg: Message, trigger: Trigger) => msg))
|
||||
|
||||
elements ~> zip.in0
|
||||
triggerSource ~> zip.in1
|
||||
zip.out ~> sink
|
||||
}
|
||||
ClosedShape
|
||||
})
|
||||
//#manually-triggered-stream-zipwith
|
||||
|
||||
graph.run()
|
||||
|
|
|
|||
|
|
@ -1,5 +1,6 @@
|
|||
package docs.stream.cookbook
|
||||
|
||||
import akka.stream.FlowShape
|
||||
import akka.stream.scaladsl._
|
||||
import akka.testkit.TestProbe
|
||||
|
||||
|
|
@ -20,7 +21,7 @@ class RecipeWorkerPool extends RecipeSpec {
|
|||
def balancer[In, Out](worker: Flow[In, Out, Any], workerCount: Int): Flow[In, Out, Unit] = {
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
Flow() { implicit b =>
|
||||
Flow.fromGraph(FlowGraph.create() { implicit b =>
|
||||
val balancer = b.add(Balance[In](workerCount, waitForAllDownstreams = true))
|
||||
val merge = b.add(Merge[Out](workerCount))
|
||||
|
||||
|
|
@ -30,8 +31,8 @@ class RecipeWorkerPool extends RecipeSpec {
|
|||
balancer ~> worker ~> merge
|
||||
}
|
||||
|
||||
(balancer.in, merge.out)
|
||||
}
|
||||
FlowShape(balancer.in, merge.out)
|
||||
})
|
||||
}
|
||||
|
||||
val processedJobs: Source[Result, Unit] = myJobs.via(balancer(worker, 3))
|
||||
|
|
|
|||
|
|
@ -73,7 +73,7 @@ class StreamTcpDocSpec extends AkkaSpec {
|
|||
|
||||
connections runForeach { connection =>
|
||||
|
||||
val serverLogic = Flow() { implicit b =>
|
||||
val serverLogic = Flow.fromGraph(FlowGraph.create() { implicit b =>
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
// server logic, parses incoming commands
|
||||
|
|
@ -109,8 +109,8 @@ class StreamTcpDocSpec extends AkkaSpec {
|
|||
// then we continue using the echo-logic Flow
|
||||
echo.outlet ~> concat.in(1)
|
||||
|
||||
(echo.inlet, concat.out)
|
||||
}
|
||||
FlowShape(echo.inlet, concat.out)
|
||||
})
|
||||
|
||||
connection.handleWith(serverLogic)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -126,8 +126,8 @@ As a first example, let's look at a more complex layout:
|
|||
|
||||
The diagram shows a :class:`RunnableGraph` (remember, if there are no unwired ports, the graph is closed, and therefore
|
||||
can be materialized) that encapsulates a non-trivial stream processing network. It contains fan-in, fan-out stages,
|
||||
directed and non-directed cycles. The ``closed()`` method of the :class:`FlowGraph` object allows the creation of a
|
||||
general closed graph. For example the network on the diagram can be realized like this:
|
||||
directed and non-directed cycles. The ``runnable()`` method of the :class:`FlowGraph` object allows the creation of a
|
||||
general, closed, and runnable graph. For example the network on the diagram can be realized like this:
|
||||
|
||||
.. includecode:: code/docs/stream/CompositionDocSpec.scala#complex-graph
|
||||
|
||||
|
|
@ -141,7 +141,7 @@ explicitly, and it is not necessary to import our linear stages via ``add()``, s
|
|||
|
||||
Similar to the case in the first section, so far we have not considered modularity. We created a complex graph, but
|
||||
the layout is flat, not modularized. We will modify our example, and create a reusable component with the graph DSL.
|
||||
The way to do it is to use the ``partial()`` factory method on :class:`FlowGraph`. If we remove the sources and sinks
|
||||
The way to do it is to use the ``create()`` factory method on :class:`FlowGraph`. If we remove the sources and sinks
|
||||
from the previous example, what remains is a partial graph:
|
||||
|
||||
|
|
||||
|
|
@ -284,7 +284,7 @@ Attributes
|
|||
----------
|
||||
|
||||
We have seen that we can use ``named()`` to introduce a nesting level in the fluid DSL (and also explicit nesting by using
|
||||
``partial()`` from :class:`FlowGraph`). Apart from having the effect of adding a nesting level, ``named()`` is actually
|
||||
``create()`` from :class:`FlowGraph`). Apart from having the effect of adding a nesting level, ``named()`` is actually
|
||||
a shorthand for calling ``withAttributes(Attributes.name("someName"))``. Attributes provide a way to fine-tune certain
|
||||
aspects of the materialized running entity. For example buffer sizes can be controlled via attributes (see
|
||||
:ref:`stream-buffers-scala`). When it comes to hierarchic composition, attributes are inherited by nested modules,
|
||||
|
|
|
|||
|
|
@ -60,9 +60,6 @@ will be inferred.
|
|||
|
||||
Notice the ``import FlowGraph.Implicits._`` which brings into scope the ``~>`` operator (read as "edge", "via" or "to")
|
||||
and its inverted counterpart ``<~`` (for noting down flows in the opposite direction where appropriate).
|
||||
It is also possible to construct graphs without the ``~>`` operator in case you prefer to use the graph builder explicitly:
|
||||
|
||||
.. includecode:: code/docs/stream/FlowGraphDocSpec.scala#simple-flow-graph-no-implicits
|
||||
|
||||
By looking at the snippets above, it should be apparent that the :class:`FlowGraph.Builder` object is *mutable*.
|
||||
It is used (implicitly) by the ``~>`` operator, also making it a mutable operation as well.
|
||||
|
|
|
|||
|
|
@ -76,7 +76,7 @@ private[http] object OutgoingConnectionBlueprint {
|
|||
case (MessageStartError(_, info), _) ⇒ throw IllegalResponseException(info)
|
||||
}
|
||||
|
||||
val core = BidiFlow() { implicit b ⇒
|
||||
val core = BidiFlow.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
import FlowGraph.Implicits._
|
||||
val methodBypassFanout = b.add(Broadcast[HttpRequest](2, eagerCancel = true))
|
||||
val responseParsingMerge = b.add(new ResponseParsingMerge(rootParser))
|
||||
|
|
@ -103,7 +103,7 @@ private[http] object OutgoingConnectionBlueprint {
|
|||
wrapTls.outlet,
|
||||
unwrapTls.inlet,
|
||||
terminationFanout.out(1))
|
||||
}
|
||||
})
|
||||
|
||||
One2OneBidiFlow[HttpRequest, HttpResponse](-1) atop core
|
||||
}
|
||||
|
|
|
|||
|
|
@ -65,7 +65,7 @@ private object PoolConductor {
|
|||
|
||||
*/
|
||||
def apply(slotCount: Int, maxRetries: Int, pipeliningLimit: Int, log: LoggingAdapter): Graph[Ports, Any] =
|
||||
FlowGraph.partial() { implicit b ⇒
|
||||
FlowGraph.create() { implicit b ⇒
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
val retryMerge = b.add(MergePreferred[RequestContext](1, eagerClose = true))
|
||||
|
|
|
|||
|
|
@ -11,7 +11,7 @@ import scala.concurrent.{ Promise, Future }
|
|||
import scala.util.Try
|
||||
import akka.event.LoggingAdapter
|
||||
import akka.actor._
|
||||
import akka.stream.Materializer
|
||||
import akka.stream.{ FlowShape, Materializer }
|
||||
import akka.stream.scaladsl._
|
||||
import akka.http.scaladsl.model._
|
||||
import akka.http.scaladsl.Http
|
||||
|
|
@ -70,7 +70,7 @@ private object PoolFlow {
|
|||
def apply(connectionFlow: Flow[HttpRequest, HttpResponse, Future[Http.OutgoingConnection]],
|
||||
remoteAddress: InetSocketAddress, settings: ConnectionPoolSettings, log: LoggingAdapter)(
|
||||
implicit system: ActorSystem, fm: Materializer): Flow[RequestContext, ResponseContext, Unit] =
|
||||
Flow() { implicit b ⇒
|
||||
Flow.fromGraph(FlowGraph.create[FlowShape[RequestContext, ResponseContext]]() { implicit b ⇒
|
||||
import settings._
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
|
|
@ -87,6 +87,6 @@ private object PoolFlow {
|
|||
slot.out0 ~> responseMerge.in(ix)
|
||||
slot.out1 ~> slotEventMerge.in(ix)
|
||||
}
|
||||
(conductor.requestIn, responseMerge.out)
|
||||
}
|
||||
FlowShape(conductor.requestIn, responseMerge.out)
|
||||
})
|
||||
}
|
||||
|
|
|
|||
|
|
@ -53,7 +53,7 @@ private object PoolSlot {
|
|||
remoteAddress: InetSocketAddress, // TODO: remove after #16168 is cleared
|
||||
settings: ConnectionPoolSettings)(implicit system: ActorSystem,
|
||||
fm: Materializer): Graph[FanOutShape2[RequestContext, ResponseContext, RawSlotEvent], Any] =
|
||||
FlowGraph.partial() { implicit b ⇒
|
||||
FlowGraph.create() { implicit b ⇒
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
val slotProcessor = b.add {
|
||||
|
|
|
|||
|
|
@ -5,6 +5,7 @@
|
|||
package akka.http.impl.engine.rendering
|
||||
|
||||
import akka.parboiled2.CharUtils
|
||||
import akka.stream.SourceShape
|
||||
import akka.util.ByteString
|
||||
import akka.event.LoggingAdapter
|
||||
import akka.stream.scaladsl._
|
||||
|
|
@ -31,12 +32,12 @@ private object RenderSupport {
|
|||
val defaultLastChunkBytes: ByteString = renderChunk(HttpEntity.LastChunk)
|
||||
|
||||
def CancelSecond[T, Mat](first: Source[T, Mat], second: Source[T, Any]): Source[T, Mat] = {
|
||||
Source(first) { implicit b ⇒
|
||||
Source.fromGraph(FlowGraph.create(first) { implicit b ⇒
|
||||
frst ⇒
|
||||
import FlowGraph.Implicits._
|
||||
second ~> Sink.cancelled
|
||||
frst.outlet
|
||||
}
|
||||
SourceShape(frst.outlet)
|
||||
})
|
||||
}
|
||||
|
||||
def renderEntityContentType(r: Rendering, entity: HttpEntity) =
|
||||
|
|
|
|||
|
|
@ -116,7 +116,7 @@ private[http] object HttpServerBluePrint {
|
|||
.flatten(FlattenStrategy.concat)
|
||||
.via(Flow[ResponseRenderingOutput].transform(() ⇒ errorLogger(log, "Outgoing response stream error")).named("errorLogger"))
|
||||
|
||||
BidiFlow(requestParsingFlow, rendererPipeline, oneHundredContinueSource)((_, _, _) ⇒ ()) { implicit b ⇒
|
||||
BidiFlow.fromGraph(FlowGraph.create(requestParsingFlow, rendererPipeline, oneHundredContinueSource)((_, _, _) ⇒ ()) { implicit b ⇒
|
||||
(requestParsing, renderer, oneHundreds) ⇒
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
|
|
@ -172,7 +172,7 @@ private[http] object HttpServerBluePrint {
|
|||
wrapTls.outlet,
|
||||
unwrapTls.inlet,
|
||||
requestsIn)
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
class BypassMerge(settings: ServerSettings, log: LoggingAdapter)
|
||||
|
|
@ -319,7 +319,7 @@ private[http] object HttpServerBluePrint {
|
|||
val sink = StreamUtils.oneTimePublisherSink[FrameEvent](sinkCell, "frameHandler.in")
|
||||
val source = StreamUtils.oneTimeSubscriberSource[FrameEvent](sourceCell, "frameHandler.out")
|
||||
|
||||
val flow = Websocket.framing.join(Flow.wrap(sink, source)(Keep.none))
|
||||
val flow = Websocket.framing.join(Flow.fromSinkAndSourceMat(sink, source)(Keep.none))
|
||||
|
||||
new WebsocketSetup {
|
||||
def websocketFlow: Flow[ByteString, ByteString, Any] = flow
|
||||
|
|
|
|||
|
|
@ -16,7 +16,7 @@ import akka.stream.stage.{ SyncDirective, Context, StatefulStage }
|
|||
*/
|
||||
private[http] object Masking {
|
||||
def apply(serverSide: Boolean, maskRandom: () ⇒ Random): BidiFlow[ /* net in */ FrameEvent, /* app out */ FrameEvent, /* app in */ FrameEvent, /* net out */ FrameEvent, Unit] =
|
||||
BidiFlow.wrap(unmaskIf(serverSide), maskIf(!serverSide, maskRandom))(Keep.none)
|
||||
BidiFlow.fromFlowsMat(unmaskIf(serverSide), maskIf(!serverSide, maskRandom))(Keep.none)
|
||||
|
||||
def maskIf(condition: Boolean, maskRandom: () ⇒ Random): Flow[FrameEvent, FrameEvent, Unit] =
|
||||
if (condition) Flow[FrameEvent].transform(() ⇒ new Masking(maskRandom())) // new random per materialization
|
||||
|
|
|
|||
|
|
@ -39,7 +39,7 @@ private[http] object Websocket {
|
|||
|
||||
/** The lowest layer that implements the binary protocol */
|
||||
def framing: BidiFlow[ByteString, FrameEvent, FrameEvent, ByteString, Unit] =
|
||||
BidiFlow.wrap(
|
||||
BidiFlow.fromFlowsMat(
|
||||
Flow[ByteString].transform(() ⇒ new FrameEventParser),
|
||||
Flow[FrameEvent].transform(() ⇒ new FrameEventRenderer))(Keep.none)
|
||||
.named("ws-framing")
|
||||
|
|
@ -56,7 +56,7 @@ private[http] object Websocket {
|
|||
def frameHandling(serverSide: Boolean = true,
|
||||
closeTimeout: FiniteDuration,
|
||||
log: LoggingAdapter): BidiFlow[FrameEvent, FrameHandler.Output, FrameOutHandler.Input, FrameStart, Unit] =
|
||||
BidiFlow.wrap(
|
||||
BidiFlow.fromFlowsMat(
|
||||
FrameHandler.create(server = serverSide),
|
||||
FrameOutHandler.create(serverSide, closeTimeout, log))(Keep.none)
|
||||
.named("ws-frame-handling")
|
||||
|
|
@ -121,7 +121,7 @@ private[http] object Websocket {
|
|||
MessageToFrameRenderer.create(serverSide)
|
||||
.named("ws-render-messages")
|
||||
|
||||
BidiFlow() { implicit b ⇒
|
||||
BidiFlow.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
val split = b.add(BypassRouter)
|
||||
|
|
@ -146,7 +146,7 @@ private[http] object Websocket {
|
|||
messagePreparation.outlet,
|
||||
messageRendering.inlet,
|
||||
merge.out)
|
||||
}.named("ws-message-api")
|
||||
}.named("ws-message-api"))
|
||||
}
|
||||
|
||||
private object BypassRouter extends GraphStage[FanOutShape2[Output, BypassEvent, MessagePart]] {
|
||||
|
|
|
|||
|
|
@ -109,7 +109,7 @@ object WebsocketClientBlueprint {
|
|||
}
|
||||
}
|
||||
|
||||
BidiFlow() { implicit b ⇒
|
||||
BidiFlow.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
val networkIn = b.add(Flow[ByteString].transform(() ⇒ new UpgradeStage))
|
||||
|
|
@ -126,11 +126,11 @@ object WebsocketClientBlueprint {
|
|||
networkIn.outlet,
|
||||
wsIn.inlet,
|
||||
httpRequestBytesAndThenWSBytes.out)
|
||||
} mapMaterializedValue (_ ⇒ result.future)
|
||||
}) mapMaterializedValue (_ ⇒ result.future)
|
||||
}
|
||||
|
||||
def simpleTls: BidiFlow[SslTlsInbound, ByteString, ByteString, SendBytes, Unit] =
|
||||
BidiFlow.wrap(
|
||||
BidiFlow.fromFlowsMat(
|
||||
Flow[SslTlsInbound].collect { case SessionBytes(_, bytes) ⇒ bytes },
|
||||
Flow[ByteString].map(SendBytes))(Keep.none)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -113,7 +113,7 @@ private[http] object JavaMapping {
|
|||
def toScala(javaObject: javadsl.Flow[JIn, JOut, M]): S =
|
||||
scaladsl.Flow[SIn].map(inMapping.toJava(_)).viaMat(javaObject)(scaladsl.Keep.right).map(outMapping.toScala(_))
|
||||
def toJava(scalaObject: scaladsl.Flow[SIn, SOut, M]): J =
|
||||
javadsl.Flow.wrap {
|
||||
javadsl.Flow.fromGraph {
|
||||
scaladsl.Flow[JIn].map(inMapping.toScala(_)).viaMat(scalaObject)(scaladsl.Keep.right).map(outMapping.toJava(_))
|
||||
}
|
||||
}
|
||||
|
|
@ -123,7 +123,7 @@ private[http] object JavaMapping {
|
|||
def javaToScalaAdapterFlow[J, S](implicit mapping: JavaMapping[J, S]): scaladsl.Flow[J, S, Unit] =
|
||||
scaladsl.Flow[J].map(mapping.toScala(_))
|
||||
def adapterBidiFlow[JIn, SIn, SOut, JOut](implicit inMapping: JavaMapping[JIn, SIn], outMapping: JavaMapping[JOut, SOut]): scaladsl.BidiFlow[JIn, SIn, SOut, JOut, Unit] =
|
||||
scaladsl.BidiFlow.wrap(javaToScalaAdapterFlow(inMapping), scalaToJavaAdapterFlow(outMapping))(scaladsl.Keep.none)
|
||||
scaladsl.BidiFlow.fromFlowsMat(javaToScalaAdapterFlow(inMapping), scalaToJavaAdapterFlow(outMapping))(scaladsl.Keep.none)
|
||||
|
||||
implicit def pairMapping[J1, J2, S1, S2](implicit _1Mapping: JavaMapping[J1, S1], _2Mapping: JavaMapping[J2, S2]): JavaMapping[Pair[J1, J2], (S1, S2)] =
|
||||
new JavaMapping[Pair[J1, J2], (S1, S2)] {
|
||||
|
|
|
|||
|
|
@ -86,7 +86,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
* [[ServerBinding]].
|
||||
*/
|
||||
def bind(interface: String, port: Int, materializer: Materializer): Source[IncomingConnection, Future[ServerBinding]] =
|
||||
Source.adapt(delegate.bind(interface, port)(materializer)
|
||||
new Source(delegate.bind(interface, port)(materializer)
|
||||
.map(new IncomingConnection(_))
|
||||
.mapMaterializedValue(_.map(new ServerBinding(_))(ec)))
|
||||
|
||||
|
|
@ -105,7 +105,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
httpsContext: Option[HttpsContext],
|
||||
log: LoggingAdapter,
|
||||
materializer: Materializer): Source[IncomingConnection, Future[ServerBinding]] =
|
||||
Source.adapt(delegate.bind(interface, port, settings, httpsContext, log)(materializer)
|
||||
new Source(delegate.bind(interface, port, settings, httpsContext, log)(materializer)
|
||||
.map(new IncomingConnection(_))
|
||||
.mapMaterializedValue(_.map(new ServerBinding(_))(ec)))
|
||||
|
||||
|
|
@ -238,7 +238,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
* Every materialization of the produced flow will attempt to establish a new outgoing connection.
|
||||
*/
|
||||
def outgoingConnection(host: String, port: Int): Flow[HttpRequest, HttpResponse, Future[OutgoingConnection]] =
|
||||
Flow.wrap {
|
||||
Flow.fromGraph {
|
||||
akka.stream.scaladsl.Flow[HttpRequest].map(_.asScala)
|
||||
.viaMat(delegate.outgoingConnection(host, port))(Keep.right)
|
||||
.mapMaterializedValue(_.map(new OutgoingConnection(_))(ec))
|
||||
|
|
@ -248,7 +248,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
* Same as [[outgoingConnection]] but with HTTPS encryption.
|
||||
*/
|
||||
def outgoingConnectionTls(host: String, port: Int): Flow[HttpRequest, HttpResponse, Future[OutgoingConnection]] =
|
||||
Flow.wrap {
|
||||
Flow.fromGraph {
|
||||
akka.stream.scaladsl.Flow[HttpRequest].map(_.asScala)
|
||||
.viaMat(delegate.outgoingConnectionTls(host, port))(Keep.right)
|
||||
.mapMaterializedValue(_.map(new OutgoingConnection(_))(ec))
|
||||
|
|
@ -262,7 +262,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
localAddress: Option[InetSocketAddress],
|
||||
settings: ClientConnectionSettings,
|
||||
log: LoggingAdapter): Flow[HttpRequest, HttpResponse, Future[OutgoingConnection]] =
|
||||
Flow.wrap {
|
||||
Flow.fromGraph {
|
||||
akka.stream.scaladsl.Flow[HttpRequest].map(_.asScala)
|
||||
.viaMat(delegate.outgoingConnection(host, port, localAddress.asScala, settings, log))(Keep.right)
|
||||
.mapMaterializedValue(_.map(new OutgoingConnection(_))(ec))
|
||||
|
|
@ -279,7 +279,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
settings: ClientConnectionSettings,
|
||||
httpsContext: Option[HttpsContext],
|
||||
log: LoggingAdapter): Flow[HttpRequest, HttpResponse, Future[OutgoingConnection]] =
|
||||
Flow.wrap {
|
||||
Flow.fromGraph {
|
||||
akka.stream.scaladsl.Flow[HttpRequest].map(_.asScala)
|
||||
.viaMat(delegate.outgoingConnectionTls(host, port, localAddress.asScala, settings,
|
||||
httpsContext.map(_.asInstanceOf[akka.http.scaladsl.HttpsContext]), log))(Keep.right)
|
||||
|
|
@ -629,7 +629,7 @@ class Http(system: ExtendedActorSystem) extends akka.actor.Extension {
|
|||
.atopMat(wsLayer)((_, s) ⇒ adaptWsUpgradeResponse(s)))
|
||||
|
||||
private def adaptWsFlow(wsLayer: stream.scaladsl.Flow[sm.ws.Message, sm.ws.Message, Future[scaladsl.model.ws.WebsocketUpgradeResponse]]): Flow[Message, Message, Future[WebsocketUpgradeResponse]] =
|
||||
Flow.adapt(JavaMapping.adapterBidiFlow[Message, sm.ws.Message, sm.ws.Message, Message].joinMat(wsLayer)(Keep.right).mapMaterializedValue(adaptWsUpgradeResponse _))
|
||||
Flow.fromGraph(JavaMapping.adapterBidiFlow[Message, sm.ws.Message, sm.ws.Message, Message].joinMat(wsLayer)(Keep.right).mapMaterializedValue(adaptWsUpgradeResponse _))
|
||||
|
||||
private def adaptWsFlow[Mat](javaFlow: Flow[Message, Message, Mat]): stream.scaladsl.Flow[scaladsl.model.ws.Message, scaladsl.model.ws.Message, Mat] =
|
||||
stream.scaladsl.Flow[scaladsl.model.ws.Message]
|
||||
|
|
|
|||
|
|
@ -31,7 +31,7 @@ class IncomingConnection private[http] (delegate: akka.http.scaladsl.Http.Incomi
|
|||
*
|
||||
* Use `Flow.join` or one of the handleXXX methods to consume handle requests on this connection.
|
||||
*/
|
||||
def flow: Flow[HttpResponse, HttpRequest, Unit] = Flow.adapt(delegate.flow).asInstanceOf[Flow[HttpResponse, HttpRequest, Unit]]
|
||||
def flow: Flow[HttpResponse, HttpRequest, Unit] = Flow.fromGraph(delegate.flow).asInstanceOf[Flow[HttpResponse, HttpRequest, Unit]]
|
||||
|
||||
/**
|
||||
* Handles the connection with the given flow, which is materialized exactly once
|
||||
|
|
|
|||
|
|
@ -38,18 +38,18 @@ object WebsocketUpgradeResponse {
|
|||
import akka.http.impl.util.JavaMapping.Implicits._
|
||||
def adapt(scalaResponse: scaladsl.model.ws.WebsocketUpgradeResponse): WebsocketUpgradeResponse =
|
||||
scalaResponse match {
|
||||
case ValidUpgrade(response, chosen) ⇒
|
||||
case ValidUpgrade(resp, chosen) ⇒
|
||||
new WebsocketUpgradeResponse {
|
||||
def isValid: Boolean = true
|
||||
def response: HttpResponse = scalaResponse.response
|
||||
def response: HttpResponse = resp
|
||||
def chosenSubprotocol: Option[String] = chosen.asJava
|
||||
def invalidationReason: String =
|
||||
throw new UnsupportedOperationException("invalidationReason must not be called for valid response")
|
||||
}
|
||||
case InvalidUpgradeResponse(response, cause) ⇒
|
||||
case InvalidUpgradeResponse(resp, cause) ⇒
|
||||
new WebsocketUpgradeResponse {
|
||||
def isValid: Boolean = false
|
||||
def response: HttpResponse = scalaResponse.response
|
||||
def response: HttpResponse = resp
|
||||
def chosenSubprotocol: Option[String] = throw new UnsupportedOperationException("chosenSubprotocol must not be called for valid response")
|
||||
def invalidationReason: String = cause
|
||||
}
|
||||
|
|
|
|||
|
|
@ -75,7 +75,7 @@ sealed trait HttpEntity extends jm.HttpEntity {
|
|||
def withContentType(contentType: ContentType): HttpEntity
|
||||
|
||||
/** Java API */
|
||||
def getDataBytes: stream.javadsl.Source[ByteString, _] = stream.javadsl.Source.adapt(dataBytes)
|
||||
def getDataBytes: stream.javadsl.Source[ByteString, _] = stream.javadsl.Source.fromGraph(dataBytes)
|
||||
|
||||
/** Java API */
|
||||
def getContentLengthOption: japi.Option[JLong] =
|
||||
|
|
@ -296,7 +296,7 @@ object HttpEntity {
|
|||
override def productPrefix = "HttpEntity.Chunked"
|
||||
|
||||
/** Java API */
|
||||
def getChunks: stream.javadsl.Source[jm.ChunkStreamPart, Any] = stream.javadsl.Source.adapt(chunks)
|
||||
def getChunks: stream.javadsl.Source[jm.ChunkStreamPart, Any] = stream.javadsl.Source.fromGraph(chunks)
|
||||
}
|
||||
object Chunked {
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -52,7 +52,7 @@ trait UpgradeToWebsocket extends jm.ws.UpgradeToWebsocket {
|
|||
def handleMessagesWithSinkSource(inSink: Sink[Message, Any],
|
||||
outSource: Source[Message, Any],
|
||||
subprotocol: Option[String] = None): HttpResponse =
|
||||
handleMessages(Flow.wrap(inSink, outSource)((_, _) ⇒ ()), subprotocol)
|
||||
handleMessages(Flow.fromSinkAndSource(inSink, outSource), subprotocol)
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
|
|
@ -88,5 +88,5 @@ trait UpgradeToWebsocket extends jm.ws.UpgradeToWebsocket {
|
|||
handleMessages(createScalaFlow(inSink, outSource), subprotocol = Some(subprotocol))
|
||||
|
||||
private[this] def createScalaFlow(inSink: stream.javadsl.Sink[jm.ws.Message, _], outSource: stream.javadsl.Source[jm.ws.Message, _]): Flow[Message, Message, Any] =
|
||||
JavaMapping.toScala(Flow.wrap(inSink.asScala, outSource.asScala)((_, _) ⇒ ()).asJava)
|
||||
JavaMapping.toScala(Flow.fromSinkAndSourceMat(inSink.asScala, outSource.asScala)((_, _) ⇒ ()).asJava)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -63,7 +63,7 @@ public class WSEchoTestClientApp {
|
|||
.toMat(Sink.<List<String>>head(), Keep.<BoxedUnit, Future<List<String>>>right());
|
||||
|
||||
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 =
|
||||
Http.get(system).singleWebsocketRequest(
|
||||
|
|
|
|||
|
|
@ -298,12 +298,10 @@ class ConnectionPoolSpec extends AkkaSpec("""
|
|||
val incomingConnectionCounter = new AtomicInteger
|
||||
val incomingConnections = TestSubscriber.manualProbe[Http.IncomingConnection]
|
||||
val incomingConnectionsSub = {
|
||||
val rawBytesInjection = BidiFlow() { b ⇒
|
||||
val top = b.add(Flow[SslTlsOutbound].collect[ByteString] { case SendBytes(x) ⇒ mapServerSideOutboundRawBytes(x) }
|
||||
.transform(StreamUtils.recover { case NoErrorComplete ⇒ ByteString.empty }))
|
||||
val bottom = b.add(Flow[ByteString].map(SessionBytes(null, _)))
|
||||
BidiShape(top.inlet, top.outlet, bottom.inlet, bottom.outlet)
|
||||
}
|
||||
val rawBytesInjection = BidiFlow.fromFlows(
|
||||
Flow[SslTlsOutbound].collect[ByteString] { case SendBytes(x) ⇒ mapServerSideOutboundRawBytes(x) }
|
||||
.transform(StreamUtils.recover { case NoErrorComplete ⇒ ByteString.empty }),
|
||||
Flow[ByteString].map(SessionBytes(null, _)))
|
||||
val sink = if (autoAccept) Sink.foreach[Http.IncomingConnection](handleConnection) else Sink(incomingConnections)
|
||||
// TODO getHostString in Java7
|
||||
Tcp().bind(serverEndpoint.getHostName, serverEndpoint.getPort, idleTimeout = serverSettings.timeouts.idleTimeout)
|
||||
|
|
|
|||
|
|
@ -6,7 +6,7 @@ package akka.http.impl.engine.client
|
|||
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.stream.{ FlowShape, ActorMaterializer }
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.testkit.AkkaSpec
|
||||
import akka.http.scaladsl.{ Http, TestUtils }
|
||||
|
|
@ -44,7 +44,7 @@ class HighLevelOutgoingConnectionSpec extends AkkaSpec {
|
|||
val connFlow = Http().outgoingConnection(serverHostName, serverPort)
|
||||
|
||||
val C = 4
|
||||
val doubleConnection = Flow() { implicit b ⇒
|
||||
val doubleConnection = Flow.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
val bcast = b.add(Broadcast[HttpRequest](C))
|
||||
|
|
@ -52,8 +52,8 @@ class HighLevelOutgoingConnectionSpec extends AkkaSpec {
|
|||
|
||||
for (i ← 0 until C)
|
||||
bcast.out(i) ~> connFlow ~> merge.in(i)
|
||||
(bcast.in, merge.out)
|
||||
}
|
||||
FlowShape(bcast.in, merge.out)
|
||||
})
|
||||
|
||||
val N = 100
|
||||
val result = Source(() ⇒ Iterator.from(1))
|
||||
|
|
|
|||
|
|
@ -9,7 +9,7 @@ import akka.stream.io.{ SessionBytes, SslTlsOutbound, SendBytes }
|
|||
import org.scalatest.Inside
|
||||
import akka.util.ByteString
|
||||
import akka.event.NoLogging
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.stream.{ ClosedShape, ActorMaterializer }
|
||||
import akka.stream.testkit._
|
||||
import akka.stream.scaladsl._
|
||||
import akka.http.scaladsl.model.HttpEntity._
|
||||
|
|
@ -359,14 +359,15 @@ class LowLevelOutgoingConnectionSpec extends AkkaSpec("akka.loggers = []\n akka.
|
|||
val netOut = TestSubscriber.manualProbe[ByteString]
|
||||
val netIn = TestPublisher.manualProbe[ByteString]()
|
||||
|
||||
FlowGraph.closed(OutgoingConnectionBlueprint(Host("example.com"), settings, NoLogging)) { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create(OutgoingConnectionBlueprint(Host("example.com"), settings, NoLogging)) { implicit b ⇒
|
||||
client ⇒
|
||||
import FlowGraph.Implicits._
|
||||
Source(netIn) ~> Flow[ByteString].map(SessionBytes(null, _)) ~> client.in2
|
||||
client.out1 ~> Flow[SslTlsOutbound].collect { case SendBytes(x) ⇒ x } ~> Sink(netOut)
|
||||
Source(requests) ~> client.in1
|
||||
client.out2 ~> Sink(responses)
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
netOut -> netIn
|
||||
}
|
||||
|
|
|
|||
|
|
@ -15,8 +15,8 @@ import akka.actor.ActorSystem
|
|||
import akka.event.NoLogging
|
||||
import akka.util.ByteString
|
||||
|
||||
import akka.stream.Materializer
|
||||
import akka.stream.scaladsl.{ Flow, Sink, Source, FlowGraph }
|
||||
import akka.stream.{ ClosedShape, Materializer }
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.testkit.{ TestPublisher, TestSubscriber }
|
||||
|
||||
import akka.http.impl.util._
|
||||
|
|
@ -39,14 +39,15 @@ abstract class HttpServerTestSetupBase {
|
|||
val netIn = TestPublisher.probe[ByteString]()
|
||||
val netOut = ByteStringSinkProbe()
|
||||
|
||||
FlowGraph.closed(HttpServerBluePrint(settings, remoteAddress = remoteAddress, log = NoLogging)) { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create(HttpServerBluePrint(settings, remoteAddress = remoteAddress, log = NoLogging)) { implicit b ⇒
|
||||
server ⇒
|
||||
import FlowGraph.Implicits._
|
||||
Source(netIn) ~> Flow[ByteString].map(SessionBytes(null, _)) ~> server.in2
|
||||
server.out1 ~> Flow[SslTlsOutbound].collect { case SendBytes(x) ⇒ x } ~> netOut.sink
|
||||
server.out2 ~> Sink(requests)
|
||||
Source(responses) ~> server.in1
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
netIn -> netOut
|
||||
}
|
||||
|
|
|
|||
|
|
@ -50,7 +50,7 @@ object EchoTestClientApp extends App {
|
|||
.grouped(10000)
|
||||
.toMat(Sink.head)(Keep.right)
|
||||
|
||||
def echoClient = Flow.wrap(sink, source)(Keep.left)
|
||||
def echoClient = Flow.fromSinkAndSourceMat(sink, source)(Keep.left)
|
||||
|
||||
val (upgrade, res) = Http().singleWebsocketRequest("wss://echo.websocket.org", echoClient)
|
||||
res onComplete {
|
||||
|
|
|
|||
|
|
@ -817,8 +817,8 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
|||
val messageOut = TestPublisher.probe[Message]()
|
||||
|
||||
val messageHandler: Flow[Message, Message, Unit] =
|
||||
Flow.wrap {
|
||||
FlowGraph.partial() { implicit b ⇒
|
||||
Flow.fromGraph {
|
||||
FlowGraph.create() { implicit b ⇒
|
||||
val in = b.add(Sink(messageIn))
|
||||
val out = b.add(Source(messageOut))
|
||||
|
||||
|
|
|
|||
|
|
@ -199,7 +199,7 @@ object WSClientAutobahnTest extends App {
|
|||
*/
|
||||
def runToSingleText(uri: Uri): Future[String] = {
|
||||
val sink = Sink.head[Message]
|
||||
runWs(uri, Flow.wrap(sink, Source.lazyEmpty[Message])(Keep.left)).flatMap {
|
||||
runWs(uri, Flow.fromSinkAndSourceMat(sink, Source.maybe[Message])(Keep.left)).flatMap {
|
||||
case tm: TextMessage ⇒ tm.textStream.runWith(Sink.fold("")(_ + _))
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -7,6 +7,7 @@ package akka.http.impl.engine.ws
|
|||
import java.util.Random
|
||||
|
||||
import akka.http.scaladsl.model.ws.{ InvalidUpgradeResponse, WebsocketUpgradeResponse }
|
||||
import akka.stream.ClosedShape
|
||||
|
||||
import scala.concurrent.duration._
|
||||
|
||||
|
|
@ -119,7 +120,7 @@ class WebsocketClientSpec extends FreeSpec with Matchers with WithMaterializerSp
|
|||
|
||||
"don't send out frames before handshake was finished successfully" in new TestSetup {
|
||||
def clientImplementation: Flow[Message, Message, Unit] =
|
||||
Flow.wrap(Sink.ignore, Source.single(TextMessage("fast message")))(Keep.none)
|
||||
Flow.fromSinkAndSourceMat(Sink.ignore, Source.single(TextMessage("fast message")))(Keep.none)
|
||||
|
||||
expectWireData(UpgradeRequestBytes)
|
||||
expectNoWireData()
|
||||
|
|
@ -311,13 +312,14 @@ class WebsocketClientSpec extends FreeSpec with Matchers with WithMaterializerSp
|
|||
val netIn = TestPublisher.probe[ByteString]()
|
||||
|
||||
val graph =
|
||||
FlowGraph.closed(clientLayer) { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create(clientLayer) { implicit b ⇒
|
||||
client ⇒
|
||||
import FlowGraph.Implicits._
|
||||
Source(netIn) ~> Flow[ByteString].map(SessionBytes(null, _)) ~> client.in2
|
||||
client.out1 ~> Flow[SslTlsOutbound].collect { case SendBytes(x) ⇒ x } ~> netOut.sink
|
||||
client.out2 ~> clientImplementation ~> client.in1
|
||||
}
|
||||
ClosedShape
|
||||
})
|
||||
|
||||
val response = graph.run()
|
||||
|
||||
|
|
@ -365,6 +367,6 @@ class WebsocketClientSpec extends FreeSpec with Matchers with WithMaterializerSp
|
|||
lazy val messagesIn = TestSubscriber.probe[Message]()
|
||||
|
||||
override def clientImplementation: Flow[Message, Message, Unit] =
|
||||
Flow.wrap(Sink(messagesIn), Source(messagesOut))(Keep.none)
|
||||
Flow.fromSinkAndSourceMat(Sink(messagesIn), Source(messagesOut))(Keep.none)
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -38,7 +38,7 @@ class WebsocketServerSpec extends FreeSpec with Matchers with WithMaterializerSp
|
|||
|
||||
val source =
|
||||
Source(List(1, 2, 3, 4, 5)).map(num ⇒ TextMessage.Strict(s"Message $num"))
|
||||
val handler = Flow.wrap(Sink.ignore, source)(Keep.none)
|
||||
val handler = Flow.fromSinkAndSourceMat(Sink.ignore, source)(Keep.none)
|
||||
val response = upgrade.get.handleMessages(handler)
|
||||
responses.sendNext(response)
|
||||
|
||||
|
|
|
|||
|
|
@ -330,10 +330,7 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
|
|||
val sink = Sink.publisher[HttpRequest]
|
||||
val source = Source.subscriber[HttpResponse]
|
||||
|
||||
val handler = Flow(sink, source)(Keep.both) { implicit b ⇒
|
||||
(snk, src) ⇒
|
||||
(snk.inlet, src.outlet)
|
||||
}
|
||||
val handler = Flow.fromSinkAndSourceMat(sink, source)(Keep.both)
|
||||
|
||||
val (pub, sub) = incomingConnection.handleWith(handler)
|
||||
val requestSubscriberProbe = TestSubscriber.manualProbe[HttpRequest]()
|
||||
|
|
|
|||
|
|
@ -104,7 +104,7 @@ object WSProbe {
|
|||
val subscriber = TestSubscriber.probe[Message]()
|
||||
val publisher = TestPublisher.probe[Message]()
|
||||
|
||||
def flow: Flow[Message, Message, Unit] = Flow.wrap(Sink(subscriber), Source(publisher))(Keep.none)
|
||||
def flow: Flow[Message, Message, Unit] = Flow.fromSinkAndSourceMat(Sink(subscriber), Source(publisher))(Keep.none)
|
||||
|
||||
def sendMessage(message: Message): Unit = publisher.sendNext(message)
|
||||
def sendMessage(text: String): Unit = sendMessage(TextMessage(text))
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
@ -1,6 +1,6 @@
|
|||
package akka.stream.testkit
|
||||
|
||||
import akka.stream.{ ActorMaterializer, ActorMaterializerSettings, Inlet, Outlet }
|
||||
import akka.stream._
|
||||
import akka.stream.scaladsl._
|
||||
import org.reactivestreams.Publisher
|
||||
import scala.collection.immutable
|
||||
|
|
@ -19,15 +19,15 @@ abstract class TwoStreamsSetup extends BaseTwoStreamsSetup {
|
|||
|
||||
override def setup(p1: Publisher[Int], p2: Publisher[Int]) = {
|
||||
val subscriber = TestSubscriber.probe[Outputs]()
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
import FlowGraph.Implicits._
|
||||
val f = fixture(b)
|
||||
|
||||
Source(p1) ~> f.left
|
||||
Source(p2) ~> f.right
|
||||
f.out ~> Sink(subscriber)
|
||||
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
subscriber
|
||||
}
|
||||
|
|
|
|||
|
|
@ -35,21 +35,20 @@ public class BidiFlowTest extends StreamTest {
|
|||
"FlowTest", AkkaSpec.testConf());
|
||||
|
||||
private final BidiFlow<Integer, Long, ByteString, String, BoxedUnit> bidi = BidiFlow
|
||||
.factory()
|
||||
.create(
|
||||
.fromGraph(FlowGraph.create(
|
||||
new Function<FlowGraph.Builder<BoxedUnit>, BidiShape<Integer, Long, ByteString, String>>() {
|
||||
@Override
|
||||
public BidiShape<Integer, Long, ByteString, String> apply(Builder<BoxedUnit> b)
|
||||
throws Exception {
|
||||
final FlowShape<Integer, Long> top = b.graph(Flow
|
||||
.<Integer> empty().map(new Function<Integer, Long>() {
|
||||
final FlowShape<Integer, Long> top = b.add(Flow
|
||||
.of(Integer.class).map(new Function<Integer, Long>() {
|
||||
@Override
|
||||
public Long apply(Integer arg) {
|
||||
return (long) ((int) arg) + 2;
|
||||
}
|
||||
}));
|
||||
final FlowShape<ByteString, String> bottom = b.graph(Flow
|
||||
.<ByteString> empty().map(new Function<ByteString, String>() {
|
||||
final FlowShape<ByteString, String> bottom = b.add(Flow
|
||||
.of(ByteString.class).map(new Function<ByteString, String>() {
|
||||
@Override
|
||||
public String apply(ByteString arg) {
|
||||
return arg.decodeString("UTF-8");
|
||||
|
|
@ -58,24 +57,24 @@ public class BidiFlowTest extends StreamTest {
|
|||
return new BidiShape<Integer, Long, ByteString, String>(top
|
||||
.inlet(), top.outlet(), bottom.inlet(), bottom.outlet());
|
||||
}
|
||||
});
|
||||
}));
|
||||
|
||||
private final BidiFlow<Long, Integer, String, ByteString, BoxedUnit> inverse = BidiFlow
|
||||
.factory()
|
||||
.create(
|
||||
.fromGraph(
|
||||
FlowGraph.create(
|
||||
new Function<FlowGraph.Builder<BoxedUnit>, BidiShape<Long, Integer, String, ByteString>>() {
|
||||
@Override
|
||||
public BidiShape<Long, Integer, String, ByteString> apply(Builder<BoxedUnit> b)
|
||||
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>() {
|
||||
@Override
|
||||
public Integer apply(Long arg) {
|
||||
return (int) ((long) arg) + 2;
|
||||
}
|
||||
}));
|
||||
final FlowShape<String, ByteString> bottom = b.graph(Flow
|
||||
.<String> empty().map(new Function<String, ByteString>() {
|
||||
final FlowShape<String, ByteString> bottom = b.add(Flow
|
||||
.of(String.class).map(new Function<String, ByteString>() {
|
||||
@Override
|
||||
public ByteString apply(String arg) {
|
||||
return ByteString.fromString(arg);
|
||||
|
|
@ -84,26 +83,26 @@ public class BidiFlowTest extends StreamTest {
|
|||
return new BidiShape<Long, Integer, String, ByteString>(top
|
||||
.inlet(), top.outlet(), bottom.inlet(), bottom.outlet());
|
||||
}
|
||||
});
|
||||
}));
|
||||
|
||||
private final BidiFlow<Integer, Long, ByteString, String, Future<Integer>> bidiMat = BidiFlow
|
||||
.factory()
|
||||
.create(
|
||||
private final BidiFlow<Integer, Long, ByteString, String, Future<Integer>> bidiMat =
|
||||
BidiFlow.fromGraph(
|
||||
FlowGraph.create(
|
||||
Sink.<Integer>head(),
|
||||
new Function2<FlowGraph.Builder<Future<Integer>>, SinkShape<Integer>, BidiShape<Integer, Long, ByteString, String>>() {
|
||||
@Override
|
||||
public BidiShape<Integer, Long, ByteString, String> apply(Builder<Future<Integer>> b, SinkShape<Integer> sink)
|
||||
throws Exception {
|
||||
b.from(b.graph(Source.single(42))).to(sink);
|
||||
final FlowShape<Integer, Long> top = b.graph(Flow
|
||||
.<Integer> empty().map(new Function<Integer, Long>() {
|
||||
b.from(b.add(Source.single(42))).to(sink);
|
||||
final FlowShape<Integer, Long> top = b.add(Flow
|
||||
.of(Integer.class).map(new Function<Integer, Long>() {
|
||||
@Override
|
||||
public Long apply(Integer arg) {
|
||||
return (long) ((int) arg) + 2;
|
||||
}
|
||||
}));
|
||||
final FlowShape<ByteString, String> bottom = b.graph(Flow
|
||||
.<ByteString> empty().map(new Function<ByteString, String>() {
|
||||
final FlowShape<ByteString, String> bottom = b.add(Flow
|
||||
.of(ByteString.class).map(new Function<ByteString, String>() {
|
||||
@Override
|
||||
public String apply(ByteString arg) {
|
||||
return arg.decodeString("UTF-8");
|
||||
|
|
@ -112,7 +111,7 @@ public class BidiFlowTest extends StreamTest {
|
|||
return new BidiShape<Integer, Long, ByteString, String>(top
|
||||
.inlet(), top.outlet(), bottom.inlet(), bottom.outlet());
|
||||
}
|
||||
});
|
||||
}));
|
||||
|
||||
private final String str = "Hello World";
|
||||
private final ByteString bytes = ByteString.fromString(str);
|
||||
|
|
@ -126,22 +125,23 @@ public class BidiFlowTest extends StreamTest {
|
|||
|
||||
@Test
|
||||
public void mustWorkInIsolation() throws Exception {
|
||||
final Pair<Future<Long>, Future<String>> p = FlowGraph
|
||||
.factory()
|
||||
.closed(Sink.<Long> head(), Sink.<String> head(),
|
||||
final Pair<Future<Long>, Future<String>> p =
|
||||
RunnableGraph.fromGraph(FlowGraph
|
||||
.create(Sink.<Long> head(), Sink.<String> head(),
|
||||
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
|
||||
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 {
|
||||
final BidiShape<Integer, Long, ByteString, String> s = b
|
||||
.graph(bidi);
|
||||
b.from(b.graph(Source.single(1))).toInlet(s.in1());
|
||||
final BidiShape<Integer, Long, ByteString, String> s =
|
||||
b.add(bidi);
|
||||
b.from(b.add(Source.single(1))).toInlet(s.in1());
|
||||
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);
|
||||
return ClosedShape.getInstance();
|
||||
}
|
||||
}).run(materializer);
|
||||
})).run(materializer);
|
||||
|
||||
final Long rt = Await.result(p.first(), oneSec);
|
||||
final String rb = Await.result(p.second(), oneSec);
|
||||
|
|
@ -152,7 +152,7 @@ public class BidiFlowTest extends StreamTest {
|
|||
|
||||
@Test
|
||||
public void mustWorkAsAFlowThatIsOpenOnTheLeft() throws Exception {
|
||||
final Flow<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>() {
|
||||
@Override public ByteString apply(Long arg) {
|
||||
return ByteString.fromString("Hello " + arg);
|
||||
|
|
@ -164,7 +164,7 @@ public class BidiFlowTest extends StreamTest {
|
|||
|
||||
@Test
|
||||
public void mustWorkAsAFlowThatIsOpenOnTheRight() throws Exception {
|
||||
final Flow<ByteString, Long, BoxedUnit> f = Flow.<String> empty().map(
|
||||
final Flow<ByteString, Long, BoxedUnit> f = Flow.of(String.class).map(
|
||||
new Function<String, Integer>() {
|
||||
@Override public Integer apply(String arg) {
|
||||
return Integer.valueOf(arg);
|
||||
|
|
@ -177,7 +177,7 @@ public class BidiFlowTest extends StreamTest {
|
|||
|
||||
@Test
|
||||
public void mustWorkWhenAtopItsInverse() throws Exception {
|
||||
final Flow<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>() {
|
||||
@Override public String apply(Integer arg) {
|
||||
return arg.toString();
|
||||
|
|
@ -189,7 +189,7 @@ public class BidiFlowTest extends StreamTest {
|
|||
|
||||
@Test
|
||||
public void mustWorkWhenReversed() throws Exception {
|
||||
final Flow<Integer,String,BoxedUnit> f = Flow.<Integer> empty().map(
|
||||
final Flow<Integer,String,BoxedUnit> f = Flow.of(Integer.class).map(
|
||||
new Function<Integer, String>() {
|
||||
@Override public String apply(Integer arg) {
|
||||
return arg.toString();
|
||||
|
|
@ -201,39 +201,44 @@ public class BidiFlowTest extends StreamTest {
|
|||
|
||||
@Test
|
||||
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
|
||||
public void apply(Builder<Future<Integer>> b,
|
||||
public ClosedShape apply(Builder<Future<Integer>> b,
|
||||
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>() {
|
||||
@Override public Integer apply(String arg) {
|
||||
@Override
|
||||
public Integer apply(String arg) {
|
||||
return Integer.valueOf(arg);
|
||||
}
|
||||
}));
|
||||
final FlowShape<Long, ByteString> right = b.graph(Flow.<Long> empty().map(
|
||||
final FlowShape<Long, ByteString> right = b.add(Flow.of(Long.class).map(
|
||||
new Function<Long, ByteString>() {
|
||||
@Override public ByteString apply(Long arg) {
|
||||
@Override
|
||||
public ByteString apply(Long arg) {
|
||||
return ByteString.fromString("Hello " + arg);
|
||||
}
|
||||
}));
|
||||
b.from(shape.out2()).via(left).toInlet(shape.in1())
|
||||
.from(shape.out1()).via(right).toInlet(shape.in2());
|
||||
return ClosedShape.getInstance();
|
||||
}
|
||||
}).run(materializer);
|
||||
})).run(materializer);
|
||||
assertEquals((Integer) 42, Await.result(f, oneSec));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void mustCombineMaterializationValues() throws Exception {
|
||||
final Flow<String, Integer, Future<Integer>> left = Flow.factory().create(
|
||||
Sink.<Integer> head(), new Function2<Builder<Future<Integer> >, SinkShape<Integer>, Pair<Inlet<String>, Outlet<Integer>>>() {
|
||||
final Flow<String, Integer, Future<Integer>> left = Flow.fromGraph(FlowGraph.create(
|
||||
Sink.<Integer>head(), new Function2<Builder<Future<Integer>>, SinkShape<Integer>, FlowShape<String, Integer>>() {
|
||||
@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 {
|
||||
final UniformFanOutShape<Integer, Integer> bcast = b.graph(Broadcast.<Integer> create(2));
|
||||
final UniformFanInShape<Integer, Integer> merge = b.graph(Merge.<Integer> create(2));
|
||||
final FlowShape<String, Integer> flow = b.graph(Flow.<String> empty().map(
|
||||
final UniformFanOutShape<Integer, Integer> bcast = b.add(Broadcast.<Integer>create(2));
|
||||
final UniformFanInShape<Integer, Integer> merge = b.add(Merge.<Integer>create(2));
|
||||
final FlowShape<String, Integer> flow = b.add(Flow.of(String.class).map(
|
||||
new Function<String, Integer>() {
|
||||
@Override
|
||||
public Integer apply(String arg) {
|
||||
|
|
@ -241,21 +246,21 @@ public class BidiFlowTest extends StreamTest {
|
|||
}
|
||||
}));
|
||||
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);
|
||||
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(
|
||||
Sink.<List<Long>> head(), new Function2<Builder<Future<List<Long>>>, SinkShape<List<Long>>, Pair<Inlet<Long>, Outlet<ByteString>>>() {
|
||||
}));
|
||||
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>>, FlowShape<Long, ByteString>>() {
|
||||
@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 {
|
||||
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);
|
||||
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 =
|
||||
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();
|
||||
|
|
|
|||
|
|
@ -69,15 +69,16 @@ public class FlowGraphTest extends StreamTest {
|
|||
|
||||
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(
|
||||
FlowGraph.create(new Function<FlowGraph.Builder<BoxedUnit>, SourceShape<String>>() {
|
||||
@Override
|
||||
public Outlet<String> apply(Builder<BoxedUnit> b) throws Exception {
|
||||
final UniformFanInShape<String, String> merge = b.graph(Merge.<String> create(2));
|
||||
b.flow(b.source(in1), f1, merge.in(0));
|
||||
b.flow(b.source(in2), f2, merge.in(1));
|
||||
return merge.out();
|
||||
public SourceShape<String> apply(Builder<BoxedUnit> b) throws Exception {
|
||||
final UniformFanInShape<String, String> merge = b.add(Merge.<String>create(2));
|
||||
b.from(b.add(in1)).via(b.add(f1)).toInlet(merge.in(0));
|
||||
b.from(b.add(in2)).via(b.add(f2)).toInlet(merge.in(1));
|
||||
return new SourceShape<String>(merge.out());
|
||||
}
|
||||
});
|
||||
}));
|
||||
|
||||
// collecting
|
||||
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<Integer> input2 = Arrays.asList(1, 2, 3);
|
||||
|
||||
final Builder<BoxedUnit> b = FlowGraph.builder();
|
||||
RunnableGraph.fromGraph( FlowGraph.create(
|
||||
new Function<Builder<BoxedUnit>,ClosedShape>() {
|
||||
@Override
|
||||
public ClosedShape apply(final Builder<BoxedUnit> b) throws Exception {
|
||||
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.graph(Zip.<String, Integer>create());
|
||||
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.edge(b.source(in2), zip.in1());
|
||||
b.edge(zip.out(), b.sink(out));
|
||||
b.run(materializer);
|
||||
b.from(b.add(in1)).toInlet(zip.in0());
|
||||
b.from(b.add(in2)).toInlet(zip.in1());
|
||||
b.from(zip.out()).to(b.add(out));
|
||||
return ClosedShape.getInstance();
|
||||
}
|
||||
})).run(materializer);
|
||||
|
||||
List<Object> output = Arrays.asList(probe.receiveN(3));
|
||||
@SuppressWarnings("unchecked")
|
||||
|
|
@ -123,17 +129,22 @@ public class FlowGraphTest extends StreamTest {
|
|||
final Iterable<String> expected1 = Arrays.asList("A", "B", "C");
|
||||
final Iterable<Integer> expected2 = Arrays.asList(1, 2, 3);
|
||||
|
||||
final Builder<BoxedUnit> b = FlowGraph.builder();
|
||||
final Outlet<Pair<String, Integer>> in = b.source(Source.from(input));
|
||||
final FanOutShape2<Pair<String, Integer>, String, Integer> unzip = b.graph(Unzip.<String, Integer>create());
|
||||
RunnableGraph.fromGraph(FlowGraph.create(
|
||||
new Function<Builder<BoxedUnit>, ClosedShape>() {
|
||||
@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 Sink<Integer, BoxedUnit> out2 = createSink(probe2);
|
||||
final SinkShape<String> out1 = b.add(FlowGraphTest.<String>createSink(probe1));
|
||||
final SinkShape<Integer> out2 = b.add(FlowGraphTest.<Integer>createSink(probe2));
|
||||
|
||||
b.edge(in, unzip.in());
|
||||
b.edge(unzip.out0(), b.sink(out1));
|
||||
b.edge(unzip.out1(), b.sink(out2));
|
||||
b.run(materializer);
|
||||
b.from(in).toInlet(unzip.in());
|
||||
b.from(unzip.out0()).to(out1);
|
||||
b.from(unzip.out1()).to(out2);
|
||||
return ClosedShape.getInstance();
|
||||
}
|
||||
})).run(materializer);
|
||||
|
||||
List<Object> output1 = Arrays.asList(probe1.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 probe2 = new JavaTestKit(system);
|
||||
|
||||
final Builder<BoxedUnit> b = FlowGraph.builder();
|
||||
RunnableGraph.fromGraph(FlowGraph.create(
|
||||
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>>() {
|
||||
@Override public Pair<String, Integer> apply(Integer l) throws Exception {
|
||||
@Override
|
||||
public Pair<String, Integer> apply(Integer l) throws Exception {
|
||||
return new Pair<String, Integer>(l + "!", l);
|
||||
}
|
||||
})
|
||||
);
|
||||
|
||||
final Sink<String, BoxedUnit> out1 = createSink(probe1);
|
||||
final Sink<Integer, BoxedUnit> out2 = createSink(probe2);
|
||||
final SinkShape<String> out1 = b.add(FlowGraphTest.<String>createSink(probe1));
|
||||
final SinkShape<Integer> out2 = b.add(FlowGraphTest.<Integer>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);
|
||||
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);
|
||||
|
||||
Duration d = Duration.create(300, TimeUnit.MILLISECONDS);
|
||||
|
||||
|
|
@ -186,28 +204,34 @@ public class FlowGraphTest extends StreamTest {
|
|||
final JavaTestKit probe3 = new JavaTestKit(system);
|
||||
final JavaTestKit probe4 = new JavaTestKit(system);
|
||||
|
||||
final Builder<BoxedUnit> b = FlowGraph.builder();
|
||||
RunnableGraph.fromGraph(FlowGraph.create(
|
||||
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>>() {
|
||||
@Override public Tuple4<String, Integer, String, Integer> apply(Integer l) throws Exception {
|
||||
@Override
|
||||
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 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);
|
||||
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.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);
|
||||
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);
|
||||
|
||||
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
|
||||
public void apply(Builder<Future<Integer> > b, SinkShape<Integer> out) throws Exception {
|
||||
final FanInShape2<Integer, Integer, Integer> zip = b.graph(sumZip);
|
||||
b.edge(b.source(in1), zip.in0());
|
||||
b.edge(b.source(in2), zip.in1());
|
||||
b.edge(zip.out(), out.inlet());
|
||||
public ClosedShape apply(Builder<Future<Integer>> b, SinkShape<Integer> out) throws Exception {
|
||||
final FanInShape2<Integer, Integer, Integer> zip = b.add(sumZip);
|
||||
b.from(b.add(in1)).toInlet(zip.in0());
|
||||
b.from(b.add(in2)).toInlet(zip.in1());
|
||||
b.from(zip.out()).to(out);
|
||||
return ClosedShape.getInstance();
|
||||
}
|
||||
}).run(materializer);
|
||||
})).run(materializer);
|
||||
|
||||
final Integer result = Await.result(future, Duration.create(300, TimeUnit.MILLISECONDS));
|
||||
assertEquals(11, (int) result);
|
||||
|
|
@ -262,17 +288,20 @@ public class FlowGraphTest extends StreamTest {
|
|||
}
|
||||
});
|
||||
|
||||
final Future<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
|
||||
public void apply(Builder<Future<Integer>> b, SinkShape<Integer> out) throws Exception {
|
||||
final FanInShape4<Integer, Integer, Integer, Integer, Integer> zip = b.graph(sumZip);
|
||||
b.edge(b.source(in1), zip.in0());
|
||||
b.edge(b.source(in2), zip.in1());
|
||||
b.edge(b.source(in3), zip.in2());
|
||||
b.edge(b.source(in4), zip.in3());
|
||||
b.edge(zip.out(), out.inlet());
|
||||
public ClosedShape apply(Builder<Future<Integer>> b, SinkShape<Integer> out) throws Exception {
|
||||
final FanInShape4<Integer, Integer, Integer, Integer, Integer> zip = b.add(sumZip);
|
||||
b.from(b.add(in1)).toInlet(zip.in0());
|
||||
b.from(b.add(in2)).toInlet(zip.in1());
|
||||
b.from(b.add(in3)).toInlet(zip.in2());
|
||||
b.from(b.add(in4)).toInlet(zip.in3());
|
||||
b.from(zip.out()).to(out);
|
||||
return ClosedShape.getInstance();
|
||||
}
|
||||
}).run(materializer);
|
||||
})).run(materializer);
|
||||
|
||||
final Integer result = Await.result(future, Duration.create(300, TimeUnit.MILLISECONDS));
|
||||
assertEquals(1111, (int) result);
|
||||
|
|
@ -284,17 +313,19 @@ public class FlowGraphTest extends StreamTest {
|
|||
final Source<Integer, BoxedUnit> in1 = Source.single(1);
|
||||
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
|
||||
public void apply(Builder<Future<Integer>> b, SinkShape<Integer> out) throws Exception {
|
||||
b.from(b.graph(Source.single(1))).to(out);
|
||||
b.from(b.materializedValue()).to(b.graph(Sink.foreach(new Procedure<Future<Integer>>(){
|
||||
public ClosedShape apply(Builder<Future<Integer>> b, SinkShape<Integer> out) throws Exception {
|
||||
b.from(b.add(Source.single(1))).to(out);
|
||||
b.from(b.materializedValue()).to(b.add(Sink.foreach(new Procedure<Future<Integer>>(){
|
||||
public void apply(Future<Integer> mat) throws Exception {
|
||||
Patterns.pipe(mat, system.dispatcher()).to(probe.ref());
|
||||
}
|
||||
})));
|
||||
return ClosedShape.getInstance();
|
||||
}
|
||||
}).run(materializer);
|
||||
})).run(materializer);
|
||||
|
||||
final Integer result = Await.result(future, Duration.create(300, TimeUnit.MILLISECONDS));
|
||||
assertEquals(1, (int) result);
|
||||
|
|
|
|||
|
|
@ -385,15 +385,16 @@ public class FlowTest extends StreamTest {
|
|||
|
||||
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(
|
||||
FlowGraph.create(new Function<FlowGraph.Builder<BoxedUnit>, SourceShape<String>>() {
|
||||
@Override
|
||||
public Outlet<String> apply(Builder<BoxedUnit> b) throws Exception {
|
||||
final UniformFanInShape<String, String> merge = b.graph(Merge.<String> create(2));
|
||||
b.flow(b.source(in1), f1, merge.in(0));
|
||||
b.flow(b.source(in2), f2, merge.in(1));
|
||||
return merge.out();
|
||||
public SourceShape<String> apply(Builder<BoxedUnit> b) throws Exception {
|
||||
final UniformFanInShape<String, String> merge = b.add(Merge.<String>create(2));
|
||||
b.from(b.add(in1)).via(b.add(f1)).toInlet(merge.in(0));
|
||||
b.from(b.add(in2)).via(b.add(f2)).toInlet(merge.in(1));
|
||||
return new SourceShape<String>(merge.out());
|
||||
}
|
||||
});
|
||||
}));
|
||||
|
||||
// collecting
|
||||
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<Integer> input2 = Arrays.asList(1, 2, 3);
|
||||
|
||||
final Builder<BoxedUnit> b = FlowGraph.<BoxedUnit>builder();
|
||||
final Outlet<String> in1 = b.source(Source.from(input1));
|
||||
final Outlet<Integer> in2 = b.source(Source.from(input2));
|
||||
final FanInShape2<String, Integer, Pair<String, Integer>> zip = b.graph(Zip.<String, Integer> create());
|
||||
final Inlet<Pair<String, Integer>> out = b.sink(Sink
|
||||
.foreach(new Procedure<Pair<String, Integer>>() {
|
||||
RunnableGraph.fromGraph(FlowGraph.create(new Function<Builder<BoxedUnit>, ClosedShape>(){
|
||||
public ClosedShape apply(Builder<BoxedUnit> b) {
|
||||
final Outlet<String> in1 = b.add(Source.from(input1)).outlet();
|
||||
final Outlet<Integer> in2 = b.add(Source.from(input2)).outlet();
|
||||
final FanInShape2<String, Integer, Pair<String, Integer>> zip = b.add(Zip.<String, Integer>create());
|
||||
final SinkShape<Pair<String, Integer>> out =
|
||||
b.add(Sink.foreach(new Procedure<Pair<String, Integer>>() {
|
||||
@Override
|
||||
public void apply(Pair<String, Integer> param) throws Exception {
|
||||
probe.getRef().tell(param, ActorRef.noSender());
|
||||
}
|
||||
}));
|
||||
|
||||
b.edge(in1, zip.in0());
|
||||
b.edge(in2, zip.in1());
|
||||
b.edge(zip.out(), out);
|
||||
|
||||
b.run(materializer);
|
||||
b.from(in1).toInlet(zip.in0());
|
||||
b.from(in2).toInlet(zip.in1());
|
||||
b.from(zip.out()).to(out);
|
||||
return ClosedShape.getInstance();
|
||||
}
|
||||
})).run(materializer);
|
||||
|
||||
List<Object> output = Arrays.asList(probe.receiveN(3));
|
||||
@SuppressWarnings("unchecked")
|
||||
|
|
@ -638,19 +641,18 @@ public class FlowTest extends StreamTest {
|
|||
|
||||
@Test
|
||||
public void mustBeAbleToBroadcastEagerCancel() throws Exception {
|
||||
final Sink<String, BoxedUnit> out1 = Sink.cancelled();
|
||||
final Sink<String, ?> out2 = Sink.ignore();
|
||||
|
||||
final Sink<String, BoxedUnit> sink = Sink.factory().<String>create(new Function<FlowGraph.Builder<BoxedUnit>, Inlet<String>>() {
|
||||
final Sink<String, BoxedUnit> sink = Sink.fromGraph(
|
||||
FlowGraph.create(new Function<FlowGraph.Builder<BoxedUnit>, SinkShape<String>>() {
|
||||
@Override
|
||||
public Inlet<String> apply(Builder<BoxedUnit> b) throws Exception {
|
||||
final UniformFanOutShape<String, String> broadcast = b.graph(Broadcast.<String>create(2, true));
|
||||
|
||||
b.from(broadcast.out(0)).to(b.graph(out1));
|
||||
b.from(broadcast.out(1)).to(b.graph(out2));
|
||||
return broadcast.in();
|
||||
public SinkShape<String> apply(Builder<BoxedUnit> b) throws Exception {
|
||||
final UniformFanOutShape<String, String> broadcast = b.add(Broadcast.<String>create(2, true));
|
||||
final SinkShape<String> out1 = b.add(Sink.<String>cancelled());
|
||||
final SinkShape<String> out2 = b.add(Sink.<String>ignore());
|
||||
b.from(broadcast.out(0)).to(out1);
|
||||
b.from(broadcast.out(1)).to(out2);
|
||||
return new SinkShape<String>(broadcast.in());
|
||||
}
|
||||
});
|
||||
}));
|
||||
|
||||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
Source<String, ActorRef> source = Source.actorRef(1, OverflowStrategy.dropNew());
|
||||
|
|
|
|||
|
|
@ -35,7 +35,7 @@ public class TcpTest extends StreamTest {
|
|||
final Sink<IncomingConnection, Future<BoxedUnit>> echoHandler =
|
||||
Sink.foreach(new Procedure<IncomingConnection>() {
|
||||
public void apply(IncomingConnection conn) {
|
||||
conn.handleWith(Flow.<ByteString>empty(), materializer);
|
||||
conn.handleWith(Flow.of(ByteString.class), materializer);
|
||||
}
|
||||
});
|
||||
|
||||
|
|
|
|||
|
|
@ -26,7 +26,7 @@ class ActorMaterializerSpec extends AkkaSpec with ImplicitSender {
|
|||
|
||||
val m = ActorMaterializer.create(system)
|
||||
|
||||
val f = Source.lazyEmpty[Int].runFold(0)(_ + _)(m)
|
||||
val f = Source.maybe[Int].runFold(0)(_ + _)(m)
|
||||
|
||||
m.shutdown()
|
||||
|
||||
|
|
@ -43,7 +43,7 @@ class ActorMaterializerSpec extends AkkaSpec with ImplicitSender {
|
|||
"shut down the supervisor actor it encapsulates" in {
|
||||
val m = ActorMaterializer.create(system).asInstanceOf[ActorMaterializerImpl]
|
||||
|
||||
Source.lazyEmpty[Any].to(Sink.ignore).run()(m)
|
||||
Source.maybe[Any].to(Sink.ignore).run()(m)
|
||||
m.supervisor ! StreamSupervisor.GetChildren
|
||||
expectMsgType[StreamSupervisor.Children]
|
||||
m.shutdown()
|
||||
|
|
|
|||
|
|
@ -66,11 +66,11 @@ class DslFactoriesConsistencySpec extends WordSpec with Matchers {
|
|||
}
|
||||
|
||||
val testCases = Seq(
|
||||
TestCase("Source", scaladsl.Source.getClass, javadsl.Source.getClass, classOf[javadsl.SourceCreate]),
|
||||
TestCase("Flow", scaladsl.Flow.getClass, javadsl.Flow.getClass, classOf[javadsl.FlowCreate]),
|
||||
TestCase("Sink", scaladsl.Sink.getClass, javadsl.Sink.getClass, classOf[javadsl.SinkCreate]),
|
||||
TestCase("Source", scaladsl.Source.getClass, javadsl.Source.getClass),
|
||||
TestCase("Flow", scaladsl.Flow.getClass, javadsl.Flow.getClass),
|
||||
TestCase("Sink", scaladsl.Sink.getClass, javadsl.Sink.getClass),
|
||||
TestCase("BidiFlow", scaladsl.BidiFlow.getClass, javadsl.BidiFlow.getClass),
|
||||
TestCase("FlowGraph", scaladsl.FlowGraph.getClass, javadsl.FlowGraph.getClass, classOf[javadsl.GraphCreate]),
|
||||
TestCase("BidiFlow", scaladsl.BidiFlow.getClass, javadsl.BidiFlow.getClass, classOf[javadsl.BidiFlowCreate]),
|
||||
TestCase("ZipWith", Some(scaladsl.ZipWith.getClass), None, Some(javadsl.ZipWith.getClass)),
|
||||
TestCase("Merge", scaladsl.Merge.getClass, javadsl.Merge.getClass),
|
||||
TestCase("MergePreferred", scaladsl.MergePreferred.getClass, javadsl.MergePreferred.getClass),
|
||||
|
|
@ -113,8 +113,8 @@ class DslFactoriesConsistencySpec extends WordSpec with Matchers {
|
|||
Ignore(_ == akka.stream.scaladsl.Flow.getClass, _ == "apply", _ == 24, _ ⇒ true),
|
||||
Ignore(_ == akka.stream.scaladsl.Sink.getClass, _ == "apply", _ == 24, _ ⇒ true),
|
||||
Ignore(_ == akka.stream.scaladsl.BidiFlow.getClass, _ == "apply", _ == 24, _ ⇒ true),
|
||||
Ignore(_ == akka.stream.scaladsl.FlowGraph.getClass, _ == "closed", _ == 24, _ ⇒ true),
|
||||
Ignore(_ == akka.stream.scaladsl.FlowGraph.getClass, _ == "partial", _ == 24, _ ⇒ true),
|
||||
Ignore(_ == akka.stream.scaladsl.FlowGraph.getClass, _ == "runnable", _ == 24, _ ⇒ true),
|
||||
Ignore(_ == akka.stream.scaladsl.FlowGraph.getClass, _ == "create", _ == 24, _ ⇒ true),
|
||||
// all generated methods like scaladsl.Sink$.akka$stream$scaladsl$Sink$$newOnCompleteStage$1
|
||||
Ignore(_ ⇒ true, _.contains("$"), _ ⇒ true, _ ⇒ true))
|
||||
|
||||
|
|
@ -127,8 +127,8 @@ class DslFactoriesConsistencySpec extends WordSpec with Matchers {
|
|||
/**
|
||||
* Rename
|
||||
* createN => create
|
||||
* closedN => closed
|
||||
* partialN => partial
|
||||
* runnableN => runnable
|
||||
* createN => create
|
||||
*/
|
||||
private val unspecializeName: PartialFunction[Method, Method] = {
|
||||
case m ⇒ m.copy(name = m.name.filter(Character.isLetter))
|
||||
|
|
|
|||
|
|
@ -4,7 +4,7 @@
|
|||
package akka.stream.actor
|
||||
|
||||
import akka.actor.{ ActorRef, PoisonPill, Props }
|
||||
import akka.stream.{ ActorMaterializer, ActorMaterializerSettings, ActorAttributes }
|
||||
import akka.stream.{ ClosedShape, ActorMaterializer, ActorMaterializerSettings, ActorAttributes }
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.testkit._
|
||||
import akka.stream.testkit.Utils._
|
||||
|
|
@ -350,7 +350,7 @@ class ActorPublisherSpec extends AkkaSpec(ActorPublisherSpec.config) with Implic
|
|||
val sink1 = Sink(ActorSubscriber[String](system.actorOf(receiverProps(probe1.ref))))
|
||||
val sink2: Sink[String, ActorRef] = Sink.actorSubscriber(receiverProps(probe2.ref))
|
||||
|
||||
val senderRef2 = FlowGraph.closed(Source.actorPublisher[Int](senderProps)) { implicit b ⇒
|
||||
val senderRef2 = RunnableGraph.fromGraph(FlowGraph.create(Source.actorPublisher[Int](senderProps)) { implicit b ⇒
|
||||
source2 ⇒
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
|
|
@ -364,7 +364,8 @@ class ActorPublisherSpec extends AkkaSpec(ActorPublisherSpec.config) with Implic
|
|||
|
||||
bcast.out(0).map(_ + "mark") ~> sink1
|
||||
bcast.out(1) ~> sink2
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
(0 to 10).foreach {
|
||||
senderRef1 ! _
|
||||
|
|
|
|||
|
|
@ -72,7 +72,7 @@ class ActorGraphInterpreterSpec extends AkkaSpec {
|
|||
override def toString = "IdentityBidi"
|
||||
}
|
||||
|
||||
val identity = BidiFlow.wrap(identityBidi).join(Flow[Int].map { x ⇒ x })
|
||||
val identity = BidiFlow.fromGraph(identityBidi).join(Flow[Int].map { x ⇒ x })
|
||||
|
||||
Await.result(
|
||||
Source(1 to 10).via(identity).grouped(100).runWith(Sink.head),
|
||||
|
|
@ -117,7 +117,7 @@ class ActorGraphInterpreterSpec extends AkkaSpec {
|
|||
override def toString = "IdentityBidi"
|
||||
}
|
||||
|
||||
val identityBidiF = BidiFlow.wrap(identityBidi)
|
||||
val identityBidiF = BidiFlow.fromGraph(identityBidi)
|
||||
val identity = (identityBidiF atop identityBidiF atop identityBidiF).join(Flow[Int].map { x ⇒ x })
|
||||
|
||||
Await.result(
|
||||
|
|
@ -163,7 +163,7 @@ class ActorGraphInterpreterSpec extends AkkaSpec {
|
|||
override def toString = "IdentityBidi"
|
||||
}
|
||||
|
||||
val identityBidiF = BidiFlow.wrap(identityBidi)
|
||||
val identityBidiF = BidiFlow.fromGraph(identityBidi)
|
||||
val identity = (identityBidiF atop identityBidiF atop identityBidiF).join(Flow[Int].map { x ⇒ x })
|
||||
|
||||
Await.result(
|
||||
|
|
@ -214,7 +214,7 @@ class ActorGraphInterpreterSpec extends AkkaSpec {
|
|||
|
||||
val takeAll = Flow[Int].grouped(200).toMat(Sink.head)(Keep.right)
|
||||
|
||||
val (f1, f2) = FlowGraph.closed(takeAll, takeAll)(Keep.both) { implicit b ⇒
|
||||
val (f1, f2) = RunnableGraph.fromGraph(FlowGraph.create(takeAll, takeAll)(Keep.both) { implicit b ⇒
|
||||
(out1, out2) ⇒
|
||||
import FlowGraph.Implicits._
|
||||
val bidi = b.add(rotatedBidi)
|
||||
|
|
@ -224,7 +224,8 @@ class ActorGraphInterpreterSpec extends AkkaSpec {
|
|||
|
||||
bidi.in2 <~ Source(1 to 100)
|
||||
bidi.out1 ~> out1
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
Await.result(f1, 3.seconds) should ===(1 to 100)
|
||||
Await.result(f2, 3.seconds) should ===(1 to 10)
|
||||
|
|
|
|||
|
|
@ -216,7 +216,7 @@ class FramingSpec extends AkkaSpec {
|
|||
}
|
||||
|
||||
"support simple framing adapter" in {
|
||||
val rechunkBidi = BidiFlow.wrap(rechunk, rechunk)(Keep.left)
|
||||
val rechunkBidi = BidiFlow.fromFlowsMat(rechunk, rechunk)(Keep.left)
|
||||
val codecFlow =
|
||||
Framing.simpleFramingProtocol(1024)
|
||||
.atop(rechunkBidi)
|
||||
|
|
|
|||
|
|
@ -342,7 +342,7 @@ class TcpSpec extends AkkaSpec("akka.io.tcp.windows-connection-abort-workaround-
|
|||
"properly full-close if requested" in assertAllStagesStopped {
|
||||
val serverAddress = temporaryServerAddress()
|
||||
val writeButIgnoreRead: Flow[ByteString, ByteString, Unit] =
|
||||
Flow.wrap(Sink.ignore, Source.single(ByteString("Early response")))(Keep.right)
|
||||
Flow.fromSinkAndSourceMat(Sink.ignore, Source.single(ByteString("Early response")))(Keep.right)
|
||||
|
||||
val binding =
|
||||
Await.result(
|
||||
|
|
@ -350,7 +350,7 @@ class TcpSpec extends AkkaSpec("akka.io.tcp.windows-connection-abort-workaround-
|
|||
conn.flow.join(writeButIgnoreRead).run()
|
||||
})(Keep.left).run(), 3.seconds)
|
||||
|
||||
val result = Source.lazyEmpty[ByteString]
|
||||
val result = Source.maybe[ByteString]
|
||||
.via(Tcp().outgoingConnection(serverAddress.getHostName, serverAddress.getPort))
|
||||
.runFold(ByteString.empty)(_ ++ _)
|
||||
|
||||
|
|
@ -385,7 +385,7 @@ class TcpSpec extends AkkaSpec("akka.io.tcp.windows-connection-abort-workaround-
|
|||
val serverAddress = temporaryServerAddress()
|
||||
val binding = Tcp(system2).bindAndHandle(Flow[ByteString], serverAddress.getHostName, serverAddress.getPort)(mat2)
|
||||
|
||||
val result = Source.lazyEmpty[ByteString].via(Tcp(system2).outgoingConnection(serverAddress)).runFold(0)(_ + _.size)(mat2)
|
||||
val result = Source.maybe[ByteString].via(Tcp(system2).outgoingConnection(serverAddress)).runFold(0)(_ + _.size)(mat2)
|
||||
|
||||
// Getting rid of existing connection actors by using a blunt instrument
|
||||
system2.actorSelection(akka.io.Tcp(system2).getManager.path / "selectors" / "$a" / "*") ! Kill
|
||||
|
|
|
|||
|
|
@ -6,7 +6,7 @@ package akka.stream.io
|
|||
|
||||
import java.util.concurrent.TimeoutException
|
||||
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.stream.{ ClosedShape, ActorMaterializer }
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.testkit.{ AkkaSpec, TestSubscriber, TestPublisher }
|
||||
import scala.concurrent.{ Future, Await }
|
||||
|
|
@ -37,7 +37,7 @@ class TimeoutsSpec extends AkkaSpec {
|
|||
|
||||
"fail if no initial element passes until timeout" in assertAllStagesStopped {
|
||||
val downstreamProbe = TestSubscriber.probe[Int]()
|
||||
Source.lazyEmpty[Int]
|
||||
Source.maybe[Int]
|
||||
.via(Timeouts.initalTimeout(1.seconds))
|
||||
.runWith(Sink(downstreamProbe))
|
||||
|
||||
|
|
@ -142,8 +142,8 @@ class TimeoutsSpec extends AkkaSpec {
|
|||
val upstreamWriter = TestPublisher.probe[Int]()
|
||||
val downstreamWriter = TestPublisher.probe[String]()
|
||||
|
||||
val upstream = Flow.wrap(Sink.ignore, Source(upstreamWriter))(Keep.left)
|
||||
val downstream = Flow.wrap(Sink.ignore, Source(downstreamWriter))(Keep.left)
|
||||
val upstream = Flow.fromSinkAndSourceMat(Sink.ignore, Source(upstreamWriter))(Keep.left)
|
||||
val downstream = Flow.fromSinkAndSourceMat(Sink.ignore, Source(downstreamWriter))(Keep.left)
|
||||
|
||||
val assembly: RunnableGraph[(Future[Unit], Future[Unit])] = upstream
|
||||
.joinMat(Timeouts.idleTimeoutBidi[Int, String](2.seconds))(Keep.left)
|
||||
|
|
@ -172,14 +172,15 @@ class TimeoutsSpec extends AkkaSpec {
|
|||
val downWrite = TestPublisher.probe[Int]()
|
||||
val downRead = TestSubscriber.probe[String]()
|
||||
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
import FlowGraph.Implicits._
|
||||
val timeoutStage = b.add(Timeouts.idleTimeoutBidi[String, Int](2.seconds))
|
||||
Source(upWrite) ~> timeoutStage.in1;
|
||||
timeoutStage.out1 ~> Sink(downRead)
|
||||
Sink(upRead) <~ timeoutStage.out2;
|
||||
timeoutStage.in2 <~ Source(downWrite)
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
// Request enough for the whole test
|
||||
upRead.request(100)
|
||||
|
|
@ -219,14 +220,15 @@ class TimeoutsSpec extends AkkaSpec {
|
|||
val downWrite = TestPublisher.probe[Int]()
|
||||
val downRead = TestSubscriber.probe[String]()
|
||||
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
import FlowGraph.Implicits._
|
||||
val timeoutStage = b.add(Timeouts.idleTimeoutBidi[String, Int](2.seconds))
|
||||
Source(upWrite) ~> timeoutStage.in1;
|
||||
timeoutStage.out1 ~> Sink(downRead)
|
||||
Sink(upRead) <~ timeoutStage.out2;
|
||||
timeoutStage.in2 <~ Source(downWrite)
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
val te = TE("test")
|
||||
|
||||
|
|
|
|||
|
|
@ -12,7 +12,7 @@ import scala.util.Random
|
|||
|
||||
import akka.actor.ActorSystem
|
||||
import akka.pattern.{ after ⇒ later }
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.stream.{ ClosedShape, ActorMaterializer }
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.stage._
|
||||
import akka.stream.testkit._
|
||||
|
|
@ -385,7 +385,7 @@ class TlsSpec extends AkkaSpec("akka.loglevel=INFO\nakka.actor.debug.receive=off
|
|||
.recover { case e: SSLException ⇒ Right(e) }
|
||||
.collect { case Right(e) ⇒ e }.toMat(Sink.head)(Keep.right)
|
||||
|
||||
val simple = Flow.wrap(getError, Source.lazyEmpty[SslTlsOutbound])(Keep.left)
|
||||
val simple = Flow.fromSinkAndSourceMat(getError, Source.maybe[SslTlsOutbound])(Keep.left)
|
||||
|
||||
// The creation of actual TCP connections is necessary. It is the easiest way to decouple the client and server
|
||||
// under error conditions, and has the bonus of matching most actual SSL deployments.
|
||||
|
|
@ -406,12 +406,13 @@ class TlsSpec extends AkkaSpec("akka.loglevel=INFO\nakka.actor.debug.receive=off
|
|||
"reliably cancel subscriptions when TransportIn fails early" in assertAllStagesStopped {
|
||||
val ex = new Exception("hello")
|
||||
val (sub, out1, out2) =
|
||||
FlowGraph.closed(Source.subscriber[SslTlsOutbound], Sink.head[ByteString], Sink.head[SslTlsInbound])((_, _, _)) { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create(Source.subscriber[SslTlsOutbound], Sink.head[ByteString], Sink.head[SslTlsInbound])((_, _, _)) { implicit b ⇒
|
||||
(s, o1, o2) ⇒
|
||||
val tls = b.add(clientTls(EagerClose))
|
||||
s ~> tls.in1; tls.out1 ~> o1
|
||||
o2 <~ tls.out2; tls.in2 <~ Source.failed(ex)
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
the[Exception] thrownBy Await.result(out1, 1.second) should be(ex)
|
||||
the[Exception] thrownBy Await.result(out2, 1.second) should be(ex)
|
||||
Thread.sleep(500)
|
||||
|
|
@ -423,12 +424,13 @@ class TlsSpec extends AkkaSpec("akka.loglevel=INFO\nakka.actor.debug.receive=off
|
|||
"reliably cancel subscriptions when UserIn fails early" in assertAllStagesStopped {
|
||||
val ex = new Exception("hello")
|
||||
val (sub, out1, out2) =
|
||||
FlowGraph.closed(Source.subscriber[ByteString], Sink.head[ByteString], Sink.head[SslTlsInbound])((_, _, _)) { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create(Source.subscriber[ByteString], Sink.head[ByteString], Sink.head[SslTlsInbound])((_, _, _)) { implicit b ⇒
|
||||
(s, o1, o2) ⇒
|
||||
val tls = b.add(clientTls(EagerClose))
|
||||
Source.failed[SslTlsOutbound](ex) ~> tls.in1; tls.out1 ~> o1
|
||||
o2 <~ tls.out2; tls.in2 <~ s
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
the[Exception] thrownBy Await.result(out1, 1.second) should be(ex)
|
||||
the[Exception] thrownBy Await.result(out2, 1.second) should be(ex)
|
||||
Thread.sleep(500)
|
||||
|
|
|
|||
|
|
@ -7,12 +7,10 @@ import akka.stream.testkit.AkkaSpec
|
|||
import akka.stream.testkit.Utils._
|
||||
import org.scalactic.ConversionCheckedTripleEquals
|
||||
import akka.util.ByteString
|
||||
import akka.stream.BidiShape
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.stream._
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
import scala.collection.immutable
|
||||
import akka.stream.Attributes
|
||||
|
||||
class BidiFlowSpec extends AkkaSpec with ConversionCheckedTripleEquals {
|
||||
import Attributes._
|
||||
|
|
@ -20,26 +18,22 @@ class BidiFlowSpec extends AkkaSpec with ConversionCheckedTripleEquals {
|
|||
|
||||
implicit val mat = ActorMaterializer()
|
||||
|
||||
val bidi = BidiFlow() { b ⇒
|
||||
val top = b.add(Flow[Int].map(x ⇒ x.toLong + 2).withAttributes(name("top")))
|
||||
val bottom = b.add(Flow[ByteString].map(_.decodeString("UTF-8")).withAttributes(name("bottom")))
|
||||
BidiShape(top.inlet, top.outlet, bottom.inlet, bottom.outlet)
|
||||
}
|
||||
val bidi = BidiFlow.fromFlows(
|
||||
Flow[Int].map(x ⇒ x.toLong + 2).withAttributes(name("top")),
|
||||
Flow[ByteString].map(_.decodeString("UTF-8")).withAttributes(name("bottom")))
|
||||
|
||||
val inverse = BidiFlow() { b ⇒
|
||||
val top = b.add(Flow[Long].map(x ⇒ x.toInt + 2).withAttributes(name("top")))
|
||||
val bottom = b.add(Flow[String].map(ByteString(_)).withAttributes(name("bottom")))
|
||||
BidiShape(top.inlet, top.outlet, bottom.inlet, bottom.outlet)
|
||||
}
|
||||
val inverse = BidiFlow.fromFlows(
|
||||
Flow[Long].map(x ⇒ x.toInt + 2).withAttributes(name("top")),
|
||||
Flow[String].map(ByteString(_)).withAttributes(name("bottom")))
|
||||
|
||||
val bidiMat = BidiFlow(Sink.head[Int]) { implicit b ⇒
|
||||
val bidiMat = BidiFlow.fromGraph(FlowGraph.create(Sink.head[Int]) { implicit b ⇒
|
||||
s ⇒
|
||||
Source.single(42) ~> s
|
||||
|
||||
val top = b.add(Flow[Int].map(x ⇒ x.toLong + 2))
|
||||
val bottom = b.add(Flow[ByteString].map(_.decodeString("UTF-8")))
|
||||
BidiShape(top.inlet, top.outlet, bottom.inlet, bottom.outlet)
|
||||
}
|
||||
})
|
||||
|
||||
val str = "Hello World"
|
||||
val bytes = ByteString(str)
|
||||
|
|
@ -47,13 +41,14 @@ class BidiFlowSpec extends AkkaSpec with ConversionCheckedTripleEquals {
|
|||
"A BidiFlow" must {
|
||||
|
||||
"work top/bottom in isolation" in {
|
||||
val (top, bottom) = FlowGraph.closed(Sink.head[Long], Sink.head[String])(Keep.both) { implicit b ⇒
|
||||
val (top, bottom) = RunnableGraph.fromGraph(FlowGraph.create(Sink.head[Long], Sink.head[String])(Keep.both) { implicit b ⇒
|
||||
(st, sb) ⇒
|
||||
val s = b.add(bidi)
|
||||
|
||||
Source.single(1) ~> s.in1; s.out1 ~> st
|
||||
sb <~ s.out2; s.in2 <~ Source.single(bytes)
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
Await.result(top, 1.second) should ===(3)
|
||||
Await.result(bottom, 1.second) should ===(str)
|
||||
|
|
@ -85,15 +80,16 @@ class BidiFlowSpec extends AkkaSpec with ConversionCheckedTripleEquals {
|
|||
}
|
||||
|
||||
"materialize to its value" in {
|
||||
val f = FlowGraph.closed(bidiMat) { implicit b ⇒
|
||||
val f = RunnableGraph.fromGraph(FlowGraph.create(bidiMat) { implicit b ⇒
|
||||
bidi ⇒
|
||||
Flow[String].map(Integer.valueOf(_).toInt) <~> bidi <~> Flow[Long].map(x ⇒ ByteString(s"Hello $x"))
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
Await.result(f, 1.second) should ===(42)
|
||||
}
|
||||
|
||||
"combine materialization values" in assertAllStagesStopped {
|
||||
val left = Flow(Sink.head[Int]) { implicit b ⇒
|
||||
val left = Flow.fromGraph(FlowGraph.create(Sink.head[Int]) { implicit b ⇒
|
||||
sink ⇒
|
||||
val bcast = b.add(Broadcast[Int](2))
|
||||
val merge = b.add(Merge[Int](2))
|
||||
|
|
@ -101,14 +97,14 @@ class BidiFlowSpec extends AkkaSpec with ConversionCheckedTripleEquals {
|
|||
bcast ~> sink
|
||||
Source.single(1) ~> bcast ~> merge
|
||||
flow ~> merge
|
||||
(flow.inlet, merge.out)
|
||||
}
|
||||
val right = Flow(Sink.head[immutable.Seq[Long]]) { implicit b ⇒
|
||||
FlowShape(flow.inlet, merge.out)
|
||||
})
|
||||
val right = Flow.fromGraph(FlowGraph.create(Sink.head[immutable.Seq[Long]]) { implicit b ⇒
|
||||
sink ⇒
|
||||
val flow = b.add(Flow[Long].grouped(10))
|
||||
flow ~> sink
|
||||
(flow.inlet, b.add(Source.single(ByteString("10"))))
|
||||
}
|
||||
FlowShape(flow.inlet, b.add(Source.single(ByteString("10"))))
|
||||
})
|
||||
val ((l, m), r) = left.joinMat(bidiMat)(Keep.both).joinMat(right)(Keep.both).run()
|
||||
Await.result(l, 1.second) should ===(1)
|
||||
Await.result(m, 1.second) should ===(42)
|
||||
|
|
|
|||
|
|
@ -4,8 +4,7 @@
|
|||
package akka.stream.scaladsl
|
||||
|
||||
import akka.stream.Attributes._
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.stream.OverflowStrategy
|
||||
import akka.stream.{ ClosedShape, ActorMaterializer, OverflowStrategy }
|
||||
import akka.stream.testkit._
|
||||
import akka.stream.stage._
|
||||
|
||||
|
|
@ -41,47 +40,52 @@ class FlowGraphCompileSpec extends AkkaSpec {
|
|||
val out2 = Sink.head[String]
|
||||
|
||||
"A Graph" should {
|
||||
import FlowGraph.Implicits._
|
||||
"build simple merge" in {
|
||||
FlowGraph.closed() { b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val merge = b.add(Merge[String](2))
|
||||
b.addEdge(b.add(in1), f1, merge.in(0))
|
||||
b.addEdge(b.add(in2), f2, merge.in(1))
|
||||
b.addEdge(merge.out, f3, b.add(out1))
|
||||
}.run()
|
||||
in1 ~> f1 ~> merge.in(0)
|
||||
in2 ~> f2 ~> merge.in(1)
|
||||
merge.out ~> f3 ~> out1
|
||||
ClosedShape
|
||||
}).run()
|
||||
}
|
||||
|
||||
"build simple broadcast" in {
|
||||
FlowGraph.closed() { b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val bcast = b.add(Broadcast[String](2))
|
||||
b.addEdge(b.add(in1), f1, bcast.in)
|
||||
b.addEdge(bcast.out(0), f2, b.add(out1))
|
||||
b.addEdge(bcast.out(1), f3, b.add(out2))
|
||||
}.run()
|
||||
in1 ~> f1 ~> bcast.in
|
||||
bcast.out(0) ~> f2 ~> out1
|
||||
bcast.out(1) ~> f3 ~> out2
|
||||
ClosedShape
|
||||
}).run()
|
||||
}
|
||||
|
||||
"build simple balance" in {
|
||||
FlowGraph.closed() { b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val balance = b.add(Balance[String](2))
|
||||
b.addEdge(b.add(in1), f1, balance.in)
|
||||
b.addEdge(balance.out(0), f2, b.add(out1))
|
||||
b.addEdge(balance.out(1), f3, b.add(out2))
|
||||
}
|
||||
in1 ~> f1 ~> balance.in
|
||||
balance.out(0) ~> f2 ~> out1
|
||||
balance.out(1) ~> f3 ~> out2
|
||||
ClosedShape
|
||||
})
|
||||
}
|
||||
|
||||
"build simple merge - broadcast" in {
|
||||
FlowGraph.closed() { b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val merge = b.add(Merge[String](2))
|
||||
val bcast = b.add(Broadcast[String](2))
|
||||
b.addEdge(b.add(in1), f1, merge.in(0))
|
||||
b.addEdge(b.add(in2), f2, merge.in(1))
|
||||
b.addEdge(merge.out, f3, bcast.in)
|
||||
b.addEdge(bcast.out(0), f4, b.add(out1))
|
||||
b.addEdge(bcast.out(1), f5, b.add(out2))
|
||||
}.run()
|
||||
in1 ~> f1 ~> merge.in(0)
|
||||
in2 ~> f2 ~> merge.in(1)
|
||||
merge ~> f3 ~> bcast
|
||||
bcast.out(0) ~> f4 ~> out1
|
||||
bcast.out(1) ~> f5 ~> out2
|
||||
ClosedShape
|
||||
}).run()
|
||||
}
|
||||
|
||||
"build simple merge - broadcast with implicits" in {
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
import FlowGraph.Implicits._
|
||||
val merge = b.add(Merge[String](2))
|
||||
val bcast = b.add(Broadcast[String](2))
|
||||
|
|
@ -90,7 +94,8 @@ class FlowGraphCompileSpec extends AkkaSpec {
|
|||
bcast.out(0) ~> f3 ~> b.add(out1)
|
||||
b.add(in2) ~> f4 ~> merge.in(1)
|
||||
bcast.out(1) ~> f5 ~> b.add(out2)
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
}
|
||||
|
||||
/*
|
||||
|
|
@ -105,24 +110,25 @@ class FlowGraphCompileSpec extends AkkaSpec {
|
|||
"detect cycle in " in {
|
||||
pending // FIXME needs cycle detection capability
|
||||
intercept[IllegalArgumentException] {
|
||||
FlowGraph.closed() { b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val merge = b.add(Merge[String](2))
|
||||
val bcast1 = b.add(Broadcast[String](2))
|
||||
val bcast2 = b.add(Broadcast[String](2))
|
||||
val feedbackLoopBuffer = Flow[String].buffer(10, OverflowStrategy.dropBuffer)
|
||||
b.addEdge(b.add(in1), f1, merge.in(0))
|
||||
b.addEdge(merge.out, f2, bcast1.in)
|
||||
b.addEdge(bcast1.out(0), f3, b.add(out1))
|
||||
b.addEdge(bcast1.out(1), feedbackLoopBuffer, bcast2.in)
|
||||
b.addEdge(bcast2.out(0), f5, merge.in(1)) // cycle
|
||||
b.addEdge(bcast2.out(1), f6, b.add(out2))
|
||||
}
|
||||
in1 ~> f1 ~> merge.in(0)
|
||||
merge ~> f2 ~> bcast1
|
||||
bcast1.out(0) ~> f3 ~> out1
|
||||
bcast1.out(1) ~> feedbackLoopBuffer ~> bcast2
|
||||
bcast2.out(0) ~> f5 ~> merge.in(1) // cycle
|
||||
bcast2.out(1) ~> f6 ~> out2
|
||||
ClosedShape
|
||||
})
|
||||
}.getMessage.toLowerCase should include("cycle")
|
||||
|
||||
}
|
||||
|
||||
"express complex topologies in a readable way" in {
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val merge = b.add(Merge[String](2))
|
||||
val bcast1 = b.add(Broadcast[String](2))
|
||||
val bcast2 = b.add(Broadcast[String](2))
|
||||
|
|
@ -131,22 +137,24 @@ class FlowGraphCompileSpec extends AkkaSpec {
|
|||
b.add(in1) ~> f1 ~> merge ~> f2 ~> bcast1 ~> f3 ~> b.add(out1)
|
||||
bcast1 ~> feedbackLoopBuffer ~> bcast2 ~> f5 ~> merge
|
||||
bcast2 ~> f6 ~> b.add(out2)
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
}
|
||||
|
||||
"build broadcast - merge" in {
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val bcast = b.add(Broadcast[String](2))
|
||||
val merge = b.add(Merge[String](2))
|
||||
import FlowGraph.Implicits._
|
||||
in1 ~> f1 ~> bcast ~> f2 ~> merge ~> f3 ~> out1
|
||||
bcast ~> f4 ~> merge
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
}
|
||||
|
||||
"build wikipedia Topological_sorting" in {
|
||||
// see https://en.wikipedia.org/wiki/Topological_sorting#mediaviewer/File:Directed_acyclic_graph.png
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val b3 = b.add(Broadcast[String](2))
|
||||
val b7 = b.add(Broadcast[String](2))
|
||||
val b11 = b.add(Broadcast[String](3))
|
||||
|
|
@ -170,22 +178,24 @@ class FlowGraphCompileSpec extends AkkaSpec {
|
|||
in5 ~> f("k") ~> m11
|
||||
in3 ~> f("l") ~> b3 ~> f("m") ~> m8
|
||||
b3 ~> f("n") ~> m10
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
}
|
||||
|
||||
"make it optional to specify flows" in {
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val merge = b.add(Merge[String](2))
|
||||
val bcast = b.add(Broadcast[String](2))
|
||||
import FlowGraph.Implicits._
|
||||
in1 ~> merge ~> bcast ~> out1
|
||||
in2 ~> merge
|
||||
bcast ~> out2
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
}
|
||||
|
||||
"build unzip - zip" in {
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val zip = b.add(Zip[Int, String]())
|
||||
val unzip = b.add(Unzip[Int, String]())
|
||||
val out = Sink.publisher[(Int, String)]
|
||||
|
|
@ -194,12 +204,13 @@ class FlowGraphCompileSpec extends AkkaSpec {
|
|||
unzip.out0 ~> Flow[Int].map(_ * 2) ~> zip.in0
|
||||
unzip.out1 ~> zip.in1
|
||||
zip.out ~> out
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
}
|
||||
|
||||
"distinguish between input and output ports" in {
|
||||
intercept[IllegalArgumentException] {
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val zip = b.add(Zip[Int, String]())
|
||||
val unzip = b.add(Unzip[Int, String]())
|
||||
val wrongOut = Sink.publisher[(Int, Int)]
|
||||
|
|
@ -210,22 +221,24 @@ class FlowGraphCompileSpec extends AkkaSpec {
|
|||
"zip.left ~> zip.right" shouldNot compile
|
||||
"Flow(List(1, 2, 3)) ~> zip.left ~> wrongOut" shouldNot compile
|
||||
"""Flow(List(1 -> "a", 2 -> "b", 3 -> "c")) ~> unzip.in ~> whatever""" shouldNot compile
|
||||
}
|
||||
}.getMessage should include("unconnected")
|
||||
ClosedShape
|
||||
})
|
||||
}.getMessage should include("must correspond to")
|
||||
}
|
||||
|
||||
"build with variance" in {
|
||||
val out = Sink(TestSubscriber.manualProbe[Fruit]())
|
||||
FlowGraph.closed() { b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
import FlowGraph.Implicits._
|
||||
val merge = b.add(Merge[Fruit](2))
|
||||
b.addEdge(b add Source[Fruit](apples), Flow[Fruit], merge.in(0))
|
||||
b.addEdge(b add Source[Apple](apples), Flow[Apple], merge.in(1))
|
||||
b.addEdge(merge.out, Flow[Fruit].map(identity), b add out)
|
||||
}
|
||||
Source[Fruit](apples) ~> Flow[Fruit] ~> merge.in(0)
|
||||
Source[Apple](apples) ~> Flow[Apple] ~> merge.in(1)
|
||||
merge.out ~> Flow[Fruit].map(identity) ~> Sink(TestSubscriber.manualProbe[Fruit]())
|
||||
ClosedShape
|
||||
})
|
||||
}
|
||||
|
||||
"build with variance when indices are not specified" in {
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
import FlowGraph.Implicits._
|
||||
val fruitMerge = b.add(Merge[Fruit](2))
|
||||
Source[Fruit](apples) ~> fruitMerge
|
||||
|
|
@ -253,11 +266,12 @@ class FlowGraphCompileSpec extends AkkaSpec {
|
|||
Source[Apple](apples) ~> appleBcast
|
||||
appleBcast ~> Sink.head[Fruit]
|
||||
appleBcast ~> Sink.head[Apple]
|
||||
}
|
||||
ClosedShape
|
||||
})
|
||||
}
|
||||
|
||||
"build with implicits and variance" in {
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
def appleSource = b.add(Source(TestPublisher.manualProbe[Apple]()))
|
||||
def fruitSource = b.add(Source(TestPublisher.manualProbe[Fruit]()))
|
||||
val outA = b add Sink(TestSubscriber.manualProbe[Fruit]())
|
||||
|
|
@ -289,40 +303,40 @@ class FlowGraphCompileSpec extends AkkaSpec {
|
|||
"merge.out ~> b.add(Broadcast[Apple](2))" shouldNot compile
|
||||
"merge.out ~> Flow[Fruit].map(identity) ~> b.add(Broadcast[Apple](2))" shouldNot compile
|
||||
"fruitSource ~> merge ~> b.add(Broadcast[Apple](2))" shouldNot compile
|
||||
}
|
||||
ClosedShape
|
||||
})
|
||||
}
|
||||
|
||||
"build with plain flow without junctions" in {
|
||||
FlowGraph.closed() { b ⇒
|
||||
b.addEdge(b.add(in1), f1, b.add(out1))
|
||||
}.run()
|
||||
FlowGraph.closed() { b ⇒
|
||||
b.addEdge(b.add(in1), f1, b.add(f2.to(out1)))
|
||||
}.run()
|
||||
FlowGraph.closed() { b ⇒
|
||||
b.addEdge(b.add(in1 via f1), f2, b.add(out1))
|
||||
}.run()
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
import FlowGraph.Implicits._
|
||||
b.add(in1) ~> f1 ~> b.add(out1)
|
||||
}.run()
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
import FlowGraph.Implicits._
|
||||
b.add(in1) ~> b.add(out1)
|
||||
}.run()
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
import FlowGraph.Implicits._
|
||||
b.add(in1) ~> b.add(f1 to out1)
|
||||
}.run()
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
import FlowGraph.Implicits._
|
||||
b.add(in1 via f1) ~> b.add(out1)
|
||||
}.run()
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
import FlowGraph.Implicits._
|
||||
b.add(in1 via f1) ~> b.add(f2 to out1)
|
||||
}.run()
|
||||
}
|
||||
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
in1 ~> f1 ~> out1
|
||||
ClosedShape
|
||||
}).run()
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
in1 ~> f1 ~> f2.to(out1)
|
||||
ClosedShape
|
||||
}).run()
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
(in1 via f1) ~> f2 ~> out1
|
||||
ClosedShape
|
||||
}).run()
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
in1 ~> out1
|
||||
ClosedShape
|
||||
}).run()
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
in1 ~> (f1 to out1)
|
||||
ClosedShape
|
||||
}).run()
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
(in1 via f1) ~> out1
|
||||
ClosedShape
|
||||
}).run()
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
(in1 via f1) ~> (f2 to out1)
|
||||
ClosedShape
|
||||
}).run()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -3,7 +3,7 @@
|
|||
*/
|
||||
package akka.stream.scaladsl
|
||||
|
||||
import akka.stream.{ ActorMaterializer, ActorMaterializerSettings }
|
||||
import akka.stream.{ FlowShape, ActorMaterializer, ActorMaterializerSettings }
|
||||
import akka.stream.testkit._
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import scala.concurrent.Await
|
||||
|
|
@ -24,7 +24,7 @@ class FlowJoinSpec extends AkkaSpec(ConfigFactory.parseString("akka.loglevel=INF
|
|||
val source = Source(0 to end)
|
||||
val probe = TestSubscriber.manualProbe[Seq[Int]]()
|
||||
|
||||
val flow1 = Flow() { implicit b ⇒
|
||||
val flow1 = Flow.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
import FlowGraph.Implicits._
|
||||
val merge = b.add(Merge[Int](2))
|
||||
val broadcast = b.add(Broadcast[Int](2))
|
||||
|
|
@ -32,8 +32,8 @@ class FlowJoinSpec extends AkkaSpec(ConfigFactory.parseString("akka.loglevel=INF
|
|||
merge.out ~> broadcast.in
|
||||
broadcast.out(0).grouped(1000) ~> Sink(probe)
|
||||
|
||||
(merge.in(1), broadcast.out(1))
|
||||
}
|
||||
FlowShape(merge.in(1), broadcast.out(1))
|
||||
})
|
||||
|
||||
val flow2 = Flow[Int].filter(_ % 2 == 1).map(_ * 10).take((end + 1) / 2)
|
||||
|
||||
|
|
|
|||
|
|
@ -41,7 +41,7 @@ class FlowScanSpec extends AkkaSpec {
|
|||
}
|
||||
|
||||
"emit values promptly" in {
|
||||
val f = Source.single(1).concat(Source.lazyEmpty).scan(0)(_ + _).grouped(2).runWith(Sink.head)
|
||||
val f = Source.single(1).concat(Source.maybe[Int]).scan(0)(_ + _).grouped(2).runWith(Sink.head)
|
||||
Await.result(f, 1.second) should ===(Seq(0, 1))
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -5,7 +5,7 @@ package akka.stream.scaladsl
|
|||
|
||||
import akka.stream.Attributes._
|
||||
import akka.stream.ActorAttributes._
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.stream.{ FlowShape, ActorMaterializer }
|
||||
import akka.stream.testkit.AkkaSpec
|
||||
import akka.actor.ActorRef
|
||||
import akka.testkit.TestProbe
|
||||
|
|
@ -33,13 +33,13 @@ class FlowSectionSpec extends AkkaSpec(FlowSectionSpec.config) {
|
|||
}
|
||||
|
||||
"have a nested flow with a different dispatcher" in {
|
||||
val flow = Flow() { implicit b ⇒
|
||||
val flow = Flow.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
import FlowGraph.Implicits._
|
||||
val bcast1 = b.add(Broadcast[Int](1))
|
||||
val bcast2 = b.add(Broadcast[Int](1))
|
||||
bcast1 ~> Flow[Int].map(sendThreadNameTo(testActor)) ~> bcast2.in
|
||||
(bcast1.in, bcast2.out(0))
|
||||
}.withAttributes(dispatcher("my-dispatcher1"))
|
||||
FlowShape(bcast1.in, bcast2.out(0))
|
||||
}).withAttributes(dispatcher("my-dispatcher1"))
|
||||
|
||||
Source.single(1).via(flow).to(Sink.ignore).run()
|
||||
|
||||
|
|
@ -51,21 +51,21 @@ class FlowSectionSpec extends AkkaSpec(FlowSectionSpec.config) {
|
|||
val probe1 = TestProbe()
|
||||
val probe2 = TestProbe()
|
||||
|
||||
val flow1 = Flow() { implicit b ⇒
|
||||
val flow1 = Flow.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
import FlowGraph.Implicits._
|
||||
val bcast1 = b.add(Broadcast[Int](1))
|
||||
val bcast2 = b.add(Broadcast[Int](1))
|
||||
bcast1 ~> Flow[Int].map(sendThreadNameTo(probe1.ref)) ~> bcast2.in
|
||||
(bcast1.in, bcast2.out(0))
|
||||
}.withAttributes(dispatcher("my-dispatcher1"))
|
||||
FlowShape(bcast1.in, bcast2.out(0))
|
||||
}).withAttributes(dispatcher("my-dispatcher1"))
|
||||
|
||||
val flow2 = Flow() { implicit b ⇒
|
||||
val flow2 = Flow.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
import FlowGraph.Implicits._
|
||||
val bcast1 = b.add(Broadcast[Int](1))
|
||||
val bcast2 = b.add(Broadcast[Int](1))
|
||||
bcast1 ~> flow1.via(Flow[Int].map(sendThreadNameTo(probe2.ref))) ~> bcast2.in
|
||||
(bcast1.in, bcast2.out(0))
|
||||
}.withAttributes(dispatcher("my-dispatcher2"))
|
||||
FlowShape(bcast1.in, bcast2.out(0))
|
||||
}).withAttributes(dispatcher("my-dispatcher2"))
|
||||
|
||||
Source.single(1).via(flow2).to(Sink.ignore).run()
|
||||
|
||||
|
|
|
|||
|
|
@ -41,8 +41,8 @@ class FlowTakeSpec extends AkkaSpec with ScriptedTest {
|
|||
}
|
||||
|
||||
"complete eagerly when zero or less is taken independently of upstream completion" in {
|
||||
Await.result(Source.lazyEmpty.take(0).runWith(Sink.ignore), 3.second)
|
||||
Await.result(Source.lazyEmpty.take(-1).runWith(Sink.ignore), 3.second)
|
||||
Await.result(Source.maybe[Int].take(0).runWith(Sink.ignore), 3.second)
|
||||
Await.result(Source.maybe[Int].take(-1).runWith(Sink.ignore), 3.second)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -12,7 +12,7 @@ import org.reactivestreams.Subscriber
|
|||
object GraphFlowSpec {
|
||||
val source1 = Source(0 to 3)
|
||||
|
||||
val partialGraph = FlowGraph.partial() { implicit b ⇒
|
||||
val partialGraph = FlowGraph.create() { implicit b ⇒
|
||||
import FlowGraph.Implicits._
|
||||
val source2 = Source(4 to 9)
|
||||
val source3 = Source.empty[Int]
|
||||
|
|
@ -62,12 +62,11 @@ class GraphFlowSpec extends AkkaSpec {
|
|||
"work with a Source and Sink" in {
|
||||
val probe = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
val flow = Flow(partialGraph) { implicit b ⇒
|
||||
val flow = Flow.fromGraph(FlowGraph.create(partialGraph) { implicit b ⇒
|
||||
partial ⇒
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
(partial.inlet, partial.outlet.map(_.toInt).outlet)
|
||||
}
|
||||
FlowShape(partial.inlet, partial.outlet.map(_.toInt).outlet)
|
||||
})
|
||||
|
||||
source1.via(flow).to(Sink(probe)).run()
|
||||
|
||||
|
|
@ -77,10 +76,9 @@ class GraphFlowSpec extends AkkaSpec {
|
|||
"be transformable with a Pipe" in {
|
||||
val probe = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
val flow = Flow(partialGraph) { implicit b ⇒
|
||||
partial ⇒
|
||||
(partial.inlet, partial.outlet)
|
||||
}
|
||||
val flow = Flow.fromGraph(FlowGraph.create(partialGraph) { implicit b ⇒
|
||||
partial ⇒ FlowShape(partial.inlet, partial.outlet)
|
||||
})
|
||||
|
||||
source1.via(flow).map(_.toInt).to(Sink(probe)).run()
|
||||
|
||||
|
|
@ -90,15 +88,15 @@ class GraphFlowSpec extends AkkaSpec {
|
|||
"work with another GraphFlow" in {
|
||||
val probe = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
val flow1 = Flow(partialGraph) { implicit b ⇒
|
||||
val flow1 = Flow.fromGraph(FlowGraph.create(partialGraph) { implicit b ⇒
|
||||
partial ⇒
|
||||
(partial.inlet, partial.outlet)
|
||||
}
|
||||
FlowShape(partial.inlet, partial.outlet)
|
||||
})
|
||||
|
||||
val flow2 = Flow(Flow[String].map(_.toInt)) { implicit b ⇒
|
||||
val flow2 = Flow.fromGraph(FlowGraph.create(Flow[String].map(_.toInt)) { implicit b ⇒
|
||||
importFlow ⇒
|
||||
(importFlow.inlet, importFlow.outlet)
|
||||
}
|
||||
FlowShape(importFlow.inlet, importFlow.outlet)
|
||||
})
|
||||
|
||||
source1.via(flow1).via(flow2).to(Sink(probe)).run()
|
||||
|
||||
|
|
@ -108,15 +106,15 @@ class GraphFlowSpec extends AkkaSpec {
|
|||
"be reusable multiple times" in {
|
||||
val probe = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
val flow = Flow(Flow[Int].map(_ * 2)) { implicit b ⇒
|
||||
importFlow ⇒
|
||||
(importFlow.inlet, importFlow.outlet)
|
||||
}
|
||||
val flow = Flow.fromGraph(FlowGraph.create(Flow[Int].map(_ * 2)) { implicit b ⇒
|
||||
importFlow ⇒ FlowShape(importFlow.inlet, importFlow.outlet)
|
||||
})
|
||||
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
import FlowGraph.Implicits._
|
||||
Source(1 to 5) ~> flow ~> flow ~> Sink(probe)
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
validateProbe(probe, 5, Set(4, 8, 12, 16, 20))
|
||||
}
|
||||
|
|
@ -126,12 +124,12 @@ class GraphFlowSpec extends AkkaSpec {
|
|||
"work with a Sink" in {
|
||||
val probe = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
val source = Source(partialGraph) { implicit b ⇒
|
||||
val source = Source.fromGraph(FlowGraph.create(partialGraph) { implicit b ⇒
|
||||
partial ⇒
|
||||
import FlowGraph.Implicits._
|
||||
source1 ~> partial.inlet
|
||||
partial.outlet.map(_.toInt).outlet
|
||||
}
|
||||
SourceShape(partial.outlet.map(_.toInt).outlet)
|
||||
})
|
||||
|
||||
source.to(Sink(probe)).run()
|
||||
|
||||
|
|
@ -141,10 +139,10 @@ class GraphFlowSpec extends AkkaSpec {
|
|||
"work with a Sink when having KeyedSource inside" in {
|
||||
val probe = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
val source = Source.apply(Source.subscriber[Int]) { implicit b ⇒
|
||||
val source = Source.fromGraph(FlowGraph.create(Source.subscriber[Int]) { implicit b ⇒
|
||||
subSource ⇒
|
||||
subSource.outlet
|
||||
}
|
||||
SourceShape(subSource.outlet)
|
||||
})
|
||||
|
||||
val mm: Subscriber[Int] = source.to(Sink(probe)).run()
|
||||
source1.to(Sink(mm)).run()
|
||||
|
|
@ -156,12 +154,12 @@ class GraphFlowSpec extends AkkaSpec {
|
|||
|
||||
val probe = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
val source = Source(partialGraph) { implicit b ⇒
|
||||
val source = Source.fromGraph(FlowGraph.create(partialGraph) { implicit b ⇒
|
||||
partial ⇒
|
||||
import FlowGraph.Implicits._
|
||||
source1 ~> partial.inlet
|
||||
partial.outlet
|
||||
}
|
||||
SourceShape(partial.outlet)
|
||||
})
|
||||
|
||||
source.map(_.toInt).to(Sink(probe)).run()
|
||||
|
||||
|
|
@ -171,17 +169,17 @@ class GraphFlowSpec extends AkkaSpec {
|
|||
"work with an GraphFlow" in {
|
||||
val probe = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
val source = Source(partialGraph) { implicit b ⇒
|
||||
val source = Source.fromGraph(FlowGraph.create(partialGraph) { implicit b ⇒
|
||||
partial ⇒
|
||||
import FlowGraph.Implicits._
|
||||
source1 ~> partial.inlet
|
||||
partial.outlet
|
||||
}
|
||||
SourceShape(partial.outlet)
|
||||
})
|
||||
|
||||
val flow = Flow(Flow[String].map(_.toInt)) { implicit b ⇒
|
||||
val flow = Flow.fromGraph(FlowGraph.create(Flow[String].map(_.toInt)) { implicit b ⇒
|
||||
importFlow ⇒
|
||||
(importFlow.inlet, importFlow.outlet)
|
||||
}
|
||||
FlowShape(importFlow.inlet, importFlow.outlet)
|
||||
})
|
||||
|
||||
source.via(flow).to(Sink(probe)).run()
|
||||
|
||||
|
|
@ -191,20 +189,21 @@ class GraphFlowSpec extends AkkaSpec {
|
|||
"be reusable multiple times" in {
|
||||
val probe = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
val source = Source(Source(1 to 5)) { implicit b ⇒
|
||||
val source = Source.fromGraph(FlowGraph.create(Source(1 to 5)) { implicit b ⇒
|
||||
s ⇒
|
||||
import FlowGraph.Implicits._
|
||||
s.outlet.map(_ * 2).outlet
|
||||
}
|
||||
SourceShape(s.outlet.map(_ * 2).outlet)
|
||||
})
|
||||
|
||||
FlowGraph.closed(source, source)(Keep.both) { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create(source, source)(Keep.both) { implicit b ⇒
|
||||
(s1, s2) ⇒
|
||||
import FlowGraph.Implicits._
|
||||
val merge = b.add(Merge[Int](2))
|
||||
s1.outlet ~> merge.in(0)
|
||||
merge.out ~> Sink(probe)
|
||||
s2.outlet.map(_ * 10) ~> merge.in(1)
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
validateProbe(probe, 10, Set(2, 4, 6, 8, 10, 20, 40, 60, 80, 100))
|
||||
}
|
||||
|
|
@ -214,12 +213,12 @@ class GraphFlowSpec extends AkkaSpec {
|
|||
"work with a Source" in {
|
||||
val probe = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
val sink = Sink(partialGraph) { implicit b ⇒
|
||||
val sink = Sink.fromGraph(FlowGraph.create(partialGraph) { implicit b ⇒
|
||||
partial ⇒
|
||||
import FlowGraph.Implicits._
|
||||
partial.outlet.map(_.toInt) ~> Sink(probe)
|
||||
partial.inlet
|
||||
}
|
||||
SinkShape(partial.inlet)
|
||||
})
|
||||
|
||||
source1.to(sink).run()
|
||||
|
||||
|
|
@ -230,10 +229,9 @@ class GraphFlowSpec extends AkkaSpec {
|
|||
val probe = TestSubscriber.manualProbe[Int]()
|
||||
val pubSink = Sink.publisher[Int]
|
||||
|
||||
val sink = Sink(pubSink) { implicit b ⇒
|
||||
p ⇒
|
||||
p.inlet
|
||||
}
|
||||
val sink = Sink.fromGraph(FlowGraph.create(pubSink) { implicit b ⇒
|
||||
p ⇒ SinkShape(p.inlet)
|
||||
})
|
||||
|
||||
val mm = source1.runWith(sink)
|
||||
Source(mm).to(Sink(probe)).run()
|
||||
|
|
@ -244,13 +242,13 @@ class GraphFlowSpec extends AkkaSpec {
|
|||
"be transformable with a Pipe" in {
|
||||
val probe = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
val sink = Sink(partialGraph, Flow[String].map(_.toInt))(Keep.both) { implicit b ⇒
|
||||
val sink = Sink.fromGraph(FlowGraph.create(partialGraph, Flow[String].map(_.toInt))(Keep.both) { implicit b ⇒
|
||||
(partial, flow) ⇒
|
||||
import FlowGraph.Implicits._
|
||||
flow.outlet ~> partial.inlet
|
||||
partial.outlet.map(_.toInt) ~> Sink(probe)
|
||||
flow.inlet
|
||||
}
|
||||
SinkShape(flow.inlet)
|
||||
})
|
||||
|
||||
val iSink = Flow[Int].map(_.toString).to(sink)
|
||||
source1.to(iSink).run()
|
||||
|
|
@ -262,17 +260,17 @@ class GraphFlowSpec extends AkkaSpec {
|
|||
|
||||
val probe = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
val flow = Flow(partialGraph) { implicit b ⇒
|
||||
val flow = Flow.fromGraph(FlowGraph.create(partialGraph) { implicit b ⇒
|
||||
partial ⇒
|
||||
(partial.inlet, partial.outlet)
|
||||
}
|
||||
FlowShape(partial.inlet, partial.outlet)
|
||||
})
|
||||
|
||||
val sink = Sink(Flow[String].map(_.toInt)) { implicit b ⇒
|
||||
val sink = Sink.fromGraph(FlowGraph.create(Flow[String].map(_.toInt)) { implicit b ⇒
|
||||
flow ⇒
|
||||
import FlowGraph.Implicits._
|
||||
flow.outlet ~> Sink(probe)
|
||||
flow.inlet
|
||||
}
|
||||
SinkShape(flow.inlet)
|
||||
})
|
||||
|
||||
source1.via(flow).to(sink).run()
|
||||
|
||||
|
|
@ -286,32 +284,33 @@ class GraphFlowSpec extends AkkaSpec {
|
|||
val inSource = Source.subscriber[Int]
|
||||
val outSink = Sink.publisher[Int]
|
||||
|
||||
val flow = Flow(partialGraph) { implicit b ⇒
|
||||
val flow = Flow.fromGraph(FlowGraph.create(partialGraph) { implicit b ⇒
|
||||
partial ⇒
|
||||
import FlowGraph.Implicits._
|
||||
(partial.inlet, partial.outlet.map(_.toInt).outlet)
|
||||
}
|
||||
FlowShape(partial.inlet, partial.outlet.map(_.toInt).outlet)
|
||||
})
|
||||
|
||||
val source = Source(Flow[Int].map(_.toString), inSource)(Keep.right) { implicit b ⇒
|
||||
val source = Source.fromGraph(FlowGraph.create(Flow[Int].map(_.toString), inSource)(Keep.right) { implicit b ⇒
|
||||
(flow, src) ⇒
|
||||
import FlowGraph.Implicits._
|
||||
src.outlet ~> flow.inlet
|
||||
flow.outlet
|
||||
}
|
||||
SourceShape(flow.outlet)
|
||||
})
|
||||
|
||||
val sink = Sink(Flow[String].map(_.toInt), outSink)(Keep.right) { implicit b ⇒
|
||||
val sink = Sink.fromGraph(FlowGraph.create(Flow[String].map(_.toInt), outSink)(Keep.right) { implicit b ⇒
|
||||
(flow, snk) ⇒
|
||||
import FlowGraph.Implicits._
|
||||
flow.outlet ~> snk.inlet
|
||||
flow.inlet
|
||||
}
|
||||
SinkShape(flow.inlet)
|
||||
})
|
||||
|
||||
val (m1, m2, m3) = FlowGraph.closed(source, flow, sink)(Tuple3.apply) { implicit b ⇒
|
||||
val (m1, m2, m3) = RunnableGraph.fromGraph(FlowGraph.create(source, flow, sink)(Tuple3.apply) { implicit b ⇒
|
||||
(src, f, snk) ⇒
|
||||
import FlowGraph.Implicits._
|
||||
src.outlet.map(_.toInt) ~> f.inlet
|
||||
f.outlet.map(_.toString) ~> snk.inlet
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
val subscriber = m1
|
||||
val publisher = m3
|
||||
|
|
@ -326,21 +325,22 @@ class GraphFlowSpec extends AkkaSpec {
|
|||
val inSource = Source.subscriber[Int]
|
||||
val outSink = Sink.publisher[Int]
|
||||
|
||||
val source = Source(inSource) { implicit b ⇒
|
||||
val source = Source.fromGraph(FlowGraph.create(inSource) { implicit b ⇒
|
||||
src ⇒
|
||||
src.outlet
|
||||
}
|
||||
SourceShape(src.outlet)
|
||||
})
|
||||
|
||||
val sink = Sink(outSink) { implicit b ⇒
|
||||
val sink = Sink.fromGraph(FlowGraph.create(outSink) { implicit b ⇒
|
||||
snk ⇒
|
||||
snk.inlet
|
||||
}
|
||||
SinkShape(snk.inlet)
|
||||
})
|
||||
|
||||
val (m1, m2) = FlowGraph.closed(source, sink)(Keep.both) { implicit b ⇒
|
||||
val (m1, m2) = RunnableGraph.fromGraph(FlowGraph.create(source, sink)(Keep.both) { implicit b ⇒
|
||||
(src, snk) ⇒
|
||||
import FlowGraph.Implicits._
|
||||
src.outlet ~> snk.inlet
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
val subscriber = m1
|
||||
val publisher = m2
|
||||
|
|
|
|||
|
|
@ -4,9 +4,7 @@ import scala.concurrent.Await
|
|||
import scala.concurrent.duration._
|
||||
import scala.concurrent.Future
|
||||
|
||||
import akka.stream.ActorMaterializer
|
||||
|
||||
import akka.stream.ActorMaterializerSettings
|
||||
import akka.stream.{ ClosedShape, ActorMaterializer, ActorMaterializerSettings }
|
||||
import akka.stream.testkit._
|
||||
import akka.stream.testkit.scaladsl._
|
||||
import akka.stream.testkit.Utils._
|
||||
|
|
@ -25,12 +23,13 @@ class GraphBalanceSpec extends AkkaSpec {
|
|||
val c1 = TestSubscriber.manualProbe[Int]()
|
||||
val c2 = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val balance = b.add(Balance[Int](2))
|
||||
Source(List(1, 2, 3)) ~> balance.in
|
||||
balance.out(0) ~> Sink(c1)
|
||||
balance.out(1) ~> Sink(c2)
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
val sub1 = c1.expectSubscription()
|
||||
val sub2 = c2.expectSubscription()
|
||||
|
|
@ -48,13 +47,14 @@ class GraphBalanceSpec extends AkkaSpec {
|
|||
|
||||
"support waiting for demand from all downstream subscriptions" in {
|
||||
val s1 = TestSubscriber.manualProbe[Int]()
|
||||
val p2 = FlowGraph.closed(Sink.publisher[Int]) { implicit b ⇒
|
||||
val p2 = RunnableGraph.fromGraph(FlowGraph.create(Sink.publisher[Int]) { implicit b ⇒
|
||||
p2Sink ⇒
|
||||
val balance = b.add(Balance[Int](2, waitForAllDownstreams = true))
|
||||
Source(List(1, 2, 3)) ~> balance.in
|
||||
balance.out(0) ~> Sink(s1)
|
||||
balance.out(1) ~> p2Sink
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
val sub1 = s1.expectSubscription()
|
||||
sub1.request(1)
|
||||
|
|
@ -78,14 +78,15 @@ class GraphBalanceSpec extends AkkaSpec {
|
|||
"support waiting for demand from all non-cancelled downstream subscriptions" in assertAllStagesStopped {
|
||||
val s1 = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
val (p2, p3) = FlowGraph.closed(Sink.publisher[Int], Sink.publisher[Int])(Keep.both) { implicit b ⇒
|
||||
val (p2, p3) = RunnableGraph.fromGraph(FlowGraph.create(Sink.publisher[Int], Sink.publisher[Int])(Keep.both) { implicit b ⇒
|
||||
(p2Sink, p3Sink) ⇒
|
||||
val balance = b.add(Balance[Int](3, waitForAllDownstreams = true))
|
||||
Source(List(1, 2, 3)) ~> balance.in
|
||||
balance.out(0) ~> Sink(s1)
|
||||
balance.out(1) ~> p2Sink
|
||||
balance.out(2) ~> p3Sink
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
val sub1 = s1.expectSubscription()
|
||||
sub1.request(1)
|
||||
|
|
@ -112,7 +113,7 @@ class GraphBalanceSpec extends AkkaSpec {
|
|||
"work with 5-way balance" in {
|
||||
|
||||
val sink = Sink.head[Seq[Int]]
|
||||
val (s1, s2, s3, s4, s5) = FlowGraph.closed(sink, sink, sink, sink, sink)(Tuple5.apply) {
|
||||
val (s1, s2, s3, s4, s5) = RunnableGraph.fromGraph(FlowGraph.create(sink, sink, sink, sink, sink)(Tuple5.apply) {
|
||||
implicit b ⇒
|
||||
(f1, f2, f3, f4, f5) ⇒
|
||||
val balance = b.add(Balance[Int](5, waitForAllDownstreams = true))
|
||||
|
|
@ -122,7 +123,8 @@ class GraphBalanceSpec extends AkkaSpec {
|
|||
balance.out(2).grouped(15) ~> f3
|
||||
balance.out(3).grouped(15) ~> f4
|
||||
balance.out(4).grouped(15) ~> f5
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
Set(s1, s2, s3, s4, s5) flatMap (Await.result(_, 3.seconds)) should be((0 to 14).toSet)
|
||||
}
|
||||
|
|
@ -131,14 +133,15 @@ class GraphBalanceSpec extends AkkaSpec {
|
|||
val numElementsForSink = 10000
|
||||
val outputs = Sink.fold[Int, Int](0)(_ + _)
|
||||
|
||||
val results = FlowGraph.closed(outputs, outputs, outputs)(List(_, _, _)) { implicit b ⇒
|
||||
val results = RunnableGraph.fromGraph(FlowGraph.create(outputs, outputs, outputs)(List(_, _, _)) { implicit b ⇒
|
||||
(o1, o2, o3) ⇒
|
||||
val balance = b.add(Balance[Int](3, waitForAllDownstreams = true))
|
||||
Source.repeat(1).take(numElementsForSink * 3) ~> balance.in
|
||||
balance.out(0) ~> o1
|
||||
balance.out(1) ~> o2
|
||||
balance.out(2) ~> o3
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
import system.dispatcher
|
||||
val sum = Future.sequence(results).map { res ⇒
|
||||
|
|
@ -150,14 +153,15 @@ class GraphBalanceSpec extends AkkaSpec {
|
|||
|
||||
"fairly balance between three outputs" in {
|
||||
val probe = TestSink.probe[Int]
|
||||
val (p1, p2, p3) = FlowGraph.closed(probe, probe, probe)(Tuple3.apply) { implicit b ⇒
|
||||
val (p1, p2, p3) = RunnableGraph.fromGraph(FlowGraph.create(probe, probe, probe)(Tuple3.apply) { implicit b ⇒
|
||||
(o1, o2, o3) ⇒
|
||||
val balance = b.add(Balance[Int](3))
|
||||
Source(1 to 7) ~> balance.in
|
||||
balance.out(0) ~> o1
|
||||
balance.out(1) ~> o2
|
||||
balance.out(2) ~> o3
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
p1.requestNext(1)
|
||||
p2.requestNext(2)
|
||||
|
|
@ -176,12 +180,13 @@ class GraphBalanceSpec extends AkkaSpec {
|
|||
val c1 = TestSubscriber.manualProbe[Int]()
|
||||
val c2 = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val balance = b.add(Balance[Int](2))
|
||||
Source(List(1, 2, 3)) ~> balance.in
|
||||
balance.out(0) ~> Sink(c1)
|
||||
balance.out(1) ~> Sink(c2)
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
val sub1 = c1.expectSubscription()
|
||||
sub1.cancel()
|
||||
|
|
@ -197,12 +202,13 @@ class GraphBalanceSpec extends AkkaSpec {
|
|||
val c1 = TestSubscriber.manualProbe[Int]()
|
||||
val c2 = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val balance = b.add(Balance[Int](2))
|
||||
Source(List(1, 2, 3)) ~> balance.in
|
||||
balance.out(0) ~> Sink(c1)
|
||||
balance.out(1) ~> Sink(c2)
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
val sub1 = c1.expectSubscription()
|
||||
val sub2 = c2.expectSubscription()
|
||||
|
|
@ -219,12 +225,13 @@ class GraphBalanceSpec extends AkkaSpec {
|
|||
val c1 = TestSubscriber.manualProbe[Int]()
|
||||
val c2 = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val balance = b.add(Balance[Int](2))
|
||||
Source(p1.getPublisher) ~> balance.in
|
||||
balance.out(0) ~> Sink(c1)
|
||||
balance.out(1) ~> Sink(c2)
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
val bsub = p1.expectSubscription()
|
||||
val sub1 = c1.expectSubscription()
|
||||
|
|
|
|||
|
|
@ -5,8 +5,7 @@ import akka.stream.testkit.scaladsl.TestSink
|
|||
import scala.concurrent.{ Future, Await }
|
||||
import scala.concurrent.duration._
|
||||
|
||||
import akka.stream.{ OverflowStrategy, ActorMaterializerSettings }
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.stream._
|
||||
import akka.stream.testkit._
|
||||
import akka.stream.testkit.Utils._
|
||||
|
||||
|
|
@ -24,12 +23,13 @@ class GraphBroadcastSpec extends AkkaSpec {
|
|||
val c1 = TestSubscriber.manualProbe[Int]()
|
||||
val c2 = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val bcast = b.add(Broadcast[Int](2))
|
||||
Source(List(1, 2, 3)) ~> bcast.in
|
||||
bcast.out(0) ~> Flow[Int].buffer(16, OverflowStrategy.backpressure) ~> Sink(c1)
|
||||
bcast.out(1) ~> Flow[Int].buffer(16, OverflowStrategy.backpressure) ~> Sink(c2)
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
val sub1 = c1.expectSubscription()
|
||||
val sub2 = c2.expectSubscription()
|
||||
|
|
@ -53,7 +53,7 @@ class GraphBroadcastSpec extends AkkaSpec {
|
|||
val headSink = Sink.head[Seq[Int]]
|
||||
|
||||
import system.dispatcher
|
||||
val result = FlowGraph.closed(
|
||||
val result = RunnableGraph.fromGraph(FlowGraph.create(
|
||||
headSink,
|
||||
headSink,
|
||||
headSink,
|
||||
|
|
@ -68,7 +68,8 @@ class GraphBroadcastSpec extends AkkaSpec {
|
|||
bcast.out(2).grouped(5) ~> p3.inlet
|
||||
bcast.out(3).grouped(5) ~> p4.inlet
|
||||
bcast.out(4).grouped(5) ~> p5.inlet
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
Await.result(result, 3.seconds) should be(List.fill(5)(List(1, 2, 3)))
|
||||
}
|
||||
|
|
@ -83,7 +84,7 @@ class GraphBroadcastSpec extends AkkaSpec {
|
|||
(f1, f2, f3, f4, f5, f6, f7, f8, f9, f10, f11, f12, f13, f14, f15, f16, f17, f18, f19, f20, f21, f22) ⇒
|
||||
Future.sequence(List(f1, f2, f3, f4, f5, f6, f7, f8, f9, f10, f11, f12, f13, f14, f15, f16, f17, f18, f19, f20, f21, f22))
|
||||
|
||||
val result = FlowGraph.closed(
|
||||
val result = RunnableGraph.fromGraph(FlowGraph.create(
|
||||
headSink, headSink, headSink, headSink, headSink,
|
||||
headSink, headSink, headSink, headSink, headSink,
|
||||
headSink, headSink, headSink, headSink, headSink,
|
||||
|
|
@ -115,7 +116,8 @@ class GraphBroadcastSpec extends AkkaSpec {
|
|||
bcast.out(19).grouped(5) ~> p20.inlet
|
||||
bcast.out(20).grouped(5) ~> p21.inlet
|
||||
bcast.out(21).grouped(5) ~> p22.inlet
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
Await.result(result, 3.seconds) should be(List.fill(22)(List(1, 2, 3)))
|
||||
}
|
||||
|
|
@ -124,12 +126,13 @@ class GraphBroadcastSpec extends AkkaSpec {
|
|||
val c1 = TestSubscriber.manualProbe[Int]()
|
||||
val c2 = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val bcast = b.add(Broadcast[Int](2))
|
||||
Source(List(1, 2, 3)) ~> bcast.in
|
||||
bcast.out(0) ~> Flow[Int] ~> Sink(c1)
|
||||
bcast.out(1) ~> Flow[Int] ~> Sink(c2)
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
val sub1 = c1.expectSubscription()
|
||||
sub1.cancel()
|
||||
|
|
@ -145,12 +148,13 @@ class GraphBroadcastSpec extends AkkaSpec {
|
|||
val c1 = TestSubscriber.manualProbe[Int]()
|
||||
val c2 = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val bcast = b.add(Broadcast[Int](2))
|
||||
Source(List(1, 2, 3)) ~> bcast.in
|
||||
bcast.out(0) ~> Flow[Int].named("identity-a") ~> Sink(c1)
|
||||
bcast.out(1) ~> Flow[Int].named("identity-b") ~> Sink(c2)
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
val sub1 = c1.expectSubscription()
|
||||
val sub2 = c2.expectSubscription()
|
||||
|
|
@ -167,12 +171,13 @@ class GraphBroadcastSpec extends AkkaSpec {
|
|||
val c1 = TestSubscriber.manualProbe[Int]()
|
||||
val c2 = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val bcast = b.add(Broadcast[Int](2))
|
||||
Source(p1.getPublisher) ~> bcast.in
|
||||
bcast.out(0) ~> Flow[Int] ~> Sink(c1)
|
||||
bcast.out(1) ~> Flow[Int] ~> Sink(c2)
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
val bsub = p1.expectSubscription()
|
||||
val sub1 = c1.expectSubscription()
|
||||
|
|
@ -195,12 +200,12 @@ class GraphBroadcastSpec extends AkkaSpec {
|
|||
val c1 = TestSubscriber.manualProbe[Int]()
|
||||
val c2 = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
val sink = Sink() { implicit b ⇒
|
||||
val sink = Sink.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val bcast = b.add(Broadcast[Int](2))
|
||||
bcast.out(0) ~> Sink(c1)
|
||||
bcast.out(1) ~> Sink(c2)
|
||||
bcast.in
|
||||
}
|
||||
SinkShape(bcast.in)
|
||||
})
|
||||
|
||||
val s = Source.subscriber[Int].to(sink).run()
|
||||
|
||||
|
|
|
|||
|
|
@ -30,7 +30,7 @@ class GraphConcatSpec extends TwoStreamsSetup {
|
|||
"work in the happy case" in assertAllStagesStopped {
|
||||
val probe = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
|
||||
val concat1 = b add Concat[Int]()
|
||||
val concat2 = b add Concat[Int]()
|
||||
|
|
@ -42,7 +42,8 @@ class GraphConcatSpec extends TwoStreamsSetup {
|
|||
Source(5 to 10) ~> concat2.in(1)
|
||||
|
||||
concat2.out ~> Sink(probe)
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
val subscription = probe.expectSubscription()
|
||||
|
||||
|
|
@ -140,12 +141,13 @@ class GraphConcatSpec extends TwoStreamsSetup {
|
|||
val promise = Promise[Int]()
|
||||
val subscriber = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val concat = b add Concat[Int]()
|
||||
Source(List(1, 2, 3)) ~> concat.in(0)
|
||||
Source(promise.future) ~> concat.in(1)
|
||||
concat.out ~> Sink(subscriber)
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
val subscription = subscriber.expectSubscription()
|
||||
subscription.request(4)
|
||||
|
|
|
|||
|
|
@ -3,7 +3,7 @@
|
|||
*/
|
||||
package akka.stream.scaladsl
|
||||
|
||||
import akka.stream.{ ActorMaterializer, ActorMaterializerSettings }
|
||||
import akka.stream.{ ClosedShape, SourceShape, ActorMaterializer, ActorMaterializerSettings }
|
||||
import akka.stream.testkit._
|
||||
|
||||
import scala.concurrent.Await
|
||||
|
|
@ -25,11 +25,12 @@ class GraphMatValueSpec extends AkkaSpec {
|
|||
|
||||
"expose the materialized value as source" in {
|
||||
val sub = TestSubscriber.manualProbe[Int]()
|
||||
val f = FlowGraph.closed(foldSink) { implicit b ⇒
|
||||
val f = RunnableGraph.fromGraph(FlowGraph.create(foldSink) { implicit b ⇒
|
||||
fold ⇒
|
||||
Source(1 to 10) ~> fold
|
||||
b.materializedValue.mapAsync(4)(identity) ~> Sink(sub)
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
val r1 = Await.result(f, 3.seconds)
|
||||
sub.expectSubscription().request(1)
|
||||
|
|
@ -41,7 +42,7 @@ class GraphMatValueSpec extends AkkaSpec {
|
|||
"expose the materialized value as source multiple times" in {
|
||||
val sub = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
val f = FlowGraph.closed(foldSink) { implicit b ⇒
|
||||
val f = RunnableGraph.fromGraph(FlowGraph.create(foldSink) { implicit b ⇒
|
||||
fold ⇒
|
||||
val zip = b.add(ZipWith[Int, Int, Int](_ + _))
|
||||
Source(1 to 10) ~> fold
|
||||
|
|
@ -49,7 +50,8 @@ class GraphMatValueSpec extends AkkaSpec {
|
|||
b.materializedValue.mapAsync(4)(identity) ~> zip.in1
|
||||
|
||||
zip.out ~> Sink(sub)
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
val r1 = Await.result(f, 3.seconds)
|
||||
sub.expectSubscription().request(1)
|
||||
|
|
@ -59,11 +61,11 @@ class GraphMatValueSpec extends AkkaSpec {
|
|||
}
|
||||
|
||||
// Exposes the materialized value as a stream value
|
||||
val foldFeedbackSource: Source[Future[Int], Future[Int]] = Source(foldSink) { implicit b ⇒
|
||||
val foldFeedbackSource: Source[Future[Int], Future[Int]] = Source.fromGraph(FlowGraph.create(foldSink) { implicit b ⇒
|
||||
fold ⇒
|
||||
Source(1 to 10) ~> fold
|
||||
b.materializedValue
|
||||
}
|
||||
SourceShape(b.materializedValue)
|
||||
})
|
||||
|
||||
"allow exposing the materialized value as port" in {
|
||||
val (f1, f2) = foldFeedbackSource.mapAsync(4)(identity).map(_ + 100).toMat(Sink.head)(Keep.both).run()
|
||||
|
|
@ -77,23 +79,22 @@ class GraphMatValueSpec extends AkkaSpec {
|
|||
}
|
||||
|
||||
"work properly with nesting and reusing" in {
|
||||
val compositeSource1 = Source(foldFeedbackSource, foldFeedbackSource)(Keep.both) { implicit b ⇒
|
||||
val compositeSource1 = Source.fromGraph(FlowGraph.create(foldFeedbackSource, foldFeedbackSource)(Keep.both) { implicit b ⇒
|
||||
(s1, s2) ⇒
|
||||
val zip = b.add(ZipWith[Int, Int, Int](_ + _))
|
||||
|
||||
s1.outlet.mapAsync(4)(identity) ~> zip.in0
|
||||
s2.outlet.mapAsync(4)(identity).map(_ * 100) ~> zip.in1
|
||||
zip.out
|
||||
}
|
||||
SourceShape(zip.out)
|
||||
})
|
||||
|
||||
val compositeSource2 = Source(compositeSource1, compositeSource1)(Keep.both) { implicit b ⇒
|
||||
val compositeSource2 = Source.fromGraph(FlowGraph.create(compositeSource1, compositeSource1)(Keep.both) { implicit b ⇒
|
||||
(s1, s2) ⇒
|
||||
val zip = b.add(ZipWith[Int, Int, Int](_ + _))
|
||||
|
||||
s1.outlet ~> zip.in0
|
||||
s2.outlet.map(_ * 10000) ~> zip.in1
|
||||
zip.out
|
||||
}
|
||||
SourceShape(zip.out)
|
||||
})
|
||||
|
||||
val (((f1, f2), (f3, f4)), result) = compositeSource2.toMat(Sink.head)(Keep.both).run()
|
||||
|
||||
|
|
|
|||
|
|
@ -3,7 +3,7 @@
|
|||
*/
|
||||
package akka.stream.scaladsl
|
||||
|
||||
import akka.stream.{ ActorMaterializer, ActorMaterializerSettings, Inlet, Outlet }
|
||||
import akka.stream._
|
||||
|
||||
import scala.concurrent.duration._
|
||||
|
||||
|
|
@ -33,7 +33,7 @@ class GraphMergeSpec extends TwoStreamsSetup {
|
|||
val source3 = Source(List[Int]())
|
||||
val probe = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val m1 = b.add(Merge[Int](2))
|
||||
val m2 = b.add(Merge[Int](2))
|
||||
|
||||
|
|
@ -43,7 +43,8 @@ class GraphMergeSpec extends TwoStreamsSetup {
|
|||
source2 ~> m1.in(1)
|
||||
source3 ~> m2.in(1)
|
||||
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
val subscription = probe.expectSubscription()
|
||||
|
||||
|
|
@ -67,7 +68,7 @@ class GraphMergeSpec extends TwoStreamsSetup {
|
|||
|
||||
val probe = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val merge = b.add(Merge[Int](6))
|
||||
|
||||
source1 ~> merge.in(0)
|
||||
|
|
@ -78,7 +79,8 @@ class GraphMergeSpec extends TwoStreamsSetup {
|
|||
source6 ~> merge.in(5)
|
||||
merge.out ~> Sink(probe)
|
||||
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
val subscription = probe.expectSubscription()
|
||||
|
||||
|
|
@ -152,13 +154,14 @@ class GraphMergeSpec extends TwoStreamsSetup {
|
|||
val src1 = Source.subscriber[Int]
|
||||
val src2 = Source.subscriber[Int]
|
||||
|
||||
val (graphSubscriber1, graphSubscriber2) = FlowGraph.closed(src1, src2)((_, _)) { implicit b ⇒
|
||||
val (graphSubscriber1, graphSubscriber2) = RunnableGraph.fromGraph(FlowGraph.create(src1, src2)((_, _)) { implicit b ⇒
|
||||
(s1, s2) ⇒
|
||||
val merge = b.add(Merge[Int](2))
|
||||
s1.outlet ~> merge.in(0)
|
||||
s2.outlet ~> merge.in(1)
|
||||
merge.out ~> Sink(down)
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
val downstream = down.expectSubscription()
|
||||
downstream.cancel()
|
||||
|
|
|
|||
|
|
@ -3,13 +3,10 @@ package akka.stream.scaladsl
|
|||
import scala.collection.immutable
|
||||
import scala.concurrent.{ Future, Await }
|
||||
import scala.concurrent.duration._
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.stream.ActorMaterializerSettings
|
||||
import akka.stream._
|
||||
import akka.stream.testkit._
|
||||
import akka.util.ByteString
|
||||
import akka.stream.{ Inlet, Outlet, Shape, Graph }
|
||||
import org.scalactic.ConversionCheckedTripleEquals
|
||||
import akka.stream.Attributes
|
||||
|
||||
object GraphOpsIntegrationSpec {
|
||||
import FlowGraph.Implicits._
|
||||
|
|
@ -33,7 +30,7 @@ object GraphOpsIntegrationSpec {
|
|||
}
|
||||
|
||||
def apply[In, Out](pipeline: Flow[In, Out, _]): Graph[ShufflePorts[In, Out], Unit] = {
|
||||
FlowGraph.partial() { implicit b ⇒
|
||||
FlowGraph.create() { implicit b ⇒
|
||||
val merge = b.add(Merge[In](2))
|
||||
val balance = b.add(Balance[Out](2))
|
||||
merge.out ~> pipeline ~> balance.in
|
||||
|
|
@ -57,7 +54,7 @@ class GraphOpsIntegrationSpec extends AkkaSpec with ConversionCheckedTripleEqual
|
|||
"FlowGraphs" must {
|
||||
|
||||
"support broadcast - merge layouts" in {
|
||||
val resultFuture = FlowGraph.closed(Sink.head[Seq[Int]]) { implicit b ⇒
|
||||
val resultFuture = RunnableGraph.fromGraph(FlowGraph.create(Sink.head[Seq[Int]]) { implicit b ⇒
|
||||
(sink) ⇒
|
||||
val bcast = b.add(Broadcast[Int](2))
|
||||
val merge = b.add(Merge[Int](2))
|
||||
|
|
@ -66,14 +63,15 @@ class GraphOpsIntegrationSpec extends AkkaSpec with ConversionCheckedTripleEqual
|
|||
bcast.out(0) ~> merge.in(0)
|
||||
bcast.out(1).map(_ + 3) ~> merge.in(1)
|
||||
merge.out.grouped(10) ~> sink.inlet
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
Await.result(resultFuture, 3.seconds).sorted should be(List(1, 2, 3, 4, 5, 6))
|
||||
}
|
||||
|
||||
"support balance - merge (parallelization) layouts" in {
|
||||
val elements = 0 to 10
|
||||
val out = FlowGraph.closed(Sink.head[Seq[Int]]) { implicit b ⇒
|
||||
val out = RunnableGraph.fromGraph(FlowGraph.create(Sink.head[Seq[Int]]) { implicit b ⇒
|
||||
(sink) ⇒
|
||||
val balance = b.add(Balance[Int](5))
|
||||
val merge = b.add(Merge[Int](5))
|
||||
|
|
@ -83,7 +81,8 @@ class GraphOpsIntegrationSpec extends AkkaSpec with ConversionCheckedTripleEqual
|
|||
for (i ← 0 until 5) balance.out(i) ~> merge.in(i)
|
||||
|
||||
merge.out.grouped(elements.size * 2) ~> sink.inlet
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
Await.result(out, 3.seconds).sorted should be(elements)
|
||||
}
|
||||
|
|
@ -93,7 +92,7 @@ class GraphOpsIntegrationSpec extends AkkaSpec with ConversionCheckedTripleEqual
|
|||
// see https://en.wikipedia.org/wiki/Topological_sorting#mediaviewer/File:Directed_acyclic_graph.png
|
||||
val seqSink = Sink.head[Seq[Int]]
|
||||
|
||||
val (resultFuture2, resultFuture9, resultFuture10) = FlowGraph.closed(seqSink, seqSink, seqSink)(Tuple3.apply) { implicit b ⇒
|
||||
val (resultFuture2, resultFuture9, resultFuture10) = RunnableGraph.fromGraph(FlowGraph.create(seqSink, seqSink, seqSink)(Tuple3.apply) { implicit b ⇒
|
||||
(sink2, sink9, sink10) ⇒
|
||||
val b3 = b.add(Broadcast[Int](2))
|
||||
val b7 = b.add(Broadcast[Int](2))
|
||||
|
|
@ -129,7 +128,8 @@ class GraphOpsIntegrationSpec extends AkkaSpec with ConversionCheckedTripleEqual
|
|||
m9.out.grouped(1000) ~> sink9.inlet
|
||||
m10.out.grouped(1000) ~> sink10.inlet
|
||||
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
Await.result(resultFuture2, 3.seconds).sorted should be(List(5, 7))
|
||||
Await.result(resultFuture9, 3.seconds).sorted should be(List(3, 5, 7, 7))
|
||||
|
|
@ -139,7 +139,7 @@ class GraphOpsIntegrationSpec extends AkkaSpec with ConversionCheckedTripleEqual
|
|||
|
||||
"allow adding of flows to sources and sinks to flows" in {
|
||||
|
||||
val resultFuture = FlowGraph.closed(Sink.head[Seq[Int]]) { implicit b ⇒
|
||||
val resultFuture = RunnableGraph.fromGraph(FlowGraph.create(Sink.head[Seq[Int]]) { implicit b ⇒
|
||||
(sink) ⇒
|
||||
val bcast = b.add(Broadcast[Int](2))
|
||||
val merge = b.add(Merge[Int](2))
|
||||
|
|
@ -148,7 +148,8 @@ class GraphOpsIntegrationSpec extends AkkaSpec with ConversionCheckedTripleEqual
|
|||
bcast.out(0) ~> merge.in(0)
|
||||
bcast.out(1).map(_ + 3) ~> merge.in(1)
|
||||
merge.out.grouped(10) ~> sink.inlet
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
Await.result(resultFuture, 3.seconds) should contain theSameElementsAs (Seq(2, 4, 6, 5, 7, 9))
|
||||
}
|
||||
|
|
@ -157,9 +158,10 @@ class GraphOpsIntegrationSpec extends AkkaSpec with ConversionCheckedTripleEqual
|
|||
val p = Source(List(1, 2, 3)).runWith(Sink.publisher)
|
||||
val s = TestSubscriber.manualProbe[Int]
|
||||
val flow = Flow[Int].map(_ * 2)
|
||||
FlowGraph.closed() { implicit builder ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit builder ⇒
|
||||
Source(p) ~> flow ~> Sink(s)
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
val sub = s.expectSubscription()
|
||||
sub.request(10)
|
||||
s.expectNext(1 * 2)
|
||||
|
|
@ -171,7 +173,7 @@ class GraphOpsIntegrationSpec extends AkkaSpec with ConversionCheckedTripleEqual
|
|||
"be possible to use as lego bricks" in {
|
||||
val shuffler = Shuffle(Flow[Int].map(_ + 1))
|
||||
|
||||
val f: Future[Seq[Int]] = FlowGraph.closed(shuffler, shuffler, shuffler, Sink.head[Seq[Int]])((_, _, _, fut) ⇒ fut) { implicit b ⇒
|
||||
val f: Future[Seq[Int]] = RunnableGraph.fromGraph(FlowGraph.create(shuffler, shuffler, shuffler, Sink.head[Seq[Int]])((_, _, _, fut) ⇒ fut) { implicit b ⇒
|
||||
(s1, s2, s3, sink) ⇒
|
||||
val merge = b.add(Merge[Int](2))
|
||||
|
||||
|
|
@ -188,7 +190,8 @@ class GraphOpsIntegrationSpec extends AkkaSpec with ConversionCheckedTripleEqual
|
|||
s3.out2 ~> merge.in(1)
|
||||
|
||||
merge.out.grouped(1000) ~> sink
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
val result = Await.result(f, 3.seconds)
|
||||
|
||||
|
|
|
|||
|
|
@ -1,6 +1,6 @@
|
|||
package akka.stream.scaladsl
|
||||
|
||||
import akka.stream.{ ActorMaterializer, ActorMaterializerSettings, FlowShape }
|
||||
import akka.stream.{ ClosedShape, ActorMaterializer, ActorMaterializerSettings, FlowShape }
|
||||
import akka.stream.testkit.AkkaSpec
|
||||
|
||||
import scala.concurrent.{ Await, Future }
|
||||
|
|
@ -18,7 +18,7 @@ class GraphPartialSpec extends AkkaSpec {
|
|||
import FlowGraph.Implicits._
|
||||
|
||||
"be able to build and reuse simple partial graphs" in {
|
||||
val doubler = FlowGraph.partial() { implicit b ⇒
|
||||
val doubler = FlowGraph.create() { implicit b ⇒
|
||||
val bcast = b.add(Broadcast[Int](2))
|
||||
val zip = b.add(ZipWith((a: Int, b: Int) ⇒ a + b))
|
||||
|
||||
|
|
@ -27,18 +27,19 @@ class GraphPartialSpec extends AkkaSpec {
|
|||
FlowShape(bcast.in, zip.out)
|
||||
}
|
||||
|
||||
val (_, _, result) = FlowGraph.closed(doubler, doubler, Sink.head[Seq[Int]])(Tuple3.apply) { implicit b ⇒
|
||||
val (_, _, result) = RunnableGraph.fromGraph(FlowGraph.create(doubler, doubler, Sink.head[Seq[Int]])(Tuple3.apply) { implicit b ⇒
|
||||
(d1, d2, sink) ⇒
|
||||
Source(List(1, 2, 3)) ~> d1.inlet
|
||||
d1.outlet ~> d2.inlet
|
||||
d2.outlet.grouped(100) ~> sink.inlet
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
Await.result(result, 3.seconds) should be(List(4, 8, 12))
|
||||
}
|
||||
|
||||
"be able to build and reuse simple materializing partial graphs" in {
|
||||
val doubler = FlowGraph.partial(Sink.head[Seq[Int]]) { implicit b ⇒
|
||||
val doubler = FlowGraph.create(Sink.head[Seq[Int]]) { implicit b ⇒
|
||||
sink ⇒
|
||||
val bcast = b.add(Broadcast[Int](3))
|
||||
val zip = b.add(ZipWith((a: Int, b: Int) ⇒ a + b))
|
||||
|
|
@ -49,12 +50,13 @@ class GraphPartialSpec extends AkkaSpec {
|
|||
FlowShape(bcast.in, zip.out)
|
||||
}
|
||||
|
||||
val (sub1, sub2, result) = FlowGraph.closed(doubler, doubler, Sink.head[Seq[Int]])(Tuple3.apply) { implicit b ⇒
|
||||
val (sub1, sub2, result) = RunnableGraph.fromGraph(FlowGraph.create(doubler, doubler, Sink.head[Seq[Int]])(Tuple3.apply) { implicit b ⇒
|
||||
(d1, d2, sink) ⇒
|
||||
Source(List(1, 2, 3)) ~> d1.inlet
|
||||
d1.outlet ~> d2.inlet
|
||||
d2.outlet.grouped(100) ~> sink.inlet
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
Await.result(result, 3.seconds) should be(List(4, 8, 12))
|
||||
Await.result(sub1, 3.seconds) should be(List(1, 2, 3))
|
||||
|
|
@ -64,7 +66,7 @@ class GraphPartialSpec extends AkkaSpec {
|
|||
"be able to build and reuse complex materializing partial graphs" in {
|
||||
val summer = Sink.fold[Int, Int](0)(_ + _)
|
||||
|
||||
val doubler = FlowGraph.partial(summer, summer)(Tuple2.apply) { implicit b ⇒
|
||||
val doubler = FlowGraph.create(summer, summer)(Tuple2.apply) { implicit b ⇒
|
||||
(s1, s2) ⇒
|
||||
val bcast = b.add(Broadcast[Int](3))
|
||||
val bcast2 = b.add(Broadcast[Int](2))
|
||||
|
|
@ -80,12 +82,13 @@ class GraphPartialSpec extends AkkaSpec {
|
|||
FlowShape(bcast.in, bcast2.out(1))
|
||||
}
|
||||
|
||||
val (sub1, sub2, result) = FlowGraph.closed(doubler, doubler, Sink.head[Seq[Int]])(Tuple3.apply) { implicit b ⇒
|
||||
val (sub1, sub2, result) = RunnableGraph.fromGraph(FlowGraph.create(doubler, doubler, Sink.head[Seq[Int]])(Tuple3.apply) { implicit b ⇒
|
||||
(d1, d2, sink) ⇒
|
||||
Source(List(1, 2, 3)) ~> d1.inlet
|
||||
d1.outlet ~> d2.inlet
|
||||
d2.outlet.grouped(100) ~> sink.inlet
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
Await.result(result, 3.seconds) should be(List(4, 8, 12))
|
||||
Await.result(sub1._1, 3.seconds) should be(6)
|
||||
|
|
@ -95,17 +98,18 @@ class GraphPartialSpec extends AkkaSpec {
|
|||
}
|
||||
|
||||
"be able to expose the ports of imported graphs" in {
|
||||
val p = FlowGraph.partial(Flow[Int].map(_ + 1)) { implicit b ⇒
|
||||
val p = FlowGraph.create(Flow[Int].map(_ + 1)) { implicit b ⇒
|
||||
flow ⇒
|
||||
FlowShape(flow.inlet, flow.outlet)
|
||||
}
|
||||
|
||||
val fut = FlowGraph.closed(Sink.head[Int], p)(Keep.left) { implicit b ⇒
|
||||
val fut = RunnableGraph.fromGraph(FlowGraph.create(Sink.head[Int], p)(Keep.left) { implicit b ⇒
|
||||
(sink, flow) ⇒
|
||||
import FlowGraph.Implicits._
|
||||
Source.single(0) ~> flow.inlet
|
||||
flow.outlet ~> sink.inlet
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
Await.result(fut, 3.seconds) should be(1)
|
||||
|
||||
|
|
|
|||
|
|
@ -32,7 +32,7 @@ class GraphPreferredMergeSpec extends TwoStreamsSetup {
|
|||
val preferred = Source(Stream.fill(numElements)(1))
|
||||
val aux = Source(Stream.fill(numElements)(2))
|
||||
|
||||
val result = FlowGraph.closed(Sink.head[Seq[Int]]) { implicit b ⇒
|
||||
val result = RunnableGraph.fromGraph(FlowGraph.create(Sink.head[Seq[Int]]) { implicit b ⇒
|
||||
sink ⇒
|
||||
val merge = b.add(MergePreferred[Int](3))
|
||||
preferred ~> merge.preferred
|
||||
|
|
@ -41,13 +41,14 @@ class GraphPreferredMergeSpec extends TwoStreamsSetup {
|
|||
aux ~> merge.in(0)
|
||||
aux ~> merge.in(1)
|
||||
aux ~> merge.in(2)
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
Await.result(result, 3.seconds).filter(_ == 1).size should be(numElements)
|
||||
}
|
||||
|
||||
"eventually pass through all elements" in {
|
||||
val result = FlowGraph.closed(Sink.head[Seq[Int]]) { implicit b ⇒
|
||||
val result = RunnableGraph.fromGraph(FlowGraph.create(Sink.head[Seq[Int]]) { implicit b ⇒
|
||||
sink ⇒
|
||||
val merge = b.add(MergePreferred[Int](3))
|
||||
Source(1 to 100) ~> merge.preferred
|
||||
|
|
@ -56,7 +57,8 @@ class GraphPreferredMergeSpec extends TwoStreamsSetup {
|
|||
Source(101 to 200) ~> merge.in(0)
|
||||
Source(201 to 300) ~> merge.in(1)
|
||||
Source(301 to 400) ~> merge.in(2)
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
Await.result(result, 3.seconds).toSet should ===((1 to 400).toSet)
|
||||
}
|
||||
|
|
@ -65,7 +67,7 @@ class GraphPreferredMergeSpec extends TwoStreamsSetup {
|
|||
val s = Source(0 to 3)
|
||||
|
||||
(the[IllegalArgumentException] thrownBy {
|
||||
val g = FlowGraph.closed() { implicit b ⇒
|
||||
val g = RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val merge = b.add(MergePreferred[Int](1))
|
||||
|
||||
s ~> merge.preferred
|
||||
|
|
@ -73,7 +75,8 @@ class GraphPreferredMergeSpec extends TwoStreamsSetup {
|
|||
s ~> merge.in(0)
|
||||
|
||||
merge.out ~> Sink.head[Int]
|
||||
}
|
||||
ClosedShape
|
||||
})
|
||||
}).getMessage should include("[MergePreferred.preferred] is already connected")
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -23,12 +23,12 @@ object GraphStageTimersSpec {
|
|||
|
||||
class SideChannel {
|
||||
@volatile var asyncCallback: AsyncCallback[Any] = _
|
||||
@volatile var stopPromise: Promise[Unit] = _
|
||||
@volatile var stopPromise: Promise[Option[Nothing]] = _
|
||||
|
||||
def isReady: Boolean = asyncCallback ne null
|
||||
def !(msg: Any) = asyncCallback.invoke(msg)
|
||||
|
||||
def stopStage(): Unit = stopPromise.trySuccess(())
|
||||
def stopStage(): Unit = stopPromise.trySuccess(None)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -81,7 +81,7 @@ class GraphStageTimersSpec extends AkkaSpec {
|
|||
|
||||
def setupIsolatedStage: SideChannel = {
|
||||
val channel = new SideChannel
|
||||
val stopPromise = Source.lazyEmpty[Int].via(new TestStage(testActor, channel)).to(Sink.ignore).run()
|
||||
val stopPromise = Source.maybe[Nothing].via(new TestStage(testActor, channel)).to(Sink.ignore).run()
|
||||
channel.stopPromise = stopPromise
|
||||
awaitCond(channel.isReady)
|
||||
channel
|
||||
|
|
|
|||
|
|
@ -5,8 +5,7 @@ package akka.stream.scaladsl
|
|||
|
||||
import scala.concurrent.duration._
|
||||
|
||||
import akka.stream.{ OverflowStrategy, ActorMaterializerSettings }
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.stream.{ ClosedShape, OverflowStrategy, ActorMaterializerSettings, ActorMaterializer }
|
||||
import akka.stream.testkit._
|
||||
import akka.stream.testkit.Utils._
|
||||
|
||||
|
|
@ -24,12 +23,13 @@ class GraphUnzipSpec extends AkkaSpec {
|
|||
val c1 = TestSubscriber.manualProbe[Int]()
|
||||
val c2 = TestSubscriber.manualProbe[String]()
|
||||
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val unzip = b.add(Unzip[Int, String]())
|
||||
Source(List(1 -> "a", 2 -> "b", 3 -> "c")) ~> unzip.in
|
||||
unzip.out1 ~> Flow[String].buffer(16, OverflowStrategy.backpressure) ~> Sink(c2)
|
||||
unzip.out0 ~> Flow[Int].buffer(16, OverflowStrategy.backpressure).map(_ * 2) ~> Sink(c1)
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
val sub1 = c1.expectSubscription()
|
||||
val sub2 = c2.expectSubscription()
|
||||
|
|
@ -53,12 +53,13 @@ class GraphUnzipSpec extends AkkaSpec {
|
|||
val c1 = TestSubscriber.manualProbe[Int]()
|
||||
val c2 = TestSubscriber.manualProbe[String]()
|
||||
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val unzip = b.add(Unzip[Int, String]())
|
||||
Source(List(1 -> "a", 2 -> "b", 3 -> "c")) ~> unzip.in
|
||||
unzip.out0 ~> Sink(c1)
|
||||
unzip.out1 ~> Sink(c2)
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
val sub1 = c1.expectSubscription()
|
||||
val sub2 = c2.expectSubscription()
|
||||
|
|
@ -74,12 +75,13 @@ class GraphUnzipSpec extends AkkaSpec {
|
|||
val c1 = TestSubscriber.manualProbe[Int]()
|
||||
val c2 = TestSubscriber.manualProbe[String]()
|
||||
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val unzip = b.add(Unzip[Int, String]())
|
||||
Source(List(1 -> "a", 2 -> "b", 3 -> "c")) ~> unzip.in
|
||||
unzip.out0 ~> Sink(c1)
|
||||
unzip.out1 ~> Sink(c2)
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
val sub1 = c1.expectSubscription()
|
||||
val sub2 = c2.expectSubscription()
|
||||
|
|
@ -96,12 +98,13 @@ class GraphUnzipSpec extends AkkaSpec {
|
|||
val c1 = TestSubscriber.manualProbe[Int]()
|
||||
val c2 = TestSubscriber.manualProbe[String]()
|
||||
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val unzip = b.add(Unzip[Int, String]())
|
||||
Source(p1.getPublisher) ~> unzip.in
|
||||
unzip.out0 ~> Sink(c1)
|
||||
unzip.out1 ~> Sink(c2)
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
val p1Sub = p1.expectSubscription()
|
||||
val sub1 = c1.expectSubscription()
|
||||
|
|
@ -122,14 +125,15 @@ class GraphUnzipSpec extends AkkaSpec {
|
|||
|
||||
"work with zip" in assertAllStagesStopped {
|
||||
val c1 = TestSubscriber.manualProbe[(Int, String)]()
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val zip = b.add(Zip[Int, String]())
|
||||
val unzip = b.add(Unzip[Int, String]())
|
||||
Source(List(1 -> "a", 2 -> "b", 3 -> "c")) ~> unzip.in
|
||||
unzip.out0 ~> zip.in0
|
||||
unzip.out1 ~> zip.in1
|
||||
zip.out ~> Sink(c1)
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
val sub1 = c1.expectSubscription()
|
||||
sub1.request(5)
|
||||
|
|
|
|||
|
|
@ -48,14 +48,15 @@ class GraphUnzipWithSpec extends AkkaSpec {
|
|||
val leftSubscriber = TestSubscriber.probe[LeftOutput]()
|
||||
val rightSubscriber = TestSubscriber.probe[RightOutput]()
|
||||
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val f = fixture(b)
|
||||
|
||||
Source(p) ~> f.in
|
||||
f.left ~> Sink(leftSubscriber)
|
||||
f.right ~> Sink(rightSubscriber)
|
||||
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
(leftSubscriber, rightSubscriber)
|
||||
}
|
||||
|
|
@ -96,13 +97,15 @@ class GraphUnzipWithSpec extends AkkaSpec {
|
|||
val leftProbe = TestSubscriber.manualProbe[LeftOutput]()
|
||||
val rightProbe = TestSubscriber.manualProbe[RightOutput]()
|
||||
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val unzip = b.add(UnzipWith(f))
|
||||
Source(1 to 4) ~> unzip.in
|
||||
|
||||
unzip.out0 ~> Flow[LeftOutput].buffer(4, OverflowStrategy.backpressure) ~> Sink(leftProbe)
|
||||
unzip.out1 ~> Flow[RightOutput].buffer(4, OverflowStrategy.backpressure) ~> Sink(rightProbe)
|
||||
}.run()
|
||||
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
val leftSubscription = leftProbe.expectSubscription()
|
||||
val rightSubscription = rightProbe.expectSubscription()
|
||||
|
|
@ -144,14 +147,16 @@ class GraphUnzipWithSpec extends AkkaSpec {
|
|||
val leftProbe = TestSubscriber.manualProbe[LeftOutput]()
|
||||
val rightProbe = TestSubscriber.manualProbe[RightOutput]()
|
||||
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val unzip = b.add(UnzipWith[Int, Int, String]((b: Int) ⇒ (1 / b, 1 + "/" + b)))
|
||||
|
||||
Source(-2 to 2) ~> unzip.in
|
||||
|
||||
unzip.out0 ~> Sink(leftProbe)
|
||||
unzip.out1 ~> Sink(rightProbe)
|
||||
}.run()
|
||||
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
val leftSubscription = leftProbe.expectSubscription()
|
||||
val rightSubscription = rightProbe.expectSubscription()
|
||||
|
|
@ -187,7 +192,7 @@ class GraphUnzipWithSpec extends AkkaSpec {
|
|||
|
||||
case class Person(name: String, surname: String, int: Int)
|
||||
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val unzip = b.add(UnzipWith((a: Person) ⇒ Person.unapply(a).get))
|
||||
|
||||
Source.single(Person("Caplin", "Capybara", 3)) ~> unzip.in
|
||||
|
|
@ -195,7 +200,9 @@ class GraphUnzipWithSpec extends AkkaSpec {
|
|||
unzip.out0 ~> Sink(probe0)
|
||||
unzip.out1 ~> Sink(probe1)
|
||||
unzip.out2 ~> Sink(probe2)
|
||||
}.run()
|
||||
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
val subscription0 = probe0.expectSubscription()
|
||||
val subscription1 = probe1.expectSubscription()
|
||||
|
|
@ -221,7 +228,7 @@ class GraphUnzipWithSpec extends AkkaSpec {
|
|||
val probe15 = TestSubscriber.manualProbe[String]()
|
||||
val probe19 = TestSubscriber.manualProbe[String]()
|
||||
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
|
||||
val split20 = (a: (List[Int])) ⇒
|
||||
(a(0), a(0).toString,
|
||||
|
|
@ -268,7 +275,8 @@ class GraphUnzipWithSpec extends AkkaSpec {
|
|||
|
||||
unzip.out19 ~> Sink(probe19)
|
||||
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
probe0.expectSubscription().request(1)
|
||||
probe5.expectSubscription().request(1)
|
||||
|
|
|
|||
|
|
@ -25,14 +25,16 @@ class GraphZipSpec extends TwoStreamsSetup {
|
|||
"work in the happy case" in assertAllStagesStopped {
|
||||
val probe = TestSubscriber.manualProbe[(Int, String)]()
|
||||
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val zip = b.add(Zip[Int, String]())
|
||||
|
||||
Source(1 to 4) ~> zip.in0
|
||||
Source(List("A", "B", "C", "D", "E", "F")) ~> zip.in1
|
||||
|
||||
zip.out ~> Sink(probe)
|
||||
}.run()
|
||||
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
val subscription = probe.expectSubscription()
|
||||
|
||||
|
|
|
|||
|
|
@ -21,13 +21,15 @@ class GraphZipWithSpec extends TwoStreamsSetup {
|
|||
"work in the happy case" in {
|
||||
val probe = TestSubscriber.manualProbe[Outputs]()
|
||||
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val zip = b.add(ZipWith((_: Int) + (_: Int)))
|
||||
Source(1 to 4) ~> zip.in0
|
||||
Source(10 to 40 by 10) ~> zip.in1
|
||||
|
||||
zip.out ~> Sink(probe)
|
||||
}.run()
|
||||
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
val subscription = probe.expectSubscription()
|
||||
|
||||
|
|
@ -46,14 +48,16 @@ class GraphZipWithSpec extends TwoStreamsSetup {
|
|||
"work in the sad case" in {
|
||||
val probe = TestSubscriber.manualProbe[Outputs]()
|
||||
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val zip = b.add(ZipWith[Int, Int, Int]((_: Int) / (_: Int)))
|
||||
|
||||
Source(1 to 4) ~> zip.in0
|
||||
Source(-2 to 2) ~> zip.in1
|
||||
|
||||
zip.out ~> Sink(probe)
|
||||
}.run()
|
||||
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
val subscription = probe.expectSubscription()
|
||||
|
||||
|
|
@ -107,7 +111,7 @@ class GraphZipWithSpec extends TwoStreamsSetup {
|
|||
|
||||
case class Person(name: String, surname: String, int: Int)
|
||||
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val zip = b.add(ZipWith(Person.apply _))
|
||||
|
||||
Source.single("Caplin") ~> zip.in0
|
||||
|
|
@ -115,7 +119,9 @@ class GraphZipWithSpec extends TwoStreamsSetup {
|
|||
Source.single(3) ~> zip.in2
|
||||
|
||||
zip.out ~> Sink(probe)
|
||||
}.run()
|
||||
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
val subscription = probe.expectSubscription()
|
||||
|
||||
|
|
@ -128,7 +134,7 @@ class GraphZipWithSpec extends TwoStreamsSetup {
|
|||
"work with up to 22 inputs" in {
|
||||
val probe = TestSubscriber.manualProbe[String]()
|
||||
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
|
||||
val sum19 = (v1: Int, v2: String, v3: Int, v4: String, v5: Int, v6: String, v7: Int, v8: String, v9: Int, v10: String,
|
||||
v11: Int, v12: String, v13: Int, v14: String, v15: Int, v16: String, v17: Int, v18: String, v19: Int) ⇒
|
||||
|
|
@ -159,7 +165,9 @@ class GraphZipWithSpec extends TwoStreamsSetup {
|
|||
Source.single(19) ~> zip.in18
|
||||
|
||||
zip.out ~> Sink(probe)
|
||||
}.run()
|
||||
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
val subscription = probe.expectSubscription()
|
||||
|
||||
|
|
|
|||
|
|
@ -65,7 +65,7 @@ class One2OneBidiFlowSpec extends AkkaSpec with ConversionCheckedTripleEquals {
|
|||
|
||||
Source(1 to 1000)
|
||||
.log("", seen.set)
|
||||
.via(One2OneBidiFlow[Int, Int](MAX_PENDING) join Flow.wrap(Sink.ignore, Source(out))(Keep.left))
|
||||
.via(One2OneBidiFlow[Int, Int](MAX_PENDING) join Flow.fromSinkAndSourceMat(Sink.ignore, Source(out))(Keep.left))
|
||||
.runWith(Sink.ignore)
|
||||
|
||||
Thread.sleep(50)
|
||||
|
|
@ -82,6 +82,6 @@ class One2OneBidiFlowSpec extends AkkaSpec with ConversionCheckedTripleEquals {
|
|||
val outIn = TestPublisher.probe[Int]()
|
||||
val outOut = TestSubscriber.probe[Int]()
|
||||
|
||||
Source(inIn).via(One2OneBidiFlow[Int, Int](maxPending) join Flow.wrap(Sink(inOut), Source(outIn))(Keep.left)).runWith(Sink(outOut))
|
||||
Source(inIn).via(One2OneBidiFlow[Int, Int](maxPending) join Flow.fromSinkAndSourceMat(Sink(inOut), Source(outIn))(Keep.left)).runWith(Sink(outOut))
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -3,7 +3,7 @@
|
|||
*/
|
||||
package akka.stream.scaladsl
|
||||
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.stream.{ ClosedShape, ActorMaterializer }
|
||||
|
||||
import akka.stream.testkit.AkkaSpec
|
||||
import akka.stream.testkit.Utils._
|
||||
|
|
@ -19,7 +19,7 @@ class PublisherSinkSpec extends AkkaSpec {
|
|||
|
||||
"be unique when created twice" in assertAllStagesStopped {
|
||||
|
||||
val (pub1, pub2) = FlowGraph.closed(Sink.publisher[Int], Sink.publisher[Int])(Keep.both) { implicit b ⇒
|
||||
val (pub1, pub2) = RunnableGraph.fromGraph(FlowGraph.create(Sink.publisher[Int], Sink.publisher[Int])(Keep.both) { implicit b ⇒
|
||||
(p1, p2) ⇒
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
|
|
@ -28,7 +28,8 @@ class PublisherSinkSpec extends AkkaSpec {
|
|||
Source(0 to 5) ~> bcast.in
|
||||
bcast.out(0).map(_ * 2) ~> p1.inlet
|
||||
bcast.out(1) ~> p2.inlet
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
val f1 = Source(pub1).map(identity).runFold(0)(_ + _)
|
||||
val f2 = Source(pub2).map(identity).runFold(0)(_ + _)
|
||||
|
|
|
|||
|
|
@ -16,43 +16,48 @@ class ReverseArrowSpec extends AkkaSpec with ConversionCheckedTripleEquals {
|
|||
"Reverse Arrows in the Graph DSL" must {
|
||||
|
||||
"work from Inlets" in {
|
||||
Await.result(FlowGraph.closed(sink) { implicit b ⇒
|
||||
Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b ⇒
|
||||
s ⇒
|
||||
s.inlet <~ source
|
||||
}.run(), 1.second) should ===(Seq(1, 2, 3))
|
||||
ClosedShape
|
||||
}).run(), 1.second) should ===(Seq(1, 2, 3))
|
||||
}
|
||||
|
||||
"work from SinkShape" in {
|
||||
Await.result(FlowGraph.closed(sink) { implicit b ⇒
|
||||
Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b ⇒
|
||||
s ⇒
|
||||
s <~ source
|
||||
}.run(), 1.second) should ===(Seq(1, 2, 3))
|
||||
ClosedShape
|
||||
}).run(), 1.second) should ===(Seq(1, 2, 3))
|
||||
}
|
||||
|
||||
"work from Sink" in {
|
||||
val sub = TestSubscriber.manualProbe[Int]
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
Sink(sub) <~ source
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
sub.expectSubscription().request(10)
|
||||
sub.expectNext(1, 2, 3)
|
||||
sub.expectComplete()
|
||||
}
|
||||
|
||||
"not work from Outlets" in {
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val o: Outlet[Int] = b.add(source)
|
||||
"o <~ source" shouldNot compile
|
||||
sink <~ o
|
||||
}
|
||||
ClosedShape
|
||||
})
|
||||
}
|
||||
|
||||
"not work from SourceShape" in {
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val o: SourceShape[Int] = b.add(source)
|
||||
"o <~ source" shouldNot compile
|
||||
sink <~ o
|
||||
}
|
||||
ClosedShape
|
||||
})
|
||||
}
|
||||
|
||||
"not work from Source" in {
|
||||
|
|
@ -60,114 +65,127 @@ class ReverseArrowSpec extends AkkaSpec with ConversionCheckedTripleEquals {
|
|||
}
|
||||
|
||||
"work from FlowShape" in {
|
||||
Await.result(FlowGraph.closed(sink) { implicit b ⇒
|
||||
Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b ⇒
|
||||
s ⇒
|
||||
val f: FlowShape[Int, Int] = b.add(Flow[Int])
|
||||
f <~ source
|
||||
f ~> s
|
||||
}.run(), 1.second) should ===(Seq(1, 2, 3))
|
||||
ClosedShape
|
||||
}).run(), 1.second) should ===(Seq(1, 2, 3))
|
||||
}
|
||||
|
||||
"work from UniformFanInShape" in {
|
||||
Await.result(FlowGraph.closed(sink) { implicit b ⇒
|
||||
Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b ⇒
|
||||
s ⇒
|
||||
val f: UniformFanInShape[Int, Int] = b.add(Merge[Int](1))
|
||||
f <~ source
|
||||
f ~> s
|
||||
}.run(), 1.second) should ===(Seq(1, 2, 3))
|
||||
ClosedShape
|
||||
}).run(), 1.second) should ===(Seq(1, 2, 3))
|
||||
}
|
||||
|
||||
"work from UniformFanOutShape" in {
|
||||
Await.result(FlowGraph.closed(sink) { implicit b ⇒
|
||||
Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b ⇒
|
||||
s ⇒
|
||||
val f: UniformFanOutShape[Int, Int] = b.add(Broadcast[Int](1))
|
||||
f <~ source
|
||||
f ~> s
|
||||
}.run(), 1.second) should ===(Seq(1, 2, 3))
|
||||
ClosedShape
|
||||
}).run(), 1.second) should ===(Seq(1, 2, 3))
|
||||
}
|
||||
|
||||
"work towards Outlets" in {
|
||||
Await.result(FlowGraph.closed(sink) { implicit b ⇒
|
||||
Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b ⇒
|
||||
s ⇒
|
||||
val o: Outlet[Int] = b.add(source)
|
||||
s <~ o
|
||||
}.run(), 1.second) should ===(Seq(1, 2, 3))
|
||||
ClosedShape
|
||||
}).run(), 1.second) should ===(Seq(1, 2, 3))
|
||||
}
|
||||
|
||||
"work towards SourceShape" in {
|
||||
Await.result(FlowGraph.closed(sink) { implicit b ⇒
|
||||
Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b ⇒
|
||||
s ⇒
|
||||
val o: SourceShape[Int] = b.add(source)
|
||||
s <~ o
|
||||
}.run(), 1.second) should ===(Seq(1, 2, 3))
|
||||
ClosedShape
|
||||
}).run(), 1.second) should ===(Seq(1, 2, 3))
|
||||
}
|
||||
|
||||
"work towards Source" in {
|
||||
Await.result(FlowGraph.closed(sink) { implicit b ⇒
|
||||
Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b ⇒
|
||||
s ⇒
|
||||
s <~ source
|
||||
}.run(), 1.second) should ===(Seq(1, 2, 3))
|
||||
ClosedShape
|
||||
}).run(), 1.second) should ===(Seq(1, 2, 3))
|
||||
}
|
||||
|
||||
"work towards FlowShape" in {
|
||||
Await.result(FlowGraph.closed(sink) { implicit b ⇒
|
||||
Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b ⇒
|
||||
s ⇒
|
||||
val f: FlowShape[Int, Int] = b.add(Flow[Int])
|
||||
s <~ f
|
||||
source ~> f
|
||||
}.run(), 1.second) should ===(Seq(1, 2, 3))
|
||||
ClosedShape
|
||||
}).run(), 1.second) should ===(Seq(1, 2, 3))
|
||||
}
|
||||
|
||||
"work towards UniformFanInShape" in {
|
||||
Await.result(FlowGraph.closed(sink) { implicit b ⇒
|
||||
Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b ⇒
|
||||
s ⇒
|
||||
val f: UniformFanInShape[Int, Int] = b.add(Merge[Int](1))
|
||||
s <~ f
|
||||
source ~> f
|
||||
}.run(), 1.second) should ===(Seq(1, 2, 3))
|
||||
ClosedShape
|
||||
}).run(), 1.second) should ===(Seq(1, 2, 3))
|
||||
}
|
||||
|
||||
"fail towards already full UniformFanInShape" in {
|
||||
Await.result(FlowGraph.closed(sink) { implicit b ⇒
|
||||
Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b ⇒
|
||||
s ⇒
|
||||
val f: UniformFanInShape[Int, Int] = b.add(Merge[Int](1))
|
||||
val src = b.add(source)
|
||||
src ~> f
|
||||
(the[IllegalArgumentException] thrownBy (s <~ f <~ src)).getMessage should include("no more inlets free")
|
||||
}.run(), 1.second) should ===(Seq(1, 2, 3))
|
||||
ClosedShape
|
||||
}).run(), 1.second) should ===(Seq(1, 2, 3))
|
||||
}
|
||||
|
||||
"work towards UniformFanOutShape" in {
|
||||
Await.result(FlowGraph.closed(sink) { implicit b ⇒
|
||||
Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b ⇒
|
||||
s ⇒
|
||||
val f: UniformFanOutShape[Int, Int] = b.add(Broadcast[Int](1))
|
||||
s <~ f
|
||||
source ~> f
|
||||
}.run(), 1.second) should ===(Seq(1, 2, 3))
|
||||
ClosedShape
|
||||
}).run(), 1.second) should ===(Seq(1, 2, 3))
|
||||
}
|
||||
|
||||
"fail towards already full UniformFanOutShape" in {
|
||||
Await.result(FlowGraph.closed(sink) { implicit b ⇒
|
||||
Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b ⇒
|
||||
s ⇒
|
||||
val f: UniformFanOutShape[Int, Int] = b.add(Broadcast[Int](1))
|
||||
val src = b.add(source)
|
||||
src ~> f
|
||||
(the[IllegalArgumentException] thrownBy (s <~ f <~ src)).getMessage should include("already connected")
|
||||
}.run(), 1.second) should ===(Seq(1, 2, 3))
|
||||
ClosedShape
|
||||
}).run(), 1.second) should ===(Seq(1, 2, 3))
|
||||
}
|
||||
|
||||
"work across a Flow" in {
|
||||
Await.result(FlowGraph.closed(sink) { implicit b ⇒
|
||||
Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b ⇒
|
||||
s ⇒
|
||||
s <~ Flow[Int] <~ source
|
||||
}.run(), 1.second) should ===(Seq(1, 2, 3))
|
||||
ClosedShape
|
||||
}).run(), 1.second) should ===(Seq(1, 2, 3))
|
||||
}
|
||||
|
||||
"work across a FlowShape" in {
|
||||
Await.result(FlowGraph.closed(sink) { implicit b ⇒
|
||||
Await.result(RunnableGraph.fromGraph(FlowGraph.create(sink) { implicit b ⇒
|
||||
s ⇒
|
||||
s <~ b.add(Flow[Int]) <~ source
|
||||
}.run(), 1.second) should ===(Seq(1, 2, 3))
|
||||
ClosedShape
|
||||
}).run(), 1.second) should ===(Seq(1, 2, 3))
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -3,7 +3,7 @@
|
|||
*/
|
||||
package akka.stream.scaladsl
|
||||
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.stream.{ SinkShape, ActorMaterializer }
|
||||
import akka.stream.testkit.TestPublisher.ManualProbe
|
||||
import akka.stream.testkit._
|
||||
|
||||
|
|
@ -17,11 +17,11 @@ class SinkSpec extends AkkaSpec {
|
|||
|
||||
"be composable without importing modules" in {
|
||||
val probes = Array.fill(3)(TestSubscriber.manualProbe[Int])
|
||||
val sink = Sink() { implicit b ⇒
|
||||
val sink = Sink.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
val bcast = b.add(Broadcast[Int](3))
|
||||
for (i ← 0 to 2) bcast.out(i).filter(_ == i) ~> Sink(probes(i))
|
||||
bcast.in
|
||||
}
|
||||
SinkShape(bcast.in)
|
||||
})
|
||||
Source(List(0, 1, 2)).runWith(sink)
|
||||
for (i ← 0 to 2) {
|
||||
val p = probes(i)
|
||||
|
|
@ -34,13 +34,13 @@ class SinkSpec extends AkkaSpec {
|
|||
|
||||
"be composable with importing 1 module" in {
|
||||
val probes = Array.fill(3)(TestSubscriber.manualProbe[Int])
|
||||
val sink = Sink(Sink(probes(0))) { implicit b ⇒
|
||||
val sink = Sink.fromGraph(FlowGraph.create(Sink(probes(0))) { implicit b ⇒
|
||||
s0 ⇒
|
||||
val bcast = b.add(Broadcast[Int](3))
|
||||
bcast.out(0) ~> Flow[Int].filter(_ == 0) ~> s0.inlet
|
||||
for (i ← 1 to 2) bcast.out(i).filter(_ == i) ~> Sink(probes(i))
|
||||
bcast.in
|
||||
}
|
||||
SinkShape(bcast.in)
|
||||
})
|
||||
Source(List(0, 1, 2)).runWith(sink)
|
||||
for (i ← 0 to 2) {
|
||||
val p = probes(i)
|
||||
|
|
@ -53,14 +53,14 @@ class SinkSpec extends AkkaSpec {
|
|||
|
||||
"be composable with importing 2 modules" in {
|
||||
val probes = Array.fill(3)(TestSubscriber.manualProbe[Int])
|
||||
val sink = Sink(Sink(probes(0)), Sink(probes(1)))(List(_, _)) { implicit b ⇒
|
||||
val sink = Sink.fromGraph(FlowGraph.create(Sink(probes(0)), Sink(probes(1)))(List(_, _)) { implicit b ⇒
|
||||
(s0, s1) ⇒
|
||||
val bcast = b.add(Broadcast[Int](3))
|
||||
bcast.out(0).filter(_ == 0) ~> s0.inlet
|
||||
bcast.out(1).filter(_ == 1) ~> s1.inlet
|
||||
bcast.out(2).filter(_ == 2) ~> Sink(probes(2))
|
||||
bcast.in
|
||||
}
|
||||
SinkShape(bcast.in)
|
||||
})
|
||||
Source(List(0, 1, 2)).runWith(sink)
|
||||
for (i ← 0 to 2) {
|
||||
val p = probes(i)
|
||||
|
|
@ -73,14 +73,14 @@ class SinkSpec extends AkkaSpec {
|
|||
|
||||
"be composable with importing 3 modules" in {
|
||||
val probes = Array.fill(3)(TestSubscriber.manualProbe[Int])
|
||||
val sink = Sink(Sink(probes(0)), Sink(probes(1)), Sink(probes(2)))(List(_, _, _)) { implicit b ⇒
|
||||
val sink = Sink.fromGraph(FlowGraph.create(Sink(probes(0)), Sink(probes(1)), Sink(probes(2)))(List(_, _, _)) { implicit b ⇒
|
||||
(s0, s1, s2) ⇒
|
||||
val bcast = b.add(Broadcast[Int](3))
|
||||
bcast.out(0).filter(_ == 0) ~> s0.inlet
|
||||
bcast.out(1).filter(_ == 1) ~> s1.inlet
|
||||
bcast.out(2).filter(_ == 2) ~> s2.inlet
|
||||
bcast.in
|
||||
}
|
||||
SinkShape(bcast.in)
|
||||
})
|
||||
Source(List(0, 1, 2)).runWith(sink)
|
||||
for (i ← 0 to 2) {
|
||||
val p = probes(i)
|
||||
|
|
|
|||
|
|
@ -7,10 +7,9 @@ import scala.concurrent.Await
|
|||
import scala.concurrent.duration._
|
||||
import scala.util.{ Success, Failure }
|
||||
import scala.util.control.NoStackTrace
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.stream.{ SourceShape, ActorMaterializer }
|
||||
import akka.stream.testkit._
|
||||
import akka.stream.impl.PublisherSource
|
||||
import akka.stream.impl.ReactiveStreamsCompliance
|
||||
import akka.stream.impl.{ PublisherSource, ReactiveStreamsCompliance }
|
||||
|
||||
class SourceSpec extends AkkaSpec {
|
||||
|
||||
|
|
@ -73,14 +72,14 @@ class SourceSpec extends AkkaSpec {
|
|||
}
|
||||
}
|
||||
|
||||
"Lazy Empty Source" must {
|
||||
"complete materialized future when stream cancels" in {
|
||||
val neverSource = Source.lazyEmpty
|
||||
val pubSink = Sink.publisher
|
||||
"Maybe Source" must {
|
||||
"complete materialized future with None when stream cancels" in {
|
||||
val neverSource = Source.maybe[Int]
|
||||
val pubSink = Sink.publisher[Int]
|
||||
|
||||
val (f, neverPub) = neverSource.toMat(pubSink)(Keep.both).run()
|
||||
|
||||
val c = TestSubscriber.manualProbe()
|
||||
val c = TestSubscriber.manualProbe[Int]()
|
||||
neverPub.subscribe(c)
|
||||
val subs = c.expectSubscription()
|
||||
|
||||
|
|
@ -88,24 +87,35 @@ class SourceSpec extends AkkaSpec {
|
|||
c.expectNoMsg(300.millis)
|
||||
|
||||
subs.cancel()
|
||||
Await.result(f.future, 500.millis)
|
||||
Await.result(f.future, 500.millis) shouldEqual None
|
||||
}
|
||||
|
||||
"allow external triggering of completion" in {
|
||||
val neverSource = Source.lazyEmpty[Int]
|
||||
"allow external triggering of empty completion" in {
|
||||
val neverSource = Source.maybe[Int].filter(_ ⇒ false)
|
||||
val counterSink = Sink.fold[Int, Int](0) { (acc, _) ⇒ acc + 1 }
|
||||
|
||||
val (neverPromise, counterFuture) = neverSource.toMat(counterSink)(Keep.both).run()
|
||||
|
||||
// external cancellation
|
||||
neverPromise.success(())
|
||||
neverPromise.trySuccess(None) shouldEqual true
|
||||
|
||||
val ready = Await.ready(counterFuture, 500.millis)
|
||||
val Success(0) = ready.value.get
|
||||
Await.result(counterFuture, 500.millis) shouldEqual 0
|
||||
}
|
||||
|
||||
"allow external triggering of non-empty completion" in {
|
||||
val neverSource = Source.maybe[Int]
|
||||
val counterSink = Sink.head[Int]
|
||||
|
||||
val (neverPromise, counterFuture) = neverSource.toMat(counterSink)(Keep.both).run()
|
||||
|
||||
// external cancellation
|
||||
neverPromise.trySuccess(Some(6)) shouldEqual true
|
||||
|
||||
Await.result(counterFuture, 500.millis) shouldEqual 6
|
||||
}
|
||||
|
||||
"allow external triggering of onError" in {
|
||||
val neverSource = Source.lazyEmpty
|
||||
val neverSource = Source.maybe[Int]
|
||||
val counterSink = Sink.fold[Int, Int](0) { (acc, _) ⇒ acc + 1 }
|
||||
|
||||
val (neverPromise, counterFuture) = neverSource.toMat(counterSink)(Keep.both).run()
|
||||
|
|
@ -126,7 +136,7 @@ class SourceSpec extends AkkaSpec {
|
|||
val source = Source.subscriber[Int]
|
||||
val out = TestSubscriber.manualProbe[Int]
|
||||
|
||||
val s = Source(source, source, source, source, source)(Seq(_, _, _, _, _)) { implicit b ⇒
|
||||
val s = Source.fromGraph(FlowGraph.create(source, source, source, source, source)(Seq(_, _, _, _, _)) { implicit b ⇒
|
||||
(i0, i1, i2, i3, i4) ⇒
|
||||
import FlowGraph.Implicits._
|
||||
val m = b.add(Merge[Int](5))
|
||||
|
|
@ -135,8 +145,8 @@ class SourceSpec extends AkkaSpec {
|
|||
i2.outlet ~> m.in(2)
|
||||
i3.outlet ~> m.in(3)
|
||||
i4.outlet ~> m.in(4)
|
||||
m.out
|
||||
}.to(Sink(out)).run()
|
||||
SourceShape(m.out)
|
||||
}).to(Sink(out)).run()
|
||||
|
||||
for (i ← 0 to 4) probes(i).subscribe(s(i))
|
||||
val sub = out.expectSubscription()
|
||||
|
|
|
|||
|
|
@ -7,10 +7,9 @@ import akka.actor.Cancellable
|
|||
|
||||
import scala.concurrent.duration._
|
||||
import scala.util.control.NoStackTrace
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.stream.{ ClosedShape, ActorMaterializer, ActorMaterializerSettings }
|
||||
import akka.stream.testkit._
|
||||
import akka.stream.testkit.Utils._
|
||||
import akka.stream.ActorMaterializerSettings
|
||||
|
||||
class TickSourceSpec extends AkkaSpec {
|
||||
|
||||
|
|
@ -68,13 +67,14 @@ class TickSourceSpec extends AkkaSpec {
|
|||
"be usable with zip for a simple form of rate limiting" in {
|
||||
val c = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
FlowGraph.closed() { implicit b ⇒
|
||||
RunnableGraph.fromGraph(FlowGraph.create() { implicit b ⇒
|
||||
import FlowGraph.Implicits._
|
||||
val zip = b.add(Zip[Int, String]())
|
||||
Source(1 to 100) ~> zip.in0
|
||||
Source(1.second, 1.second, "tick") ~> zip.in1
|
||||
zip.out ~> Flow[(Int, String)].map { case (n, _) ⇒ n } ~> Sink(c)
|
||||
}.run()
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
val sub = c.expectSubscription()
|
||||
sub.request(1000)
|
||||
|
|
|
|||
|
|
@ -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#]) })#
|
||||
|
||||
]
|
||||
|
||||
}
|
||||
|
|
@ -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#]) })#
|
||||
|
||||
]
|
||||
|
||||
}
|
||||
|
|
@ -7,91 +7,36 @@ import akka.stream.scaladsl
|
|||
import akka.stream.{ Inlet, Shape, Graph }
|
||||
import akka.japi.function
|
||||
|
||||
trait GraphCreate {
|
||||
|
||||
import language.implicitConversions
|
||||
private implicit def r[M](run: scaladsl.RunnableGraph[M]): RunnableGraph[M] = new RunnableGraphAdapter(run)
|
||||
|
||||
private[stream] abstract class GraphCreate {
|
||||
/**
|
||||
* Creates a new fully connected graph by passing a [[FlowGraph.Builder]] to the given create function.
|
||||
*
|
||||
* The created graph must have all ports connected, otherwise an [[IllegalArgumentException]] is thrown.
|
||||
* Creates a new [[Graph]] of the given [[Shape]] by passing a [[FlowGraph.Builder]] to the given create function.
|
||||
*/
|
||||
@throws(classOf[IllegalArgumentException])
|
||||
def closed(block: function.Procedure[FlowGraph.Builder[Unit]]): RunnableGraph[Unit] =
|
||||
scaladsl.FlowGraph.closed() { b ⇒ block.apply(b.asJava) }
|
||||
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 [[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]]
|
||||
* 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 closed[S1 <: Shape, M](g1: Graph[S1, M],
|
||||
block: function.Procedure2[FlowGraph.Builder[M], S1]): RunnableGraph[M] =
|
||||
scaladsl.FlowGraph.closed(g1) { b ⇒ s => block.apply(b.asJava, s) }
|
||||
|
||||
/**
|
||||
* Creates a new [[Graph]] by importing the given graph `g1` (using `builder.graph()`) and passing its resulting [[Shape]]
|
||||
* along with the[[FlowGraph.Builder]] to the given create function.
|
||||
*
|
||||
* Partial graphs are allowed to have unconnected ports.
|
||||
*/
|
||||
def partial[S1 <: Shape, S <: Shape, M](g1: Graph[S1, M],
|
||||
def create[S1 <: Shape, S <: Shape, M](g1: Graph[S1, M],
|
||||
block: function.Function2[FlowGraph.Builder[M], S1, S]): Graph[S, M] =
|
||||
scaladsl.FlowGraph.partial(g1) { b ⇒ s => block.apply(b.asJava, s) }
|
||||
scaladsl.FlowGraph.create(g1) { b ⇒ s => block.apply(b.asJava, s) }
|
||||
|
||||
/**
|
||||
* Creates a new fully connected graph by importing the given graphs (using `builder.graph()`) and passing their resulting [[Shape]]s
|
||||
* Creates a new [[Graph]] by importing the given graphs and passing their [[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.
|
||||
*/
|
||||
@throws(classOf[IllegalArgumentException])
|
||||
def closed[S1 <: Shape, S2 <: Shape, M1, M2, M](g1: Graph[S1, M1], g2: Graph[S2, M2], combineMat: function.Function2[M1, M2, M],
|
||||
block: function.Procedure3[FlowGraph.Builder[M], S1, S2]): RunnableGraph[M] =
|
||||
scaladsl.FlowGraph.closed(g1, g2)(combineMat.apply _) { b => (s1, s2) => block.apply(b.asJava, s1, s2) }
|
||||
|
||||
/**
|
||||
* Creates a new [[Graph]] by importing the given graphs (using `builder.graph()`) and passing their resulting [[Shape]]s
|
||||
* along with the[[FlowGraph.Builder]] to the given create function.
|
||||
*
|
||||
* Partial graphs are allowed to have unconnected ports.
|
||||
*/
|
||||
def partial[S1 <: Shape, S2 <: Shape, S <: Shape, M1, M2, M](g1: Graph[S1, M1], g2: Graph[S2, M2], combineMat: function.Function2[M1, M2, M],
|
||||
def create[S1 <: Shape, S2 <: Shape, S <: Shape, M1, M2, M](g1: Graph[S1, M1], g2: Graph[S2, M2], combineMat: function.Function2[M1, M2, M],
|
||||
block: function.Function3[FlowGraph.Builder[M], S1, S2, S]): Graph[S, M] =
|
||||
scaladsl.FlowGraph.partial(g1, g2)(combineMat.apply _) { b => (s1, s2) => block.apply(b.asJava, s1, s2) }
|
||||
scaladsl.FlowGraph.create(g1, g2)(combineMat.apply) { b => (s1, s2) => block.apply(b.asJava, s1, s2) }
|
||||
|
||||
[3..21#/**
|
||||
* Creates a new fully connected graph by importing the given graphs (using `builder.graph()`) and passing their resulting [[Shape]]s
|
||||
* Creates a new [[Graph]] by importing the given graphs and passing their [[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.
|
||||
*/
|
||||
@throws(classOf[IllegalArgumentException])
|
||||
def closed1[[#S1 <: Shape#], [#M1#], M]([#g1: Graph[S1, M1]#], combineMat: function.Function1[[#M1#], M],
|
||||
block: function.Procedure2[FlowGraph.Builder[M], [#S1#]]): RunnableGraph[M] =
|
||||
scaladsl.FlowGraph.closed([#g1#])(combineMat.apply _) { b => ([#s1#]) => block.apply(b.asJava, [#s1#]) }
|
||||
|
||||
/**
|
||||
* Creates a new [[Graph]] by importing the given graphs (using `builder.graph()`) and passing their resulting [[Shape]]s
|
||||
* along with the[[FlowGraph.Builder]] to the given create function.
|
||||
*
|
||||
* Partial graphs are allowed to have unconnected ports.
|
||||
*/
|
||||
def partial1[[#S1 <: Shape#], S <: Shape, [#M1#], M]([#g1: Graph[S1, M1]#], combineMat: function.Function1[[#M1#], M],
|
||||
def create1[[#S1 <: Shape#], S <: Shape, [#M1#], M]([#g1: Graph[S1, M1]#], combineMat: function.Function1[[#M1#], M],
|
||||
block: function.Function2[FlowGraph.Builder[M], [#S1#], S]): Graph[S, M] =
|
||||
scaladsl.FlowGraph.partial([#g1#])(combineMat.apply _) { b => ([#s1#]) => block.apply(b.asJava, [#s1#]) }#
|
||||
scaladsl.FlowGraph.create([#g1#])(combineMat.apply) { b => ([#s1#]) => block.apply(b.asJava, [#s1#]) }#
|
||||
|
||||
]
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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#]) })#
|
||||
|
||||
]
|
||||
|
||||
}
|
||||
|
|
@ -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#]) })#
|
||||
|
||||
]
|
||||
|
||||
}
|
||||
|
|
@ -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)
|
||||
}#
|
||||
|
||||
]
|
||||
|
||||
}
|
||||
|
|
@ -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)
|
||||
}#
|
||||
|
||||
]
|
||||
|
||||
}
|
||||
|
|
@ -8,24 +8,10 @@ import akka.stream.impl.StreamLayout.Module
|
|||
import akka.stream.{ Graph, Attributes, Shape }
|
||||
|
||||
trait GraphApply {
|
||||
|
||||
/**
|
||||
* Creates a new fully connected graph by passing a [[FlowGraph.Builder]] to the given create function.
|
||||
*
|
||||
* The created graph must have all ports connected, otherwise an [[IllegalArgumentException]] is thrown.
|
||||
*/
|
||||
def closed()(buildBlock: (FlowGraph.Builder[Unit]) ⇒ Unit): RunnableGraph[Unit] = {
|
||||
val builder = new FlowGraph.Builder
|
||||
buildBlock(builder)
|
||||
builder.buildRunnable()
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new [[Graph]] by passing a [[FlowGraph.Builder]] to the given create function.
|
||||
*
|
||||
* Partial graphs are allowed to have unconnected ports.
|
||||
*/
|
||||
def partial[S <: Shape]()(buildBlock: FlowGraph.Builder[Unit] ⇒ S): Graph[S, Unit] = {
|
||||
def create[S <: Shape]()(buildBlock: FlowGraph.Builder[Unit] ⇒ S): Graph[S, Unit] = {
|
||||
val builder = new FlowGraph.Builder
|
||||
val s = buildBlock(builder)
|
||||
val mod = builder.module.nest().replaceShape(s)
|
||||
|
|
@ -34,25 +20,10 @@ trait GraphApply {
|
|||
}
|
||||
|
||||
/**
|
||||
* Creates a new fully connected graph by importing the given graph `g1` (using `builder.graph()`) and passing its resulting [[Shape]]
|
||||
* Creates a new [[Graph]] by importing the given graph `g1` and passing 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.
|
||||
*/
|
||||
def closed[Mat](g1: Graph[Shape, Mat])(buildBlock: FlowGraph.Builder[Mat] ⇒ (g1.Shape) ⇒ Unit): RunnableGraph[Mat] = {
|
||||
val builder = new FlowGraph.Builder
|
||||
val p1 = builder.add(g1)
|
||||
buildBlock(builder)(p1)
|
||||
builder.buildRunnable()
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new [[Graph]] by importing the given graph `g1` (using `builder.graph()`) and passing its resulting [[Shape]]
|
||||
* along with the[[FlowGraph.Builder]] to the given create function.
|
||||
*
|
||||
* Partial graphs are allowed to have unconnected ports.
|
||||
*/
|
||||
def partial[S <: Shape, Mat](g1: Graph[Shape, Mat])(buildBlock: FlowGraph.Builder[Mat] ⇒ (g1.Shape) ⇒ S): Graph[S, Mat] = {
|
||||
def create[S <: Shape, Mat](g1: Graph[Shape, Mat])(buildBlock: FlowGraph.Builder[Mat] ⇒ (g1.Shape) ⇒ S): Graph[S, Mat] = {
|
||||
val builder = new FlowGraph.Builder
|
||||
val s1 = builder.add(g1)
|
||||
val s = buildBlock(builder)(s1)
|
||||
|
|
@ -62,29 +33,13 @@ trait GraphApply {
|
|||
}
|
||||
|
||||
|
||||
[2..#/**
|
||||
* Creates a new fully connected graph by importing the given graphs (using `builder.graph()`) and passing their resulting [[Shape]]s
|
||||
* along with the[[FlowGraph.Builder]] to the given create function.
|
||||
*
|
||||
* The created graph must have all ports connected, otherwise an [[IllegalArgumentException]] is thrown.
|
||||
*/
|
||||
def closed[Mat, [#M1#]]([#g1: Graph[Shape, M1]#])(combineMat: ([#M1#]) ⇒ Mat)(buildBlock: FlowGraph.Builder[Mat] ⇒ ([#g1.Shape#]) ⇒ Unit): RunnableGraph[Mat] = {
|
||||
val builder = new FlowGraph.Builder
|
||||
val curried = combineMat.curried
|
||||
val s##1 = builder.add(g##1, (m##1: M##1) ⇒ curried(m##1))
|
||||
[2..#val s1 = builder.add(g1, (f: M1 ⇒ Any, m1: M1) ⇒ f(m1))#
|
||||
]
|
||||
buildBlock(builder)([#s1#])
|
||||
builder.buildRunnable()
|
||||
}
|
||||
[2..#
|
||||
|
||||
/**
|
||||
* Creates a new [[Graph]] by importing the given graphs (using `builder.graph()`) and passing their resulting [[Shape]]s
|
||||
* Creates a new [[Graph]] by importing the given graphs and passing their [[Shape]]s
|
||||
* 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 curried = combineMat.curried
|
||||
val s##1 = builder.add(g##1, (m##1: M##1) ⇒ curried(m##1))
|
||||
|
|
@ -97,15 +52,13 @@ trait GraphApply {
|
|||
}#
|
||||
|
||||
]
|
||||
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[stream] object GraphApply {
|
||||
class GraphImpl[S <: Shape, Mat](override val shape: S, private[stream] override val module: StreamLayout.Module)
|
||||
final class GraphImpl[S <: Shape, Mat](override val shape: S, private[stream] override val module: StreamLayout.Module)
|
||||
extends Graph[S, Mat] {
|
||||
|
||||
override def withAttributes(attr: Attributes): Graph[S, Mat] =
|
||||
|
|
@ -113,5 +66,4 @@ private[stream] object GraphApply {
|
|||
|
||||
override def named(name: String): Graph[S, Mat] = withAttributes(Attributes.name(name))
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}#
|
||||
|
||||
]
|
||||
|
||||
}
|
||||
|
|
@ -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)
|
||||
}#
|
||||
|
||||
]
|
||||
|
||||
}
|
||||
|
|
@ -177,7 +177,7 @@ object ClosedShape extends ClosedShape {
|
|||
/**
|
||||
* 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
Loading…
Add table
Add a link
Reference in a new issue