diff --git a/akka-bench-jmh/src/main/scala/akka/stream/NewLayoutBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/stream/NewLayoutBenchmark.scala new file mode 100644 index 0000000000..db786586ff --- /dev/null +++ b/akka-bench-jmh/src/main/scala/akka/stream/NewLayoutBenchmark.scala @@ -0,0 +1,359 @@ +package akka.stream + +import java.util +import java.util.concurrent.TimeUnit + +import akka.stream.impl.MaterializerSession +import akka.stream.impl.NewLayout._ +import akka.stream.impl.StreamLayout.{ AtomicModule, Module } +import org.openjdk.jmh.annotations._ +import org.reactivestreams.{ Publisher, Subscriber, Subscription } + +@State(Scope.Benchmark) +@OutputTimeUnit(TimeUnit.MILLISECONDS) +@BenchmarkMode(Array(Mode.Throughput)) +class NewLayoutBenchmark { + + // TODO: This benchmark is heavily copy-pasta. This is a temporary benchmark as these two implementations + // will never exist at the same time. This needs to be turned into a better one once the design + // settles. + + // --- These test classes do not use the optimized linear builder, for testing the composite builder instead + class CompositeTestSource extends AtomicModule { + val out = Outlet[Any]("testSourceC.out") + override val shape: Shape = SourceShape(out) + override val attributes: Attributes = Attributes.name("testSource") + val traversalBuilder = TraversalBuilder.atomic(this) + + override def withAttributes(attributes: Attributes): Module = ??? + override def carbonCopy: Module = ??? + override def replaceShape(s: Shape): Module = ??? + override def toString = "TestSource" + } + + class CompositeTestSink extends AtomicModule { + val in = Inlet[Any]("testSinkC.in") + override val shape: Shape = SinkShape(in) + override val attributes: Attributes = Attributes.name("testSink") + val traversalBuilder = TraversalBuilder.atomic(this) + + override def withAttributes(attributes: Attributes): Module = ??? + override def carbonCopy: Module = ??? + override def replaceShape(s: Shape): Module = ??? + override def toString = "TestSink" + } + + class CompositeTestFlow(tag: String) extends AtomicModule { + val in = Inlet[Any](s"testFlowC$tag.in") + val out = Outlet[Any](s"testFlowC$tag.out") + override val shape: Shape = FlowShape(in, out) + override val attributes: Attributes = Attributes.name(s"testFlow$tag") + val traversalBuilder = TraversalBuilder.atomic(this) + + override def withAttributes(attributes: Attributes): Module = ??? + override def carbonCopy: Module = ??? + override def replaceShape(s: Shape): Module = ??? + override def toString = s"TestFlow$tag" + } + + // --- These test classes DO use the optimized linear builder, for testing the composite builder instead + class LinearTestSource extends AtomicModule { + val out = Outlet[Any]("testSource.out") + override val shape: Shape = SourceShape(out) + override val attributes: Attributes = Attributes.name("testSource") + val traversalBuilder = TraversalBuilder.linear(this) + + override def withAttributes(attributes: Attributes): Module = ??? + override def carbonCopy: Module = ??? + override def replaceShape(s: Shape): Module = ??? + override def toString = "TestSource" + } + + class LinearTestSink extends AtomicModule { + val in = Inlet[Any]("testSink.in") + override val shape: Shape = SinkShape(in) + override val attributes: Attributes = Attributes.name("testSink") + val traversalBuilder = TraversalBuilder.linear(this) + + override def withAttributes(attributes: Attributes): Module = ??? + override def carbonCopy: Module = ??? + override def replaceShape(s: Shape): Module = ??? + override def toString = "TestSink" + } + + class LinearTestFlow(tag: String) extends AtomicModule { + val in = Inlet[Any](s"testFlow$tag.in") + val out = Outlet[Any](s"testFlow$tag.out") + override val shape: Shape = FlowShape(in, out) + override val attributes: Attributes = Attributes.name(s"testFlow$tag") + val traversalBuilder = TraversalBuilder.linear(this) + + override def withAttributes(attributes: Attributes): Module = ??? + override def carbonCopy: Module = ??? + override def replaceShape(s: Shape): Module = ??? + override def toString = s"TestFlow$tag" + } + + class MaterializationResult( + val connections: Int, + val inlets: Array[InPort], + val outlets: Array[OutPort] + ) { + + override def toString = { + outlets.iterator.zip(inlets.iterator).mkString("connections: ", ", ", "") + } + } + + class OldSource extends AtomicModule { + val out = Outlet[Any]("out") + override val shape = SourceShape(out) + + override def replaceShape(s: Shape): Module = ??? + + override def carbonCopy: Module = ??? + + override def attributes: Attributes = Attributes.none + override def withAttributes(attributes: Attributes): Module = this + } + + class OldSink extends AtomicModule { + val in = Inlet[Any]("in") + override val shape = SinkShape(in) + + override def replaceShape(s: Shape): Module = ??? + + override def carbonCopy: Module = ??? + + override def attributes: Attributes = Attributes.none + override def withAttributes(attributes: Attributes): Module = this + } + + class OldFlow extends AtomicModule { + val in = Inlet[Any]("in") + val out = Outlet[Any]("out") + override val shape = FlowShape(in, out) + + override def replaceShape(s: Shape): Module = ??? + + override def carbonCopy: Module = ??? + + override def attributes: Attributes = Attributes.none + override def withAttributes(attributes: Attributes): Module = this + } + + val linearSource = new LinearTestSource + val linearSink = new LinearTestSink + val linearFlow = new LinearTestFlow("linearFlow") + + val compositeSource = new CompositeTestSource + val compositeSink = new CompositeTestSink + val compositeFlow = new CompositeTestFlow("linearFlow") + + val oldSource = new OldSource + val oldSink = new OldSink + val oldFlow = new OldFlow + + def testMaterializeNew(b: TraversalBuilder): MaterializationResult = { + require(b.isTraversalComplete, "Traversal builder must be complete") + + val connections = b.inSlots + val inlets = Array.ofDim[InPort](connections) + val outlets = Array.ofDim[OutPort](connections) + + // Track next assignable number for input ports + var inOffs = 0 + + var current: Traversal = b.traversal.get + val traversalStack = new util.ArrayList[Traversal](16) + traversalStack.add(current) + + // Due to how Concat works, we need a stack. This probably can be optimized for the most common cases. + while (!traversalStack.isEmpty) { + current = traversalStack.remove(traversalStack.size() - 1) + + while (current ne EmptyTraversal) { + current match { + case MaterializeAtomic(mod, outToSlot) ⇒ + var i = 0 + val inletsIter = mod.shape.inlets.iterator + while (inletsIter.hasNext) { + val in = inletsIter.next() + inlets(inOffs + i) = in + i += 1 + } + + val outletsIter = mod.shape.outlets.iterator + while (outletsIter.hasNext) { + val out = outletsIter.next() + outlets(inOffs + outToSlot(out.id)) = out + } + inOffs += mod.shape.inlets.size + current = current.next + // And that's it ;) + case Concat(first, next) ⇒ + traversalStack.add(next) + current = first + case _ ⇒ + current = current.next + } + } + } + new MaterializationResult(connections, inlets, outlets) + } + + case class TestPublisher(owner: Module, port: OutPort) extends Publisher[Any] with Subscription { + var downstreamModule: Module = _ + var downstreamPort: InPort = _ + + override def subscribe(s: Subscriber[_ >: Any]): Unit = s match { + case TestSubscriber(o, p) ⇒ + downstreamModule = o + downstreamPort = p + s.onSubscribe(this) + } + + override def request(n: Long): Unit = () + override def cancel(): Unit = () + } + + case class TestSubscriber(owner: Module, port: InPort) extends Subscriber[Any] { + var upstreamModule: Module = _ + var upstreamPort: OutPort = _ + + override def onSubscribe(s: Subscription): Unit = s match { + case TestPublisher(o, p) ⇒ + upstreamModule = o + upstreamPort = p + } + + override def onError(t: Throwable): Unit = () + override def onComplete(): Unit = () + override def onNext(t: Any): Unit = () + } + + class FlatTestMaterializer(_module: Module) extends MaterializerSession(_module, Attributes()) { + private var i = 0 + var publishers = Array.ofDim[TestPublisher](1024) + var subscribers = Array.ofDim[TestSubscriber](1024) + + override protected def materializeAtomic(atomic: AtomicModule, effectiveAttributes: Attributes, + matVal: java.util.Map[Module, Any]): Unit = { + for (inPort ← atomic.inPorts) { + val subscriber = TestSubscriber(atomic, inPort) + subscribers(i) = subscriber + i += 1 + assignPort(inPort, subscriber) + } + for (outPort ← atomic.outPorts) { + val publisher = TestPublisher(atomic, outPort) + publishers(i) = publisher + i += 1 + assignPort(outPort, publisher) + } + } + } + + def testMaterializeOld(m: Module, blackhole: org.openjdk.jmh.infra.Blackhole): Unit = { + val mat = new FlatTestMaterializer(m) + mat.materialize() + blackhole.consume(mat.publishers) + } + + @Benchmark + def source_and_sink_new_linear(): TraversalBuilder = { + linearSource.traversalBuilder.append(linearSink.traversalBuilder, linearSink.shape) + } + + @Benchmark + def source_and_sink_new_composite(): TraversalBuilder = { + compositeSource.traversalBuilder + .add(compositeSink.traversalBuilder, compositeSink.shape) + .wire(compositeSource.out, compositeSink.in) + } + + @Benchmark + def source_and_sink_old(): Module = { + oldSource + .compose(oldSink) + .wire(oldSource.out, oldSink.in) + } + + @Benchmark + def source_flow_and_sink_new_linear(): TraversalBuilder = { + linearSource.traversalBuilder + .append(linearFlow.traversalBuilder, linearFlow.shape) + .append(linearSink.traversalBuilder, linearSink.shape) + } + + @Benchmark + def source_flow_and_sink_new_composite(): TraversalBuilder = { + compositeSource.traversalBuilder + .add(compositeFlow.traversalBuilder, compositeFlow.shape) + .wire(compositeSource.out, compositeFlow.in) + .add(compositeSink.traversalBuilder, compositeSink.shape) + .wire(compositeFlow.out, compositeSink.in) + } + + @Benchmark + def source_flow_and_sink_old(): Module = { + oldSource + .compose(oldFlow) + .wire(oldSource.out, oldFlow.in) + .compose(oldSink) + .wire(oldFlow.out, oldSink.in) + } + + val sourceSinkLinear = linearSource.traversalBuilder.append(linearSink.traversalBuilder, linearSink.shape) + val sourceSinkComposite = compositeSource.traversalBuilder + .add(compositeSink.traversalBuilder, compositeSink.shape) + .wire(compositeSource.out, compositeSink.in) + val sourceSinkOld = oldSource + .compose(oldSink) + .wire(oldSource.out, oldSink.in) + + val sourceFlowSinkLinear = linearSource.traversalBuilder + .append(linearFlow.traversalBuilder, linearFlow.shape) + .append(linearSink.traversalBuilder, linearSink.shape) + val sourceFlowSinkComposite = compositeSource.traversalBuilder + .add(compositeFlow.traversalBuilder, compositeFlow.shape) + .wire(compositeSource.out, compositeFlow.in) + .add(compositeSink.traversalBuilder, compositeSink.shape) + .wire(compositeFlow.out, compositeSink.in) + val sourceFlowSinkOld = oldSource + .compose(oldFlow) + .wire(oldSource.out, oldFlow.in) + .compose(oldSink) + .wire(oldFlow.out, oldSink.in) + + @Benchmark + def mat_source_and_sink_new_linear(): MaterializationResult = { + testMaterializeNew(sourceSinkLinear) + } + + @Benchmark + def mat_source_and_sink_new_composite(): MaterializationResult = { + testMaterializeNew(sourceSinkComposite) + } + + @Benchmark + def mat_source_and_sink_old(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = { + testMaterializeOld(sourceSinkOld, blackhole: org.openjdk.jmh.infra.Blackhole) + } + + @Benchmark + def mat_source_flow_and_sink_new_linear(): MaterializationResult = { + testMaterializeNew(sourceFlowSinkLinear) + } + + @Benchmark + def mat_source_flow_and_sink_new_composite(): MaterializationResult = { + testMaterializeNew(sourceFlowSinkComposite) + } + + @Benchmark + def mat_source_flow_and_sink_old(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = { + testMaterializeOld(sourceFlowSinkOld, blackhole: org.openjdk.jmh.infra.Blackhole) + } + +} diff --git a/akka-stream-testkit/src/main/scala/akka/stream/testkit/StreamTestKit.scala b/akka-stream-testkit/src/main/scala/akka/stream/testkit/StreamTestKit.scala index 719ac4b459..69804a0728 100644 --- a/akka-stream-testkit/src/main/scala/akka/stream/testkit/StreamTestKit.scala +++ b/akka-stream-testkit/src/main/scala/akka/stream/testkit/StreamTestKit.scala @@ -5,7 +5,6 @@ package akka.stream.testkit import akka.actor.{ ActorSystem, DeadLetterSuppression, NoSerializationVerificationNeeded } import akka.stream._ -import akka.stream.impl.StreamLayout.Module import akka.stream.impl._ import akka.testkit.TestProbe import org.reactivestreams.{ Publisher, Subscriber, Subscription } @@ -758,7 +757,7 @@ private[testkit] object StreamTestKit { (probe, probe) } override protected def newInstance(shape: SourceShape[T]): SourceModule[T, TestPublisher.Probe[T]] = new ProbeSource[T](attributes, shape) - override def withAttributes(attr: Attributes): Module = new ProbeSource[T](attr, amendShape(attr)) + override def withAttributes(attr: Attributes): SourceModule[T, TestPublisher.Probe[T]] = new ProbeSource[T](attr, amendShape(attr)) } final class ProbeSink[T](val attributes: Attributes, shape: SinkShape[T])(implicit system: ActorSystem) extends SinkModule[T, TestSubscriber.Probe[T]](shape) { @@ -767,7 +766,7 @@ private[testkit] object StreamTestKit { (probe, probe) } override protected def newInstance(shape: SinkShape[T]): SinkModule[T, TestSubscriber.Probe[T]] = new ProbeSink[T](attributes, shape) - override def withAttributes(attr: Attributes): Module = new ProbeSink[T](attr, amendShape(attr)) + override def withAttributes(attr: Attributes): SinkModule[T, TestSubscriber.Probe[T]] = new ProbeSink[T](attr, amendShape(attr)) } } diff --git a/akka-stream-testkit/src/main/scala/akka/stream/testkit/scaladsl/TestSink.scala b/akka-stream-testkit/src/main/scala/akka/stream/testkit/scaladsl/TestSink.scala index af4049c0cb..788aa7db98 100644 --- a/akka-stream-testkit/src/main/scala/akka/stream/testkit/scaladsl/TestSink.scala +++ b/akka-stream-testkit/src/main/scala/akka/stream/testkit/scaladsl/TestSink.scala @@ -19,6 +19,6 @@ object TestSink { * A Sink that materialized to a [[akka.stream.testkit.TestSubscriber.Probe]]. */ def probe[T](implicit system: ActorSystem): Sink[T, Probe[T]] = - new Sink[T, TestSubscriber.Probe[T]](new StreamTestKit.ProbeSink(none, SinkShape(Inlet("ProbeSink.in")))) + Sink.fromGraph[T, TestSubscriber.Probe[T]](new StreamTestKit.ProbeSink(none, SinkShape(Inlet("ProbeSink.in")))) } diff --git a/akka-stream-testkit/src/main/scala/akka/stream/testkit/scaladsl/TestSource.scala b/akka-stream-testkit/src/main/scala/akka/stream/testkit/scaladsl/TestSource.scala index 84e158b201..14614c3070 100644 --- a/akka-stream-testkit/src/main/scala/akka/stream/testkit/scaladsl/TestSource.scala +++ b/akka-stream-testkit/src/main/scala/akka/stream/testkit/scaladsl/TestSource.scala @@ -18,6 +18,6 @@ object TestSource { /** * A Source that materializes to a [[akka.stream.testkit.TestPublisher.Probe]]. */ - def probe[T](implicit system: ActorSystem) = new Source[T, TestPublisher.Probe[T]](new StreamTestKit.ProbeSource(none, SourceShape(Outlet("ProbeSource.out")))) + def probe[T](implicit system: ActorSystem) = Source.fromGraph[T, TestPublisher.Probe[T]](new StreamTestKit.ProbeSource(none, SourceShape(Outlet("ProbeSource.out")))) } diff --git a/akka-stream-testkit/src/test/scala/akka/stream/testkit/Utils.scala b/akka-stream-testkit/src/test/scala/akka/stream/testkit/Utils.scala index 4b6131d381..d265b03dd3 100644 --- a/akka-stream-testkit/src/test/scala/akka/stream/testkit/Utils.scala +++ b/akka-stream-testkit/src/test/scala/akka/stream/testkit/Utils.scala @@ -18,7 +18,7 @@ object Utils { def assertAllStagesStopped[T](block: ⇒ T)(implicit materializer: Materializer): T = materializer match { - case impl: ActorMaterializerImpl ⇒ + case impl: PhasedFusingActorMaterializer ⇒ val probe = TestProbe()(impl.system) probe.send(impl.supervisor, StreamSupervisor.StopChildren) probe.expectMsg(StreamSupervisor.StoppedChildren) diff --git a/akka-stream-tests/src/test/scala/akka/stream/ActorMaterializerSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/ActorMaterializerSpec.scala index 1e3f7e879f..de64c4793d 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/ActorMaterializerSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/ActorMaterializerSpec.scala @@ -1,10 +1,10 @@ package akka.stream import akka.actor.{ ActorSystem, Props } -import akka.stream.impl.{ StreamSupervisor, ActorMaterializerImpl } +import akka.stream.impl.{ PhasedFusingActorMaterializer, StreamSupervisor } import akka.stream.scaladsl.{ Sink, Source } import akka.stream.testkit.StreamSpec -import akka.testkit.{ TestActor, ImplicitSender } +import akka.testkit.{ ImplicitSender, TestActor } import scala.concurrent.Await import scala.concurrent.duration._ @@ -39,7 +39,7 @@ class ActorMaterializerSpec extends StreamSpec with ImplicitSender { } "shut down the supervisor actor it encapsulates" in { - val m = ActorMaterializer.create(system).asInstanceOf[ActorMaterializerImpl] + val m = ActorMaterializer.create(system).asInstanceOf[PhasedFusingActorMaterializer] Source.maybe[Any].to(Sink.ignore).run()(m) m.supervisor ! StreamSupervisor.GetChildren diff --git a/akka-stream-tests/src/test/scala/akka/stream/FusingSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/FusingSpec.scala index c38de4cdcc..b26947960b 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/FusingSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/FusingSpec.scala @@ -3,14 +3,8 @@ */ package akka.stream -import akka.NotUsed import akka.stream.scaladsl._ import akka.stream.testkit.StreamSpec -import akka.stream.Attributes._ -import akka.stream.Fusing.FusedGraph - -import scala.annotation.tailrec -import akka.stream.impl.StreamLayout._ import akka.stream.impl.fusing.GraphInterpreter import akka.event.BusLogging @@ -24,126 +18,6 @@ class FusingSpec extends StreamSpec { .alsoTo(Flow[Int].fold(0)(_ + _).to(Sink.head.named("otherSink")).addAttributes(if (async) Attributes.asyncBoundary else Attributes.none)) .via(Flow[Int].fold(1)(_ + _).named("mainSink")) - def singlePath[S <: Shape, M](fg: FusedGraph[S, M], from: Attribute, to: Attribute): Unit = { - val starts = fg.module.info.subModules.filter(_.attributes.contains(from)) - starts.size should ===(1) - val start = starts.head - val ups = fg.module.info.upstreams - val owner = fg.module.info.outOwners - - @tailrec def rec(curr: Module): Unit = { - if (Debug) println(extractName(curr, "unknown")) - curr match { - case CopiedModule(_, attributes, copyOf) if (attributes and copyOf.attributes).contains(to) ⇒ () - case other if other.attributes.contains(to) ⇒ () - case _ ⇒ - val outs = curr.inPorts.map(ups) - outs.size should ===(1) - rec(owner(outs.head)) - } - } - - rec(start) - } - - case object NoSubModulesModule extends AtomicModule { - override def shape = ClosedShape - override def replaceShape(s: Shape) = - if (s != shape) throw new UnsupportedOperationException("cannot replace shape") - else this - - override def compose(that: Module): Module = compose(that, scaladsl.Keep.left) - - override def compose[A, B, C](that: Module, f: (A, B) ⇒ C): Module = { - if (f eq scaladsl.Keep.right) { - that - } else if (f eq scaladsl.Keep.left) { - val mat = - if (IgnorableMatValComp(that)) { - Ignore - } else { - Transform(_ ⇒ NotUsed, that.materializedValueComputation) - } - CompositeModule( - if (that.isSealed) Set(that) else that.subModules, - that.shape, - that.downstreams, - that.upstreams, - mat, - if (this.isSealed) Attributes.none else attributes) - } else { - throw new UnsupportedOperationException( - "It is invalid to combine materialized value with BogusModule " + - "except with Keep.left or Keep.right") - } - } - - override def withAttributes(attributes: Attributes): Module = - throw new UnsupportedOperationException("BogusModule cannot carry attributes") - - override def attributes = Attributes.none - override def carbonCopy: Module = this - override def isRunnable: Boolean = true - override def isAtomic: Boolean = true - override def materializedValueComputation: MaterializedValueNode = Ignore - } - - "Fusing" must { - - def verify[S <: Shape, M](fused: FusedGraph[S, M], modules: Int, downstreams: Int): Unit = { - val module = fused.module - module.subModules.size should ===(modules) - module.downstreams.size should ===(modules - 1) - module.info.downstreams.size should be >= downstreams - module.info.upstreams.size should be >= downstreams - singlePath(fused, Attributes.Name("mainSink"), Attributes.Name("unfold")) - singlePath(fused, Attributes.Name("otherSink"), Attributes.Name("unfold")) - } - - "fuse a moderately complex graph" in { - val g = graph(false) - val fused = Fusing.aggressive(g) - verify(fused, modules = 1, downstreams = 5) - } - - "not fuse across AsyncBoundary" in { - val g = graph(true) - val fused = Fusing.aggressive(g) - verify(fused, modules = 2, downstreams = 5) - } - - "not fuse a FusedGraph again" in { - val g = Fusing.aggressive(graph(false)) - Fusing.aggressive(g) should be theSameInstanceAs g - } - - "properly fuse a FusedGraph that has been extended (no AsyncBoundary)" in { - val src = Fusing.aggressive(graph(false)) - val fused = Fusing.aggressive(Source.fromGraph(src).to(Sink.head)) - verify(fused, modules = 1, downstreams = 6) - } - - "properly fuse a FusedGraph that has been extended (with AsyncBoundary)" in { - val src = Fusing.aggressive(graph(true)) - val fused = Fusing.aggressive(Source.fromGraph(src).to(Sink.head)) - verify(fused, modules = 2, downstreams = 6) - } - - "fuse a Module with no subModules" in { - val structuralInfoModule = Fusing.structuralInfo(RunnableGraph(NoSubModulesModule), Attributes.none) - structuralInfoModule.matValues.size > 0 - } - - "fuse a Module with empty graph" in { - val g = GraphDSL.create() { implicit b ⇒ - ClosedShape - } - val structuralInfoModule = Fusing.structuralInfo(g, Attributes.none) - structuralInfoModule.matValues.size > 0 - } - - } - "SubFusingActorMaterializer" must { "work with asynchronous boundaries in the subflows" in { diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/LinearTraversalBuilderSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/LinearTraversalBuilderSpec.scala new file mode 100644 index 0000000000..cf58201d74 --- /dev/null +++ b/akka-stream-tests/src/test/scala/akka/stream/impl/LinearTraversalBuilderSpec.scala @@ -0,0 +1,811 @@ +/** + * Copyright (C) 2015-2016 Lightbend Inc. + */ + +package akka.stream.impl + +import akka.NotUsed +import akka.stream._ +import akka.stream.impl.StreamLayout.AtomicModule +import akka.stream.impl.TraversalTestUtils._ +import akka.stream.scaladsl.Keep +import akka.testkit.AkkaSpec + +class LinearTraversalBuilderSpec extends AkkaSpec { + + "LinearTraversalBuilder" must { + val source = new LinearTestSource + val sink = new LinearTestSink + val flow1 = new LinearTestFlow("1") + val flow2 = new LinearTestFlow("2") + + val compositeSource = new CompositeTestSource + val compositeSink = new CompositeTestSink + val compositeFlow1 = new CompositeTestFlow("C1") + val compositeFlow2 = new CompositeTestFlow("C2") + + // ADD closed shape, (and composite closed shape) + + "work with a single Source and Sink" in { + val builder = + source.traversalBuilder + .append(sink.traversalBuilder, sink.shape, Keep.left) + + val mat = testMaterialize(builder) + + mat.connections should ===(1) + + mat.outlets(0) should ===(source.out) + mat.inlets(0) should ===(sink.in) + } + + "work with two Flows" in { + val builder = source.traversalBuilder + .append(flow1.traversalBuilder, flow1.shape, Keep.left) + .append(flow2.traversalBuilder, flow2.shape, Keep.left) + .append(sink.traversalBuilder, sink.shape, Keep.left) + + val mat = testMaterialize(builder) + + mat.connections should ===(3) + mat.outlets(2) should ===(source.out) + mat.inlets(2) should ===(flow1.in) + mat.outlets(1) should ===(flow1.out) + mat.inlets(1) should ===(flow2.in) + mat.outlets(0) should ===(flow2.out) + mat.inlets(0) should ===(sink.in) + } + + "work with two Flows wired in opposite order" in { + val s1 = flow2.traversalBuilder.append(sink.traversalBuilder, sink.shape, Keep.left) + val s2 = flow1.traversalBuilder.append(s1, SinkShape(flow2.in), Keep.left) + val builder = source.traversalBuilder.append(s2, SinkShape(flow1.in), Keep.left) + + val mat = testMaterialize(builder) + + mat.connections should ===(3) + mat.outlets(2) should ===(source.out) + mat.inlets(2) should ===(flow1.in) + mat.outlets(1) should ===(flow1.out) + mat.inlets(1) should ===(flow2.in) + mat.outlets(0) should ===(flow2.out) + mat.inlets(0) should ===(sink.in) + } + + "work with two Flows wired in an irregular order" in { + val source2 = source.traversalBuilder.append(flow1.traversalBuilder, flow1.shape, Keep.left) + val sink2 = flow2.traversalBuilder.append(sink.traversalBuilder, sink.shape, Keep.left) + + val builder = source2.append(sink2, SinkShape(flow2.in), Keep.left) + + val mat = testMaterialize(builder) + + mat.connections should ===(3) + mat.outlets(2) should ===(source.out) + mat.inlets(2) should ===(flow1.in) + mat.outlets(1) should ===(flow1.out) + mat.inlets(1) should ===(flow2.in) + mat.outlets(0) should ===(flow2.out) + mat.inlets(0) should ===(sink.in) + } + + "work with a Flow appended to its imported self" in { + val remappedShape = FlowShape(Inlet[Any]("Remapped.in"), Outlet[Any]("Remapped.out")) + remappedShape.in.mappedTo = flow1.in + remappedShape.out.mappedTo = flow1.out + + val builder = source.traversalBuilder + .append(flow1.traversalBuilder, flow1.shape, Keep.left) + .append(flow1.traversalBuilder, flow1.shape, Keep.left) + .append(sink.traversalBuilder, sink.shape, Keep.left) + + val mat = testMaterialize(builder) + + mat.connections should ===(3) + mat.outlets(2) should ===(source.out) + mat.inlets(2) should ===(flow1.in) + mat.outlets(1) should ===(flow1.out) + mat.inlets(1) should ===(flow1.in) + mat.outlets(0) should ===(flow1.out) + mat.inlets(0) should ===(sink.in) + } + + "work with a nested Flow chain" in { + val nestedFlows = + flow1.traversalBuilder + .append(flow2.traversalBuilder, flow2.shape, Keep.left) + + val builder = source.traversalBuilder + .append(nestedFlows, FlowShape(flow1.in, flow2.out), Keep.left) + .append(sink.traversalBuilder, sink.shape, Keep.left) + + val mat = testMaterialize(builder) + + mat.connections should ===(3) + mat.outlets(2) should ===(source.out) + mat.inlets(2) should ===(flow1.in) + mat.outlets(1) should ===(flow1.out) + mat.inlets(1) should ===(flow2.in) + mat.outlets(0) should ===(flow2.out) + mat.inlets(0) should ===(sink.in) + } + + "work with a nested Flow chain used twice (appended to self)" in { + val nestedFlows = + flow1.traversalBuilder + .append(flow2.traversalBuilder, flow2.shape, Keep.left) + + val builder = source.traversalBuilder + .append(nestedFlows, FlowShape(flow1.in, flow2.out), Keep.left) + .append(nestedFlows, FlowShape(flow1.in, flow2.out), Keep.left) + .append(sink.traversalBuilder, sink.shape, Keep.left) + + val mat = testMaterialize(builder) + + mat.connections should ===(5) + mat.outlets(4) should ===(source.out) + mat.inlets(4) should ===(flow1.in) + mat.outlets(3) should ===(flow1.out) + mat.inlets(3) should ===(flow2.in) + mat.outlets(2) should ===(flow2.out) + mat.inlets(2) should ===(flow1.in) + mat.outlets(1) should ===(flow1.out) + mat.inlets(1) should ===(flow2.in) + mat.outlets(0) should ===(flow2.out) + mat.inlets(0) should ===(sink.in) + } + + "work with a Flow wired to self" in { + val builder = flow1.traversalBuilder.wire(flow1.out, flow1.in) + + val mat = testMaterialize(builder) + + mat.connections should ===(1) + mat.outlets(0) should ===(flow1.out) + mat.inlets(0) should ===(flow1.in) + } + + "work with a two Flows wired back to self" in { + val builder = + flow1.traversalBuilder + .append(flow2.traversalBuilder, flow2.shape, Keep.left) + .wire(flow2.out, flow1.in) + + val mat = testMaterialize(builder) + + mat.connections should ===(2) + mat.outlets(0) should ===(flow1.out) + mat.inlets(0) should ===(flow2.in) + mat.outlets(1) should ===(flow2.out) + mat.inlets(1) should ===(flow1.in) + } + + "work with Flow appended to self then wired back to self" in { + val builder = + flow1.traversalBuilder + .append(flow1.traversalBuilder, flow1.shape, Keep.left) + .wire(flow1.out, flow1.in) + + val mat = testMaterialize(builder) + + mat.connections should ===(2) + mat.outlets(0) should ===(flow1.out) + mat.inlets(0) should ===(flow1.in) + mat.outlets(1) should ===(flow1.out) + mat.inlets(1) should ===(flow1.in) + } + + "be able to be used with a composite source" in { + val builder = + compositeSource.traversalBuilder + .add(sink.traversalBuilder, sink.shape, Keep.left) + .wire(compositeSource.out, sink.in) + + val mat = testMaterialize(builder) + + mat.connections should ===(1) + mat.outlets(0) should ===(compositeSource.out) + mat.inlets(0) should ===(sink.in) + } + + "be able to be used with a composite sink" in { + val builder = + compositeSink.traversalBuilder + .add(source.traversalBuilder, source.shape, Keep.left) + .wire(source.out, compositeSink.in) + + val mat = testMaterialize(builder) + + mat.connections should ===(1) + mat.outlets(0) should ===(source.out) + mat.inlets(0) should ===(compositeSink.in) + } + + "be able to be joined with a composite flow" in { + val embeddedFlow = + flow1.traversalBuilder + .append(flow2.traversalBuilder, flow2.shape, Keep.left) + + val builder = + compositeFlow1.traversalBuilder + .add(embeddedFlow, FlowShape(flow1.in, flow2.out), Keep.left) + .wire(compositeFlow1.out, flow1.in) + .wire(flow2.out, compositeFlow1.in) + + val mat = testMaterialize(builder) + + mat.connections should ===(3) + mat.outlets(0) should ===(flow2.out) + mat.inlets(0) should ===(compositeFlow1.in) + mat.outlets(1) should ===(flow1.out) + mat.inlets(1) should ===(flow2.in) + mat.outlets(2) should ===(compositeFlow1.out) + mat.inlets(2) should ===(flow1.in) + } + + "be able to use a linear flow with composite source and sink" in { + val builder = + compositeSource.traversalBuilder + .add(compositeSink.traversalBuilder, compositeSink.shape, Keep.left) + .add(flow1.traversalBuilder, flow1.shape, Keep.left) + .wire(compositeSource.out, flow1.in) + .wire(flow1.out, compositeSink.in) + + val mat = testMaterialize(builder) + + mat.connections should ===(2) + mat.outlets(0) should ===(flow1.out) + mat.inlets(0) should ===(compositeSink.in) + mat.outlets(1) should ===(compositeSource.out) + mat.inlets(1) should ===(flow1.in) + } + + "be able to add a flow to an empty composite and join to itself" in { + val builder = + TraversalBuilder.empty() + .add(flow1.traversalBuilder, flow1.shape, Keep.left) + .wire(flow1.out, flow1.in) + + val mat = testMaterialize(builder) + + mat.connections should ===(1) + mat.outlets(0) should ===(flow1.out) + mat.inlets(0) should ===(flow1.in) + } + + "be able to embed a composite sink in a linear traversal" in { + val builder = + source.traversalBuilder + .append(compositeSink.traversalBuilder, compositeSink.shape, Keep.left) + + val mat = testMaterialize(builder) + + mat.connections should ===(1) + mat.outlets(0) should ===(source.out) + mat.inlets(0) should ===(compositeSink.in) + } + + "be able to start from a composite source" in pending + + "be able to start from a composite flow" in pending + + "be able to embed a composite flow in a linear traversal" in { + val builder = + source.traversalBuilder + .append(compositeFlow1.traversalBuilder, compositeFlow1.shape, Keep.left) + .append(sink.traversalBuilder, sink.shape, Keep.left) + + val mat = testMaterialize(builder) + + mat.connections should ===(2) + mat.outlets(1) should ===(source.out) + mat.inlets(1) should ===(compositeFlow1.in) + mat.outlets(0) should ===(compositeFlow1.out) + mat.inlets(0) should ===(sink.in) + } + + "be able to embed a composite sink with an irregular wiring" in { + val sinkBuilder = + TraversalBuilder.empty() + .add(compositeFlow2.traversalBuilder, compositeFlow2.shape, Keep.left) + .add(compositeSink.traversalBuilder, compositeSink.shape, Keep.left) + .add(compositeFlow1.traversalBuilder, compositeFlow1.shape, Keep.left) + .wire(compositeFlow2.out, compositeSink.in) + .wire(compositeFlow1.out, compositeFlow2.in) + + val builder = + source.traversalBuilder + .append(sinkBuilder, SinkShape(compositeFlow1.in), Keep.left) + + val mat = testMaterialize(builder) + + mat.connections should ===(3) + mat.outlets(2) should ===(source.out) + mat.inlets(2) should ===(compositeFlow1.in) + mat.outlets(1) should ===(compositeFlow2.out) + mat.inlets(1) should ===(compositeSink.in) + mat.outlets(0) should ===(compositeFlow1.out) + mat.inlets(0) should ===(compositeFlow2.in) + } + + "be able to embed a composite flow multiple times appended to self" in { + val builder = + source.traversalBuilder + .append(compositeFlow1.traversalBuilder, compositeFlow1.shape, Keep.left) + .append(compositeFlow1.traversalBuilder, compositeFlow1.shape, Keep.left) + .append(sink.traversalBuilder, sink.shape, Keep.left) + + val mat = testMaterialize(builder) + + mat.connections should ===(3) + mat.outlets(2) should ===(source.out) + mat.inlets(2) should ===(compositeFlow1.in) + mat.outlets(1) should ===(compositeFlow1.out) + mat.inlets(1) should ===(compositeFlow1.in) + mat.outlets(0) should ===(compositeFlow1.out) + mat.inlets(0) should ===(sink.in) + } + + "be able to embed a composite flow multiple times appended to self alternating with linear flow" in { + + val builder = + source.traversalBuilder + .append(compositeFlow1.traversalBuilder, compositeFlow1.shape, Keep.left) + .append(flow1.traversalBuilder, flow1.shape, Keep.left) + .append(compositeFlow1.traversalBuilder, compositeFlow1.shape, Keep.left) + .append(flow1.traversalBuilder, flow1.shape, Keep.left) + .append(sink.traversalBuilder, sink.shape, Keep.left) + + val mat = testMaterialize(builder) + + mat.connections should ===(5) + + mat.outlets(4) should ===(source.out) + mat.inlets(4) should ===(compositeFlow1.in) + mat.outlets(3) should ===(compositeFlow1.out) + mat.inlets(3) should ===(flow1.in) + mat.outlets(2) should ===(flow1.out) + mat.inlets(2) should ===(compositeFlow1.in) + mat.outlets(1) should ===(compositeFlow1.out) + mat.inlets(1) should ===(flow1.in) + mat.outlets(0) should ===(flow1.out) + mat.inlets(0) should ===(sink.in) + } + + "be able to embed a composite flow wrapped in linear, if the input port is of the second flow" in { + val compositeBuilder = + compositeFlow2.traversalBuilder + .add(compositeFlow1.traversalBuilder, compositeFlow1.shape, Keep.left) + .add(flow1.traversalBuilder, flow1.shape, Keep.left) + .wire(compositeFlow1.out, compositeFlow2.in) + .wire(compositeFlow2.out, flow1.in) + + val shape = FlowShape(compositeFlow1.in, flow1.out) + + val embeddedBuilder = + LinearTraversalBuilder.empty() + .append(compositeBuilder, shape, Keep.left) + + val builder = + source.traversalBuilder + .append(embeddedBuilder, shape, Keep.left) + .append(sink.traversalBuilder, sink.shape, Keep.left) + + val mat = testMaterialize(builder) + + mat.connections should ===(4) + + mat.outlets(3) should ===(compositeFlow2.out) + mat.inlets(3) should ===(flow1.in) + mat.outlets(2) should ===(source.out) + mat.inlets(2) should ===(compositeFlow1.in) + mat.outlets(1) should ===(compositeFlow1.out) + mat.inlets(1) should ===(compositeFlow2.in) + mat.outlets(0) should ===(flow1.out) + mat.inlets(0) should ===(sink.in) + } + + "properly materialize empty builder" in { + val builder = LinearTraversalBuilder.empty() + + val mat = testMaterialize(builder) + mat.connections should ===(0) + mat.outlets.length should ===(0) + mat.inlets.length should ===(0) + mat.matValue should ===(NotUsed) + } + + "properly propagate materialized value with Keep.left" in { + val builder = + source.traversalBuilder + .append(sink.traversalBuilder, sink.shape, Keep.left) + + val mat = testMaterialize(builder) + + mat.matValue should ===("TestSource") + } + + "keep mapped materialized value of empty builder" in { + val builder = + LinearTraversalBuilder.empty() + .transformMat[NotUsed, String](_ ⇒ "NOTUSED") + .append(source.traversalBuilder, source.shape, Keep.left) + .append(sink.traversalBuilder, sink.shape, Keep.left) + + val mat = testMaterialize(builder) + + mat.matValue should ===("NOTUSED") + } + + "properly propagate materialized value with Keep.right" in { + val builder = + source.traversalBuilder + .append(sink.traversalBuilder, sink.shape, Keep.right) + + val mat = testMaterialize(builder) + + mat.matValue should ===("TestSink") + } + + "properly propagate materialized value with Keep.both" in { + val builder = + source.traversalBuilder + .append(sink.traversalBuilder, sink.shape, Keep.both) + + val mat = testMaterialize(builder) + + mat.matValue should ===(("TestSource", "TestSink")) + } + + "properly propagate materialized value with Keep.left with Flow in middle" in { + val builder = source.traversalBuilder + .append(flow1.traversalBuilder, flow1.shape, Keep.left) + .append(sink.traversalBuilder, sink.shape, Keep.left) + + val mat = testMaterialize(builder) + + mat.matValue should ===("TestSource") + } + + "properly propagate materialized value with Keep.right with Flow in middle (1)" in { + val builder = source.traversalBuilder + .append(flow1.traversalBuilder, flow1.shape, Keep.right) + .append(sink.traversalBuilder, sink.shape, Keep.left) + + val mat = testMaterialize(builder) + + mat.matValue should ===("TestFlow1") + } + + "properly propagate materialized value with Keep.right with composite Flow in middle" in { + val builder = source.traversalBuilder + .append(compositeFlow1.traversalBuilder, compositeFlow1.shape, Keep.right) + .append(sink.traversalBuilder, sink.shape, Keep.left) + + val mat = testMaterialize(builder) + + mat.matValue should ===("TestFlowC1") + } + + "properly propagate materialized value with Keep.right with composite Source as start" in { + val builder = + LinearTraversalBuilder.empty() + .append(compositeSource.traversalBuilder, compositeSource.shape, Keep.right) + .append(flow1.traversalBuilder, flow1.shape, Keep.left) + .append(sink.traversalBuilder, sink.shape, Keep.left) + + val mat = testMaterialize(builder) + + mat.matValue should ===("TestSource") + } + + "properly propagate materialized value with Keep.right with composite Sink as end" in { + val builder = + source.traversalBuilder + .append(flow1.traversalBuilder, flow1.shape, Keep.left) + .append(compositeSink.traversalBuilder, compositeSink.shape, Keep.right) + + val mat = testMaterialize(builder) + + mat.matValue should ===("TestSink") + } + + "properly propagate materialized value with Keep.both and all composite" in { + val builder = + LinearTraversalBuilder.empty() + .append(compositeSource.traversalBuilder, compositeSource.shape, Keep.both) + .append(compositeFlow1.traversalBuilder, compositeFlow1.shape, Keep.both) + .append(compositeSink.traversalBuilder, compositeSink.shape, Keep.both) + + val mat = testMaterialize(builder) + + mat.matValue should ===((((NotUsed, "TestSource"), "TestFlowC1"), "TestSink")) + } + + "properly propagate materialized value with Keep.right with Flow in middle (2)" in { + val builder = source.traversalBuilder + .append(flow1.traversalBuilder, flow1.shape, Keep.left) + .append(sink.traversalBuilder, sink.shape, Keep.right) + + val mat = testMaterialize(builder) + + mat.matValue should ===("TestSink") + } + + "properly propagate materialized value with Keep.both with Flow in middle (1)" in { + val builder = source.traversalBuilder + .append(flow1.traversalBuilder, flow1.shape, Keep.both) + .append(sink.traversalBuilder, sink.shape, Keep.left) + + val mat = testMaterialize(builder) + + mat.matValue should ===(("TestSource", "TestFlow1")) + } + + "properly propagate materialized value with Keep.both with Flow in middle (2)" in { + val builder = source.traversalBuilder + .append(flow1.traversalBuilder, flow1.shape, Keep.both) + .append(sink.traversalBuilder, sink.shape, Keep.both) + + val mat = testMaterialize(builder) + + mat.matValue should ===((("TestSource", "TestFlow1"), "TestSink")) + } + + "properly map materialized value" in { + val builder = source.traversalBuilder + .append(flow1.traversalBuilder, flow1.shape, Keep.right) + .append(sink.traversalBuilder, sink.shape, Keep.left) + .transformMat[String, String]("MAPPED: " + _) + + val mat = testMaterialize(builder) + + mat.matValue should ===("MAPPED: TestFlow1") + } + + "properly map materialized value (nested)" in { + val flowBuilder = + flow1.traversalBuilder + .transformMat[String, String]("M1: " + _) + + val builder = source.traversalBuilder + .append(flowBuilder, flow1.shape, Keep.right) + .append(sink.traversalBuilder, sink.shape, Keep.left) + .transformMat[String, String]("M2: " + _) + + val mat = testMaterialize(builder) + + mat.matValue should ===("M2: M1: TestFlow1") + } + + "properly set attributes for whole chain" in { + val builder = source.traversalBuilder + .append(sink.traversalBuilder, sink.shape, Keep.left) + .setAttributes(Attributes.name("test")) + + val mat = testMaterialize(builder) + + mat.attributesAssignments should ===(List( + sink → (Attributes.name("test") and Attributes.name("testSink")), + source → (Attributes.name("test") and Attributes.name("testSource")) + )) + } + + "properly accumulate attributes in chain" in { + val builder = source.traversalBuilder + .setAttributes(Attributes.name("source")) + .append(sink.traversalBuilder, sink.shape, Keep.left) + .setAttributes(Attributes.name("test")) + + val mat = testMaterialize(builder) + + mat.attributesAssignments should ===(List( + sink → (Attributes.name("test") and Attributes.name("testSink")), + source → (Attributes.name("test") and Attributes.name("source")) + )) + } + + "overwrite last attributes until a new module is added" in { + val builder = source.traversalBuilder + .setAttributes(Attributes.name("source")) + .setAttributes(Attributes.name("source2")) + .append(sink.traversalBuilder, sink.shape, Keep.left) + .setAttributes(Attributes.name("test")) + .setAttributes(Attributes.name("test2")) + + val mat = testMaterialize(builder) + + mat.attributesAssignments should ===(List( + sink → (Attributes.name("test2") and Attributes.name("testSink")), + source → (Attributes.name("test2") and Attributes.name("source2")) + )) + } + + "propagate attributes to embedded linear sink and source" in { + val builder = + source.traversalBuilder + .setAttributes(Attributes.name("source")) + .append(sink.traversalBuilder.setAttributes(Attributes.name("sink")), sink.shape, Keep.left) + .setAttributes(Attributes.name("test")) + + val mat = testMaterialize(builder) + + mat.attributesAssignments should ===(List( + sink → (Attributes.name("test") and Attributes.name("sink")), + source → (Attributes.name("test") and Attributes.name("source")) + )) + } + + "propagate attributes to embedded linear flow" in { + val builder = + source.traversalBuilder + .setAttributes(Attributes.name("source")) + .append(flow1.traversalBuilder.setAttributes(Attributes.name("flow")), flow1.shape, Keep.left) + .setAttributes(Attributes.name("compositeSource")) + .append(sink.traversalBuilder.setAttributes(Attributes.name("sink")), sink.shape, Keep.left) + .setAttributes(Attributes.name("test")) + + val mat = testMaterialize(builder) + + mat.attributesAssignments should ===(List( + sink → (Attributes.name("test") and Attributes.name("sink")), + flow1 → (Attributes.name("test") and Attributes.name("compositeSource") and Attributes.name("flow")), + source → (Attributes.name("test") and Attributes.name("compositeSource") and Attributes.name("source")) + )) + } + + "propagate attributes to embedded composite sink" in { + val builder = + source.traversalBuilder + .setAttributes(Attributes.name("source")) + .append(flow1.traversalBuilder.setAttributes(Attributes.name("flow")), flow1.shape, Keep.left) + .append(compositeSink.traversalBuilder.setAttributes(Attributes.name("sink")), compositeSink.shape, Keep.left) + .setAttributes(Attributes.name("test")) + + val mat = testMaterialize(builder) + + mat.attributesAssignments should ===(List( + compositeSink → (Attributes.name("test") and Attributes.name("sink")), + flow1 → (Attributes.name("test") and Attributes.name("flow")), + source → (Attributes.name("test") and Attributes.name("source")) + )) + } + + "propagate attributes to embedded composite source" in { + val builder = + LinearTraversalBuilder.empty() + .append( + compositeSource.traversalBuilder + .setAttributes(Attributes.name("source")), + compositeSource.shape, + Keep.left) + .setAttributes(Attributes.name("source-outer")) + .append(flow1.traversalBuilder.setAttributes(Attributes.name("flow")), flow1.shape, Keep.left) + .append(sink.traversalBuilder.setAttributes(Attributes.name("sink")), compositeSink.shape, Keep.left) + .setAttributes(Attributes.name("test")) + + val mat = testMaterialize(builder) + + mat.attributesAssignments should ===(List( + sink → (Attributes.name("test") and Attributes.name("sink")), + flow1 → (Attributes.name("test") and Attributes.name("flow")), + compositeSource → (Attributes.name("test") and Attributes.name("source-outer") and Attributes.name("source")) + )) + } + + "propagate attributes to embedded composite flow" in { + val builder = + source.traversalBuilder + .setAttributes(Attributes.name("source")) + .append(compositeFlow1.traversalBuilder.setAttributes(Attributes.name("flow")), compositeFlow1.shape, Keep.left) + .append(sink.traversalBuilder.setAttributes(Attributes.name("sink")), compositeSink.shape, Keep.left) + .setAttributes(Attributes.name("test")) + + val mat = testMaterialize(builder) + + mat.attributesAssignments should ===(List( + sink → (Attributes.name("test") and Attributes.name("sink")), + compositeFlow1 → (Attributes.name("test") and Attributes.name("flow")), + source → (Attributes.name("test") and Attributes.name("source")) + )) + } + + "properly append a Source to empty linear" in { + val builder = + LinearTraversalBuilder.empty() + .append(source.traversalBuilder, source.shape, Keep.right) + .append(sink.traversalBuilder, sink.shape, Keep.right) + + val mat = testMaterialize(builder) + + mat.connections should ===(1) + mat.outlets(0) should ===(source.out) + mat.inlets(0) should ===(sink.in) + } + + "properly append a Sink to empty linear" in { + val nestedSink = + LinearTraversalBuilder.empty() + .append(sink.traversalBuilder, sink.shape, Keep.right) + + val builder = + source.traversalBuilder + .append(nestedSink, sink.shape, Keep.right) + + val mat = testMaterialize(builder) + + mat.connections should ===(1) + mat.outlets(0) should ===(source.out) + mat.inlets(0) should ===(sink.in) + } + + "properly append a Flow to empty linear" in { + val nestedFlow = + LinearTraversalBuilder.empty() + .append(flow1.traversalBuilder, flow1.shape, Keep.right) + + val builder = + source.traversalBuilder + .append(nestedFlow, flow1.shape, Keep.right) + .append(sink.traversalBuilder, sink.shape, Keep.right) + + val mat = testMaterialize(builder) + + mat.connections should ===(2) + mat.outlets(1) should ===(source.out) + mat.inlets(1) should ===(flow1.in) + mat.outlets(0) should ===(flow1.out) + mat.inlets(0) should ===(sink.in) + } + + "properly append a composite Source to empty linear" in { + val builder = + LinearTraversalBuilder.empty() + .append(compositeSource.traversalBuilder, compositeSource.shape, Keep.right) + .append(sink.traversalBuilder, sink.shape, Keep.right) + + val mat = testMaterialize(builder) + + mat.connections should ===(1) + mat.outlets(0) should ===(compositeSource.out) + mat.inlets(0) should ===(sink.in) + } + + "properly append a composite Sink to empty linear" in { + val nestedSink = + LinearTraversalBuilder.empty() + .append(compositeSink.traversalBuilder, compositeSink.shape, Keep.right) + + val builder = + source.traversalBuilder + .append(nestedSink, compositeSink.shape, Keep.right) + + val mat = testMaterialize(builder) + + mat.connections should ===(1) + mat.outlets(0) should ===(source.out) + mat.inlets(0) should ===(compositeSink.in) + } + + "properly append a composite Flow to empty linear" in { + val nestedFlow = + LinearTraversalBuilder.empty() + .append(compositeFlow1.traversalBuilder, compositeFlow1.shape, Keep.right) + + val builder = + source.traversalBuilder + .append(nestedFlow, compositeFlow1.shape, Keep.right) + .append(sink.traversalBuilder, sink.shape, Keep.right) + + val mat = testMaterialize(builder) + + mat.connections should ===(2) + mat.outlets(1) should ===(source.out) + mat.inlets(1) should ===(compositeFlow1.in) + mat.outlets(0) should ===(compositeFlow1.out) + mat.inlets(0) should ===(sink.in) + } + + } + +} diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/StreamLayoutSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/StreamLayoutSpec.scala deleted file mode 100644 index 99c7c7fdd9..0000000000 --- a/akka-stream-tests/src/test/scala/akka/stream/impl/StreamLayoutSpec.scala +++ /dev/null @@ -1,286 +0,0 @@ -/** - * Copyright (C) 2015-2017 Lightbend Inc. - */ -package akka.stream.impl - -import akka.stream._ -import akka.stream.scaladsl._ -import akka.stream.testkit.StreamSpec -import org.reactivestreams.{ Publisher, Subscriber, Subscription } -import org.scalatest.concurrent.PatienceConfiguration.Timeout - -import scala.concurrent.duration._ - -class StreamLayoutSpec extends StreamSpec { - import StreamLayout._ - - def testAtomic(inPortCount: Int, outPortCount: Int): Module = new AtomicModule { - override val shape = AmorphousShape(List.fill(inPortCount)(Inlet("")), List.fill(outPortCount)(Outlet(""))) - override def replaceShape(s: Shape): Module = ??? - - override def carbonCopy: Module = ??? - - override def attributes: Attributes = Attributes.none - override def withAttributes(attributes: Attributes): Module = this - } - - def testStage(): Module = testAtomic(1, 1) - def testSource(): Module = testAtomic(0, 1) - def testSink(): Module = testAtomic(1, 0) - - implicit val materializer = ActorMaterializer(ActorMaterializerSettings(system).withAutoFusing(false)) - - "StreamLayout" must { - - "be able to model simple linear stages" in { - val stage1 = testStage() - - stage1.inPorts.size should be(1) - stage1.outPorts.size should be(1) - stage1.isRunnable should be(false) - stage1.isFlow should be(true) - stage1.isSink should be(false) - stage1.isSource should be(false) - - val stage2 = testStage() - val flow12 = stage1.compose(stage2, Keep.none).wire(stage1.outPorts.head, stage2.inPorts.head) - - flow12.inPorts should be(stage1.inPorts) - flow12.outPorts should be(stage2.outPorts) - flow12.isRunnable should be(false) - flow12.isFlow should be(true) - flow12.isSink should be(false) - flow12.isSource should be(false) - - val source0 = testSource() - source0.inPorts.size should be(0) - source0.outPorts.size should be(1) - source0.isRunnable should be(false) - source0.isFlow should be(false) - source0.isSink should be(false) - source0.isSource should be(true) - - val sink3 = testSink() - sink3.inPorts.size should be(1) - sink3.outPorts.size should be(0) - sink3.isRunnable should be(false) - sink3.isFlow should be(false) - sink3.isSink should be(true) - sink3.isSource should be(false) - - val source012 = source0.compose(flow12, Keep.none).wire(source0.outPorts.head, flow12.inPorts.head) - source012.inPorts.size should be(0) - source012.outPorts should be(flow12.outPorts) - source012.isRunnable should be(false) - source012.isFlow should be(false) - source012.isSink should be(false) - source012.isSource should be(true) - - val sink123 = flow12.compose(sink3, Keep.none).wire(flow12.outPorts.head, sink3.inPorts.head) - sink123.inPorts should be(flow12.inPorts) - sink123.outPorts.size should be(0) - sink123.isRunnable should be(false) - sink123.isFlow should be(false) - sink123.isSink should be(true) - sink123.isSource should be(false) - - val runnable0123a = source0.compose(sink123, Keep.none).wire(source0.outPorts.head, sink123.inPorts.head) - val runnable0123b = source012.compose(sink3, Keep.none).wire(source012.outPorts.head, sink3.inPorts.head) - - val runnable0123c = - source0 - .compose(flow12, Keep.none).wire(source0.outPorts.head, flow12.inPorts.head) - .compose(sink3, Keep.none).wire(flow12.outPorts.head, sink3.inPorts.head) - - runnable0123a.inPorts.size should be(0) - runnable0123a.outPorts.size should be(0) - runnable0123a.isRunnable should be(true) - runnable0123a.isFlow should be(false) - runnable0123a.isSink should be(false) - runnable0123a.isSource should be(false) - } - - "be able to materialize linear layouts" in { - val source = testSource() - val stage1 = testStage() - val stage2 = testStage() - val sink = testSink() - - val runnable = source.compose(stage1, Keep.none).wire(source.outPorts.head, stage1.inPorts.head) - .compose(stage2, Keep.none).wire(stage1.outPorts.head, stage2.inPorts.head) - .compose(sink, Keep.none).wire(stage2.outPorts.head, sink.inPorts.head) - - checkMaterialized(runnable) - } - - val tooDeepForStack = 50000 - - "fail fusing when value computation is too complex" in { - // this tests that the canary in to coal mine actually works - val g = (1 to tooDeepForStack) - .foldLeft(Flow[Int].mapMaterializedValue(_ ⇒ 1)) { (flow, i) ⇒ - flow.mapMaterializedValue(x ⇒ x + i) - } - a[StackOverflowError] shouldBe thrownBy { - Fusing.aggressive(g) - } - } - - // Seen tests run in 9-10 seconds, these test cases are heavy on the GC - val veryPatient = Timeout(20.seconds) - - "not fail materialization when building a large graph with simple computation" when { - - "starting from a Source" in { - val g = (1 to tooDeepForStack) - .foldLeft(Source.single(42).mapMaterializedValue(_ ⇒ 1))( - (f, i) ⇒ f.map(identity)) - val (mat, fut) = g.toMat(Sink.seq)(Keep.both).run() - mat should ===(1) - fut.futureValue(veryPatient) should ===(List(42)) - } - - "starting from a Flow" in { - val g = (1 to tooDeepForStack).foldLeft(Flow[Int])((f, i) ⇒ f.map(identity)) - val (mat, fut) = g.runWith(Source.single(42).mapMaterializedValue(_ ⇒ 1), Sink.seq) - mat should ===(1) - fut.futureValue(veryPatient) should ===(List(42)) - } - - "using .via" in { - val g = (1 to tooDeepForStack) - .foldLeft(Source.single(42).mapMaterializedValue(_ ⇒ 1))( - (f, i) ⇒ f.via(Flow[Int].map(identity))) - val (mat, fut) = g.toMat(Sink.seq)(Keep.both).run() - mat should ===(1) - fut.futureValue(veryPatient) should ===(List(42)) - } - } - - "not fail fusing & materialization when building a large graph with simple computation" when { - - "starting from a Source" in { - val g = Source fromGraph Fusing.aggressive((1 to tooDeepForStack) - .foldLeft(Source.single(42).mapMaterializedValue(_ ⇒ 1))( - (f, i) ⇒ f.map(identity))) - val (mat, fut) = g.toMat(Sink.seq)(Keep.both).run() - mat should ===(1) - fut.futureValue(veryPatient) should ===(List(42)) - } - - "starting from a Flow" in { - val g = Flow fromGraph Fusing.aggressive((1 to tooDeepForStack).foldLeft(Flow[Int])((f, i) ⇒ f.map(identity))) - val (mat, fut) = g.runWith(Source.single(42).mapMaterializedValue(_ ⇒ 1), Sink.seq) - mat should ===(1) - fut.futureValue(veryPatient) should ===(List(42)) - } - - "using .via" in { - val g = Source fromGraph Fusing.aggressive((1 to tooDeepForStack) - .foldLeft(Source.single(42).mapMaterializedValue(_ ⇒ 1))( - (f, i) ⇒ f.via(Flow[Int].map(identity)))) - val (mat, fut) = g.toMat(Sink.seq)(Keep.both).run() - mat should ===(1) - fut.futureValue(veryPatient) should ===(List(42)) - } - } - - } - - case class TestPublisher(owner: Module, port: OutPort) extends Publisher[Any] with Subscription { - var downstreamModule: Module = _ - var downstreamPort: InPort = _ - - override def subscribe(s: Subscriber[_ >: Any]): Unit = s match { - case TestSubscriber(o, p) ⇒ - downstreamModule = o - downstreamPort = p - s.onSubscribe(this) - } - - override def request(n: Long): Unit = () - override def cancel(): Unit = () - } - - case class TestSubscriber(owner: Module, port: InPort) extends Subscriber[Any] { - var upstreamModule: Module = _ - var upstreamPort: OutPort = _ - - override def onSubscribe(s: Subscription): Unit = s match { - case TestPublisher(o, p) ⇒ - upstreamModule = o - upstreamPort = p - } - - override def onError(t: Throwable): Unit = () - override def onComplete(): Unit = () - override def onNext(t: Any): Unit = () - } - - class FlatTestMaterializer(_module: Module) extends MaterializerSession(_module, Attributes()) { - var publishers = Vector.empty[TestPublisher] - var subscribers = Vector.empty[TestSubscriber] - - override protected def materializeAtomic(atomic: AtomicModule, effectiveAttributes: Attributes, - matVal: java.util.Map[Module, Any]): Unit = { - for (inPort ← atomic.inPorts) { - val subscriber = TestSubscriber(atomic, inPort) - subscribers :+= subscriber - assignPort(inPort, subscriber) - } - for (outPort ← atomic.outPorts) { - val publisher = TestPublisher(atomic, outPort) - publishers :+= publisher - assignPort(outPort, publisher) - } - } - } - - def checkMaterialized(topLevel: Module): (Set[TestPublisher], Set[TestSubscriber]) = { - val materializer = new FlatTestMaterializer(topLevel) - materializer.materialize() - materializer.publishers.isEmpty should be(false) - materializer.subscribers.isEmpty should be(false) - - materializer.subscribers.size should be(materializer.publishers.size) - - val inToSubscriber: Map[InPort, TestSubscriber] = materializer.subscribers.map(s ⇒ s.port → s).toMap - val outToPublisher: Map[OutPort, TestPublisher] = materializer.publishers.map(s ⇒ s.port → s).toMap - - for (publisher ← materializer.publishers) { - publisher.owner.isAtomic should be(true) - topLevel.upstreams(publisher.downstreamPort) should be(publisher.port) - } - - for (subscriber ← materializer.subscribers) { - subscriber.owner.isAtomic should be(true) - topLevel.downstreams(subscriber.upstreamPort) should be(subscriber.port) - } - - def getAllAtomic(module: Module): Set[Module] = { - val (atomics, composites) = module.subModules.partition(_.isAtomic) - atomics ++ composites.flatMap(getAllAtomic) - } - - val allAtomic = getAllAtomic(topLevel) - - for (atomic ← allAtomic) { - for (in ← atomic.inPorts; subscriber = inToSubscriber(in)) { - subscriber.owner should be(atomic) - subscriber.upstreamPort should be(topLevel.upstreams(in)) - subscriber.upstreamModule.outPorts.exists(outToPublisher(_).downstreamPort == in) - } - for (out ← atomic.outPorts; publisher = outToPublisher(out)) { - publisher.owner should be(atomic) - publisher.downstreamPort should be(topLevel.downstreams(out)) - publisher.downstreamModule.inPorts.exists(inToSubscriber(_).upstreamPort == out) - } - } - - materializer.publishers.distinct.size should be(materializer.publishers.size) - materializer.subscribers.distinct.size should be(materializer.subscribers.size) - - (materializer.publishers.toSet, materializer.subscribers.toSet) - } - -} diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/TraversalBuilderSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/TraversalBuilderSpec.scala new file mode 100644 index 0000000000..58151ebd60 --- /dev/null +++ b/akka-stream-tests/src/test/scala/akka/stream/impl/TraversalBuilderSpec.scala @@ -0,0 +1,518 @@ +/** + * Copyright (C) 2015-2016 Lightbend Inc. + */ + +package akka.stream.impl + +import akka.NotUsed +import akka.stream._ +import akka.stream.impl.TraversalBuilder.printTraversal +import akka.stream.impl.TraversalTestUtils._ +import akka.stream.scaladsl.{ BidiFlow, Flow, Keep, Sink, Source } +import akka.stream.testkit.{ TestPublisher, TestSubscriber } +import akka.testkit.AkkaSpec + +import scala.concurrent.Await + +class TraversalBuilderSpec extends AkkaSpec { + + "CompositeTraversalBuilder" must { + val source = new CompositeTestSource + val sink = new CompositeTestSink + val flow1 = new CompositeTestFlow("1") + val flow2 = new CompositeTestFlow("2") + + // ADD closed shape, (and composite closed shape) + + "work with a single Source and Sink" in { + val builder = + source.traversalBuilder + .add(sink.traversalBuilder, sink.shape, Keep.left) + .wire(source.out, sink.in) + + val mat = testMaterialize(builder) + + mat.connections should ===(1) + + mat.outlets(0) should ===(source.out) + mat.inlets(0) should ===(sink.in) + } + + "work with a nested Source and Sink" in { + val nestedBuilder = + TraversalBuilder.empty() + .add(source.traversalBuilder, source.shape, Keep.left) + + val builder = + sink.traversalBuilder + .add(nestedBuilder, source.shape, Keep.left) + .wire(source.out, sink.in) + + val mat = testMaterialize(builder) + + mat.connections should ===(1) + mat.outlets(0) should ===(source.out) + mat.inlets(0) should ===(sink.in) + } + + "work with a remapped Source and Sink" in { + val remappedShape = SourceShape(Outlet[Any]("remapped.out")) + remappedShape.out.mappedTo = source.out + + val builder = sink.traversalBuilder + .add(source.traversalBuilder, remappedShape, Keep.left) + .wire(remappedShape.out, sink.in) + + val mat = testMaterialize(builder) + + mat.connections should ===(1) + mat.outlets(0) should ===(source.out) + mat.inlets(0) should ===(sink.in) + } + + "work with two Flows" in { + val builder = source.traversalBuilder + .add(flow1.traversalBuilder, flow1.shape, Keep.left) + .add(flow2.traversalBuilder, flow2.shape, Keep.left) + .add(sink.traversalBuilder, sink.shape, Keep.left) + .wire(source.out, flow1.in) + .wire(flow1.out, flow2.in) + .wire(flow2.out, sink.in) + + val mat = testMaterialize(builder) + + mat.connections should ===(3) + mat.outlets(0) should ===(source.out) + mat.inlets(0) should ===(flow1.in) + mat.outlets(1) should ===(flow1.out) + mat.inlets(1) should ===(flow2.in) + mat.outlets(2) should ===(flow2.out) + mat.inlets(2) should ===(sink.in) + } + + "work with two Flows wired in opposite order" in { + val builder = source.traversalBuilder + .add(flow1.traversalBuilder, flow1.shape, Keep.left) + .add(flow2.traversalBuilder, flow2.shape, Keep.left) + .add(sink.traversalBuilder, sink.shape, Keep.left) + .wire(flow2.out, sink.in) + .wire(flow1.out, flow2.in) + .wire(source.out, flow1.in) + + val mat = testMaterialize(builder) + + mat.connections should ===(3) + mat.outlets(0) should ===(source.out) + mat.inlets(0) should ===(flow1.in) + mat.outlets(1) should ===(flow1.out) + mat.inlets(1) should ===(flow2.in) + mat.outlets(2) should ===(flow2.out) + mat.inlets(2) should ===(sink.in) + } + + "work with two Flows wired in an irregular order" in { + val builder = source.traversalBuilder + .add(sink.traversalBuilder, sink.shape, Keep.left) + .add(flow2.traversalBuilder, flow2.shape, Keep.left) + .wire(flow2.out, sink.in) + .add(flow1.traversalBuilder, flow1.shape, Keep.left) + .wire(source.out, flow1.in) + .wire(flow1.out, flow2.in) + + val mat = testMaterialize(builder) + + mat.connections should ===(3) + mat.outlets(0) should ===(flow2.out) + mat.inlets(0) should ===(sink.in) + mat.outlets(1) should ===(flow1.out) + mat.inlets(1) should ===(flow2.in) + mat.outlets(2) should ===(source.out) + mat.inlets(2) should ===(flow1.in) + } + + "work with a Flow wired to its imported self" in { + val remappedShape = flow1.shape.deepCopy() + + val builder = source.traversalBuilder + .add(flow1.traversalBuilder, flow1.shape, Keep.left) + .add(flow1.traversalBuilder, remappedShape, Keep.left) + .add(sink.traversalBuilder, sink.shape, Keep.left) + .wire(source.out, flow1.in) + .wire(flow1.out, remappedShape.in) + .wire(remappedShape.out, sink.in) + + val mat = testMaterialize(builder) + + mat.connections should ===(3) + mat.outlets(0) should ===(source.out) + mat.inlets(0) should ===(flow1.in) + mat.outlets(1) should ===(flow1.out) + mat.inlets(1) should ===(flow1.in) + mat.outlets(2) should ===(flow1.out) + mat.inlets(2) should ===(sink.in) + } + + "work with a nested Flow chain" in { + val nestedFlowShape = FlowShape(flow1.in, flow2.out) + + val nestedFlows = + flow1.traversalBuilder + .add(flow2.traversalBuilder, flow2.shape, Keep.left) + .wire(flow1.out, flow2.in) + + val builder = source.traversalBuilder + .add(nestedFlows, nestedFlowShape, Keep.left) + .add(sink.traversalBuilder, sink.shape, Keep.left) + .wire(source.out, flow1.in) + .wire(flow2.out, sink.in) + + val mat = testMaterialize(builder) + + mat.connections should ===(3) + mat.outlets(0) should ===(source.out) + mat.inlets(0) should ===(flow1.in) + mat.outlets(1) should ===(flow1.out) + mat.inlets(1) should ===(flow2.in) + mat.outlets(2) should ===(flow2.out) + mat.inlets(2) should ===(sink.in) + } + + "work with a nested Flow chain, imported" in { + val importedFlowShape = FlowShape(Inlet[Any]("imported.in"), Outlet[Any]("imported.out")) + importedFlowShape.in.mappedTo = flow1.in + importedFlowShape.out.mappedTo = flow2.out + + val nestedFlows = + flow1.traversalBuilder + .add(flow2.traversalBuilder, flow2.shape, Keep.left) + .wire(flow1.out, flow2.in) + + val builder = source.traversalBuilder + .add(nestedFlows, importedFlowShape, Keep.left) + .add(sink.traversalBuilder, sink.shape, Keep.left) + .wire(source.out, importedFlowShape.in) + .wire(importedFlowShape.out, sink.in) + + val mat = testMaterialize(builder) + + mat.connections should ===(3) + mat.outlets(0) should ===(source.out) + mat.inlets(0) should ===(flow1.in) + mat.outlets(1) should ===(flow1.out) + mat.inlets(1) should ===(flow2.in) + mat.outlets(2) should ===(flow2.out) + mat.inlets(2) should ===(sink.in) + } + + "work with a Flow wired to self" in { + val builder = flow1.traversalBuilder.wire(flow1.out, flow1.in) + + val mat = testMaterialize(builder) + + mat.connections should ===(1) + mat.outlets(0) should ===(flow1.out) + mat.inlets(0) should ===(flow1.in) + } + + "properly materialize empty builder" in { + val builder = TraversalBuilder.empty() + + val mat = testMaterialize(builder) + mat.connections should ===(0) + mat.outlets.length should ===(0) + mat.inlets.length should ===(0) + mat.matValue should ===(NotUsed) + } + + "properly propagate materialized value with Keep.left" in { + val builder = + source.traversalBuilder + .add(sink.traversalBuilder, sink.shape, Keep.left) + .wire(source.out, sink.in) + + val mat = testMaterialize(builder) + + mat.matValue should ===("TestSource") + } + + "keep mapped materialized value of empty builder" in { + val builder = + TraversalBuilder.empty() + .transformMat[NotUsed, String](_ ⇒ "NOTUSED") + .add(source.traversalBuilder, source.shape, Keep.left) + .add(sink.traversalBuilder, sink.shape, Keep.left) + .wire(source.out, sink.in) + + val mat = testMaterialize(builder) + + mat.matValue should ===("NOTUSED") + } + + "properly propagate materialized value with Keep.right" in { + val builder = + source.traversalBuilder + .add(sink.traversalBuilder, sink.shape, Keep.right) + .wire(source.out, sink.in) + + val mat = testMaterialize(builder) + + mat.matValue should ===("TestSink") + } + + "properly propagate materialized value with Keep.both" in { + val builder = + source.traversalBuilder + .add(sink.traversalBuilder, sink.shape, Keep.both) + .wire(source.out, sink.in) + + val mat = testMaterialize(builder) + + mat.matValue should ===(("TestSource", "TestSink")) + } + + "properly propagate materialized value with Keep.left with Flow in middle" in { + val builder = source.traversalBuilder + .add(flow1.traversalBuilder, flow1.shape, Keep.left) + .add(sink.traversalBuilder, sink.shape, Keep.left) + .wire(source.out, flow1.in) + .wire(flow1.out, sink.in) + + val mat = testMaterialize(builder) + + mat.matValue should ===("TestSource") + } + + "properly propagate materialized value with Keep.right with Flow in middle (1)" in { + val builder = source.traversalBuilder + .add(flow1.traversalBuilder, flow1.shape, Keep.right) + .add(sink.traversalBuilder, sink.shape, Keep.left) + .wire(source.out, flow1.in) + .wire(flow1.out, sink.in) + + val mat = testMaterialize(builder) + + mat.matValue should ===("TestFlow1") + } + + "properly propagate materialized value with Keep.right with Flow in middle (2)" in { + val builder = source.traversalBuilder + .add(flow1.traversalBuilder, flow1.shape, Keep.left) + .add(sink.traversalBuilder, sink.shape, Keep.right) + .wire(source.out, flow1.in) + .wire(flow1.out, sink.in) + + val mat = testMaterialize(builder) + + mat.matValue should ===("TestSink") + } + + "properly propagate materialized value with Keep.both with Flow in middle (1)" in { + val builder = source.traversalBuilder + .add(flow1.traversalBuilder, flow1.shape, Keep.both) + .add(sink.traversalBuilder, sink.shape, Keep.left) + .wire(source.out, flow1.in) + .wire(flow1.out, sink.in) + + val mat = testMaterialize(builder) + + mat.matValue should ===(("TestSource", "TestFlow1")) + } + + "properly propagate materialized value with Keep.both with Flow in middle (2)" in { + val builder = source.traversalBuilder + .add(flow1.traversalBuilder, flow1.shape, Keep.both) + .add(sink.traversalBuilder, sink.shape, Keep.both) + .wire(source.out, flow1.in) + .wire(flow1.out, sink.in) + + val mat = testMaterialize(builder) + + mat.matValue should ===((("TestSource", "TestFlow1"), "TestSink")) + } + + "properly map materialized value" in { + val builder = source.traversalBuilder + .add(flow1.traversalBuilder, flow1.shape, Keep.right) + .add(sink.traversalBuilder, sink.shape, Keep.left) + .wire(source.out, flow1.in) + .wire(flow1.out, sink.in) + .transformMat[String, String]("MAPPED: " + _) + + val mat = testMaterialize(builder) + + mat.matValue should ===("MAPPED: TestFlow1") + } + + "properly map materialized value (nested)" in { + val flowBuilder = + flow1.traversalBuilder + .transformMat[String, String]("M1: " + _) + + val builder = source.traversalBuilder + .add(flowBuilder, flow1.shape, Keep.right) + .add(sink.traversalBuilder, sink.shape, Keep.left) + .wire(source.out, flow1.in) + .wire(flow1.out, sink.in) + .transformMat[String, String]("M2: " + _) + + val mat = testMaterialize(builder) + + mat.matValue should ===("M2: M1: TestFlow1") + } + + "properly set attributes for whole chain" in { + val builder = source.traversalBuilder + .add(sink.traversalBuilder, sink.shape, Keep.left) + .wire(source.out, sink.in) + .setAttributes(Attributes.name("test")) + + val mat = testMaterialize(builder) + + mat.attributesAssignments should ===(List( + source → (Attributes.name("test") and Attributes.name("testSource")), + sink → (Attributes.name("test") and Attributes.name("testSink")) + )) + } + + "overwrite last attributes until embedded in other builder" in { + val innerBuilder = source.traversalBuilder + .add(sink.traversalBuilder.setAttributes(Attributes.name("testSinkB")), sink.shape, Keep.left) + .wire(source.out, sink.in) + .setAttributes(Attributes.name("test")) + .setAttributes(Attributes.name("test2")) + + val builder = + TraversalBuilder.empty() + .add(innerBuilder, ClosedShape, Keep.left) + .setAttributes(Attributes.name("outer")) + .setAttributes(Attributes.name("outer2")) + + val mat = testMaterialize(builder) + + mat.attributesAssignments should ===(List( + source → (Attributes.name("outer2") and Attributes.name("test2") and Attributes.name("testSource")), + sink → (Attributes.name("outer2") and Attributes.name("test2") and Attributes.name("testSinkB")) + )) + } + + "propagate attributes to embedded flow" in { + val flowBuilder = + flow1.traversalBuilder + .setAttributes(Attributes.name("flow")) + + val builder = source.traversalBuilder + .add(flowBuilder, flow1.shape, Keep.left) + .add(sink.traversalBuilder, sink.shape, Keep.left) + .wire(source.out, flow1.in) + .wire(flow1.out, sink.in) + .setAttributes(Attributes.name("test")) + + val mat = testMaterialize(builder) + + mat.attributesAssignments should ===(List( + source → (Attributes.name("test") and Attributes.name("testSource")), + flow1 → (Attributes.name("test") and Attributes.name("flow")), + sink → (Attributes.name("test") and Attributes.name("testSink")) + )) + } + + //TODO: Dummy test cases just for smoke-testing. Should be removed. + + "foo" in { + implicit val mat = PhasedFusingActorMaterializer(ActorMaterializerSettings(system).withSyncProcessingLimit(5000)) + import scala.concurrent.duration._ + + val graph = Source.repeat(1).take(10).toMat(Sink.fold(0)(_ + _))(Keep.right) + + Await.result(graph.run(), 3.seconds) should ===(10) + } + + "islands 1" in { + implicit val mat = PhasedFusingActorMaterializer(ActorMaterializerSettings(system).withSyncProcessingLimit(5000)) + val sub = TestSubscriber.probe[Int]() + val graph = Source.repeat(1).take(10).toMat(Sink.asPublisher(false))(Keep.right) + + val pub = graph.run().subscribe(sub) + + sub.request(10) + sub.expectNextN(List(1, 1, 1, 1, 1, 1, 1, 1, 1, 1)) + sub.expectComplete() + } + + "islands 2" in { + implicit val mat = PhasedFusingActorMaterializer(ActorMaterializerSettings(system).withSyncProcessingLimit(5000)) + val pub = TestPublisher.probe[Int]() + import scala.concurrent.duration._ + + val graph = Source.asSubscriber[Int].toMat(Sink.fold(0)(_ + _))(Keep.both) + + val (sub, future) = graph.run() + pub.subscribe(sub) + + pub.sendNext(0) + pub.sendNext(1) + pub.sendNext(2) + pub.sendNext(3) + pub.sendComplete() + + Await.result(future, 3.seconds) should ===(6) + } + + "islands 3" in { + implicit val mat = PhasedFusingActorMaterializer(ActorMaterializerSettings(system).withSyncProcessingLimit(5000)) + val sub = TestSubscriber.probe[Int]() + Source + .repeat(1) + .take(10) + .runWith(Sink.fromSubscriber(sub)) + + sub.request(10) + sub.expectNextN(List(1, 1, 1, 1, 1, 1, 1, 1, 1, 1)) + sub.expectComplete() + } + + "islands 4" in { + implicit val mat = PhasedFusingActorMaterializer(ActorMaterializerSettings(system).withSyncProcessingLimit(5000)) + val pub = TestPublisher.probe[Int]() + import scala.concurrent.duration._ + + val future = Source.fromPublisher(pub).runWith(Sink.fold(0)(_ + _)) + pub.sendNext(0) + pub.sendNext(1) + pub.sendNext(2) + pub.sendNext(3) + pub.sendComplete() + + Await.result(future, 3.seconds) should ===(6) + } + + "bidiflow1" in { + implicit val mat = PhasedFusingActorMaterializer(ActorMaterializerSettings(system).withSyncProcessingLimit(5000)) + val flow1 = Flow.fromGraph(new fusing.Map((x: Int) ⇒ x + 1)) + val flow2 = Flow.fromGraph(new fusing.Map((x: Int) ⇒ x + 1)) + + val bidi = BidiFlow.fromFlowsMat(flow1, flow2)(Keep.none) + + val flow = bidi.join(Flow[Int]) + + Source.single(1).via(flow).runWith(Sink.ignore) + } + + "bidiflow reverse" in { + implicit val mat = PhasedFusingActorMaterializer(ActorMaterializerSettings(system).withSyncProcessingLimit(5000)) + val flow1 = Flow.fromGraph(new fusing.Map((x: Int) ⇒ x + 1)) + val flow2 = Flow.fromGraph(new fusing.Map((x: Int) ⇒ x + 1)) + + val bidi = BidiFlow.fromFlowsMat(flow1, flow2)(Keep.none) + + val flow = Flow[Int].join(bidi.reversed) + + Source.single(1).via(flow).runWith(Sink.ignore) + } + + // Keep.none + // Test self-closed atomic module + + } + +} diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/TraversalTestUtils.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/TraversalTestUtils.scala new file mode 100644 index 0000000000..f67c5b5162 --- /dev/null +++ b/akka-stream-tests/src/test/scala/akka/stream/impl/TraversalTestUtils.scala @@ -0,0 +1,168 @@ +/** + * Copyright (C) 2015-2016 Lightbend Inc. + */ + +package akka.stream.impl + +import akka.NotUsed +import akka.stream.impl.StreamLayout.AtomicModule +import akka.stream._ + +object TraversalTestUtils { + + // --- These test classes do not use the optimized linear builder, for testing the composite builder instead + class CompositeTestSource extends AtomicModule[SourceShape[Any], Any] { + val out = Outlet[Any]("testSourceC.out") + override val shape: Shape = SourceShape(out) + override val traversalBuilder = TraversalBuilder.atomic(this, Attributes.name("testSource")) + + override def withAttributes(attributes: Attributes): AtomicModule[SourceShape[Any], Any] = ??? + override def toString = "TestSource" + } + + class CompositeTestSink extends AtomicModule[SinkShape[Any], Any] { + val in = Inlet[Any]("testSinkC.in") + override val shape: Shape = SinkShape(in) + override val traversalBuilder = TraversalBuilder.atomic(this, Attributes.name("testSink")) + + override def withAttributes(attributes: Attributes): AtomicModule[SinkShape[Any], Any] = ??? + override def toString = "TestSink" + } + + class CompositeTestFlow(tag: String) extends AtomicModule[FlowShape[Any, Any], Any] { + val in = Inlet[Any](s"testFlowC$tag.in") + val out = Outlet[Any](s"testFlowC$tag.out") + override val shape: Shape = FlowShape(in, out) + override val traversalBuilder = TraversalBuilder.atomic(this, Attributes.name(s"testFlow$tag")) + + override def withAttributes(attributes: Attributes): AtomicModule[FlowShape[Any, Any], Any] = ??? + override def toString = s"TestFlow$tag" + } + + // --- These test classes DO use the optimized linear builder, for testing the composite builder instead + class LinearTestSource extends AtomicModule[SourceShape[Any], Any] { + val out = Outlet[Any]("testSource.out") + override val shape: Shape = SourceShape(out) + override val traversalBuilder = LinearTraversalBuilder.fromModule(this, Attributes.name("testSource")) + + override def withAttributes(attributes: Attributes): AtomicModule[SourceShape[Any], Any] = ??? + override def toString = "TestSource" + } + + class LinearTestSink extends AtomicModule[SinkShape[Any], Any] { + val in = Inlet[Any]("testSink.in") + override val shape: Shape = SinkShape(in) + override val traversalBuilder = LinearTraversalBuilder.fromModule(this, Attributes.name("testSink")) + + override def withAttributes(attributes: Attributes): AtomicModule[SinkShape[Any], Any] = ??? + override def toString = "TestSink" + } + + class LinearTestFlow(tag: String) extends AtomicModule[FlowShape[Any, Any], Any] { + val in = Inlet[Any](s"testFlow$tag.in") + val out = Outlet[Any](s"testFlow$tag.out") + override val shape: Shape = FlowShape(in, out) + override val traversalBuilder = LinearTraversalBuilder.fromModule(this, Attributes.name(s"testFlow$tag")) + + override def withAttributes(attributes: Attributes): AtomicModule[FlowShape[Any, Any], Any] = ??? + override def toString = s"TestFlow$tag" + } + + class MaterializationResult( + val connections: Int, + val inlets: Array[InPort], + val outlets: Array[OutPort], + val matValue: Any, + val attributesAssignments: List[(AtomicModule[Shape, Any], Attributes)] + ) { + + override def toString = { + outlets.iterator.zip(inlets.iterator).mkString("connections: ", ", ", "") + } + } + + /** + * This test method emulates a materialization run. It simply puts input and output ports into slots of an Array. + * After running this method, it can be tested that ports that are meant to be wired together have been put into + * corresponding slots of the [[MaterializationResult]]. + */ + def testMaterialize(b: TraversalBuilder): MaterializationResult = { + require(b.isTraversalComplete, "Traversal builder must be complete") + + var attributesResult: List[(AtomicModule[Shape, Any], Attributes)] = Nil + + val connections = b.inSlots + val inlets = Array.ofDim[InPort](connections) + val outlets = Array.ofDim[OutPort](connections) + + // Track next assignable number for input ports + var inOffs = 0 + + var current: Traversal = b.traversal + val traversalStack = new java.util.ArrayList[Traversal](16) + traversalStack.add(current) + + val matValueStack = new java.util.ArrayDeque[Any](8) + val attributesStack = new java.util.ArrayDeque[Attributes](8) + attributesStack.addLast(Attributes.none) + + // Due to how Concat works, we need a stack. This probably can be optimized for the most common cases. + while (!traversalStack.isEmpty) { + current = traversalStack.remove(traversalStack.size() - 1) + + while (current ne EmptyTraversal) { + var nextStep: Traversal = EmptyTraversal + + current match { + case MaterializeAtomic(mod, outToSlot) ⇒ + var i = 0 + val inletsIter = mod.shape.inlets.iterator + while (inletsIter.hasNext) { + val in = inletsIter.next() + inlets(inOffs + i) = in + i += 1 + } + + val outletsIter = mod.shape.outlets.iterator + while (outletsIter.hasNext) { + val out = outletsIter.next() + outlets(inOffs + outToSlot(out.id)) = out + } + + // Recording attributes assignment results for testing purposes + attributesResult = (mod → attributesStack.getLast) :: attributesResult + + // Dummy materialized value is simply the name of the module + matValueStack.addLast(mod.toString) + inOffs += mod.shape.inlets.size + case Pop ⇒ + matValueStack.removeLast() + case PushNotUsed ⇒ + matValueStack.addLast(NotUsed) + case Transform(f) ⇒ + val prev = matValueStack.removeLast() + val result = f(prev) + matValueStack.addLast(result) + case Compose(f) ⇒ + val second = matValueStack.removeLast() + val first = matValueStack.removeLast() + val result = f(first, second) + matValueStack.addLast(result) + case PushAttributes(attr) ⇒ + attributesStack.addLast(attributesStack.getLast and attr) + case PopAttributes ⇒ + attributesStack.removeLast() + case Concat(first, next) ⇒ + if (next ne EmptyTraversal) traversalStack.add(next) + nextStep = first + case _ ⇒ + } + + current = nextStep + } + } + + new MaterializationResult(connections, inlets, outlets, matValueStack.peekLast(), attributesResult.reverse) + } + +} diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/FileSinkSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/io/FileSinkSpec.scala index 7fc7b6021c..f095fbfa92 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/io/FileSinkSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/io/FileSinkSpec.scala @@ -7,7 +7,7 @@ import java.nio.file.{ Files, Path, StandardOpenOption } import akka.actor.ActorSystem import akka.dispatch.ExecutionContexts -import akka.stream.impl.ActorMaterializerImpl +import akka.stream.impl.PhasedFusingActorMaterializer import akka.stream.impl.StreamSupervisor import akka.stream.impl.StreamSupervisor.Children import akka.stream.scaladsl.{ FileIO, Sink, Source } @@ -109,7 +109,7 @@ class FileSinkSpec extends StreamSpec(UnboundedMailboxConfig) { try { Source.fromIterator(() ⇒ Iterator.continually(TestByteStrings.head)).runWith(FileIO.toPath(f))(materializer) - materializer.asInstanceOf[ActorMaterializerImpl].supervisor.tell(StreamSupervisor.GetChildren, testActor) + materializer.asInstanceOf[PhasedFusingActorMaterializer].supervisor.tell(StreamSupervisor.GetChildren, testActor) val ref = expectMsgType[Children].children.find(_.path.toString contains "fileSink").get assertDispatcher(ref, "akka.stream.default-blocking-io-dispatcher") } finally shutdown(sys) @@ -130,7 +130,7 @@ class FileSinkSpec extends StreamSpec(UnboundedMailboxConfig) { .withAttributes(ActorAttributes.dispatcher("akka.actor.default-dispatcher")) .run()(materializer) - materializer.asInstanceOf[ActorMaterializerImpl].supervisor.tell(StreamSupervisor.GetChildren, testActor) + materializer.asInstanceOf[PhasedFusingActorMaterializer].supervisor.tell(StreamSupervisor.GetChildren, testActor) val ref = expectMsgType[Children].children.find(_.path.toString contains "File").get assertDispatcher(ref, "akka.actor.default-dispatcher") } finally shutdown(sys) diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/FileSourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/io/FileSourceSpec.scala index 72b6e9df46..1feb87e0ea 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/io/FileSourceSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/io/FileSourceSpec.scala @@ -12,8 +12,7 @@ import akka.stream.ActorMaterializer import akka.stream.ActorMaterializerSettings import akka.stream.ActorAttributes import akka.stream.Attributes -import akka.stream.impl.ActorMaterializerImpl -import akka.stream.impl.StreamSupervisor +import akka.stream.impl.{ PhasedFusingActorMaterializer, StreamSupervisor } import akka.stream.impl.StreamSupervisor.Children import akka.stream.io.FileSourceSpec.Settings import akka.stream.scaladsl.{ FileIO, Keep, Sink } @@ -177,7 +176,7 @@ class FileSourceSpec extends StreamSpec(UnboundedMailboxConfig) { try { val p = FileIO.fromPath(manyLines).runWith(TestSink.probe)(materializer) - materializer.asInstanceOf[ActorMaterializerImpl].supervisor.tell(StreamSupervisor.GetChildren, testActor) + materializer.asInstanceOf[PhasedFusingActorMaterializer].supervisor.tell(StreamSupervisor.GetChildren, testActor) val ref = expectMsgType[Children].children.find(_.path.toString contains "fileSource").get try assertDispatcher(ref, "akka.stream.default-blocking-io-dispatcher") finally p.cancel() } finally shutdown(sys) @@ -195,7 +194,7 @@ class FileSourceSpec extends StreamSpec(UnboundedMailboxConfig) { .withAttributes(ActorAttributes.dispatcher("akka.actor.default-dispatcher")) .runWith(TestSink.probe)(materializer) - materializer.asInstanceOf[ActorMaterializerImpl].supervisor.tell(StreamSupervisor.GetChildren, testActor) + materializer.asInstanceOf[PhasedFusingActorMaterializer].supervisor.tell(StreamSupervisor.GetChildren, testActor) val ref = expectMsgType[Children].children.find(_.path.toString contains "File").get try assertDispatcher(ref, "akka.actor.default-dispatcher") finally p.cancel() } finally shutdown(sys) diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/InputStreamSinkSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/io/InputStreamSinkSpec.scala index c6a0f491f8..9548bfc846 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/io/InputStreamSinkSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/io/InputStreamSinkSpec.scala @@ -5,20 +5,23 @@ package akka.stream.io import java.io.{ IOException, InputStream } import java.util.concurrent.TimeoutException + import akka.actor.ActorSystem import akka.stream._ import akka.stream.Attributes.inputBuffer import akka.stream.impl.StreamSupervisor.Children import akka.stream.impl.io.InputStreamSinkStage -import akka.stream.impl.{ ActorMaterializerImpl, StreamSupervisor } +import akka.stream.impl.{ PhasedFusingActorMaterializer, StreamSupervisor } import akka.stream.scaladsl.{ Keep, Source, StreamConverters } import akka.stream.testkit.Utils._ import akka.stream.testkit.scaladsl.TestSource -import akka.stream.testkit.{ StreamSpec, GraphStageMessages, TestSinkStage } +import akka.stream.testkit.{ GraphStageMessages, StreamSpec, TestSinkStage } import akka.testkit.TestProbe import akka.util.ByteString + import scala.concurrent.duration._ import java.util.concurrent.ThreadLocalRandom + import scala.concurrent.{ Await, Future } import scala.util.control.NoStackTrace @@ -203,7 +206,7 @@ class InputStreamSinkSpec extends StreamSpec(UnboundedMailboxConfig) { val materializer = ActorMaterializer()(sys) try { TestSource.probe[ByteString].runWith(StreamConverters.asInputStream())(materializer) - materializer.asInstanceOf[ActorMaterializerImpl].supervisor.tell(StreamSupervisor.GetChildren, testActor) + materializer.asInstanceOf[PhasedFusingActorMaterializer].supervisor.tell(StreamSupervisor.GetChildren, testActor) val ref = expectMsgType[Children].children.find(_.path.toString contains "inputStreamSink").get assertDispatcher(ref, "akka.stream.default-blocking-io-dispatcher") } finally shutdown(sys) diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/OutputStreamSourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/io/OutputStreamSourceSpec.scala index 4e1bf1e83e..04bd25fe0d 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/io/OutputStreamSourceSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/io/OutputStreamSourceSpec.scala @@ -10,8 +10,7 @@ import java.util.concurrent.TimeoutException import akka.actor.ActorSystem import akka.stream.Attributes.inputBuffer import akka.stream._ -import akka.stream.impl.ActorMaterializerImpl -import akka.stream.impl.StreamSupervisor +import akka.stream.impl.{ PhasedFusingActorMaterializer, StreamSupervisor } import akka.stream.impl.StreamSupervisor.Children import akka.stream.impl.io.OutputStreamSourceStage import akka.stream.scaladsl.{ Keep, Sink, Source, StreamConverters } @@ -139,7 +138,7 @@ class OutputStreamSourceSpec extends StreamSpec(UnboundedMailboxConfig) { try { StreamConverters.asOutputStream().runWith(TestSink.probe[ByteString])(materializer) - materializer.asInstanceOf[ActorMaterializerImpl].supervisor.tell(StreamSupervisor.GetChildren, testActor) + materializer.asInstanceOf[PhasedFusingActorMaterializer].supervisor.tell(StreamSupervisor.GetChildren, testActor) val ref = expectMsgType[Children].children.find(_.path.toString contains "outputStreamSource").get assertDispatcher(ref, "akka.stream.default-blocking-io-dispatcher") } finally shutdown(sys) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/AttributesSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/AttributesSpec.scala index aec5e72c28..23e43cf895 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/AttributesSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/AttributesSpec.scala @@ -13,14 +13,13 @@ import akka.stream.testkit._ import scala.concurrent.Future import scala.concurrent.Promise import akka.stream.impl.SinkModule -import akka.stream.impl.StreamLayout.Module import akka.stream.impl.SinkholeSubscriber object AttributesSpec { object AttributesSink { def apply(): Sink[Nothing, Future[Attributes]] = - new Sink(new AttributesSink(Attributes.name("attributesSink"), Sink.shape("attributesSink"))) + Sink.fromGraph[Nothing, Future[Attributes]](new AttributesSink(Attributes.name("attributesSink"), Sink.shape("attributesSink"))) } final class AttributesSink(val attributes: Attributes, shape: SinkShape[Nothing]) extends SinkModule[Nothing, Future[Attributes]](shape) { @@ -30,7 +29,7 @@ object AttributesSpec { override protected def newInstance(shape: SinkShape[Nothing]): SinkModule[Nothing, Future[Attributes]] = new AttributesSink(attributes, shape) - override def withAttributes(attr: Attributes): Module = + override def withAttributes(attr: Attributes): SinkModule[Nothing, Future[Attributes]] = new AttributesSink(attr, amendShape(attr)) } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/BidiFlowSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/BidiFlowSpec.scala index 77ca6ecf40..a45f3d5634 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/BidiFlowSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/BidiFlowSpec.scala @@ -110,8 +110,8 @@ class BidiFlowSpec extends StreamSpec { import Attributes._ val b: BidiFlow[Int, Long, ByteString, String, NotUsed] = bidi.async.addAttributes(none).named("name") - b.module.attributes.getFirst[Name] shouldEqual Some(Name("name")) - b.module.attributes.getFirst[AsyncBoundary.type] shouldEqual Some(AsyncBoundary) + b.traversalBuilder.attributes.getFirst[Name] shouldEqual Some(Name("name")) + b.traversalBuilder.attributes.getFirst[AsyncBoundary.type] shouldEqual Some(AsyncBoundary) } } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGroupBySpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGroupBySpec.scala index 1833c0bd14..16c208f119 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGroupBySpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGroupBySpec.scala @@ -9,7 +9,6 @@ import akka.NotUsed import akka.actor.ActorSystem import akka.stream.Attributes._ import akka.stream.impl.SinkModule -import akka.stream.impl.StreamLayout.Module import akka.util.ByteString import scala.annotation.tailrec @@ -433,7 +432,7 @@ class FlowGroupBySpec extends StreamSpec { (probe, probe) } override protected def newInstance(shape: SinkShape[ByteString]): SinkModule[ByteString, TestSubscriber.Probe[ByteString]] = new ProbeSink(attributes, shape) - override def withAttributes(attr: Attributes): Module = new ProbeSink(attr, amendShape(attr)) + override def withAttributes(attr: Attributes): SinkModule[ByteString, TestSubscriber.Probe[ByteString]] = new ProbeSink(attr, amendShape(attr)) } @tailrec @@ -463,7 +462,7 @@ class FlowGroupBySpec extends StreamSpec { val publisherProbe = TestPublisher.manualProbe[ByteString]() Source.fromPublisher[ByteString](publisherProbe) - .groupBy(100, elem ⇒ Math.abs(elem.head % 100)).to(new Sink(new ProbeSink(none, SinkShape(Inlet("ProbeSink.in"))))).run()(mat) + .groupBy(100, elem ⇒ Math.abs(elem.head % 100)).to(Sink.fromGraph(new ProbeSink(none, SinkShape(Inlet("ProbeSink.in"))))).run()(mat) val upstreamSubscription = publisherProbe.expectSubscription() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSpec.scala index 5d15600885..2fb1662f1b 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSpec.scala @@ -8,7 +8,6 @@ import akka.actor._ import akka.stream.Supervision._ import akka.stream.impl._ import akka.stream.impl.fusing.ActorGraphInterpreter -import akka.stream.impl.fusing.GraphInterpreter.GraphAssembly import akka.stream.testkit.Utils._ import akka.stream.testkit._ import akka.stream._ @@ -41,46 +40,47 @@ class FlowSpec extends StreamSpec(ConfigFactory.parseString("akka.actor.debug.re val identity: Flow[Any, Any, NotUsed] ⇒ Flow[Any, Any, NotUsed] = in ⇒ in.map(e ⇒ e) val identity2: Flow[Any, Any, NotUsed] ⇒ Flow[Any, Any, NotUsed] = in ⇒ identity(in) - class BrokenActorInterpreter(_shell: GraphInterpreterShell, brokenMessage: Any) - extends ActorGraphInterpreter(_shell) { + // TODO: Reenable these tests + // class BrokenActorInterpreter(_shell: GraphInterpreterShell, brokenMessage: Any) + // extends ActorGraphInterpreter(_shell) { + // + // override protected[akka] def aroundReceive(receive: Receive, msg: Any) = { + // msg match { + // case ActorGraphInterpreter.OnNext(_, 0, m) if m == brokenMessage ⇒ + // throw new NullPointerException(s"I'm so broken [$m]") + // case _ ⇒ super.aroundReceive(receive, msg) + // } + // } + // } - override protected[akka] def aroundReceive(receive: Receive, msg: Any) = { - msg match { - case ActorGraphInterpreter.OnNext(_, 0, m) if m == brokenMessage ⇒ - throw new NullPointerException(s"I'm so broken [$m]") - case _ ⇒ super.aroundReceive(receive, msg) - } - } - } - - val faultyFlow: Flow[Any, Any, NotUsed] ⇒ Flow[Any, Any, NotUsed] = in ⇒ in.via({ - val stage = fusing.Map({ x: Any ⇒ x }) - - val assembly = new GraphAssembly( - Array(stage), - Array(Attributes.none), - Array(stage.shape.in, null), - Array(0, -1), - Array(null, stage.shape.out), - Array(-1, 0)) - - val (connections, logics) = - assembly.materialize(Attributes.none, assembly.stages.map(_.module), new java.util.HashMap, _ ⇒ ()) - - val shell = new GraphInterpreterShell(assembly, connections, logics, stage.shape, settings, - materializer.asInstanceOf[ActorMaterializerImpl]) - - val props = Props(new BrokenActorInterpreter(shell, "a3")) - .withDispatcher("akka.test.stream-dispatcher").withDeploy(Deploy.local) - val impl = system.actorOf(props, "borken-stage-actor") - - val subscriber = new ActorGraphInterpreter.BoundarySubscriber(impl, shell, 0) - val publisher = new ActorPublisher[Any](impl) { override val wakeUpMsg = ActorGraphInterpreter.SubscribePending(shell, 0) } - - impl ! ActorGraphInterpreter.ExposedPublisher(shell, 0, publisher) - - Flow.fromSinkAndSource(Sink.fromSubscriber(subscriber), Source.fromPublisher(publisher)) - }) + // val faultyFlow: Flow[Any, Any, NotUsed] ⇒ Flow[Any, Any, NotUsed] = in ⇒ in.via({ + // val stage = fusing.Map({ x: Any ⇒ x }) + // + // val assembly = new GraphAssembly( + // Array(stage), + // Array(Attributes.none), + // Array(stage.shape.in, null), + // Array(0, -1), + // Array(null, stage.shape.out), + // Array(-1, 0)) + // + // val (connections, logics) = + // assembly.materialize(Attributes.none, assembly.stages.map(_.module), new java.util.HashMap, _ ⇒ ()) + // + // val shell = new GraphInterpreterShell(assembly, connections, logics, stage.shape, settings, + // materializer.asInstanceOf[ActorMaterializerImpl]) + // + // val props = Props(new BrokenActorInterpreter(shell, "a3")) + // .withDispatcher("akka.test.stream-dispatcher").withDeploy(Deploy.local) + // val impl = system.actorOf(props, "borken-stage-actor") + // + // val subscriber = new ActorGraphInterpreter.BoundarySubscriber(impl, shell, 0) + // val publisher = new ActorPublisher[Any](impl) { override val wakeUpMsg = ActorGraphInterpreter.SubscribePending(shell, 0) } + // + // impl ! ActorGraphInterpreter.ExposedPublisher(shell, 0, publisher) + // + // Flow.fromSinkAndSource(Sink.fromSubscriber(subscriber), Source.fromPublisher(publisher)) + // }) val toPublisher: (Source[Any, _], ActorMaterializer) ⇒ Publisher[Any] = (f, m) ⇒ f.runWith(Sink.asPublisher(false))(m) @@ -535,67 +535,67 @@ class FlowSpec extends StreamSpec(ConfigFactory.parseString("akka.actor.debug.re Source(0 to 9).via(Flow.fromFunction(_ + 1)).runWith(Sink.seq).futureValue should ===(1 to 10) } } - - "A broken Flow" must { - "cancel upstream and call onError on current and future downstream subscribers if an internal error occurs" in { - new ChainSetup(faultyFlow, settings.withInputBuffer(initialSize = 1, maxSize = 1), toFanoutPublisher(16)) { - - def checkError(sprobe: TestSubscriber.ManualProbe[Any]): Unit = { - val error = sprobe.expectError() - error.isInstanceOf[AbruptTerminationException] should be(true) - error.getMessage should startWith("Processor actor") - } - - val downstream2 = TestSubscriber.manualProbe[Any]() - publisher.subscribe(downstream2) - val downstream2Subscription = downstream2.expectSubscription() - - downstreamSubscription.request(5) - downstream2Subscription.request(5) - upstream.expectRequest(upstreamSubscription, 1) - upstreamSubscription.sendNext("a1") - downstream.expectNext("a1") - downstream2.expectNext("a1") - - upstream.expectRequest(upstreamSubscription, 1) - upstreamSubscription.sendNext("a2") - downstream.expectNext("a2") - downstream2.expectNext("a2") - - val filters = immutable.Seq( - EventFilter[NullPointerException](), - EventFilter[IllegalStateException](), - EventFilter[PostRestartException]()) // This is thrown because we attach the dummy failing actor to toplevel - try { - system.eventStream.publish(Mute(filters)) - - upstream.expectRequest(upstreamSubscription, 1) - upstreamSubscription.sendNext("a3") - upstreamSubscription.expectCancellation() - - // IllegalStateException terminated abruptly - checkError(downstream) - checkError(downstream2) - - val downstream3 = TestSubscriber.manualProbe[Any]() - publisher.subscribe(downstream3) - downstream3.expectSubscription() - // IllegalStateException terminated abruptly - checkError(downstream3) - } finally { - system.eventStream.publish(UnMute(filters)) - } - } - } - - "suitably override attribute handling methods" in { - import Attributes._ - val f: Flow[Int, Int, NotUsed] = Flow[Int].map(_ + 1).async.addAttributes(none).named("name") - - f.module.attributes.getFirst[Name] shouldEqual Some(Name("name")) - f.module.attributes.getFirst[Attributes.AsyncBoundary.type] shouldEqual Some(AsyncBoundary) - } - } + // + // "A broken Flow" must { + // "cancel upstream and call onError on current and future downstream subscribers if an internal error occurs" in { + // new ChainSetup(faultyFlow, settings.withInputBuffer(initialSize = 1, maxSize = 1), toFanoutPublisher(16)) { + // + // def checkError(sprobe: TestSubscriber.ManualProbe[Any]): Unit = { + // val error = sprobe.expectError() + // error.isInstanceOf[AbruptTerminationException] should be(true) + // error.getMessage should startWith("Processor actor") + // } + // + // val downstream2 = TestSubscriber.manualProbe[Any]() + // publisher.subscribe(downstream2) + // val downstream2Subscription = downstream2.expectSubscription() + // + // downstreamSubscription.request(5) + // downstream2Subscription.request(5) + // upstream.expectRequest(upstreamSubscription, 1) + // upstreamSubscription.sendNext("a1") + // downstream.expectNext("a1") + // downstream2.expectNext("a1") + // + // upstream.expectRequest(upstreamSubscription, 1) + // upstreamSubscription.sendNext("a2") + // downstream.expectNext("a2") + // downstream2.expectNext("a2") + // + // val filters = immutable.Seq( + // EventFilter[NullPointerException](), + // EventFilter[IllegalStateException](), + // EventFilter[PostRestartException]()) // This is thrown because we attach the dummy failing actor to toplevel + // try { + // system.eventStream.publish(Mute(filters)) + // + // upstream.expectRequest(upstreamSubscription, 1) + // upstreamSubscription.sendNext("a3") + // upstreamSubscription.expectCancellation() + // + // // IllegalStateException terminated abruptly + // checkError(downstream) + // checkError(downstream2) + // + // val downstream3 = TestSubscriber.manualProbe[Any]() + // publisher.subscribe(downstream3) + // downstream3.expectSubscription() + // // IllegalStateException terminated abruptly + // checkError(downstream3) + // } finally { + // system.eventStream.publish(UnMute(filters)) + // } + // } + // } + // + // "suitably override attribute handling methods" in { + // import Attributes._ + // val f: Flow[Int, Int, NotUsed] = Flow[Int].map(_ + 1).async.addAttributes(none).named("name") + // + // f.module.attributes.getFirst[Name] shouldEqual Some(Name("name")) + // f.module.attributes.getFirst[Attributes.AsyncBoundary.type] shouldEqual Some(AsyncBoundary) + // } + // } object TestException extends RuntimeException with NoStackTrace diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphDSLCompileSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphDSLCompileSpec.scala index 6bae7b59e9..2d966df927 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphDSLCompileSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphDSLCompileSpec.scala @@ -356,8 +356,8 @@ class GraphDSLCompileSpec extends StreamSpec { FlowShape(id.in, id.out) }.async.addAttributes(none).named("useless") - ga.module.attributes.getFirst[Name] shouldEqual Some(Name("useless")) - ga.module.attributes.getFirst[AsyncBoundary.type] shouldEqual (Some(AsyncBoundary)) + ga.traversalBuilder.attributes.getFirst[Name] shouldEqual Some(Name("useless")) + ga.traversalBuilder.attributes.getFirst[AsyncBoundary.type] shouldEqual (Some(AsyncBoundary)) } } } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphOpsIntegrationSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphOpsIntegrationSpec.scala index 07d837f67a..1b2f38cc61 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphOpsIntegrationSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphOpsIntegrationSpec.scala @@ -19,13 +19,6 @@ object GraphOpsIntegrationSpec { override def deepCopy() = ShufflePorts( in1.carbonCopy(), in2.carbonCopy(), out1.carbonCopy(), out2.carbonCopy()) - override def copyFromPorts(inlets: immutable.Seq[Inlet[_]], outlets: immutable.Seq[Outlet[_]]): ShufflePorts[In, Out] = { - assert(inlets.size == this.inlets.size) - assert(outlets.size == this.outlets.size) - val i = inlets.asInstanceOf[Seq[Inlet[In]]] - val o = outlets.asInstanceOf[Seq[Outlet[Out]]] - ShufflePorts(i(0), i(1), o(0), o(1)) - } } def apply[In, Out](pipeline: Flow[In, Out, _]): Graph[ShufflePorts[In, Out], NotUsed] = { diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/RunnableGraphSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/RunnableGraphSpec.scala index 6be8bd63c5..053ebf772a 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/RunnableGraphSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/RunnableGraphSpec.scala @@ -17,8 +17,8 @@ class RunnableGraphSpec extends StreamSpec { import Attributes._ val r: RunnableGraph[NotUsed] = RunnableGraph.fromGraph(Source.empty.to(Sink.ignore)).async.addAttributes(none).named("useless") - r.module.attributes.getFirst[Name] shouldEqual Some(Name("useless")) - r.module.attributes.getFirst[AsyncBoundary.type] shouldEqual (Some(AsyncBoundary)) + r.traversalBuilder.attributes.getFirst[Name] shouldEqual Some(Name("useless")) + r.traversalBuilder.attributes.getFirst[AsyncBoundary.type] shouldEqual (Some(AsyncBoundary)) } } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkAsJavaStreamSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkAsJavaStreamSpec.scala index 0a6602c73a..ba28afde6c 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkAsJavaStreamSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkAsJavaStreamSpec.scala @@ -4,10 +4,11 @@ package akka.stream.scaladsl import java.util.stream.Collectors + import akka.actor.ActorSystem -import akka.stream.impl.StreamSupervisor.Children -import akka.stream.impl.{ StreamSupervisor, ActorMaterializerImpl } import akka.stream._ +import akka.stream.impl.{ PhasedFusingActorMaterializer, StreamSupervisor } +import akka.stream.impl.StreamSupervisor.Children import akka.stream.testkit.Utils._ import akka.stream.testkit._ import akka.stream.testkit.scaladsl.TestSource @@ -53,7 +54,7 @@ class SinkAsJavaStreamSpec extends StreamSpec(UnboundedMailboxConfig) { try { TestSource.probe[ByteString].runWith(StreamConverters.asJavaStream())(materializer) - materializer.asInstanceOf[ActorMaterializerImpl].supervisor.tell(StreamSupervisor.GetChildren, testActor) + materializer.asInstanceOf[PhasedFusingActorMaterializer].supervisor.tell(StreamSupervisor.GetChildren, testActor) val ref = expectMsgType[Children].children.find(_.path.toString contains "asJavaStream").get assertDispatcher(ref, "akka.stream.default-blocking-io-dispatcher") } finally shutdown(sys) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkSpec.scala index 3627c7c936..3a66eb6207 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SinkSpec.scala @@ -128,50 +128,50 @@ class SinkSpec extends StreamSpec with DefaultTimeout with ScalaFutures { import Attributes._ val s: Sink[Int, Future[Int]] = Sink.head[Int].async.addAttributes(none).named("name") - s.module.attributes.getFirst[Name] shouldEqual Some(Name("name")) - s.module.attributes.getFirst[AsyncBoundary.type] shouldEqual (Some(AsyncBoundary)) + s.traversalBuilder.attributes.getFirst[Name] shouldEqual Some(Name("name")) + s.traversalBuilder.attributes.getFirst[AsyncBoundary.type] shouldEqual (Some(AsyncBoundary)) } "given one attribute of a class should correctly get it as first attribute with default value" in { import Attributes._ val s: Sink[Int, Future[Int]] = Sink.head[Int].async.addAttributes(none).named("name") - s.module.attributes.getFirst[Name](Name("default")) shouldEqual Name("name") + s.traversalBuilder.attributes.getFirst[Name](Name("default")) shouldEqual Name("name") } "given one attribute of a class should correctly get it as last attribute with default value" in { import Attributes._ val s: Sink[Int, Future[Int]] = Sink.head[Int].async.addAttributes(none).named("name") - s.module.attributes.get[Name](Name("default")) shouldEqual Name("name") + s.traversalBuilder.attributes.get[Name](Name("default")) shouldEqual Name("name") } "given no attributes of a class when getting first attribute with default value should get default value" in { import Attributes._ val s: Sink[Int, Future[Int]] = Sink.head[Int].async.addAttributes(none) - s.module.attributes.getFirst[Name](Name("default")) shouldEqual Name("default") + s.traversalBuilder.attributes.getFirst[Name](Name("default")) shouldEqual Name("default") } "given no attributes of a class when getting last attribute with default value should get default value" in { import Attributes._ val s: Sink[Int, Future[Int]] = Sink.head[Int].async.addAttributes(none) - s.module.attributes.get[Name](Name("default")) shouldEqual Name("default") + s.traversalBuilder.attributes.get[Name](Name("default")) shouldEqual Name("default") } "given multiple attributes of a class when getting first attribute with default value should get first attribute" in { import Attributes._ val s: Sink[Int, Future[Int]] = Sink.head[Int].async.addAttributes(none).named("name").named("another_name") - s.module.attributes.getFirst[Name](Name("default")) shouldEqual Name("name") + s.traversalBuilder.attributes.getFirst[Name](Name("default")) shouldEqual Name("name") } "given multiple attributes of a class when getting last attribute with default value should get last attribute" in { import Attributes._ val s: Sink[Int, Future[Int]] = Sink.head[Int].async.addAttributes(none).named("name").named("another_name") - s.module.attributes.get[Name](Name("default")) shouldEqual Name("another_name") + s.traversalBuilder.attributes.get[Name](Name("default")) shouldEqual Name("another_name") } "support contramap" in { diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/UnfoldResourceAsyncSourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/UnfoldResourceAsyncSourceSpec.scala index 0564f79590..8f62d6ebd6 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/UnfoldResourceAsyncSourceSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/UnfoldResourceAsyncSourceSpec.scala @@ -11,7 +11,7 @@ import akka.stream.ActorAttributes._ import akka.stream.Supervision._ import akka.stream.{ ActorMaterializer, _ } import akka.stream.impl.StreamSupervisor.Children -import akka.stream.impl.{ ActorMaterializerImpl, StreamSupervisor } +import akka.stream.impl.{ PhasedFusingActorMaterializer, StreamSupervisor } import akka.stream.testkit.{ StreamSpec, TestSubscriber } import akka.stream.testkit.Utils._ import akka.stream.testkit.scaladsl.TestSink @@ -217,7 +217,7 @@ class UnfoldResourceAsyncSourceSpec extends StreamSpec(UnboundedMailboxConfig) { open, read, close).runWith(TestSink.probe)(materializer) - materializer.asInstanceOf[ActorMaterializerImpl].supervisor.tell(StreamSupervisor.GetChildren, testActor) + materializer.asInstanceOf[PhasedFusingActorMaterializer].supervisor.tell(StreamSupervisor.GetChildren, testActor) val ref = expectMsgType[Children].children.find(_.path.toString contains "unfoldResourceSourceAsync").get try assertDispatcher(ref, "akka.stream.default-blocking-io-dispatcher") finally p.cancel() } finally shutdown(sys) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/UnfoldResourceSourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/UnfoldResourceSourceSpec.scala index 4afbdc5b70..e396394092 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/UnfoldResourceSourceSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/UnfoldResourceSourceSpec.scala @@ -10,7 +10,7 @@ import akka.stream.ActorAttributes._ import akka.stream.Supervision._ import akka.stream.{ ActorMaterializer, _ } import akka.stream.impl.StreamSupervisor.Children -import akka.stream.impl.{ ActorMaterializerImpl, StreamSupervisor } +import akka.stream.impl.{ PhasedFusingActorMaterializer, StreamSupervisor } import akka.stream.testkit.{ StreamSpec, TestSubscriber } import akka.stream.testkit.Utils._ import akka.stream.testkit.scaladsl.TestSink @@ -150,7 +150,7 @@ class UnfoldResourceSourceSpec extends StreamSpec(UnboundedMailboxConfig) { reader ⇒ Option(reader.readLine()), reader ⇒ reader.close()).runWith(TestSink.probe)(materializer) - materializer.asInstanceOf[ActorMaterializerImpl].supervisor.tell(StreamSupervisor.GetChildren, testActor) + materializer.asInstanceOf[PhasedFusingActorMaterializer].supervisor.tell(StreamSupervisor.GetChildren, testActor) val ref = expectMsgType[Children].children.find(_.path.toString contains "unfoldResourceSource").get try assertDispatcher(ref, "akka.stream.default-blocking-io-dispatcher") finally p.cancel() } finally shutdown(sys) diff --git a/akka-stream/src/main/boilerplate/akka/stream/scaladsl/GraphApply.scala.template b/akka-stream/src/main/boilerplate/akka/stream/scaladsl/GraphApply.scala.template index c5113c60a4..7d790c457f 100644 --- a/akka-stream/src/main/boilerplate/akka/stream/scaladsl/GraphApply.scala.template +++ b/akka-stream/src/main/boilerplate/akka/stream/scaladsl/GraphApply.scala.template @@ -4,8 +4,7 @@ package akka.stream.scaladsl import akka.NotUsed -import akka.stream.impl.StreamLayout -import akka.stream.impl.StreamLayout.Module +import akka.stream.impl.TraversalBuilder import akka.stream.{ Graph, Attributes, Shape } trait GraphApply { @@ -15,9 +14,8 @@ trait GraphApply { def create[S <: Shape]()(buildBlock: GraphDSL.Builder[NotUsed] ⇒ S): Graph[S, NotUsed] = { val builder = new GraphDSL.Builder val s = buildBlock(builder) - val mod = builder.module.replaceShape(s) - new GraphApply.GraphImpl(s, mod) + new GraphApply.GraphImpl(s, builder.traversalBuilder) } /** @@ -28,9 +26,8 @@ trait GraphApply { val builder = new GraphDSL.Builder val s1 = builder.add(g1, Keep.right) val s = buildBlock(builder)(s1) - val mod = builder.module.replaceShape(s) - new GraphApply.GraphImpl(s, mod) + new GraphApply.GraphImpl(s, builder.traversalBuilder) } @@ -47,9 +44,8 @@ trait GraphApply { [2..#val s1 = builder.add(g1, (f: M1 ⇒ Any, m1: M1) ⇒ f(m1))# ] val s = buildBlock(builder)([#s1#]) - val mod = builder.module.replaceShape(s) - new GraphApply.GraphImpl(s, mod) + new GraphApply.GraphImpl(s, builder.traversalBuilder) }# ] @@ -59,26 +55,15 @@ trait GraphApply { * INTERNAL API */ object GraphApply { - final class GraphImpl[S <: Shape, Mat](override val shape: S, override val module: StreamLayout.Module) + final class GraphImpl[S <: Shape, Mat](override val shape: S, override val traversalBuilder: TraversalBuilder) extends Graph[S, Mat] { - override def toString: String = s"Graph($shape, $module)" + override def toString: String = s"Graph($shape)" override def withAttributes(attr: Attributes): Graph[S, Mat] = - new GraphImpl(shape, module.withAttributes(attr)) + new GraphImpl(shape, traversalBuilder.setAttributes(attr)) override def named(name: String): Graph[S, Mat] = addAttributes(Attributes.name(name)) } } -/** - * INTERNAL API - */ -object ModuleExtractor { - def unapply[S <: Shape, Mat](graph: Graph[S, Mat]): Option[Module] = graph match { - case module: Module => - Some(module) - case _ => - None - } -} diff --git a/akka-stream/src/main/scala/akka/stream/ActorMaterializer.scala b/akka-stream/src/main/scala/akka/stream/ActorMaterializer.scala index f846c3b38b..b851c2145d 100644 --- a/akka-stream/src/main/scala/akka/stream/ActorMaterializer.scala +++ b/akka-stream/src/main/scala/akka/stream/ActorMaterializer.scala @@ -57,7 +57,7 @@ object ActorMaterializer { val haveShutDown = new AtomicBoolean(false) val system = actorSystemOf(context) - new ActorMaterializerImpl( + new PhasedFusingActorMaterializer( system, materializerSettings, system.dispatchers, @@ -86,7 +86,7 @@ object ActorMaterializer { private[akka] def systemMaterializer(materializerSettings: ActorMaterializerSettings, namePrefix: String, system: ExtendedActorSystem): ActorMaterializer = { val haveShutDown = new AtomicBoolean(false) - new ActorMaterializerImpl( + new PhasedFusingActorMaterializer( system, materializerSettings, system.dispatchers, diff --git a/akka-stream/src/main/scala/akka/stream/Attributes.scala b/akka-stream/src/main/scala/akka/stream/Attributes.scala index 8dbe2121cb..62687603ca 100644 --- a/akka-stream/src/main/scala/akka/stream/Attributes.scala +++ b/akka-stream/src/main/scala/akka/stream/Attributes.scala @@ -6,11 +6,15 @@ package akka.stream import java.util.Optional import akka.event.Logging + import scala.annotation.tailrec -import scala.reflect.{ classTag, ClassTag } +import scala.reflect.{ ClassTag, classTag } import akka.japi.function import akka.stream.impl.StreamLayout._ import java.net.URLEncoder + +import akka.stream.impl.TraversalBuilder + import scala.compat.java8.OptionConverters._ /** @@ -233,11 +237,8 @@ object Attributes { * Compute a name by concatenating all Name attributes that the given module * has, returning the given default value if none are found. */ - def extractName(mod: Module, default: String): String = { - mod match { - case CopiedModule(_, attr, copyOf) ⇒ (attr and copyOf.attributes).nameOrDefault(default) - case _ ⇒ mod.attributes.nameOrDefault(default) - } + def extractName(builder: TraversalBuilder, default: String): String = { + builder.attributes.nameOrDefault(default) } } diff --git a/akka-stream/src/main/scala/akka/stream/Fusing.scala b/akka-stream/src/main/scala/akka/stream/Fusing.scala deleted file mode 100644 index dee53b6263..0000000000 --- a/akka-stream/src/main/scala/akka/stream/Fusing.scala +++ /dev/null @@ -1,62 +0,0 @@ -/** - * Copyright (C) 2015-2017 Lightbend Inc. - */ -package akka.stream - -import scala.collection.immutable -import akka.stream.impl.StreamLayout._ -import akka.stream.impl.fusing.{ Fusing ⇒ Impl } -import scala.annotation.unchecked.uncheckedVariance - -/** - * This class holds some graph transformation functions that can fuse together - * multiple operation stages into synchronous execution islands. The purpose is - * to reduce the number of Actors that are created in order to execute the stream - * and thereby improve start-up cost as well as reduce element traversal latency - * for large graphs. Fusing itself is a time-consuming operation, meaning that - * usually it is best to cache the result of this computation and reuse it instead - * of fusing the same graph many times. - * - * Fusing together all operations which allow this treatment will reduce the - * parallelism that is available in the stream graph’s execution—in the worst case - * it will become single-threaded and not benefit from multiple CPU cores at all. - * Where parallelism is required, the [[akka.stream.Attributes#AsyncBoundary]] - * attribute can be used to declare subgraph boundaries across which the graph - * shall not be fused. - */ -object Fusing { - - /** - * Fuse all operations where this is technically possible (i.e. all - * implementations based on [[akka.stream.stage.GraphStage]]) and not forbidden - * via [[akka.stream.Attributes#AsyncBoundary]]. - */ - def aggressive[S <: Shape, M](g: Graph[S, M]): FusedGraph[S, M] = Impl.aggressive(g) - - /** - * Return the StructuralInfo for this Graph without any fusing - */ - def structuralInfo[S <: Shape, M](g: Graph[S, M], attributes: Attributes): StructuralInfoModule = - Impl.structuralInfo(g, attributes) - - /** - * A fused graph of the right shape, containing a [[FusedModule]] which - * holds more information on the operation structure of the contained stream - * topology for convenient graph traversal. - */ - case class FusedGraph[+S <: Shape @uncheckedVariance, +M]( - override val module: FusedModule, - override val shape: S) extends Graph[S, M] { - // the @uncheckedVariance look like a compiler bug ... why does it work in Graph but not here? - override def withAttributes(attr: Attributes) = copy(module = module.withAttributes(attr)) - } - - object FusedGraph { - def unapply[S <: Shape, M](g: Graph[S, M]): Option[(FusedModule, S)] = - g.module match { - case f: FusedModule ⇒ Some((f, g.shape)) - case _ ⇒ None - } - } - -} diff --git a/akka-stream/src/main/scala/akka/stream/Graph.scala b/akka-stream/src/main/scala/akka/stream/Graph.scala index 0eaab0a74b..c015ecc749 100644 --- a/akka-stream/src/main/scala/akka/stream/Graph.scala +++ b/akka-stream/src/main/scala/akka/stream/Graph.scala @@ -3,7 +3,8 @@ */ package akka.stream -import akka.stream.impl.StreamLayout +import akka.stream.impl.TraversalBuilder + import scala.annotation.unchecked.uncheckedVariance trait Graph[+S <: Shape, +M] { @@ -20,7 +21,7 @@ trait Graph[+S <: Shape, +M] { * * Every materializable element must be backed by a stream layout module */ - private[stream] def module: StreamLayout.Module + private[stream] def traversalBuilder: TraversalBuilder def withAttributes(attr: Attributes): Graph[S, M] @@ -29,7 +30,8 @@ trait Graph[+S <: Shape, +M] { /** * Put an asynchronous boundary around this `Graph` */ + // TODO: no longer encoded as attributes!!!! def async: Graph[S, M] = addAttributes(Attributes.asyncBoundary) - def addAttributes(attr: Attributes): Graph[S, M] = withAttributes(module.attributes and attr) + def addAttributes(attr: Attributes): Graph[S, M] = withAttributes(traversalBuilder.attributes and attr) } diff --git a/akka-stream/src/main/scala/akka/stream/Shape.scala b/akka-stream/src/main/scala/akka/stream/Shape.scala index 29d2efdabd..af67f904be 100644 --- a/akka-stream/src/main/scala/akka/stream/Shape.scala +++ b/akka-stream/src/main/scala/akka/stream/Shape.scala @@ -21,7 +21,13 @@ sealed abstract class InPort { self: Inlet[_] ⇒ /** * INTERNAL API */ - private[stream] var id: Int = -1 + @volatile private[stream] var id: Int = -1 + + /** + * INTERNAL API + */ + @volatile private[stream] var mappedTo: InPort = this + /** * INTERNAL API */ @@ -40,7 +46,13 @@ sealed abstract class OutPort { self: Outlet[_] ⇒ /** * INTERNAL API */ - private[stream] var id: Int = -1 + @volatile private[stream] var id: Int = -1 + + /** + * INTERNAL API + */ + @volatile private[stream] var mappedTo: OutPort = this + /** * INTERNAL API */ @@ -70,12 +82,20 @@ object Inlet { def create[T](name: String): Inlet[T] = Inlet(name) } -final class Inlet[T] private (override val toString: String) extends InPort { - def carbonCopy(): Inlet[T] = Inlet(toString) +final class Inlet[T] private (val s: String) extends InPort { + def carbonCopy(): Inlet[T] = { + val in = Inlet[T](s) + in.mappedTo = this + in + } /** * INTERNAL API. */ def as[U]: Inlet[U] = this.asInstanceOf[Inlet[U]] + + override def toString: String = s + "(" + this.hashCode + s")" + + (if (mappedTo eq this) "" + else s" mapped to $mappedTo") } /** @@ -102,12 +122,20 @@ object Outlet { def create[T](name: String): Outlet[T] = Outlet(name) } -final class Outlet[T] private (override val toString: String) extends OutPort { - def carbonCopy(): Outlet[T] = Outlet(toString) +final class Outlet[T] private (val s: String) extends OutPort { + def carbonCopy(): Outlet[T] = { + val out = Outlet[T](s) + out.mappedTo = this + out + } /** * INTERNAL API. */ def as[U]: Outlet[U] = this.asInstanceOf[Outlet[U]] + + override def toString: String = s + "(" + this.hashCode + s")" + + (if (mappedTo eq this) "" + else s" mapped to $mappedTo") } /** @@ -134,12 +162,6 @@ abstract class Shape { */ def deepCopy(): Shape - /** - * Create a copy of this Shape object, returning the same type as the - * original but containing the ports given within the passed-in Shape. - */ - def copyFromPorts(inlets: immutable.Seq[Inlet[_]], outlets: immutable.Seq[Outlet[_]]): Shape - /** * Java API: get a list of all input ports */ @@ -205,11 +227,6 @@ object ClosedShape extends ClosedShape { override val inlets: immutable.Seq[Inlet[_]] = EmptyImmutableSeq override val outlets: immutable.Seq[Outlet[_]] = EmptyImmutableSeq override def deepCopy() = this - override def copyFromPorts(inlets: immutable.Seq[Inlet[_]], outlets: immutable.Seq[Outlet[_]]): Shape = { - require(inlets.isEmpty, s"proposed inlets [${inlets.mkString(", ")}] do not fit ClosedShape") - require(outlets.isEmpty, s"proposed outlets [${outlets.mkString(", ")}] do not fit ClosedShape") - this - } /** * Java API: obtain ClosedShape instance @@ -227,7 +244,6 @@ object ClosedShape extends ClosedShape { */ case class AmorphousShape(inlets: immutable.Seq[Inlet[_]], outlets: immutable.Seq[Outlet[_]]) extends Shape { override def deepCopy() = AmorphousShape(inlets.map(_.carbonCopy()), outlets.map(_.carbonCopy())) - override def copyFromPorts(inlets: immutable.Seq[Inlet[_]], outlets: immutable.Seq[Outlet[_]]): Shape = AmorphousShape(inlets, outlets) } /** @@ -239,11 +255,6 @@ final case class SourceShape[+T](out: Outlet[T @uncheckedVariance]) extends Shap override val outlets: immutable.Seq[Outlet[_]] = List(out) override def deepCopy(): SourceShape[T] = SourceShape(out.carbonCopy()) - override def copyFromPorts(inlets: immutable.Seq[Inlet[_]], outlets: immutable.Seq[Outlet[_]]): Shape = { - require(inlets.isEmpty, s"proposed inlets [${inlets.mkString(", ")}] do not fit SourceShape") - require(outlets.size == 1, s"proposed outlets [${outlets.mkString(", ")}] do not fit SourceShape") - SourceShape(outlets.head) - } } object SourceShape { /** Java API */ @@ -261,11 +272,6 @@ final case class FlowShape[-I, +O](in: Inlet[I @uncheckedVariance], out: Outlet[ override val outlets: immutable.Seq[Outlet[_]] = List(out) override def deepCopy(): FlowShape[I, O] = FlowShape(in.carbonCopy(), out.carbonCopy()) - override def copyFromPorts(inlets: immutable.Seq[Inlet[_]], outlets: immutable.Seq[Outlet[_]]): Shape = { - require(inlets.size == 1, s"proposed inlets [${inlets.mkString(", ")}] do not fit FlowShape") - require(outlets.size == 1, s"proposed outlets [${outlets.mkString(", ")}] do not fit FlowShape") - FlowShape(inlets.head, outlets.head) - } } object FlowShape { /** Java API */ @@ -281,11 +287,6 @@ final case class SinkShape[-T](in: Inlet[T @uncheckedVariance]) extends Shape { override val outlets: immutable.Seq[Outlet[_]] = EmptyImmutableSeq override def deepCopy(): SinkShape[T] = SinkShape(in.carbonCopy()) - override def copyFromPorts(inlets: immutable.Seq[Inlet[_]], outlets: immutable.Seq[Outlet[_]]): Shape = { - require(inlets.size == 1, s"proposed inlets [${inlets.mkString(", ")}] do not fit SinkShape") - require(outlets.isEmpty, s"proposed outlets [${outlets.mkString(", ")}] do not fit SinkShape") - SinkShape(inlets.head) - } } object SinkShape { /** Java API */ @@ -322,12 +323,7 @@ final case class BidiShape[-In1, +Out1, -In2, +Out2]( override def deepCopy(): BidiShape[In1, Out1, In2, Out2] = BidiShape(in1.carbonCopy(), out1.carbonCopy(), in2.carbonCopy(), out2.carbonCopy()) - override def copyFromPorts(inlets: immutable.Seq[Inlet[_]], outlets: immutable.Seq[Outlet[_]]): Shape = { - require(inlets.size == 2, s"proposed inlets [${inlets.mkString(", ")}] do not fit BidiShape") - require(outlets.size == 2, s"proposed outlets [${outlets.mkString(", ")}] do not fit BidiShape") - BidiShape(inlets(0), outlets(0), inlets(1), outlets(1)) - } - def reversed: Shape = copyFromPorts(inlets.reverse, outlets.reverse) + //#implementation-details-elided } //#bidi-shape diff --git a/akka-stream/src/main/scala/akka/stream/impl/ActorMaterializerImpl.scala b/akka-stream/src/main/scala/akka/stream/impl/ActorMaterializerImpl.scala index 8613a2b07c..1e3864f6c3 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/ActorMaterializerImpl.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/ActorMaterializerImpl.scala @@ -4,26 +4,15 @@ package akka.stream.impl import java.util.concurrent.atomic.AtomicBoolean -import java.{ util ⇒ ju } -import akka.NotUsed import akka.actor._ -import akka.event.{ Logging, LoggingAdapter } -import akka.dispatch.Dispatchers +import akka.event.LoggingAdapter import akka.pattern.ask import akka.stream._ -import akka.stream.impl.StreamLayout.{ AtomicModule, Module } -import akka.stream.impl.fusing.{ ActorGraphInterpreter, GraphModule } -import akka.stream.impl.io.TLSActor -import akka.stream.impl.io.TlsModule -import org.reactivestreams._ +import akka.stream.impl.fusing.GraphInterpreterShell import scala.concurrent.duration.FiniteDuration import scala.concurrent.{ Await, ExecutionContextExecutor } -import akka.stream.impl.fusing.GraphStageModule -import akka.stream.impl.fusing.GraphInterpreter.GraphAssembly -import akka.stream.impl.fusing.Fusing -import akka.stream.impl.fusing.GraphInterpreterShell /** * ExtendedActorMaterializer used by subtypes which materializer using GraphInterpreterShell @@ -60,7 +49,8 @@ abstract class ExtendedActorMaterializer extends ActorMaterializer { /** * INTERNAL API */ - protected def actorOf(props: Props, name: String, dispatcher: String): ActorRef = { + // TODO: hide it again + def actorOf(props: Props, name: String, dispatcher: String): ActorRef = { supervisor match { case ref: LocalActorRef ⇒ ref.underlying.attachChild(props.withDispatcher(dispatcher), name, systemService = false) @@ -89,183 +79,6 @@ abstract class ExtendedActorMaterializer extends ActorMaterializer { } -/** - * INTERNAL API - */ -private[akka] case class ActorMaterializerImpl( - system: ActorSystem, - override val settings: ActorMaterializerSettings, - dispatchers: Dispatchers, - supervisor: ActorRef, - haveShutDown: AtomicBoolean, - flowNames: SeqActorName) extends ExtendedActorMaterializer { - import akka.stream.impl.Stages._ - private val _logger = Logging.getLogger(system, this) - override def logger = _logger - - if (settings.fuzzingMode && !system.settings.config.hasPath("akka.stream.secret-test-fuzzing-warning-disable")) { - _logger.warning("Fuzzing mode is enabled on this system. If you see this warning on your production system then " + - "set akka.stream.materializer.debug.fuzzing-mode to off.") - } - - override def shutdown(): Unit = - if (haveShutDown.compareAndSet(false, true)) supervisor ! PoisonPill - - override def isShutdown: Boolean = haveShutDown.get() - - override def withNamePrefix(name: String): ActorMaterializerImpl = this.copy(flowNames = flowNames.copy(name)) - - private[this] def createFlowName(): String = flowNames.next() - - private val defaultInitialAttributes = Attributes( - Attributes.InputBuffer(settings.initialInputBufferSize, settings.maxInputBufferSize) :: - ActorAttributes.Dispatcher(settings.dispatcher) :: - ActorAttributes.SupervisionStrategy(settings.supervisionDecider) :: - Nil) - - override def effectiveSettings(opAttr: Attributes): ActorMaterializerSettings = { - import Attributes._ - import ActorAttributes._ - opAttr.attributeList.foldLeft(settings) { (s, attr) ⇒ - attr match { - case InputBuffer(initial, max) ⇒ s.withInputBuffer(initial, max) - case Dispatcher(dispatcher) ⇒ s.withDispatcher(dispatcher) - case SupervisionStrategy(decider) ⇒ s.withSupervisionStrategy(decider) - case _ ⇒ s - } - } - } - - override def schedulePeriodically(initialDelay: FiniteDuration, interval: FiniteDuration, task: Runnable) = - system.scheduler.schedule(initialDelay, interval, task)(executionContext) - - override def scheduleOnce(delay: FiniteDuration, task: Runnable) = - system.scheduler.scheduleOnce(delay, task)(executionContext) - - override def materialize[Mat](_runnableGraph: Graph[ClosedShape, Mat]): Mat = - materialize(_runnableGraph, null, defaultInitialAttributes) - - override def materialize[Mat](_runnableGraph: Graph[ClosedShape, Mat], initialAttributes: Attributes): Mat = - materialize(_runnableGraph, null, initialAttributes) - - override def materialize[Mat](_runnableGraph: Graph[ClosedShape, Mat], subflowFuser: (GraphInterpreterShell) ⇒ ActorRef): Mat = - materialize(_runnableGraph, subflowFuser, defaultInitialAttributes) - - override def materialize[Mat]( - _runnableGraph: Graph[ClosedShape, Mat], - subflowFuser: GraphInterpreterShell ⇒ ActorRef, - initialAttributes: Attributes - ): Mat = { - val runnableGraph = - if (settings.autoFusing) Fusing.aggressive(_runnableGraph) - else _runnableGraph - - if (haveShutDown.get()) - throw new IllegalStateException("Attempted to call materialize() after the ActorMaterializer has been shut down.") - if (StreamLayout.Debug) StreamLayout.validate(runnableGraph.module) - - val session = new MaterializerSession(runnableGraph.module, initialAttributes) { - private val flowName = createFlowName() - private var nextId = 0 - private def stageName(attr: Attributes): String = { - val name = s"$flowName-$nextId-${attr.nameOrDefault()}" - nextId += 1 - name - } - - override protected def materializeAtomic(atomic: AtomicModule, effectiveAttributes: Attributes, matVal: ju.Map[Module, Any]): Unit = { - if (MaterializerSession.Debug) println(s"materializing $atomic") - - def newMaterializationContext() = - new MaterializationContext(ActorMaterializerImpl.this, effectiveAttributes, stageName(effectiveAttributes)) - atomic match { - case sink: SinkModule[_, _] ⇒ - val (sub, mat) = sink.create(newMaterializationContext()) - assignPort(sink.shape.in, sub) - matVal.put(atomic, mat) - case source: SourceModule[_, _] ⇒ - val (pub, mat) = source.create(newMaterializationContext()) - assignPort(source.shape.out, pub.asInstanceOf[Publisher[Any]]) - matVal.put(atomic, mat) - - case stage: ProcessorModule[_, _, _] ⇒ - val (processor, mat) = stage.createProcessor() - assignPort(stage.inPort, processor) - assignPort(stage.outPort, processor.asInstanceOf[Publisher[Any]]) - matVal.put(atomic, mat) - - case tls: TlsModule ⇒ // TODO solve this so TlsModule doesn't need special treatment here - val es = effectiveSettings(effectiveAttributes) - val props = - TLSActor.props(es, tls.createSSLEngine, tls.verifySession, tls.closing) - val impl = actorOf(props, stageName(effectiveAttributes), es.dispatcher) - def factory(id: Int) = new ActorPublisher[Any](impl) { - override val wakeUpMsg = FanOut.SubstreamSubscribePending(id) - } - val publishers = Vector.tabulate(2)(factory) - impl ! FanOut.ExposedPublishers(publishers) - - assignPort(tls.plainOut, publishers(TLSActor.UserOut)) - assignPort(tls.cipherOut, publishers(TLSActor.TransportOut)) - - assignPort(tls.plainIn, FanIn.SubInput[Any](impl, TLSActor.UserIn)) - assignPort(tls.cipherIn, FanIn.SubInput[Any](impl, TLSActor.TransportIn)) - - matVal.put(atomic, NotUsed) - - case graph: GraphModule ⇒ - matGraph(graph, effectiveAttributes, matVal) - - case stage: GraphStageModule ⇒ - val graph = - GraphModule( - GraphAssembly(stage.shape.inlets, stage.shape.outlets, stage.stage), - stage.shape, stage.attributes, Array(stage)) - matGraph(graph, effectiveAttributes, matVal) - } - } - - private def matGraph(graph: GraphModule, effectiveAttributes: Attributes, matVal: ju.Map[Module, Any]): Unit = { - val calculatedSettings = effectiveSettings(effectiveAttributes) - val (connections, logics) = graph.assembly.materialize(effectiveAttributes, graph.matValIDs, matVal, registerSrc) - - val shell = new GraphInterpreterShell(graph.assembly, connections, logics, graph.shape, - calculatedSettings, ActorMaterializerImpl.this) - - val impl = - if (subflowFuser != null && !effectiveAttributes.contains(Attributes.AsyncBoundary)) { - subflowFuser(shell) - } else { - val props = ActorGraphInterpreter.props(shell) - actorOf(props, stageName(effectiveAttributes), calculatedSettings.dispatcher) - } - - for ((inlet, i) ← graph.shape.inlets.iterator.zipWithIndex) { - val subscriber = new ActorGraphInterpreter.BoundarySubscriber(impl, shell, i) - assignPort(inlet, subscriber) - } - for ((outlet, i) ← graph.shape.outlets.iterator.zipWithIndex) { - val publisher = new ActorGraphInterpreter.BoundaryPublisher(impl, shell, i) - impl ! ActorGraphInterpreter.ExposedPublisher(shell, i, publisher) - assignPort(outlet, publisher) - } - } - - } - - session.materialize().asInstanceOf[Mat] - } - - override def makeLogger(logSource: Class[_]): LoggingAdapter = - Logging(system, logSource) - - override lazy val executionContext: ExecutionContextExecutor = dispatchers.lookup(settings.dispatcher match { - case Deploy.NoDispatcherGiven ⇒ Dispatchers.DefaultDispatcherId - case other ⇒ other - }) - -} - private[akka] class SubFusingActorMaterializerImpl(val delegate: ExtendedActorMaterializer, registerShell: GraphInterpreterShell ⇒ ActorRef) extends Materializer { override def executionContext: ExecutionContextExecutor = delegate.executionContext diff --git a/akka-stream/src/main/scala/akka/stream/impl/Modules.scala b/akka-stream/src/main/scala/akka/stream/impl/Modules.scala index d30462484f..3f2966e751 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/Modules.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/Modules.scala @@ -8,6 +8,7 @@ import akka.actor._ import akka.stream._ import akka.stream.impl.StreamLayout.AtomicModule import org.reactivestreams._ + import scala.annotation.unchecked.uncheckedVariance import scala.concurrent.Promise import akka.event.Logging @@ -15,29 +16,30 @@ import akka.event.Logging /** * INTERNAL API */ -abstract class SourceModule[+Out, +Mat](val shape: SourceShape[Out]) extends AtomicModule { +abstract class SourceModule[+Out, +Mat](val shape: SourceShape[Out]) extends AtomicModule[SourceShape[Out], Mat] { protected def label: String = Logging.simpleName(this) final override def toString: String = f"$label [${System.identityHashCode(this)}%08x]" def create(context: MaterializationContext): (Publisher[Out] @uncheckedVariance, Mat) - override def replaceShape(s: Shape): AtomicModule = - if (s != shape) throw new UnsupportedOperationException("cannot replace the shape of a Source, you need to wrap it in a Graph for that") - else this - - // This is okay since the only caller of this method is right below. + // TODO: Remove this, no longer needed? protected def newInstance(shape: SourceShape[Out] @uncheckedVariance): SourceModule[Out, Mat] - override def carbonCopy: AtomicModule = newInstance(SourceShape(shape.out.carbonCopy())) + // TODO: Amendshape changed the name of ports. Is it needed anymore? + + def attributes: Attributes protected def amendShape(attr: Attributes): SourceShape[Out] = { - val thisN = attributes.nameOrDefault(null) + val thisN = traversalBuilder.attributes.nameOrDefault(null) val thatN = attr.nameOrDefault(null) if ((thatN eq null) || thisN == thatN) shape else shape.copy(out = Outlet(thatN + ".out")) } + + override private[stream] def traversalBuilder = LinearTraversalBuilder.fromModule(this, attributes).makeIsland(SourceModuleIslandTag) + } /** @@ -53,7 +55,7 @@ final class SubscriberSource[Out](val attributes: Attributes, shape: SourceShape } override protected def newInstance(shape: SourceShape[Out]): SourceModule[Out, Subscriber[Out]] = new SubscriberSource[Out](attributes, shape) - override def withAttributes(attr: Attributes): AtomicModule = new SubscriberSource[Out](attr, amendShape(attr)) + override def withAttributes(attr: Attributes): SourceModule[Out, Subscriber[Out]] = new SubscriberSource[Out](attr, amendShape(attr)) } /** @@ -70,7 +72,7 @@ final class PublisherSource[Out](p: Publisher[Out], val attributes: Attributes, override def create(context: MaterializationContext) = (p, NotUsed) override protected def newInstance(shape: SourceShape[Out]): SourceModule[Out, NotUsed] = new PublisherSource[Out](p, attributes, shape) - override def withAttributes(attr: Attributes): AtomicModule = new PublisherSource[Out](p, attr, amendShape(attr)) + override def withAttributes(attr: Attributes): SourceModule[Out, NotUsed] = new PublisherSource[Out](p, attr, amendShape(attr)) } /** @@ -83,7 +85,7 @@ final class MaybeSource[Out](val attributes: Attributes, shape: SourceShape[Out] new MaybePublisher[Out](p, attributes.nameOrDefault("MaybeSource"))(context.materializer.executionContext) → p } override protected def newInstance(shape: SourceShape[Out]): SourceModule[Out, Promise[Option[Out]]] = new MaybeSource[Out](attributes, shape) - override def withAttributes(attr: Attributes): AtomicModule = new MaybeSource(attr, amendShape(attr)) + override def withAttributes(attr: Attributes): SourceModule[Out, Promise[Option[Out]]] = new MaybeSource(attr, amendShape(attr)) } /** @@ -100,7 +102,7 @@ final class ActorPublisherSource[Out](props: Props, val attributes: Attributes, override protected def newInstance(shape: SourceShape[Out]): SourceModule[Out, ActorRef] = new ActorPublisherSource[Out](props, attributes, shape) - override def withAttributes(attr: Attributes): AtomicModule = new ActorPublisherSource(props, attr, amendShape(attr)) + override def withAttributes(attr: Attributes): SourceModule[Out, ActorRef] = new ActorPublisherSource(props, attr, amendShape(attr)) } /** @@ -120,6 +122,6 @@ final class ActorRefSource[Out]( override protected def newInstance(shape: SourceShape[Out]): SourceModule[Out, ActorRef] = new ActorRefSource[Out](bufferSize, overflowStrategy, attributes, shape) - override def withAttributes(attr: Attributes): AtomicModule = + override def withAttributes(attr: Attributes): SourceModule[Out, ActorRef] = new ActorRefSource(bufferSize, overflowStrategy, attr, amendShape(attr)) } diff --git a/akka-stream/src/main/scala/akka/stream/impl/PhasedFusingActorMaterializer.scala b/akka-stream/src/main/scala/akka/stream/impl/PhasedFusingActorMaterializer.scala new file mode 100644 index 0000000000..03478a917f --- /dev/null +++ b/akka-stream/src/main/scala/akka/stream/impl/PhasedFusingActorMaterializer.scala @@ -0,0 +1,685 @@ +package akka.stream.impl + +import java.util.ArrayList +import java.util.concurrent.atomic.AtomicBoolean + +import akka.NotUsed +import akka.actor.{ ActorContext, ActorRef, ActorRefFactory, ActorSystem, Cancellable, Deploy, ExtendedActorSystem, PoisonPill, Props } +import akka.dispatch.Dispatchers +import akka.event.{ Logging, LoggingAdapter } +import akka.stream._ +import akka.stream.impl.StreamLayout.AtomicModule +import akka.stream.impl.fusing.ActorGraphInterpreter.{ ActorOutputBoundary, BatchingActorInputBoundary } +import akka.stream.impl.fusing.GraphInterpreter.Connection +import akka.stream.impl.fusing._ +import akka.stream.stage.{ GraphStageLogic, InHandler, OutHandler } +import org.reactivestreams.{ Publisher, Subscriber, Subscription } + +import scala.collection.immutable.Map +import scala.concurrent.duration.FiniteDuration +import scala.concurrent.{ ExecutionContextExecutor, Future } +import scala.util.Random + +object PhasedFusingActorMaterializer { + + val Debug = false + + val DefaultPhase: Phase[Any] = new Phase[Any] { + override def apply(settings: ActorMaterializerSettings, materializer: PhasedFusingActorMaterializer): PhaseIsland[Any] = + new GraphStageIsland(settings, materializer).asInstanceOf[PhaseIsland[Any]] + } + + val DefaultPhases: Map[IslandTag, Phase[Any]] = Map[IslandTag, Phase[Any]]( + SinkModuleIslandTag → new Phase[Any] { + override def apply(settings: ActorMaterializerSettings, materializer: PhasedFusingActorMaterializer): PhaseIsland[Any] = + (new SinkModulePhase(materializer)).asInstanceOf[PhaseIsland[Any]] + }, + SourceModuleIslandTag → new Phase[Any] { + override def apply(settings: ActorMaterializerSettings, materializer: PhasedFusingActorMaterializer): PhaseIsland[Any] = + new SourceModulePhase(materializer).asInstanceOf[PhaseIsland[Any]] + }, + GraphStageTag → DefaultPhase + ) + + def apply(settings: ActorMaterializerSettings)(implicit context: ActorRefFactory): ActorMaterializer = { + val haveShutDown = new AtomicBoolean(false) + val system = actorSystemOf(context) + val materializerSettings = ActorMaterializerSettings(system) + + PhasedFusingActorMaterializer( + system, + materializerSettings, + system.dispatchers, + context.actorOf(StreamSupervisor.props(materializerSettings, haveShutDown).withDispatcher(materializerSettings.dispatcher), StreamSupervisor.nextName()), + haveShutDown, + FlowNames(system).name.copy("flow")) + } + + private def actorSystemOf(context: ActorRefFactory): ActorSystem = { + val system = context match { + case s: ExtendedActorSystem ⇒ s + case c: ActorContext ⇒ c.system + case null ⇒ throw new IllegalArgumentException("ActorRefFactory context must be defined") + case _ ⇒ + throw new IllegalArgumentException(s"ActorRefFactory context must be an ActorSystem or ActorContext, got [${context.getClass.getName}]") + } + system + } + +} + +private case class SegmentInfo( + globalislandOffset: Int, // The island to which the segment belongs + length: Int, // How many slots are contained by the segment + globalBaseOffset: Int, // The global slot where this segment starts + relativeBaseOffset: Int, // the local offset of the slot where this segment starts + phase: PhaseIsland[Any]) { + + override def toString: String = + s""" + | Segment + | globalislandOffset = $globalislandOffset + | length = $length + | globalBaseOffset = $globalBaseOffset + | relativeBaseOffset = $relativeBaseOffset + | phase = $phase + """.stripMargin +} + +private case class ForwardWire( + islandGlobalOffset: Int, + from: OutPort, + toGlobalOffset: Int, + outStage: Any, + phase: PhaseIsland[Any]) { + + override def toString: String = s"ForwardWire(islandId = $islandGlobalOffset, from = $from, toGlobal = $toGlobalOffset, phase = $phase)" +} + +class IslandTracking( + val phases: Map[IslandTag, Phase[Any]], + val settings: ActorMaterializerSettings, + defaultPhase: Phase[Any], + val materializer: PhasedFusingActorMaterializer +) { + + import PhasedFusingActorMaterializer.Debug + + private var currentGlobalOffset = 0 + private var currentSegmentGlobalOffset = 0 + private var currentIslandGlobalOffset = 0 + // The number of slots that belong to segments of other islands encountered so far, from the + // beginning of the island + private var currentIslandSkippetSlots = 0 + + private var segments: java.util.ArrayList[SegmentInfo] = null + private var forwardWires: java.util.ArrayList[ForwardWire] = null + + private var currentPhase: PhaseIsland[Any] = defaultPhase.apply(settings, materializer) + + def getCurrentPhase: PhaseIsland[Any] = currentPhase + def getCurrentOffset: Int = currentGlobalOffset + + private def completeSegment(): Int = { + val length = currentGlobalOffset - currentSegmentGlobalOffset + + if (length > 0) { + // We just finished a segment by entering an island. + val previousSegment = SegmentInfo( + globalislandOffset = currentIslandGlobalOffset, + length = currentGlobalOffset - currentSegmentGlobalOffset, + globalBaseOffset = currentSegmentGlobalOffset, + relativeBaseOffset = currentSegmentGlobalOffset - currentIslandGlobalOffset - currentIslandSkippetSlots, + currentPhase + ) + + // Segment tracking is by demand, we only allocate this list if it is used. + // If there are no islands, then there is no need to track segments + if (segments eq null) segments = new java.util.ArrayList[SegmentInfo](8) + segments.add(previousSegment) + if (Debug) println(s"Completed segment $previousSegment") + } else { + if (Debug) println(s"Skipped zero length segment") + } + + length + } + + def enterIsland(tag: IslandTag): ExitIsland = { + completeSegment() + val previousPhase = currentPhase + val previousIslandOffset = currentIslandGlobalOffset + + currentPhase = phases(tag)(settings, materializer) + if (Debug) println(s"Entering island starting at offset = $currentIslandGlobalOffset phase = $currentPhase") + + // Resolve the phase to be used to materialize this island + currentIslandGlobalOffset = currentGlobalOffset + + // The base offset of this segment is the current global offset + currentSegmentGlobalOffset = currentGlobalOffset + ExitIsland(previousIslandOffset, currentIslandSkippetSlots, previousPhase) + } + + def exitIsland(exitIsland: ExitIsland): Unit = { + val previousSegmentLength = completeSegment() + + // Closing previous island + currentPhase.onIslandReady() + + // We start a new segment + currentSegmentGlobalOffset = currentGlobalOffset + + // We restore data for the island + currentIslandGlobalOffset = exitIsland.islandGlobalOffset + currentPhase = exitIsland.phase + currentIslandSkippetSlots = exitIsland.skippedSlots + previousSegmentLength + + if (Debug) println(s"Exited to island starting at offset = $currentIslandGlobalOffset phase = $currentPhase") + } + + def wireIn(in: InPort, logic: Any): Unit = { + // The slot for this InPort always belong to the current segment, so resolving its local + // offset/slot is simple + val localInSlot = currentGlobalOffset - currentIslandGlobalOffset - currentIslandSkippetSlots + if (Debug) println(s" wiring port $in inOffs absolute = $currentGlobalOffset local = $localInSlot") + + // Assign the logic belonging to the current port to its calculated local slot in the island + currentPhase.assignPort(in, localInSlot, logic) + + // Check if there was any forward wiring that has this offset/slot as its target + // First try to find such wiring + var forwardWire: ForwardWire = null + if ((forwardWires ne null) && !forwardWires.isEmpty) { + var i = 0 + while (i < forwardWires.size()) { + forwardWire = forwardWires.get(i) + if (forwardWire.toGlobalOffset == currentGlobalOffset) { + if (Debug) println(s" there is a forward wire to this slot $forwardWire") + forwardWires.remove(i) + i = Int.MaxValue // Exit the loop + } else { + forwardWire = null // Didn't found it yet + i += 1 + } + } + } + + // If there is a forward wiring we need to resolve it + if (forwardWire ne null) { + // The forward wire ends up in the same island + if (forwardWire.phase eq currentPhase) { + if (Debug) println(s" in-island forward wiring from port ${forwardWire.from} wired to local slot = $localInSlot") + forwardWire.phase.assignPort(forwardWire.from, localInSlot, forwardWire.outStage) + } else { + if (Debug) println(s" cross island forward wiring from port ${forwardWire.from} wired to local slot = $localInSlot") + val publisher = forwardWire.phase.createPublisher(forwardWire.from, forwardWire.outStage) + currentPhase.takePublisher(localInSlot, publisher) + } + } + + currentGlobalOffset += 1 + } + + def wireOut(out: OutPort, absoluteOffset: Int, logic: Any): Unit = { + // TODO: forward wires + if (Debug) println(s" wiring $out to absolute = $absoluteOffset") + + // First check if we are wiring backwards. This is important since we can only do resolution for backward wires. + // In other cases we need to record the forward wire and resolve it later once its target inSlot has been visited. + if (absoluteOffset < currentGlobalOffset) { + if (Debug) println(" backward wiring") + + if (absoluteOffset >= currentSegmentGlobalOffset) { + // Wiring is in the same segment, no complex lookup needed + val localInSlot = absoluteOffset - currentIslandGlobalOffset - currentIslandSkippetSlots + if (Debug) println(s" in-segment wiring to local ($absoluteOffset - $currentIslandGlobalOffset - $currentIslandSkippetSlots) = $localInSlot") + currentPhase.assignPort(out, localInSlot, logic) + } else { + // Wiring is cross-segment, but we don't know if it is cross-island or not yet + // We must find the segment to which this slot belongs first + var i = segments.size() - 1 + var targetSegment: SegmentInfo = segments.get(i) + // Skip segments that have a higher offset than our slot, until we find the containing segment + while (i > 0 && targetSegment.globalBaseOffset > absoluteOffset) { + i -= 1 + targetSegment = segments.get(i) + } + + // Independently of the target island the local slot for the target island is calculated the same: + // - Calculate the relative offset of the local slot in the segment + // - calculate the island relative offset by adding the island relative base offset of the segment + val distanceFromSegmentStart = absoluteOffset - targetSegment.globalBaseOffset + val localInSlot = distanceFromSegmentStart + targetSegment.relativeBaseOffset + + if (targetSegment.phase eq currentPhase) { + if (Debug) println(s" cross-segment, in-island wiring to local slot $localInSlot") + currentPhase.assignPort(out, localInSlot, logic) + } else { + if (Debug) println(s" cross-island wiring to local slot $localInSlot in target island") + val publisher = currentPhase.createPublisher(out, logic) + targetSegment.phase.takePublisher(localInSlot, publisher) + } + } + } else { + // We need to record the forward wiring so we can resolve it later + + // The forward wire tracking data structure is only allocated when needed. Many graphs have no forward wires + // even though it might have islands. + if (forwardWires eq null) { + forwardWires = new java.util.ArrayList[ForwardWire](8) + } + + val forwardWire = ForwardWire( + islandGlobalOffset = currentIslandGlobalOffset, + from = out, + toGlobalOffset = absoluteOffset, + logic, + currentPhase + ) + + if (Debug) println(s" wiring is forward, recording $forwardWire") + forwardWires.add(forwardWire) + } + + } + +} + +case class PhasedFusingActorMaterializer( + system: ActorSystem, + override val settings: ActorMaterializerSettings, + dispatchers: Dispatchers, + supervisor: ActorRef, + haveShutDown: AtomicBoolean, + flowNames: SeqActorName +) extends ExtendedActorMaterializer { + import PhasedFusingActorMaterializer._ + + private val _logger = Logging.getLogger(system, this) + override def logger: LoggingAdapter = _logger + + if (settings.fuzzingMode && !system.settings.config.hasPath("akka.stream.secret-test-fuzzing-warning-disable")) { + _logger.warning("Fuzzing mode is enabled on this system. If you see this warning on your production system then " + + "set akka.stream.materializer.debug.fuzzing-mode to off.") + } + + override def shutdown(): Unit = + if (haveShutDown.compareAndSet(false, true)) supervisor ! PoisonPill + + override def isShutdown: Boolean = haveShutDown.get() + + override def withNamePrefix(name: String): PhasedFusingActorMaterializer = this.copy(flowNames = flowNames.copy(name)) + + private[this] def createFlowName(): String = flowNames.next() + + private val defaultInitialAttributes = Attributes( + Attributes.InputBuffer(settings.initialInputBufferSize, settings.maxInputBufferSize) :: + ActorAttributes.Dispatcher(settings.dispatcher) :: + ActorAttributes.SupervisionStrategy(settings.supervisionDecider) :: + Nil) + + override def effectiveSettings(opAttr: Attributes): ActorMaterializerSettings = { + import ActorAttributes._ + import Attributes._ + opAttr.attributeList.foldLeft(settings) { (s, attr) ⇒ + attr match { + case InputBuffer(initial, max) ⇒ s.withInputBuffer(initial, max) + case Dispatcher(dispatcher) ⇒ s.withDispatcher(dispatcher) + case SupervisionStrategy(decider) ⇒ s.withSupervisionStrategy(decider) + case _ ⇒ s + } + } + } + + override def schedulePeriodically(initialDelay: FiniteDuration, interval: FiniteDuration, task: Runnable): Cancellable = + system.scheduler.schedule(initialDelay, interval, task)(executionContext) + + override def scheduleOnce(delay: FiniteDuration, task: Runnable): Cancellable = + system.scheduler.scheduleOnce(delay, task)(executionContext) + + override def materialize[Mat](_runnableGraph: Graph[ClosedShape, Mat]): Mat = + materialize(_runnableGraph, null, defaultInitialAttributes) + + override def materialize[Mat](_runnableGraph: Graph[ClosedShape, Mat], initialAttributes: Attributes): Mat = + materialize(_runnableGraph, null, initialAttributes) + + override def materialize[Mat](_runnableGraph: Graph[ClosedShape, Mat], subflowFuser: (GraphInterpreterShell) ⇒ ActorRef): Mat = + materialize(_runnableGraph, subflowFuser, defaultInitialAttributes) + + override def makeLogger(logSource: Class[_]): LoggingAdapter = + Logging(system, logSource) + + override lazy val executionContext: ExecutionContextExecutor = dispatchers.lookup(settings.dispatcher match { + case Deploy.NoDispatcherGiven ⇒ Dispatchers.DefaultDispatcherId + case other ⇒ other + }) + + override def materialize[Mat]( + _runnableGraph: Graph[ClosedShape, Mat], + subflowFuser: (GraphInterpreterShell) ⇒ ActorRef, + initialAttributes: Attributes): Mat = { + materialize( + _runnableGraph, + subflowFuser, + initialAttributes, + PhasedFusingActorMaterializer.DefaultPhase, + PhasedFusingActorMaterializer.DefaultPhases + ) + } + + def materialize[Mat]( + graph: Graph[ClosedShape, Mat], + subflowFuser: GraphInterpreterShell ⇒ ActorRef, + initialAttributes: Attributes, + defaultPhase: Phase[Any], + phases: Map[IslandTag, Phase[Any]] + ): Mat = { + val islandTracking = new IslandTracking(phases, settings, defaultPhase, this) + + var current: Traversal = graph.traversalBuilder.traversal + + val attributesStack = new java.util.ArrayDeque[Attributes](8) + attributesStack.addLast(initialAttributes) + + // TODO: No longer need for a stack + val traversalStack = new java.util.ArrayDeque[Traversal](16) + traversalStack.addLast(current) + + val matValueStack = new java.util.ArrayDeque[Any](8) + + if (Debug) { + println(s"--- Materializing layout:") + TraversalBuilder.printTraversal(current) + println(s"--- Start materialization") + } + + // Due to how Concat works, we need a stack. This probably can be optimized for the most common cases. + while (!traversalStack.isEmpty) { + current = traversalStack.removeLast() + + while (current ne EmptyTraversal) { + var nextStep: Traversal = EmptyTraversal + current match { + case MaterializeAtomic(mod, outToSlot) ⇒ + if (Debug) println(s"materializing module: $mod") + val matAndStage = islandTracking.getCurrentPhase.materializeAtomic(mod, attributesStack.getLast) + val logic = matAndStage._1 + val matValue = matAndStage._2 + if (Debug) println(s" materialized value is $matValue") + matValueStack.addLast(matValue) + + val ins = mod.shape.inlets.iterator + val stageGlobalOffset = islandTracking.getCurrentOffset + + while (ins.hasNext) { + val in = ins.next() + islandTracking.wireIn(in, logic) + } + + val outs = mod.shape.outlets.iterator + while (outs.hasNext) { + val out = outs.next() + val absoluteTargetSlot = stageGlobalOffset + outToSlot(out.id) + if (Debug) println(s" wiring offset: ${outToSlot.mkString("[", ",", "]")}") + islandTracking.wireOut(out, absoluteTargetSlot, logic) + } + + if (Debug) println(s"PUSH: $matValue => $matValueStack") + + case Concat(first, next) ⇒ + if (next ne EmptyTraversal) traversalStack.add(next) + nextStep = first + case Pop ⇒ + val popped = matValueStack.removeLast() + if (Debug) println(s"POP: $popped => $matValueStack") + case PushNotUsed ⇒ + matValueStack.addLast(NotUsed) + if (Debug) println(s"PUSH: NotUsed => $matValueStack") + case Transform(f) ⇒ + val prev = matValueStack.removeLast() + val result = f(prev) + matValueStack.addLast(result) + if (Debug) println(s"TRFM: $matValueStack") + case Compose(f) ⇒ + val second = matValueStack.removeLast() + val first = matValueStack.removeLast() + val result = f(first, second) + matValueStack.addLast(result) + if (Debug) println(s"COMP: $matValueStack") + case PushAttributes(attr) ⇒ + attributesStack.addLast(attributesStack.getLast and attr) + if (Debug) println(s"ATTR PUSH: $attr") + case PopAttributes ⇒ + attributesStack.removeLast() + if (Debug) println(s"ATTR POP") + case EnterIsland(tag, island) ⇒ + traversalStack.addLast(islandTracking.enterIsland(tag)) + nextStep = island + case ex: ExitIsland ⇒ + islandTracking.exitIsland(ex) + case _ ⇒ + } + current = nextStep + } + } + + islandTracking.getCurrentPhase.onIslandReady() + + if (Debug) println("--- Finished materialization") + matValueStack.peekLast().asInstanceOf[Mat] + } + +} + +trait IslandTag + +trait Phase[M] { + def apply(settings: ActorMaterializerSettings, materializer: PhasedFusingActorMaterializer): PhaseIsland[M] +} + +trait PhaseIsland[M] { + + def name: String + + def materializeAtomic(mod: AtomicModule[Shape, Any], attributes: Attributes): (M, Any) + + def assignPort(in: InPort, slot: Int, logic: M): Unit + + def assignPort(out: OutPort, slot: Int, logic: M): Unit + + def createPublisher(out: OutPort, logic: M): Publisher[Any] + + def takePublisher(slot: Int, publisher: Publisher[Any]): Unit + + def onIslandReady(): Unit + +} + +object GraphStageTag extends IslandTag + +final class GraphStageIsland( + settings: ActorMaterializerSettings, + materializer: PhasedFusingActorMaterializer +) extends PhaseIsland[GraphStageLogic] { + // TODO: remove these + private val logicArrayType = Array.empty[GraphStageLogic] + private[this] val logics = new ArrayList[GraphStageLogic](64) + // TODO: Resize + private val connections = Array.ofDim[Connection](64) + private var maxConnections = 0 + private var outConnections: List[Connection] = Nil + + val shell = new GraphInterpreterShell( + connections = null, + logics = null, + settings, + materializer) + + override def name: String = "Fusing GraphStages phase" + + override def materializeAtomic(mod: AtomicModule[Shape, Any], attributes: Attributes): (GraphStageLogic, Any) = { + // TODO: bail on unknown types + val stageModule = mod.asInstanceOf[GraphStageModule[Shape, Any]] + val matAndLogic = stageModule.stage.createLogicAndMaterializedValue(attributes) + val logic = matAndLogic._1 + logics.add(logic) + logic.stageId = logics.size() - 1 + matAndLogic + } + + def conn(slot: Int): Connection = { + maxConnections = math.max(slot, maxConnections) + val c = connections(slot) + if (c ne null) c + else { + val c2 = new Connection(0, 0, null, 0, null, null, null) + connections(slot) = c2 + c2 + } + } + + def outConn(): Connection = { + val connection = new Connection(0, 0, null, 0, null, null, null) + outConnections ::= connection + connection + } + + override def assignPort(in: InPort, slot: Int, logic: GraphStageLogic): Unit = { + val connection = conn(slot) + connection.inOwner = logic + connection.id = slot + connection.inOwnerId = logic.stageId + connection.inHandler = logic.handlers(in.id).asInstanceOf[InHandler] + logic.portToConn(in.id) = connection + } + + override def assignPort(out: OutPort, slot: Int, logic: GraphStageLogic): Unit = { + val connection = conn(slot) + connection.outOwner = logic + connection.id = slot + connection.outOwnerId = logic.stageId + connection.outHandler = logic.handlers(logic.inCount + out.id).asInstanceOf[OutHandler] + logic.portToConn(logic.inCount + out.id) = connection + } + + override def createPublisher(out: OutPort, logic: GraphStageLogic): Publisher[Any] = { + val boundary = new ActorOutputBoundary(shell, out.toString) + logics.add(boundary) + boundary.stageId = logics.size() - 1 + + val connection = outConn() + boundary.portToConn(boundary.in.id) = connection + connection.inHandler = boundary.handlers(0).asInstanceOf[InHandler] + connection.inOwner = boundary + connection.inOwnerId = boundary.stageId + + connection.outOwner = logic + connection.id = -1 // Will be filled later + connection.outOwnerId = logic.stageId + connection.outHandler = logic.handlers(logic.inCount + out.id).asInstanceOf[OutHandler] + logic.portToConn(logic.inCount + out.id) = connection + + boundary.publisher + } + + override def takePublisher(slot: Int, publisher: Publisher[Any]): Unit = { + // TODO: proper input buffer sizes from attributes + val connection = conn(slot) + // TODO: proper input port debug string (currently prints the stage) + val boundary = new BatchingActorInputBoundary(16, shell, publisher, connection.inOwner.toString) + logics.add(boundary) + boundary.stageId = logics.size() - 1 + + boundary.portToConn(boundary.out.id + boundary.inCount) = connection + connection.outHandler = boundary.handlers(0).asInstanceOf[OutHandler] + connection.outOwner = boundary + connection.outOwnerId = boundary.stageId + } + + override def onIslandReady(): Unit = { + + val totalConnections = maxConnections + outConnections.size + 1 + val finalConnections = java.util.Arrays.copyOf(connections, totalConnections) + + var i = maxConnections + 1 + var outConns = outConnections + while (i < totalConnections) { + val conn = outConns.head + outConns = outConns.tail + finalConnections(i) = conn + conn.id = i + i += 1 + } + + shell.connections = finalConnections + shell.logics = logics.toArray(logicArrayType) + + // TODO: Subfusing + // if (subflowFuser != null) { + // subflowFuser(shell) + // } else { + val props = ActorGraphInterpreter.props(shell) + // TODO: actor names + materializer.actorOf(props, "fused" + Random.nextInt(), settings.dispatcher) + // } + + } + + override def toString: String = "GraphStagePhase" +} + +object SourceModuleIslandTag extends IslandTag + +final class SourceModulePhase(materializer: PhasedFusingActorMaterializer) extends PhaseIsland[Publisher[Any]] { + override def name: String = s"SourceModule phase" + + override def materializeAtomic(mod: AtomicModule[Shape, Any], attributes: Attributes): (Publisher[Any], Any) = { + // TODO: proper stage name + mod.asInstanceOf[SourceModule[Any, Any]].create(MaterializationContext(materializer, attributes, "stageName")) + } + + override def assignPort(in: InPort, slot: Int, logic: Publisher[Any]): Unit = () + + override def assignPort(out: OutPort, slot: Int, logic: Publisher[Any]): Unit = () + + override def createPublisher(out: OutPort, logic: Publisher[Any]): Publisher[Any] = logic + + override def takePublisher(slot: Int, publisher: Publisher[Any]): Unit = + throw new UnsupportedOperationException("A Source cannot take a Publisher") + + override def onIslandReady(): Unit = () +} + +object SinkModuleIslandTag extends IslandTag + +final class SinkModulePhase(materializer: PhasedFusingActorMaterializer) extends PhaseIsland[AnyRef] { + override def name: String = s"SourceModule phase" + var subscriberOrVirtualPublisher: AnyRef = _ + + override def materializeAtomic(mod: AtomicModule[Shape, Any], attributes: Attributes): (AnyRef, Any) = { + // TODO: proper stage name + val subAndMat = + mod.asInstanceOf[SinkModule[Any, Any]].create(MaterializationContext(materializer, attributes, "stageName")) + + subscriberOrVirtualPublisher = subAndMat._1 + (subscriberOrVirtualPublisher, subAndMat._2) + } + + override def assignPort(in: InPort, slot: Int, logic: AnyRef): Unit = () + + override def assignPort(out: OutPort, slot: Int, logic: AnyRef): Unit = () + + override def createPublisher(out: OutPort, logic: AnyRef): Publisher[Any] = { + throw new UnsupportedOperationException("A Sink cannot create a Publisher") + } + + override def takePublisher(slot: Int, publisher: Publisher[Any]): Unit = { + subscriberOrVirtualPublisher match { + case v: VirtualPublisher[Any] ⇒ v.registerPublisher(publisher) + case s: Subscriber[Any] ⇒ publisher.subscribe(s) + } + } + + override def onIslandReady(): Unit = () +} diff --git a/akka-stream/src/main/scala/akka/stream/impl/Sinks.scala b/akka-stream/src/main/scala/akka/stream/impl/Sinks.scala index 9eba8b717f..c3009e0559 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/Sinks.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/Sinks.scala @@ -5,40 +5,43 @@ package akka.stream.impl import akka.dispatch.ExecutionContexts import akka.stream.ActorAttributes.SupervisionStrategy -import akka.stream.Supervision.{ stoppingDecider, Stop } +import akka.stream.Supervision.{ Stop, stoppingDecider } import akka.stream.impl.QueueSink.{ Output, Pull } import akka.stream.impl.fusing.GraphInterpreter import akka.{ Done, NotUsed } -import akka.actor.{ ActorRef, Actor, Props } +import akka.actor.{ Actor, ActorRef, Props } import akka.stream.Attributes.InputBuffer import akka.stream._ import akka.stream.impl.Stages.DefaultAttributes import akka.stream.impl.StreamLayout.AtomicModule import java.util.concurrent.atomic.AtomicReference import java.util.function.BiConsumer + import akka.actor.{ ActorRef, Props } import akka.stream.Attributes.InputBuffer import akka.stream._ -import akka.stream.impl.StreamLayout.Module import akka.stream.stage._ import org.reactivestreams.{ Publisher, Subscriber } + import scala.annotation.unchecked.uncheckedVariance import scala.collection.immutable -import scala.concurrent.{ ExecutionContext, Promise, Future } +import scala.concurrent.{ ExecutionContext, Future, Promise } import scala.language.postfixOps import scala.util.control.NonFatal import scala.util.{ Failure, Success, Try } -import akka.stream.scaladsl.{ Source, Sink, SinkQueueWithCancel, SinkQueue } +import akka.stream.scaladsl.{ Sink, SinkQueue, SinkQueueWithCancel, Source } import java.util.concurrent.CompletionStage + import scala.compat.java8.FutureConverters._ import scala.compat.java8.OptionConverters._ import java.util.Optional + import akka.event.Logging /** * INTERNAL API */ -abstract class SinkModule[-In, Mat](val shape: SinkShape[In]) extends AtomicModule { +abstract class SinkModule[-In, Mat](val shape: SinkShape[In]) extends AtomicModule[SinkShape[In], Mat] { /** * Create the Subscriber or VirtualPublisher that consumes the incoming @@ -49,17 +52,15 @@ abstract class SinkModule[-In, Mat](val shape: SinkShape[In]) extends AtomicModu */ def create(context: MaterializationContext): (AnyRef, Mat) - override def replaceShape(s: Shape): AtomicModule = - if (s != shape) throw new UnsupportedOperationException("cannot replace the shape of a Sink, you need to wrap it in a Graph for that") - else this + override def traversalBuilder: TraversalBuilder = + LinearTraversalBuilder.fromModule(this).makeIsland(SinkModuleIslandTag) // This is okay since we the only caller of this method is right below. + // TODO: Remove this, no longer needed protected def newInstance(s: SinkShape[In] @uncheckedVariance): SinkModule[In, Mat] - override def carbonCopy: AtomicModule = newInstance(SinkShape(shape.in.carbonCopy())) - protected def amendShape(attr: Attributes): SinkShape[In] = { - val thisN = attributes.nameOrDefault(null) + val thisN = traversalBuilder.attributes.nameOrDefault(null) val thatN = attr.nameOrDefault(null) if ((thatN eq null) || thisN == thatN) shape @@ -91,7 +92,7 @@ private[akka] class PublisherSink[In](val attributes: Attributes, shape: SinkSha } override protected def newInstance(shape: SinkShape[In]): SinkModule[In, Publisher[In]] = new PublisherSink[In](attributes, shape) - override def withAttributes(attr: Attributes): AtomicModule = new PublisherSink[In](attr, amendShape(attr)) + override def withAttributes(attr: Attributes): SinkModule[In, Publisher[In]] = new PublisherSink[In](attr, amendShape(attr)) } /** @@ -116,7 +117,7 @@ private[akka] final class FanoutPublisherSink[In]( override protected def newInstance(shape: SinkShape[In]): SinkModule[In, Publisher[In]] = new FanoutPublisherSink[In](attributes, shape) - override def withAttributes(attr: Attributes): AtomicModule = + override def withAttributes(attr: Attributes): SinkModule[In, Publisher[In]] = new FanoutPublisherSink[In](attr, amendShape(attr)) } @@ -129,7 +130,7 @@ final class SubscriberSink[In](subscriber: Subscriber[In], val attributes: Attri override def create(context: MaterializationContext) = (subscriber, NotUsed) override protected def newInstance(shape: SinkShape[In]): SinkModule[In, NotUsed] = new SubscriberSink[In](subscriber, attributes, shape) - override def withAttributes(attr: Attributes): AtomicModule = new SubscriberSink[In](subscriber, attr, amendShape(attr)) + override def withAttributes(attr: Attributes): SinkModule[In, NotUsed] = new SubscriberSink[In](subscriber, attr, amendShape(attr)) } /** @@ -139,7 +140,7 @@ final class SubscriberSink[In](subscriber: Subscriber[In], val attributes: Attri final class CancelSink(val attributes: Attributes, shape: SinkShape[Any]) extends SinkModule[Any, NotUsed](shape) { override def create(context: MaterializationContext): (Subscriber[Any], NotUsed) = (new CancellingSubscriber[Any], NotUsed) override protected def newInstance(shape: SinkShape[Any]): SinkModule[Any, NotUsed] = new CancelSink(attributes, shape) - override def withAttributes(attr: Attributes): AtomicModule = new CancelSink(attr, amendShape(attr)) + override def withAttributes(attr: Attributes): SinkModule[Any, NotUsed] = new CancelSink(attr, amendShape(attr)) } /** @@ -155,7 +156,7 @@ final class ActorSubscriberSink[In](props: Props, val attributes: Attributes, sh } override protected def newInstance(shape: SinkShape[In]): SinkModule[In, ActorRef] = new ActorSubscriberSink[In](props, attributes, shape) - override def withAttributes(attr: Attributes): AtomicModule = new ActorSubscriberSink[In](props, attr, amendShape(attr)) + override def withAttributes(attr: Attributes): SinkModule[In, ActorRef] = new ActorSubscriberSink[In](props, attr, amendShape(attr)) } /** @@ -176,7 +177,7 @@ final class ActorRefSink[In](ref: ActorRef, onCompleteMessage: Any, override protected def newInstance(shape: SinkShape[In]): SinkModule[In, NotUsed] = new ActorRefSink[In](ref, onCompleteMessage, attributes, shape) - override def withAttributes(attr: Attributes): AtomicModule = + override def withAttributes(attr: Attributes): SinkModule[In, NotUsed] = new ActorRefSink[In](ref, onCompleteMessage, attr, amendShape(attr)) } diff --git a/akka-stream/src/main/scala/akka/stream/impl/StreamLayout.scala b/akka-stream/src/main/scala/akka/stream/impl/StreamLayout.scala index fd57c1dbad..cacffef7f9 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/StreamLayout.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/StreamLayout.scala @@ -4,23 +4,13 @@ package akka.stream.impl import java.util.concurrent.atomic.AtomicReference -import java.{ util ⇒ ju } -import akka.NotUsed -import akka.event.Logging -import akka.event.Logging.simpleName import akka.stream._ -import akka.stream.impl.MaterializerSession.MaterializationPanic import akka.stream.impl.Stages.DefaultAttributes -import akka.stream.impl.StreamLayout.Module -import akka.stream.impl.fusing.GraphModule -import akka.stream.impl.fusing.GraphStages.MaterializedValueSource -import akka.stream.scaladsl.Keep import org.reactivestreams.{ Processor, Publisher, Subscriber, Subscription } import scala.annotation.tailrec -import scala.collection.JavaConverters._ -import scala.util.control.{ NoStackTrace, NonFatal } +import scala.util.control.NonFatal /** * INTERNAL API @@ -30,497 +20,12 @@ object StreamLayout { // compile-time constant final val Debug = false - final def validate(m: Module, level: Int = 0, doPrint: Boolean = false, idMap: ju.Map[AnyRef, Integer] = new ju.HashMap): Unit = { - val ids = Iterator from 1 - def id(obj: AnyRef) = idMap get obj match { - case null ⇒ - val x = ids.next() - idMap.put(obj, x) - x - case x ⇒ x - } - def in(i: InPort) = s"${i.toString}@${id(i)}" - def out(o: OutPort) = s"${o.toString}@${id(o)}" - def ins(i: Iterable[InPort]) = i.map(in).mkString("In[", ",", "]") - def outs(o: Iterable[OutPort]) = o.map(out).mkString("Out[", ",", "]") - def pair(p: (OutPort, InPort)) = s"${in(p._2)}->${out(p._1)}" - def pairs(p: Iterable[(OutPort, InPort)]) = p.map(pair).mkString("[", ",", "]") - - import m._ - - val inset: Set[InPort] = shape.inlets.toSet - val outset: Set[OutPort] = shape.outlets.toSet - var problems: List[String] = Nil - - if (inset.size != shape.inlets.size) problems ::= "shape has duplicate inlets: " + ins(shape.inlets) - if (inset != inPorts) problems ::= s"shape has extra ${ins(inset diff inPorts)}, module has extra ${ins(inPorts diff inset)}" - if (inset.intersect(upstreams.keySet).nonEmpty) problems ::= s"found connected inlets ${inset.intersect(upstreams.keySet)}" - if (outset.size != shape.outlets.size) problems ::= "shape has duplicate outlets: " + outs(shape.outlets) - if (outset != outPorts) problems ::= s"shape has extra ${outs(outset diff outPorts)}, module has extra ${outs(outPorts diff outset)}" - if (outset.intersect(downstreams.keySet).nonEmpty) problems ::= s"found connected outlets ${outset.intersect(downstreams.keySet)}" - val ups = upstreams.toSet - val ups2 = ups.map(_.swap) - val downs = downstreams.toSet - val inter = ups2.intersect(downs) - if (downs != ups2) problems ::= s"inconsistent maps: ups ${pairs(ups2 diff inter)} downs ${pairs(downs diff inter)}" - val (allIn, dupIn, allOut, dupOut) = - subModules.foldLeft((Set.empty[InPort], Set.empty[InPort], Set.empty[OutPort], Set.empty[OutPort])) { - case ((ai, di, ao, doo), sm) ⇒ - (ai ++ sm.inPorts, di ++ ai.intersect(sm.inPorts), ao ++ sm.outPorts, doo ++ ao.intersect(sm.outPorts)) - } - if (dupIn.nonEmpty) problems ::= s"duplicate ports in submodules ${ins(dupIn)}" - if (dupOut.nonEmpty) problems ::= s"duplicate ports in submodules ${outs(dupOut)}" - if (!isSealed && (inset diff allIn).nonEmpty) problems ::= s"foreign inlets ${ins(inset diff allIn)}" - if (!isSealed && (outset diff allOut).nonEmpty) problems ::= s"foreign outlets ${outs(outset diff allOut)}" - val unIn = allIn diff inset diff upstreams.keySet - if (unIn.nonEmpty && !isCopied) problems ::= s"unconnected inlets ${ins(unIn)}" - val unOut = allOut diff outset diff downstreams.keySet - if (unOut.nonEmpty && !isCopied) problems ::= s"unconnected outlets ${outs(unOut)}" - - def atomics(n: MaterializedValueNode): Set[Module] = - n match { - case Ignore ⇒ Set.empty - case Transform(f, dep) ⇒ atomics(dep) - case Atomic(module) ⇒ Set(module) - case Combine(f, left, right) ⇒ atomics(left) ++ atomics(right) - } - val atomic = atomics(materializedValueComputation) - val graphValues = subModules.flatMap { - case GraphModule(_, _, _, mvids) ⇒ mvids - case _ ⇒ Nil - } - if (((atomic diff subModules diff graphValues) - m).nonEmpty) - problems ::= s"computation refers to non-existent modules [${(atomic diff subModules diff graphValues) - m mkString ","}]" - - val print = doPrint || problems.nonEmpty - - if (print) { - val indent = " " * (level * 2) - println(s"$indent${simpleName(this)}($shape): ${ins(inPorts)} ${outs(outPorts)}") - downstreams foreach { case (o, i) ⇒ println(s"$indent ${out(o)} -> ${in(i)}") } - problems foreach (p ⇒ println(s"$indent -!- $p")) - } - - subModules foreach (sm ⇒ validate(sm, level + 1, print, idMap)) - - if (problems.nonEmpty && !doPrint) throw new IllegalStateException(s"module inconsistent, found ${problems.size} problems") - } - - object IgnorableMatValComp { - def apply(comp: MaterializedValueNode): Boolean = - comp match { - case Atomic(module) ⇒ IgnorableMatValComp(module) - case _: Combine | _: Transform ⇒ false - case Ignore ⇒ true - } - def apply(module: Module): Boolean = - module match { - case _: AtomicModule | EmptyModule ⇒ true - case CopiedModule(_, _, module) ⇒ IgnorableMatValComp(module) - case CompositeModule(_, _, _, _, comp, _) ⇒ IgnorableMatValComp(comp) - case FusedModule(_, _, _, _, comp, _, _) ⇒ IgnorableMatValComp(comp) - case StructuralInfoModule(_, _, _, _, _, _, _, comp, _) ⇒ IgnorableMatValComp(comp) - } - } - - sealed trait MaterializedValueNode { - /* - * These nodes are used in hash maps and therefore must have efficient implementations - * of hashCode and equals. There is no value in allowing aliases to be equal, so using - * reference equality. - */ - override def hashCode: Int = super.hashCode - override def equals(other: Any): Boolean = super.equals(other) - } - case class Combine(f: (Any, Any) ⇒ Any, dep1: MaterializedValueNode, dep2: MaterializedValueNode) extends MaterializedValueNode { - override def toString: String = s"Combine($dep1,$dep2)" - } - case class Atomic(module: Module) extends MaterializedValueNode { - override def toString: String = f"Atomic(${module.attributes.nameOrDefault(module.getClass.getName)}[${System.identityHashCode(module)}%08x])" - } - case class Transform(f: Any ⇒ Any, dep: MaterializedValueNode) extends MaterializedValueNode { - override def toString: String = s"Transform($dep)" - } - case object Ignore extends MaterializedValueNode - - sealed trait Module { - - def shape: Shape - /** - * Verify that the given Shape has the same ports and return a new module with that shape. - * Concrete implementations may throw UnsupportedOperationException where applicable. - * - * Please note that this method MUST NOT be implemented using a CopiedModule since - * the purpose of replaceShape can also be to rearrange the ports (as in BidiFlow.reversed) - * and that purpose would be defeated. - */ - def replaceShape(s: Shape): Module - - final lazy val inPorts: Set[InPort] = shape.inlets.toSet - final lazy val outPorts: Set[OutPort] = shape.outlets.toSet - - def isRunnable: Boolean = inPorts.isEmpty && outPorts.isEmpty - final def isSink: Boolean = (inPorts.size == 1) && outPorts.isEmpty - final def isSource: Boolean = (outPorts.size == 1) && inPorts.isEmpty - final def isFlow: Boolean = (inPorts.size == 1) && (outPorts.size == 1) - final def isBidiFlow: Boolean = (inPorts.size == 2) && (outPorts.size == 2) - def isAtomic: Boolean = subModules.isEmpty - def isCopied: Boolean = false - def isFused: Boolean = false - - /** - * Fuses this Module to `that` Module by wiring together `from` and `to`, - * retaining the materialized value of `this` in the result - * - * @param that a Module to fuse with - * @param from the data source to wire - * @param to the data sink to wire - * @return a Module representing the fusion of `this` and `that` - */ - final def fuse(that: Module, from: OutPort, to: InPort): Module = - fuse(that, from, to, Keep.left) - - /** - * Fuses this Module to `that` Module by wiring together `from` and `to`, - * transforming the materialized values of `this` and `that` using the - * provided function `f` - * - * @param that a Module to fuse with - * @param from the data source to wire - * @param to the data sink to wire - * @param f the function to apply to the materialized values - * @return a Module representing the fusion of `this` and `that` - */ - final def fuse[A, B, C](that: Module, from: OutPort, to: InPort, f: (A, B) ⇒ C): Module = - this.compose(that, f).wire(from, to) - - /** - * Creates a new Module based on the current Module but with - * the given OutPort wired to the given InPort. - * - * @param from the OutPort to wire - * @param to the InPort to wire - * @return a new Module with the ports wired - */ - final def wire(from: OutPort, to: InPort): Module = { - if (Debug) validate(this) - - require( - outPorts(from), - if (downstreams.contains(from)) s"The output port [$from] is already connected" - else s"The output port [$from] is not part of the underlying graph.") - require( - inPorts(to), - if (upstreams.contains(to)) s"The input port [$to] is already connected" - else s"The input port [$to] is not part of the underlying graph.") - - CompositeModule( - if (isSealed) Set(this) else subModules, - AmorphousShape(shape.inlets.filterNot(_ == to), shape.outlets.filterNot(_ == from)), - downstreams.updated(from, to), - upstreams.updated(to, from), - materializedValueComputation, - if (isSealed) Attributes.none else attributes) - } - - final def transformMaterializedValue(f: Any ⇒ Any): Module = { - if (Debug) validate(this) - - CompositeModule( - if (this.isSealed) Set(this) else this.subModules, - shape, - downstreams, - upstreams, - Transform(f, if (this.isSealed) Atomic(this) else this.materializedValueComputation), - if (this.isSealed) Attributes.none else attributes) - } - - /** - * Creates a new Module which is `this` Module composed with `that` Module. - * - * @param that a Module to be composed with (cannot be itself) - * @return a Module that represents the composition of `this` and `that` - */ - def compose(that: Module): Module = compose(that, Keep.left) - - /** - * Creates a new Module which is `this` Module composed with `that` Module, - * using the given function `f` to compose the materialized value of `this` with - * the materialized value of `that`. - * - * @param that a Module to be composed with (cannot be itself) - * @param f a function which combines the materialized values - * @tparam A the type of the materialized value of `this` - * @tparam B the type of the materialized value of `that` - * @tparam C the type of the materialized value of the returned Module - * @return a Module that represents the composition of `this` and `that` - */ - def compose[A, B, C](that: Module, f: (A, B) ⇒ C): Module = { - if (Debug) validate(this) - - require(that ne this, "A module cannot be added to itself. You should pass a separate instance to compose().") - require(!subModules(that), "An existing submodule cannot be added again. All contained modules must be unique.") - - val modulesLeft = if (this.isSealed) Set(this) else this.subModules - val modulesRight = if (that.isSealed) Set(that) else that.subModules - - val matCompLeft = if (this.isSealed) Atomic(this) else this.materializedValueComputation - val matCompRight = if (that.isSealed) Atomic(that) else that.materializedValueComputation - - val mat = - { - val comp = - if (f == scaladsl.Keep.left) { - if (IgnorableMatValComp(matCompRight)) matCompLeft else null - } else if (f == scaladsl.Keep.right) { - if (IgnorableMatValComp(matCompLeft)) matCompRight else null - } else null - if (comp == null) Combine(f.asInstanceOf[(Any, Any) ⇒ Any], matCompLeft, matCompRight) - else comp - } - - CompositeModule( - modulesLeft union modulesRight, - AmorphousShape(shape.inlets ++ that.shape.inlets, shape.outlets ++ that.shape.outlets), - downstreams ++ that.downstreams, - upstreams ++ that.upstreams, - mat, - Attributes.none) - } - - /** - * Creates a new Module which is `this` Module composed with `that` Module. - * - * The difference to compose(that) is that this version completely ignores the materialized value - * computation of `that` while the normal version executes the computation and discards its result. - * This means that this version must not be used for user-provided `that` modules because users may - * transform materialized values only to achieve some side-effect; it can only be - * used where we know that there is no meaningful computation to be done (like for - * MaterializedValueSource). - * - * @param that a Module to be composed with (cannot be itself) - * @return a Module that represents the composition of `this` and `that` - */ - def composeNoMat(that: Module): Module = { - if (Debug) validate(this) - - require(that ne this, "A module cannot be added to itself. You should pass a separate instance to compose().") - require(!subModules(that), "An existing submodule cannot be added again. All contained modules must be unique.") - - val modules1 = if (this.isSealed) Set(this) else this.subModules - val modules2 = if (that.isSealed) Set(that) else that.subModules - - val matComputation = if (this.isSealed) Atomic(this) else this.materializedValueComputation - - CompositeModule( - modules1 ++ modules2, - AmorphousShape(shape.inlets ++ that.shape.inlets, shape.outlets ++ that.shape.outlets), - downstreams ++ that.downstreams, - upstreams ++ that.upstreams, - // would like to optimize away this allocation for Keep.{left,right} but that breaks side-effecting transformations - matComputation, - Attributes.none) - } - - def subModules: Set[Module] - final def isSealed: Boolean = isAtomic || isCopied || isFused || attributes.attributeList.nonEmpty - - def downstreams: Map[OutPort, InPort] = Map.empty - def upstreams: Map[InPort, OutPort] = Map.empty - - def materializedValueComputation: MaterializedValueNode = Atomic(this) - - /** - * The purpose of this method is to create a copy to be included in a larger - * graph such that port identity clashes are avoided. Where a full copy is not - * possible or desirable, use a CopiedModule. The shape of the resulting - * module MUST NOT contain the same ports as this module’s shape. - */ - def carbonCopy: Module - - def attributes: Attributes - def withAttributes(attributes: Attributes): Module - - final override def hashCode(): Int = super.hashCode() - final override def equals(obj: scala.Any): Boolean = super.equals(obj) - } - - case object EmptyModule extends Module { - override def shape = ClosedShape - override def replaceShape(s: Shape) = - if (s != shape) throw new UnsupportedOperationException("cannot replace the shape of the EmptyModule") - else this - - override def compose(that: Module): Module = compose(that, scaladsl.Keep.left) - - override def compose[A, B, C](that: Module, f: (A, B) ⇒ C): Module = { - if (f eq scaladsl.Keep.right) { - that - } else if (f eq scaladsl.Keep.left) { - // If "that" has a fully ignorable materialized value, we ignore it, otherwise we keep the side effect and - // explicitly map to NotUsed - val mat = - if (IgnorableMatValComp(that)) - Ignore - else - Transform(_ ⇒ NotUsed, that.materializedValueComputation) - - CompositeModule( - if (that.isSealed) Set(that) else that.subModules, - that.shape, - that.downstreams, - that.upstreams, - mat, - if (this.isSealed) Attributes.none else attributes) - } else { - throw new UnsupportedOperationException("It is invalid to combine materialized value with EmptyModule " + - "except with Keep.left or Keep.right") - } - } - - override def withAttributes(attributes: Attributes): Module = - throw new UnsupportedOperationException("EmptyModule cannot carry attributes") - - override def subModules: Set[Module] = Set.empty - override def attributes = Attributes.none - override def carbonCopy: Module = this - override def isRunnable: Boolean = false - override def isAtomic: Boolean = false - override def materializedValueComputation: MaterializedValueNode = Ignore - } - - final case class CopiedModule( - override val shape: Shape, - override val attributes: Attributes, - copyOf: Module) extends Module { - override val subModules: Set[Module] = Set(copyOf) - - override def withAttributes(attr: Attributes): Module = - if (attr ne attributes) this.copy(attributes = attr) - else this - - override def carbonCopy: Module = this.copy(shape = shape.deepCopy()) - - override def replaceShape(s: Shape): Module = - if (s != shape) { - shape.requireSamePortsAs(s) - CompositeModule(this, s) - } else this - - override val materializedValueComputation: MaterializedValueNode = Atomic(copyOf) - - override def isCopied: Boolean = true - - override def toString: String = f"[${System.identityHashCode(this)}%08x] copy of $copyOf" - } - - final case class CompositeModule( - override val subModules: Set[Module], - override val shape: Shape, - override val downstreams: Map[OutPort, InPort], - override val upstreams: Map[InPort, OutPort], - override val materializedValueComputation: MaterializedValueNode, - override val attributes: Attributes) extends Module { - - override def replaceShape(s: Shape): Module = - if (s != shape) { - shape.requireSamePortsAs(s) - copy(shape = s) - } else this - - override def carbonCopy: Module = CopiedModule(shape.deepCopy(), attributes, copyOf = this) - - override def withAttributes(attributes: Attributes): Module = copy(attributes = attributes) - - override def toString = - f"""CompositeModule [${System.identityHashCode(this)}%08x] - | Name: ${this.attributes.nameOrDefault("unnamed")} - | Modules: - | ${subModules.iterator.map(m ⇒ s"(${m.attributes.nameLifted.getOrElse("unnamed")}) ${m.toString.replaceAll("\n", "\n ")}").mkString("\n ")} - | Downstreams: ${downstreams.iterator.map { case (in, out) ⇒ s"\n $in -> $out" }.mkString("")} - | Upstreams: ${upstreams.iterator.map { case (out, in) ⇒ s"\n $out -> $in" }.mkString("")} - | MatValue: $materializedValueComputation""".stripMargin - } - - object CompositeModule { - def apply(m: Module, s: Shape): CompositeModule = CompositeModule(Set(m), s, Map.empty, Map.empty, Atomic(m), Attributes.none) - } - - /** - * INTERNAL API - * - * When fusing a [[Graph]] a part of the internal stage wirings are hidden within - * [[akka.stream.impl.fusing.GraphInterpreter#GraphAssembly]] objects that are - * optimized for high-speed execution. This structural information module contains - * the wirings in a more accessible form, allowing traversal from port to upstream - * or downstream port and from there to the owning module (or graph vertex). - */ - final case class StructuralInfoModule( - override val subModules: Set[Module], - override val shape: Shape, - override val downstreams: Map[OutPort, InPort], - override val upstreams: Map[InPort, OutPort], - inOwners: Map[InPort, Module], - outOwners: Map[OutPort, Module], - matValues: List[(Module, MaterializedValueNode)], - override val materializedValueComputation: MaterializedValueNode, - override val attributes: Attributes) extends Module { - - override def isFused: Boolean = false - - override def replaceShape(s: Shape): Module = - if (s != shape) { - shape.requireSamePortsAs(s) - copy(shape = s) - } else this - - override def carbonCopy: Module = CopiedModule(shape.deepCopy(), attributes, copyOf = this) - - override def withAttributes(attributes: Attributes): StructuralInfoModule = copy(attributes = attributes) - - } - - final case class FusedModule( - override val subModules: Set[Module], - override val shape: Shape, - override val downstreams: Map[OutPort, InPort], - override val upstreams: Map[InPort, OutPort], - override val materializedValueComputation: MaterializedValueNode, - override val attributes: Attributes, - info: StructuralInfoModule) extends Module { - - override def isFused: Boolean = true - - override def replaceShape(s: Shape): Module = - if (s != shape) { - shape.requireSamePortsAs(s) - copy(shape = s) - } else this - - override def carbonCopy: Module = CopiedModule(shape.deepCopy(), attributes, copyOf = this) - - override def withAttributes(attributes: Attributes): FusedModule = copy(attributes = attributes) - - override def toString = - f"""FusedModule [${System.identityHashCode(this)}%08x] - | Name: ${this.attributes.nameOrDefault("unnamed")} - | Modules: - | ${subModules.iterator.map(m ⇒ m.attributes.nameLifted.getOrElse(m.toString.replaceAll("\n", "\n "))).mkString("\n ")} - | Downstreams: ${downstreams.iterator.map { case (in, out) ⇒ s"\n $in -> $out" }.mkString("")} - | Upstreams: ${upstreams.iterator.map { case (out, in) ⇒ s"\n $out -> $in" }.mkString("")} - | MatValue: $materializedValueComputation""".stripMargin - } - /** * This is the only extension point for the sealed type hierarchy: composition * (i.e. the module tree) is managed strictly within this file, only leaf nodes * may be declared elsewhere. */ - abstract class AtomicModule extends Module { - final override def subModules: Set[Module] = Set.empty - final override def downstreams: Map[OutPort, InPort] = super.downstreams - final override def upstreams: Map[InPort, OutPort] = super.upstreams - } + trait AtomicModule[+S <: Shape, +M] extends Graph[S, M] } /** @@ -590,6 +95,8 @@ private[stream] final class VirtualProcessor[T] extends AtomicReference[AnyRef] import ReactiveStreamsCompliance._ import VirtualProcessor._ + override def toString: String = s"VirtualProcessor(${this.hashCode()})" + override def subscribe(s: Subscriber[_ >: T]): Unit = { @tailrec def rec(sub: Subscriber[Any]): Unit = get() match { @@ -612,6 +119,7 @@ private[stream] final class VirtualProcessor[T] extends AtomicReference[AnyRef] } override final def onSubscribe(s: Subscription): Unit = { + @tailrec def rec(obj: AnyRef): Unit = get() match { case null ⇒ if (!compareAndSet(null, obj)) rec(obj) @@ -831,209 +339,8 @@ private[impl] class VirtualPublisher[T] extends AtomicReference[AnyRef] with Pub pub.asInstanceOf[Publisher[r]].subscribe(sub) case _ ⇒ throw new IllegalStateException("internal error") } -} - -/** - * INERNAL API - */ -object MaterializerSession { - class MaterializationPanic(cause: Throwable) extends RuntimeException("Materialization aborted.", cause) with NoStackTrace - - final val Debug = false -} - -/** - * INTERNAL API - */ -abstract class MaterializerSession(val topLevel: StreamLayout.Module, val initialAttributes: Attributes) { - import StreamLayout._ - - // the contained maps store either Subscriber[Any] or VirtualPublisher, but the type system cannot express that - private var subscribersStack: List[ju.Map[InPort, AnyRef]] = - new ju.HashMap[InPort, AnyRef] :: Nil - private var publishersStack: List[ju.Map[OutPort, Publisher[Any]]] = - new ju.HashMap[OutPort, Publisher[Any]] :: Nil - private var matValSrcStack: List[ju.Map[MaterializedValueNode, List[MaterializedValueSource[Any]]]] = - new ju.HashMap[MaterializedValueNode, List[MaterializedValueSource[Any]]] :: Nil - - /* - * Please note that this stack keeps track of the scoped modules wrapped in CopiedModule but not the CopiedModule - * itself. The reason is that the CopiedModule itself is only needed for the enterScope and exitScope methods but - * not elsewhere. For this reason they are just simply passed as parameters to those methods. - * - * The reason why the encapsulated (copied) modules are stored as mutable state to save subclasses of this class - * from passing the current scope around or even knowing about it. - */ - private var moduleStack: List[Module] = topLevel :: Nil - - private def subscribers: ju.Map[InPort, AnyRef] = subscribersStack.head - private def publishers: ju.Map[OutPort, Publisher[Any]] = publishersStack.head - private def currentLayout: Module = moduleStack.head - private def matValSrc: ju.Map[MaterializedValueNode, List[MaterializedValueSource[Any]]] = matValSrcStack.head - - // Enters a copied module and establishes a scope that prevents internals to leak out and interfere with copies - // of the same module. - // We don't store the enclosing CopiedModule itself as state since we don't use it anywhere else than exit and enter - protected def enterScope(enclosing: CopiedModule): Unit = { - if (MaterializerSession.Debug) println(f"entering scope [${System.identityHashCode(enclosing)}%08x]") - subscribersStack ::= new ju.HashMap - publishersStack ::= new ju.HashMap - matValSrcStack ::= new ju.HashMap - moduleStack ::= enclosing.copyOf - } - - // Exits the scope of the copied module and propagates Publishers/Subscribers to the enclosing scope assigning - // them to the copied ports instead of the original ones (since there might be multiple copies of the same module - // leading to port identity collisions) - // We don't store the enclosing CopiedModule itself as state since we don't use it anywhere else than exit and enter - protected def exitScope(enclosing: CopiedModule): Unit = { - if (MaterializerSession.Debug) println(f"exiting scope [${System.identityHashCode(enclosing)}%08x]") - val scopeSubscribers = subscribers - val scopePublishers = publishers - subscribersStack = subscribersStack.tail - publishersStack = publishersStack.tail - matValSrcStack = matValSrcStack.tail - moduleStack = moduleStack.tail - - if (MaterializerSession.Debug) println(s" subscribers = $scopeSubscribers\n publishers = $scopePublishers") - - // When we exit the scope of a copied module, pick up the Subscribers/Publishers belonging to exposed ports of - // the original module and assign them to the copy ports in the outer scope that we will return to - enclosing.copyOf.shape.inlets.iterator.zip(enclosing.shape.inlets.iterator).foreach { - case (original, exposed) ⇒ assignPort(exposed, scopeSubscribers.get(original)) - } - - enclosing.copyOf.shape.outlets.iterator.zip(enclosing.shape.outlets.iterator).foreach { - case (original, exposed) ⇒ assignPort(exposed, scopePublishers.get(original)) - } - } - - final def materialize(): Any = { - if (MaterializerSession.Debug) println(s"beginning materialization of $topLevel") - require(topLevel ne EmptyModule, "An empty module cannot be materialized (EmptyModule was given)") - require( - topLevel.isRunnable, - s"The top level module cannot be materialized because it has unconnected ports: ${(topLevel.inPorts ++ topLevel.outPorts).mkString(", ")}") - try materializeModule(topLevel, initialAttributes and topLevel.attributes) - catch { - case NonFatal(cause) ⇒ - // PANIC!!! THE END OF THE MATERIALIZATION IS NEAR! - // Cancels all intermediate Publishers and fails all intermediate Subscribers. - // (This is an attempt to clean up after an exception during materialization) - val errorPublisher = new ErrorPublisher(new MaterializationPanic(cause), "") - for (subMap ← subscribersStack; sub ← subMap.asScala.valuesIterator) - doSubscribe(errorPublisher, sub) - - for (pubMap ← publishersStack; pub ← pubMap.asScala.valuesIterator) - pub.subscribe(new CancellingSubscriber) - - throw cause - } - } - - protected def mergeAttributes(parent: Attributes, current: Attributes): Attributes = - parent and current - - def registerSrc(ms: MaterializedValueSource[Any]): Unit = { - if (MaterializerSession.Debug) println(s"registering source $ms") - matValSrc.get(ms.computation) match { - case null ⇒ matValSrc.put(ms.computation, ms :: Nil) - case xs ⇒ matValSrc.put(ms.computation, ms :: xs) - } - } - - protected def materializeModule(module: Module, effectiveAttributes: Attributes): Any = { - val materializedValues: ju.Map[Module, Any] = new ju.HashMap - - if (MaterializerSession.Debug) println(f"entering module [${System.identityHashCode(module)}%08x] (${Logging.simpleName(module)})") - - for (submodule ← module.subModules) { - val subEffectiveAttributes = mergeAttributes(effectiveAttributes, submodule.attributes) - submodule match { - case atomic: AtomicModule ⇒ - materializeAtomic(atomic, subEffectiveAttributes, materializedValues) - case copied: CopiedModule ⇒ - enterScope(copied) - materializedValues.put(copied, materializeModule(copied, subEffectiveAttributes)) - exitScope(copied) - case composite @ (_: CompositeModule | _: FusedModule | _: StructuralInfoModule) ⇒ - materializedValues.put(composite, materializeComposite(composite, subEffectiveAttributes)) - case EmptyModule ⇒ // nothing to do or say - } - } - - if (MaterializerSession.Debug) { - println(f"resolving module [${System.identityHashCode(module)}%08x] computation ${module.materializedValueComputation}") - println(s" matValSrc = $matValSrc") - println(s" matVals =\n ${materializedValues.asScala.map(p ⇒ "%08x".format(System.identityHashCode(p._1)) → p._2).mkString("\n ")}") - } - - val ret = resolveMaterialized(module.materializedValueComputation, materializedValues, 2) - while (!matValSrc.isEmpty) { - val node = matValSrc.keySet.iterator.next() - if (MaterializerSession.Debug) println(s" delayed computation of $node") - resolveMaterialized(node, materializedValues, 4) - } - - if (MaterializerSession.Debug) println(f"exiting module [${System.identityHashCode(module)}%08x]") - - ret - } - - protected def materializeComposite(composite: Module, effectiveAttributes: Attributes): Any = { - materializeModule(composite, effectiveAttributes) - } - - protected def materializeAtomic(atomic: AtomicModule, effectiveAttributes: Attributes, matVal: ju.Map[Module, Any]): Unit - - private def resolveMaterialized(matNode: MaterializedValueNode, matVal: ju.Map[Module, Any], spaces: Int): Any = { - if (MaterializerSession.Debug) println(" " * spaces + matNode) - val ret = matNode match { - case Atomic(m) ⇒ matVal.get(m) - case Combine(f, d1, d2) ⇒ f(resolveMaterialized(d1, matVal, spaces + 2), resolveMaterialized(d2, matVal, spaces + 2)) - case Transform(f, d) ⇒ f(resolveMaterialized(d, matVal, spaces + 2)) - case Ignore ⇒ NotUsed - } - if (MaterializerSession.Debug) println(" " * spaces + s"result = $ret") - matValSrc.remove(matNode) match { - case null ⇒ // nothing to do - case srcs ⇒ - if (MaterializerSession.Debug) println(" " * spaces + s"triggering sources $srcs") - srcs.foreach(_.setValue(ret)) - } - ret - } - - protected def assignPort(in: InPort, subscriberOrVirtual: AnyRef): Unit = { - subscribers.put(in, subscriberOrVirtual) - - currentLayout.upstreams.get(in) match { - case Some(upstream) ⇒ - val publisher = publishers.get(upstream) - if (publisher ne null) doSubscribe(publisher, subscriberOrVirtual) - // Interface (unconnected) ports of the current scope will be wired when exiting the scope (or some parent scope) - case None ⇒ - } - } - - protected def assignPort(out: OutPort, publisher: Publisher[Any]): Unit = { - publishers.put(out, publisher) - - currentLayout.downstreams.get(out) match { - case Some(downstream) ⇒ - val subscriber = subscribers.get(downstream) - if (subscriber ne null) doSubscribe(publisher, subscriber) - // Interface (unconnected) ports of the current scope will be wired when exiting the scope - case None ⇒ - } - } - - private def doSubscribe(publisher: Publisher[_ <: Any], subscriberOrVirtual: AnyRef): Unit = - subscriberOrVirtual match { - case s: Subscriber[_] ⇒ publisher.subscribe(s.asInstanceOf[Subscriber[Any]]) - case v: VirtualPublisher[_] ⇒ v.registerPublisher(publisher) - } + override def toString: String = s"VirtualProcessor(state = ${get()})" } /** @@ -1041,15 +348,13 @@ abstract class MaterializerSession(val topLevel: StreamLayout.Module, val initia */ final case class ProcessorModule[In, Out, Mat]( val createProcessor: () ⇒ (Processor[In, Out], Mat), - attributes: Attributes = DefaultAttributes.processor) extends StreamLayout.AtomicModule { + attributes: Attributes = DefaultAttributes.processor) extends StreamLayout.AtomicModule[FlowShape[In, Out], Mat] { val inPort = Inlet[In]("ProcessorModule.in") val outPort = Outlet[Out]("ProcessorModule.out") override val shape = new FlowShape(inPort, outPort) - override def replaceShape(s: Shape) = if (s != shape) throw new UnsupportedOperationException("cannot replace the shape of a FlowModule") - else this - override def withAttributes(attributes: Attributes) = copy(attributes = attributes) - override def carbonCopy: Module = withAttributes(attributes) override def toString: String = f"ProcessorModule [${System.identityHashCode(this)}%08x]" + + override private[stream] def traversalBuilder = LinearTraversalBuilder.fromModule(this) } diff --git a/akka-stream/src/main/scala/akka/stream/impl/TraversalBuilder.scala b/akka-stream/src/main/scala/akka/stream/impl/TraversalBuilder.scala new file mode 100644 index 0000000000..660262cb85 --- /dev/null +++ b/akka-stream/src/main/scala/akka/stream/impl/TraversalBuilder.scala @@ -0,0 +1,985 @@ +/** + * Copyright (C) 2015-2016 Lightbend Inc. + */ + +package akka.stream.impl + +import akka.stream._ +import akka.stream.impl.StreamLayout.AtomicModule +import akka.stream.scaladsl.Keep + +/** + * Graphs to be materialized are defined by their traversal. There is no explicit graph information tracked, instead + * a sequence of steps required to "reconstruct" the graph. + * + * "Reconstructing" a graph here has a very clear-cut definition: assign a gapless range of integers from + * 0..connectionCount to inputs and outputs of modules, so that those that are wired together receive the same + * number (and those which are not receive different numbers). This feature can be used to + * - materialize a graph, using the slots as indices to an array of Publishers/Subscribers that need to be wired + * together + * - fuse a graph, using the slots to construct a [[akka.stream.impl.fusing.GraphInterpreter.GraphAssembly]] which + * uses a similar layout + * - create a DOT formatted output for visualization + * - convert the graph to another data structure + * + * The Traversal is designed to be position independent so that multiple traversals can be composed relatively + * simply. This particular feature also avoids issues with multiply imported modules where the identity must + * be encoded somehow. The two imports don't need any special treatment as they are at different positions in + * the traversal. See [[MaterializeAtomic]] for more details. + */ +sealed trait Traversal { + + /** + * Concatenates two traversals building a new Traversal which traverses both. + */ + def concat(that: Traversal): Traversal = { + Concat.normalizeConcat(this, that) + } + + def rewireFirstTo(relativeOffset: Int): Traversal = null +} + +object Concat { + + def normalizeConcat(first: Traversal, second: Traversal): Traversal = { + if (second eq EmptyTraversal) first + else if (first eq PushNotUsed) { + // No need to push NotUsed and Pop it immediately + second match { + case Pop ⇒ EmptyTraversal + case Concat(Pop, rest) ⇒ rest + case _ ⇒ Concat(PushNotUsed, second) + } + } else { + // Limit the tree by rotations + first match { + case Concat(firstfirst, firstsecond) ⇒ + // Note that we DON'T use firstfirst.concat(firstsecond.concat(second)) here, + // although that would fully linearize the tree. + // The reason is to simply avoid going n^2. The rotation below is of constant time and good enough. + Concat(firstfirst, Concat(firstsecond, second)) + case _ ⇒ Concat(first, second) + } + } + } + +} + +/** + * A Traversal that consists of two traversals. The linked traversals must be traversed in first, next order. + */ +final case class Concat(first: Traversal, next: Traversal) extends Traversal { + override def rewireFirstTo(relativeOffset: Int): Traversal = { + val firstResult = first.rewireFirstTo(relativeOffset) + if (firstResult ne null) + firstResult.concat(next) + else + first.concat(next.rewireFirstTo(relativeOffset)) + + } +} + +/** + * Arriving at this step means that an atomic module needs to be materialized (or any other activity which + * assigns something to wired output-input port pairs). + * + * The traversing party must assign port numbers in the following way: + * - input ports are implicitly assigned to numbers. Every module's input ports are assigned to consecutive numbers + * according to their order in the shape. In other words, the materializer only needs to keep a counter and + * increment it for every visited input port. + * - the assigned number of the first input port for every module should be saved while materializing the module. + * every output port should be assigned to (base + outToSlots(out.id)) where base is the number of the first + * input port of the module (or the last unused input number if it has no input ports) and outToSlots is the + * array provided by the traversal step. + * + * Since the above two rules always require local only computations (except a counter) this achieves + * positional independence of materializations. + * + * See the `TraversalTestUtils` class and the `testMaterialize` method for a simple example. + */ +final case class MaterializeAtomic(module: AtomicModule[Shape, Any], outToSlots: Array[Int]) extends Traversal { + override def toString: String = s"MaterializeAtomic($module, ${outToSlots.mkString("[", ", ", "]")})" + + override def rewireFirstTo(relativeOffset: Int): Traversal = copy(outToSlots = Array(relativeOffset)) +} + +/** + * Traversal with no steps. + */ +object EmptyTraversal extends Traversal { + override def concat(that: Traversal): Traversal = that +} + +sealed trait MaterializedValueOp extends Traversal + +case object Pop extends MaterializedValueOp +case object PushNotUsed extends MaterializedValueOp +final case class Transform(mapper: Any ⇒ Any) extends MaterializedValueOp +final case class Compose(composer: (Any, Any) ⇒ Any) extends MaterializedValueOp + +final case class PushAttributes(attributes: Attributes) extends Traversal +final case object PopAttributes extends Traversal + +final case class EnterIsland(islandTag: IslandTag, island: Traversal) extends Traversal { + override def rewireFirstTo(relativeOffset: Int): Traversal = + copy(island = island.rewireFirstTo(relativeOffset)) +} + +// Never embedded into actual traversal, used as a marker in AbsoluteTraversal +final case class ExitIsland(islandGlobalOffset: Int, skippedSlots: Int, phase: PhaseIsland[Any]) extends Traversal + +object TraversalBuilder { + + private val cachedEmptyCompleted = CompletedTraversalBuilder(PushNotUsed, 0, Map.empty, Attributes.none) + + /** + * Assign ports their id, which is their position inside the Shape. This is used both by the GraphInterpreter + * and the layout system here. + */ + private[impl] def initShape(shape: Shape): Unit = { + // Initialize port IDs + val inIter = shape.inlets.iterator + var i = 0 + while (inIter.hasNext) { + inIter.next.id = i + i += 1 + } + + val outIter = shape.outlets.iterator + i = 0 + while (outIter.hasNext) { + outIter.next.id = i + i += 1 + } + } + + def empty(attributes: Attributes = Attributes.none): TraversalBuilder = { + if (attributes eq Attributes.none) cachedEmptyCompleted + else CompletedTraversalBuilder(PushNotUsed, 0, Map.empty, attributes) + } + + /** + * Create a generic traversal builder starting from an atomic module. + */ + def atomic(module: AtomicModule[Shape, Any], attributes: Attributes = Attributes.none): TraversalBuilder = { + initShape(module.shape) + + if (module.shape.outlets.isEmpty) { + val b = CompletedTraversalBuilder( + traversalSoFar = MaterializeAtomic(module, Array.ofDim[Int](module.shape.outlets.size)), + inSlots = module.shape.inlets.size, + inToOffset = module.shape.inlets.map(in ⇒ in → in.id).toMap, + attributes + ) + b + } else { + AtomicTraversalBuilder( + module, + Array.ofDim[Int](module.shape.outlets.size), + module.shape.outlets.size, + attributes + ) + } + } + + def printTraversal(t: Traversal, indent: Int = 0): Unit = { + var current: Traversal = t + var slot = 0 + + def prindent(s: String): Unit = println(" | " * indent + s) + + while (current != EmptyTraversal) { + var nextStep: Traversal = EmptyTraversal + + current match { + case PushNotUsed ⇒ prindent("push NotUsed") + case Pop ⇒ prindent("pop mat") + case _: Transform ⇒ prindent("transform mat") + case _: Compose ⇒ prindent("compose mat") + case PushAttributes(attr) ⇒ prindent("push attr " + attr) + case PopAttributes ⇒ prindent("pop attr") + case EnterIsland(tag, island) ⇒ + prindent("enter island " + tag) + printTraversal(island, indent + 1) + case MaterializeAtomic(mod, outToSlots) ⇒ prindent("materialize " + mod + " " + outToSlots.mkString("[", ", ", "]")) + case Concat(first, next) ⇒ + printTraversal(first, indent + 1) + nextStep = next + case _ ⇒ + } + + current = nextStep + } + } + + def printWiring(t: Traversal, baseSlot: Int = 0): Int = { + var current: Traversal = t + var slot = baseSlot + + while (current != EmptyTraversal) { + var nextStep: Traversal = EmptyTraversal + + current match { + case MaterializeAtomic(mod, outToSlots) ⇒ + println(s"materialize $mod") + val base = slot + mod.shape.inlets.foreach { in ⇒ + println(s" wiring $in to $slot") + slot += 1 + } + mod.shape.outlets.foreach { out ⇒ + println(s" wiring $out to ${base + outToSlots(out.id)}") + } + case Concat(first, next) ⇒ + slot = printWiring(first, slot) + nextStep = next + case EnterIsland(_, traversal) ⇒ + nextStep = traversal + case _ ⇒ + } + + current = nextStep + } + slot + } +} + +/** + * A builder for a Traversal. The purpose of subclasses of this trait is to eventually build a Traversal that + * describes the graph. Depending on whether the graph is linear or generic different approaches can be used but + * they still result in a Traversal. + * + * The resulting Traversal can be accessed via the `traversal` method once the graph is completed (all ports are + * wired). The Traversal may be accessed earlier, depending on the type of the builder and certain conditions. + * See [[CompositeTraversalBuilder]] and [[LinearTraversalBuilder]]. + */ +sealed trait TraversalBuilder { + + /** + * Adds a module to the builder. It is possible to add a module with a different Shape (import), in this + * case the ports of the shape MUST have their `mappedTo` field pointing to the original ports. The act of being + * imported will not be reflected in the final Traversal, the Shape is only used by the builder to disambiguate + * between multiple imported instances of the same module. + * + * See append in the [[LinearTraversalBuilder]] for a more efficient alternative for linear graphs. + */ + def add[A, B, C](submodule: TraversalBuilder, shape: Shape, combineMat: (A, B) ⇒ C): TraversalBuilder + + /** + * Maps the materialized value produced by the module built-up so far with the provided function, providing a new + * TraversalBuilder returning the mapped materialized value. + */ + def transformMat[A, B](f: A ⇒ B): TraversalBuilder + + def setAttributes(attributes: Attributes): TraversalBuilder + + def attributes: Attributes + + /** + * Connects two unwired ports in the graph. For imported modules, use the ports of their "import shape". These + * ports must have their `mappedTo` field set and point to the original ports. + * + * See append in the [[LinearTraversalBuilder]] for a more efficient alternative for linear graphs. + */ + def wire(out: OutPort, in: InPort): TraversalBuilder + + /** + * Returns the base offset (the first number an input port would receive if there is any) of the module to which + * the port belongs *relative to this builder*. This is used to calculate the relative offset of output port mappings + * (see [[MaterializeAtomic]]). + * + * This method only guarantees to return the offset of modules for output ports that have not been wired. + */ + def offsetOfModule(out: OutPort): Int + + /** + * Returns whether the given output port has been wired in the graph or not. + */ + def isUnwired(out: OutPort): Boolean + + /** + * Returns whether the given output port has been wired in the graph or not. + */ + def isUnwired(in: InPort): Boolean + + /** + * Returns the number assigned to a certain input port *relative* to this module. + * + * This method only guarantees to return the offset of input ports that have not been wired. + */ + def offsetOf(in: InPort): Int + + /** + * Finish the wiring of an output port to an input port by assigning the relative slot for the output port. + * + * (see [[MaterializeAtomic]] for details of the resolution process) + */ + def assign(out: OutPort, relativeSlot: Int): TraversalBuilder + + /** + * Returns true if the Traversal is available. Not all builders are able to build up the Traversal incrementally. + * Generally a traversal is complete if there are no unwired output ports. + */ + def isTraversalComplete: Boolean + + /** + * The total number of input ports encountered so far. Gives the first slot to which a new input port can be + * assigned (if a new module is added). + */ + def inSlots: Int + + /** + * Returns the Traversal if ready for this (sub)graph. + */ + def traversal: Traversal = throw new IllegalStateException("Traversal can be only acquired from a completed builder") + + /** + * The number of output ports that have not been wired. + */ + def unwiredOuts: Int + + /** + * Wraps the builder in an island that can be materialized differently, using async boundaries to bridge + * between islands. + */ + def makeIsland(islandTag: IslandTag): TraversalBuilder +} + +/** + * Returned by [[CompositeTraversalBuilder]] once all output ports of a subgraph has been wired. + */ +final case class CompletedTraversalBuilder( + traversalSoFar: Traversal, + inSlots: Int, + inToOffset: Map[InPort, Int], + attributes: Attributes +) extends TraversalBuilder { + + override def add[A, B, C](submodule: TraversalBuilder, shape: Shape, combineMat: (A, B) ⇒ C): TraversalBuilder = { + val key = new BuilderKey + CompositeTraversalBuilder( + reverseBuildSteps = key :: Nil, + inSlots = inSlots, + inOffsets = inToOffset, + pendingBuilders = Map(key → this), + attributes = attributes + ).add(submodule, shape, combineMat) + } + + override def traversal: Traversal = + if (attributes eq Attributes.none) traversalSoFar + else PushAttributes(attributes).concat(traversalSoFar).concat(PopAttributes) + + override def transformMat[A, B](f: (A) ⇒ B): TraversalBuilder = + copy(traversalSoFar = traversalSoFar.concat(Transform(f.asInstanceOf[Any ⇒ Any]))) + + override def offsetOf(in: InPort): Int = inToOffset(in) + + override def isTraversalComplete: Boolean = true + + override def wire(out: OutPort, in: InPort): TraversalBuilder = + throw new UnsupportedOperationException("Cannot wire ports in a completed builder.") + + override def setAttributes(attributes: Attributes): TraversalBuilder = + copy(attributes = attributes) + + override def unwiredOuts: Int = 0 + + override def makeIsland(islandTag: IslandTag): TraversalBuilder = + copy(traversalSoFar = EnterIsland(islandTag, traversalSoFar)) + + override def assign(out: OutPort, relativeSlot: Int): TraversalBuilder = + throw new UnsupportedOperationException("Cannot assign ports to slots in a completed builder.") + + override def offsetOfModule(out: OutPort): Int = + throw new UnsupportedOperationException("Cannot look up offsets in a completed builder.") + + override def isUnwired(out: OutPort): Boolean = false + override def isUnwired(in: InPort): Boolean = inToOffset.contains(in) +} + +/** + * Represents a builder that contains a single atomic module. Its primary purpose is to track and build the + * outToSlot array which will be then embedded in a [[MaterializeAtomic]] Traversal step. + */ +final case class AtomicTraversalBuilder( + module: AtomicModule[Shape, Any], + outToSlot: Array[Int], + unwiredOuts: Int, + attributes: Attributes +) extends TraversalBuilder { + + override def add[A, B, C](submodule: TraversalBuilder, shape: Shape, combineMat: (A, B) ⇒ C): TraversalBuilder = { + // TODO: Use automatically a linear builder if applicable + // Create a composite, add ourselves, then the other. + CompositeTraversalBuilder(attributes = attributes) + .add(this, module.shape, Keep.right) + .add(submodule, shape, combineMat) + } + + override def transformMat[A, B](f: (A) ⇒ B): TraversalBuilder = + TraversalBuilder.empty().add(this, module.shape, Keep.right).transformMat(f) + + override val inSlots: Int = module.shape.inlets.size + + override def offsetOfModule(out: OutPort): Int = 0 + override def offsetOf(in: InPort): Int = in.id + override def isTraversalComplete: Boolean = false + override def isUnwired(out: OutPort): Boolean = true + override def isUnwired(in: InPort): Boolean = true + + override def wire(out: OutPort, in: InPort): TraversalBuilder = { + assign(out, offsetOf(in) - offsetOfModule(out)) + } + + override def assign(out: OutPort, relativeSlot: Int): TraversalBuilder = { + // Create a new array, with the output port assigned to its relative slot + val newOutToSlot = java.util.Arrays.copyOf(outToSlot, outToSlot.length) + newOutToSlot(out.id) = relativeSlot + + // Check if every output port has been assigned, if yes, we have a Traversal for this module. + val newUnwiredOuts = unwiredOuts - 1 + if (newUnwiredOuts == 0) { + CompletedTraversalBuilder( + traversalSoFar = MaterializeAtomic(module, newOutToSlot), + inSlots = inSlots, + // TODO Optimize Map creation + inToOffset = module.shape.inlets.iterator.map(in ⇒ in → in.id).toMap, + attributes = attributes + ) + } else copy(outToSlot = newOutToSlot, unwiredOuts = newUnwiredOuts) + } + + override def setAttributes(attributes: Attributes): TraversalBuilder = + copy(attributes = attributes) + + override def makeIsland(islandTag: IslandTag): TraversalBuilder = + TraversalBuilder.empty().add(this, module.shape, Keep.right).makeIsland(islandTag) +} + +object LinearTraversalBuilder { + + // TODO: Remove + private val cachedEmptyLinear = LinearTraversalBuilder(None, None, 0, 0, PushNotUsed, None, Attributes.none) + + private[this] final val wireBackward: Array[Int] = Array(-1) + private[this] final val noWire: Array[Int] = Array() + + def empty(attributes: Attributes = Attributes.none): LinearTraversalBuilder = + if (attributes eq Attributes.none) cachedEmptyLinear + else LinearTraversalBuilder(None, None, 0, 0, PushNotUsed, None, attributes) + + /** + * Create a traversal builder specialized for linear graphs. This is designed to be much faster and lightweight + * than its generic counterpart. It can be freely mixed with the generic builder in both ways. + */ + def fromModule(module: AtomicModule[Shape, Any], attributes: Attributes = Attributes.none): LinearTraversalBuilder = { + require(module.shape.inlets.size <= 1, "Modules with more than one input port cannot be linear.") + require(module.shape.outlets.size <= 1, "Modules with more than one input port cannot be linear.") + TraversalBuilder.initShape(module.shape) + + val inPortOpt = module.shape.inlets.headOption + val outPortOpt = module.shape.outlets.headOption + + val wiring = if (outPortOpt.isDefined) wireBackward else noWire + + LinearTraversalBuilder( + inPortOpt, + outPortOpt, + inOffset = 0, + if (inPortOpt.isDefined) 1 else 0, + traversalSoFar = MaterializeAtomic(module, wiring), + pendingBuilder = None, + attributes + ) + } + + def addMatCompose[A, B](t: Traversal, matCompose: (A, B) ⇒ Any): Traversal = { + if (matCompose eq Keep.left) + Pop.concat(t) + else if (matCompose eq Keep.right) + t.concat(Pop) + else // TODO: Optimize this case so the extra function allocation is not needed. Maybe ReverseCompose? + t.concat(Compose((second, first) ⇒ matCompose.asInstanceOf[(Any, Any) ⇒ Any](first, second))) + } + + def fromBuilder[A, B]( + traversalBuilder: TraversalBuilder, + shape: Shape, + combine: (A, B) ⇒ Any = Keep.right[A, B] + ): LinearTraversalBuilder = { + traversalBuilder match { + case linear: LinearTraversalBuilder ⇒ + if (combine eq Keep.right) linear + else empty().append(linear, combine) + + case completed: CompletedTraversalBuilder ⇒ + val inOpt = shape.inlets.headOption + val inOffs = inOpt match { + case Some(in) ⇒ completed.offsetOf(in) + case None ⇒ 0 + } + + LinearTraversalBuilder( + inPort = inOpt, + outPort = None, + inOffset = inOffs, + inSlots = completed.inSlots, + completed.traversal.concat(addMatCompose(PushNotUsed, combine)), + pendingBuilder = None, + Attributes.none + ) + + case composite ⇒ + val inOpt = shape.inlets.headOption + val out = shape.outlets.head // Cannot be empty, otherwise it would be a CompletedTraversalBuilder + val inOffs = inOpt match { + case Some(in) ⇒ composite.offsetOf(in) + case None ⇒ 0 + } + + LinearTraversalBuilder( + inPort = inOpt, + outPort = Some(out), + inOffset = inOffs, + inSlots = composite.inSlots, + addMatCompose(PushNotUsed, combine), + pendingBuilder = Some(composite), + Attributes.none, + beforeBuilder = if (inOpt.isDefined) PushAttributes(composite.attributes) else EmptyTraversal + ) + + } + } +} + +/** + * Traversal builder that is optimized for linear graphs (those that contain modules with at most one input and + * at most one output port). The Traversal is simply built up in reverse order and output ports are automatically + * assigned to -1 due to the nature of the graph. The only exception is when composites created by + * [[CompositeTraversalBuilder]] are embedded. These are not guaranteed to have their unwired input/output ports + * in a fixed location, therefore the last step of the Traversal might need to be changed in those cases from the + * -1 relative offset to something else (see rewireLastOutTo). + */ +final case class LinearTraversalBuilder( + inPort: Option[InPort], + outPort: Option[OutPort], + inOffset: Int, + override val inSlots: Int, + traversalSoFar: Traversal, + pendingBuilder: Option[TraversalBuilder], + attributes: Attributes, + beforeBuilder: Traversal = EmptyTraversal +) extends TraversalBuilder { + + protected def isEmpty: Boolean = inSlots == 0 && outPort.isEmpty + + override def add[A, B, C](submodule: TraversalBuilder, shape: Shape, combineMat: (A, B) ⇒ C): TraversalBuilder = { + throw new UnsupportedOperationException("LinearTraversal does not support free-form addition. Add it into a" + + "composite builder instead and add the second module to that.") + } + + /** + * This builder can always return a traversal. + */ + override def traversal: Traversal = applyAttributes(traversalSoFar) + + override def setAttributes(attributes: Attributes): LinearTraversalBuilder = + copy(attributes = attributes) + + private def applyAttributes(t: Traversal): Traversal = { + val withBuilder = beforeBuilder.concat(t) + + if (attributes eq Attributes.none) withBuilder + else PushAttributes(attributes).concat(withBuilder).concat(PopAttributes) + } + + /** + * In case the default relative wiring of -1 is not applicable (due to for example an embedded composite + * [[CompositeTraversalBuilder]] created traversal) we need to change the mapping for the module we added + * last. This method tears down the traversal until it finds that [[MaterializeAtomic]], changes the mapping, + * then rebuilds the Traversal. + */ + private def rewireLastOutTo(traversal: Traversal, relativeOffset: Int): Traversal = { + // If, by luck, the offset is the same as it would be in the normal case, no transformation is needed + if (relativeOffset == -1) traversal + else traversal.rewireFirstTo(relativeOffset) + } + + /** + * Since this is a linear traversal, this should not be called in most of the cases. The only notable + * exception is when a Flow is wired to itself. + */ + override def wire(out: OutPort, in: InPort): TraversalBuilder = { + if (outPort.contains(out) && inPort.contains(in)) { + pendingBuilder match { + case Some(composite) ⇒ + copy( + inPort = None, + outPort = None, + traversalSoFar = + composite + .assign(out, inOffset - composite.offsetOfModule(out)) + .traversal + .concat(traversalSoFar), + pendingBuilder = None + ) + case None ⇒ + copy(inPort = None, outPort = None, traversalSoFar = rewireLastOutTo(traversalSoFar, inOffset)) + } + } else + throw new IllegalArgumentException(s"The ports $in and $out cannot be accessed in this builder.") + } + + override def offsetOfModule(out: OutPort): Int = { + if (outPort.contains(out)) { + pendingBuilder match { + case Some(composite) ⇒ composite.offsetOfModule(out) + case None ⇒ 0 // Output belongs to the last module, which will be materialized *first* + } + } else + throw new IllegalArgumentException(s"Port $out cannot be accessed in this builder") + } + + override def isUnwired(out: OutPort): Boolean = outPort.contains(out) + override def isUnwired(in: InPort): Boolean = inPort.contains(in) + + override def offsetOf(in: InPort): Int = { + if (inPort.contains(in)) inOffset + else + throw new IllegalArgumentException(s"Port $in cannot be accessed in this builder") + } + + override def assign(out: OutPort, relativeSlot: Int): TraversalBuilder = { + if (outPort.contains(out)) { + pendingBuilder match { + case Some(composite) ⇒ + copy( + outPort = None, + traversalSoFar = + composite + .assign(out, relativeSlot) + .traversal + .concat(traversalSoFar), + pendingBuilder = None + ) + case None ⇒ + copy(outPort = None, traversalSoFar = rewireLastOutTo(traversalSoFar, relativeSlot)) + } + } else + throw new IllegalArgumentException(s"Port $out cannot be assigned in this builder") + } + + override def isTraversalComplete: Boolean = outPort.isEmpty + + override def unwiredOuts: Int = if (outPort.isDefined) 1 else 0 + + def append[A, B, C](toAppend: TraversalBuilder, shape: Shape, matCompose: (A, B) ⇒ C): LinearTraversalBuilder = + append(LinearTraversalBuilder.fromBuilder(toAppend, shape, Keep.right), matCompose) + + // We don't really need the Shape for the linear append, but it is nicer to keep the API uniform here + /** + * Append any builder that is linear shaped (have at most one input and at most one output port) to the + * end of this graph, connecting the output of the last module to the input of the appended module. + */ + def append[A, B, C](toAppend: LinearTraversalBuilder, matCompose: (A, B) ⇒ C): LinearTraversalBuilder = { + + if (toAppend.isEmpty) this + else if (this.isEmpty) { + toAppend.copy( + traversalSoFar = toAppend.traversalSoFar.concat(LinearTraversalBuilder.addMatCompose(traversal, matCompose)) + ) + } else { + if (outPort.nonEmpty) { + require(toAppend.inPort.isDefined, "Appended linear module must have an unwired input port " + + "because there is a dangling output.") + + val traversalWithWiringCorrected = pendingBuilder match { + case Some(composite) ⇒ + val out = outPort.get + applyAttributes( + composite + .assign(out, -composite.offsetOfModule(out) - toAppend.inSlots + toAppend.inOffset) + .traversal + .concat(traversalSoFar) + ) + case None ⇒ + // No need to rewire if input port is at the expected position + if (toAppend.inOffset == (toAppend.inSlots - 1)) + traversal + else + applyAttributes(rewireLastOutTo(traversalSoFar, toAppend.inOffset - toAppend.inSlots)) + } + + val newTraversal = + if (toAppend.pendingBuilder.isEmpty) + toAppend.traversal + .concat(LinearTraversalBuilder.addMatCompose(traversalWithWiringCorrected, matCompose)) + else { + toAppend.traversalSoFar + .concat(PopAttributes) + .concat(LinearTraversalBuilder.addMatCompose(traversalWithWiringCorrected, matCompose)) + } + + // Simply just take the new unwired ports, increase the number of inSlots and concatenate the traversals + LinearTraversalBuilder( + inPort = inPort, + outPort = toAppend.outPort, + inSlots = inSlots + toAppend.inSlots, + inOffset = inOffset + toAppend.inSlots, + // Build in reverse so it yields a more efficient layout for left-to-right building + traversalSoFar = newTraversal, + pendingBuilder = toAppend.pendingBuilder, + attributes = Attributes.none, + beforeBuilder = if (toAppend.pendingBuilder.isEmpty) EmptyTraversal else PushAttributes(toAppend.attributes) + ) + } else throw new Exception("should this happen?") + + } + + } + + override def transformMat[A, B](f: (A) ⇒ B): LinearTraversalBuilder = { + copy(traversalSoFar = traversalSoFar.concat(Transform(f.asInstanceOf[Any ⇒ Any]))) + } + + /** + * Wraps the builder in an island that can be materialized differently, using async boundaries to bridge + * between islands. + */ + override def makeIsland(islandTag: IslandTag): LinearTraversalBuilder = + copy(traversalSoFar = EnterIsland(islandTag, traversalSoFar)) +} + +sealed trait TraversalBuildStep +/** + * Helper class that is only used to identify a [[TraversalBuilder]] in a [[CompositeTraversalBuilder]]. The + * reason why this is needed is because the builder is referenced at various places, while it needs to be mutated. + * In an immutable data structure this is best done with an indirection, i.e. places refer to this immutable key and + * look up the current state in an extra Map. + */ +final class BuilderKey extends TraversalBuildStep { + override def toString = s"K:$hashCode" +} +final case class AppendTraversal(traversal: Traversal) extends TraversalBuildStep + +/** + * A generic builder that builds a traversal for graphs of arbitrary shape. The memory retained by this class + * usually decreases as ports are wired since auxiliary data is only maintained for ports that are unwired. + * + * This builder MUST construct its Traversal in the *exact* same order as its modules were added, since the first + * (and subsequent) input port of a module is implicitly assigned by its traversal order. Emitting Traversal nodes + * in a non-deterministic order (depending on wiring order) would mess up all relative addressing. This is the + * primary technical reason why a reverseTraversal list is maintained and the Traversal can only be completed once + * all output ports have been wired. + * + * @param reverseBuildSteps Keeps track of traversal steps that needs to be concatenated. This is basically + * a "queue" of BuilderKeys that point to builders of submodules/subgraphs. Since it is + * unknown in which order will the submodules "complete" (have all of their outputs assigned) + * we delay the creation of the actual Traversal. + * @param inSlots The number of input ports this graph has in total. + * @param inOffsets Map to look up the offset of input ports not yet wired + * @param inBaseOffsetForOut Map to look up the base (input) offset of a module that owns the given output port + * @param pendingBuilders Map to contain the "mutable" builders referred by BuilderKeys + * @param outOwners Map of output ports to their parent builders (actually the BuilderKey) + * @param unwiredOuts Number of output ports that have not yet been wired/assigned + */ +final case class CompositeTraversalBuilder( + finalSteps: Traversal = EmptyTraversal, + reverseBuildSteps: List[TraversalBuildStep] = AppendTraversal(PushNotUsed) :: Nil, + inSlots: Int = 0, + inOffsets: Map[InPort, Int] = Map.empty, + inBaseOffsetForOut: Map[OutPort, Int] = Map.empty, + pendingBuilders: Map[BuilderKey, TraversalBuilder] = Map.empty, + outOwners: Map[OutPort, BuilderKey] = Map.empty, + unwiredOuts: Int = 0, + attributes: Attributes, + islandTag: IslandTag = null +) extends TraversalBuilder { + + override def toString: String = + s""" + |CompositeTraversal( + | reverseTraversal = $reverseBuildSteps + | pendingBuilders = $pendingBuilders + | inSlots = $inSlots + | inOffsets = $inOffsets + | inBaseOffsetForOut = $inBaseOffsetForOut + | outOwners = $outOwners + | unwiredOuts = $unwiredOuts + |) + """.stripMargin + + override def offsetOfModule(out: OutPort): Int = inBaseOffsetForOut(out) + override def isUnwired(out: OutPort): Boolean = inBaseOffsetForOut.contains(out) + override def isUnwired(in: InPort): Boolean = inOffsets.contains(in) + + override def offsetOf(in: InPort): Int = inOffsets(in) + override def isTraversalComplete = false + + override def setAttributes(attributes: Attributes): TraversalBuilder = + copy(attributes = attributes) + + /** + * Convert this builder to a [[CompositeTraversalBuilder]] if there are no more unwired outputs. + */ + def completeIfPossible: TraversalBuilder = { + if (unwiredOuts == 0) { + var traversal: Traversal = finalSteps + var remaining = reverseBuildSteps + while (remaining.nonEmpty) { + remaining.head match { + case key: BuilderKey ⇒ + // At this point all the builders we have are completed and we can finally build our traversal + traversal = pendingBuilders(key).traversal.concat(traversal) + case AppendTraversal(toAppend) ⇒ + traversal = toAppend.concat(traversal) + } + remaining = remaining.tail + } + + val finalTraversal = if (islandTag == null) traversal else EnterIsland(islandTag, traversal) + + // The CompleteTraversalBuilder only keeps the minimum amount of necessary information that is needed for it + // to be embedded in a larger graph, making partial graph reuse much more efficient. + CompletedTraversalBuilder( + traversalSoFar = finalTraversal, + inSlots, + inOffsets, + attributes + ) + } else this + } + + /** + * Assign an output port a relative slot (relative to the base input slot of its module, see [[MaterializeAtomic]]) + */ + override def assign(out: OutPort, relativeSlot: Int): TraversalBuilder = { + // Which module out belongs to (indirection via BuilderKey and pendingBuilders) + val builderKey = outOwners(out) + val submodule = pendingBuilders(builderKey) + + // Do the assignment in the submodule + val result = submodule.assign(out.mappedTo, relativeSlot) + val wired = if (result.isTraversalComplete) { + // Remove the builder (and associated data). + // We can't simply append its Traversal as there might be uncompleted builders that come earlier in the + // final traversal (remember, input ports are assigned in traversal order of modules, and the inOffsets + // and inBaseOffseForOut Maps are updated when adding a module; we must respect addition order). + + copy( + inBaseOffsetForOut = inBaseOffsetForOut - out, + outOwners = outOwners - out, + // TODO Optimize Map access + pendingBuilders = pendingBuilders.updated(builderKey, result), + // pendingBuilders = pendingBuilders - builderKey, + unwiredOuts = unwiredOuts - 1 + ) + } else { + // Update structures with result + copy( + inBaseOffsetForOut = inBaseOffsetForOut - out, + unwiredOuts = unwiredOuts - 1, + pendingBuilders = pendingBuilders.updated(builderKey, result) + ) + } + + // If we have no more unconnected outputs, we can finally build the Traversal and shed most of the auxiliary data. + wired.completeIfPossible + } + + // Requires that a remapped Shape's ports contain the same ID as their target ports! + def add[A, B, C](submodule: TraversalBuilder, shape: Shape, combineMat: (A, B) ⇒ C): TraversalBuilder = { + val builderKey = new BuilderKey + + val newBuildSteps = + if (combineMat == Keep.left) { + AppendTraversal(Pop) :: + builderKey :: + reverseBuildSteps + } else if (combineMat == Keep.right) { + builderKey :: + AppendTraversal(Pop) :: + reverseBuildSteps + } else if (combineMat == Keep.none) { + AppendTraversal(PushNotUsed) :: + AppendTraversal(Pop) :: + AppendTraversal(Pop) :: + builderKey :: + reverseBuildSteps + } else { + AppendTraversal(Compose(combineMat.asInstanceOf[(Any, Any) ⇒ Any])) :: + builderKey :: + reverseBuildSteps + } + + val added = if (submodule.isTraversalComplete) { + // We only need to keep track of the offsets of unwired inputs. Outputs have all been wired + // (isTraversalComplete = true). + + var newInOffsets = inOffsets + val inIterator = shape.inlets.iterator + while (inIterator.hasNext) { + val in = inIterator.next() + // Calculate offset in the current scope. This is the our first unused input slot plus + // the relative offset of the input port in the submodule. + // TODO Optimize Map access + newInOffsets = newInOffsets.updated(in, inSlots + submodule.offsetOf(in.mappedTo)) + } + + copy( + reverseBuildSteps = newBuildSteps, + inSlots = inSlots + submodule.inSlots, + pendingBuilders = pendingBuilders.updated(builderKey, submodule), + inOffsets = newInOffsets + ) + } else { + // Added module have unwired outputs. + + var newInOffsets = inOffsets + var newBaseOffsetsForOut = inBaseOffsetForOut + var newOutOwners = outOwners + + // See the other if case for explanation of this + val inIterator = shape.inlets.iterator + while (inIterator.hasNext) { + val in = inIterator.next() + // Calculate offset in the current scope + // TODO Optimize Map access + newInOffsets = newInOffsets.updated(in, inSlots + submodule.offsetOf(in.mappedTo)) + } + + val outIterator = shape.outlets.iterator + while (outIterator.hasNext) { + val out = outIterator.next() + // Record the base offsets of all the modules we included and which have unwired output ports. We need + // to adjust their offset by inSlots as that would be their new position in this module. + newBaseOffsetsForOut = newBaseOffsetsForOut.updated(out, inSlots + submodule.offsetOfModule(out.mappedTo)) + // TODO Optimize Map access + newOutOwners = newOutOwners.updated(out, builderKey) + } + + copy( + reverseBuildSteps = newBuildSteps, + inSlots = inSlots + submodule.inSlots, + inOffsets = newInOffsets, + inBaseOffsetForOut = newBaseOffsetsForOut, + outOwners = newOutOwners, + pendingBuilders = pendingBuilders.updated(builderKey, submodule), + unwiredOuts = unwiredOuts + submodule.unwiredOuts + ) + } + + added.completeIfPossible + } + + def wire(out: OutPort, in: InPort): TraversalBuilder = { + copy(inOffsets = inOffsets - in).assign(out, offsetOf(in) - offsetOfModule(out)) + } + + override def transformMat[A, B](f: (A) ⇒ B): TraversalBuilder = { + copy(finalSteps = finalSteps.concat(Transform(f.asInstanceOf[Any ⇒ Any]))) + } + + override def makeIsland(islandTag: IslandTag): TraversalBuilder = copy(islandTag = islandTag) +} diff --git a/akka-stream/src/main/scala/akka/stream/impl/fusing/ActorGraphInterpreter.scala b/akka-stream/src/main/scala/akka/stream/impl/fusing/ActorGraphInterpreter.scala index f12b581177..69ff2d421b 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/fusing/ActorGraphInterpreter.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/fusing/ActorGraphInterpreter.scala @@ -5,98 +5,93 @@ package akka.stream.impl.fusing import java.util import java.util.concurrent.TimeoutException +import java.util.concurrent.atomic.AtomicReference import akka.actor._ import akka.event.Logging import akka.stream._ import akka.stream.impl.ReactiveStreamsCompliance._ -import akka.stream.impl.StreamLayout.{ AtomicModule, CompositeModule, CopiedModule, Module } +import akka.stream.impl.fusing.GraphInterpreter.{ Connection, DownstreamBoundaryStageLogic, UpstreamBoundaryStageLogic } import akka.stream.impl.{ SubFusingActorMaterializerImpl, _ } -import akka.stream.impl.fusing.GraphInterpreter.{ Connection, DownstreamBoundaryStageLogic, GraphAssembly, UpstreamBoundaryStageLogic } import akka.stream.stage.{ GraphStageLogic, InHandler, OutHandler } -import org.reactivestreams.{ Subscriber, Subscription } +import org.reactivestreams.{ Publisher, Subscriber, Subscription } import scala.annotation.tailrec +import scala.collection.immutable import scala.util.control.NonFatal -/** - * INTERNAL API - */ -final case class GraphModule(assembly: GraphAssembly, shape: Shape, attributes: Attributes, - matValIDs: Array[Module]) extends AtomicModule { - - override def withAttributes(newAttr: Attributes): Module = copy(attributes = newAttr) - - override final def carbonCopy: Module = CopiedModule(shape.deepCopy(), Attributes.none, this) - - override final def replaceShape(newShape: Shape): Module = - if (newShape != shape) CompositeModule(this, newShape) - else this - - override def toString: String = - s"""GraphModule - | ${assembly.toString.replace("\n", "\n ")} - | shape=$shape, attributes=$attributes - | matVals= - | ${matValIDs.mkString("\n ")}""".stripMargin -} - /** * INTERNAL API */ object ActorGraphInterpreter { + + object ResumeActor extends DeadLetterSuppression with NoSerializationVerificationNeeded + trait BoundaryEvent extends DeadLetterSuppression with NoSerializationVerificationNeeded { def shell: GraphInterpreterShell + def execute(eventLimit: Int): Int } - final case class OnError(shell: GraphInterpreterShell, id: Int, cause: Throwable) extends BoundaryEvent - final case class OnComplete(shell: GraphInterpreterShell, id: Int) extends BoundaryEvent - final case class OnNext(shell: GraphInterpreterShell, id: Int, e: Any) extends BoundaryEvent - final case class OnSubscribe(shell: GraphInterpreterShell, id: Int, subscription: Subscription) extends BoundaryEvent - - final case class RequestMore(shell: GraphInterpreterShell, id: Int, demand: Long) extends BoundaryEvent - final case class Cancel(shell: GraphInterpreterShell, id: Int) extends BoundaryEvent - final case class SubscribePending(shell: GraphInterpreterShell, id: Int) extends BoundaryEvent - final case class ExposedPublisher(shell: GraphInterpreterShell, id: Int, publisher: ActorPublisher[Any]) extends BoundaryEvent - - final case class AsyncInput(shell: GraphInterpreterShell, logic: GraphStageLogic, evt: Any, handler: (Any) ⇒ Unit) extends BoundaryEvent - - case class Resume(shell: GraphInterpreterShell) extends BoundaryEvent - case class Abort(shell: GraphInterpreterShell) extends BoundaryEvent - - final class BoundaryPublisher(parent: ActorRef, shell: GraphInterpreterShell, id: Int) extends ActorPublisher[Any](parent) { - override val wakeUpMsg = ActorGraphInterpreter.SubscribePending(shell, id) - } - - final class BoundarySubscription(parent: ActorRef, shell: GraphInterpreterShell, id: Int) extends Subscription { - override def request(elements: Long): Unit = parent ! RequestMore(shell, id, elements) - override def cancel(): Unit = parent ! Cancel(shell, id) - override def toString = s"BoundarySubscription[$parent, $id]" - } - - final class BoundarySubscriber(parent: ActorRef, shell: GraphInterpreterShell, id: Int) extends Subscriber[Any] { - override def onError(cause: Throwable): Unit = { - ReactiveStreamsCompliance.requireNonNullException(cause) - parent ! OnError(shell, id, cause) - } - override def onComplete(): Unit = parent ! OnComplete(shell, id) - override def onNext(element: Any): Unit = { - ReactiveStreamsCompliance.requireNonNullElement(element) - parent ! OnNext(shell, id, element) - } - override def onSubscribe(subscription: Subscription): Unit = { - ReactiveStreamsCompliance.requireNonNullSubscription(subscription) - parent ! OnSubscribe(shell, id, subscription) + trait SimpleBoundaryEvent extends BoundaryEvent { + final override def execute(eventLimit: Int): Int = { + val wasNotShutdown = !shell.interpreter.isStageCompleted(logic) + execute() + if (wasNotShutdown) shell.interpreter.afterStageHasRun(logic) + shell.runBatch(eventLimit) } + + def logic: GraphStageLogic + + def execute(): Unit } def props(shell: GraphInterpreterShell): Props = Props(new ActorGraphInterpreter(shell)).withDeploy(Deploy.local) - class BatchingActorInputBoundary(size: Int, id: Int) extends UpstreamBoundaryStageLogic[Any] { + class BatchingActorInputBoundary( + size: Int, + shell: GraphInterpreterShell, + publisher: Publisher[Any], + internalPortName: String) extends UpstreamBoundaryStageLogic[Any] { + + final case class OnError(shell: GraphInterpreterShell, cause: Throwable) extends SimpleBoundaryEvent { + override def execute(): Unit = { + if (GraphInterpreter.Debug) println(s"${interpreter.Name} onError port=$internalPortName") + BatchingActorInputBoundary.this.onError(cause) + } + + override def logic: GraphStageLogic = BatchingActorInputBoundary.this + } + final case class OnComplete(shell: GraphInterpreterShell) extends SimpleBoundaryEvent { + override def execute(): Unit = { + if (GraphInterpreter.Debug) println(s"${interpreter.Name} onComplete port=$internalPortName") + BatchingActorInputBoundary.this.onComplete() + } + + override def logic: GraphStageLogic = BatchingActorInputBoundary.this + } + final case class OnNext(shell: GraphInterpreterShell, e: Any) extends SimpleBoundaryEvent { + override def execute(): Unit = { + if (GraphInterpreter.Debug) println(s"${interpreter.Name} onNext $e port=$internalPortName") + BatchingActorInputBoundary.this.onNext(e) + } + + override def logic: GraphStageLogic = BatchingActorInputBoundary.this + } + final case class OnSubscribe(shell: GraphInterpreterShell, subscription: Subscription) extends SimpleBoundaryEvent { + override def execute(): Unit = { + if (GraphInterpreter.Debug) println(s"${interpreter.Name} onSubscribe port=$internalPortName") + shell.subscribeArrived() + BatchingActorInputBoundary.this.onSubscribe(subscription) + } + + override def logic: GraphStageLogic = BatchingActorInputBoundary.this + } + require(size > 0, "buffer size cannot be zero") require((size & (size - 1)) == 0, "buffer size must be a power of two") + private var actor: ActorRef = ActorRef.noSender private var upstream: Subscription = _ private val inputBuffer = Array.ofDim[AnyRef](size) private var inputBufferElements = 0 @@ -108,9 +103,36 @@ object ActorGraphInterpreter { private def requestBatchSize = math.max(1, inputBuffer.length / 2) private var batchRemaining = requestBatchSize - val out: Outlet[Any] = Outlet[Any]("UpstreamBoundary" + id) + val out: Outlet[Any] = Outlet[Any]("UpstreamBoundary:" + internalPortName) out.id = 0 + def setActor(actor: ActorRef): Unit = this.actor = actor + + override def preStart(): Unit = { + publisher.subscribe( + new Subscriber[Any] { + override def onError(t: Throwable): Unit = { + ReactiveStreamsCompliance.requireNonNullException(t) + actor ! OnError(shell, t) + } + + override def onSubscribe(s: Subscription): Unit = { + ReactiveStreamsCompliance.requireNonNullSubscription(s) + actor ! OnSubscribe(shell, s) + } + + override def onComplete(): Unit = { + actor ! OnComplete(shell) + } + + override def onNext(t: Any): Unit = { + ReactiveStreamsCompliance.requireNonNullElement(t) + actor ! OnNext(shell, t) + } + } + ) + } + private def dequeue(): Any = { val elem = inputBuffer(nextInputElementCursor) require(elem ne null, "Internal queue must never contain a null") @@ -174,9 +196,9 @@ object ActorGraphInterpreter { def onSubscribe(subscription: Subscription): Unit = { require(subscription != null, "Subscription cannot be null") - if (upstreamCompleted) + if (upstreamCompleted) { tryCancel(subscription) - else if (downstreamCanceled) { + } else if (downstreamCanceled) { upstreamCompleted = true tryCancel(subscription) } else { @@ -204,14 +226,107 @@ object ActorGraphInterpreter { override def toString: String = BatchingActorInputBoundary.this.toString }) - override def toString: String = s"BatchingActorInputBoundary(id=$id, fill=$inputBufferElements/$size, completed=$upstreamCompleted, canceled=$downstreamCanceled)" + override def toString: String = s"BatchingActorInputBoundary(forPort=$internalPortName, fill=$inputBufferElements/$size, completed=$upstreamCompleted, canceled=$downstreamCanceled)" } - private[stream] class ActorOutputBoundary(actor: ActorRef, shell: GraphInterpreterShell, id: Int) extends DownstreamBoundaryStageLogic[Any] { - val in: Inlet[Any] = Inlet[Any]("UpstreamBoundary" + id) + final case class SubscribePending(boundary: ActorOutputBoundary) extends SimpleBoundaryEvent { + override def execute(): Unit = boundary.subscribePending() + + override def shell: GraphInterpreterShell = boundary.shell + + override def logic: GraphStageLogic = boundary + } + + final case class RequestMore(boundary: ActorOutputBoundary, demand: Long) extends SimpleBoundaryEvent { + override def execute(): Unit = { + if (GraphInterpreter.Debug) println(s"${boundary.shell.interpreter.Name} request $demand port=${boundary.internalPortName}") + boundary.requestMore(demand) + } + override def shell: GraphInterpreterShell = boundary.shell + override def logic: GraphStageLogic = boundary + } + final case class Cancel(boundary: ActorOutputBoundary) extends SimpleBoundaryEvent { + override def execute(): Unit = { + if (GraphInterpreter.Debug) println(s"${boundary.shell.interpreter.Name} cancel port=${boundary.internalPortName}") + boundary.cancel() + } + + override def shell: GraphInterpreterShell = boundary.shell + override def logic: GraphStageLogic = boundary + } + + private[stream] class OutputBoundaryPublisher(boundary: ActorOutputBoundary, internalPortName: String) extends Publisher[Any] { + import ReactiveStreamsCompliance._ + + // The subscriber of an subscription attempt is first placed in this list of pending subscribers. + // The actor will call takePendingSubscribers to remove it from the list when it has received the + // SubscribePending message. The AtomicReference is set to null by the shutdown method, which is + // called by the actor from postStop. Pending (unregistered) subscription attempts are denied by + // the shutdown method. Subscription attempts after shutdown can be denied immediately. + private val pendingSubscribers = new AtomicReference[immutable.Seq[Subscriber[Any]]](Nil) + + protected val wakeUpMsg: Any = SubscribePending(boundary) + + override def subscribe(subscriber: Subscriber[_ >: Any]): Unit = { + requireNonNullSubscriber(subscriber) + @tailrec def doSubscribe(): Unit = { + val current = pendingSubscribers.get + if (current eq null) + reportSubscribeFailure(subscriber) + else { + if (pendingSubscribers.compareAndSet(current, subscriber +: current)) { + if (boundary.getActor ne null) boundary.getActor ! wakeUpMsg + } else { + doSubscribe() // CAS retry + } + } + } + + doSubscribe() + } + + def takePendingSubscribers(): immutable.Seq[Subscriber[Any]] = { + val pending = pendingSubscribers.getAndSet(Nil) + if (pending eq null) Nil else pending.reverse + } + + def shutdown(reason: Option[Throwable]): Unit = { + shutdownReason = reason + pendingSubscribers.getAndSet(null) match { + case null ⇒ // already called earlier + case pending ⇒ pending foreach reportSubscribeFailure + } + } + + @volatile private var shutdownReason: Option[Throwable] = None + + private def reportSubscribeFailure(subscriber: Subscriber[Any]): Unit = + try shutdownReason match { + case Some(e: SpecViolation) ⇒ // ok, not allowed to call onError + case Some(e) ⇒ + tryOnSubscribe(subscriber, CancelledSubscription) + tryOnError(subscriber, e) + case None ⇒ + tryOnSubscribe(subscriber, CancelledSubscription) + tryOnComplete(subscriber) + } catch { + case _: SpecViolation ⇒ // nothing to do + } + + override def toString: String = s"Publisher[$internalPortName]" + } + + private[stream] class ActorOutputBoundary(val shell: GraphInterpreterShell, val internalPortName: String) + extends DownstreamBoundaryStageLogic[Any] { + + val in: Inlet[Any] = Inlet[Any]("UpstreamBoundary:" + internalPortName) in.id = 0 - private var exposedPublisher: ActorPublisher[Any] = _ + val publisher = new OutputBoundaryPublisher(this, internalPortName) + + @volatile private var actor: ActorRef = null + def setActor(actor: ActorRef): Unit = this.actor = actor + def getActor: ActorRef = this.actor private var subscriber: Subscriber[Any] = _ private var downstreamDemand: Long = 0L @@ -231,7 +346,7 @@ object ActorGraphInterpreter { // No need to complete if had already been cancelled, or we closed earlier if (!(upstreamCompleted || downstreamCompleted)) { upstreamCompleted = true - if (exposedPublisher ne null) exposedPublisher.shutdown(None) + publisher.shutdown(None) if (subscriber ne null) tryOnComplete(subscriber) } } @@ -241,7 +356,7 @@ object ActorGraphInterpreter { if (!(downstreamCompleted || upstreamCompleted)) { upstreamCompleted = true upstreamFailed = Some(e) - if (exposedPublisher ne null) exposedPublisher.shutdown(Some(e)) + publisher.shutdown(Some(e)) if ((subscriber ne null) && !e.isInstanceOf[SpecViolation]) tryOnError(subscriber, e) } } @@ -261,25 +376,22 @@ object ActorGraphInterpreter { }) def subscribePending(): Unit = - exposedPublisher.takePendingSubscribers() foreach { sub ⇒ + publisher.takePendingSubscribers() foreach { sub ⇒ if (subscriber eq null) { subscriber = sub - tryOnSubscribe(subscriber, new BoundarySubscription(actor, shell, id)) - if (GraphInterpreter.Debug) println(s"${interpreter.Name} subscribe subscriber=$sub") + val subscription = new Subscription { + override def request(elements: Long): Unit = actor ! RequestMore(ActorOutputBoundary.this, elements) + override def cancel(): Unit = actor ! Cancel(ActorOutputBoundary.this) + override def toString = s"BoundarySubscription[$actor, $internalPortName]" + } + + tryOnSubscribe(subscriber, subscription) + // TODO: add interpreter name back + if (GraphInterpreter.Debug) println(s" subscribe subscriber=$sub") } else rejectAdditionalSubscriber(subscriber, s"${Logging.simpleName(this)}") } - def exposedPublisher(publisher: ActorPublisher[Any]): Unit = { - exposedPublisher = publisher - upstreamFailed match { - case _: Some[_] ⇒ - publisher.shutdown(upstreamFailed) - case _ ⇒ - if (upstreamCompleted) publisher.shutdown(None) - } - } - def requestMore(elements: Long): Unit = { if (elements < 1) { cancel(in) @@ -295,11 +407,11 @@ object ActorGraphInterpreter { def cancel(): Unit = { downstreamCompleted = true subscriber = null - exposedPublisher.shutdown(Some(new ActorPublisher.NormalShutdownException)) + publisher.shutdown(Some(new ActorPublisher.NormalShutdownException)) cancel(in) } - override def toString: String = s"ActorOutputBoundary(id=$id, demand=$downstreamDemand, finished=$downstreamCompleted)" + override def toString: String = s"ActorOutputBoundary(port=$internalPortName, demand=$downstreamDemand, finished=$downstreamCompleted)" } } @@ -308,21 +420,51 @@ object ActorGraphInterpreter { * INTERNAL API */ final class GraphInterpreterShell( - assembly: GraphAssembly, - connections: Array[Connection], - logics: Array[GraphStageLogic], - shape: Shape, - settings: ActorMaterializerSettings, - val mat: ExtendedActorMaterializer) { + var connections: Array[Connection], + var logics: Array[GraphStageLogic], + settings: ActorMaterializerSettings, + val mat: ExtendedActorMaterializer) { import ActorGraphInterpreter._ private var self: ActorRef = _ lazy val log = Logging(mat.system.eventStream, self) + final case class AsyncInput(shell: GraphInterpreterShell, logic: GraphStageLogic, evt: Any, handler: (Any) ⇒ Unit) extends BoundaryEvent { + override def execute(eventLimit: Int): Int = { + if (!waitingForShutdown) { + interpreter.runAsyncInput(logic, evt, handler) + if (eventLimit == 1 && interpreter.isSuspended) { + sendResume(true) + 0 + } else runBatch(eventLimit - 1) + } else eventLimit + } + } + + final case class ResumeShell(shell: GraphInterpreterShell) extends BoundaryEvent { + override def execute(eventLimit: Int): Int = { + if (!waitingForShutdown) { + if (GraphInterpreter.Debug) println(s"${interpreter.Name} resume") + if (interpreter.isSuspended) runBatch(eventLimit) else eventLimit + } else eventLimit + } + } + + final case class Abort(shell: GraphInterpreterShell) extends BoundaryEvent { + override def execute(eventLimit: Int): Int = { + if (waitingForShutdown) { + subscribesPending = 0 + tryAbort(new TimeoutException("Streaming actor has been already stopped processing (normally), but not all of its " + + s"inputs or outputs have been subscribed in [${settings.subscriptionTimeoutSettings.timeout}}]. Aborting actor now.")) + } + 0 + } + } + private var enqueueToShortCircuit: (Any) ⇒ Unit = _ - lazy val interpreter: GraphInterpreter = new GraphInterpreter(assembly, mat, log, logics, connections, + lazy val interpreter: GraphInterpreter = new GraphInterpreter(mat, log, logics, connections, (logic, event, handler) ⇒ { val asyncInput = AsyncInput(this, logic, event, handler) val currentInterpreter = GraphInterpreter.currentInterpreterOrNull @@ -331,11 +473,11 @@ final class GraphInterpreterShell( else enqueueToShortCircuit(asyncInput) }, settings.fuzzingMode, self) - private val inputs = new Array[BatchingActorInputBoundary](shape.inlets.size) - private val outputs = new Array[ActorOutputBoundary](shape.outlets.size) + // TODO: really needed? + private var subscribesPending = 0 - private var subscribesPending = inputs.length - private var publishersPending = outputs.length + private var inputs: List[BatchingActorInputBoundary] = Nil + private var outputs: List[ActorOutputBoundary] = Nil def dumpWaits(): Unit = interpreter.dumpWaits() @@ -352,7 +494,8 @@ final class GraphInterpreterShell( * - if the event limit is greater than maxInputBufferSize * (ins + outs) than there will always be expand activity * because no data can enter “fast enough” from the outside */ - val shellEventLimit = settings.maxInputBufferSize * (assembly.ins.length + assembly.outs.length) + // TODO: Fix event limit heuristic + val shellEventLimit = settings.maxInputBufferSize * 16 // Limits the number of events processed by the interpreter on an abort event. // TODO: Better heuristic here private val abortLimit = shellEventLimit * 2 @@ -363,101 +506,42 @@ final class GraphInterpreterShell( this.self = self this.enqueueToShortCircuit = enqueueToShortCircuit var i = 0 - while (i < inputs.length) { - val in = new BatchingActorInputBoundary(settings.maxInputBufferSize, i) - inputs(i) = in - interpreter.attachUpstreamBoundary(connections(i), in) - i += 1 - } - val offset = assembly.connectionCount - outputs.length - i = 0 - while (i < outputs.length) { - val out = new ActorOutputBoundary(self, this, i) - outputs(i) = out - interpreter.attachDownstreamBoundary(connections(i + offset), out) + while (i < logics.length) { + logics(i) match { + case in: BatchingActorInputBoundary ⇒ + in.setActor(self) + subscribesPending += 1 + inputs ::= in + case out: ActorOutputBoundary ⇒ + out.setActor(self) + out.subscribePending() + outputs ::= out + case _ ⇒ + } i += 1 } + interpreter.init(subMat) runBatch(eventLimit) } - def receive(event: BoundaryEvent, eventLimit: Int): Int = { + def processEvent(event: BoundaryEvent, eventLimit: Int): Int = { resumeScheduled = false - if (waitingForShutdown) { - event match { - case ExposedPublisher(_, id, publisher) ⇒ - outputs(id).exposedPublisher(publisher) - publishersPending -= 1 - if (canShutDown) interpreterCompleted = true - case OnSubscribe(_, _, sub) ⇒ - tryCancel(sub) - subscribesPending -= 1 - if (canShutDown) interpreterCompleted = true - case Abort(_) ⇒ - // Not waiting anymore, leak is unlikely due to the timeout we have waited - publishersPending = 0 - subscribesPending = 0 - tryAbort(new TimeoutException("Streaming actor has been already stopped processing (normally), but not all of its " + - s"inputs or outputs have been subscribed in [${settings.subscriptionTimeoutSettings.timeout}}]. Aborting actor now.")) - case _ ⇒ // Ignore, there is nothing to do anyway - } - eventLimit - } else event match { - // Cases that are most likely on the hot path, in decreasing order of frequency - case OnNext(_, id: Int, e: Any) ⇒ - if (GraphInterpreter.Debug) println(s"${interpreter.Name} onNext $e id=$id") - inputs(id).onNext(e) - runBatch(eventLimit) - case RequestMore(_, id: Int, demand: Long) ⇒ - if (GraphInterpreter.Debug) println(s"${interpreter.Name} request $demand id=$id") - outputs(id).requestMore(demand) - runBatch(eventLimit) - case Resume(_) ⇒ - if (GraphInterpreter.Debug) println(s"${interpreter.Name} resume") - if (interpreter.isSuspended) runBatch(eventLimit) else eventLimit - case AsyncInput(_, logic, event, handler) ⇒ - interpreter.runAsyncInput(logic, event, handler) - if (eventLimit == 1 && interpreter.isSuspended) { - sendResume(true) - 0 - } else runBatch(eventLimit - 1) - - // Initialization and completion messages - case OnError(_, id: Int, cause: Throwable) ⇒ - if (GraphInterpreter.Debug) println(s"${interpreter.Name} onError id=$id") - inputs(id).onError(cause) - runBatch(eventLimit) - case OnComplete(_, id: Int) ⇒ - if (GraphInterpreter.Debug) println(s"${interpreter.Name} onComplete id=$id") - inputs(id).onComplete() - runBatch(eventLimit) - case OnSubscribe(_, id: Int, subscription: Subscription) ⇒ - if (GraphInterpreter.Debug) println(s"${interpreter.Name} onSubscribe id=$id") - subscribesPending -= 1 - inputs(id).onSubscribe(subscription) - runBatch(eventLimit) - case Cancel(_, id: Int) ⇒ - if (GraphInterpreter.Debug) println(s"${interpreter.Name} cancel id=$id") - outputs(id).cancel() - runBatch(eventLimit) - case SubscribePending(_, id: Int) ⇒ - outputs(id).subscribePending() - eventLimit - case ExposedPublisher(_, id, publisher) ⇒ - publishersPending -= 1 - outputs(id).exposedPublisher(publisher) - eventLimit - } + event.execute(eventLimit) } private var interpreterCompleted = false def isTerminated: Boolean = interpreterCompleted && canShutDown - private def canShutDown: Boolean = subscribesPending + publishersPending == 0 + private def canShutDown: Boolean = subscribesPending == 0 + + def subscribeArrived(): Unit = { + subscribesPending -= 1 + } private var waitingForShutdown: Boolean = false - private val resume = Resume(this) + private val resume = ResumeShell(this) def sendResume(sendResume: Boolean): Unit = { resumeScheduled = true @@ -495,6 +579,7 @@ final class GraphInterpreterShell( * - a new error is encountered */ def tryAbort(ex: Throwable): Unit = { + ex.printStackTrace() val reason = ex match { case s: SpecViolation ⇒ new IllegalStateException("Shutting down because of violation of the Reactive Streams specification.", s) @@ -520,7 +605,8 @@ final class GraphInterpreterShell( } } - override def toString: String = s"GraphInterpreterShell\n ${assembly.toString.replace("\n", "\n ")}" + // TODO: Fix debug string + override def toString: String = s"GraphInterpreterShell\n" // ${assembly.toString.replace("\n", "\n ")}" } /** @@ -561,7 +647,7 @@ class ActorGraphInterpreter(_initial: GraphInterpreterShell) extends Actor with def registerShell(shell: GraphInterpreterShell): ActorRef = { newShells ::= shell - enqueueToShortCircuit(Resume) + enqueueToShortCircuit(shell.ResumeShell(shell)) self } @@ -592,15 +678,15 @@ class ActorGraphInterpreter(_initial: GraphInterpreterShell) extends Actor with while (!shortCircuitBuffer.isEmpty && currentLimit > 0 && activeInterpreters.nonEmpty) shortCircuitBuffer.poll() match { case b: BoundaryEvent ⇒ processEvent(b) - case Resume ⇒ finishShellRegistration() + case ResumeActor ⇒ finishShellRegistration() } - if (!shortCircuitBuffer.isEmpty && currentLimit == 0) self ! Resume + if (!shortCircuitBuffer.isEmpty && currentLimit == 0) self ! ResumeActor } private def processEvent(b: BoundaryEvent): Unit = { val shell = b.shell if (!shell.isTerminated && (shell.isInitialized || tryInit(shell))) { - try currentLimit = shell.receive(b, currentLimit) + try currentLimit = shell.processEvent(b, currentLimit) catch { case NonFatal(e) ⇒ shell.tryAbort(e) } @@ -618,7 +704,7 @@ class ActorGraphInterpreter(_initial: GraphInterpreterShell) extends Actor with processEvent(b) if (shortCircuitBuffer != null) shortCircuitBatch() - case Resume ⇒ + case ResumeActor ⇒ currentLimit = eventLimit if (shortCircuitBuffer != null) shortCircuitBatch() diff --git a/akka-stream/src/main/scala/akka/stream/impl/fusing/Fusing.scala b/akka-stream/src/main/scala/akka/stream/impl/fusing/Fusing.scala deleted file mode 100644 index 0580f8c02d..0000000000 --- a/akka-stream/src/main/scala/akka/stream/impl/fusing/Fusing.scala +++ /dev/null @@ -1,765 +0,0 @@ -/** - * Copyright (C) 2015-2017 Lightbend Inc. - */ -package akka.stream.impl.fusing - -import java.util.Arrays -import java.{ util ⇒ ju } - -import akka.stream.Attributes.AsyncBoundary -import akka.stream.Fusing.FusedGraph -import akka.stream._ -import akka.stream.impl.StreamLayout -import akka.stream.impl.StreamLayout._ -import akka.stream.impl.fusing.GraphStages.MaterializedValueSource -import akka.stream.stage.GraphStageWithMaterializedValue - -import scala.annotation.tailrec -import scala.collection.JavaConverters._ -import scala.collection.immutable -import scala.util.control.NonFatal - -/** - * INTERNAL API - */ -private[stream] object Fusing { - - final val Debug = false - - /** - * Fuse everything that is not forbidden via AsyncBoundary attribute. - */ - def aggressive[S <: Shape, M](g: Graph[S, M]): FusedGraph[S, M] = - g match { - case fg: FusedGraph[_, _] ⇒ fg - case FusedGraph(module, shape) ⇒ FusedGraph(module, shape) - case _ ⇒ doAggressive(g) - } - - def structuralInfo[S <: Shape, M](g: Graph[S, M], attributes: Attributes): StructuralInfoModule = { - val struct = new BuildStructuralInfo - /* - * First perform normalization by descending the module tree and recording - * information in the BuildStructuralInfo instance. - */ - val matValue = - try descend(g.module, Attributes.none, struct, struct.newGroup(0), 0) - catch { - case NonFatal(ex) ⇒ - if (Debug) struct.dump() - throw ex - } - /* - * Then create a copy of the original Shape with the new copied ports. - */ - val shape = g.shape.copyFromPorts( - struct.newInlets(g.shape.inlets), - struct.newOutlets(g.shape.outlets)).asInstanceOf[S] - /* - * Extract the full topological information from the builder - */ - struct.toInfo(shape, matValue, attributes) - } - - private def doAggressive[S <: Shape, M](g: Graph[S, M]): FusedGraph[S, M] = { - val struct = new BuildStructuralInfo - /* - * First perform normalization by descending the module tree and recording - * information in the BuildStructuralInfo instance. - */ - val matValue = - try descend(g.module, Attributes.none, struct, struct.newGroup(0), 0) - catch { - case NonFatal(ex) ⇒ - if (Debug) struct.dump() - throw ex - } - /* - * Then create a copy of the original Shape with the new copied ports. - */ - val shape = g.shape.copyFromPorts( - struct.newInlets(g.shape.inlets), - struct.newOutlets(g.shape.outlets)).asInstanceOf[S] - /* - * Extract the full topological information from the builder before - * removing assembly-internal (fused) wirings in the next step. - */ - val info = struct.toInfo(shape, matValue) - /* - * Perform the fusing of `struct.groups` into GraphModules (leaving them - * as they are for non-fusable modules). - */ - struct.removeInternalWires() - struct.breakUpGroupsByDispatcher() - val modules = fuse(struct) - /* - * Now we have everything ready for a FusedModule. - */ - val module = FusedModule( - modules, - shape, - immutable.Map.empty ++ struct.downstreams.asScala, - immutable.Map.empty ++ struct.upstreams.asScala, - matValue.head._2, - Attributes.none, - info) - - if (StreamLayout.Debug) validate(module) - if (Debug) println(module) - - FusedGraph(module, shape) - } - - /** - * Take the fusable islands identified by `descend` in the `groups` list - * and execute their fusion; only fusable islands will have multiple modules - * in their set. - */ - private def fuse(struct: BuildStructuralInfo): Set[Module] = - struct.groups.asScala.flatMap { group ⇒ - if (group.size == 0) Nil - else if (group.size == 1) group.iterator.next() :: Nil - else fuseGroup(struct, group) :: Nil - }(collection.breakOut) - - /** - * Transform a set of GraphStageModules into a single GraphModule. This is done - * by performing a traversal of all their Inlets, sorting them into those without - * internal connections (the exposed inlets) and those with internal connections - * (where the corresponding Outlet is recorded in a map so that it will be wired - * to the same slot number in the GraphAssembly). Then all Outlets are traversed, - * completing internal connections using the aforementioned maps and appending - * the others to the list of exposed Outlets. - */ - private def fuseGroup(struct: BuildStructuralInfo, group: ju.Set[Module]): GraphModule = { - val stages = new Array[GraphStageWithMaterializedValue[Shape, Any]](group.size) - val matValIDs = new Array[Module](group.size) - val attributes = new Array[Attributes](group.size) - - /* - * The overall GraphAssembly arrays are constructed in three parts: - * - 1) exposed inputs (ins) - * - 2) connections (ins and outs) - * - 3) exposed outputs (outs) - */ - val insB1, insB2 = new ju.ArrayList[Inlet[_]] - val outsB3 = new ju.ArrayList[Outlet[_]] - val inOwnersB1, inOwnersB2 = new ju.ArrayList[Int] - val outOwnersB3 = new ju.ArrayList[Int] - - // for the shape of the GraphModule - val inlets = new ju.ArrayList[Inlet[_]] - val outlets = new ju.ArrayList[Outlet[_]] - - // connection slots are allocated from the inputs side, outs find their place by this map - val outConns: ju.Map[OutPort, Int] = new ju.HashMap - - /* - * First traverse all Inlets and sort them into exposed and internal, - * taking note of their partner Outlets where appropriate. - */ - var pos = 0 - var it = group.iterator - val ups = struct.upstreams - val downs = struct.downstreams - val outGroup = struct.outGroup - while (it.hasNext) it.next() match { - case copy @ CopiedModule(shape, attr, gsm: GraphStageModule) ⇒ - stages(pos) = gsm.stage - matValIDs(pos) = copy - attributes(pos) = attr and gsm.attributes - - shape.inlets.iterator.zip(gsm.shape.inlets.iterator).foreach { - case (in, orig) ⇒ - val out = ups.get(in) - val internal = (out != null) && (outGroup.get(out) eq group) - if (internal) { - ups.remove(in) - downs.remove(out) - outConns.put(out, insB2.size) - insB2.add(orig) - inOwnersB2.add(pos) - } else { - insB1.add(orig) - inOwnersB1.add(pos) - inlets.add(in) - } - } - - pos += 1 - case _ ⇒ throw new IllegalArgumentException("unexpected module structure") - } - - val outsB2 = new Array[Outlet[_]](insB2.size) - val outOwnersB2 = new Array[Int](insB2.size) - - /* - * Then traverse all Outlets and complete connections. - */ - pos = 0 - it = group.iterator - while (it.hasNext) it.next() match { - case CopiedModule(shape, _, gsm: GraphStageModule) ⇒ - shape.outlets.iterator.zip(gsm.shape.outlets.iterator).foreach { - case (out, orig) ⇒ - if (outConns.containsKey(out)) { - val idx = outConns.remove(out) - outsB2(idx) = orig - outOwnersB2(idx) = pos - } else { - outsB3.add(orig) - outOwnersB3.add(pos) - outlets.add(out) - } - } - pos += 1 - case _ ⇒ throw new IllegalArgumentException("unexpected module structure") - } - - /* - * Now mechanically gather together the GraphAssembly arrays from their various pieces. - */ - - val shape = AmorphousShape(inlets.asScala.to[immutable.Seq], outlets.asScala.to[immutable.Seq]) - - val connStart = insB1.size - val conns = insB2.size - val outStart = connStart + conns - val size = outStart + outsB3.size - - val ins = new Array[Inlet[_]](size) - copyToArray(insB2.iterator, ins, copyToArray(insB1.iterator, ins, 0)) - - val inOwners = new Array[Int](size) - Arrays.fill(inOwners, copyToArray(inOwnersB2.iterator, inOwners, copyToArray(inOwnersB1.iterator, inOwners, 0)), size, -1) - - val outs = new Array[Outlet[_]](size) - System.arraycopy(outsB2, 0, outs, connStart, conns) - copyToArray(outsB3.iterator, outs, outStart) - - val outOwners = new Array[Int](size) - Arrays.fill(outOwners, 0, connStart, -1) - System.arraycopy(outOwnersB2, 0, outOwners, connStart, conns) - copyToArray(outOwnersB3.iterator, outOwners, outStart) - - // FIXME attributes should contain some naming info and async boundary where needed - val firstModule = group.iterator.next() match { - case c: CopiedModule ⇒ c - case _ ⇒ throw new IllegalArgumentException("unexpected module structure") - } - val async = if (isAsync(firstModule)) Attributes(AsyncBoundary) else Attributes.none - val disp = dispatcher(firstModule) match { - case None ⇒ Attributes.none - case Some(d) ⇒ Attributes(d) - } - val attr = async and disp - - GraphModule(new GraphInterpreter.GraphAssembly(stages, attributes, ins, inOwners, outs, outOwners), shape, attr, matValIDs) - } - - @tailrec private def copyToArray[T](it: ju.Iterator[T], array: Array[T], idx: Int): Int = - if (it.hasNext) { - array(idx) = it.next() - copyToArray(it, array, idx + 1) - } else idx - - /** - * This is a normalization step for the graph that also collects the needed - * information for later fusing. The goal is to transform an arbitrarily deep - * module tree into one that has exactly two levels: all direct submodules are - * CopiedModules where each contains exactly one atomic module. This way all - * modules have their own identity and all necessary port copies have been - * made. The upstreams/downstreams in the BuildStructuralInfo are rewritten - * to point to the shapes of the copied modules. - * - * The materialized value computation is rewritten as well in that all - * leaf nodes point to the copied modules and all nested computations are - * “inlined”, resulting in only one big computation tree for the whole - * normalized overall module. The contained MaterializedValueSource stages - * are also rewritten to point to the copied MaterializedValueNodes. This - * correspondence is then used during materialization to trigger these sources - * when “their” node has received its value. - */ - private def descend( - m: Module, - inheritedAttributes: Attributes, - struct: BuildStructuralInfo, - openGroup: ju.Set[Module], - indent: Int): List[(Module, MaterializedValueNode)] = { - def log(msg: String): Unit = println(" " * indent + msg) - val async = m match { - case _: GraphStageModule ⇒ m.attributes.contains(AsyncBoundary) - case _: GraphModule ⇒ m.attributes.contains(AsyncBoundary) - case _ if m.isAtomic ⇒ true // non-GraphStage atomic or has AsyncBoundary - case _ ⇒ m.attributes.contains(AsyncBoundary) - } - if (Debug) log(s"entering ${m.getClass} (hash=${struct.hash(m)}, async=$async, name=${m.attributes.nameLifted}, dispatcher=${dispatcher(m)})") - val localGroup = - if (async) struct.newGroup(indent) - else openGroup - - if (m.isAtomic) { - m match { - case gm: GraphModule if !async ⇒ - // need to dissolve previously fused GraphStages to allow further fusion - if (Debug) log(s"dissolving graph module ${m.toString.replace("\n", "\n" + " " * indent)}") - val attributes = inheritedAttributes and m.attributes - gm.matValIDs.flatMap(sub ⇒ descend(sub, attributes, struct, localGroup, indent + 1))(collection.breakOut) - case gm @ GraphModule(_, oldShape, _, mvids) ⇒ - /* - * Importing a GraphModule that has an AsyncBoundary attribute is a little more work: - * - * - we need to copy all the CopiedModules that are in matValIDs - * - we need to rewrite the corresponding MaterializedValueNodes - * - we need to match up the new (copied) GraphModule shape with the individual Shape copies - * - we need to register the contained modules but take care to not include the internal - * wirings into the final result, see also `struct.removeInternalWires()` - */ - if (Debug) log(s"graph module ${m.toString.replace("\n", "\n" + " " * indent)}") - - // storing the old Shape in arrays for in-place updating as we clone the contained GraphStages - val oldIns = oldShape.inlets.toArray - val oldOuts = oldShape.outlets.toArray - - val newids = mvids.map { - case CopiedModule(shape, attr, copyOf) ⇒ - val newShape = shape.deepCopy - val copy = CopiedModule(newShape, attr, copyOf): Module - - // rewrite shape: first the inlets - val oldIn = shape.inlets.iterator - val newIn = newShape.inlets.iterator - while (oldIn.hasNext) { - val o = oldIn.next() - val n = newIn.next() - findInArray(o, oldIns) match { - case -1 ⇒ // nothing to do - case idx ⇒ oldIns(idx) = n - } - } - // ... then the outlets - val oldOut = shape.outlets.iterator - val newOut = newShape.outlets.iterator - while (oldOut.hasNext) { - val o = oldOut.next() - val n = newOut.next() - findInArray(o, oldOuts) match { - case -1 ⇒ // nothing to do - case idx ⇒ oldOuts(idx) = n - } - } - - // need to add the module so that the structural (internal) wirings can be rewritten as well - // but these modules must not be added to any of the groups - struct.addModule(copy, new ju.HashSet, inheritedAttributes, indent, shape) - struct.registerInternals(newShape, indent) - - copy - case _ ⇒ throw new IllegalArgumentException("unexpected module structure") - } - val newgm = gm.copy(shape = oldShape.copyFromPorts(oldIns.toList, oldOuts.toList), matValIDs = newids) - // make sure to add all the port mappings from old GraphModule Shape to new shape - struct.addModule(newgm, localGroup, inheritedAttributes, indent, _oldShape = oldShape) - // now compute the list of all materialized value computation updates - var result = List.empty[(Module, MaterializedValueNode)] - var i = 0 - while (i < mvids.length) { - result ::= mvids(i) → Atomic(newids(i)) - i += 1 - } - result ::= m → Atomic(newgm) - result - case _ ⇒ - if (Debug) log(s"atomic module $m") - List(m → struct.addModule(m, localGroup, inheritedAttributes, indent)) - } - } else { - val attributes = inheritedAttributes and m.attributes - m match { - case CopiedModule(shape, _, copyOf) ⇒ - val ret = - descend(copyOf, attributes, struct, localGroup, indent + 1) match { - case xs @ (_, mat) :: _ ⇒ (m → mat) :: xs - case _ ⇒ throw new IllegalArgumentException("cannot happen") - } - struct.rewire(copyOf.shape, shape, indent) - ret - case _ ⇒ - // we need to keep track of all MaterializedValueSource nodes that get pushed into the current - // computation context (i.e. that need the same value). - struct.enterMatCtx() - // now descend into submodules and collect their computations (plus updates to `struct`) - val subMatBuilder = Predef.Map.newBuilder[Module, MaterializedValueNode] - val subIterator = m.subModules.iterator - while (subIterator.hasNext) { - val sub = subIterator.next() - val res = descend(sub, attributes, struct, localGroup, indent + 1) - subMatBuilder ++= res - } - val subMat = subMatBuilder.result() - if (Debug) log(subMat.map(p ⇒ s"${p._1.getClass.getName}[${struct.hash(p._1)}] -> ${p._2}").mkString("subMat\n " + " " * indent, "\n " + " " * indent, "")) - // we need to remove all wirings that this module copied from nested modules so that we - // don’t do wirings twice - val oldDownstreams = m match { - case f: FusedModule ⇒ f.info.downstreams.toSet - case _ ⇒ m.downstreams.toSet - } - val down = m.subModules.foldLeft(oldDownstreams)((set, m) ⇒ set -- m.downstreams) - down.foreach { - case (start, end) ⇒ struct.wire(start, end, indent) - } - // now rewrite the materialized value computation based on the copied modules and their computation nodes - val matNodeMapping: ju.Map[MaterializedValueNode, MaterializedValueNode] = new ju.HashMap - val newMat = rewriteMat(subMat, m.materializedValueComputation, matNodeMapping) - if (Debug) log(matNodeMapping.asScala.map(p ⇒ s"${p._1} -> ${p._2}").mkString("matNodeMapping\n " + " " * indent, "\n " + " " * indent, "")) - // and finally rewire all MaterializedValueSources to their new computation nodes - val matSrcs = struct.exitMatCtx() - matSrcs.foreach { c ⇒ - val ms = c.copyOf.asInstanceOf[GraphStageModule].stage.asInstanceOf[MaterializedValueSource[Any]] - val mapped = ms.computation match { - case Atomic(sub) ⇒ subMat(sub) - case Ignore ⇒ Ignore - case other ⇒ matNodeMapping.get(other) - } - if (Debug) log(s"materialized value source: ${c.copyOf} -> $mapped") - require(mapped != null, s"mismatch:\n ${ms.computation}\n ${m.materializedValueComputation}") - val newSrc = new MaterializedValueSource[Any](mapped, ms.out) - val replacement = CopiedModule(c.shape, c.attributes, newSrc.module) - struct.replace(c, replacement, localGroup) - } - // the result for each level is the materialized value computation - List(m → newMat) - } - } - } - - @tailrec - private def findInArray[T](elem: T, arr: Array[T], idx: Int = 0): Int = - if (idx >= arr.length) -1 - else if (arr(idx) == elem) idx - else findInArray(elem, arr, idx + 1) - - /** - * Given a mapping from old modules to new MaterializedValueNode, rewrite the given - * computation while also populating a mapping from old computation nodes to new ones. - * That mapping is needed to rewrite the MaterializedValueSource stages later-on in - * descend(). - */ - private def rewriteMat(subMat: Predef.Map[Module, MaterializedValueNode], mat: MaterializedValueNode, - mapping: ju.Map[MaterializedValueNode, MaterializedValueNode]): MaterializedValueNode = - mat match { - case Atomic(sub) ⇒ - val ret = subMat(sub) - mapping.put(mat, ret) - ret - case Combine(f, left, right) ⇒ - val ret = Combine(f, rewriteMat(subMat, left, mapping), rewriteMat(subMat, right, mapping)) - mapping.put(mat, ret) - ret - case Transform(f, dep) ⇒ - val ret = Transform(f, rewriteMat(subMat, dep, mapping)) - mapping.put(mat, ret) - ret - case Ignore ⇒ Ignore - } - - /** - * INTERNAL API - * - * Collect structural information about a module tree while descending into - * it and performing normalization. - */ - final class BuildStructuralInfo { - def toInfo[S <: Shape](shape: S, matValues: List[(Module, MaterializedValueNode)], - attributes: Attributes = Attributes.none): StructuralInfoModule = - StructuralInfoModule( - Set.empty ++ modules.asScala, - shape, - immutable.Map.empty ++ downstreams.asScala, - immutable.Map.empty ++ upstreams.asScala, - immutable.Map.empty ++ inOwners.asScala, - immutable.Map.empty ++ outOwners.asScala, - matValues, - matValues.head._2, - attributes) - - /** - * the set of all contained modules - */ - val modules: ju.Set[Module] = new ju.HashSet - - /** - * the list of all groups of modules that are within each async boundary - */ - val groups: ju.Deque[ju.Set[Module]] = new ju.LinkedList - - /** - * Fusable groups may contain modules with differing dispatchers, in which - * case the group needs to be broken up. - */ - def breakUpGroupsByDispatcher(): Unit = { - val newGroups: ju.List[ju.Set[Module]] = new ju.LinkedList - val it = groups.iterator() - while (it.hasNext) { - val group = it.next() - if (group.size > 1) { - val subgroups = group.asScala.groupBy(dispatcher) - if (subgroups.size > 1) { - group.clear() - subgroups.valuesIterator.foreach(g ⇒ newGroups.add(g.asJava)) - } - } - } - groups.addAll(newGroups) - } - - /** - * a mapping from OutPort to its containing group, needed when determining - * whether an upstream connection is internal or not - */ - val outGroup: ju.Map[OutPort, ju.Set[Module]] = new ju.HashMap - - def replace(oldMod: Module, newMod: Module, localGroup: ju.Set[Module]): Unit = { - modules.remove(oldMod) - modules.add(newMod) - localGroup.remove(oldMod) - localGroup.add(newMod) - } - - /** - * A stack of mappings for a given non-copied InPort. - */ - val newIns: ju.Map[InPort, List[InPort]] = new ju.HashMap - /** - * A stack of mappings for a given non-copied OutPort. - */ - val newOuts: ju.Map[OutPort, List[OutPort]] = new ju.HashMap - - private def addMapping[T](orig: T, mapd: T, map: ju.Map[T, List[T]]): Unit = { - if (map.containsKey(orig)) { - map.put(orig, mapd :: map.get(orig)) - } else map.put(orig, mapd :: Nil) - } - - private def removeMapping[T](orig: T, map: ju.Map[T, List[T]]): T = - map.remove(orig) match { - case null ⇒ null.asInstanceOf[T] - case Nil ⇒ throw new IllegalStateException("mappings corrupted") - case x :: Nil ⇒ x - case x :: xs ⇒ - map.put(orig, xs) - x - } - - /** - * A stack of materialized value sources, grouped by materialized computation context. - */ - private var matSrc: List[List[CopiedModule]] = Nil - - def enterMatCtx(): Unit = matSrc ::= Nil - def exitMatCtx(): List[CopiedModule] = - matSrc match { - case x :: xs ⇒ - matSrc = xs - x - case Nil ⇒ throw new IllegalArgumentException("exitMatCtx with empty stack") - } - def pushMatSrc(m: CopiedModule): Unit = - matSrc match { - case x :: xs ⇒ matSrc = (m :: x) :: xs - case Nil ⇒ throw new IllegalArgumentException("pushMatSrc without context") - } - - /** - * The downstreams relationships of the original module rewritten in terms of - * the copied ports. - */ - val downstreams: ju.Map[OutPort, InPort] = new ju.HashMap - /** - * The upstreams relationships of the original module rewritten in terms of - * the copied ports. - */ - val upstreams: ju.Map[InPort, OutPort] = new ju.HashMap - - /** - * The owner mapping for the copied InPorts. - */ - val inOwners: ju.Map[InPort, Module] = new ju.HashMap - /** - * The owner mapping for the copied OutPorts. - */ - val outOwners: ju.Map[OutPort, Module] = new ju.HashMap - - /** - * List of internal wirings of GraphModules that were incorporated. - */ - val internalOuts: ju.Set[OutPort] = new ju.HashSet - - /** - * Register the outlets of the given Shape as sources for internal - * connections within imported (and not dissolved) GraphModules. - * See also the comment in addModule where this is partially undone. - */ - def registerInternals(s: Shape, indent: Int): Unit = { - if (Debug) println(" " * indent + s"registerInternals(${s.outlets.map(hash)})") - internalOuts.addAll(s.outlets.asJava) - } - - /** - * Remove wirings that belong to the fused stages contained in GraphModules - * that were incorporated in this fusing run. - */ - def removeInternalWires(): Unit = { - val it = internalOuts.iterator() - while (it.hasNext) { - val out = it.next() - val in = downstreams.remove(out) - if (in != null) upstreams.remove(in) - } - } - - def dump(): Unit = { - println("StructuralInfo:") - println(" newIns:") - newIns.asScala.foreach { case (k, v) ⇒ println(s" $k (${hash(k)}) -> ${v.map(hash).mkString(",")}") } - println(" newOuts:") - newOuts.asScala.foreach { case (k, v) ⇒ println(s" $k (${hash(k)}) -> ${v.map(hash).mkString(",")}") } - } - - def hash(obj: AnyRef) = f"${System.identityHashCode(obj)}%08x" - def printShape(s: Shape) = s"${s.getClass.getSimpleName}(ins=${s.inlets.map(hash).mkString(",")} outs=${s.outlets.map(hash).mkString(",")})" - - /** - * Create and return a new grouping (i.e. an AsyncBoundary-delimited context) - */ - def newGroup(indent: Int): ju.Set[Module] = { - val group = new ju.HashSet[Module] - if (Debug) println(" " * indent + s"creating new group ${hash(group)}") - groups.add(group) - group - } - - /** - * Add a module to the given group, performing normalization (i.e. giving it a unique port identity). - */ - def addModule(m: Module, group: ju.Set[Module], inheritedAttributes: Attributes, indent: Int, - _oldShape: Shape = null): Atomic = { - val copy = - if (_oldShape == null) CopiedModule(m.shape.deepCopy(), inheritedAttributes, realModule(m)) - else m - val oldShape = if (_oldShape == null) m.shape else _oldShape - if (Debug) println(" " * indent + s"adding copy ${hash(copy)} ${printShape(copy.shape)} of ${printShape(oldShape)}") - group.add(copy) - modules.add(copy) - copy.shape.outlets.foreach(o ⇒ outGroup.put(o, group)) - val orig1 = oldShape.inlets.iterator - val mapd1 = copy.shape.inlets.iterator - while (orig1.hasNext) { - val orig = orig1.next() - val mapd = mapd1.next() - addMapping(orig, mapd, newIns) - inOwners.put(mapd, copy) - } - val orig2 = oldShape.outlets.iterator - val mapd2 = copy.shape.outlets.iterator - while (orig2.hasNext) { - val orig = orig2.next() - val mapd = mapd2.next() - addMapping(orig, mapd, newOuts) - outOwners.put(mapd, copy) - } - /* - * In descend() we add internalOuts entries for all shapes that belong to stages that - * are part of a GraphModule that is not dissolved. This includes the exposed Outlets, - * which of course are external and thus need to be removed again from the internalOuts - * set. - */ - if (m.isInstanceOf[GraphModule]) internalOuts.removeAll(m.shape.outlets.asJava) - copy match { - case c @ CopiedModule(_, _, GraphStageModule(_, _, _: MaterializedValueSource[_])) ⇒ pushMatSrc(c) - case GraphModule(_, _, _, mvids) ⇒ - var i = 0 - while (i < mvids.length) { - mvids(i) match { - case c @ CopiedModule(_, _, GraphStageModule(_, _, _: MaterializedValueSource[_])) ⇒ pushMatSrc(c) - case _ ⇒ - } - i += 1 - } - case _ ⇒ - } - Atomic(copy) - } - - /** - * Record a wiring between two copied ports, using (and reducing) the port - * mappings. - */ - def wire(out: OutPort, in: InPort, indent: Int): Unit = { - if (Debug) println(" " * indent + s"wiring $out (${hash(out)}) -> $in (${hash(in)})") - val newOut = nonNullForPort(removeMapping(out, newOuts), out) - val newIn = nonNullForPort(removeMapping(in, newIns), in) - downstreams.put(newOut, newIn) - upstreams.put(newIn, newOut) - } - - /** - * Replace all mappings for a given shape with its new (copied) form. - */ - def rewire(oldShape: Shape, newShape: Shape, indent: Int): Unit = { - if (Debug) println(" " * indent + s"rewiring ${printShape(oldShape)} -> ${printShape(newShape)}") - oldShape.inlets.iterator.zip(newShape.inlets.iterator).foreach { - case (oldIn, newIn) ⇒ addMapping(newIn, nonNullForPort(removeMapping(oldIn, newIns), oldIn), newIns) - } - oldShape.outlets.iterator.zip(newShape.outlets.iterator).foreach { - case (oldOut, newOut) ⇒ addMapping(newOut, nonNullForPort(removeMapping(oldOut, newOuts), oldOut), newOuts) - } - } - - /** - * Transform original into copied Inlets. - */ - def newInlets(old: immutable.Seq[Inlet[_]]): immutable.Seq[Inlet[_]] = - old.map(i ⇒ newIns.get(i).head.inlet) - - /** - * Transform original into copied Inlets. - */ - def newOutlets(old: immutable.Seq[Outlet[_]]): immutable.Seq[Outlet[_]] = - old.map(o ⇒ newOuts.get(o).head.outlet) - - // optimization - specialized null check avoiding allocation or creation of unused strings - private def nonNullForPort[T](t: T, port: AnyRef): T = { - if (t != null) t - else throw new IllegalArgumentException(s"null encountered: $port (${hash(port)})") - } - - } - - /** - * Determine whether the given CopiedModule has an AsyncBoundary attribute. - */ - private def isAsync(m: CopiedModule): Boolean = m match { - case CopiedModule(_, inherited, orig) ⇒ - val attr = inherited and orig.attributes - attr.contains(AsyncBoundary) - } - - /** - * Figure out the dispatcher setting of a module. - */ - private def dispatcher(m: Module): Option[ActorAttributes.Dispatcher] = m match { - case CopiedModule(_, inherited, orig) ⇒ - val attr = inherited and orig.attributes - attr.get[ActorAttributes.Dispatcher] - case x ⇒ x.attributes.get[ActorAttributes.Dispatcher] - } - - /** - * See through copied modules to the “real” module. - */ - private def realModule(m: Module): Module = m match { - case CopiedModule(_, _, of) ⇒ realModule(of) - case other ⇒ other - } -} diff --git a/akka-stream/src/main/scala/akka/stream/impl/fusing/GraphInterpreter.scala b/akka-stream/src/main/scala/akka/stream/impl/fusing/GraphInterpreter.scala index dd74167ce9..8ddc235072 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/fusing/GraphInterpreter.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/fusing/GraphInterpreter.scala @@ -3,18 +3,12 @@ */ package akka.stream.impl.fusing -import java.util.Arrays import akka.actor.ActorRef import akka.event.LoggingAdapter import akka.stream.stage._ -import scala.annotation.tailrec -import scala.collection.immutable import akka.stream._ -import akka.stream.impl.StreamLayout._ import java.util.concurrent.ThreadLocalRandom import scala.util.control.NonFatal -import java.{ util ⇒ ju } -import akka.stream.impl.fusing.GraphStages.MaterializedValueSource /** * INTERNAL API @@ -28,7 +22,6 @@ object GraphInterpreter { final val Debug = false final val NoEvent = null - final val Boundary = -1 final val InReady = 1 final val Pulling = 2 @@ -81,11 +74,11 @@ object GraphInterpreter { * @param outHandler The handler that contains the callback for output events. */ final class Connection( - val id: Int, - val inOwnerId: Int, - val inOwner: GraphStageLogic, - val outOwnerId: Int, - val outOwner: GraphStageLogic, + var id: Int, + var inOwnerId: Int, + var inOwner: GraphStageLogic, + var outOwnerId: Int, + var outOwner: GraphStageLogic, var inHandler: InHandler, var outHandler: OutHandler ) { @@ -95,195 +88,6 @@ object GraphInterpreter { override def toString = s"Connection($id, $portState, $slot, $inHandler, $outHandler)" } - /** - * INTERNAL API - * - * A GraphAssembly represents a small stream processing graph to be executed by the interpreter. Instances of this - * class **must not** be mutated after construction. - * - * The array ``originalAttributes`` may contain the attribute information of the original atomic module, otherwise - * it must contain a none (otherwise the enclosing module could not overwrite attributes defined in this array). - * - * The arrays [[ins]] and [[outs]] correspond to the notion of a *connection* in the [[GraphInterpreter]]. Each slot - * *i* contains the input and output port corresponding to connection *i*. Slots where the graph is not closed (i.e. - * ports are exposed to the external world) are marked with *null* values. For example if an input port *p* is - * exposed, then outs(p) will contain a *null*. - * - * The arrays [[inOwners]] and [[outOwners]] are lookup tables from a connection id (the index of the slot) - * to a slot in the [[stages]] array, indicating which stage is the owner of the given input or output port. - * Slots which would correspond to non-existent stages (where the corresponding port is null since it represents - * the currently unknown external context) contain the value [[GraphInterpreter#Boundary]]. - * - * The current assumption by the infrastructure is that the layout of these arrays looks like this: - * - * +---------------------------------------+-----------------+ - * inOwners: | index to stages array | Boundary (-1) | - * +----------------+----------------------+-----------------+ - * ins: | exposed inputs | internal connections | nulls | - * +----------------+----------------------+-----------------+ - * outs: | nulls | internal connections | exposed outputs | - * +----------------+----------------------+-----------------+ - * outOwners: | Boundary (-1) | index to stages array | - * +----------------+----------------------------------------+ - * - * In addition, it is also assumed by the infrastructure that the order of exposed inputs and outputs in the - * corresponding segments of these arrays matches the exact same order of the ports in the [[Shape]]. - * - */ - final class GraphAssembly( - val stages: Array[GraphStageWithMaterializedValue[Shape, Any]], - val originalAttributes: Array[Attributes], - val ins: Array[Inlet[_]], - val inOwners: Array[Int], - val outs: Array[Outlet[_]], - val outOwners: Array[Int]) { - require(ins.length == inOwners.length && inOwners.length == outs.length && outs.length == outOwners.length) - - def connectionCount: Int = ins.length - - /** - * Takes an interpreter and returns three arrays required by the interpreter containing the input, output port - * handlers and the stage logic instances. - * - * Returns a tuple of - * - lookup table for Connections - * - array of the logics - */ - def materialize( - inheritedAttributes: Attributes, - copiedModules: Array[Module], - matVal: ju.Map[Module, Any], - register: MaterializedValueSource[Any] ⇒ Unit): (Array[Connection], Array[GraphStageLogic]) = { - val logics = Array.ofDim[GraphStageLogic](stages.length) - - var i = 0 - while (i < stages.length) { - // Port initialization loops, these must come first - val shape = stages(i).shape - - var idx = 0 - val inletItr = shape.inlets.iterator - while (inletItr.hasNext) { - val inlet = inletItr.next() - require(inlet.id == -1 || inlet.id == idx, s"Inlet $inlet was shared among multiple stages. This is illegal.") - inlet.id = idx - idx += 1 - } - - idx = 0 - val outletItr = shape.outlets.iterator - while (outletItr.hasNext) { - val outlet = outletItr.next() - require(outlet.id == -1 || outlet.id == idx, s"Outlet $outlet was shared among multiple stages. This is illegal.") - outlet.id = idx - idx += 1 - } - - val stage = stages(i) match { - case mv: MaterializedValueSource[_] ⇒ - val copy = mv.copySrc.asInstanceOf[MaterializedValueSource[Any]] - register(copy) - copy - case x ⇒ x - } - - val logicAndMat = stage.createLogicAndMaterializedValue(inheritedAttributes and originalAttributes(i)) - matVal.put(copiedModules(i), logicAndMat._2) - - logics(i) = logicAndMat._1 - i += 1 - } - - val connections = Array.ofDim[Connection](connectionCount) - - i = 0 - while (i < connectionCount) { - connections(i) = new Connection( - id = i, - inOwner = if (inOwners(i) == Boundary) null else logics(inOwners(i)), - inOwnerId = inOwners(i), - outOwner = if (outOwners(i) == Boundary) null else logics(outOwners(i)), - outOwnerId = outOwners(i), - inHandler = null, - outHandler = null - ) - - if (ins(i) ne null) { - val logic = logics(inOwners(i)) - logic.handlers(ins(i).id) match { - case null ⇒ throw new IllegalStateException(s"no handler defined in stage $logic for port ${ins(i)}") - case h: InHandler ⇒ - connections(i).inHandler = h - } - logics(inOwners(i)).portToConn(ins(i).id) = connections(i) - } - if (outs(i) ne null) { - val logic = logics(outOwners(i)) - val inCount = logic.inCount - logic.handlers(outs(i).id + inCount) match { - case null ⇒ throw new IllegalStateException(s"no handler defined in stage $logic for port ${outs(i)}") - case h: OutHandler ⇒ - connections(i).outHandler = h - } - logic.portToConn(outs(i).id + inCount) = connections(i) - } - i += 1 - } - - (connections, logics) - } - - override def toString: String = { - val stageList = stages.iterator.zip(originalAttributes.iterator).map { - case (stage, attr) ⇒ s"${stage.module} [${attr.attributeList.mkString(", ")}]" - } - "GraphAssembly\n " + - stageList.mkString("[ ", "\n ", "\n ]") + "\n " + - ins.mkString("[", ",", "]") + "\n " + - inOwners.mkString("[", ",", "]") + "\n " + - outs.mkString("[", ",", "]") + "\n " + - outOwners.mkString("[", ",", "]") - } - } - - object GraphAssembly { - /** - * INTERNAL API - */ - final def apply( - inlets: immutable.Seq[Inlet[_]], - outlets: immutable.Seq[Outlet[_]], - stages: GraphStageWithMaterializedValue[Shape, _]*): GraphAssembly = { - // add the contents of an iterator to an array starting at idx - @tailrec def add[T](i: Iterator[T], a: Array[T], idx: Int): Array[T] = - if (i.hasNext) { - a(idx) = i.next() - add(i, a, idx + 1) - } else a - - // fill array slots with Boundary - def markBoundary(owners: Array[Int], from: Int, to: Int): Array[Int] = { - Arrays.fill(owners, from, to, Boundary) - owners - } - - val inletsSize = inlets.size - val outletsSize = outlets.size - val connectionCount = inletsSize + outletsSize - require(connectionCount > 0, s"sum of inlets ({$inletsSize}) & outlets ({$outletsSize}) must be > 0") - - val assembly = new GraphAssembly( - stages.toArray, - GraphInterpreter.singleNoAttribute, - add(inlets.iterator, Array.ofDim(connectionCount), 0), - markBoundary(Array.ofDim(connectionCount), inletsSize, connectionCount), - add(outlets.iterator, Array.ofDim(connectionCount), inletsSize), - markBoundary(Array.ofDim(connectionCount), 0, inletsSize)) - - assembly - } - } - /** * INTERNAL API */ @@ -389,14 +193,14 @@ object GraphInterpreter { * edge of a balance is pulled, dissolving the original cycle). */ final class GraphInterpreter( - private val assembly: GraphInterpreter.GraphAssembly, - val materializer: Materializer, - val log: LoggingAdapter, - val logics: Array[GraphStageLogic], // Array of stage logics - val connections: Array[GraphInterpreter.Connection], - val onAsyncInput: (GraphStageLogic, Any, (Any) ⇒ Unit) ⇒ Unit, - val fuzzingMode: Boolean, - val context: ActorRef) { + val materializer: Materializer, + val log: LoggingAdapter, + val logics: Array[GraphStageLogic], // Array of stage logics + val connections: Array[GraphInterpreter.Connection], + val onAsyncInput: (GraphStageLogic, Any, (Any) ⇒ Unit) ⇒ Unit, + val fuzzingMode: Boolean, + val context: ActorRef) { + import GraphInterpreter._ private[this] val ChaseLimit = if (fuzzingMode) 0 else 16 @@ -408,12 +212,11 @@ final class GraphInterpreter( // The number of currently running stages. Once this counter reaches zero, the interpreter is considered to be // completed - private[this] var runningStages = assembly.stages.length + private[this] var runningStages = logics.length // Counts how many active connections a stage has. Once it reaches zero, the stage is automatically stopped. - private[this] val shutdownCounter = Array.tabulate(assembly.stages.length) { i ⇒ - val shape = assembly.stages(i).shape - shape.inlets.size + shape.outlets.size + private[this] val shutdownCounter = Array.tabulate(logics.length) { i ⇒ + logics(i).handlers.length } private[this] var _subFusingMaterializer: Materializer = _ @@ -421,7 +224,7 @@ final class GraphInterpreter( // An event queue implemented as a circular buffer // FIXME: This calculates the maximum size ever needed, but most assemblies can run on a smaller queue - private[this] val eventQueue = Array.ofDim[Connection](1 << (32 - Integer.numberOfLeadingZeros(assembly.connectionCount - 1))) + private[this] val eventQueue = Array.ofDim[Connection](1 << (32 - Integer.numberOfLeadingZeros(connections.length - 1))) private[this] val mask = eventQueue.length - 1 private[this] var queueHead: Int = 0 private[this] var queueTail: Int = 0 @@ -449,32 +252,6 @@ final class GraphInterpreter( */ private[stream] def nonNull: GraphInterpreter = this - /** - * Assign the boundary logic to a given connection. This will serve as the interface to the external world - * (outside the interpreter) to process and inject events. - */ - def attachUpstreamBoundary(connection: Connection, logic: UpstreamBoundaryStageLogic[_]): Unit = { - logic.portToConn(logic.out.id + logic.inCount) = connection - logic.interpreter = this - connection.outHandler = logic.handlers(0).asInstanceOf[OutHandler] - } - - def attachUpstreamBoundary(connection: Int, logic: UpstreamBoundaryStageLogic[_]): Unit = - attachUpstreamBoundary(connections(connection), logic) - - /** - * Assign the boundary logic to a given connection. This will serve as the interface to the external world - * (outside the interpreter) to process and inject events. - */ - def attachDownstreamBoundary(connection: Connection, logic: DownstreamBoundaryStageLogic[_]): Unit = { - logic.portToConn(logic.in.id) = connection - logic.interpreter = this - connection.inHandler = logic.handlers(0).asInstanceOf[InHandler] - } - - def attachDownstreamBoundary(connection: Int, logic: DownstreamBoundaryStageLogic[_]): Unit = - attachDownstreamBoundary(connections(connection), logic) - /** * Dynamic handler changes are communicated from a GraphStageLogic by this method. */ @@ -513,14 +290,14 @@ final class GraphInterpreter( var i = 0 while (i < logics.length) { val logic = logics(i) - logic.stageId = i logic.interpreter = this try { logic.beforePreStart() logic.preStart() } catch { case NonFatal(e) ⇒ - log.error(e, "Error during preStart in [{}]", assembly.stages(logic.stageId)) + e.printStackTrace() + // log.error(e, "Error during preStart in [{}]", assembly.stages(logic.stageId)) logic.failStage(e) } afterStageHasRun(logic) @@ -541,32 +318,16 @@ final class GraphInterpreter( } // Debug name for a connections input part - private def inOwnerName(connection: Connection): String = - assembly.inOwners(connection.id) match { - case Boundary ⇒ "DownstreamBoundary" - case owner ⇒ assembly.stages(owner).toString - } + private def inOwnerName(connection: Connection): String = connection.inOwner.toString // Debug name for a connections output part - private def outOwnerName(connection: Connection): String = - assembly.outOwners(connection.id) match { - case Boundary ⇒ "UpstreamBoundary" - case owner ⇒ assembly.stages(owner).toString - } + private def outOwnerName(connection: Connection): String = connection.outOwner.toString // Debug name for a connections input part - private def inLogicName(connection: Connection): String = - assembly.inOwners(connection.id) match { - case Boundary ⇒ "DownstreamBoundary" - case owner ⇒ logics(owner).toString - } + private def inLogicName(connection: Connection): String = logics(connection.inOwnerId).toString // Debug name for a connections output part - private def outLogicName(connection: Connection): String = - assembly.outOwners(connection.id) match { - case Boundary ⇒ "UpstreamBoundary" - case owner ⇒ logics(owner).toString - } + private def outLogicName(connection: Connection): String = logics(connection.outOwnerId).toString private def shutdownCounters: String = shutdownCounter.map(x ⇒ if (x >= KeepGoingFlag) s"${x & KeepGoingMask}(KeepGoing)" else x.toString).mkString(",") @@ -590,9 +351,10 @@ final class GraphInterpreter( def reportStageError(e: Throwable): Unit = { if (activeStage == null) throw e else { - val stage = assembly.stages(activeStage.stageId) + // TODO: Get error reporting back + //val stage = stages(activeStage.stageId) - log.error(e, "Error in stage [{}]: {}", stage, e.getMessage) + log.error(e, "Error in stage [{}]: {}", activeStage, e.getMessage) activeStage.failStage(e) // Abort chasing @@ -786,10 +548,8 @@ final class GraphInterpreter( // Register that a connection in which the given stage participated has been completed and therefore the stage // itself might stop, too. private def completeConnection(stageId: Int): Unit = { - if (stageId != Boundary) { - val activeConnections = shutdownCounter(stageId) - if (activeConnections > 0) shutdownCounter(stageId) = activeConnections - 1 - } + val activeConnections = shutdownCounter(stageId) + if (activeConnections > 0) shutdownCounter(stageId) = activeConnections - 1 } private[stream] def setKeepGoing(logic: GraphStageLogic, enabled: Boolean): Unit = @@ -802,7 +562,7 @@ final class GraphInterpreter( logic.afterPostStop() } catch { case NonFatal(e) ⇒ - log.error(e, s"Error during postStop in [{}]: {}", assembly.stages(logic.stageId), e.getMessage) + //log.error(e, s"Error during postStop in [{}]: {}", assembly.stages(logic.stageId), e.getMessage) } } @@ -877,39 +637,39 @@ final class GraphInterpreter( */ def dumpWaits(): Unit = println(toString) - override def toString: String = { - val builder = new StringBuilder("digraph waits {\n") - - for (i ← assembly.stages.indices) - builder.append(s"""N$i [label="${assembly.stages(i)}"]""" + "\n") - - def nameIn(port: Int): String = { - val owner = assembly.inOwners(port) - if (owner == Boundary) "Out" + port - else "N" + owner - } - - def nameOut(port: Int): String = { - val owner = assembly.outOwners(port) - if (owner == Boundary) "In" + port - else "N" + owner - } - - for (i ← connections.indices) { - connections(i).portState match { - case InReady ⇒ - builder.append(s""" ${nameIn(i)} -> ${nameOut(i)} [label=shouldPull; color=blue]""") - case OutReady ⇒ - builder.append(s""" ${nameOut(i)} -> ${nameIn(i)} [label=shouldPush; color=red];""") - case x if (x | InClosed | OutClosed) == (InClosed | OutClosed) ⇒ - builder.append(s""" ${nameIn(i)} -> ${nameOut(i)} [style=dotted; label=closed dir=both];""") - case _ ⇒ - } - builder.append("\n") - } - - builder.append("}\n") - builder.append(s"// $queueStatus (running=$runningStages, shutdown=${shutdownCounter.mkString(",")})") - builder.toString() - } + // override def toString: String = { + // val builder = new StringBuilder("digraph waits {\n") + // + // for (i ← assembly.stages.indices) + // builder.append(s"""N$i [label="${assembly.stages(i)}"]""" + "\n") + // + // def nameIn(port: Int): String = { + // val owner = assembly.inOwners(port) + // if (owner == Boundary) "Out" + port + // else "N" + owner + // } + // + // def nameOut(port: Int): String = { + // val owner = assembly.outOwners(port) + // if (owner == Boundary) "In" + port + // else "N" + owner + // } + // + // for (i ← connections.indices) { + // connections(i).portState match { + // case InReady ⇒ + // builder.append(s""" ${nameIn(i)} -> ${nameOut(i)} [label=shouldPull; color=blue]""") + // case OutReady ⇒ + // builder.append(s""" ${nameOut(i)} -> ${nameIn(i)} [label=shouldPush; color=red];""") + // case x if (x | InClosed | OutClosed) == (InClosed | OutClosed) ⇒ + // builder.append(s""" ${nameIn(i)} -> ${nameOut(i)} [style=dotted; label=closed dir=both];""") + // case _ ⇒ + // } + // builder.append("\n") + // } + // + // builder.append("}\n") + // builder.append(s"// $queueStatus (running=$runningStages, shutdown=${shutdownCounter.mkString(",")})") + // builder.toString() + // } } diff --git a/akka-stream/src/main/scala/akka/stream/impl/fusing/GraphStages.scala b/akka-stream/src/main/scala/akka/stream/impl/fusing/GraphStages.scala index cd60fdcabb..090d625b3d 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/fusing/GraphStages.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/fusing/GraphStages.scala @@ -10,7 +10,7 @@ import akka.actor.Cancellable import akka.dispatch.ExecutionContexts import akka.event.Logging import akka.stream.FlowMonitorState._ -import akka.stream._ +import akka.stream.{ Shape, _ } import akka.stream.scaladsl._ import akka.stream.impl.Stages.DefaultAttributes import akka.stream.stage._ @@ -18,27 +18,26 @@ import akka.stream.stage._ import scala.concurrent.{ Future, Promise } import scala.concurrent.duration.FiniteDuration import akka.stream.impl.StreamLayout._ -import akka.stream.impl.ReactiveStreamsCompliance +import akka.stream.impl.{ LinearTraversalBuilder, ReactiveStreamsCompliance } +import scala.annotation.unchecked.uncheckedVariance import scala.util.Try /** * INTERNAL API */ -final case class GraphStageModule( - shape: Shape, +// TODO: Fix variance issues +final case class GraphStageModule[+S <: Shape @uncheckedVariance, +M]( + shape: S, attributes: Attributes, - stage: GraphStageWithMaterializedValue[Shape, Any]) extends AtomicModule { - override def carbonCopy: Module = CopiedModule(shape.deepCopy(), Attributes.none, this) + stage: GraphStageWithMaterializedValue[S, M]) extends AtomicModule[S, M] { - override def replaceShape(s: Shape): Module = - if (s != shape) CompositeModule(this, s) - else this - - override def withAttributes(attributes: Attributes): Module = + override def withAttributes(attributes: Attributes): AtomicModule[S, M] = if (attributes ne this.attributes) new GraphStageModule(shape, attributes, stage) else this + override private[stream] def traversalBuilder = LinearTraversalBuilder.fromModule(this) + override def toString: String = f"GraphStage($stage) [${System.identityHashCode(this)}%08x]" } @@ -260,32 +259,6 @@ object GraphStages { override def toString: String = s"TickSource($initialDelay, $interval, $tick)" } - /** - * INTERNAL API. - * - * This source is not reusable, it is only created internally. - */ - final class MaterializedValueSource[T](val computation: MaterializedValueNode, val out: Outlet[T]) extends GraphStage[SourceShape[T]] { - def this(computation: MaterializedValueNode) = this(computation, Outlet[T]("matValue")) - override def initialAttributes: Attributes = DefaultAttributes.materializedValueSource - override val shape = SourceShape(out) - - private val promise = Promise[T] - def setValue(t: T): Unit = promise.success(t) - - def copySrc: MaterializedValueSource[T] = new MaterializedValueSource(computation, out) - - override def createLogic(attr: Attributes) = new GraphStageLogic(shape) { - setHandler(out, eagerTerminateOutput) - override def preStart(): Unit = { - val cb = getAsyncCallback[T](t ⇒ emit(out, t, () ⇒ completeStage())) - promise.future.foreach(cb.invoke)(ExecutionContexts.sameThreadExecutionContext) - } - } - - override def toString: String = s"MaterializedValueSource($computation)" - } - final class SingleSource[T](val elem: T) extends GraphStage[SourceShape[T]] { override def initialAttributes: Attributes = DefaultAttributes.singleSource ReactiveStreamsCompliance.requireNonNullElement(elem) diff --git a/akka-stream/src/main/scala/akka/stream/impl/fusing/IteratorInterpreter.scala b/akka-stream/src/main/scala/akka/stream/impl/fusing/IteratorInterpreter.scala index deeff7f97a..9a7c7fb670 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/fusing/IteratorInterpreter.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/fusing/IteratorInterpreter.scala @@ -5,7 +5,7 @@ package akka.stream.impl.fusing import akka.event.NoLogging import akka.stream._ -import akka.stream.impl.fusing.GraphInterpreter.{ GraphAssembly, DownstreamBoundaryStageLogic, UpstreamBoundaryStageLogic } +import akka.stream.impl.fusing.GraphInterpreter.{ DownstreamBoundaryStageLogic, UpstreamBoundaryStageLogic } import akka.stream.stage._ import java.{ util ⇒ ju } @@ -107,7 +107,6 @@ private[akka] class IteratorInterpreter[I, O]( private val downstream = IteratorDownstream[O]() private def init(): Unit = { - import GraphInterpreter.Boundary var i = 0 val length = stages.length @@ -119,9 +118,9 @@ private[akka] class IteratorInterpreter[I, O]( val stagesArray = Array.ofDim[GraphStageWithMaterializedValue[Shape, Any]](length) ins(length) = null - inOwners(length) = Boundary + inOwners(length) = length outs(0) = null - outOwners(0) = Boundary + outOwners(0) = 0 val stagesIterator = stages.iterator while (stagesIterator.hasNext) { @@ -133,22 +132,24 @@ private[akka] class IteratorInterpreter[I, O]( outOwners(i + 1) = i i += 1 } - val assembly = new GraphAssembly(stagesArray, attributes, ins, inOwners, outs, outOwners) - val (connections, logics) = - assembly.materialize(Attributes.none, assembly.stages.map(_.module), new ju.HashMap, _ ⇒ ()) - val interpreter = new GraphInterpreter( - assembly, - NoMaterializer, - NoLogging, - logics, - connections, - (_, _, _) ⇒ throw new UnsupportedOperationException("IteratorInterpreter does not support asynchronous events."), - fuzzingMode = false, - null) - interpreter.attachUpstreamBoundary(connections(0), upstream) - interpreter.attachDownstreamBoundary(connections(length), downstream) - interpreter.init(null) + // TODO: Fix this (assembly is gone) + // val assembly = new GraphAssembly(stagesArray, attributes, ins, inOwners, outs, outOwners) + // + // val (connections, logics) + // + // val interpreter = new GraphInterpreter( + // assembly, + // NoMaterializer, + // NoLogging, + // logics, + // connections, + // (_, _, _) ⇒ throw new UnsupportedOperationException("IteratorInterpreter does not support asynchronous events."), + // fuzzingMode = false, + // null) + // interpreter.attachUpstreamBoundary(connections(0), upstream) + // interpreter.attachDownstreamBoundary(connections(length), downstream) + // interpreter.init(null) } init() diff --git a/akka-stream/src/main/scala/akka/stream/impl/fusing/StreamOfStreams.scala b/akka-stream/src/main/scala/akka/stream/impl/fusing/StreamOfStreams.scala index 59ced97cbe..179d4d391c 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/fusing/StreamOfStreams.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/fusing/StreamOfStreams.scala @@ -643,29 +643,6 @@ final class SubSink[T](name: String, externalCallback: ActorSubscriberMessage override def toString: String = name } -object SubSource { - /** - * INTERNAL API - * - * HERE ACTUALLY ARE DRAGONS, YOU HAVE BEEN WARNED! - * - * FIXME #19240 - */ - private[akka] def kill[T, M](s: Source[T, M]): Unit = { - s.module match { - case GraphStageModule(_, _, stage: SubSource[_]) ⇒ - stage.externalCallback.invoke(SubSink.Cancel) - case pub: PublisherSource[_] ⇒ - pub.create(null)._1.subscribe(new CancellingSubscriber) - case m ⇒ - GraphInterpreter.currentInterpreterOrNull match { - case null ⇒ throw new UnsupportedOperationException(s"cannot drop Source of type ${m.getClass.getName}") - case intp ⇒ s.runWith(Sink.ignore)(intp.subFusingMaterializer) - } - } - } -} - /** * INTERNAL API */ diff --git a/akka-stream/src/main/scala/akka/stream/impl/io/IOSinks.scala b/akka-stream/src/main/scala/akka/stream/impl/io/IOSinks.scala index e90b1120f1..7e256cb3e9 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/io/IOSinks.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/io/IOSinks.scala @@ -8,7 +8,6 @@ import java.nio.file.{ Path, StandardOpenOption } import akka.stream._ import akka.stream.impl.SinkModule -import akka.stream.impl.StreamLayout.Module import akka.stream.impl.Stages.DefaultAttributes.IODispatcher import akka.stream.ActorAttributes.Dispatcher import akka.util.ByteString @@ -40,7 +39,7 @@ private[akka] final class FileSink(f: Path, options: Set[StandardOpenOption], va override protected def newInstance(shape: SinkShape[ByteString]): SinkModule[ByteString, Future[IOResult]] = new FileSink(f, options, attributes, shape) - override def withAttributes(attr: Attributes): Module = + override def withAttributes(attr: Attributes): SinkModule[ByteString, Future[IOResult]] = new FileSink(f, options, attr, amendShape(attr)) } @@ -67,6 +66,6 @@ private[akka] final class OutputStreamSink(createOutput: () ⇒ OutputStream, va override protected def newInstance(shape: SinkShape[ByteString]): SinkModule[ByteString, Future[IOResult]] = new OutputStreamSink(createOutput, attributes, shape, autoFlush) - override def withAttributes(attr: Attributes): Module = + override def withAttributes(attr: Attributes): SinkModule[ByteString, Future[IOResult]] = new OutputStreamSink(createOutput, attr, amendShape(attr), autoFlush) } diff --git a/akka-stream/src/main/scala/akka/stream/impl/io/IOSources.scala b/akka-stream/src/main/scala/akka/stream/impl/io/IOSources.scala index db93a76966..2c9698e626 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/io/IOSources.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/io/IOSources.scala @@ -9,7 +9,6 @@ import java.nio.file.Path import akka.stream._ import akka.stream.ActorAttributes.Dispatcher import akka.stream.IOResult -import akka.stream.impl.StreamLayout.Module import akka.stream.impl.Stages.DefaultAttributes.IODispatcher import akka.stream.impl.{ ErrorPublisher, SourceModule } import akka.util.ByteString @@ -40,7 +39,7 @@ private[akka] final class FileSource(f: Path, chunkSize: Int, val attributes: At override protected def newInstance(shape: SourceShape[ByteString]): SourceModule[ByteString, Future[IOResult]] = new FileSource(f, chunkSize, attributes, shape) - override def withAttributes(attr: Attributes): Module = + override def withAttributes(attr: Attributes): SourceModule[ByteString, Future[IOResult]] = new FileSource(f, chunkSize, attr, amendShape(attr)) override protected def label: String = s"FileSource($f, $chunkSize)" @@ -75,6 +74,6 @@ private[akka] final class InputStreamSource(createInputStream: () ⇒ InputStrea override protected def newInstance(shape: SourceShape[ByteString]): SourceModule[ByteString, Future[IOResult]] = new InputStreamSource(createInputStream, chunkSize, attributes, shape) - override def withAttributes(attr: Attributes): Module = + override def withAttributes(attr: Attributes): SourceModule[ByteString, Future[IOResult]] = new InputStreamSource(createInputStream, chunkSize, attr, amendShape(attr)) } diff --git a/akka-stream/src/main/scala/akka/stream/impl/io/TlsModule.scala b/akka-stream/src/main/scala/akka/stream/impl/io/TlsModule.scala index 21d787b4c3..a0362d1d03 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/io/TlsModule.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/io/TlsModule.scala @@ -2,10 +2,12 @@ package akka.stream.impl.io import javax.net.ssl.{ SSLContext, SSLEngine, SSLSession } +import akka.NotUsed import akka.actor.ActorSystem import akka.stream._ -import akka.stream.impl.StreamLayout.{ AtomicModule, CompositeModule } +import akka.stream.impl.StreamLayout.AtomicModule import akka.stream.TLSProtocol._ +import akka.stream.impl.TraversalBuilder import akka.util.ByteString import com.typesafe.sslconfig.akka.AkkaSSLConfig @@ -16,21 +18,18 @@ import scala.util.Try */ private[stream] final case class TlsModule(plainIn: Inlet[SslTlsOutbound], plainOut: Outlet[SslTlsInbound], cipherIn: Inlet[ByteString], cipherOut: Outlet[ByteString], - shape: Shape, attributes: Attributes, + shape: BidiShape[SslTlsOutbound, ByteString, ByteString, SslTlsInbound], + attributes: Attributes, createSSLEngine: ActorSystem ⇒ SSLEngine, // ActorSystem is only needed to support the AkkaSSLConfig legacy, see #21753 verifySession: (ActorSystem, SSLSession) ⇒ Try[Unit], // ActorSystem is only needed to support the AkkaSSLConfig legacy, see #21753 - closing: TLSClosing) extends AtomicModule { + closing: TLSClosing) + extends AtomicModule[BidiShape[SslTlsOutbound, ByteString, ByteString, SslTlsInbound], NotUsed] { override def withAttributes(att: Attributes): TlsModule = copy(attributes = att) - override def carbonCopy: TlsModule = TlsModule(attributes, createSSLEngine, verifySession, closing) - - override def replaceShape(s: Shape) = - if (s != shape) { - shape.requireSamePortsAs(s) - CompositeModule(this, s) - } else this override def toString: String = f"TlsModule($closing) [${System.identityHashCode(this)}%08x]" + + override private[stream] def traversalBuilder = TraversalBuilder.atomic(this) } /** diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/BidiFlow.scala b/akka-stream/src/main/scala/akka/stream/javadsl/BidiFlow.scala index 4c3b8911f6..e1788357fa 100644 --- a/akka-stream/src/main/scala/akka/stream/javadsl/BidiFlow.scala +++ b/akka-stream/src/main/scala/akka/stream/javadsl/BidiFlow.scala @@ -96,7 +96,7 @@ object BidiFlow { } final class BidiFlow[-I1, +O1, -I2, +O2, +Mat](delegate: scaladsl.BidiFlow[I1, O1, I2, O2, Mat]) extends Graph[BidiShape[I1, O1, I2, O2], Mat] { - override def module = delegate.module + override def traversalBuilder = delegate.traversalBuilder override def shape = delegate.shape def asScala: scaladsl.BidiFlow[I1, O1, I2, O2, Mat] = delegate diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/Flow.scala b/akka-stream/src/main/scala/akka/stream/javadsl/Flow.scala index 18f3e2ac94..2e4521a7f0 100644 --- a/akka-stream/src/main/scala/akka/stream/javadsl/Flow.scala +++ b/akka-stream/src/main/scala/akka/stream/javadsl/Flow.scala @@ -49,9 +49,8 @@ object Flow { */ def fromGraph[I, O, M](g: Graph[FlowShape[I, O], M]): Flow[I, O, M] = g match { - case f: Flow[I, O, M] ⇒ f - case f: scaladsl.Flow[I, O, M] if f.isIdentity ⇒ _identity.asInstanceOf[Flow[I, O, M]] - case other ⇒ new Flow(scaladsl.Flow.fromGraph(other)) + case f: Flow[I, O, M] ⇒ f + case other ⇒ new Flow(scaladsl.Flow.fromGraph(other)) } /** @@ -74,7 +73,7 @@ final class Flow[-In, +Out, +Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends import scala.collection.JavaConverters._ override def shape: FlowShape[In, Out] = delegate.shape - def module: StreamLayout.Module = delegate.module + override def traversalBuilder = delegate.traversalBuilder override def toString: String = delegate.toString @@ -2047,8 +2046,8 @@ object RunnableGraph { /** INTERNAL API */ private final class RunnableGraphAdapter[Mat](runnable: scaladsl.RunnableGraph[Mat]) extends RunnableGraph[Mat] { - def shape = ClosedShape - def module = runnable.module + override def shape = ClosedShape + override def traversalBuilder = runnable.traversalBuilder override def toString: String = runnable.toString @@ -2083,7 +2082,7 @@ abstract class RunnableGraph[+Mat] extends Graph[ClosedShape, Mat] { override def withAttributes(attr: Attributes): RunnableGraph[Mat] override def addAttributes(attr: Attributes): RunnableGraph[Mat] = - withAttributes(module.attributes and attr) + withAttributes(traversalBuilder.attributes and attr) override def named(name: String): RunnableGraph[Mat] = withAttributes(Attributes.name(name)) diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/Sink.scala b/akka-stream/src/main/scala/akka/stream/javadsl/Sink.scala index 9767bcdb3a..88cb76446e 100644 --- a/akka-stream/src/main/scala/akka/stream/javadsl/Sink.scala +++ b/akka-stream/src/main/scala/akka/stream/javadsl/Sink.scala @@ -4,17 +4,20 @@ package akka.stream.javadsl import java.util.Optional + import akka.{ Done, NotUsed } import akka.actor.{ ActorRef, Props } import akka.dispatch.ExecutionContexts import akka.japi.function -import akka.stream.impl.{ StreamLayout, SinkQueueAdapter } +import akka.stream.impl.{ LinearTraversalBuilder, SinkQueueAdapter, StreamLayout } import akka.stream.{ javadsl, scaladsl, _ } import org.reactivestreams.{ Publisher, Subscriber } + import scala.compat.java8.OptionConverters._ import scala.concurrent.ExecutionContext import scala.util.Try import java.util.concurrent.CompletionStage + import scala.compat.java8.FutureConverters._ /** Java API */ @@ -284,7 +287,7 @@ object Sink { final class Sink[-In, +Mat](delegate: scaladsl.Sink[In, Mat]) extends Graph[SinkShape[In], Mat] { override def shape: SinkShape[In] = delegate.shape - def module: StreamLayout.Module = delegate.module + override def traversalBuilder: LinearTraversalBuilder = delegate.traversalBuilder override def toString: String = delegate.toString diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/Source.scala b/akka-stream/src/main/scala/akka/stream/javadsl/Source.scala index 09aac730a3..97d1f9e013 100644 --- a/akka-stream/src/main/scala/akka/stream/javadsl/Source.scala +++ b/akka-stream/src/main/scala/akka/stream/javadsl/Source.scala @@ -5,13 +5,15 @@ package akka.stream.javadsl import java.util import java.util.Optional + import akka.{ Done, NotUsed } import akka.actor.{ ActorRef, Cancellable, Props } import akka.event.LoggingAdapter import akka.japi.{ Pair, Util, function } import akka.stream._ -import akka.stream.impl.{ ConstantFun, StreamLayout, SourceQueueAdapter } +import akka.stream.impl.{ ConstantFun, LinearTraversalBuilder, SourceQueueAdapter, StreamLayout } import org.reactivestreams.{ Publisher, Subscriber } + import scala.annotation.unchecked.uncheckedVariance import scala.collection.JavaConverters._ import scala.collection.immutable @@ -21,6 +23,7 @@ import scala.concurrent.{ Future, Promise } import scala.compat.java8.OptionConverters._ import java.util.concurrent.CompletionStage import java.util.concurrent.CompletableFuture + import scala.compat.java8.FutureConverters._ /** Java API */ @@ -423,7 +426,7 @@ final class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Grap override def shape: SourceShape[Out] = delegate.shape - def module: StreamLayout.Module = delegate.module + override def traversalBuilder: LinearTraversalBuilder = delegate.traversalBuilder override def toString: String = delegate.toString diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/BidiFlow.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/BidiFlow.scala index 2a5431db97..9d740bbd9f 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl/BidiFlow.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl/BidiFlow.scala @@ -4,14 +4,15 @@ package akka.stream.scaladsl import akka.NotUsed -import akka.stream._ -import akka.stream.impl.StreamLayout.Module -import akka.stream.impl.Timers +import akka.stream.{ BidiShape, _ } +import akka.stream.impl.{ LinearTraversalBuilder, Timers, TraversalBuilder } import scala.concurrent.duration.FiniteDuration -final class BidiFlow[-I1, +O1, -I2, +O2, +Mat](override val module: Module) extends Graph[BidiShape[I1, O1, I2, O2], Mat] { - override def shape = module.shape.asInstanceOf[BidiShape[I1, O1, I2, O2]] +final class BidiFlow[-I1, +O1, -I2, +O2, +Mat]( + override val traversalBuilder: TraversalBuilder, + override val shape: BidiShape[I1, O1, I2, O2] +) extends Graph[BidiShape[I1, O1, I2, O2], Mat] { def asJava: javadsl.BidiFlow[I1, O1, I2, O2, Mat] = new javadsl.BidiFlow(this) @@ -55,14 +56,14 @@ final class BidiFlow[-I1, +O1, -I2, +O2, +Mat](override val module: Module) exte * flow into the materialized value of the resulting BidiFlow. */ def atopMat[OO1, II2, Mat2, M](bidi: Graph[BidiShape[O1, OO1, II2, I2], Mat2])(combine: (Mat, Mat2) ⇒ M): BidiFlow[I1, OO1, II2, O2, M] = { - val copy = bidi.module.carbonCopy - val ins = copy.shape.inlets - val outs = copy.shape.outlets - new BidiFlow(module - .compose(copy, combine) - .wire(shape.out1, ins(0)) - .wire(outs(1), shape.in2) - .replaceShape(BidiShape(shape.in1, outs(0), ins(1), shape.out2))) + val newBidiShape = bidi.shape.deepCopy() + + new BidiFlow( + traversalBuilder.add(bidi.traversalBuilder, newBidiShape, combine) + .wire(shape.out1, newBidiShape.in1) + .wire(newBidiShape.out2, shape.in2), + BidiShape(shape.in1, newBidiShape.out1, newBidiShape.in2, shape.out2) + ) } /** @@ -105,26 +106,38 @@ final class BidiFlow[-I1, +O1, -I2, +O2, +Mat](override val module: Module) exte * flow into the materialized value of the resulting [[Flow]]. */ def joinMat[Mat2, M](flow: Graph[FlowShape[O1, I2], Mat2])(combine: (Mat, Mat2) ⇒ M): Flow[I1, O2, M] = { - val copy = flow.module.carbonCopy - val in = copy.shape.inlets.head - val out = copy.shape.outlets.head - new Flow(module - .compose(copy, combine) - .wire(shape.out1, in) - .wire(out, shape.in2) - .replaceShape(FlowShape(shape.in1, shape.out2))) + val newFlowShape = flow.shape.deepCopy() + + val resultBuilder = traversalBuilder + .add(flow.traversalBuilder, newFlowShape, combine) + .wire(shape.out1, newFlowShape.in) + .wire(newFlowShape.out, shape.in2) + + val newShape = FlowShape(shape.in1, shape.out2) + + new Flow( + LinearTraversalBuilder.fromBuilder(resultBuilder, newShape, Keep.right), + newShape + ) } /** * Turn this BidiFlow around by 180 degrees, logically flipping it upside down in a protocol stack. */ - def reversed: BidiFlow[I2, O2, I1, O1, Mat] = new BidiFlow(module.replaceShape(BidiShape(shape.in2, shape.out2, shape.in1, shape.out1))) + def reversed: BidiFlow[I2, O2, I1, O1, Mat] = + new BidiFlow( + traversalBuilder, + BidiShape(shape.in2, shape.out2, shape.in1, shape.out1) + ) /** * Transform only the materialized value of this BidiFlow, leaving all other properties as they were. */ def mapMaterializedValue[Mat2](f: Mat ⇒ Mat2): BidiFlow[I1, O1, I2, O2, Mat2] = - new BidiFlow(module.transformMaterializedValue(f.asInstanceOf[Any ⇒ Any])) + new BidiFlow( + traversalBuilder.transformMat(f.asInstanceOf[Any ⇒ Any]), + shape + ) /** * Change the attributes of this [[Source]] to the given ones and seal the list @@ -134,7 +147,10 @@ final class BidiFlow[-I1, +O1, -I2, +O2, +Mat](override val module: Module) exte * only to the contained processing stages). */ override def withAttributes(attr: Attributes): BidiFlow[I1, O1, I2, O2, Mat] = - new BidiFlow(module.withAttributes(attr)) + new BidiFlow( + traversalBuilder.setAttributes(attr), + shape + ) /** * Add the given attributes to this Source. Further calls to `withAttributes` @@ -143,7 +159,7 @@ final class BidiFlow[-I1, +O1, -I2, +O2, +Mat](override val module: Module) exte * only to the contained processing stages). */ override def addAttributes(attr: Attributes): BidiFlow[I1, O1, I2, O2, Mat] = - withAttributes(module.attributes and attr) + withAttributes(traversalBuilder.attributes and attr) /** * Add a ``name`` attribute to this Flow. @@ -169,7 +185,11 @@ object BidiFlow { graph match { case bidi: BidiFlow[I1, O1, I2, O2, Mat] ⇒ bidi case bidi: javadsl.BidiFlow[I1, O1, I2, O2, Mat] ⇒ bidi.asScala - case other ⇒ new BidiFlow(other.module) + case other ⇒ + new BidiFlow( + other.traversalBuilder, + other.shape + ) } /** @@ -193,9 +213,17 @@ object BidiFlow { */ def fromFlowsMat[I1, O1, I2, O2, M1, M2, M]( flow1: Graph[FlowShape[I1, O1], M1], - flow2: Graph[FlowShape[I2, O2], M2])(combine: (M1, M2) ⇒ M): BidiFlow[I1, O1, I2, O2, M] = - fromGraph(GraphDSL.create(flow1, flow2)(combine) { implicit b ⇒ (f1, f2) ⇒ BidiShape(f1.in, f1.out, f2.in, f2.out) - }) + flow2: Graph[FlowShape[I2, O2], M2])(combine: (M1, M2) ⇒ M): BidiFlow[I1, O1, I2, O2, M] = { + val newFlow1Shape = flow1.shape.deepCopy() + val newFlow2Shape = flow2.shape.deepCopy() + + new BidiFlow( + TraversalBuilder.empty() + .add(flow1.traversalBuilder, newFlow1Shape, Keep.right) + .add(flow2.traversalBuilder, newFlow2Shape, combine), + BidiShape(newFlow1Shape.in, newFlow1Shape.out, newFlow2Shape.in, newFlow2Shape.out) + ) + } /** * Wraps two Flows to create a ''BidiFlow''. The materialized value of the resulting BidiFlow is Unit. diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/FileIO.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/FileIO.scala index 854aff78d2..1f0bab0615 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl/FileIO.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl/FileIO.scala @@ -55,7 +55,7 @@ object FileIO { * @param chunkSize the size of each read operation, defaults to 8192 */ def fromPath(f: Path, chunkSize: Int = 8192): Source[ByteString, Future[IOResult]] = - new Source(new FileSource(f, chunkSize, DefaultAttributes.fileSource, sourceShape("FileSource"))) + Source.fromGraph(new FileSource(f, chunkSize, DefaultAttributes.fileSource, sourceShape("FileSource"))) /** * Creates a Sink which writes incoming [[ByteString]] elements to the given file. Overwrites existing files by default. @@ -86,5 +86,5 @@ object FileIO { * @param options File open options, defaults to Set(WRITE, CREATE) */ def toPath(f: Path, options: Set[StandardOpenOption] = Set(WRITE, CREATE)): Sink[ByteString, Future[IOResult]] = - new Sink(new FileSink(f, options, DefaultAttributes.fileSink, sinkShape("FileSink"))) + Sink.fromGraph(new FileSink(f, options, DefaultAttributes.fileSink, sinkShape("FileSink"))) } diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala index d20bbffc73..0bd4b83c8e 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala @@ -6,7 +6,6 @@ package akka.stream.scaladsl import akka.event.LoggingAdapter import akka.stream._ import akka.Done -import akka.stream.impl.StreamLayout.Module import akka.stream.impl._ import akka.stream.impl.fusing._ import akka.stream.stage._ @@ -24,12 +23,14 @@ import akka.annotation.DoNotInherit /** * A `Flow` is a set of stream processing steps that has one open input and one open output. */ -final class Flow[-In, +Out, +Mat](override val module: Module) +final class Flow[-In, +Out, +Mat]( + override val traversalBuilder: LinearTraversalBuilder, + override val shape: FlowShape[In, Out] +) extends FlowOpsMat[Out, Mat] with Graph[FlowShape[In, Out], Mat] { - override val shape: FlowShape[In, Out] = module.shape.asInstanceOf[FlowShape[In, Out]] - - override def toString: String = s"Flow($shape, $module)" + // TODO: debug string + override def toString: String = s"Flow($shape)" override type Repr[+O] = Flow[In @uncheckedVariance, O, Mat @uncheckedVariance] override type ReprMat[+O, +M] = Flow[In @uncheckedVariance, O, M] @@ -37,27 +38,23 @@ final class Flow[-In, +Out, +Mat](override val module: Module) override type Closed = Sink[In @uncheckedVariance, Mat @uncheckedVariance] override type ClosedMat[+M] = Sink[In @uncheckedVariance, M] - private[stream] def isIdentity: Boolean = this.module eq GraphStages.Identity.module + private[stream] def isIdentity: Boolean = this.traversalBuilder eq Flow.identityTraversalBuilder override def via[T, Mat2](flow: Graph[FlowShape[Out, T], Mat2]): Repr[T] = viaMat(flow)(Keep.left) - override def viaMat[T, Mat2, Mat3](flow: Graph[FlowShape[Out, T], Mat2])(combine: (Mat, Mat2) ⇒ Mat3): Flow[In, T, Mat3] = + override def viaMat[T, Mat2, Mat3](flow: Graph[FlowShape[Out, T], Mat2])(combine: (Mat, Mat2) ⇒ Mat3): Flow[In, T, Mat3] = { if (this.isIdentity) { - import Predef.Map.empty - import StreamLayout.{ CompositeModule, Ignore, IgnorableMatValComp, Transform, Atomic, Combine } - val m = flow.module - val mat = - if (combine == Keep.left) { - if (IgnorableMatValComp(m)) Ignore else Transform(_ ⇒ NotUsed, Atomic(m)) - } else Combine(combine.asInstanceOf[(Any, Any) ⇒ Any], Ignore, Atomic(m)) - new Flow(CompositeModule(Set(m), m.shape, empty, empty, mat, Attributes.none)) - } else { - val flowCopy = flow.module.carbonCopy new Flow( - module - .fuse(flowCopy, shape.out, flowCopy.shape.inlets.head, combine) - .replaceShape(FlowShape(shape.in, flowCopy.shape.outlets.head))) + LinearTraversalBuilder.fromBuilder(flow.traversalBuilder, flow.shape, combine), + flow.shape + ).asInstanceOf[Flow[In, T, Mat3]] + } else { + new Flow( + traversalBuilder.append(flow.traversalBuilder, flow.shape, combine), + FlowShape[In, T](shape.in, flow.shape.out) + ) } + } /** * Connect this [[Flow]] to a [[Sink]], concatenating the processing steps of both. @@ -98,15 +95,16 @@ final class Flow[-In, +Out, +Mat](override val module: Module) * where appropriate instead of manually writing functions that pass through one of the values. */ def toMat[Mat2, Mat3](sink: Graph[SinkShape[Out], Mat2])(combine: (Mat, Mat2) ⇒ Mat3): Sink[In, Mat3] = { - if (isIdentity) - Sink.fromGraph(sink.asInstanceOf[Graph[SinkShape[In], Mat2]]) - .mapMaterializedValue(combine(NotUsed.asInstanceOf[Mat], _)) - else { - val sinkCopy = sink.module.carbonCopy + if (isIdentity) { new Sink( - module - .fuse(sinkCopy, shape.out, sinkCopy.shape.inlets.head, combine) - .replaceShape(SinkShape(shape.in))) + LinearTraversalBuilder.fromBuilder(sink.traversalBuilder, sink.shape, combine), + SinkShape(sink.shape.in) + ).asInstanceOf[Sink[In, Mat3]] + } else { + new Sink( + traversalBuilder.append(sink.traversalBuilder, sink.shape, combine), + SinkShape(shape.in) + ) } } @@ -114,7 +112,10 @@ final class Flow[-In, +Out, +Mat](override val module: Module) * Transform the materialized value of this Flow, leaving all other properties as they were. */ override def mapMaterializedValue[Mat2](f: Mat ⇒ Mat2): ReprMat[Out, Mat2] = - new Flow(module.transformMaterializedValue(f.asInstanceOf[Any ⇒ Any])) + new Flow( + traversalBuilder.transformMat(f), + shape + ) /** * Join this [[Flow]] to another [[Flow]], by cross connecting the inputs and outputs, creating a [[RunnableGraph]]. @@ -147,12 +148,11 @@ final class Flow[-In, +Out, +Mat](override val module: Module) * where appropriate instead of manually writing functions that pass through one of the values. */ def joinMat[Mat2, Mat3](flow: Graph[FlowShape[Out, In], Mat2])(combine: (Mat, Mat2) ⇒ Mat3): RunnableGraph[Mat3] = { - val flowCopy = flow.module.carbonCopy - RunnableGraph( - module - .compose(flowCopy, combine) - .wire(shape.out, flowCopy.shape.inlets.head) - .wire(flowCopy.shape.outlets.head, shape.in)) + val resultBuilder = traversalBuilder + .append(flow.traversalBuilder, flow.shape, combine) + .wire(flow.shape.out, shape.in) + + RunnableGraph(resultBuilder) } /** @@ -194,14 +194,22 @@ final class Flow[-In, +Out, +Mat](override val module: Module) * where appropriate instead of manually writing functions that pass through one of the values. */ def joinMat[I2, O2, Mat2, M](bidi: Graph[BidiShape[Out, O2, I2, In], Mat2])(combine: (Mat, Mat2) ⇒ M): Flow[I2, O2, M] = { - val copy = bidi.module.carbonCopy - val ins = copy.shape.inlets - val outs = copy.shape.outlets - new Flow(module - .compose(copy, combine) - .wire(shape.out, ins.head) - .wire(outs(1), shape.in) - .replaceShape(FlowShape(ins(1), outs.head))) + val newBidiShape = bidi.shape.deepCopy() + val newFlowShape = shape.deepCopy() + + val resultBuilder = + TraversalBuilder.empty() + .add(traversalBuilder, newFlowShape, Keep.right) + .add(bidi.traversalBuilder, newBidiShape, combine) + .wire(newFlowShape.out, newBidiShape.in1) + .wire(newBidiShape.out2, newFlowShape.in) + + val newShape = FlowShape(newBidiShape.in2, newBidiShape.out1) + + new Flow( + LinearTraversalBuilder.fromBuilder(resultBuilder, newShape, Keep.right), + newShape + ) } /** @@ -212,8 +220,10 @@ final class Flow[-In, +Out, +Mat](override val module: Module) * only to the contained processing stages). */ override def withAttributes(attr: Attributes): Repr[Out] = - if (isIdentity) this - else new Flow(module.withAttributes(attr)) + new Flow( + traversalBuilder.setAttributes(attr), + shape + ) /** * Add the given attributes to this Flow. Further calls to `withAttributes` @@ -221,7 +231,7 @@ final class Flow[-In, +Out, +Mat](override val module: Module) * operation has no effect on an empty Flow (because the attributes apply * only to the contained processing stages). */ - override def addAttributes(attr: Attributes): Repr[Out] = withAttributes(module.attributes and attr) + override def addAttributes(attr: Attributes): Repr[Out] = withAttributes(traversalBuilder.attributes and attr) /** * Add a ``name`` attribute to this Flow. @@ -231,7 +241,12 @@ final class Flow[-In, +Out, +Mat](override val module: Module) /** * Put an asynchronous boundary around this `Flow` */ - override def async: Repr[Out] = addAttributes(Attributes.asyncBoundary) + override def async: Repr[Out] = { + new Flow( + traversalBuilder.makeIsland(GraphStageTag), + shape + ) + } /** * Connect the `Source` to this `Flow` and then connect it to the `Sink` and run it. The returned tuple contains @@ -265,7 +280,13 @@ final class Flow[-In, +Out, +Mat](override val module: Module) } object Flow { - private[this] val identity: Flow[Any, Any, NotUsed] = new Flow[Any, Any, NotUsed](GraphStages.Identity.module) + private[stream] val identityTraversalBuilder = + LinearTraversalBuilder.fromBuilder(GraphStages.Identity.traversalBuilder, GraphStages.Identity.shape, Keep.right) + + private[this] val identity: Flow[Any, Any, NotUsed] = new Flow[Any, Any, NotUsed]( + identityTraversalBuilder, + GraphStages.Identity.shape + ) /** * Creates a Flow from a Reactive Streams [[org.reactivestreams.Processor]] @@ -278,7 +299,7 @@ object Flow { * Creates a Flow from a Reactive Streams [[org.reactivestreams.Processor]] and returns a materialized value. */ def fromProcessorMat[I, O, M](processorFactory: () ⇒ (Processor[I, O], M)): Flow[I, O, M] = - new Flow(ProcessorModule(processorFactory)) + fromGraph(ProcessorModule(processorFactory)) /** * Returns a `Flow` which outputs all its inputs. @@ -299,7 +320,10 @@ object Flow { g match { case f: Flow[I, O, M] ⇒ f case f: javadsl.Flow[I, O, M] ⇒ f.asScala - case other ⇒ new Flow(other.module) + case other ⇒ new Flow( + LinearTraversalBuilder.fromBuilder(g.traversalBuilder, g.shape, Keep.right), + g.shape + ) } /** @@ -328,21 +352,20 @@ object RunnableGraph { def fromGraph[Mat](g: Graph[ClosedShape, Mat]): RunnableGraph[Mat] = g match { case r: RunnableGraph[Mat] ⇒ r - case other ⇒ RunnableGraph(other.module) + case other ⇒ RunnableGraph(other.traversalBuilder) } } /** * Flow with attached input and output, can be executed. */ -final case class RunnableGraph[+Mat](val module: StreamLayout.Module) extends Graph[ClosedShape, Mat] { - require(module.isRunnable) +final case class RunnableGraph[+Mat](override val traversalBuilder: TraversalBuilder) extends Graph[ClosedShape, Mat] { override def shape = ClosedShape /** * Transform only the materialized value of this RunnableGraph, leaving all other properties as they were. */ def mapMaterializedValue[Mat2](f: Mat ⇒ Mat2): RunnableGraph[Mat2] = - copy(module.transformMaterializedValue(f.asInstanceOf[Any ⇒ Any])) + copy(traversalBuilder.transformMat(f.asInstanceOf[Any ⇒ Any])) /** * Run this flow and return the materialized instance from the flow. @@ -350,16 +373,16 @@ final case class RunnableGraph[+Mat](val module: StreamLayout.Module) extends Gr def run()(implicit materializer: Materializer): Mat = materializer.materialize(this) override def addAttributes(attr: Attributes): RunnableGraph[Mat] = - withAttributes(module.attributes and attr) + withAttributes(traversalBuilder.attributes and attr) override def withAttributes(attr: Attributes): RunnableGraph[Mat] = - new RunnableGraph(module.withAttributes(attr)) + new RunnableGraph(traversalBuilder.setAttributes(attr)) override def named(name: String): RunnableGraph[Mat] = addAttributes(Attributes.name(name)) override def async: RunnableGraph[Mat] = - addAttributes(Attributes.asyncBoundary) + new RunnableGraph(traversalBuilder.makeIsland(GraphStageTag)) } /** diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/Graph.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/Graph.scala index a596e2b174..be5ae694d1 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl/Graph.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl/Graph.scala @@ -7,15 +7,14 @@ import akka.NotUsed import akka.stream._ import akka.stream.impl._ import akka.stream.impl.fusing.GraphStages -import akka.stream.impl.fusing.GraphStages.MaterializedValueSource import akka.stream.impl.Stages.DefaultAttributes -import akka.stream.impl.StreamLayout._ import akka.stream.scaladsl.Partition.PartitionOutOfBoundsException import akka.stream.stage.{ GraphStage, GraphStageLogic, InHandler, OutHandler } import scala.annotation.unchecked.uncheckedVariance import scala.annotation.tailrec import scala.collection.immutable +import scala.concurrent.Promise import scala.util.control.NoStackTrace object Merge { @@ -991,13 +990,13 @@ private[stream] final class OrElse[T] extends GraphStage[UniformFanInShape[T, T] object GraphDSL extends GraphApply { class Builder[+M] private[stream] () { - private var moduleInProgress: Module = EmptyModule + private var traversalBuilderInProgress: TraversalBuilder = TraversalBuilder.empty() /** * INTERNAL API */ private[GraphDSL] def addEdge[T, U >: T](from: Outlet[T], to: Inlet[U]): Unit = - moduleInProgress = moduleInProgress.wire(from, to) + traversalBuilderInProgress = traversalBuilderInProgress.wire(from, to) /** * Import a graph into this module, performing a deep copy, discarding its @@ -1005,10 +1004,9 @@ object GraphDSL extends GraphApply { * connected. */ def add[S <: Shape](graph: Graph[S, _]): S = { - if (StreamLayout.Debug) StreamLayout.validate(graph.module) - val copy = graph.module.carbonCopy - moduleInProgress = moduleInProgress.compose(copy, Keep.left) - graph.shape.copyFromPorts(copy.shape.inlets, copy.shape.outlets).asInstanceOf[S] + val newShape = graph.shape.deepCopy() + traversalBuilderInProgress = traversalBuilderInProgress.add(graph.traversalBuilder, newShape, Keep.left) + newShape.asInstanceOf[S] } /** @@ -1018,10 +1016,13 @@ object GraphDSL extends GraphApply { * Flow, Sink and Graph. */ private[stream] def add[S <: Shape, A](graph: Graph[S, _], transform: (A) ⇒ Any): S = { - if (StreamLayout.Debug) StreamLayout.validate(graph.module) - val copy = graph.module.carbonCopy - moduleInProgress = moduleInProgress.compose(copy.transformMaterializedValue(transform.asInstanceOf[Any ⇒ Any]), Keep.right) - graph.shape.copyFromPorts(copy.shape.inlets, copy.shape.outlets).asInstanceOf[S] + val newShape = graph.shape.deepCopy() + traversalBuilderInProgress = traversalBuilderInProgress.add( + graph.traversalBuilder.transformMat(transform), + newShape, + Keep.right + ) + newShape.asInstanceOf[S] } /** @@ -1031,10 +1032,9 @@ object GraphDSL extends GraphApply { * Flow, Sink and Graph. */ private[stream] def add[S <: Shape, A, B](graph: Graph[S, _], combine: (A, B) ⇒ Any): S = { - if (StreamLayout.Debug) StreamLayout.validate(graph.module) - val copy = graph.module.carbonCopy - moduleInProgress = moduleInProgress.compose(copy, combine) - graph.shape.copyFromPorts(copy.shape.inlets, copy.shape.outlets).asInstanceOf[S] + val newShape = graph.shape.deepCopy() + traversalBuilderInProgress = traversalBuilderInProgress.add(graph.traversalBuilder, newShape, combine) + newShape.asInstanceOf[S] } /** @@ -1052,26 +1052,19 @@ object GraphDSL extends GraphApply { * @return The outlet that will emit the materialized value. */ def materializedValue: Outlet[M @uncheckedVariance] = { - /* - * This brings the graph into a homogenous shape: if only one `add` has - * been performed so far, the moduleInProgress will be a CopiedModule - * that upon the next `composeNoMat` will be wrapped together with the - * MaterializedValueSource into a CompositeModule, leading to its - * relevant computation being an Atomic() for the CopiedModule. This is - * what we must reference, and we can only get this reference if we - * create that computation up-front: just making one up will not work - * because that computation node would not be part of the tree and - * the source would not be triggered. - */ - if (moduleInProgress.isInstanceOf[CopiedModule]) { - moduleInProgress = CompositeModule(moduleInProgress, moduleInProgress.shape) - } - val source = new MaterializedValueSource[M](moduleInProgress.materializedValueComputation) - moduleInProgress = moduleInProgress.composeNoMat(source.module) - source.out + val promise = Promise[M] + val source = Source.fromFuture(promise.future) + + traversalBuilderInProgress = traversalBuilderInProgress + .transformMat { mat: M ⇒ + promise.trySuccess(mat) + mat + } + .add(source.traversalBuilder, source.shape, Keep.left) + source.shape.out } - private[stream] def module: Module = moduleInProgress + private[stream] def traversalBuilder: TraversalBuilder = traversalBuilderInProgress /** Converts this Scala DSL element to it's Java DSL counterpart. */ def asJava: javadsl.GraphDSL.Builder[M] = new javadsl.GraphDSL.Builder()(this) @@ -1083,7 +1076,7 @@ object GraphDSL extends GraphApply { private[stream] def findOut[I, O](b: Builder[_], junction: UniformFanOutShape[I, O], n: Int): Outlet[O] = { if (n == junction.outArray.length) throw new IllegalArgumentException(s"no more outlets free on $junction") - else if (b.module.downstreams.contains(junction.out(n))) findOut(b, junction, n + 1) + else if (!b.traversalBuilder.isUnwired(junction.out(n))) findOut(b, junction, n + 1) else junction.out(n) } @@ -1091,7 +1084,7 @@ object GraphDSL extends GraphApply { private[stream] def findIn[I, O](b: Builder[_], junction: UniformFanInShape[I, O], n: Int): Inlet[I] = { if (n == junction.inSeq.length) throw new IllegalArgumentException(s"no more inlets free on $junction") - else if (b.module.upstreams.contains(junction.in(n))) findIn(b, junction, n + 1) + else if (!b.traversalBuilder.isUnwired(junction.in(n))) findIn(b, junction, n + 1) else junction.in(n) } @@ -1111,7 +1104,7 @@ object GraphDSL extends GraphApply { def bind(n: Int): Unit = { if (n == junction.inSeq.length) throw new IllegalArgumentException(s"no more inlets free on $junction") - else if (b.module.upstreams.contains(junction.in(n))) bind(n + 1) + else if (!b.traversalBuilder.isUnwired(junction.in(n))) bind(n + 1) else b.addEdge(importAndGetPort(b), junction.in(n)) } bind(0) @@ -1154,7 +1147,7 @@ object GraphDSL extends GraphApply { def bind(n: Int): Unit = { if (n == junction.outArray.length) throw new IllegalArgumentException(s"no more outlets free on $junction") - else if (b.module.downstreams.contains(junction.out(n))) bind(n + 1) + else if (!b.traversalBuilder.isUnwired(junction.out(n))) bind(n + 1) else b.addEdge(junction.out(n), importAndGetPortReverse(b)) } bind(0) diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala index 029d7b165b..240672b3fd 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala @@ -8,7 +8,6 @@ import akka.dispatch.ExecutionContexts import akka.actor.{ ActorRef, Props, Status } import akka.stream.actor.ActorSubscriber import akka.stream.impl.Stages.DefaultAttributes -import akka.stream.impl.StreamLayout.Module import akka.stream.impl._ import akka.stream.impl.fusing.GraphStages import akka.stream.stage.{ GraphStage, GraphStageLogic, InHandler, OutHandler } @@ -24,12 +23,14 @@ import scala.util.{ Failure, Success, Try } * A `Sink` is a set of stream processing steps that has one open input. * Can be used as a `Subscriber` */ -final class Sink[-In, +Mat](override val module: Module) +final class Sink[-In, +Mat]( + override val traversalBuilder: LinearTraversalBuilder, + override val shape: SinkShape[In] +) extends Graph[SinkShape[In], Mat] { - override val shape: SinkShape[In] = module.shape.asInstanceOf[SinkShape[In]] - - override def toString: String = s"Sink($shape, $module)" + // TODO: Debug string + override def toString: String = s"Sink($shape)" /** * Transform this Sink by applying a function to each *incoming* upstream element before @@ -52,7 +53,10 @@ final class Sink[-In, +Mat](override val module: Module) * Transform only the materialized value of this Sink, leaving all other properties as they were. */ def mapMaterializedValue[Mat2](f: Mat ⇒ Mat2): Sink[In, Mat2] = - new Sink(module.transformMaterializedValue(f.asInstanceOf[Any ⇒ Any])) + new Sink( + traversalBuilder.transformMat(f.asInstanceOf[Any ⇒ Any]), + shape + ) /** * Change the attributes of this [[Sink]] to the given ones and seal the list @@ -62,7 +66,10 @@ final class Sink[-In, +Mat](override val module: Module) * only to the contained processing stages). */ override def withAttributes(attr: Attributes): Sink[In, Mat] = - new Sink(module.withAttributes(attr)) + new Sink( + traversalBuilder.setAttributes(attr), + shape + ) /** * Add the given attributes to this Sink. Further calls to `withAttributes` @@ -71,7 +78,7 @@ final class Sink[-In, +Mat](override val module: Module) * only to the contained processing stages). */ override def addAttributes(attr: Attributes): Sink[In, Mat] = - withAttributes(module.attributes and attr) + withAttributes(traversalBuilder.attributes and attr) /** * Add a ``name`` attribute to this Sink. @@ -81,7 +88,11 @@ final class Sink[-In, +Mat](override val module: Module) /** * Put an asynchronous boundary around this `Sink` */ - override def async: Sink[In, Mat] = addAttributes(Attributes.asyncBoundary) + override def async: Sink[In, Mat] = + new Sink( + traversalBuilder.makeIsland(GraphStageTag), + shape + ) /** * Converts this Scala DSL element to it's Java DSL counterpart. @@ -102,20 +113,23 @@ object Sink { g match { case s: Sink[T, M] ⇒ s case s: javadsl.Sink[T, M] ⇒ s.asScala - case other ⇒ new Sink(other.module) + case other ⇒ new Sink( + LinearTraversalBuilder.fromBuilder(other.traversalBuilder, other.shape, Keep.right), + other.shape + ) } /** * Helper to create [[Sink]] from `Subscriber`. */ def fromSubscriber[T](subscriber: Subscriber[T]): Sink[T, NotUsed] = - new Sink(new SubscriberSink(subscriber, DefaultAttributes.subscriberSink, shape("SubscriberSink"))) + fromGraph(new SubscriberSink(subscriber, DefaultAttributes.subscriberSink, shape("SubscriberSink"))) /** * A `Sink` that immediately cancels its upstream after materialization. */ def cancelled[T]: Sink[T, NotUsed] = - new Sink[Any, NotUsed](new CancelSink(DefaultAttributes.cancelledSink, shape("CancelledSink"))) + fromGraph[Any, NotUsed](new CancelSink(DefaultAttributes.cancelledSink, shape("CancelledSink"))) /** * A `Sink` that materializes into a `Future` of the first value received. @@ -181,7 +195,7 @@ object Sink { * reject any additional `Subscriber`s. */ def asPublisher[T](fanout: Boolean): Sink[T, Publisher[T]] = - new Sink( + fromGraph( if (fanout) new FanoutPublisherSink[T](DefaultAttributes.fanoutPublisherSink, shape("FanoutPublisherSink")) else new PublisherSink[T](DefaultAttributes.publisherSink, shape("PublisherSink"))) @@ -327,7 +341,7 @@ object Sink { * limiting stage in front of this `Sink`. */ def actorRef[T](ref: ActorRef, onCompleteMessage: Any): Sink[T, NotUsed] = - new Sink(new ActorRefSink(ref, onCompleteMessage, DefaultAttributes.actorRefSink, shape("ActorRefSink"))) + fromGraph(new ActorRefSink(ref, onCompleteMessage, DefaultAttributes.actorRefSink, shape("ActorRefSink"))) /** * Sends the elements of the stream to the given `ActorRef` that sends back back-pressure signal. @@ -356,7 +370,7 @@ object Sink { @deprecated("Use `akka.stream.stage.GraphStage` and `fromGraph` instead, it allows for all operations an Actor would and is more type-safe as well as guaranteed to be ReactiveStreams compliant.", since = "2.5.0") def actorSubscriber[T](props: Props): Sink[T, ActorRef] = { require(classOf[ActorSubscriber].isAssignableFrom(props.actorClass()), "Actor must be ActorSubscriber") - new Sink(new ActorSubscriberSink(props, DefaultAttributes.actorSubscriberSink, shape("ActorSubscriberSink"))) + fromGraph(new ActorSubscriberSink(props, DefaultAttributes.actorSubscriberSink, shape("ActorSubscriberSink"))) } /** diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala index 930b7fda25..ac43033479 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala @@ -7,18 +7,19 @@ import akka.stream.impl.Stages.DefaultAttributes import akka.{ Done, NotUsed } import akka.actor.{ ActorRef, Cancellable, Props } import akka.stream.actor.ActorPublisher -import akka.stream.impl.StreamLayout.Module import akka.stream.impl.fusing.GraphStages import akka.stream.impl.fusing.GraphStages._ -import akka.stream.impl.{ EmptyPublisher, ErrorPublisher, _ } +import akka.stream.impl.{ EmptyPublisher, ErrorPublisher, PublisherSource, _ } import akka.stream.{ Outlet, SourceShape, _ } import org.reactivestreams.{ Publisher, Subscriber } + import scala.annotation.tailrec import scala.annotation.unchecked.uncheckedVariance import scala.collection.immutable import scala.concurrent.duration.FiniteDuration import scala.concurrent.{ Future, Promise } import java.util.concurrent.CompletionStage + import scala.compat.java8.FutureConverters._ /** @@ -27,7 +28,10 @@ import scala.compat.java8.FutureConverters._ * an “atomic” source, e.g. from a collection or a file. Materialization turns a Source into * a Reactive Streams `Publisher` (at least conceptually). */ -final class Source[+Out, +Mat](override val module: Module) +final class Source[+Out, +Mat]( + override val traversalBuilder: LinearTraversalBuilder, + override val shape: SourceShape[Out] +) extends FlowOpsMat[Out, Mat] with Graph[SourceShape[Out], Mat] { override type Repr[+O] = Source[O, Mat @uncheckedVariance] @@ -36,26 +40,18 @@ final class Source[+Out, +Mat](override val module: Module) override type Closed = RunnableGraph[Mat @uncheckedVariance] override type ClosedMat[+M] = RunnableGraph[M] - override val shape: SourceShape[Out] = module.shape.asInstanceOf[SourceShape[Out]] - - override def toString: String = s"Source($shape, $module)" + override def toString: String = s"Source($shape)" override def via[T, Mat2](flow: Graph[FlowShape[Out, T], Mat2]): Repr[T] = viaMat(flow)(Keep.left) override def viaMat[T, Mat2, Mat3](flow: Graph[FlowShape[Out, T], Mat2])(combine: (Mat, Mat2) ⇒ Mat3): Source[T, Mat3] = { - if (flow.module eq GraphStages.Identity.module) { - if (combine eq Keep.left) - this.asInstanceOf[Source[T, Mat3]] - else if (combine eq Keep.right) - this.mapMaterializedValue((_) ⇒ NotUsed).asInstanceOf[Source[T, Mat3]] - else - this.mapMaterializedValue(combine(_, NotUsed.asInstanceOf[Mat2])).asInstanceOf[Source[T, Mat3]] + if (flow.traversalBuilder eq Flow.identityTraversalBuilder) { + traversalBuilder.append(LinearTraversalBuilder.empty(), combine).asInstanceOf[Source[T, Mat3]] } else { - val flowCopy = flow.module.carbonCopy - new Source( - module - .fuse(flowCopy, shape.out, flowCopy.shape.inlets.head, combine) - .replaceShape(SourceShape(flowCopy.shape.outlets.head))) + new Source[T, Mat3]( + traversalBuilder.append(flow.traversalBuilder, flow.shape, combine), + SourceShape(flow.shape.out) + ) } } @@ -70,15 +66,14 @@ final class Source[+Out, +Mat](override val module: Module) * concatenating the processing steps of both. */ def toMat[Mat2, Mat3](sink: Graph[SinkShape[Out], Mat2])(combine: (Mat, Mat2) ⇒ Mat3): RunnableGraph[Mat3] = { - val sinkCopy = sink.module.carbonCopy - RunnableGraph(module.fuse(sinkCopy, shape.out, sinkCopy.shape.inlets.head, combine)) + RunnableGraph(traversalBuilder.append(sink.traversalBuilder, sink.shape, combine)) } /** * Transform only the materialized value of this Source, leaving all other properties as they were. */ override def mapMaterializedValue[Mat2](f: Mat ⇒ Mat2): ReprMat[Out, Mat2] = - new Source[Out, Mat2](module.transformMaterializedValue(f.asInstanceOf[Any ⇒ Any])) + new Source[Out, Mat2](traversalBuilder.transformMat(f.asInstanceOf[Any ⇒ Any]), shape) /** * Connect this `Source` to a `Sink` and run it. The returned value is the materialized value @@ -140,7 +135,7 @@ final class Source[+Out, +Mat](override val module: Module) * only to the contained processing stages). */ override def withAttributes(attr: Attributes): Repr[Out] = - new Source(module.withAttributes(attr)) + new Source(traversalBuilder.setAttributes(attr), shape) /** * Add the given attributes to this Source. Further calls to `withAttributes` @@ -148,7 +143,7 @@ final class Source[+Out, +Mat](override val module: Module) * operation has no effect on an empty Flow (because the attributes apply * only to the contained processing stages). */ - override def addAttributes(attr: Attributes): Repr[Out] = withAttributes(module.attributes and attr) + override def addAttributes(attr: Attributes): Repr[Out] = withAttributes(traversalBuilder.attributes and attr) /** * Add a ``name`` attribute to this Source. @@ -158,7 +153,10 @@ final class Source[+Out, +Mat](override val module: Module) /** * Put an asynchronous boundary around this `Source` */ - override def async: Repr[Out] = addAttributes(Attributes.asyncBoundary) + override def async: Repr[Out] = new Source( + traversalBuilder.makeIsland(GraphStageTag), + shape + ) /** * Converts this Scala DSL element to it's Java DSL counterpart. @@ -198,7 +196,7 @@ object Source { * back-pressure upstream. */ def fromPublisher[T](publisher: Publisher[T]): Source[T, NotUsed] = - new Source(new PublisherSource(publisher, DefaultAttributes.publisherSource, shape("PublisherSource"))) + fromGraph(new PublisherSource(publisher, DefaultAttributes.publisherSource, shape("PublisherSource"))) /** * Helper to create [[Source]] from `Iterator`. @@ -234,7 +232,10 @@ object Source { def fromGraph[T, M](g: Graph[SourceShape[T], M]): Source[T, M] = g match { case s: Source[T, M] ⇒ s case s: javadsl.Source[T, M] ⇒ s.asScala - case other ⇒ new Source(other.module) + case other ⇒ new Source( + LinearTraversalBuilder.fromBuilder(other.traversalBuilder, other.shape, Keep.right), + other.shape + ) } /** @@ -331,12 +332,10 @@ object Source { */ def empty[T]: Source[T, NotUsed] = _empty private[this] val _empty: Source[Nothing, NotUsed] = - new Source( - new PublisherSource[Nothing]( - EmptyPublisher, - DefaultAttributes.emptySource, - shape("EmptySource"))) - + fromGraph(new PublisherSource[Nothing]( + EmptyPublisher, + DefaultAttributes.emptySource, + shape("EmptySource"))) /** * Create a `Source` which materializes a [[scala.concurrent.Promise]] which controls what element * will be emitted by the Source. @@ -349,17 +348,16 @@ object Source { * with None. */ def maybe[T]: Source[T, Promise[Option[T]]] = - new Source(new MaybeSource[T](DefaultAttributes.maybeSource, shape("MaybeSource"))) + fromGraph(new MaybeSource[T](DefaultAttributes.maybeSource, shape("MaybeSource"))) /** * Create a `Source` that immediately ends the stream with the `cause` error to every connected `Sink`. */ def failed[T](cause: Throwable): Source[T, NotUsed] = - new Source( - new PublisherSource( - ErrorPublisher(cause, "FailedSource")[T], - DefaultAttributes.failedSource, - shape("FailedSource"))) + fromGraph(new PublisherSource( + ErrorPublisher(cause, "FailedSource")[T], + DefaultAttributes.failedSource, + shape("FailedSource"))) /** * Creates a `Source` that is not materialized until there is downstream demand, when the source gets materialized @@ -373,7 +371,7 @@ object Source { * Creates a `Source` that is materialized as a [[org.reactivestreams.Subscriber]] */ def asSubscriber[T]: Source[T, Subscriber[T]] = - new Source(new SubscriberSource[T](DefaultAttributes.subscriberSource, shape("SubscriberSource"))) + fromGraph(new SubscriberSource[T](DefaultAttributes.subscriberSource, shape("SubscriberSource"))) /** * Creates a `Source` that is materialized to an [[akka.actor.ActorRef]] which points to an Actor @@ -385,7 +383,7 @@ object Source { @deprecated("Use `akka.stream.stage.GraphStage` and `fromGraph` instead, it allows for all operations an Actor would and is more type-safe as well as guaranteed to be ReactiveStreams compliant.", since = "2.5.0") def actorPublisher[T](props: Props): Source[T, ActorRef] = { require(classOf[ActorPublisher[_]].isAssignableFrom(props.actorClass()), "Actor must be ActorPublisher") - new Source(new ActorPublisherSource(props, DefaultAttributes.actorPublisherSource, shape("ActorPublisherSource"))) + fromGraph(new ActorPublisherSource(props, DefaultAttributes.actorPublisherSource, shape("ActorPublisherSource"))) } /** @@ -423,7 +421,7 @@ object Source { def actorRef[T](bufferSize: Int, overflowStrategy: OverflowStrategy): Source[T, ActorRef] = { require(bufferSize >= 0, "bufferSize must be greater than or equal to 0") require(overflowStrategy != OverflowStrategies.Backpressure, "Backpressure overflowStrategy not supported") - new Source(new ActorRefSource(bufferSize, overflowStrategy, DefaultAttributes.actorRefSource, shape("ActorRefSource"))) + fromGraph(new ActorRefSource(bufferSize, overflowStrategy, DefaultAttributes.actorRefSource, shape("ActorRefSource"))) } /** diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/StreamConverters.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/StreamConverters.scala index 478eeb5c10..de5f39a9bf 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl/StreamConverters.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl/StreamConverters.scala @@ -44,7 +44,7 @@ object StreamConverters { * @param chunkSize the size of each read operation, defaults to 8192 */ def fromInputStream(in: () ⇒ InputStream, chunkSize: Int = 8192): Source[ByteString, Future[IOResult]] = - new Source(new InputStreamSource(in, chunkSize, DefaultAttributes.inputStreamSource, sourceShape("InputStreamSource"))) + Source.fromGraph(new InputStreamSource(in, chunkSize, DefaultAttributes.inputStreamSource, sourceShape("InputStreamSource"))) /** * Creates a Source which when materialized will return an [[OutputStream]] which it is possible @@ -77,7 +77,7 @@ object StreamConverters { * will cancel the stream when the [[OutputStream]] is no longer writable. */ def fromOutputStream(out: () ⇒ OutputStream, autoFlush: Boolean = false): Sink[ByteString, Future[IOResult]] = - new Sink(new OutputStreamSink(out, DefaultAttributes.outputStreamSink, sinkShape("OutputStreamSink"), autoFlush)) + Sink.fromGraph(new OutputStreamSink(out, DefaultAttributes.outputStreamSink, sinkShape("OutputStreamSink"), autoFlush)) /** * Creates a Sink which when materialized will return an [[InputStream]] which it is possible diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/TLS.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/TLS.scala index 50723a31e9..7955bedd3a 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl/TLS.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl/TLS.scala @@ -111,7 +111,7 @@ object TLS { case None ⇒ (_, _) ⇒ Success(()) } - new scaladsl.BidiFlow(TlsModule(Attributes.none, createSSLEngine, verifySession, closing)) + scaladsl.BidiFlow.fromGraph(TlsModule(Attributes.none, createSSLEngine, verifySession, closing)) } /** @@ -166,7 +166,7 @@ object TLS { verifySession: SSLSession ⇒ Try[Unit], // we don't offer the internal API that provides `ActorSystem` here, see #21753 closing: TLSClosing ): scaladsl.BidiFlow[SslTlsOutbound, ByteString, ByteString, SslTlsInbound, NotUsed] = - new scaladsl.BidiFlow(TlsModule(Attributes.none, _ ⇒ createSSLEngine(), (_, session) ⇒ verifySession(session), closing)) + scaladsl.BidiFlow.fromGraph(TlsModule(Attributes.none, _ ⇒ createSSLEngine(), (_, session) ⇒ verifySession(session), closing)) /** * Create a StreamTls [[akka.stream.scaladsl.BidiFlow]]. This is a low-level interface. diff --git a/akka-stream/src/main/scala/akka/stream/stage/GraphStage.scala b/akka-stream/src/main/scala/akka/stream/stage/GraphStage.scala index 4ae5352a51..37e987c09a 100644 --- a/akka-stream/src/main/scala/akka/stream/stage/GraphStage.scala +++ b/akka-stream/src/main/scala/akka/stream/stage/GraphStage.scala @@ -12,9 +12,9 @@ import akka.actor._ import akka.annotation.ApiMayChange import akka.japi.function.{ Effect, Procedure } import akka.stream._ -import akka.stream.impl.StreamLayout.Module +import akka.stream.impl.StreamLayout.AtomicModule import akka.stream.impl.fusing.{ GraphInterpreter, GraphStageModule, SubSink, SubSource } -import akka.stream.impl.ReactiveStreamsCompliance +import akka.stream.impl.{ ReactiveStreamsCompliance, TraversalBuilder } import scala.collection.mutable.ArrayBuffer import scala.collection.{ immutable, mutable } @@ -28,11 +28,11 @@ abstract class GraphStageWithMaterializedValue[+S <: Shape, +M] extends Graph[S, protected def initialAttributes: Attributes = Attributes.none - final override lazy val module: Module = GraphStageModule(shape, initialAttributes, this) + final override lazy val traversalBuilder: TraversalBuilder = TraversalBuilder.atomic(GraphStageModule(shape, initialAttributes, this)) final override def withAttributes(attr: Attributes): Graph[S, M] = new Graph[S, M] { override def shape = GraphStageWithMaterializedValue.this.shape - override def module = GraphStageWithMaterializedValue.this.module.withAttributes(attr) + override def traversalBuilder = GraphStageWithMaterializedValue.this.traversalBuilder.setAttributes(attr) override def withAttributes(attr: Attributes) = GraphStageWithMaterializedValue.this.withAttributes(attr) }