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

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

View file

@ -20,7 +20,7 @@ object MaterializationBenchmark {
}
val graphWithJunctionsBuilder = (numOfJunctions: Int) =>
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)

View file

@ -126,8 +126,8 @@ As a first example, let's look at a more complex layout:
The diagram shows a :class:`RunnableGraph` (remember, if there are no unwired ports, the graph is closed, and therefore
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,

View file

@ -85,8 +85,8 @@ Constructing and combining Partial Flow Graphs
Sometimes it is not possible (or needed) to construct the entire computation graph in one place, but instead construct
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

View file

@ -119,7 +119,7 @@ detail in :ref:`constructing-sources-sinks-flows-from-partial-graphs-java`. Flow
.. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/TwitterStreamQuickstartDocTest.java#flow-graph-broadcast
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`.

View file

@ -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 {

View file

@ -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

View file

@ -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
}

View file

@ -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
}

View file

@ -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

View file

@ -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
}

View file

@ -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
}

View file

@ -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

View file

@ -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

View file

@ -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()

View file

@ -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)

View file

@ -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()

View file

@ -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()

View file

@ -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))

View file

@ -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)
}

View file

@ -126,8 +126,8 @@ As a first example, let's look at a more complex layout:
The diagram shows a :class:`RunnableGraph` (remember, if there are no unwired ports, the graph is closed, and therefore
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,

View file

@ -60,9 +60,6 @@ will be inferred.
Notice the ``import FlowGraph.Implicits._`` which brings into scope the ``~>`` operator (read as "edge", "via" or "to")
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.

View file

@ -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
}

View file

@ -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))

View file

@ -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)
})
}

View file

@ -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 {

View file

@ -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) =

View file

@ -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

View file

@ -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

View file

@ -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]] {

View file

@ -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)
}

View file

@ -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)] {

View file

@ -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]

View file

@ -31,7 +31,7 @@ class IncomingConnection private[http] (delegate: akka.http.scaladsl.Http.Incomi
*
* Use `Flow.join` or one of the handleXXX methods to consume handle requests on this connection.
*/
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

View file

@ -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
}

View file

@ -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 {
/**

View file

@ -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)
}

View file

@ -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(

View file

@ -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)

View file

@ -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))

View file

@ -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
}

View file

@ -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
}

View file

@ -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 {

View file

@ -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))

View file

@ -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("")(_ + _))
}
}

View file

@ -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)
}
}

View file

@ -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)

View file

@ -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]()

View file

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

View file

@ -1,18 +0,0 @@
/**
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.stream.tck
import org.reactivestreams.Publisher
import org.reactivestreams.Subscriber
import akka.stream.scaladsl.Source
import akka.stream.scaladsl.Sink
class LazyEmptySourceTest extends AkkaPublisherVerification[Int] {
def createPublisher(elements: Long): Publisher[Int] =
Source.lazyEmpty[Int].runWith(Sink.publisher)
override def maxElementsFromPublisher(): Long = 0
}

View file

@ -0,0 +1,20 @@
/**
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.stream.tck
import org.reactivestreams.Publisher
import org.reactivestreams.Subscriber
import akka.stream.scaladsl.{ Keep, Source, Sink }
class MaybeSourceTest extends AkkaPublisherVerification[Int] {
def createPublisher(elements: Long): Publisher[Int] = {
val (p, pub) = Source.maybe[Int].toMat(Sink.publisher)(Keep.both).run()
p success Some(1)
pub
}
override def maxElementsFromPublisher(): Long = 1
}

View file

@ -1,6 +1,6 @@
package akka.stream.testkit
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
}

View file

@ -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();

View file

@ -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);

View file

@ -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());

View file

@ -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);
}
});

View file

@ -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()

View file

@ -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))

View file

@ -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 ! _

View file

@ -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)

View file

@ -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)

View file

@ -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

View file

@ -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")

View file

@ -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)

View file

@ -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)

View file

@ -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()
}
}
}

View file

@ -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)

View file

@ -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))
}

View file

@ -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()

View file

@ -41,8 +41,8 @@ class FlowTakeSpec extends AkkaSpec with ScriptedTest {
}
"complete eagerly when zero or less is taken independently of upstream completion" in {
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)
}
}

View file

@ -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

View file

@ -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()

View file

@ -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()

View file

@ -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)

View file

@ -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()

View file

@ -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()

View file

@ -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)

View file

@ -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)

View file

@ -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")
}

View file

@ -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

View file

@ -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)

View file

@ -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)

View file

@ -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()

View file

@ -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()

View file

@ -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))
}
}

View file

@ -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)(_ + _)

View file

@ -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))
}
}

View file

@ -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)

View file

@ -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()

View file

@ -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)

View file

@ -1,48 +0,0 @@
/**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.stream.javadsl
import akka.stream.scaladsl
import akka.stream.{ Inlet, Outlet, Shape, Graph, BidiShape }
import akka.japi.Pair
import akka.japi.function
trait BidiFlowCreate {
import language.implicitConversions
private implicit def p[A, B](pair: Pair[A, B]): (A, B) = pair.first -> pair.second
/**
* Creates a BidiFlow by applying a [[FlowGraph.Builder]] to the given create function.
*/
def create[I1, O1, I2, O2](block: function.Function[FlowGraph.Builder[Unit], BidiShape[I1, O1, I2, O2]]): BidiFlow[I1, O1, I2, O2, Unit] =
new BidiFlow(scaladsl.BidiFlow() { b ⇒ block.apply(b.asJava) })
/**
* Creates a BidiFlow by applying a [[FlowGraph.Builder]] to the given create function.
* The given graph will be imported (using `builder.graph()`) and the resulting [[Shape]] will be passed to the create function along with the builder.
*/
def create[I1, O1, I2, O2, S <: Shape, M](g1: Graph[S, M],
block: function.Function2[FlowGraph.Builder[M], S, BidiShape[I1, O1, I2, O2]]): BidiFlow[I1, O1, I2, O2, M] =
new BidiFlow(scaladsl.BidiFlow(g1) { b ⇒ s => block.apply(b.asJava, s) })
/**
* Creates a BidiFlow by applying a [[FlowGraph.Builder]] to the given create function.
* The given graphs will be imported (using `builder.graph()`) and the resulting [[Shape]]s will be passed to the create function along with the builder.
*/
def create[I1, O1, I2, O2, S1 <: Shape, S2 <: Shape, M1, M2, M](g1: Graph[S1, M1], g2: Graph[S2, M2], combineMat: function.Function2[M1, M2, M],
block: function.Function3[FlowGraph.Builder[M], S1, S2, BidiShape[I1, O1, I2, O2]]): BidiFlow[I1, O1, I2, O2, M] =
new BidiFlow(scaladsl.BidiFlow(g1, g2)(combineMat.apply _) { b => (s1, s2) => block.apply(b.asJava, s1, s2) })
[3..21#/**
* Creates a BidiFlow by applying a [[FlowGraph.Builder]] to the given create function.
* The given graphs will be imported (using `builder.graph()`) and the resulting [[Shape]]s will be passed to the create function along with the builder.
*/
def create1[I##1, O##1, I##2, O##2, [#S1 <: Shape#], [#M1#], M]([#g1: Graph[S1, M1]#], combineMat: function.Function1[[#M1#], M],
block: function.Function2[FlowGraph.Builder[M], [#S1#], BidiShape[I##1, O##1, I##2, O##2]]): BidiFlow[I##1, O##1, I##2, O##2, M] =
new BidiFlow(scaladsl.BidiFlow([#g1#])(combineMat.apply _) { b => ([#s1#]) => block.apply(b.asJava, [#s1#]) })#
]
}

View file

@ -1,55 +0,0 @@
/**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.stream.javadsl
import akka.stream.scaladsl
import akka.stream.{ Inlet, Outlet, Shape, Graph }
import akka.japi.Pair
import akka.japi.function
trait FlowCreate {
import language.implicitConversions
private implicit def p[A, B](pair: Pair[A, B]): (A, B) = pair.first -> pair.second
/**
* Creates a Flow by passing a [[FlowGraph.Builder]] to the given create function.
*
* The create function is expected to return a pair of [[Inlet]] and [[Outlet]] which correspond to the created Flows input and output ports.
*/
def create[I, O](block: function.Function[FlowGraph.Builder[Unit], Inlet[I] Pair Outlet[O]]): Flow[I, O, Unit] =
new Flow(scaladsl.Flow() { b ⇒ block.apply(b.asJava) })
/**
* Creates a Flow by passing a [[FlowGraph.Builder]] to the given create function.
* The given graph will be imported (using `builder.graph()`) and the resulting shape will be passed to the create function along with the builder.
*
* The create function is expected to return a pair of [[Inlet]] and [[Outlet]] which correspond to the created Flows input and output ports.
*/
def create[I, O, S <: Shape, M](g1: Graph[S, M], block: function.Function2[FlowGraph.Builder[M], S, Inlet[I] Pair Outlet[O]]): Flow[I, O, M] =
new Flow(scaladsl.Flow(g1) { b ⇒ s => block.apply(b.asJava, s) })
/**
* Creates a Flow by passing a [[FlowGraph.Builder]] to the given create function.
* The given graph will be imported (using `builder.graph()`) and the resulting shape will be passed to the create function along with the builder.
*
* The create function is expected to return a pair of [[Inlet]] and [[Outlet]] which correspond to the created Flows input and output ports.
*/
def create[I, O, S1 <: Shape, S2 <: Shape, M1, M2, M](g1: Graph[S1, M1], g2: Graph[S2, M2], combineMat: function.Function2[M1, M2, M],
block: function.Function3[FlowGraph.Builder[M], S1, S2, Inlet[I] Pair Outlet[O]]): Flow[I, O, M] =
new Flow(scaladsl.Flow(g1, g2)(combineMat.apply _) { b => (s1, s2) => block.apply(b.asJava, s1, s2) })
[3..21#/**
* Creates a Flow by passing a [[FlowGraph.Builder]] to the given create function.
* The given graph will be imported (using `builder.graph()`) and the resulting shape will be passed to the create function along with the builder.
*
* The create function is expected to return a pair of [[Inlet]] and [[Outlet]] which correspond to the created Flows input and output ports.
*/
def create1[I, O, [#S1 <: Shape#], [#M1#], M]([#g1: Graph[S1, M1]#], combineMat: function.Function1[[#M1#], M],
block: function.Function2[FlowGraph.Builder[M], [#S1#], Inlet[I] Pair Outlet[O]]): Flow[I, O, M] =
new Flow(scaladsl.Flow([#g1#])(combineMat.apply _) { b => ([#s1#]) => block.apply(b.asJava, [#s1#]) })#
]
}

View file

@ -7,91 +7,36 @@ import akka.stream.scaladsl
import akka.stream.{ Inlet, Shape, Graph }
import akka.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#]) }#
]
}

View file

@ -1,48 +0,0 @@
/**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.stream.javadsl
import akka.stream.scaladsl
import akka.stream.{ Inlet, Shape, Graph }
import akka.japi.function
trait SinkCreate {
/**
* Creates a `Sink` by using a `FlowGraph.Builder[Unit]` on a block that expects
* a [[FlowGraph.Builder]] and returns an [[Inlet]].
*/
def create[T](block: function.Function[FlowGraph.Builder[Unit], Inlet[T]]): Sink[T, Unit] =
new Sink(scaladsl.Sink() { b ⇒ block.apply(b.asJava) })
/**
* Creates a `Sink` by importing the given graph (using `builder.graph()`) and calling the provided create function
* with the `FlowGraph.Builder[M]` and the [[Shape]] resulting from importing the graph.
* The create function is expected to return the created Sink's [[Inlet]].
*/
def create[T, S <: Shape, M](g1: Graph[S, M],
block: function.Function2[FlowGraph.Builder[M], S, Inlet[T]]): Sink[T, M] =
new Sink(scaladsl.Sink(g1) { b ⇒ s => block.apply(b.asJava, s) })
/**
* Creates a `Sink` by importing the given graphs (using `builder.graph()`) and calling the provided create function
* with the `FlowGraph.Builder[M]` and the [[Shape]]s resulting from importing the graphs.
* The create function is expected to return the created Sink's [[Inlet]].
*/
def create[T, S1 <: Shape, S2 <: Shape, M1, M2, M](g1: Graph[S1, M1], g2: Graph[S2, M2], combineMat: function.Function2[M1, M2, M],
block: function.Function3[FlowGraph.Builder[M], S1, S2, Inlet[T]]): Sink[T, M] =
new Sink(scaladsl.Sink(g1, g2)(combineMat.apply _) { b => (s1, s2) => block.apply(b.asJava, s1, s2) })
[3..21#/**
* Creates a `Sink` by importing the given graphs (using `builder.graph()`) and calling the provided create function
* with the `FlowGraph.Builder[M]` and the [[Shape]]s resulting from importing the graphs.
* The create function is expected to return the created Sink's [[Inlet]].
*/
def create1[T, [#S1 <: Shape#], [#M1#], M]([#g1: Graph[S1, M1]#], combineMat: function.Function1[[#M1#], M],
block: function.Function2[FlowGraph.Builder[M], [#S1#], Inlet[T]]): Sink[T, M] =
new Sink(scaladsl.Sink([#g1#])(combineMat.apply _) { b => ([#s1#]) => block.apply(b.asJava, [#s1#]) })#
]
}

View file

@ -1,51 +0,0 @@
/**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.stream.javadsl
import akka.stream.scaladsl
import akka.stream.{ Outlet, Shape, Graph }
import akka.japi.function
trait SourceCreate {
/**
* Creates a `Source` by using a `FlowGraph.Builder[Unit]` on a block that expects
* a [[FlowGraph.Builder]] and returns an [[Outlet]].
*/
def create[T](block: function.Function[FlowGraph.Builder[Unit], Outlet[T]]): Source[T, Unit] =
new Source(scaladsl.Source() { b ⇒ block.apply(b.asJava) })
/**
* Creates a `Source` by using a `FlowGraph.Builder[M]` on a block that expects
* a [[FlowGraph.Builder]] and 1 graph and then returns an [[Outlet]].
* The graph will be imported (using `Builder.graph()`) and the resulting shape
* will be passed into the create block.
*/
def create[T, S <: Shape, M](g1: Graph[S, M],
block: function.Function2[FlowGraph.Builder[M], S, Outlet[T]]): Source[T, M] =
new Source(scaladsl.Source(g1) { b ⇒ s => block.apply(b.asJava, s) })
/**
* Creates a `Source` by using a `FlowGraph.Builder[M]` on a block that expects
* a [[FlowGraph.Builder]] and 2 graphs and then returns an [[Outlet]].
* The graphs will be imported (using `Builder.graph()`) and the resulting shapes
* will be passed into the create block.
*/
def create[T, S1 <: Shape, S2 <: Shape, M1, M2, M](g1: Graph[S1, M1], g2: Graph[S2, M2], combineMat: function.Function2[M1, M2, M],
block: function.Function3[FlowGraph.Builder[M], S1, S2, Outlet[T]]): Source[T, M] =
new Source(scaladsl.Source(g1, g2)(combineMat.apply _) { b => (s1, s2) => block.apply(b.asJava, s1, s2) })
[3..21#/**
* Creates a `Source` by using a `FlowGraph.Builder[M]` on a block that expects
* a [[FlowGraph.Builder]] and 1 graphs and then returns an [[Outlet]].
* The graphs will be imported (using `Builder.graph()`) and the resulting shapes
* will be passed into the create block.
*/
def create1[T, [#S1 <: Shape#], [#M1#], M]([#g1: Graph[S1, M1]#], combineMat: function.Function1[[#M1#], M],
block: function.Function2[FlowGraph.Builder[M], [#S1#], Outlet[T]]): Source[T, M] =
new Source(scaladsl.Source([#g1#])(combineMat.apply _) { b => ([#s1#]) => block.apply(b.asJava, [#s1#]) })#
]
}

View file

@ -1,47 +0,0 @@
/**
* Copyright (C) 2014-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.stream.scaladsl
import akka.stream.{ Shape, Inlet, Outlet, Graph, BidiShape }
trait BidiFlowApply {
/**
* Creates a BidiFlow by applying a [[FlowGraph.Builder]] to the given create block.
*/
def apply[I1, O1, I2, O2]()(block: FlowGraph.Builder[Unit] ⇒ BidiShape[I1, O1, I2, O2]): BidiFlow[I1, O1, I2, O2, Unit] = {
val builder = new FlowGraph.Builder
val shape = block(builder)
builder.buildBidiFlow(shape)
}
/**
* Creates a BidiFlow by applying a [[FlowGraph.Builder]] to the given create block.
* The given graph will be imported (using `builder.graph()`) and the resulting [[Shape]] will be passed to the create block along with the builder.
*/
def apply[I1, O1, I2, O2, Mat](g1: Graph[Shape, Mat])(buildBlock: FlowGraph.Builder[Mat] => (g1.Shape) ⇒ BidiShape[I1, O1, I2, O2]): BidiFlow[I1, O1, I2, O2, Mat] = {
val builder = new FlowGraph.Builder
val p = builder.add(g1)
val shape = buildBlock(builder)(p)
builder.buildBidiFlow(shape)
}
[2..#/**
* Creates a BidiFlow by applying a [[FlowGraph.Builder]] to the given create block.
* The given graph will be imported (using `builder.graph()`) and the resulting [[Shape]] will be passed to the create block along with the builder.
*/
def apply[I##1, O##1, I##2, O##2, [#M1#], Mat]([#g1: Graph[Shape, M1]#])(combineMat: ([#M1#]) => Mat)(
buildBlock: FlowGraph.Builder[Mat] => ([#g1.Shape#]) ⇒ BidiShape[I##1, O##1, I##2, O##2]): BidiFlow[I##1, O##1, I##2, O##2, Mat] = {
val builder = new FlowGraph.Builder
val curried = combineMat.curried
val p##1 = builder.add(g##1, (m##1: M##1) ⇒ curried(m##1))
[2..#val p1 = builder.add(g1, (f: M1 ⇒ Any, m1: M1) ⇒ f(m1))#
]
val shape = buildBlock(builder)([#p1#])
builder.buildBidiFlow(shape)
}#
]
}

View file

@ -1,53 +0,0 @@
/**
* Copyright (C) 2014-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.stream.scaladsl
import akka.stream.{ Shape, Inlet, Outlet, Graph }
trait FlowApply {
/**
* Creates a Flow by passing a [[FlowGraph.Builder]] to the given create function.
*
* The create function is expected to return a pair of [[Inlet]] and [[Outlet]] which correspond to the created Flows input and output ports.
*/
def apply[I, O]()(block: FlowGraph.Builder[Unit] ⇒ (Inlet[I], Outlet[O])): Flow[I, O, Unit] = {
val builder = new FlowGraph.Builder
val (inlet, outlet) = block(builder)
builder.buildFlow(inlet, outlet)
}
/**
* Creates a Flow by passing a [[FlowGraph.Builder]] to the given create function.
* The given graph will be imported (using `builder.graph()`) and the resulting shape will be passed to the create function along with the builder.
*
* The create function is expected to return a pair of [[Inlet]] and [[Outlet]] which correspond to the created Flows input and output ports.
*/
def apply[I, O, Mat](g1: Graph[Shape, Mat])(buildBlock: FlowGraph.Builder[Mat] => (g1.Shape) ⇒ (Inlet[I], Outlet[O])): Flow[I, O, Mat] = {
val builder = new FlowGraph.Builder
val p = builder.add(g1)
val (inlet, outlet) = buildBlock(builder)(p)
builder.buildFlow(inlet, outlet)
}
[2..#/**
* Creates a Flow by passing a [[FlowGraph.Builder]] to the given create function.
* The given graph will be imported (using `builder.graph()`) and the resulting shape will be passed to the create function along with the builder.
*
* The create function is expected to return a pair of [[Inlet]] and [[Outlet]] which correspond to the created Flows input and output ports.
*/
def apply[I, O, [#M1#], Mat]([#g1: Graph[Shape, M1]#])(combineMat: ([#M1#]) => Mat)(
buildBlock: FlowGraph.Builder[Mat] => ([#g1.Shape#]) ⇒ (Inlet[I], Outlet[O])): Flow[I, O, Mat] = {
val builder = new FlowGraph.Builder
val curried = combineMat.curried
val p##1 = builder.add(g##1, (m##1: M##1) ⇒ curried(m##1))
[2..#val p1 = builder.add(g1, (f: M1 ⇒ Any, m1: M1) ⇒ f(m1))#
]
val (inlet, outlet) = buildBlock(builder)([#p1#])
builder.buildFlow(inlet, outlet)
}#
]
}

View file

@ -8,24 +8,10 @@ import akka.stream.impl.StreamLayout.Module
import akka.stream.{ Graph, Attributes, Shape }
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))
}
}

View file

@ -1,50 +0,0 @@
/**
* Copyright (C) 2014-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.stream.scaladsl
import akka.stream.{ Inlet, Graph, Shape }
trait SinkApply {
/**
* Creates a `Sink` by using a `FlowGraph.Builder[Unit]` on a block that expects
* a [[FlowGraph.Builder]] and returns an [[Inlet]].
*/
def apply[In]()(buildBlock: FlowGraph.Builder[Unit] => Inlet[In]): Sink[In, Unit] = {
val builder = new FlowGraph.Builder
val inlet = buildBlock(builder)
builder.buildSink(inlet)
}
/**
* Creates a `Sink` by importing the given graph (using `builder.graph()`) and calling the provided create function
* with the `FlowGraph.Builder[M]` and the [[Shape]] resulting from importing the graph.
* The create function is expected to return the created Sink's [[Inlet]].
*/
def apply[In, Mat](g1: Graph[Shape, Mat])(buildBlock: FlowGraph.Builder[Mat] => (g1.Shape) => Inlet[In]): Sink[In, Mat] = {
val builder = new FlowGraph.Builder
val s = builder.add(g1)
val inlet = buildBlock(builder)(s)
builder.buildSink(inlet)
}
[2..#/**
* Creates a `Sink` by importing the given graphs (using `builder.graph()`) and calling the provided create function
* with the `FlowGraph.Builder[M]` and the [[Shape]]s resulting from importing the graphs.
* The create function is expected to return the created Sink's [[Inlet]].
*/
def apply[In, [#M1#], Mat]([#g1: Graph[Shape, M1]#])(combineMat: ([#M1#]) ⇒ Mat)(
buildBlock: FlowGraph.Builder[Mat] ⇒ ([#g1.Shape#]) ⇒ Inlet[In]): Sink[In, Mat] = {
val builder = new FlowGraph.Builder
val curried = combineMat.curried
val s##1 = builder.add(g##1, (m##1: M##1) ⇒ curried(m##1))
[2..#val s1 = builder.add(g1, (f: M1 ⇒ Any, m1: M1) ⇒ f(m1))#
]
val inlet = buildBlock(builder)([#s1#])
builder.buildSink(inlet)
}#
]
}

View file

@ -1,52 +0,0 @@
/**
* Copyright (C) 2014-2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.stream.scaladsl
import akka.stream.{ Outlet, Shape, Graph }
trait SourceApply {
/**
* Creates a `Source` by using a `FlowGraph.Builder[Unit]` on a block that expects
* a [[FlowGraph.Builder]] and returns an [[Outlet]].
*/
def apply[Out]()(buildBlock: FlowGraph.Builder[Unit] => Outlet[Out]): Source[Out, Unit] = {
val builder = new FlowGraph.Builder
val port = buildBlock(builder)
builder.buildSource(port)
}
/**
* Creates a `Source` by using a `FlowGraph.Builder[M]` on a block that expects
* a [[FlowGraph.Builder]] and 1 graph and then returns an [[Outlet]].
* The graph will be imported (using `Builder.graph()`) and the resulting shape
* will be passed into the create block.
*/
def apply[Out, Mat](g1: Graph[Shape, Mat])(buildBlock: FlowGraph.Builder[Mat] => (g1.Shape) => Outlet[Out]): Source[Out, Mat] = {
val builder = new FlowGraph.Builder
val p = builder.add(g1)
val port = buildBlock(builder)(p)
builder.buildSource(port)
}
[2..#/**
* Creates a `Source` by using a `FlowGraph.Builder[M]` on a block that expects
* a [[FlowGraph.Builder]] and 2 graphs and then returns an [[Outlet]].
* The graphs will be imported (using `Builder.graph()`) and the resulting shapes
* will be passed into the create block.
*/
def apply[Out, [#M1#], Mat]([#g1: Graph[Shape, M1]#])(combineMat: ([#M1#]) ⇒ Mat)(
buildBlock: FlowGraph.Builder[Mat] ⇒ ([#g1.Shape#]) ⇒ Outlet[Out]): Source[Out, Mat] = {
val builder = new FlowGraph.Builder
val curried = combineMat.curried
val p##1 = builder.add(g##1, (m##1: M##1) ⇒ curried(m##1))
[2..#val p1 = builder.add(g1, (f: M1 ⇒ Any, m1: M1) ⇒ f(m1))#
]
val port = buildBlock(builder)([#p1#])
builder.buildSource(port)
}#
]
}

View file

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

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