!str #16902: Unify stream internal representation
also =str #16912: Fix StreamTcpSpec flakiness
This commit is contained in:
parent
cac9c9f2fb
commit
8d77fa8b29
230 changed files with 7814 additions and 9596 deletions
|
|
@ -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()
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue