!str #16902: Unify stream internal representation

also =str #16912: Fix StreamTcpSpec flakiness
This commit is contained in:
Endre Sándor Varga 2015-01-28 14:19:50 +01:00
parent cac9c9f2fb
commit 8d77fa8b29
230 changed files with 7814 additions and 9596 deletions

View file

@ -1,59 +1,51 @@
package akka.stream.scaladsl
import scala.concurrent.Await
import scala.collection.immutable
import scala.concurrent.{ Future, Await }
import scala.concurrent.duration._
import akka.stream.ActorFlowMaterializer
import akka.stream.ActorFlowMaterializerSettings
import akka.stream.scaladsl.FlowGraphImplicits._
import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.StreamTestKit.{ OnNext, SubscriberProbe }
import akka.util.ByteString
import akka.stream.{ Inlet, Outlet, Shape, Graph }
object GraphOpsIntegrationSpec {
import FlowGraph.Implicits._
object Lego {
def apply(pipeline: Flow[String, String]): Lego = {
val in = UndefinedSource[String]
val out = UndefinedSink[ByteString]
val graph = PartialFlowGraph { implicit builder
val balance = Balance[String]
val merge = Merge[String]
in ~> Flow[String].map(_.trim) ~> balance
balance ~> pipeline ~> merge
balance ~> pipeline ~> merge
balance ~> pipeline ~> merge
merge ~> Flow[String].map(_.trim).map(ByteString.fromString) ~> out
object Shuffle {
case class ShufflePorts[In, Out](in1: Inlet[In], in2: Inlet[In], out1: Outlet[Out], out2: Outlet[Out]) extends Shape {
override def inlets: immutable.Seq[Inlet[_]] = List(in1, in2)
override def outlets: immutable.Seq[Outlet[_]] = List(out1, out2)
override def deepCopy() = ShufflePorts(
new Inlet[In](in1.toString), new Inlet[In](in2.toString),
new Outlet[Out](out1.toString), new Outlet[Out](out2.toString))
override def copyFromPorts(inlets: immutable.Seq[Inlet[_]], outlets: immutable.Seq[Outlet[_]]) = {
assert(inlets.size == this.inlets.size)
assert(outlets.size == this.outlets.size)
ShufflePorts(inlets(0), inlets(1), outlets(0), outlets(1))
}
new Lego(in, out, graph)
}
}
class Lego private (
private val in: UndefinedSource[String],
private val out: UndefinedSink[ByteString],
private val graph: PartialFlowGraph) {
def connect(that: Lego, adapter: Flow[ByteString, String]): Lego = {
val newGraph = PartialFlowGraph { builder
builder.importPartialFlowGraph(this.graph)
builder.importPartialFlowGraph(that.graph)
builder.connect(this.out, adapter, that.in)
}
new Lego(this.in, that.out, newGraph)
}
def run(source: Source[String], sink: Sink[ByteString])(implicit materializer: ActorFlowMaterializer): Unit =
FlowGraph(graph) { builder
builder.attachSource(in, source)
builder.attachSink(out, sink)
}.run()
def apply[In, Out](pipeline: Flow[In, Out, _]): Graph[ShufflePorts[In, Out], Unit] = {
FlowGraph.partial() { implicit b
val merge = b.add(Merge[In](2))
val balance = b.add(Balance[Out](2))
merge.out ~> pipeline ~> balance.in
ShufflePorts(merge.in(0), merge.in(1), balance.out(0), balance.out(1))
}
}
}
}
class GraphOpsIntegrationSpec extends AkkaSpec {
import akka.stream.scaladsl.GraphOpsIntegrationSpec._
import FlowGraph.Implicits._
val settings = ActorFlowMaterializerSettings(system)
.withInputBuffer(initialSize = 2, maxSize = 16)
@ -63,113 +55,107 @@ class GraphOpsIntegrationSpec extends AkkaSpec {
"FlowGraphs" must {
"support broadcast - merge layouts" in {
val resultFuture = Sink.head[Seq[Int]]
val resultFuture = FlowGraph.closed(Sink.head[Seq[Int]]) { implicit b
(sink)
val bcast = b.add(Broadcast[Int](2))
val merge = b.add(Merge[Int](2))
val g = FlowGraph { implicit b
val bcast = Broadcast[Int]
val merge = Merge[Int]
Source(List(1, 2, 3)) ~> bcast
bcast ~> merge
bcast ~> Flow[Int].map(_ + 3) ~> merge
merge ~> Flow[Int].grouped(10) ~> resultFuture
Source(List(1, 2, 3)) ~> bcast.in
bcast.out(0) ~> merge.in(0)
bcast.out(1).map(_ + 3) ~> merge.in(1)
merge.out.grouped(10) ~> sink.inlet
}.run()
Await.result(g.get(resultFuture), 3.seconds).sorted should be(List(1, 2, 3, 4, 5, 6))
Await.result(resultFuture, 3.seconds).sorted should be(List(1, 2, 3, 4, 5, 6))
}
"support balance - merge (parallelization) layouts" in {
val elements = 0 to 10
val in = Source(elements)
val f = Flow[Int]
val out = Sink.head[Seq[Int]]
val out = FlowGraph.closed(Sink.head[Seq[Int]]) { implicit b
(sink)
val balance = b.add(Balance[Int](5))
val merge = b.add(Merge[Int](5))
val g = FlowGraph { implicit b
val balance = Balance[Int]
val merge = Merge[Int]
Source(elements) ~> balance.in
in ~> balance ~> f ~> merge
balance ~> f ~> merge
balance ~> f ~> merge
balance ~> f ~> merge
balance ~> f ~> merge ~> Flow[Int].grouped(elements.size * 2) ~> out
for (i 0 until 5) balance.out(i) ~> merge.in(i)
merge.out.grouped(elements.size * 2) ~> sink.inlet
}.run()
Await.result(g.get(out), 3.seconds).sorted should be(elements)
Await.result(out, 3.seconds).sorted should be(elements)
}
"support wikipedia Topological_sorting 2" in {
import OperationAttributes.name
// see https://en.wikipedia.org/wiki/Topological_sorting#mediaviewer/File:Directed_acyclic_graph.png
val resultFuture2 = Sink.head[Seq[Int]]
val resultFuture9 = Sink.head[Seq[Int]]
val resultFuture10 = Sink.head[Seq[Int]]
val seqSink = Sink.head[Seq[Int]]
val g = FlowGraph { implicit b
val b3 = Broadcast[Int](name("b3"))
val b7 = Broadcast[Int](name("b7"))
val b11 = Broadcast[Int](name("b11"))
val m8 = Merge[Int](name("m8"))
val m9 = Merge[Int](name("m9"))
val m10 = Merge[Int](name("m10"))
val m11 = Merge[Int](name("m11"))
val in3 = Source(List(3))
val in5 = Source(List(5))
val in7 = Source(List(7))
val (resultFuture2, resultFuture9, resultFuture10) = FlowGraph.closed(seqSink, seqSink, seqSink)(Tuple3.apply) { implicit b
(sink2, sink9, sink10)
val b3 = b.add(Broadcast[Int](2))
val b7 = b.add(Broadcast[Int](2))
val b11 = b.add(Broadcast[Int](3))
val m8 = b.add(Merge[Int](2))
val m9 = b.add(Merge[Int](2))
val m10 = b.add(Merge[Int](2))
val m11 = b.add(Merge[Int](2))
val in3 = Source(List(3))
val in5 = Source(List(5))
val in7 = Source(List(7))
// First layer
in7 ~> b7
b7 ~> m11
b7 ~> m8
// First layer
in7 ~> b7.in
b7.out(0) ~> m11.in(0)
b7.out(1) ~> m8.in(0)
in5 ~> m11
in5 ~> m11.in(1)
in3 ~> b3
b3 ~> m8
b3 ~> m10
in3 ~> b3.in
b3.out(0) ~> m8.in(1)
b3.out(1) ~> m10.in(0)
// Second layer
m11 ~> b11
b11 ~> Flow[Int].grouped(1000) ~> resultFuture2 // Vertex 2 is omitted since it has only one in and out
b11 ~> m9
b11 ~> m10
// Second layer
m11.out ~> b11.in
b11.out(0).grouped(1000) ~> sink2.inlet // Vertex 2 is omitted since it has only one in and out
b11.out(1) ~> m9.in(0)
b11.out(2) ~> m10.in(1)
m8 ~> m9
m8.out ~> m9.in(1)
// Third layer
m9 ~> Flow[Int].grouped(1000) ~> resultFuture9
m10 ~> Flow[Int].grouped(1000) ~> resultFuture10
// Third layer
m9.out.grouped(1000) ~> sink9.inlet
m10.out.grouped(1000) ~> sink10.inlet
}.run()
Await.result(g.get(resultFuture2), 3.seconds).sorted should be(List(5, 7))
Await.result(g.get(resultFuture9), 3.seconds).sorted should be(List(3, 5, 7, 7))
Await.result(g.get(resultFuture10), 3.seconds).sorted should be(List(3, 5, 7))
Await.result(resultFuture2, 3.seconds).sorted should be(List(5, 7))
Await.result(resultFuture9, 3.seconds).sorted should be(List(3, 5, 7, 7))
Await.result(resultFuture10, 3.seconds).sorted should be(List(3, 5, 7))
}
"allow adding of flows to sources and sinks to flows" in {
val resultFuture = Sink.head[Seq[Int]]
val g = FlowGraph { implicit b
val bcast = Broadcast[Int]
val merge = Merge[Int]
val resultFuture = FlowGraph.closed(Sink.head[Seq[Int]]) { implicit b
(sink)
val bcast = b.add(Broadcast[Int](2))
val merge = b.add(Merge[Int](2))
Source(List(1, 2, 3)) ~> Flow[Int].map(_ * 2) ~> bcast
bcast ~> merge
bcast ~> Flow[Int].map(_ + 3) ~> merge
merge ~> Flow[Int].grouped(10).to(resultFuture)
Source(List(1, 2, 3)).map(_ * 2) ~> bcast.in
bcast.out(0) ~> merge.in(0)
bcast.out(1).map(_ + 3) ~> merge.in(1)
merge.out.grouped(10) ~> sink.inlet
}.run()
Await.result(g.get(resultFuture), 3.seconds) should contain theSameElementsAs (Seq(2, 4, 6, 5, 7, 9))
Await.result(resultFuture, 3.seconds) should contain theSameElementsAs (Seq(2, 4, 6, 5, 7, 9))
}
"be able to run plain flow" in {
val p = Source(List(1, 2, 3)).runWith(Sink.publisher)
val p = Source(List(1, 2, 3)).runWith(Sink.publisher())
val s = SubscriberProbe[Int]
val flow = Flow[Int].map(_ * 2)
FlowGraph { implicit builder
import FlowGraphImplicits._
FlowGraph.closed() { implicit builder
Source(p) ~> flow ~> Sink(s)
}.run()
val sub = s.expectSubscription()
@ -180,52 +166,38 @@ class GraphOpsIntegrationSpec extends AkkaSpec {
s.expectComplete()
}
"support continued transformation from undefined source/sink" in {
val input1 = UndefinedSource[Int]
val output1 = UndefinedSink[Int]
val output2 = UndefinedSink[String]
val partial = PartialFlowGraph { implicit builder
val bcast = Broadcast[String]
input1 ~> Flow[Int].map(_.toString) ~> bcast ~> Flow[String].map(_.toInt) ~> output1
bcast ~> Flow[String].map("elem-" + _) ~> output2
}
"be possible to use as lego bricks" in {
val shuffler = Shuffle(Flow[Int].map(_ + 1))
val s1 = SubscriberProbe[Int]
val s2 = SubscriberProbe[String]
FlowGraph(partial) { builder
builder.attachSource(input1, Source(List(0, 1, 2).map(_ + 1)))
builder.attachSink(output1, Flow[Int].filter(n (n % 2) != 0).to(Sink(s1)))
builder.attachSink(output2, Flow[String].map(_.toUpperCase).to(Sink(s2)))
val f: Future[Seq[Int]] = FlowGraph.closed(shuffler, shuffler, shuffler, Sink.head[Seq[Int]])((_, _, _, fut) fut) { implicit b
(s1, s2, s3, sink)
val merge = b.add(Merge[Int](2))
Source(List(1, 2, 3)) ~> s1.in1
Source(List(10, 11, 12)) ~> s1.in2
s1.out1 ~> s2.in1
s1.out2 ~> s2.in2
s2.out1 ~> s3.in1
s2.out2 ~> s3.in2
s3.out1 ~> merge.in(0)
s3.out2 ~> merge.in(1)
merge.out.grouped(1000) ~> sink.inlet
}.run()
val sub1 = s1.expectSubscription()
val sub2 = s2.expectSubscription()
sub1.request(10)
sub2.request(10)
s1.expectNext(1)
s1.expectNext(3)
s1.expectComplete()
s2.expectNext("ELEM-1")
s2.expectNext("ELEM-2")
s2.expectNext("ELEM-3")
s2.expectComplete()
}
val result = Await.result(f, 3.seconds)
result.sorted should be(List(4, 5, 6, 13, 14, 15))
result.indexOf(4) < result.indexOf(5) should be(true)
result.indexOf(5) < result.indexOf(6) should be(true)
result.indexOf(13) < result.indexOf(14) should be(true)
result.indexOf(14) < result.indexOf(15) should be(true)
"be possible to use as lego bricks" in {
val lego1 = Lego(Flow[String].filter(_.length > 3).map(s s" $s "))
val lego2 = Lego(Flow[String].map(_.toUpperCase))
val lego3 = lego1.connect(lego2, Flow[ByteString].map(_.utf8String))
val source = Source(List("green ", "blue", "red", "yellow", "black"))
val s = SubscriberProbe[ByteString]
val sink = Sink(s)
lego3.run(source, sink)
val sub = s.expectSubscription()
sub.request(100)
val result = (s.probe.receiveN(4) collect {
case OnNext(b: ByteString) b.utf8String
}).sorted
result should be(Vector("BLACK", "BLUE", "GREEN", "YELLOW"))
s.expectComplete()
}
}