diff --git a/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala index 3dc5621f17..ae4141065d 100644 --- a/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala @@ -10,11 +10,6 @@ import java.nio.ByteOrder import java.util.concurrent.CountDownLatch import java.util.concurrent.TimeUnit -import scala.annotation.tailrec -import scala.concurrent.Await -import scala.concurrent.Future -import scala.concurrent.duration._ - import akka.Done import akka.NotUsed import akka.actor._ @@ -24,14 +19,18 @@ import akka.remote.artery.compress._ import akka.serialization.BaseSerializer import akka.serialization.ByteBufferSerializer import akka.serialization.SerializationExtension -import akka.stream.ActorMaterializer -import akka.stream.ActorMaterializerSettings +import akka.stream.SystemMaterializer import akka.stream.scaladsl._ import akka.util.OptionVal import com.github.ghik.silencer.silent import com.typesafe.config.ConfigFactory import org.openjdk.jmh.annotations._ +import scala.annotation.tailrec +import scala.concurrent.Await +import scala.concurrent.Future +import scala.concurrent.duration._ + @State(Scope.Benchmark) @OutputTimeUnit(TimeUnit.MILLISECONDS) @BenchmarkMode(Array(Mode.Throughput)) @@ -45,7 +44,7 @@ class CodecBenchmark { @Param(Array(Standard, RemoteInstrument)) private var configType: String = _ - var system: ActorSystem = _ + implicit var system: ActorSystem = _ var systemB: ActorSystem = _ private val envelopePool = new EnvelopeBufferPool(1024 * 1024, 128) @@ -70,7 +69,6 @@ class CodecBenchmark { override def publishDropped(inbound: InboundEnvelope, reason: String): Unit = () } - @silent("never used") private var materializer: ActorMaterializer = _ @silent("never used") private var remoteRefB: RemoteActorRef = _ @silent("never used") private var resolvedRef: InternalActorRef = _ @silent("never used") private var senderStringA: String = _ @@ -106,8 +104,8 @@ class CodecBenchmark { system = ActorSystem("CodecBenchmark", config) systemB = ActorSystem("systemB", system.settings.config) - val settings = ActorMaterializerSettings(system) - materializer = ActorMaterializer(settings)(system) + // eager init of materializer + SystemMaterializer(system).materializer uniqueLocalAddress = UniqueAddress( system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress, @@ -205,7 +203,7 @@ class CodecBenchmark { val latch = new CountDownLatch(1) val N = OperationsPerInvocation - Source.fromGraph(new BenchTestSourceSameElement(N, "elem")).runWith(new LatchSink(N, latch))(materializer) + Source.fromGraph(new BenchTestSourceSameElement(N, "elem")).runWith(new LatchSink(N, latch)) if (!latch.await(30, TimeUnit.SECONDS)) throw new RuntimeException("Latch didn't complete in time") @@ -217,10 +215,7 @@ class CodecBenchmark { val latch = new CountDownLatch(1) val N = OperationsPerInvocation - Source - .fromGraph(new BenchTestSourceSameElement(N, "elem")) - .via(encodeGraph) - .runWith(new LatchSink(N, latch))(materializer) + Source.fromGraph(new BenchTestSourceSameElement(N, "elem")).via(encodeGraph).runWith(new LatchSink(N, latch)) if (!latch.await(30, TimeUnit.SECONDS)) throw new RuntimeException("Latch didn't complete in time") @@ -232,10 +227,7 @@ class CodecBenchmark { val latch = new CountDownLatch(1) val N = OperationsPerInvocation - Source - .fromGraph(new BenchTestSourceSameElement(N, "elem")) - .via(decodeGraph) - .runWith(new LatchSink(N, latch))(materializer) + Source.fromGraph(new BenchTestSourceSameElement(N, "elem")).via(decodeGraph).runWith(new LatchSink(N, latch)) if (!latch.await(30, TimeUnit.SECONDS)) throw new RuntimeException("Latch didn't complete in time") @@ -247,10 +239,7 @@ class CodecBenchmark { val latch = new CountDownLatch(1) val N = OperationsPerInvocation - Source - .fromGraph(new BenchTestSourceSameElement(N, "elem")) - .via(encodeDecodeGraph) - .runWith(new LatchSink(N, latch))(materializer) + Source.fromGraph(new BenchTestSourceSameElement(N, "elem")).via(encodeDecodeGraph).runWith(new LatchSink(N, latch)) if (!latch.await(30, TimeUnit.SECONDS)) throw new RuntimeException("Latch didn't complete in time") diff --git a/akka-bench-jmh/src/main/scala/akka/remote/artery/SendQueueBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/remote/artery/SendQueueBenchmark.scala index aef6deeedb..3801d9b7a4 100644 --- a/akka-bench-jmh/src/main/scala/akka/remote/artery/SendQueueBenchmark.scala +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/SendQueueBenchmark.scala @@ -4,20 +4,21 @@ package akka.remote.artery +import java.util.concurrent.CountDownLatch +import java.util.concurrent.CyclicBarrier import java.util.concurrent.TimeUnit + import akka.actor.ActorSystem +import akka.stream.KillSwitches +import akka.stream.OverflowStrategy +import akka.stream.SystemMaterializer import akka.stream.scaladsl._ import com.typesafe.config.ConfigFactory +import org.agrona.concurrent.ManyToOneConcurrentArrayQueue import org.openjdk.jmh.annotations._ + import scala.concurrent.Await import scala.concurrent.duration._ -import akka.stream.ActorMaterializer -import akka.stream.ActorMaterializerSettings -import akka.stream.OverflowStrategy -import java.util.concurrent.CyclicBarrier -import java.util.concurrent.CountDownLatch -import akka.stream.KillSwitches -import org.agrona.concurrent.ManyToOneConcurrentArrayQueue @State(Scope.Benchmark) @OutputTimeUnit(TimeUnit.MILLISECONDS) @@ -32,12 +33,10 @@ class SendQueueBenchmark { implicit val system = ActorSystem("SendQueueBenchmark", config) - var materializer: ActorMaterializer = _ - @Setup def setup(): Unit = { - val settings = ActorMaterializerSettings(system) - materializer = ActorMaterializer(settings) + // eager init of materializer + SystemMaterializer(system).materializer } @TearDown @@ -58,7 +57,7 @@ class SendQueueBenchmark { val (queue, killSwitch) = source .viaMat(KillSwitches.single)(Keep.both) .toMat(new BarrierSink(N, latch, burstSize, barrier))(Keep.left) - .run()(materializer) + .run() var n = 1 while (n <= N) { @@ -87,7 +86,7 @@ class SendQueueBenchmark { val (ref, killSwitch) = source .viaMat(KillSwitches.single)(Keep.both) .toMat(new BarrierSink(N, latch, burstSize, barrier))(Keep.left) - .run()(materializer) + .run() var n = 1 while (n <= N) { @@ -117,7 +116,7 @@ class SendQueueBenchmark { val (sendQueue, killSwitch) = source .viaMat(KillSwitches.single)(Keep.both) .toMat(new BarrierSink(N, latch, burstSize, barrier))(Keep.left) - .run()(materializer) + .run() sendQueue.inject(queue) var n = 1 diff --git a/akka-bench-jmh/src/main/scala/akka/stream/AskBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/stream/AskBenchmark.scala index afb178afa2..d738424258 100644 --- a/akka-bench-jmh/src/main/scala/akka/stream/AskBenchmark.scala +++ b/akka-bench-jmh/src/main/scala/akka/stream/AskBenchmark.scala @@ -4,19 +4,24 @@ package akka.stream -import java.util.concurrent.{ CountDownLatch, TimeUnit } +import java.util.concurrent.CountDownLatch +import java.util.concurrent.TimeUnit import akka.NotUsed -import akka.actor.{ Actor, ActorRef, ActorSystem, Props } -import akka.remote.artery.{ BenchTestSource, LatchSink } +import akka.actor.Actor +import akka.actor.ActorRef +import akka.actor.ActorSystem +import akka.actor.Props +import akka.remote.artery.BenchTestSource +import akka.remote.artery.LatchSink import akka.stream.scaladsl._ import akka.stream.testkit.scaladsl.StreamTestKit import akka.util.Timeout import com.typesafe.config.ConfigFactory import org.openjdk.jmh.annotations._ -import scala.concurrent.duration._ import scala.concurrent.Await +import scala.concurrent.duration._ object AskBenchmark { final val OperationsPerInvocation = 100000 @@ -40,8 +45,6 @@ class AskBenchmark { implicit val system = ActorSystem("MapAsyncBenchmark", config) import system.dispatcher - var materializer: ActorMaterializer = _ - var testSource: Source[java.lang.Integer, NotUsed] = _ var actor: ActorRef = _ @@ -56,15 +59,14 @@ class AskBenchmark { @Setup def setup(): Unit = { - val settings = ActorMaterializerSettings(system) - materializer = ActorMaterializer(settings) - testSource = Source.fromGraph(new BenchTestSource(OperationsPerInvocation)) actor = system.actorOf(Props(new Actor { override def receive = { case element => sender() ! element } })) + // eager init of materializer + SystemMaterializer(system).materializer } @TearDown @@ -77,14 +79,14 @@ class AskBenchmark { def mapAsync(): Unit = { val latch = new CountDownLatch(1) - testSource.ask[Int](parallelism)(actor).runWith(new LatchSink(OperationsPerInvocation, latch))(materializer) + testSource.ask[Int](parallelism)(actor).runWith(new LatchSink(OperationsPerInvocation, latch)) awaitLatch(latch) } private def awaitLatch(latch: CountDownLatch): Unit = { if (!latch.await(30, TimeUnit.SECONDS)) { - StreamTestKit.printDebugDump(materializer.supervisor) + StreamTestKit.printDebugDump(ActorMaterializerHelper.downcast(SystemMaterializer(system).materializer).supervisor) throw new RuntimeException("Latch didn't complete in time") } } diff --git a/akka-bench-jmh/src/main/scala/akka/stream/EmptySourceBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/stream/EmptySourceBenchmark.scala index b4ec60085f..75fbd77095 100644 --- a/akka-bench-jmh/src/main/scala/akka/stream/EmptySourceBenchmark.scala +++ b/akka-bench-jmh/src/main/scala/akka/stream/EmptySourceBenchmark.scala @@ -18,8 +18,6 @@ import scala.concurrent.duration._ @BenchmarkMode(Array(Mode.Throughput)) class EmptySourceBenchmark { implicit val system = ActorSystem("EmptySourceBenchmark") - val materializerSettings = ActorMaterializerSettings(system).withDispatcher("akka.test.stream-dispatcher") - implicit val materializer = ActorMaterializer(materializerSettings) @TearDown def shutdown(): Unit = { diff --git a/akka-bench-jmh/src/main/scala/akka/stream/FlatMapConcatBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/stream/FlatMapConcatBenchmark.scala index 64bc3cd82d..465df53e2f 100644 --- a/akka-bench-jmh/src/main/scala/akka/stream/FlatMapConcatBenchmark.scala +++ b/akka-bench-jmh/src/main/scala/akka/stream/FlatMapConcatBenchmark.scala @@ -4,11 +4,13 @@ package akka.stream -import java.util.concurrent.{ CountDownLatch, TimeUnit } +import java.util.concurrent.CountDownLatch +import java.util.concurrent.TimeUnit import akka.NotUsed import akka.actor.ActorSystem -import akka.remote.artery.{ BenchTestSource, LatchSink } +import akka.remote.artery.BenchTestSource +import akka.remote.artery.LatchSink import akka.stream.impl.fusing.GraphStages import akka.stream.scaladsl._ import akka.stream.testkit.scaladsl.StreamTestKit @@ -39,15 +41,12 @@ class FlatMapConcatBenchmark { private implicit val system: ActorSystem = ActorSystem("FlatMapConcatBenchmark", config) - var materializer: ActorMaterializer = _ - var testSource: Source[java.lang.Integer, NotUsed] = _ @Setup def setup(): Unit = { - val settings = ActorMaterializerSettings(system) - materializer = ActorMaterializer(settings) - + // eager init of materializer + SystemMaterializer(system).materializer testSource = Source.fromGraph(new BenchTestSource(OperationsPerInvocation)) } @@ -61,7 +60,7 @@ class FlatMapConcatBenchmark { def sourceDotSingle(): Unit = { val latch = new CountDownLatch(1) - testSource.flatMapConcat(Source.single).runWith(new LatchSink(OperationsPerInvocation, latch))(materializer) + testSource.flatMapConcat(Source.single).runWith(new LatchSink(OperationsPerInvocation, latch)) awaitLatch(latch) } @@ -73,7 +72,7 @@ class FlatMapConcatBenchmark { testSource .flatMapConcat(elem => new GraphStages.SingleSource(elem)) - .runWith(new LatchSink(OperationsPerInvocation, latch))(materializer) + .runWith(new LatchSink(OperationsPerInvocation, latch)) awaitLatch(latch) } @@ -83,7 +82,7 @@ class FlatMapConcatBenchmark { def oneElementList(): Unit = { val latch = new CountDownLatch(1) - testSource.flatMapConcat(n => Source(n :: Nil)).runWith(new LatchSink(OperationsPerInvocation, latch))(materializer) + testSource.flatMapConcat(n => Source(n :: Nil)).runWith(new LatchSink(OperationsPerInvocation, latch)) awaitLatch(latch) } @@ -93,15 +92,15 @@ class FlatMapConcatBenchmark { def mapBaseline(): Unit = { val latch = new CountDownLatch(1) - testSource.map(elem => elem).runWith(new LatchSink(OperationsPerInvocation, latch))(materializer) + testSource.map(elem => elem).runWith(new LatchSink(OperationsPerInvocation, latch)) awaitLatch(latch) } private def awaitLatch(latch: CountDownLatch): Unit = { if (!latch.await(30, TimeUnit.SECONDS)) { - implicit val ec = materializer.system.dispatcher - StreamTestKit.printDebugDump(materializer.supervisor) + implicit val ec = system.dispatcher + StreamTestKit.printDebugDump(ActorMaterializerHelper.downcast(SystemMaterializer(system).materializer).supervisor) throw new RuntimeException("Latch didn't complete in time") } } diff --git a/akka-bench-jmh/src/main/scala/akka/stream/FlatMapMergeBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/stream/FlatMapMergeBenchmark.scala index 35dd5962a8..38c9296b6f 100644 --- a/akka-bench-jmh/src/main/scala/akka/stream/FlatMapMergeBenchmark.scala +++ b/akka-bench-jmh/src/main/scala/akka/stream/FlatMapMergeBenchmark.scala @@ -18,8 +18,6 @@ import scala.concurrent.duration._ @BenchmarkMode(Array(Mode.Throughput)) class FlatMapMergeBenchmark { implicit val system = ActorSystem("FlatMapMergeBenchmark") - val materializerSettings = ActorMaterializerSettings(system).withDispatcher("akka.test.stream-dispatcher") - implicit val materializer = ActorMaterializer(materializerSettings) val NumberOfElements = 100000 @@ -42,6 +40,8 @@ class FlatMapMergeBenchmark { Source.repeat(()).take(n).flatMapMerge(n, _ => subSource) } graph = Source.fromGraph(source).toMat(Sink.ignore)(Keep.right) + // eager init of materializer + SystemMaterializer(system).materializer } @TearDown diff --git a/akka-bench-jmh/src/main/scala/akka/stream/FlowMapBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/stream/FlowMapBenchmark.scala index 1435636859..f9b484e72d 100644 --- a/akka-bench-jmh/src/main/scala/akka/stream/FlowMapBenchmark.scala +++ b/akka-bench-jmh/src/main/scala/akka/stream/FlowMapBenchmark.scala @@ -50,8 +50,6 @@ class FlowMapBenchmark { implicit val system = ActorSystem("test", config) - var materializer: ActorMaterializer = _ - @Param(Array("true", "false")) var UseGraphStageIdentity = false @@ -69,16 +67,14 @@ class FlowMapBenchmark { @Setup def setup(): Unit = { - val settings = ActorMaterializerSettings(system).withInputBuffer(initialInputBufferSize, initialInputBufferSize) - - materializer = ActorMaterializer(settings) - flow = mkMaps(Source.fromGraph(new BenchTestSource(100000)), numberOfMapOps) { if (UseGraphStageIdentity) GraphStages.identity[java.lang.Integer] else Flow[java.lang.Integer].map(identity) } + // eager init of materializer + SystemMaterializer(system).materializer } @TearDown @@ -92,7 +88,10 @@ class FlowMapBenchmark { val lock = new Semaphore(1) // todo rethink what is the most lightweight way to await for a streams completion lock.acquire() - flow.runWith(Sink.onComplete(_ => lock.release()))(materializer) + flow + .toMat(Sink.onComplete(_ => lock.release()))(Keep.right) + .withAttributes(Attributes.inputBuffer(initialInputBufferSize, initialInputBufferSize)) + .run() lock.acquire() } diff --git a/akka-bench-jmh/src/main/scala/akka/stream/FusedGraphsBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/stream/FusedGraphsBenchmark.scala index adcf7be542..10eb3d6294 100644 --- a/akka-bench-jmh/src/main/scala/akka/stream/FusedGraphsBenchmark.scala +++ b/akka-bench-jmh/src/main/scala/akka/stream/FusedGraphsBenchmark.scala @@ -4,13 +4,16 @@ package akka.stream -import java.util.concurrent.{ CountDownLatch, TimeUnit } +import java.util.concurrent.CountDownLatch +import java.util.concurrent.TimeUnit import akka.NotUsed import akka.actor.ActorSystem import akka.stream.scaladsl._ import akka.stream.stage._ -import org.openjdk.jmh.annotations.{ OperationsPerInvocation, _ } +import com.typesafe.config.ConfigFactory +import org.openjdk.jmh.annotations.OperationsPerInvocation +import org.openjdk.jmh.annotations._ import scala.concurrent.Await import scala.concurrent.duration._ @@ -93,7 +96,11 @@ class IdentityStage extends GraphStage[FlowShape[MutableElement, MutableElement] class FusedGraphsBenchmark { import FusedGraphsBenchmark._ - implicit val system = ActorSystem("test") + implicit val system = ActorSystem( + "test", + ConfigFactory.parseString(s""" + akka.stream.materializer.sync-processing-limit = ${Int.MaxValue} + """)) var materializer: ActorMaterializer = _ var testElements: Array[MutableElement] = _ @@ -110,9 +117,8 @@ class FusedGraphsBenchmark { @Setup def setup(): Unit = { - val settings = ActorMaterializerSettings(system).withFuzzing(false).withSyncProcessingLimit(Int.MaxValue) - - materializer = ActorMaterializer(settings) + // eager init of materializer + SystemMaterializer(system).materializer testElements = Array.fill(ElementCount)(new MutableElement(0)) val addFunc = (x: MutableElement) => { x.value += 1; x } diff --git a/akka-bench-jmh/src/main/scala/akka/stream/InvokeWithFeedbackBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/stream/InvokeWithFeedbackBenchmark.scala index bd6b509684..d707bc9427 100644 --- a/akka-bench-jmh/src/main/scala/akka/stream/InvokeWithFeedbackBenchmark.scala +++ b/akka-bench-jmh/src/main/scala/akka/stream/InvokeWithFeedbackBenchmark.scala @@ -18,7 +18,6 @@ import scala.concurrent.duration._ @BenchmarkMode(Array(Mode.Throughput)) class InvokeWithFeedbackBenchmark { implicit val system = ActorSystem("InvokeWithFeedbackBenchmark") - val materializerSettings = ActorMaterializerSettings(system).withDispatcher("akka.test.stream-dispatcher") var sourceQueue: SourceQueueWithComplete[Int] = _ var sinkQueue: SinkQueueWithCancel[Int] = _ @@ -27,10 +26,6 @@ class InvokeWithFeedbackBenchmark { @Setup def setup(): Unit = { - val settings = ActorMaterializerSettings(system) - - implicit val materializer = ActorMaterializer(settings) - // these are currently the only two built in stages using invokeWithFeedback val (in, out) = Source diff --git a/akka-bench-jmh/src/main/scala/akka/stream/MapAsyncBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/stream/MapAsyncBenchmark.scala index 209d52ffb8..f925cda01c 100644 --- a/akka-bench-jmh/src/main/scala/akka/stream/MapAsyncBenchmark.scala +++ b/akka-bench-jmh/src/main/scala/akka/stream/MapAsyncBenchmark.scala @@ -39,8 +39,6 @@ class MapAsyncBenchmark { implicit val system = ActorSystem("MapAsyncBenchmark", config) import system.dispatcher - var materializer: ActorMaterializer = _ - var testSource: Source[java.lang.Integer, NotUsed] = _ @Param(Array("1", "4")) @@ -51,9 +49,8 @@ class MapAsyncBenchmark { @Setup def setup(): Unit = { - val settings = ActorMaterializerSettings(system) - materializer = ActorMaterializer(settings) - + // eager init of materializer + SystemMaterializer(system).materializer testSource = Source.fromGraph(new BenchTestSource(OperationsPerInvocation)) } @@ -69,7 +66,7 @@ class MapAsyncBenchmark { testSource .mapAsync(parallelism)(elem => if (spawn) Future(elem) else Future.successful(elem)) - .runWith(new LatchSink(OperationsPerInvocation, latch))(materializer) + .runWith(new LatchSink(OperationsPerInvocation, latch)) awaitLatch(latch) } @@ -81,14 +78,14 @@ class MapAsyncBenchmark { testSource .mapAsyncUnordered(parallelism)(elem => if (spawn) Future(elem) else Future.successful(elem)) - .runWith(new LatchSink(OperationsPerInvocation, latch))(materializer) + .runWith(new LatchSink(OperationsPerInvocation, latch)) awaitLatch(latch) } private def awaitLatch(latch: CountDownLatch): Unit = { if (!latch.await(30, TimeUnit.SECONDS)) { - StreamTestKit.printDebugDump(materializer.supervisor) + StreamTestKit.printDebugDump(ActorMaterializerHelper.downcast(SystemMaterializer(system).materializer).supervisor) throw new RuntimeException("Latch didn't complete in time") } } diff --git a/akka-bench-jmh/src/main/scala/akka/stream/PartitionHubBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/stream/PartitionHubBenchmark.scala index f63f57f682..fc1a8ff385 100644 --- a/akka-bench-jmh/src/main/scala/akka/stream/PartitionHubBenchmark.scala +++ b/akka-bench-jmh/src/main/scala/akka/stream/PartitionHubBenchmark.scala @@ -38,8 +38,6 @@ class PartitionHubBenchmark { implicit val system = ActorSystem("PartitionHubBenchmark", config) - var materializer: ActorMaterializer = _ - @Param(Array("2", "5", "10", "20", "30")) var NumberOfStreams = 0 @@ -50,9 +48,8 @@ class PartitionHubBenchmark { @Setup def setup(): Unit = { - val settings = ActorMaterializerSettings(system) - materializer = ActorMaterializer(settings) - + // eager init of materializer + SystemMaterializer(system).materializer testSource = Source.fromGraph(new BenchTestSource(OperationsPerInvocation)) } @@ -71,10 +68,10 @@ class PartitionHubBenchmark { PartitionHub.sink[java.lang.Integer]( (_, elem) => elem.intValue % NumberOfStreams, startAfterNrOfConsumers = NumberOfStreams, - bufferSize = BufferSize))(materializer) + bufferSize = BufferSize)) for (_ <- 0 until NumberOfStreams) - source.runWith(new LatchSink(N / NumberOfStreams, latch))(materializer) + source.runWith(new LatchSink(N / NumberOfStreams, latch)) if (!latch.await(30, TimeUnit.SECONDS)) { dumpMaterializer() @@ -90,11 +87,10 @@ class PartitionHubBenchmark { val source = testSource.runWith( Sink.fromGraph( - new FixedSizePartitionHub(_.intValue % NumberOfStreams, lanes = NumberOfStreams, bufferSize = BufferSize)))( - materializer) + new FixedSizePartitionHub(_.intValue % NumberOfStreams, lanes = NumberOfStreams, bufferSize = BufferSize))) for (_ <- 0 until NumberOfStreams) - source.runWith(new LatchSink(N / NumberOfStreams, latch))(materializer) + source.runWith(new LatchSink(N / NumberOfStreams, latch)) if (!latch.await(30, TimeUnit.SECONDS)) { dumpMaterializer() @@ -103,8 +99,8 @@ class PartitionHubBenchmark { } private def dumpMaterializer(): Unit = { - implicit val ec = materializer.system.dispatcher - StreamTestKit.printDebugDump(materializer.supervisor) + implicit val ec = system.dispatcher + StreamTestKit.printDebugDump(ActorMaterializerHelper.downcast(SystemMaterializer(system).materializer).supervisor) } } diff --git a/akka-docs/src/main/paradox/project/migration-guide-2.5.x-2.6.x.md b/akka-docs/src/main/paradox/project/migration-guide-2.5.x-2.6.x.md index 32db2372f6..bdc7c9b0b1 100644 --- a/akka-docs/src/main/paradox/project/migration-guide-2.5.x-2.6.x.md +++ b/akka-docs/src/main/paradox/project/migration-guide-2.5.x-2.6.x.md @@ -506,3 +506,39 @@ Having a default materializer available means that most, if not all, usages of J and Scala `implicit val materializer = ActorMaterializer()` should be removed. Details about the stream materializer can be found in [Actor Materializer Lifecycle](../stream/stream-flows-and-basics.md#actor-materializer-lifecycle) + +### Materializer settings deprecated + +The `ActorMaterializerSettings` class has been deprecated. + +All materializer settings are available as configuration to change the system default or through attributes that can be +used for individual streams when they are materialized. + +| Materializer setting | Corresponding attribute | Setting | +-------------------------|-------------------------|---------| +| `initialInputBufferSize` | `Attributes.inputBuffer(initial, max)` | `akka.stream.materializer.initial-input-buffer-size` | +| `maxInputBufferSize` | `Attributes.inputBuffer(initial, max)` | `akka.stream.materializer.max-input-buffer-size` | +| `dispatcher` | `ActorAttributes.dispatcher(name)` | `akka.stream.materializer.dispatcher` | +| `supervisionDecider` | `ActorAttributes.supervisionStrategy(strategy)` | na | +| `debugLogging` | `ActorAttributes.debugLogging` | `akka.stream.materializer.debug-logging` | +| `outputBurstLimit` | `ActorAttributes.outputBurstLimit` | `akka.stream.materializer.output-burst-limit` | +| `fuzzingMode` | `ActorAttributes.fuzzingMode` | `akka.stream.materializer.debug.fuzzing-mode` | +| `autoFusing` | no longer used (since 2.5.0) | na | +| `maxFixedBufferSize` | `ActorAttributes.maxFixedBufferSize` | `akka.stream.materializer.max-fixed-buffer-size` | +| `syncProcessingLimit` | `ActorAttributes.syncProcessingLimit` | `akka.stream.materializer.sync-processing-limit` | +| `ioSettings.tcpWriteBufferSize` | `Tcp.writeBufferSize` | `akka.stream.materializer.io.tcp.write-buffer-size` | +| `streamRefSettings.bufferCapacity` | `StreamRefAttributes.bufferCapacity` | `akka.stream.materializer.stream-ref.buffer-capacity` | +| `streamRefSettings.demandRedeliveryInterval` | `StreamRefAttributes.demandRedeliveryInterval` | `akka.stream.materializer.stream-ref.demand-redelivery-interval` | +| `streamRefSettings.subscriptionTimeout` | `StreamRefAttributes.subscriptionTimeout` | `akka.stream.materializer.stream-ref.subscription-timeout` | +| `streamRefSettings.finalTerminationSignalDeadline` | `StreamRefAttributes.finalTerminationSignalDeadline` | `akka.stream.materializer.stream-ref.final-termination-signal-deadline` | +| `blockingIoDispatcher` | `ActorAttributes.blockingIoDispatcher` | `akka.stream.materializer.blocking-io-dispatcher` | +| `subscriptionTimeoutSettings.mode` | `ActorAttributes.streamSubscriptionTimeoutMode` | `akka.stream.materializer.subscription-timeout.mode` | +| `subscriptionTimeoutSettings.timeout` | `ActorAttributes.streamSubscriptionTimeout` | `akka.stream.materializer.subscription-timeout.timeout` | + +Setting attributes on individual streams can be done like so: + +Scala +: @@snip [StreamAttributeDocSpec.scala](/akka-stream-tests/src/test/scala/akka/stream/StreamAttributeDocSpec.scala) { #attributes-on-stream } + +Java +: @@snip [StreamAttributeDocTest.java](/akka-stream-tests/src/test/java/akka/stream/StreamAttributeDocTest.java) { #attributes-on-stream } diff --git a/akka-docs/src/main/paradox/stream/stream-error.md b/akka-docs/src/main/paradox/stream/stream-error.md index 320b3b151d..47aa365343 100644 --- a/akka-docs/src/main/paradox/stream/stream-error.md +++ b/akka-docs/src/main/paradox/stream/stream-error.md @@ -198,7 +198,7 @@ Scala Java : @@snip [FlowErrorDocTest.java](/akka-docs/src/test/java/jdocs/stream/FlowErrorDocTest.java) { #stop } -The default supervision strategy for a stream can be defined on the settings of the materializer. +The default supervision strategy for a stream can be defined on the complete `RunnableGraph`. Scala : @@snip [FlowErrorDocSpec.scala](/akka-docs/src/test/scala/docs/stream/FlowErrorDocSpec.scala) { #resume } diff --git a/akka-docs/src/main/paradox/stream/stream-integrations.md b/akka-docs/src/main/paradox/stream/stream-integrations.md index 5c7c1c9762..a1c51370e5 100644 --- a/akka-docs/src/main/paradox/stream/stream-integrations.md +++ b/akka-docs/src/main/paradox/stream/stream-integrations.md @@ -384,7 +384,7 @@ is `completed` before `g`, but still emitted afterwards. The numbers in parenthesis illustrates how many calls that are in progress at the same time. Here the downstream demand and thereby the number of concurrent -calls are limited by the buffer size (4) of the `ActorMaterializerSettings`. +calls are limited by the buffer size (4) set with an attribute. Here is how we can use the same service with `mapAsyncUnordered`: @@ -444,7 +444,7 @@ Note that `after` lines are not in the same order as the `before` lines. For exa The numbers in parenthesis illustrates how many calls that are in progress at the same time. Here the downstream demand and thereby the number of concurrent -calls are limited by the buffer size (4) of the `ActorMaterializerSettings`. +calls are limited by the buffer size (4) set with an attribute. ## Integrating with Reactive Streams diff --git a/akka-docs/src/main/paradox/stream/stream-quickstart.md b/akka-docs/src/main/paradox/stream/stream-quickstart.md index 928ee33bb6..17b75c66c3 100644 --- a/akka-docs/src/main/paradox/stream/stream-quickstart.md +++ b/akka-docs/src/main/paradox/stream/stream-quickstart.md @@ -237,10 +237,6 @@ Scala Java : @@snip [TwitterStreamQuickstartDocTest.java](/akka-docs/src/test/java/jdocs/stream/TwitterStreamQuickstartDocTest.java) { #system-setup } -The `ActorMaterializer` can optionally take `ActorMaterializerSettings` which can be used to define -materialization properties, such as default buffer sizes (see also @ref:[Buffers for asynchronous operators](stream-rate.md#async-stream-buffers)), the dispatcher to -be used by the pipeline etc. These can be overridden with `withAttributes` on `Flow`, `Source`, `Sink` and `Graph`. - Let's assume we have a stream of tweets readily available. In Akka this is expressed as a @scala[`Source[Out, M]`]@java[`Source`]: Scala diff --git a/akka-docs/src/main/paradox/stream/stream-rate.md b/akka-docs/src/main/paradox/stream/stream-rate.md index 134700216d..38849a2953 100644 --- a/akka-docs/src/main/paradox/stream/stream-rate.md +++ b/akka-docs/src/main/paradox/stream/stream-rate.md @@ -75,16 +75,8 @@ can be set through configuration: akka.stream.materializer.max-input-buffer-size = 16 ``` -Alternatively they can be set by passing a -@scala[@scaladoc[`ActorMaterializerSettings`](akka.stream.ActorMaterializerSettings)]@java[@javadoc[`ActorMaterializerSettings`](akka.stream.ActorMaterializerSettings)] to the materializer: - -Scala -: @@snip [StreamBuffersRateSpec.scala](/akka-docs/src/test/scala/docs/stream/StreamBuffersRateSpec.scala) { #materializer-buffer } - -Java -: @@snip [StreamBuffersRateDocTest.java](/akka-docs/src/test/java/jdocs/stream/StreamBuffersRateDocTest.java) { #materializer-buffer } - -If the buffer size needs to be set for segments of a @scala[@scaladoc[`Flow`](akka.stream.scaladsl.Flow)]@java[@javadoc[`Flow`](akka.stream.javadsl.Flow)] only, it is possible by defining a separate +Alternatively they can be set per stream by adding an attribute to the complete `RunnableGraph` or on smaller segments +of the stream it is possible by defining a separate @scala[@scaladoc[`Flow`](akka.stream.scaladsl.Flow)]@java[@javadoc[`Flow`](akka.stream.javadsl.Flow)] with these attributes: Scala diff --git a/akka-docs/src/test/java/jdocs/stream/FlowErrorDocTest.java b/akka-docs/src/test/java/jdocs/stream/FlowErrorDocTest.java index 1ee2654602..de33dbc567 100644 --- a/akka-docs/src/test/java/jdocs/stream/FlowErrorDocTest.java +++ b/akka-docs/src/test/java/jdocs/stream/FlowErrorDocTest.java @@ -13,6 +13,7 @@ import java.util.concurrent.TimeUnit; import akka.NotUsed; import akka.japi.pf.PFBuilder; +import akka.stream.javadsl.*; import jdocs.AbstractJavaTest; import akka.testkit.javadsl.TestKit; import org.junit.AfterClass; @@ -22,10 +23,7 @@ import org.junit.Test; import akka.actor.ActorSystem; import akka.stream.Materializer; import akka.stream.Supervision; -import akka.stream.javadsl.Flow; import akka.stream.ActorAttributes; -import akka.stream.javadsl.Sink; -import akka.stream.javadsl.Source; import akka.japi.function.Function; public class FlowErrorDocTest extends AbstractJavaTest { @@ -71,7 +69,13 @@ public class FlowErrorDocTest extends AbstractJavaTest { .map(elem -> 100 / elem) .withAttributes(ActorAttributes.withSupervisionStrategy(decider)); final Sink> fold = Sink.fold(0, (acc, elem) -> acc + elem); - final CompletionStage result = source.runWith(fold, system); + + final RunnableGraph> runnableGraph = source.toMat(fold, Keep.right()); + + final RunnableGraph> withCustomSupervision = + runnableGraph.withAttributes(ActorAttributes.withSupervisionStrategy(decider)); + + final CompletionStage result = withCustomSupervision.run(system); // the element causing division by zero will be dropped // result here will be a CompletionStage completed with 228 // #resume diff --git a/akka-docs/src/test/java/jdocs/stream/IntegrationDocTest.java b/akka-docs/src/test/java/jdocs/stream/IntegrationDocTest.java index 171326afce..56e23e2887 100644 --- a/akka-docs/src/test/java/jdocs/stream/IntegrationDocTest.java +++ b/akka-docs/src/test/java/jdocs/stream/IntegrationDocTest.java @@ -663,10 +663,6 @@ public class IntegrationDocTest extends AbstractJavaTest { final Executor blockingEc = system.dispatchers().lookup("blocking-dispatcher"); final SometimesSlowService service = new SometimesSlowService(blockingEc); - final ActorMaterializer mat = - ActorMaterializer.create( - ActorMaterializerSettings.create(system).withInputBuffer(4, 4), system); - Source.from(Arrays.asList("a", "B", "C", "D", "e", "F", "g", "H", "i", "J")) .map( elem -> { @@ -674,7 +670,9 @@ public class IntegrationDocTest extends AbstractJavaTest { return elem; }) .mapAsync(4, service::convert) - .runForeach(elem -> System.out.println("after: " + elem), system); + .to(Sink.foreach(elem -> System.out.println("after: " + elem))) + .withAttributes(Attributes.inputBuffer(4, 4)) + .run(system); // #sometimes-slow-mapAsync probe.expectMsg("after: A"); diff --git a/akka-docs/src/test/java/jdocs/stream/StreamBuffersRateDocTest.java b/akka-docs/src/test/java/jdocs/stream/StreamBuffersRateDocTest.java index ad7e33c732..522a814518 100644 --- a/akka-docs/src/test/java/jdocs/stream/StreamBuffersRateDocTest.java +++ b/akka-docs/src/test/java/jdocs/stream/StreamBuffersRateDocTest.java @@ -66,12 +66,6 @@ public class StreamBuffersRateDocTest extends AbstractJavaTest { @Test @SuppressWarnings("unused") public void demonstrateBufferSizes() { - // #materializer-buffer - final Materializer materializer = - ActorMaterializer.create( - ActorMaterializerSettings.create(system).withInputBuffer(64, 64), system); - // #materializer-buffer - // #section-buffer final Flow flow1 = Flow.of(Integer.class) @@ -81,7 +75,13 @@ public class StreamBuffersRateDocTest extends AbstractJavaTest { final Flow flow2 = flow1 .via(Flow.of(Integer.class).map(elem -> elem / 2)) - .async(); // the buffer size of this map is the default + .async(); // the buffer size of this map is the value from the surrounding graph it is + // used in + final RunnableGraph runnableGraph = + Source.range(1, 10).via(flow1).to(Sink.foreach(elem -> System.out.println(elem))); + + final RunnableGraph withOverridenDefaults = + runnableGraph.withAttributes(Attributes.inputBuffer(64, 64)); // #section-buffer } diff --git a/akka-docs/src/test/scala/docs/stream/FlowErrorDocSpec.scala b/akka-docs/src/test/scala/docs/stream/FlowErrorDocSpec.scala index 0da418c593..e2f5b005ce 100644 --- a/akka-docs/src/test/scala/docs/stream/FlowErrorDocSpec.scala +++ b/akka-docs/src/test/scala/docs/stream/FlowErrorDocSpec.scala @@ -5,12 +5,9 @@ package docs.stream import scala.concurrent.Await -import akka.stream.ActorMaterializer -import akka.stream.ActorMaterializerSettings import akka.stream.Supervision import akka.stream.scaladsl._ import akka.testkit.AkkaSpec -import akka.stream.Attributes import akka.stream.ActorAttributes import scala.concurrent.duration._ @@ -35,9 +32,13 @@ class FlowErrorDocSpec extends AkkaSpec { case _: ArithmeticException => Supervision.Resume case _ => Supervision.Stop } - implicit val materializer = ActorMaterializer(ActorMaterializerSettings(system).withSupervisionStrategy(decider)) val source = Source(0 to 5).map(100 / _) - val result = source.runWith(Sink.fold(0)(_ + _)) + val runnableGraph = + source.toMat(Sink.fold(0)(_ + _))(Keep.right) + + val withCustomSupervision = runnableGraph.withAttributes(ActorAttributes.supervisionStrategy(decider)) + + val result = withCustomSupervision.run() // the element causing division by zero will be dropped // result here will be a Future completed with Success(228) //#resume diff --git a/akka-docs/src/test/scala/docs/stream/IntegrationDocSpec.scala b/akka-docs/src/test/scala/docs/stream/IntegrationDocSpec.scala index 53f96397f4..49a746c1af 100644 --- a/akka-docs/src/test/scala/docs/stream/IntegrationDocSpec.scala +++ b/akka-docs/src/test/scala/docs/stream/IntegrationDocSpec.scala @@ -409,13 +409,12 @@ class IntegrationDocSpec extends AkkaSpec(IntegrationDocSpec.config) { implicit val blockingExecutionContext = system.dispatchers.lookup("blocking-dispatcher") val service = new SometimesSlowService - implicit val materializer = - ActorMaterializer(ActorMaterializerSettings(system).withInputBuffer(initialSize = 4, maxSize = 4)) - Source(List("a", "B", "C", "D", "e", "F", "g", "H", "i", "J")) .map(elem => { println(s"before: $elem"); elem }) .mapAsync(4)(service.convert) - .runForeach(elem => println(s"after: $elem")) + .to(Sink.foreach(elem => println(s"after: $elem"))) + .withAttributes(Attributes.inputBuffer(initial = 4, max = 4)) + .run() //#sometimes-slow-mapAsync probe.expectMsg("after: A") @@ -441,13 +440,12 @@ class IntegrationDocSpec extends AkkaSpec(IntegrationDocSpec.config) { implicit val blockingExecutionContext = system.dispatchers.lookup("blocking-dispatcher") val service = new SometimesSlowService - implicit val materializer = - ActorMaterializer(ActorMaterializerSettings(system).withInputBuffer(initialSize = 4, maxSize = 4)) - Source(List("a", "B", "C", "D", "e", "F", "g", "H", "i", "J")) .map(elem => { println(s"before: $elem"); elem }) .mapAsyncUnordered(4)(service.convert) - .runForeach(elem => println(s"after: $elem")) + .to(Sink.foreach(elem => println(s"after: $elem"))) + .withAttributes(Attributes.inputBuffer(initial = 4, max = 4)) + .run() //#sometimes-slow-mapAsyncUnordered probe.receiveN(10).toSet should be( diff --git a/akka-docs/src/test/scala/docs/stream/StreamBuffersRateSpec.scala b/akka-docs/src/test/scala/docs/stream/StreamBuffersRateSpec.scala index a7a4b5e61a..389afdf226 100644 --- a/akka-docs/src/test/scala/docs/stream/StreamBuffersRateSpec.scala +++ b/akka-docs/src/test/scala/docs/stream/StreamBuffersRateSpec.scala @@ -32,14 +32,13 @@ class StreamBuffersRateSpec extends AkkaSpec { } "Demonstrate buffer sizes" in { - //#materializer-buffer - val materializer = - ActorMaterializer(ActorMaterializerSettings(system).withInputBuffer(initialSize = 64, maxSize = 64)) - //#materializer-buffer - //#section-buffer val section = Flow[Int].map(_ * 2).async.addAttributes(Attributes.inputBuffer(initial = 1, max = 1)) // the buffer size of this map is 1 val flow = section.via(Flow[Int].map(_ / 2)).async // the buffer size of this map is the default + val runnableGraph = + Source(1 to 10).via(flow).to(Sink.foreach(elem => println(elem))) + + val withOverriddenDefaults = runnableGraph.withAttributes(Attributes.inputBuffer(initial = 64, max = 64)) //#section-buffer } diff --git a/akka-docs/src/test/scala/docs/stream/cookbook/RecipeCollectingMetrics.scala b/akka-docs/src/test/scala/docs/stream/cookbook/RecipeCollectingMetrics.scala index 43f6218e3f..9ac632e6f5 100644 --- a/akka-docs/src/test/scala/docs/stream/cookbook/RecipeCollectingMetrics.scala +++ b/akka-docs/src/test/scala/docs/stream/cookbook/RecipeCollectingMetrics.scala @@ -4,13 +4,7 @@ package docs.stream.cookbook -import akka.stream.{ ActorMaterializer, ActorMaterializerSettings } - -import scala.collection.immutable -import scala.concurrent.Await - class RecipeCollectingMetrics extends RecipeSpec { - implicit val m2 = ActorMaterializer(ActorMaterializerSettings(system).withInputBuffer(1, 1)) "Recipe for periodically collecting metrics" must { @@ -39,7 +33,7 @@ class RecipeCollectingMetrics extends RecipeSpec { // } // //#periodic-metrics-collection // - // val reports = graph.run().get(futureSink) + // val reports = graph.withAttributes(Attributes.inputBuffer(1, 1).run().get(futureSink) // val manualLoad = new StreamTestKit.AutoPublisher(loadPub) // val manualTick = new StreamTestKit.AutoPublisher(tickPub) // diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala b/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala index 910215b40c..717c13fcc7 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala @@ -7,8 +7,8 @@ package akka.remote.artery import java.net.InetAddress import akka.util.ccompat.JavaConverters._ -import scala.concurrent.duration._ +import scala.concurrent.duration._ import akka.NotUsed import akka.japi.Util.immutableSeq import akka.stream.ActorMaterializerSettings @@ -16,6 +16,7 @@ import akka.util.Helpers.ConfigOps import akka.util.Helpers.Requiring import akka.util.Helpers.toRootLowerCase import akka.util.WildcardIndex +import com.github.ghik.silencer.silent import com.typesafe.config.Config import com.typesafe.config.ConfigFactory @@ -95,8 +96,10 @@ private[akka] final class ArterySettings private (config: Config) { val TestMode: Boolean = getBoolean("test-mode") val Dispatcher: String = getString("use-dispatcher") val ControlStreamDispatcher: String = getString("use-control-stream-dispatcher") + @silent("deprecated") val MaterializerSettings: ActorMaterializerSettings = ActorMaterializerSettings(config.getConfig("materializer")).withDispatcher(Dispatcher) + @silent("deprecated") val ControlStreamMaterializerSettings: ActorMaterializerSettings = ActorMaterializerSettings(config.getConfig("materializer")).withDispatcher(ControlStreamDispatcher) diff --git a/akka-remote/src/test/scala/akka/remote/artery/DuplicateHandshakeSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/DuplicateHandshakeSpec.scala index 874f1d2037..4e41987f21 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/DuplicateHandshakeSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/DuplicateHandshakeSpec.scala @@ -8,8 +8,8 @@ import akka.actor.Address import akka.actor.ExtendedActorSystem import akka.remote.UniqueAddress import akka.remote.artery.OutboundHandshake.HandshakeReq -import akka.stream.ActorMaterializer -import akka.stream.ActorMaterializerSettings +import akka.serialization.SerializationExtension +import akka.serialization.SerializerWithStringManifest import akka.stream.scaladsl.Keep import akka.stream.testkit.TestPublisher import akka.stream.testkit.TestSubscriber @@ -18,13 +18,11 @@ import akka.stream.testkit.scaladsl.TestSource import akka.testkit.AkkaSpec import akka.testkit.ImplicitSender import akka.util.OptionVal -import akka.serialization.SerializationExtension -import akka.serialization.SerializerWithStringManifest -class DuplicateHandshakeSpec extends AkkaSpec with ImplicitSender { +class DuplicateHandshakeSpec extends AkkaSpec(""" + akka.stream.materializer.debug.fuzzing-mode = on + """) with ImplicitSender { - val matSettings = ActorMaterializerSettings(system).withFuzzing(true) - implicit val mat = ActorMaterializer(matSettings)(system) val pool = new EnvelopeBufferPool(1034 * 1024, 128) val serialization = SerializationExtension(system) diff --git a/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala index 2d333bc44c..eee45c8e6c 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala @@ -4,14 +4,10 @@ package akka.remote.artery -import scala.util.Try - import akka.Done import akka.actor.Address import akka.remote.UniqueAddress import akka.remote.artery.InboundControlJunction.ControlMessageObserver -import akka.stream.ActorMaterializer -import akka.stream.ActorMaterializerSettings import akka.stream.scaladsl.Keep import akka.stream.testkit.scaladsl.TestSink import akka.stream.testkit.scaladsl.TestSource @@ -20,6 +16,8 @@ import akka.testkit.ImplicitSender import akka.testkit.TestProbe import akka.util.OptionVal +import scala.util.Try + object InboundControlJunctionSpec { trait TestControlMessage extends ControlMessage @@ -33,13 +31,11 @@ class InboundControlJunctionSpec akka.actor.serialization-bindings { "akka.remote.artery.InboundControlJunctionSpec$TestControlMessage" = java } + akka.stream.materializer.debug.fuzzing-mode = on """) with ImplicitSender { import InboundControlJunctionSpec._ - val matSettings = ActorMaterializerSettings(system).withFuzzing(true) - implicit val mat = ActorMaterializer(matSettings)(system) - val addressA = UniqueAddress(Address("akka", "sysA", "hostA", 1001), 1) val addressB = UniqueAddress(Address("akka", "sysB", "hostB", 1002), 2) diff --git a/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala index a094048af6..403fc7650d 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala @@ -4,14 +4,10 @@ package akka.remote.artery -import scala.concurrent.Await -import scala.concurrent.duration._ import akka.actor.Address import akka.remote.UniqueAddress import akka.remote.artery.OutboundHandshake.HandshakeReq import akka.remote.artery.OutboundHandshake.HandshakeRsp -import akka.stream.ActorMaterializer -import akka.stream.ActorMaterializerSettings import akka.stream.scaladsl.Keep import akka.stream.testkit.TestPublisher import akka.stream.testkit.TestSubscriber @@ -22,16 +18,18 @@ import akka.testkit.ImplicitSender import akka.testkit.TestProbe import akka.util.OptionVal +import scala.concurrent.Await +import scala.concurrent.duration._ + object InboundHandshakeSpec { case object Control1 extends ControlMessage case object Control2 extends ControlMessage case object Control3 extends ControlMessage } -class InboundHandshakeSpec extends AkkaSpec with ImplicitSender { - - val matSettings = ActorMaterializerSettings(system).withFuzzing(true) - implicit val mat = ActorMaterializer(matSettings)(system) +class InboundHandshakeSpec extends AkkaSpec(""" + akka.stream.materializer.debug.fuzzing-mode = on + """) with ImplicitSender { val addressA = UniqueAddress(Address("akka", "sysA", "hostA", 1001), 1) val addressB = UniqueAddress(Address("akka", "sysB", "hostB", 1002), 2) diff --git a/akka-remote/src/test/scala/akka/remote/artery/OutboundControlJunctionSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/OutboundControlJunctionSpec.scala index 885127152e..505b93234c 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/OutboundControlJunctionSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/OutboundControlJunctionSpec.scala @@ -6,10 +6,11 @@ package akka.remote.artery import akka.actor.Address import akka.remote.UniqueAddress -import akka.stream.{ ActorMaterializer, ActorMaterializerSettings } import akka.stream.scaladsl.Keep -import akka.stream.testkit.scaladsl.{ TestSink, TestSource } -import akka.testkit.{ AkkaSpec, ImplicitSender } +import akka.stream.testkit.scaladsl.TestSink +import akka.stream.testkit.scaladsl.TestSource +import akka.testkit.AkkaSpec +import akka.testkit.ImplicitSender import akka.util.OptionVal object OutboundControlJunctionSpec { @@ -18,12 +19,11 @@ object OutboundControlJunctionSpec { case object Control3 extends ControlMessage } -class OutboundControlJunctionSpec extends AkkaSpec with ImplicitSender { +class OutboundControlJunctionSpec extends AkkaSpec(""" + akka.stream.materializer.debug.fuzzing-mode = on + """) with ImplicitSender { import OutboundControlJunctionSpec._ - val matSettings = ActorMaterializerSettings(system).withFuzzing(true) - implicit val mat = ActorMaterializer(matSettings)(system) - val addressA = UniqueAddress(Address("akka", "sysA", "hostA", 1001), 1) val addressB = UniqueAddress(Address("akka", "sysB", "hostB", 1002), 2) diff --git a/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala index 75252e2189..407ea73bdf 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala @@ -4,13 +4,10 @@ package akka.remote.artery -import scala.concurrent.duration._ import akka.actor.Address import akka.remote.UniqueAddress import akka.remote.artery.OutboundHandshake.HandshakeReq import akka.remote.artery.OutboundHandshake.HandshakeTimeoutException -import akka.stream.ActorMaterializer -import akka.stream.ActorMaterializerSettings import akka.stream.scaladsl.Keep import akka.stream.testkit.TestPublisher import akka.stream.testkit.TestSubscriber @@ -20,10 +17,11 @@ import akka.testkit.AkkaSpec import akka.testkit.ImplicitSender import akka.util.OptionVal -class OutboundHandshakeSpec extends AkkaSpec with ImplicitSender { +import scala.concurrent.duration._ - val matSettings = ActorMaterializerSettings(system).withFuzzing(true) - implicit val mat = ActorMaterializer(matSettings)(system) +class OutboundHandshakeSpec extends AkkaSpec(""" + akka.stream.materializer.debug.fuzzing-mode = on + """) with ImplicitSender { val addressA = UniqueAddress(Address("akka", "sysA", "hostA", 1001), 1) val addressB = UniqueAddress(Address("akka", "sysB", "hostB", 1002), 2) diff --git a/akka-remote/src/test/scala/akka/remote/artery/SendQueueSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/SendQueueSpec.scala index 776de1e4b2..03d1a19a14 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/SendQueueSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/SendQueueSpec.scala @@ -6,12 +6,8 @@ package akka.remote.artery import java.util.Queue -import scala.concurrent.duration._ - import akka.actor.Actor import akka.actor.Props -import akka.stream.ActorMaterializer -import akka.stream.ActorMaterializerSettings import akka.stream.scaladsl.Keep import akka.stream.scaladsl.Source import akka.stream.testkit.TestSubscriber @@ -20,6 +16,8 @@ import akka.testkit.AkkaSpec import akka.testkit.ImplicitSender import org.agrona.concurrent.ManyToOneConcurrentArrayQueue +import scala.concurrent.duration._ + object SendQueueSpec { case class ProduceToQueue(from: Int, until: Int, queue: Queue[Msg]) @@ -50,12 +48,12 @@ object SendQueueSpec { } } -class SendQueueSpec extends AkkaSpec("akka.actor.serialize-messages = off") with ImplicitSender { +class SendQueueSpec extends AkkaSpec(""" + akka.actor.serialize-messages = off + akka.stream.materializer.debug.fuzzing-mode = on + """.stripMargin) with ImplicitSender { import SendQueueSpec._ - val matSettings = ActorMaterializerSettings(system).withFuzzing(true) - implicit val mat = ActorMaterializer(matSettings)(system) - def sendToDeadLetters[T](pending: Vector[T]): Unit = pending.foreach(system.deadLetters ! _) diff --git a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageAckerSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageAckerSpec.scala index b05c48aec8..580a2f0710 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageAckerSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageAckerSpec.scala @@ -7,8 +7,6 @@ package akka.remote.artery import akka.actor.Address import akka.remote.UniqueAddress import akka.remote.artery.SystemMessageDelivery._ -import akka.stream.ActorMaterializer -import akka.stream.ActorMaterializerSettings import akka.stream.scaladsl.Keep import akka.stream.testkit.TestPublisher import akka.stream.testkit.TestSubscriber @@ -19,10 +17,9 @@ import akka.testkit.ImplicitSender import akka.testkit.TestProbe import akka.util.OptionVal -class SystemMessageAckerSpec extends AkkaSpec with ImplicitSender { - - val matSettings = ActorMaterializerSettings(system).withFuzzing(true) - implicit val mat = ActorMaterializer(matSettings)(system) +class SystemMessageAckerSpec extends AkkaSpec(""" + akka.stream.materializer.debug.fuzzing-mode = on + """) with ImplicitSender { val addressA = UniqueAddress(Address("akka", "sysA", "hostA", 1001), 1) val addressB = UniqueAddress(Address("akka", "sysB", "hostB", 1002), 2) diff --git a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala index 5ac48be5bb..466dcf2353 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala @@ -6,9 +6,6 @@ package akka.remote.artery import java.util.concurrent.ThreadLocalRandom -import scala.concurrent.Await -import scala.concurrent.duration._ - import akka.NotUsed import akka.actor.ActorIdentity import akka.actor.ActorSystem @@ -18,8 +15,6 @@ import akka.remote.AddressUidExtension import akka.remote.RARP import akka.remote.UniqueAddress import akka.remote.artery.SystemMessageDelivery._ -import akka.stream.ActorMaterializer -import akka.stream.ActorMaterializerSettings import akka.stream.ThrottleMode import akka.stream.scaladsl.Flow import akka.stream.scaladsl.Sink @@ -34,6 +29,9 @@ import akka.util.OptionVal import com.typesafe.config.Config import com.typesafe.config.ConfigFactory +import scala.concurrent.Await +import scala.concurrent.duration._ + object SystemMessageDeliverySpec { case class TestSysMsg(s: String) extends SystemMessageDelivery.AckedDeliveryMessage @@ -45,6 +43,7 @@ object SystemMessageDeliverySpec { akka.remote.watch-failure-detector.heartbeat-interval = 2 s akka.remote.artery.log-received-messages = on akka.remote.artery.log-sent-messages = on + akka.stream.materializer.debug.fuzzing-mode = on """).withFallback(ArterySpecSupport.defaultConfig) val config = @@ -59,8 +58,6 @@ abstract class AbstractSystemMessageDeliverySpec(c: Config) extends ArteryMultiN val systemB = newRemoteSystem(name = Some("systemB")) val addressB = UniqueAddress(address(systemB), AddressUidExtension(systemB).longAddressUid) val rootB = RootActorPath(addressB.address) - val matSettings = ActorMaterializerSettings(system).withFuzzing(true) - implicit val mat = ActorMaterializer(matSettings)(system) private val outboundEnvelopePool = ReusableOutboundEnvelope.createObjectPool(capacity = 16) diff --git a/akka-remote/src/test/scala/akka/remote/artery/aeron/AeronSinkSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/aeron/AeronSinkSpec.scala index 7b5644fda6..a6c3630a4b 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/aeron/AeronSinkSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/aeron/AeronSinkSpec.scala @@ -7,14 +7,8 @@ package aeron import java.io.File -import scala.concurrent.Await -import scala.concurrent.duration._ -import scala.util.control.NoStackTrace - import akka.actor.ExtendedActorSystem import akka.remote.artery.aeron.AeronSink.GaveUpMessageException -import akka.stream.ActorMaterializer -import akka.stream.ActorMaterializerSettings import akka.stream.scaladsl.Sink import akka.stream.scaladsl.Source import akka.testkit.AkkaSpec @@ -24,7 +18,13 @@ import io.aeron.Aeron import io.aeron.driver.MediaDriver import org.agrona.IoUtil -class AeronSinkSpec extends AkkaSpec with ImplicitSender { +import scala.concurrent.Await +import scala.concurrent.duration._ +import scala.util.control.NoStackTrace + +class AeronSinkSpec extends AkkaSpec(""" + akka.stream.materializer.debug.fuzzing-mode = on + """) with ImplicitSender { val driver = MediaDriver.launchEmbedded() @@ -43,9 +43,6 @@ class AeronSinkSpec extends AkkaSpec with ImplicitSender { val pool = new EnvelopeBufferPool(1034 * 1024, 128) - val matSettings = ActorMaterializerSettings(system).withFuzzing(true) - implicit val mat = ActorMaterializer(matSettings)(system) - override def afterTermination(): Unit = { taskRunner.stop() aeron.close() diff --git a/akka-remote/src/test/scala/akka/remote/artery/tcp/TcpFramingSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/tcp/TcpFramingSpec.scala index 1fe0dd72df..444e2d46ec 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/tcp/TcpFramingSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/tcp/TcpFramingSpec.scala @@ -5,11 +5,6 @@ package akka.remote.artery package tcp -import scala.util.Random - -import akka.stream.ActorMaterializer -import akka.stream.ActorMaterializerSettings -import akka.stream.impl.io.ByteStringParser.ParsingException import akka.stream.impl.io.ByteStringParser.ParsingException import akka.stream.scaladsl.Flow import akka.stream.scaladsl.Framing.FramingException @@ -19,11 +14,12 @@ import akka.testkit.AkkaSpec import akka.testkit.ImplicitSender import akka.util.ByteString -class TcpFramingSpec extends AkkaSpec with ImplicitSender { - import TcpFraming.encodeFrameHeader +import scala.util.Random - private val matSettings = ActorMaterializerSettings(system).withFuzzing(true) - private implicit val mat = ActorMaterializer(matSettings)(system) +class TcpFramingSpec extends AkkaSpec(""" + akka.stream.materializer.debug.fuzzing-mode = on + """) with ImplicitSender { + import TcpFraming.encodeFrameHeader private val afr = IgnoreEventSink diff --git a/akka-stream-testkit/src/test/scala/akka/stream/testkit/BaseTwoStreamsSetup.scala b/akka-stream-testkit/src/test/scala/akka/stream/testkit/BaseTwoStreamsSetup.scala index d36e219596..0fee49e9c2 100644 --- a/akka-stream-testkit/src/test/scala/akka/stream/testkit/BaseTwoStreamsSetup.scala +++ b/akka-stream-testkit/src/test/scala/akka/stream/testkit/BaseTwoStreamsSetup.scala @@ -4,19 +4,18 @@ package akka.stream.testkit -import akka.stream.{ ActorMaterializer, ActorMaterializerSettings } import akka.stream.scaladsl._ -import org.reactivestreams.Publisher -import scala.collection.immutable -import scala.util.control.NoStackTrace import akka.stream.testkit.scaladsl.StreamTestKit._ import akka.testkit.AkkaSpec +import org.reactivestreams.Publisher -abstract class BaseTwoStreamsSetup extends AkkaSpec { +import scala.collection.immutable +import scala.util.control.NoStackTrace - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 2) - - implicit val materializer = ActorMaterializer(settings) +abstract class BaseTwoStreamsSetup extends AkkaSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + akka.stream.materializer.max-input-buffer-size = 2 + """) { val TestException = new RuntimeException("test") with NoStackTrace diff --git a/akka-stream-testkit/src/test/scala/akka/stream/testkit/ScriptedTest.scala b/akka-stream-testkit/src/test/scala/akka/stream/testkit/ScriptedTest.scala index f465f033f8..1e86eadd8c 100644 --- a/akka-stream-testkit/src/test/scala/akka/stream/testkit/ScriptedTest.scala +++ b/akka-stream-testkit/src/test/scala/akka/stream/testkit/ScriptedTest.scala @@ -18,6 +18,7 @@ import scala.concurrent.duration._ import java.util.concurrent.ThreadLocalRandom import akka.stream.SystemMaterializer +import com.github.ghik.silencer.silent trait ScriptedTest extends Matchers { @@ -230,6 +231,7 @@ trait ScriptedTest extends Matchers { } + @silent("deprecated") def runScript[In, Out, M](script: Script[In, Out])(op: Flow[In, In, NotUsed] => Flow[In, Out, M])( implicit system: ActorSystem): Unit = runScript(script, SystemMaterializer(system).materializer.settings)(op)(system) diff --git a/akka-stream-testkit/src/test/scala/akka/stream/testkit/TestPublisherSubscriberSpec.scala b/akka-stream-testkit/src/test/scala/akka/stream/testkit/TestPublisherSubscriberSpec.scala index 2552e7adb0..71fe9476e0 100644 --- a/akka-stream-testkit/src/test/scala/akka/stream/testkit/TestPublisherSubscriberSpec.scala +++ b/akka-stream-testkit/src/test/scala/akka/stream/testkit/TestPublisherSubscriberSpec.scala @@ -4,26 +4,25 @@ package akka.stream.testkit -import akka.stream.scaladsl.{ Sink, Source } +import akka.stream.scaladsl.Sink +import akka.stream.scaladsl.Source import akka.stream.testkit.TestPublisher._ import akka.stream.testkit.TestSubscriber._ import akka.stream.testkit.scaladsl.StreamTestKit._ -import akka.stream.{ ActorMaterializer, ActorMaterializerSettings } -import org.reactivestreams.Subscription import akka.testkit.AkkaSpec +import org.reactivestreams.Subscription -class TestPublisherSubscriberSpec extends AkkaSpec { - - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 2) - - implicit val materializer = ActorMaterializer(settings) +class TestPublisherSubscriberSpec extends AkkaSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + akka.stream.materializer.max-input-buffer-size = 2 + """) { "TestPublisher and TestSubscriber" must { "have all events accessible from manual probes" in assertAllStagesStopped { val upstream = TestPublisher.manualProbe[Int]() val downstream = TestSubscriber.manualProbe[Int]() - Source.fromPublisher(upstream).runWith(Sink.asPublisher(false))(materializer).subscribe(downstream) + Source.fromPublisher(upstream).runWith(Sink.asPublisher(false)).subscribe(downstream) val upstreamSubscription = upstream.expectSubscription() val downstreamSubscription: Subscription = downstream.expectEventPF { case OnSubscribe(sub) => sub } @@ -47,7 +46,7 @@ class TestPublisherSubscriberSpec extends AkkaSpec { "handle gracefully partial function that is not suitable" in assertAllStagesStopped { val upstream = TestPublisher.manualProbe[Int]() val downstream = TestSubscriber.manualProbe[Int]() - Source.fromPublisher(upstream).runWith(Sink.asPublisher(false))(materializer).subscribe(downstream) + Source.fromPublisher(upstream).runWith(Sink.asPublisher(false)).subscribe(downstream) val upstreamSubscription = upstream.expectSubscription() val downstreamSubscription: Subscription = downstream.expectEventPF { case OnSubscribe(sub) => sub } diff --git a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/ActorSystemLifecycle.scala b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/ActorSystemLifecycle.scala index 0b8c2e6765..cfe3931ce2 100644 --- a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/ActorSystemLifecycle.scala +++ b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/ActorSystemLifecycle.scala @@ -5,6 +5,7 @@ package akka.stream.tck import java.util.concurrent.TimeoutException + import scala.concurrent.duration._ import akka.actor.ActorSystem import akka.actor.ActorSystemImpl @@ -13,20 +14,25 @@ import akka.testkit.AkkaSpec import akka.event.Logging import akka.testkit.TestEvent import akka.testkit.EventFilter +import com.typesafe.config.Config +import com.typesafe.config.ConfigFactory import org.testng.annotations.BeforeClass + import scala.concurrent.Await trait ActorSystemLifecycle { protected var _system: ActorSystem = _ - final def system: ActorSystem = _system + implicit final def system: ActorSystem = _system + + def additionalConfig: Config = ConfigFactory.empty() def shutdownTimeout: FiniteDuration = 10.seconds @BeforeClass def createActorSystem(): Unit = { - _system = ActorSystem(Logging.simpleName(getClass), AkkaSpec.testConf) + _system = ActorSystem(Logging.simpleName(getClass), additionalConfig.withFallback(AkkaSpec.testConf)) _system.eventStream.publish(TestEvent.Mute(EventFilter[RuntimeException]("Test exception"))) } diff --git a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/AkkaPublisherVerification.scala b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/AkkaPublisherVerification.scala index 6b8d8889c8..e0d4208fda 100644 --- a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/AkkaPublisherVerification.scala +++ b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/AkkaPublisherVerification.scala @@ -4,19 +4,27 @@ package akka.stream.tck -import scala.collection.immutable -import akka.stream.ActorMaterializerSettings -import akka.stream.ActorMaterializer import akka.stream.testkit.TestPublisher +import com.typesafe.config.Config +import com.typesafe.config.ConfigFactory import org.reactivestreams.Publisher -import org.reactivestreams.tck.{ PublisherVerification, TestEnvironment } +import org.reactivestreams.tck.PublisherVerification +import org.reactivestreams.tck.TestEnvironment import org.scalatestplus.testng.TestNGSuiteLike +import scala.collection.immutable + abstract class AkkaPublisherVerification[T](val env: TestEnvironment, publisherShutdownTimeout: Long) extends PublisherVerification[T](env, publisherShutdownTimeout) with TestNGSuiteLike with ActorSystemLifecycle { + override def additionalConfig: Config = + ConfigFactory.parseString(""" + akka.stream.materializer.initial-input-buffer-size = 512 + akka.stream.materializer.max-input-buffer-size = 512 + """) + def this(printlnDebug: Boolean) = this( new TestEnvironment(Timeouts.defaultTimeoutMillis, Timeouts.defaultNoSignalsTimeoutMillis, printlnDebug), @@ -24,9 +32,6 @@ abstract class AkkaPublisherVerification[T](val env: TestEnvironment, publisherS def this() = this(false) - implicit lazy val materializer = - ActorMaterializer(ActorMaterializerSettings(system).withInputBuffer(initialSize = 512, maxSize = 512))(system) - override def createFailedPublisher(): Publisher[T] = TestPublisher.error(new Exception("Unable to serve subscribers right now!")) diff --git a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/AkkaSubscriberVerification.scala b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/AkkaSubscriberVerification.scala index 01bc71b48c..7130773548 100644 --- a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/AkkaSubscriberVerification.scala +++ b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/AkkaSubscriberVerification.scala @@ -5,8 +5,6 @@ package akka.stream.tck import akka.actor.ActorSystem -import akka.stream.ActorMaterializerSettings -import akka.stream.ActorMaterializer import org.reactivestreams.tck.SubscriberBlackboxVerification import org.reactivestreams.tck.SubscriberWhiteboxVerification import org.reactivestreams.tck.TestEnvironment @@ -37,6 +35,4 @@ abstract class AkkaSubscriberWhiteboxVerification[T](env: TestEnvironment) trait AkkaSubscriberVerificationLike { implicit def system: ActorSystem - - implicit lazy val materializer = ActorMaterializer(ActorMaterializerSettings(system)) } diff --git a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/FusableProcessorTest.scala b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/FusableProcessorTest.scala index e7d10194e3..2b993b6e86 100644 --- a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/FusableProcessorTest.scala +++ b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/FusableProcessorTest.scala @@ -11,12 +11,11 @@ import org.reactivestreams.Processor class FusableProcessorTest extends AkkaIdentityProcessorVerification[Int] { override def createIdentityProcessor(maxBufferSize: Int): Processor[Int, Int] = { - val settings = - ActorMaterializerSettings(system).withInputBuffer(initialSize = maxBufferSize / 2, maxSize = maxBufferSize) - - implicit val materializer = ActorMaterializer(settings)(system) - - Flow[Int].map(identity).toProcessor.run() + Flow[Int] + .map(identity) + .toProcessor + .withAttributes(Attributes.inputBuffer(initial = maxBufferSize / 2, max = maxBufferSize)) + .run() } override def createElement(element: Int): Int = element diff --git a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/TransformProcessorTest.scala b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/TransformProcessorTest.scala index 8d015612ce..6859a3f306 100644 --- a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/TransformProcessorTest.scala +++ b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/TransformProcessorTest.scala @@ -4,20 +4,17 @@ package akka.stream.tck +import akka.stream.Attributes import akka.stream.impl.fusing.GraphStages.SimpleLinearGraphStage -import akka.stream.{ ActorMaterializer, ActorMaterializerSettings, Attributes } import akka.stream.scaladsl.Flow -import akka.stream.stage.{ GraphStageLogic, InHandler, OutHandler } +import akka.stream.stage.GraphStageLogic +import akka.stream.stage.InHandler +import akka.stream.stage.OutHandler import org.reactivestreams.Processor class TransformProcessorTest extends AkkaIdentityProcessorVerification[Int] { override def createIdentityProcessor(maxBufferSize: Int): Processor[Int, Int] = { - val settings = - ActorMaterializerSettings(system).withInputBuffer(initialSize = maxBufferSize / 2, maxSize = maxBufferSize) - - implicit val materializer = ActorMaterializer(settings)(system) - val stage = new SimpleLinearGraphStage[Int] { override def createLogic(inheritedAttributes: Attributes) = @@ -28,7 +25,11 @@ class TransformProcessorTest extends AkkaIdentityProcessorVerification[Int] { } } - Flow[Int].via(stage).toProcessor.run() + Flow[Int] + .via(stage) + .toProcessor + .withAttributes(Attributes.inputBuffer(initial = maxBufferSize / 2, max = maxBufferSize)) + .run() } override def createElement(element: Int): Int = element diff --git a/akka-stream-tests/src/test/java/akka/stream/StreamAttributeDocTest.java b/akka-stream-tests/src/test/java/akka/stream/StreamAttributeDocTest.java new file mode 100644 index 0000000000..f439490cb3 --- /dev/null +++ b/akka-stream-tests/src/test/java/akka/stream/StreamAttributeDocTest.java @@ -0,0 +1,66 @@ +/* + * Copyright (C) 2009-2019 Lightbend Inc. + */ + +package akka.stream; + +import akka.Done; +import akka.NotUsed; +import akka.stream.javadsl.Keep; +import akka.stream.javadsl.RunnableGraph; +import akka.stream.javadsl.Sink; +import akka.stream.javadsl.Source; +import akka.stream.scaladsl.TcpAttributes; +import akka.testkit.AkkaJUnitActorSystemResource; +import akka.testkit.AkkaSpec; +import com.typesafe.config.ConfigFactory; +import org.junit.ClassRule; +import org.junit.Test; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.util.concurrent.CompletionStage; +import java.util.concurrent.TimeUnit; + +public class StreamAttributeDocTest extends StreamTest { + + public StreamAttributeDocTest() { + super(actorSystemResource); + } + + @ClassRule + public static AkkaJUnitActorSystemResource actorSystemResource = + new AkkaJUnitActorSystemResource( + "StreamAttributeDocTest", + ConfigFactory.parseString("my-stream-dispatcher = akka.test.stream-dispatcher") + .withFallback(AkkaSpec.testConf())); + + @Test + public void runnableAttributesExample() throws Exception { + final PrintStream oldOut = System.out; + // no stdout from tests thank you + System.setOut(new PrintStream(new ByteArrayOutputStream())); + try { + + // #attributes-on-stream + RunnableGraph> stream = + Source.range(1, 10) + .map(Object::toString) + .toMat(Sink.foreach(System.out::println), Keep.right()) + .withAttributes( + Attributes.inputBuffer(4, 4) + .and(ActorAttributes.dispatcher("my-stream-dispatcher")) + .and(TcpAttributes.tcpWriteBufferSize(2048))); + + // #attributes-on-stream + CompletionStage done = + // #attributes-on-stream + stream.run(system); + // #attributes-on-stream + + done.toCompletableFuture().get(3, TimeUnit.SECONDS); + } finally { + System.setOut(oldOut); + } + } +} 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 741129b500..be159e44a5 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/ActorMaterializerSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/ActorMaterializerSpec.scala @@ -95,6 +95,8 @@ class ActorMaterializerSpec extends StreamSpec with ImplicitSender { } object ActorMaterializerSpec { + + @silent("deprecated") class ActorWithMaterializer(p: TestProbe) extends Actor { private val settings: ActorMaterializerSettings = ActorMaterializerSettings(context.system).withDispatcher("akka.test.stream-dispatcher") diff --git a/akka-stream-tests/src/test/scala/akka/stream/StreamAttributeDocSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/StreamAttributeDocSpec.scala new file mode 100644 index 0000000000..91208d0fd6 --- /dev/null +++ b/akka-stream-tests/src/test/scala/akka/stream/StreamAttributeDocSpec.scala @@ -0,0 +1,44 @@ +/* + * Copyright (C) 2009-2019 Lightbend Inc. + */ + +package akka.stream + +import akka.Done +import akka.stream.scaladsl.Keep +import akka.stream.scaladsl.RunnableGraph +import akka.stream.scaladsl.Sink +import akka.stream.scaladsl.Source +import akka.stream.scaladsl.TcpAttributes +import akka.stream.testkit.StreamSpec + +import scala.concurrent.Future + +class StreamAttributeDocSpec extends StreamSpec("my-stream-dispatcher = \"akka.test.stream-dispatcher\"") { + + "Setting attributes on the runnable stream" must { + + "be shown" in { + // no stdout from tests thank you + val println = (_: Any) => () + + val done = { + // #attributes-on-stream + val stream: RunnableGraph[Future[Done]] = + Source(1 to 10) + .map(_.toString) + .toMat(Sink.foreach(println))(Keep.right) + .withAttributes(Attributes.inputBuffer(4, 4) and + ActorAttributes.dispatcher("my-stream-dispatcher") and + TcpAttributes.tcpWriteBufferSize(2048)) + + stream.run() + // #attributes-on-stream + } + done.futureValue // block until stream is done + + } + + } + +} diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/FixedBufferSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/FixedBufferSpec.scala index 6b75bc7c17..e103fb1583 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/impl/FixedBufferSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/impl/FixedBufferSpec.scala @@ -4,9 +4,13 @@ package akka.stream.impl -import akka.stream.testkit.StreamSpec +import akka.stream.ActorAttributes +import akka.stream.ActorAttributes.MaxFixedBufferSize import akka.stream.ActorMaterializerSettings +import akka.stream.testkit.StreamSpec +import com.github.ghik.silencer.silent +@silent("deprecated") class FixedBufferSpec extends StreamSpec { for (size <- List(1, 3, 4)) { @@ -114,10 +118,10 @@ class FixedBufferSpec extends StreamSpec { } "Buffer factory" must { - val default = ActorMaterializerSettings(system) + val default = ActorMaterializerSettings(system).toAttributes "default to one billion for maxFixedBufferSize" in { - default.maxFixedBufferSize should ===(1000000000) + default.mandatoryAttribute[MaxFixedBufferSize].size should ===(1000000000) } "produce BoundedBuffers when capacity > max-fixed-buffer-size" in { @@ -130,7 +134,7 @@ class FixedBufferSpec extends StreamSpec { } "produce FixedSizeBuffers when max-fixed-buffer-size < BoundedBufferSize" in { - val settings = default.withMaxFixedBufferSize(9) + val settings = default and ActorAttributes.maxFixedBufferSize(9) Buffer(5, settings) shouldBe a[FixedSizeBuffer.ModuloFixedSizeBuffer[_]] Buffer(10, settings) shouldBe a[FixedSizeBuffer.ModuloFixedSizeBuffer[_]] Buffer(16, settings) shouldBe a[FixedSizeBuffer.PowerOfTwoFixedSizeBuffer[_]] diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/TimeoutsSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/TimeoutsSpec.scala index 755094c19d..b5175d2f8b 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/impl/TimeoutsSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/impl/TimeoutsSpec.scala @@ -7,15 +7,19 @@ package akka.stream.impl import java.util.concurrent.TimeoutException import akka.Done +import akka.stream._ import akka.stream.scaladsl._ +import akka.stream.testkit.StreamSpec +import akka.stream.testkit.TestPublisher +import akka.stream.testkit.TestSubscriber import akka.stream.testkit.Utils._ import akka.stream.testkit.scaladsl.StreamTestKit._ -import akka.stream.testkit.{ StreamSpec, TestPublisher, TestSubscriber } -import akka.stream._ -import org.scalatest.{ Matchers, WordSpecLike } +import org.scalatest.Matchers +import org.scalatest.WordSpecLike +import scala.concurrent.Await +import scala.concurrent.Future import scala.concurrent.duration._ -import scala.concurrent.{ Await, Future } class TimeoutsSpec extends StreamSpec { @@ -337,20 +341,29 @@ class TimeoutsSpec extends StreamSpec { "Subscription timeouts" must { - implicit val materializer = ActorMaterializer( - ActorMaterializerSettings(system).withSubscriptionTimeoutSettings( - StreamSubscriptionTimeoutSettings(StreamSubscriptionTimeoutTerminationMode.cancel, 100.millis))) + val subscriptionTimeout = + ActorAttributes.streamSubscriptionTimeout(100.millis, StreamSubscriptionTimeoutTerminationMode.cancel) "be effective for dangling downstream (no fanout)" in assertAllStagesStopped { val upstream = TestPublisher.probe() - val (sub, _) = Flow[Int].map(_.toString).runWith(Source.asSubscriber, Sink.asPublisher(false)) + val (sub, _) = + Source.asSubscriber + .viaMat(Flow[Int].map(_.toString))(Keep.left) + .toMat(Sink.asPublisher(fanout = false))(Keep.both) + .withAttributes(subscriptionTimeout) + .run() upstream.subscribe(sub) upstream.expectCancellation() } "be effective for dangling downstream (with fanout)" in assertAllStagesStopped { val upstream = TestPublisher.probe() - val (sub, _) = Flow[Int].map(_.toString).runWith(Source.asSubscriber, Sink.asPublisher(true)) + val (sub, _) = + Source.asSubscriber + .viaMat(Flow[Int].map(_.toString))(Keep.left) + .toMat(Sink.asPublisher(fanout = true))(Keep.both) + .withAttributes(subscriptionTimeout) + .run() upstream.subscribe(sub) upstream.expectCancellation() } @@ -358,7 +371,12 @@ class TimeoutsSpec extends StreamSpec { // this one seems close to impossible to actually implement "be effective for dangling upstream" in pendingUntilFixed(assertAllStagesStopped { val downstream = TestSubscriber.probe[String]() - val (_, pub) = Flow[Int].map(_.toString).runWith(Source.asSubscriber, Sink.asPublisher(false)) + val (_, pub) = + Source.asSubscriber + .viaMat(Flow[Int].map(_.toString))(Keep.left) + .toMat(Sink.asPublisher(fanout = false))(Keep.both) + .withAttributes(subscriptionTimeout) + .run() pub.subscribe(downstream) downstream.ensureSubscription() downstream.expectError() shouldBe a[SubscriptionTimeoutException] diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/ActorGraphInterpreterSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/ActorGraphInterpreterSpec.scala index 2e3fbb3c55..dc5f3c81cc 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/ActorGraphInterpreterSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/ActorGraphInterpreterSpec.scala @@ -11,15 +11,24 @@ import akka.stream._ import akka.stream.impl.ReactiveStreamsCompliance.SpecViolation import akka.stream.impl.fusing.GraphStages.SimpleLinearGraphStage import akka.stream.scaladsl._ -import akka.stream.stage.{ GraphStage, GraphStageLogic, InHandler, OutHandler } +import akka.stream.stage.GraphStage +import akka.stream.stage.GraphStageLogic +import akka.stream.stage.InHandler +import akka.stream.stage.OutHandler import akka.stream.testkit.Utils._ import akka.stream.testkit.scaladsl.StreamTestKit._ -import akka.stream.testkit.{ StreamSpec, TestPublisher, TestSubscriber } -import akka.testkit.{ EventFilter, TestLatch } +import akka.stream.testkit.StreamSpec +import akka.stream.testkit.TestPublisher +import akka.stream.testkit.TestSubscriber +import akka.testkit.EventFilter +import akka.testkit.TestLatch +import org.reactivestreams.Publisher +import org.reactivestreams.Subscriber +import org.reactivestreams.Subscription -import scala.concurrent.{ Await, Promise } import scala.concurrent.duration._ -import org.reactivestreams.{ Publisher, Subscriber, Subscription } +import scala.concurrent.Await +import scala.concurrent.Promise class ActorGraphInterpreterSpec extends StreamSpec { "ActorGraphInterpreter" must { @@ -265,10 +274,6 @@ class ActorGraphInterpreterSpec extends StreamSpec { "be able to properly handle case where a stage fails before subscription happens" in assertAllStagesStopped { - // Fuzzing needs to be off, so that the failure can propagate to the output boundary before the ExposedPublisher - // message. - val noFuzzMat = ActorMaterializer(ActorMaterializerSettings(system).withFuzzing(false)) - val te = TE("Test failure in preStart") val evilLatch = new CountDownLatch(1) @@ -321,7 +326,10 @@ class ActorGraphInterpreterSpec extends StreamSpec { ClosedShape }) - .run()(noFuzzMat) + // Fuzzing needs to be off, so that the failure can propagate to the output boundary before the ExposedPublisher + // message. + .withAttributes(ActorAttributes.fuzzingMode(false)) + .run() evilLatch.countDown() downstream0.expectSubscriptionAndError(te) diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/AsyncCallbackSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/AsyncCallbackSpec.scala index 07e70f843e..28aa075bac 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/AsyncCallbackSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/AsyncCallbackSpec.scala @@ -7,18 +7,24 @@ package akka.stream.impl.fusing import akka.Done import akka.actor.ActorRef import akka.stream._ -import akka.stream.scaladsl.{ Keep, Sink, Source } +import akka.stream.scaladsl.Keep +import akka.stream.scaladsl.Sink +import akka.stream.scaladsl.Source import akka.stream.stage._ import akka.stream.testkit.Utils.TE -import akka.stream.testkit.{ TestPublisher, TestSubscriber } -import akka.testkit.{ AkkaSpec, TestProbe } +import akka.stream.testkit.TestPublisher +import akka.stream.testkit.TestSubscriber +import akka.testkit.AkkaSpec +import akka.testkit.TestProbe -import scala.concurrent.{ Await, Future, Promise } +import scala.concurrent.Await +import scala.concurrent.Future +import scala.concurrent.Promise import scala.language.reflectiveCalls -class AsyncCallbackSpec extends AkkaSpec { - - implicit val materializer = ActorMaterializer(ActorMaterializerSettings(system).withFuzzing(false)) +class AsyncCallbackSpec extends AkkaSpec(""" + akka.stream.materializer.debug.fuzzing-mode = off + """) { case object Started case class Elem(n: Int) diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/ChasingEventsSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/ChasingEventsSpec.scala index 739c46f68a..0847c841e4 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/ChasingEventsSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/ChasingEventsSpec.scala @@ -4,16 +4,21 @@ package akka.stream.impl.fusing -import akka.stream.scaladsl.{ Sink, Source } import akka.stream._ -import akka.stream.stage.{ GraphStage, GraphStageLogic, InHandler, OutHandler } +import akka.stream.scaladsl.Sink +import akka.stream.scaladsl.Source +import akka.stream.stage.GraphStage +import akka.stream.stage.GraphStageLogic +import akka.stream.stage.InHandler +import akka.stream.stage.OutHandler import akka.stream.testkit.Utils.TE -import akka.stream.testkit.{ TestPublisher, TestSubscriber } +import akka.stream.testkit.TestPublisher +import akka.stream.testkit.TestSubscriber import akka.testkit.AkkaSpec -class ChasingEventsSpec extends AkkaSpec { - - implicit val materializer = ActorMaterializer(ActorMaterializerSettings(system).withFuzzing(false)) +class ChasingEventsSpec extends AkkaSpec(""" + akka.stream.materializer.debug.fuzzing-mode = off + """) { class CancelInChasedPull extends GraphStage[FlowShape[Int, Int]] { val in = Inlet[Int]("Propagate.in") diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/GraphInterpreterSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/GraphInterpreterSpec.scala index 96e29ae139..3d6ecab805 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/GraphInterpreterSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/GraphInterpreterSpec.scala @@ -4,7 +4,6 @@ package akka.stream.impl.fusing -import akka.stream.OverflowStrategy import akka.stream.scaladsl.{ Balance, Broadcast, Merge, Zip } import akka.stream.testkit.StreamSpec @@ -306,34 +305,6 @@ class GraphInterpreterSpec extends StreamSpec with GraphInterpreterSpecKit { interpreter.isSuspended should be(false) } - "implement buffer" in new TestSetup { - val source = new UpstreamProbe[String]("source") - val sink = new DownstreamProbe[String]("sink") - val buffer = Buffer[String](2, OverflowStrategy.backpressure) - - builder(buffer).connect(source, buffer.in).connect(buffer.out, sink).init() - - stepAll() - lastEvents() should ===(Set(RequestOne(source))) - - sink.requestOne() - lastEvents() should ===(Set.empty) - - source.onNext("A") - lastEvents() should ===(Set(RequestOne(source), OnNext(sink, "A"))) - - source.onNext("B") - lastEvents() should ===(Set(RequestOne(source))) - - source.onNext("C", eventLimit = 0) - sink.requestOne() - lastEvents() should ===(Set(OnNext(sink, "B"), RequestOne(source))) - - sink.requestOne(eventLimit = 0) - source.onComplete(eventLimit = 3) - // OnComplete arrives early due to push chasing - lastEvents() should ===(Set(OnNext(sink, "C"), OnComplete(sink))) - } } } 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 f1ffdc953a..e6b7edb4c8 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 @@ -30,9 +30,6 @@ import scala.util.control.NoStackTrace class InputStreamSinkSpec extends StreamSpec(UnboundedMailboxConfig) { import system.dispatcher - val settings = ActorMaterializerSettings(system).withDispatcher("akka.actor.default-dispatcher") - implicit val materializer = ActorMaterializer(settings) - val timeout = 300.milliseconds def randomByteString(size: Int): ByteString = { val a = new Array[Byte](size) 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 776444b8ee..f0ddb45e53 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 @@ -11,10 +11,12 @@ import java.util.concurrent.TimeoutException import akka.stream.Attributes.inputBuffer import akka.stream._ import akka.stream.impl.io.OutputStreamSourceStage -import akka.stream.scaladsl.{ Keep, Sink, StreamConverters } +import akka.stream.scaladsl.Keep +import akka.stream.scaladsl.Sink +import akka.stream.scaladsl.StreamConverters import akka.stream.testkit.Utils._ -import akka.stream.testkit.scaladsl.StreamTestKit._ import akka.stream.testkit._ +import akka.stream.testkit.scaladsl.StreamTestKit._ import akka.stream.testkit.scaladsl.TestSink import akka.util.ByteString @@ -28,9 +30,6 @@ class OutputStreamSourceSpec extends StreamSpec(UnboundedMailboxConfig) { import system.dispatcher - val settings = ActorMaterializerSettings(system).withDispatcher("akka.actor.default-dispatcher") - implicit val materializer = ActorMaterializer(settings) - val timeout = 3.seconds val bytesArray = Array.fill[Byte](3)(Random.nextInt(1024).asInstanceOf[Byte]) val byteString = ByteString(bytesArray) @@ -168,7 +167,7 @@ class OutputStreamSourceSpec extends StreamSpec(UnboundedMailboxConfig) { assertNoBlockedThreads() val (_, probe) = - StreamConverters.asOutputStream(timeout).toMat(TestSink.probe[ByteString])(Keep.both).run()(materializer) + StreamConverters.asOutputStream(timeout).toMat(TestSink.probe[ByteString])(Keep.both).run() val sub = probe.expectSubscription() @@ -181,7 +180,7 @@ class OutputStreamSourceSpec extends StreamSpec(UnboundedMailboxConfig) { } "not leave blocked threads when materializer shutdown" in { - val materializer2 = ActorMaterializer(settings) + val materializer2 = ActorMaterializer()(system) val (_, probe) = StreamConverters.asOutputStream(timeout).toMat(TestSink.probe[ByteString])(Keep.both).run()(materializer2) diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/TcpHelper.scala b/akka-stream-tests/src/test/scala/akka/stream/io/TcpHelper.scala index b17bc551c3..1a78799da2 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/io/TcpHelper.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/io/TcpHelper.scala @@ -4,17 +4,19 @@ package akka.stream.io -import akka.actor._ -import akka.io.Tcp.{ ConnectionClosed, ResumeReading } -import akka.io.{ IO, Tcp } -import akka.stream.testkit._ -import akka.stream.{ ActorMaterializer, ActorMaterializerSettings } -import akka.testkit.{ TestKitBase, TestProbe } -import akka.util.ByteString import java.net.InetSocketAddress -import scala.collection.immutable.Queue -import akka.testkit.SocketUtil.temporaryServerAddress +import akka.actor._ +import akka.io.Tcp.ConnectionClosed +import akka.io.Tcp.ResumeReading +import akka.io.IO +import akka.io.Tcp +import akka.stream.testkit._ +import akka.testkit.SocketUtil.temporaryServerAddress +import akka.testkit.TestProbe +import akka.util.ByteString + +import scala.collection.immutable.Queue import scala.concurrent.duration._ object TcpHelper { @@ -114,13 +116,9 @@ object TcpHelper { } -trait TcpHelper { this: TestKitBase => +trait TcpHelper { this: TcpSpec => import akka.stream.io.TcpHelper._ - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 4, maxSize = 4) - - implicit val materializer = ActorMaterializer(settings) - class Server(val address: InetSocketAddress = temporaryServerAddress()) { val serverProbe = TestProbe() val serverRef = system.actorOf(testServerProps(address, serverProbe.ref)) diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/TcpSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/io/TcpSpec.scala index 99496bd757..071a881efc 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/io/TcpSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/io/TcpSpec.scala @@ -84,6 +84,8 @@ class TcpSpec extends StreamSpec(""" akka.loggers = ["akka.testkit.SilenceAllTestEventListener"] akka.io.tcp.trace-logging = true akka.stream.materializer.subscription-timeout.timeout = 2s + akka.stream.materializer.initial-input-buffer-size = 2 + akka.stream.materializer.max-input-buffer-size = 2 """) with TcpHelper with WithLogCapturing { "Outgoing TCP stream" must { @@ -671,15 +673,11 @@ class TcpSpec extends StreamSpec(""" val config = ConfigFactory.parseString(""" akka.actor.serializer-messages = off akka.io.tcp.register-timeout = 42s + akka.stream.materializer.subscription-timeout.mode = cancel + akka.stream.materializer.subscription-timeout.timeout = 42s """) val serverSystem = ActorSystem("server", config) val clientSystem = ActorSystem("client", config) - val serverMaterializer = ActorMaterializer( - ActorMaterializerSettings(serverSystem).withSubscriptionTimeoutSettings( - StreamSubscriptionTimeoutSettings(StreamSubscriptionTimeoutTerminationMode.cancel, 42.seconds)))(serverSystem) - val clientMaterializer = ActorMaterializer( - ActorMaterializerSettings(clientSystem).withSubscriptionTimeoutSettings( - StreamSubscriptionTimeoutSettings(StreamSubscriptionTimeoutTerminationMode.cancel, 42.seconds)))(clientSystem) try { @@ -716,7 +714,7 @@ class TcpSpec extends StreamSpec(""" } } .to(Sink.ignore) - .run()(serverMaterializer) + .run()(SystemMaterializer(serverSystem).materializer) // make sure server is running first futureBinding.futureValue @@ -724,7 +722,7 @@ class TcpSpec extends StreamSpec(""" // then connect once, which should lead to the server cancelling val total = Source(immutable.Iterable.fill(100)(ByteString(0))) .via(Tcp(clientSystem).outgoingConnection(address)) - .runFold(0)(_ + _.size)(clientMaterializer) + .runFold(0)(_ + _.size)(SystemMaterializer(clientSystem).materializer) serverGotRequest.future.futureValue // this can take a bit of time worst case but is often swift diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/CollectionSinkSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/CollectionSinkSpec.scala index bd5da0eb8a..426646fb00 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/CollectionSinkSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/CollectionSinkSpec.scala @@ -4,17 +4,18 @@ package akka.stream.scaladsl -import akka.stream.testkit.{ StreamSpec, TestPublisher } -import akka.stream.{ AbruptTerminationException, ActorMaterializer, ActorMaterializerSettings } +import akka.stream.testkit.StreamSpec +import akka.stream.testkit.TestPublisher +import akka.stream.AbruptTerminationException +import akka.stream.ActorMaterializer import scala.collection.immutable -import scala.concurrent.{ Await, Future } +import scala.concurrent.Await +import scala.concurrent.Future -class CollectionSinkSpec extends StreamSpec { - - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 16) - - implicit val mat = ActorMaterializer(settings) +class CollectionSinkSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + """) { "Sink.collection" when { "using Seq as Collection" must { diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/CoupledTerminationFlowSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/CoupledTerminationFlowSpec.scala index f9184c55a5..203f92877e 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/CoupledTerminationFlowSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/CoupledTerminationFlowSpec.scala @@ -4,23 +4,27 @@ package akka.stream.scaladsl -import akka.{ Done, NotUsed } import akka.stream._ import akka.stream.testkit._ import akka.stream.testkit.scaladsl.TestSource import akka.testkit.TestProbe -import org.reactivestreams.{ Publisher, Subscriber, Subscription } +import akka.Done +import akka.NotUsed +import org.reactivestreams.Publisher +import org.reactivestreams.Subscriber +import org.reactivestreams.Subscription import org.scalatest.Assertion import scala.concurrent.Future -import scala.util.{ Failure, Success, Try } +import scala.util.Failure +import scala.util.Success +import scala.util.Try import scala.xml.Node -class CoupledTerminationFlowSpec extends StreamSpec with ScriptedTest { +class CoupledTerminationFlowSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + """) with ScriptedTest { - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 16) - - implicit val materializer = ActorMaterializer(settings) import system.dispatcher /** diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowBatchSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowBatchSpec.scala index 1cc7fe2a4b..32098228af 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowBatchSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowBatchSpec.scala @@ -4,17 +4,18 @@ package akka.stream.scaladsl -import scala.concurrent.Await -import scala.concurrent.duration._ import java.util.concurrent.ThreadLocalRandom -import akka.stream.{ ActorMaterializer, ActorMaterializerSettings, OverflowStrategy } + +import akka.stream.OverflowStrategy import akka.stream.testkit._ -class FlowBatchSpec extends StreamSpec { +import scala.concurrent.Await +import scala.concurrent.duration._ - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 2) - - implicit val materializer = ActorMaterializer(settings) +class FlowBatchSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + akka.stream.materializer.max-input-buffer-size = 2 + """) { "Batch" must { diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowBatchWeightedSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowBatchWeightedSpec.scala index 264f831977..e889305af1 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowBatchWeightedSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowBatchWeightedSpec.scala @@ -4,15 +4,14 @@ package akka.stream.scaladsl -import akka.stream.{ ActorMaterializer, ActorMaterializerSettings } import akka.stream.testkit._ + import scala.concurrent.duration._ -class FlowBatchWeightedSpec extends StreamSpec { - - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 2) - - implicit val materializer = ActorMaterializer(settings) +class FlowBatchWeightedSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + akka.stream.materializer.max-input-buffer-size = 2 + """) { "BatchWeighted" must { "Not aggregate heavy elements" in { diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowBufferSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowBufferSpec.scala index e3f173100d..aff349d5e0 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowBufferSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowBufferSpec.scala @@ -4,19 +4,20 @@ package akka.stream.scaladsl +import akka.stream.testkit._ +import akka.stream.testkit.scaladsl.StreamTestKit._ +import akka.stream.testkit.scaladsl._ +import akka.stream.BufferOverflowException +import akka.stream.OverflowStrategy + import scala.concurrent.Await import scala.concurrent.Future import scala.concurrent.duration._ -import akka.stream.{ ActorMaterializer, ActorMaterializerSettings, BufferOverflowException, OverflowStrategy } -import akka.stream.testkit._ -import akka.stream.testkit.scaladsl._ -import akka.stream.testkit.scaladsl.StreamTestKit._ -class FlowBufferSpec extends StreamSpec { - - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 1, maxSize = 1) - - implicit val materializer = ActorMaterializer(settings) +class FlowBufferSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 1 + akka.stream.materializer.max-input-buffer-size = 1 + """) { "Buffer" must { diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowConcatAllSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowConcatAllSpec.scala index 5dc9ea7089..92b6045340 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowConcatAllSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowConcatAllSpec.scala @@ -5,19 +5,16 @@ package akka.stream.scaladsl import akka.NotUsed +import akka.stream.testkit._ +import akka.stream.testkit.scaladsl.StreamTestKit._ import akka.util.ConstantFun import scala.util.control.NoStackTrace -import akka.stream.ActorMaterializer -import akka.stream.ActorMaterializerSettings -import akka.stream.testkit._ -import akka.stream.testkit.scaladsl.StreamTestKit._ -class FlowConcatAllSpec extends StreamSpec { - - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 2) - - implicit val materializer = ActorMaterializer(settings) +class FlowConcatAllSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + akka.stream.materializer.max-input-buffer-size = 2 + """) { "ConcatAll" must { diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowConflateSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowConflateSpec.scala index 6099e0474d..6f40f9069f 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowConflateSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowConflateSpec.scala @@ -4,22 +4,24 @@ package akka.stream.scaladsl +import java.util.concurrent.ThreadLocalRandom + import akka.stream.ActorAttributes.supervisionStrategy import akka.stream.Attributes.inputBuffer -import akka.stream.Supervision.{ restartingDecider, resumingDecider } +import akka.stream.Supervision.restartingDecider +import akka.stream.Supervision.resumingDecider +import akka.stream._ import akka.stream.testkit.Utils.TE +import akka.stream.testkit._ import akka.testkit.TestLatch + import scala.concurrent.Await import scala.concurrent.duration._ -import java.util.concurrent.ThreadLocalRandom -import akka.stream._ -import akka.stream.testkit._ -class FlowConflateSpec extends StreamSpec { - - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 2) - - implicit val materializer = ActorMaterializer(settings) +class FlowConflateSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + akka.stream.materializer.max-input-buffer-size = 2 + """) { "Conflate" must { diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDispatcherSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDispatcherSpec.scala index 0d0419c859..5209c60298 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDispatcherSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDispatcherSpec.scala @@ -4,11 +4,13 @@ package akka.stream.scaladsl -import akka.stream.testkit.StreamSpec -import akka.testkit.TestProbe import akka.stream.ActorMaterializer import akka.stream.ActorMaterializerSettings +import akka.stream.testkit.StreamSpec +import akka.testkit.TestProbe +import com.github.ghik.silencer.silent +@silent("deprecated") class FlowDispatcherSpec extends StreamSpec(s"my-dispatcher = $${akka.test.stream-dispatcher}") { val defaultSettings = ActorMaterializerSettings(system) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDropSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDropSpec.scala index ac9ec6d45d..e16ff7e2f2 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDropSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowDropSpec.scala @@ -5,15 +5,12 @@ package akka.stream.scaladsl import java.util.concurrent.ThreadLocalRandom.{ current => random } -import akka.stream.ActorMaterializer -import akka.stream.ActorMaterializerSettings + import akka.stream.testkit._ -class FlowDropSpec extends StreamSpec with ScriptedTest { - - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 16) - - implicit val materializer = ActorMaterializer(settings) +class FlowDropSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + """) with ScriptedTest { "A Drop" must { @@ -24,7 +21,7 @@ class FlowDropSpec extends StreamSpec with ScriptedTest { }: _*) TestConfig.RandomTestRange.foreach { _ => val d = Math.min(Math.max(random.nextInt(-10, 60), 0), 50) - runScript(script(d), settings)(_.drop(d)) + runScript(script(d))(_.drop(d)) } } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowExpandSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowExpandSpec.scala index dce965a9f2..f9789f014c 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowExpandSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowExpandSpec.scala @@ -4,31 +4,35 @@ package akka.stream.scaladsl +import java.util.concurrent.ThreadLocalRandom + +import akka.stream.ActorAttributes +import akka.stream.testkit._ +import akka.stream.testkit.scaladsl.TestSink +import akka.stream.testkit.scaladsl.TestSource + import scala.concurrent.Await import scala.concurrent.duration._ -import java.util.concurrent.ThreadLocalRandom -import akka.stream.{ ActorMaterializer, ActorMaterializerSettings } -import akka.stream.testkit._ -import akka.stream.testkit.scaladsl.TestSource -import akka.stream.testkit.scaladsl.TestSink -class FlowExpandSpec extends StreamSpec { - - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 2) - - implicit val materializer = ActorMaterializer(settings) +class FlowExpandSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + akka.stream.materializer.max-input-buffer-size = 2 + """) { "Expand" must { "pass-through elements unchanged when there is no rate difference" in { - // Shadow the fuzzed materializer (see the ordering guarantee needed by the for loop below). - implicit val materializer = ActorMaterializer(settings.withFuzzing(false)) - val publisher = TestPublisher.probe[Int]() val subscriber = TestSubscriber.probe[Int]() // Simply repeat the last element as an extrapolation step - Source.fromPublisher(publisher).expand(Iterator.single).to(Sink.fromSubscriber(subscriber)).run() + Source + .fromPublisher(publisher) + .expand(Iterator.single) + .to(Sink.fromSubscriber(subscriber)) + // Shadow the fuzzed materializer (see the ordering guarantee needed by the for loop below). + .withAttributes(ActorAttributes.fuzzingMode(false)) + .run() for (i <- 1 to 100) { // Order is important here: If the request comes first it will be extrapolated! diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowExtrapolateSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowExtrapolateSpec.scala index 641b75ec2f..327036a286 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowExtrapolateSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowExtrapolateSpec.scala @@ -7,24 +7,22 @@ package akka.stream.scaladsl import java.util.concurrent.ThreadLocalRandom import akka.stream.testkit._ -import akka.stream.testkit.scaladsl.{ TestSink, TestSource } -import akka.stream.{ ActorMaterializer, ActorMaterializerSettings } +import akka.stream.testkit.scaladsl.TestSink +import akka.stream.testkit.scaladsl.TestSource import scala.concurrent.Await import scala.concurrent.duration._ -class FlowExtrapolateSpec extends StreamSpec { - - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 2) - - implicit val materializer = ActorMaterializer(settings) +class FlowExtrapolateSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + akka.stream.materializer.max-input-buffer-size = 2 + # see the ordering guarantee needed by the for loop below + akka.stream.materializer.debug.fuzzing-mode = off + """) { "Extrapolate" must { "pass-through elements unchanged when there is no rate difference" in { - // Shadow the fuzzed materializer (see the ordering guarantee needed by the for loop below). - implicit val materializer = ActorMaterializer(settings.withFuzzing(false)) - val publisher = TestPublisher.probe[Int]() val subscriber = TestSubscriber.probe[Int]() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFilterSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFilterSpec.scala index b3316f9a63..e94c0b112a 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFilterSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFilterSpec.scala @@ -4,23 +4,20 @@ package akka.stream.scaladsl -import akka.stream.testkit.scaladsl.TestSink - import java.util.concurrent.ThreadLocalRandom.{ current => random } + import akka.stream.ActorAttributes._ +import akka.stream.Attributes import akka.stream.Supervision._ -import akka.stream.testkit.scaladsl.StreamTestKit._ -import akka.stream.ActorMaterializer -import akka.stream.ActorMaterializerSettings import akka.stream.testkit._ +import akka.stream.testkit.scaladsl.StreamTestKit._ +import akka.stream.testkit.scaladsl.TestSink import scala.util.control.NoStackTrace -class FlowFilterSpec extends StreamSpec with ScriptedTest { - - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 16) - - implicit val materializer = ActorMaterializer(settings) +class FlowFilterSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + """) with ScriptedTest { "A Filter" must { @@ -29,15 +26,16 @@ class FlowFilterSpec extends StreamSpec with ScriptedTest { Script(TestConfig.RandomTestRange.map { _ => val x = random.nextInt(); Seq(x) -> (if ((x & 1) == 0) Seq(x) else Seq()) }: _*) - TestConfig.RandomTestRange.foreach(_ => runScript(script, settings)(_.filter(_ % 2 == 0))) + TestConfig.RandomTestRange.foreach(_ => runScript(script)(_.filter(_ % 2 == 0))) } "not blow up with high request counts" in { - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 1, maxSize = 1) - implicit val materializer = ActorMaterializer(settings) - val probe = TestSubscriber.manualProbe[Int]() - Source(List.fill(1000)(0) ::: List(1)).filter(_ != 0).runWith(Sink.fromSubscriber(probe)) + Source(List.fill(1000)(0) ::: List(1)) + .filter(_ != 0) + .toMat(Sink.fromSubscriber(probe))(Keep.right) + .withAttributes(Attributes.inputBuffer(1, 1)) + .run() val subscription = probe.expectSubscription() for (_ <- 1 to 10000) { @@ -71,7 +69,7 @@ class FlowFilterSpec extends StreamSpec with ScriptedTest { val x = random.nextInt() Seq(x) -> (if ((x & 1) == 1) Seq(x) else Seq()) }: _*) - TestConfig.RandomTestRange.foreach(_ => runScript(script, settings)(_.filterNot(_ % 2 == 0))) + TestConfig.RandomTestRange.foreach(_ => runScript(script)(_.filterNot(_ % 2 == 0))) } } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFlattenMergeSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFlattenMergeSpec.scala index c26b19f597..e602ac0631 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFlattenMergeSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowFlattenMergeSpec.scala @@ -11,11 +11,11 @@ import akka.stream.impl.fusing.GraphStages.SingleSource import akka.stream.stage.GraphStage import akka.stream.stage.GraphStageLogic import akka.stream.stage.OutHandler +import akka.stream.testkit.StreamSpec +import akka.stream.testkit.TestPublisher import akka.stream.testkit.Utils.TE import akka.stream.testkit.scaladsl.StreamTestKit._ import akka.stream.testkit.scaladsl.TestSink -import akka.stream.testkit.StreamSpec -import akka.stream.testkit.TestPublisher import akka.testkit.TestLatch import akka.util.OptionVal import org.scalatest.exceptions.TestFailedException @@ -133,8 +133,6 @@ class FlowFlattenMergeSpec extends StreamSpec { } "cancel substreams when failing map function" in assertAllStagesStopped { - val settings = ActorMaterializerSettings(system).withSyncProcessingLimit(1).withInputBuffer(1, 1) - val mat = ActorMaterializer(settings) val p = TestPublisher.probe[Int]() val ex = new Exception("buh") val latch = TestLatch() @@ -145,7 +143,9 @@ class FlowFlattenMergeSpec extends StreamSpec { Await.ready(latch, 3.seconds) throw ex }) - .runWith(Sink.head)(mat) + .toMat(Sink.head)(Keep.right) + .withAttributes(ActorAttributes.syncProcessingLimit(1) and Attributes.inputBuffer(1, 1)) + .run() p.expectRequest() latch.countDown() p.expectCancellation() 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 826b390c17..f6e06b1f60 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 @@ -5,29 +5,30 @@ package akka.stream.scaladsl import java.util - -import akka.{ Done, NotUsed } -import akka.actor.ActorSystem -import akka.stream.Attributes._ -import akka.stream.impl.SinkModule -import akka.util.ByteString - -import scala.annotation.tailrec -import scala.concurrent.{ Await, Promise } -import scala.concurrent.duration._ -import akka.stream._ -import akka.stream.Supervision.resumingDecider -import akka.stream.impl.fusing.GroupBy -import akka.stream.testkit._ -import akka.stream.testkit.Utils._ -import akka.stream.testkit.scaladsl.StreamTestKit._ -import org.reactivestreams.Publisher -import org.scalatest.concurrent.PatienceConfiguration.Timeout -import akka.stream.testkit.scaladsl.TestSource -import akka.stream.testkit.scaladsl.TestSink import java.util.concurrent.ThreadLocalRandom +import akka.actor.ActorSystem +import akka.stream.Attributes._ +import akka.stream.Supervision.resumingDecider +import akka.stream._ +import akka.stream.impl.SinkModule +import akka.stream.impl.fusing.GroupBy +import akka.stream.testkit.Utils._ +import akka.stream.testkit._ +import akka.stream.testkit.scaladsl.StreamTestKit._ +import akka.stream.testkit.scaladsl.TestSink +import akka.stream.testkit.scaladsl.TestSource import akka.testkit.TestLatch +import akka.util.ByteString +import akka.Done +import akka.NotUsed +import org.reactivestreams.Publisher +import org.scalatest.concurrent.PatienceConfiguration.Timeout + +import scala.annotation.tailrec +import scala.concurrent.duration._ +import scala.concurrent.Await +import scala.concurrent.Promise object FlowGroupBySpec { @@ -38,13 +39,12 @@ object FlowGroupBySpec { } -class FlowGroupBySpec extends StreamSpec { +class FlowGroupBySpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + akka.stream.materializer.max-input-buffer-size = 2 + """) { import FlowGroupBySpec._ - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 2) - - implicit val materializer = ActorMaterializer(settings) - case class StreamPuppet(p: Publisher[Int]) { val probe = TestSubscriber.manualProbe[Int]() p.subscribe(probe) @@ -126,7 +126,7 @@ class FlowGroupBySpec extends StreamSpec { } } - "work in normal user scenario" in { + "work in normal user scenario" in assertAllStagesStopped { Source(List("Aaa", "Abb", "Bcc", "Cdd", "Cee")) .groupBy(3, _.substring(0, 1)) .grouped(10) @@ -137,7 +137,7 @@ class FlowGroupBySpec extends StreamSpec { .sortBy(_.head) should ===(List(List("Aaa", "Abb"), List("Bcc"), List("Cdd", "Cee"))) } - "fail when key function return null" in { + "fail when key function return null" in assertAllStagesStopped { val down = Source(List("Aaa", "Abb", "Bcc", "Cdd", "Cee")) .groupBy(3, e => if (e.startsWith("A")) null else e.substring(0, 1)) .grouped(10) @@ -266,7 +266,7 @@ class FlowGroupBySpec extends StreamSpec { upstreamSubscription.expectCancellation() } - "resume stream when groupBy function throws" in { + "resume stream when groupBy function throws" in assertAllStagesStopped { val publisherProbeProbe = TestPublisher.manualProbe[Int]() val exc = TE("test") val publisher = Source @@ -340,9 +340,10 @@ class FlowGroupBySpec extends StreamSpec { val ex = down.expectError() ex.getMessage should include("too many substreams") s1.expectError(ex) + up.expectCancellation() } - "resume when exceeding maxSubstreams" in { + "resume when exceeding maxSubstreams" in assertAllStagesStopped { val (up, down) = Flow[Int] .groupBy(0, identity) .mergeSubstreams @@ -353,6 +354,8 @@ class FlowGroupBySpec extends StreamSpec { up.sendNext(1) down.expectNoMessage(1.second) + up.sendComplete() + down.expectComplete() } "emit subscribe before completed" in assertAllStagesStopped { @@ -549,8 +552,6 @@ class FlowGroupBySpec extends StreamSpec { } "work with random demand" in assertAllStagesStopped { - val mat = ActorMaterializer(ActorMaterializerSettings(system).withInputBuffer(initialSize = 1, maxSize = 1)) - var blockingNextElement: ByteString = null.asInstanceOf[ByteString] val probes = new java.util.ArrayList[Promise[TestSubscriber.Probe[ByteString]]](100) @@ -604,11 +605,12 @@ class FlowGroupBySpec extends StreamSpec { } val publisherProbe = TestPublisher.manualProbe[ByteString]() - Source + val runnable = Source .fromPublisher[ByteString](publisherProbe) .groupBy(100, elem => Math.abs(elem.head % 100)) .to(Sink.fromGraph(new ProbeSink(none, SinkShape(Inlet("ProbeSink.in"))))) - .run()(mat) + + runnable.withAttributes(Attributes.inputBuffer(1, 1)).run() val upstreamSubscription = publisherProbe.expectSubscription() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGroupedSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGroupedSpec.scala index 3e940977df..f29eb189ed 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGroupedSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowGroupedSpec.scala @@ -4,15 +4,16 @@ package akka.stream.scaladsl -import scala.collection.immutable import java.util.concurrent.ThreadLocalRandom.{ current => random } -import akka.stream.ActorMaterializerSettings -import akka.stream.testkit.{ ScriptedTest, StreamSpec } +import akka.stream.testkit.ScriptedTest +import akka.stream.testkit.StreamSpec -class FlowGroupedSpec extends StreamSpec with ScriptedTest { +import scala.collection.immutable - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 16) +class FlowGroupedSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + """) with ScriptedTest { "A Grouped" must { @@ -25,7 +26,7 @@ class FlowGroupedSpec extends StreamSpec with ScriptedTest { Script(TestConfig.RandomTestRange.map { _ => randomTest(testLen) }: _*) - TestConfig.RandomTestRange.foreach(_ => runScript(script, settings)(_.grouped(testLen))) + TestConfig.RandomTestRange.foreach(_ => runScript(script)(_.grouped(testLen))) } "group with rest" in { @@ -34,7 +35,7 @@ class FlowGroupedSpec extends StreamSpec with ScriptedTest { Script(TestConfig.RandomTestRange.map { _ => randomTest(testLen) } :+ randomTest(1): _*) - TestConfig.RandomTestRange.foreach(_ => runScript(script, settings)(_.grouped(testLen))) + TestConfig.RandomTestRange.foreach(_ => runScript(script)(_.grouped(testLen))) } } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowIdleInjectSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowIdleInjectSpec.scala index 55ae51d9d9..835cf12622 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowIdleInjectSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowIdleInjectSpec.scala @@ -4,17 +4,17 @@ package akka.stream.scaladsl -import akka.stream.{ ActorMaterializer, ActorMaterializerSettings } -import akka.stream.testkit.{ StreamSpec, TestPublisher, TestSubscriber } import akka.stream.testkit.scaladsl.StreamTestKit._ +import akka.stream.testkit.StreamSpec +import akka.stream.testkit.TestPublisher +import akka.stream.testkit.TestSubscriber + import scala.concurrent.Await import scala.concurrent.duration._ -class FlowIdleInjectSpec extends StreamSpec { - - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 16) - - implicit val materializer = ActorMaterializer(settings) +class FlowIdleInjectSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + """) { "keepAlive" must { diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowInitialDelaySpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowInitialDelaySpec.scala index 16295a828e..47d636c2fe 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowInitialDelaySpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowInitialDelaySpec.scala @@ -5,17 +5,17 @@ package akka.stream.scaladsl import java.util.concurrent.TimeoutException -import akka.stream.{ ActorMaterializer, ActorMaterializerSettings } -import akka.stream.testkit.{ StreamSpec, TestSubscriber } + import akka.stream.testkit.scaladsl.StreamTestKit._ +import akka.stream.testkit.StreamSpec +import akka.stream.testkit.TestSubscriber + import scala.concurrent.Await import scala.concurrent.duration._ -class FlowInitialDelaySpec extends StreamSpec { - - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 16) - - implicit val materializer = ActorMaterializer(settings) +class FlowInitialDelaySpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + """) { "Flow initialDelay" must { diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowIntersperseSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowIntersperseSpec.scala index c5d549e761..685fb2d58d 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowIntersperseSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowIntersperseSpec.scala @@ -5,15 +5,14 @@ package akka.stream.scaladsl import akka.stream.testkit._ -import akka.stream.testkit.scaladsl.{ TestSink, TestSource } -import akka.stream.{ ActorMaterializer, ActorMaterializerSettings } +import akka.stream.testkit.scaladsl.TestSink +import akka.stream.testkit.scaladsl.TestSource + import scala.concurrent.duration._ -class FlowIntersperseSpec extends StreamSpec { - - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 16) - - implicit val materializer = ActorMaterializer(settings) +class FlowIntersperseSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + """) { "A Intersperse" must { "inject element between existing elements" in { diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowIteratorSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowIteratorSpec.scala index c8678d85bb..75fe413ea1 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowIteratorSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowIteratorSpec.scala @@ -5,13 +5,12 @@ package akka.stream.scaladsl import akka.NotUsed -import scala.collection.immutable -import scala.concurrent.duration._ -import akka.stream.ActorMaterializer -import akka.stream.ActorMaterializerSettings import akka.stream.testkit._ import akka.stream.testkit.scaladsl.StreamTestKit._ +import scala.collection.immutable +import scala.concurrent.duration._ + class FlowIteratorSpec extends AbstractFlowIteratorSpec { override def testName = "A Flow based on an iterator producing function" override def createSource(elements: Int): Source[Int, NotUsed] = @@ -23,8 +22,6 @@ class FlowIterableSpec extends AbstractFlowIteratorSpec { override def createSource(elements: Int): Source[Int, NotUsed] = Source(1 to elements) - implicit def mmaterializer = super.materializer - "produce onError when iterator throws" in { val iterable = new immutable.Iterable[Int] { override def iterator: Iterator[Int] = @@ -69,12 +66,10 @@ class FlowIterableSpec extends AbstractFlowIteratorSpec { } } -abstract class AbstractFlowIteratorSpec extends StreamSpec { - - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 2) - - private val m = ActorMaterializer(settings) - implicit final def materializer = m +abstract class AbstractFlowIteratorSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + akka.stream.materializer.max-input-buffer-size = 2 +""") { def testName: String diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowJoinSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowJoinSpec.scala index d2f6900e46..9b4de24617 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowJoinSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowJoinSpec.scala @@ -4,20 +4,18 @@ package akka.stream.scaladsl -import akka.stream.{ ActorMaterializer, ActorMaterializerSettings, FlowShape, OverflowStrategy } import akka.stream.testkit._ import akka.stream.testkit.scaladsl.StreamTestKit._ import akka.stream.testkit.scaladsl._ -import com.typesafe.config.ConfigFactory +import akka.stream.FlowShape +import akka.stream.OverflowStrategy import org.scalatest.time._ import scala.collection.immutable -class FlowJoinSpec extends StreamSpec(ConfigFactory.parseString("akka.loglevel=INFO")) { - - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 16) - - implicit val materializer = ActorMaterializer(settings) +class FlowJoinSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + """) { implicit val defaultPatience = PatienceConfig(timeout = Span(2, Seconds), interval = Span(200, Millis)) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLimitSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLimitSpec.scala index 16a2d6f355..df2a533b3e 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLimitSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLimitSpec.scala @@ -4,15 +4,14 @@ package akka.stream.scaladsl +import akka.stream.StreamLimitReachedException import akka.stream.testkit.StreamSpec -import akka.stream.{ ActorMaterializer, ActorMaterializerSettings, StreamLimitReachedException } + import scala.concurrent.Await -class FlowLimitSpec extends StreamSpec { - - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 16) - - implicit val mat = ActorMaterializer(settings) +class FlowLimitSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + """) { "Limit" must { "produce empty sequence when source is empty and n = 0" in { diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLimitWeightedSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLimitWeightedSpec.scala index 0597562e2a..fbd1e19796 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLimitWeightedSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLimitWeightedSpec.scala @@ -4,17 +4,15 @@ package akka.stream.scaladsl +import akka.stream.StreamLimitReachedException import akka.stream.testkit.StreamSpec -import akka.stream.{ ActorMaterializer, ActorMaterializerSettings, StreamLimitReachedException } import akka.util.unused import scala.concurrent.Await -class FlowLimitWeightedSpec extends StreamSpec { - - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 16) - - implicit val mat = ActorMaterializer(settings) +class FlowLimitWeightedSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + """) { "Limit" must { "produce empty sequence regardless of cost when source is empty and n = 0" in { diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapConcatSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapConcatSpec.scala index f42a786a46..c6b8a1a876 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapConcatSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapConcatSpec.scala @@ -4,16 +4,17 @@ package akka.stream.scaladsl -import akka.stream.testkit.scaladsl.TestSink -import akka.stream.{ ActorAttributes, ActorMaterializer, ActorMaterializerSettings, Supervision } -import akka.stream.testkit.scaladsl.StreamTestKit._ import akka.stream.testkit._ +import akka.stream.testkit.scaladsl.StreamTestKit._ +import akka.stream.testkit.scaladsl.TestSink +import akka.stream.ActorAttributes +import akka.stream.Supervision + import scala.util.control.NoStackTrace -class FlowMapConcatSpec extends StreamSpec with ScriptedTest { - - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 16) - implicit val materializer = ActorMaterializer(settings) +class FlowMapConcatSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + """) with ScriptedTest { "A MapConcat" must { @@ -25,7 +26,7 @@ class FlowMapConcatSpec extends StreamSpec with ScriptedTest { Seq(3) -> Seq(3, 3, 3), Seq(2) -> Seq(2, 2), Seq(1) -> Seq(1)) - TestConfig.RandomTestRange.foreach(_ => runScript(script, settings)(_.mapConcat(x => (1 to x).map(_ => x)))) + TestConfig.RandomTestRange.foreach(_ => runScript(script)(_.mapConcat(x => (1 to x).map(_ => x)))) } "map and concat grouping with slow downstream" in assertAllStagesStopped { diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapErrorSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapErrorSpec.scala index 358d2ee914..6b03ee4453 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapErrorSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapErrorSpec.scala @@ -7,15 +7,13 @@ package akka.stream.scaladsl import akka.stream.testkit.StreamSpec import akka.stream.testkit.scaladsl.StreamTestKit._ import akka.stream.testkit.scaladsl.TestSink -import akka.stream.{ ActorMaterializer, ActorMaterializerSettings } import scala.util.control.NoStackTrace -class FlowMapErrorSpec extends StreamSpec { - - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 1, maxSize = 1) - - implicit val materializer = ActorMaterializer(settings) +class FlowMapErrorSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 1 + akka.stream.materializer.max-input-buffer-size = 1 + """) { val ex = new RuntimeException("ex") with NoStackTrace val boom = new Exception("BOOM!") with NoStackTrace diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapSpec.scala index 1485c64bd7..3adaf16485 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMapSpec.scala @@ -6,14 +6,11 @@ package akka.stream.scaladsl import java.util.concurrent.ThreadLocalRandom.{ current => random } -import akka.stream.{ ActorMaterializer, ActorMaterializerSettings } import akka.stream.testkit._ -class FlowMapSpec extends StreamSpec with ScriptedTest { - - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 16) - - implicit val materializer = ActorMaterializer(settings) +class FlowMapSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + """) with ScriptedTest { "A Map" must { @@ -22,7 +19,7 @@ class FlowMapSpec extends StreamSpec with ScriptedTest { Script(TestConfig.RandomTestRange.map { _ => val x = random.nextInt(); Seq(x) -> Seq(x.toString) }: _*) - TestConfig.RandomTestRange.foreach(_ => runScript(script, settings)(_.map(_.toString))) + TestConfig.RandomTestRange.foreach(_ => runScript(script)(_.map(_.toString))) } "not blow up with high request counts" in { diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowOnCompleteSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowOnCompleteSpec.scala index 3d13c2abaa..208e4d0835 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowOnCompleteSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowOnCompleteSpec.scala @@ -5,20 +5,19 @@ package akka.stream.scaladsl import akka.Done -import scala.concurrent.duration._ -import scala.util.{ Failure, Success } -import scala.util.control.NoStackTrace import akka.stream.ActorMaterializer -import akka.stream.ActorMaterializerSettings import akka.stream.testkit._ import akka.stream.testkit.scaladsl.StreamTestKit._ import akka.testkit.TestProbe -class FlowOnCompleteSpec extends StreamSpec with ScriptedTest { +import scala.concurrent.duration._ +import scala.util.control.NoStackTrace +import scala.util.Failure +import scala.util.Success - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 16) - - implicit val materializer = ActorMaterializer(settings) +class FlowOnCompleteSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + """) with ScriptedTest { "A Flow with onComplete" must { diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowPrefixAndTailSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowPrefixAndTailSpec.scala index fadb955ee6..0708130ed1 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowPrefixAndTailSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowPrefixAndTailSpec.scala @@ -4,19 +4,20 @@ package akka.stream.scaladsl +import akka.stream._ +import akka.stream.testkit._ +import akka.stream.testkit.scaladsl.StreamTestKit._ +import com.github.ghik.silencer.silent + import scala.collection.immutable import scala.concurrent.Await import scala.concurrent.duration._ import scala.util.control.NoStackTrace -import akka.stream._ -import akka.stream.testkit._ -import akka.stream.testkit.scaladsl.StreamTestKit._ -class FlowPrefixAndTailSpec extends StreamSpec { - - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 2) - - implicit val materializer = ActorMaterializer(settings) +class FlowPrefixAndTailSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + akka.stream.materializer.max-input-buffer-size = 2 + """) { "PrefixAndTail" must { @@ -109,6 +110,7 @@ class FlowPrefixAndTailSpec extends StreamSpec { "signal error if substream has been not subscribed in time" in assertAllStagesStopped { val ms = 300 + @silent("deprecated") val tightTimeoutMaterializer = ActorMaterializer( ActorMaterializerSettings(system).withSubscriptionTimeoutSettings( @@ -127,6 +129,7 @@ class FlowPrefixAndTailSpec extends StreamSpec { s"Substream Source has not been materialized in ${ms} milliseconds") } "not fail the stream if substream has not been subscribed in time and configured subscription timeout is noop" in assertAllStagesStopped { + @silent("deprecated") val tightTimeoutMaterializer = ActorMaterializer( ActorMaterializerSettings(system).withSubscriptionTimeoutSettings( diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowRecoverSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowRecoverSpec.scala index b72ae91b1d..94bb58e805 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowRecoverSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowRecoverSpec.scala @@ -5,18 +5,16 @@ package akka.stream.scaladsl import akka.stream.testkit.StreamSpec -import akka.stream.testkit.scaladsl.TestSink -import akka.stream.{ ActorMaterializer, ActorMaterializerSettings } import akka.stream.testkit.scaladsl.StreamTestKit._ +import akka.stream.testkit.scaladsl.TestSink import akka.testkit.EventFilter import scala.util.control.NoStackTrace -class FlowRecoverSpec extends StreamSpec { - - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 1, maxSize = 1) - - implicit val materializer = ActorMaterializer(settings) +class FlowRecoverSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 1 + akka.stream.materializer.max-input-buffer-size = 1 + """) { val ex = new RuntimeException("ex") with NoStackTrace diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowScanSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowScanSpec.scala index 4793b35380..acc9d4ff34 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowScanSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowScanSpec.scala @@ -4,23 +4,23 @@ package akka.stream.scaladsl +import java.util.concurrent.ThreadLocalRandom.{ current => random } + import akka.NotUsed import akka.stream.testkit.StreamSpec -import akka.stream.testkit.scaladsl.TestSink -import akka.stream.{ ActorAttributes, ActorMaterializer, ActorMaterializerSettings, Supervision } import akka.stream.testkit.Utils._ import akka.stream.testkit.scaladsl.StreamTestKit._ +import akka.stream.testkit.scaladsl.TestSink +import akka.stream.ActorAttributes +import akka.stream.Supervision import scala.collection.immutable import scala.concurrent.Await import scala.concurrent.duration._ -import java.util.concurrent.ThreadLocalRandom.{ current => random } -class FlowScanSpec extends StreamSpec { - - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 16) - - implicit val materializer = ActorMaterializer(settings) +class FlowScanSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + """) { "A Scan" must { diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSplitAfterSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSplitAfterSpec.scala index a6664688c0..5a72fbf83a 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSplitAfterSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSplitAfterSpec.scala @@ -5,17 +5,19 @@ package akka.stream.scaladsl import akka.NotUsed -import akka.stream._ +import akka.stream.StreamSubscriptionTimeoutTerminationMode import akka.stream.Supervision.resumingDecider +import akka.stream._ import akka.stream.impl.SubscriptionTimeoutException -import akka.stream.testkit.{ StreamSpec, TestPublisher, TestSubscriber } +import akka.stream.testkit.StreamSpec +import akka.stream.testkit.TestPublisher +import akka.stream.testkit.TestSubscriber import akka.stream.testkit.Utils._ import akka.stream.testkit.scaladsl.StreamTestKit._ import org.reactivestreams.Publisher + import scala.concurrent.Await import scala.concurrent.duration._ -import akka.stream.StreamSubscriptionTimeoutSettings -import akka.stream.StreamSubscriptionTimeoutTerminationMode object FlowSplitAfterSpec { @@ -25,16 +27,14 @@ object FlowSplitAfterSpec { } -class FlowSplitAfterSpec extends StreamSpec { +class FlowSplitAfterSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + akka.stream.materializer.max-input-buffer-size = 2 + akka.stream.materializer.subscription-timeout.timeout = 1s + akka.stream.materializer.subscription-timeout.mode = cancel + """) { import FlowSplitAfterSpec._ - val settings = ActorMaterializerSettings(system) - .withInputBuffer(initialSize = 2, maxSize = 2) - .withSubscriptionTimeoutSettings( - StreamSubscriptionTimeoutSettings(StreamSubscriptionTimeoutTerminationMode.cancel, 1.second)) - - implicit val materializer = ActorMaterializer(settings) - case class StreamPuppet(p: Publisher[Int]) { val probe = TestSubscriber.manualProbe[Int]() p.subscribe(probe) @@ -286,17 +286,17 @@ class FlowSplitAfterSpec extends StreamSpec { } "fail stream if substream not materialized in time" in assertAllStagesStopped { - val tightTimeoutMaterializer = - ActorMaterializer( - ActorMaterializerSettings(system).withSubscriptionTimeoutSettings( - StreamSubscriptionTimeoutSettings(StreamSubscriptionTimeoutTerminationMode.cancel, 500.millisecond))) - val testSource = Source.single(1).concat(Source.maybe).splitAfter(_ => true) + val streamWithTightTimeout = + testSource.lift + .delay(1.second) + .flatMapConcat(identity) + .toMat(Sink.ignore)(Keep.right) + .withAttributes(ActorAttributes + .streamSubscriptionTimeout(500.milliseconds, StreamSubscriptionTimeoutTerminationMode.cancel)) a[SubscriptionTimeoutException] mustBe thrownBy { - Await.result( - testSource.lift.delay(1.second).flatMapConcat(identity).runWith(Sink.ignore)(tightTimeoutMaterializer), - 3.seconds) + Await.result(streamWithTightTimeout.run(), 3.seconds) } } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSplitWhenSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSplitWhenSpec.scala index 6f925970f7..9caaccfd78 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSplitWhenSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSplitWhenSpec.scala @@ -4,29 +4,29 @@ package akka.stream.scaladsl -import akka.{ Done, NotUsed } -import akka.stream._ +import akka.Done +import akka.NotUsed import akka.stream.Supervision.resumingDecider +import akka.stream._ import akka.stream.impl.SubscriptionTimeoutException import akka.stream.testkit.Utils._ -import akka.stream.testkit.scaladsl.StreamTestKit._ import akka.stream.testkit._ +import akka.stream.testkit.scaladsl.StreamTestKit._ import akka.stream.testkit.scaladsl.TestSink import org.reactivestreams.Publisher -import scala.concurrent.{ Await, Future } +import scala.concurrent.Await +import scala.concurrent.Future import scala.concurrent.duration._ -class FlowSplitWhenSpec extends StreamSpec { +class FlowSplitWhenSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + akka.stream.materializer.max-input-buffer-size = 2 + akka.stream.materializer.subscription-timeout.timeout = 1s + akka.stream.materializer.subscription-timeout.mode = cancel + """) { import FlowSplitAfterSpec._ - val settings = ActorMaterializerSettings(system) - .withInputBuffer(initialSize = 2, maxSize = 2) - .withSubscriptionTimeoutSettings( - StreamSubscriptionTimeoutSettings(StreamSubscriptionTimeoutTerminationMode.cancel, 1.second)) - - implicit val materializer = ActorMaterializer(settings) - case class StreamPuppet(p: Publisher[Int]) { val probe = TestSubscriber.manualProbe[Int]() p.subscribe(probe) @@ -263,17 +263,16 @@ class FlowSplitWhenSpec extends StreamSpec { } "fail substream if materialized twice" in assertAllStagesStopped { - implicit val mat = - ActorMaterializer(ActorMaterializerSettings(system).withInputBuffer(initialSize = 1, maxSize = 1)) import system.dispatcher - val probe = Source(1 to 5) + val stream = Source(1 to 5) .splitWhen(_ => true) .lift .map { src => - src.runWith(Sink.ignore)(mat).flatMap(_ => src.runWith(Sink.ignore)(mat)) + src.runWith(Sink.ignore).flatMap(_ => src.runWith(Sink.ignore)) } - .runWith(TestSink.probe[Future[Done]])(mat) + .toMat(TestSink.probe[Future[Done]])(Keep.right) + val probe = stream.withAttributes(Attributes.inputBuffer(1, 1)).run() probe.request(1) val future = probe.requestNext() an[IllegalStateException] mustBe thrownBy { @@ -283,17 +282,17 @@ class FlowSplitWhenSpec extends StreamSpec { } "fail stream if substream not materialized in time" in assertAllStagesStopped { - val tightTimeoutMaterializer = - ActorMaterializer( - ActorMaterializerSettings(system).withSubscriptionTimeoutSettings( - StreamSubscriptionTimeoutSettings(StreamSubscriptionTimeoutTerminationMode.cancel, 500.millisecond))) - val testSource = Source.single(1).concat(Source.maybe).splitWhen(_ => true) + val testStreamWithTightTimeout = + testSource.lift + .delay(1.second) + .flatMapConcat(identity) + .toMat(Sink.ignore)(Keep.right) + .withAttributes(ActorAttributes + .streamSubscriptionTimeout(500.milliseconds, StreamSubscriptionTimeoutTerminationMode.cancel)) a[SubscriptionTimeoutException] mustBe thrownBy { - Await.result( - testSource.lift.delay(1.second).flatMapConcat(identity).runWith(Sink.ignore)(tightTimeoutMaterializer), - 3.seconds) + Await.result(testStreamWithTightTimeout.run(), 3.seconds) } } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowStatefulMapConcatSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowStatefulMapConcatSpec.scala index 8ed7ed844d..f984ad1cb6 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowStatefulMapConcatSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowStatefulMapConcatSpec.scala @@ -4,15 +4,17 @@ package akka.stream.scaladsl -import akka.stream.testkit.scaladsl.TestSink -import akka.stream.{ ActorAttributes, ActorMaterializer, ActorMaterializerSettings, Supervision } import akka.stream.testkit._ +import akka.stream.testkit.scaladsl.TestSink +import akka.stream.ActorAttributes +import akka.stream.Supervision + import scala.util.control.NoStackTrace -class FlowStatefulMapConcatSpec extends StreamSpec with ScriptedTest { +class FlowStatefulMapConcatSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + """) with ScriptedTest { - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 16) - implicit val materializer = ActorMaterializer(settings) val ex = new Exception("TEST") with NoStackTrace "A StatefulMapConcat" must { @@ -20,7 +22,7 @@ class FlowStatefulMapConcatSpec extends StreamSpec with ScriptedTest { "work in happy case" in { val script = Script(Seq(2) -> Seq(), Seq(1) -> Seq(1, 1), Seq(3) -> Seq(3), Seq(6) -> Seq(6, 6, 6)) TestConfig.RandomTestRange.foreach(_ => - runScript(script, settings)(_.statefulMapConcat(() => { + runScript(script)(_.statefulMapConcat(() => { var prev: Option[Int] = None x => prev match { diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowTakeSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowTakeSpec.scala index 2dc282d8a1..a0b92f448e 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowTakeSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowTakeSpec.scala @@ -4,21 +4,19 @@ package akka.stream.scaladsl -import scala.concurrent.Await -import scala.concurrent.duration._ import java.util.concurrent.ThreadLocalRandom.{ current => random } -import akka.stream.ActorMaterializer -import akka.stream.ActorMaterializerSettings + import akka.stream.actor.ActorSubscriberMessage.OnComplete import akka.stream.actor.ActorSubscriberMessage.OnNext import akka.stream.impl.RequestMore import akka.stream.testkit._ -class FlowTakeSpec extends StreamSpec with ScriptedTest { +import scala.concurrent.Await +import scala.concurrent.duration._ - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 16) - - implicit val materializer = ActorMaterializer(settings) +class FlowTakeSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + """) with ScriptedTest { muteDeadLetters(classOf[OnNext], OnComplete.getClass, classOf[RequestMore[_]])() @@ -31,7 +29,7 @@ class FlowTakeSpec extends StreamSpec with ScriptedTest { }: _*) TestConfig.RandomTestRange.foreach { _ => val d = Math.min(Math.max(random.nextInt(-10, 60), 0), 50) - runScript(script(d), settings)(_.take(d)) + runScript(script(d))(_.take(d)) } } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowThrottleSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowThrottleSpec.scala index da0b9a142f..c15fb143d1 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowThrottleSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowThrottleSpec.scala @@ -21,8 +21,10 @@ import scala.util.control.NoStackTrace import akka.Done import akka.testkit.TimingTest -class FlowThrottleSpec extends StreamSpec { - implicit val materializer = ActorMaterializer(ActorMaterializerSettings(system).withInputBuffer(1, 1)) +class FlowThrottleSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + akka.stream.materializer.max-input-buffer-size = 2 + """) { def genByteString(length: Int) = ByteString(new Random().shuffle(0 to 255).take(length).map(_.toByte).toArray) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBalanceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBalanceSpec.scala index 73e4270012..e67cc48249 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBalanceSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBalanceSpec.scala @@ -4,19 +4,18 @@ package akka.stream.scaladsl -import scala.concurrent.Await -import scala.concurrent.duration._ -import scala.concurrent.Future import akka.stream._ import akka.stream.testkit._ -import akka.stream.testkit.scaladsl._ import akka.stream.testkit.scaladsl.StreamTestKit._ +import akka.stream.testkit.scaladsl._ -class GraphBalanceSpec extends StreamSpec { +import scala.concurrent.Await +import scala.concurrent.Future +import scala.concurrent.duration._ - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 16) - - implicit val materializer = ActorMaterializer(settings) +class GraphBalanceSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + """) { "A balance" must { import GraphDSL.Implicits._ diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBroadcastSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBroadcastSpec.scala index 78a76966ab..2c103aec65 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBroadcastSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphBroadcastSpec.scala @@ -4,19 +4,19 @@ package akka.stream.scaladsl -import akka.stream.testkit.scaladsl.{ TestSink, TestSource } - -import scala.concurrent.{ Await, Future } -import scala.concurrent.duration._ import akka.stream._ import akka.stream.testkit._ import akka.stream.testkit.scaladsl.StreamTestKit._ +import akka.stream.testkit.scaladsl.TestSink +import akka.stream.testkit.scaladsl.TestSource -class GraphBroadcastSpec extends StreamSpec { +import scala.concurrent.duration._ +import scala.concurrent.Await +import scala.concurrent.Future - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 16) - - implicit val materializer = ActorMaterializer(settings) +class GraphBroadcastSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + """) { "A broadcast" must { import GraphDSL.Implicits._ diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMatValueSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMatValueSpec.scala index 2aced0d47b..3b39332c1a 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMatValueSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphMatValueSpec.scala @@ -6,11 +6,14 @@ package akka.stream.scaladsl import java.util.concurrent.atomic.AtomicInteger -import akka.{ Done, NotUsed } +import akka.Done +import akka.NotUsed import akka.stream._ import akka.stream.testkit._ -import scala.concurrent.{ Await, Future, Promise } +import scala.concurrent.Await +import scala.concurrent.Future +import scala.concurrent.Promise import scala.concurrent.duration._ class GraphMatValueSpec extends StreamSpec { @@ -21,9 +24,6 @@ class GraphMatValueSpec extends StreamSpec { "A Graph with materialized value" must { - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 16) - implicit val materializer = ActorMaterializer(settings) - "expose the materialized value as source" in { val sub = TestSubscriber.manualProbe[Int]() val f = RunnableGraph @@ -206,8 +206,6 @@ class GraphMatValueSpec extends StreamSpec { } "with Identity Flow optimization even if ports are wired in an arbitrary higher nesting level" in { - val mat2 = ActorMaterializer(ActorMaterializerSettings(system)) - val subflow = GraphDSL .create() { implicit b => import GraphDSL.Implicits._ @@ -227,10 +225,9 @@ class GraphMatValueSpec extends StreamSpec { val nest4 = Flow[String].via(nest3) //fails - val matValue = Source(List("")).via(nest4).to(Sink.ignore).run()(mat2) + val matValue = Source(List("")).via(nest4).to(Sink.ignore).run() matValue should ===(NotUsed) - } "not ignore materialized value of indentity flow which is optimized away" in { 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 d60e9114a7..e2a48b1cd0 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 @@ -5,12 +5,14 @@ package akka.stream.scaladsl import akka.NotUsed -import scala.collection.immutable -import scala.concurrent.{ Await, Future } -import scala.concurrent.duration._ import akka.stream._ import akka.stream.testkit._ +import scala.collection.immutable +import scala.concurrent.duration._ +import scala.concurrent.Await +import scala.concurrent.Future + object GraphOpsIntegrationSpec { import GraphDSL.Implicits._ @@ -37,13 +39,11 @@ object GraphOpsIntegrationSpec { } -class GraphOpsIntegrationSpec extends StreamSpec { - import akka.stream.scaladsl.GraphOpsIntegrationSpec._ +class GraphOpsIntegrationSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + """) { import GraphDSL.Implicits._ - - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 16) - - implicit val materializer = ActorMaterializer(settings) + import akka.stream.scaladsl.GraphOpsIntegrationSpec._ "GraphDSLs" must { @@ -200,7 +200,7 @@ class GraphOpsIntegrationSpec extends StreamSpec { } "be possible to use with generated components" in { - implicit val ex = materializer.system.dispatcher + implicit val ex = system.dispatcher //#graph-from-list val sinks = immutable @@ -229,7 +229,7 @@ class GraphOpsIntegrationSpec extends StreamSpec { } "be possible to use with generated components if list has no tail" in { - implicit val ex = materializer.system.dispatcher + implicit val ex = system.dispatcher val sinks = immutable.Seq(Sink.seq[Int]) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphPartialSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphPartialSpec.scala index 959c348375..971e57902d 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphPartialSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphPartialSpec.scala @@ -5,16 +5,15 @@ package akka.stream.scaladsl import akka.stream.testkit.StreamSpec -import akka.stream.{ ActorMaterializer, ActorMaterializerSettings, ClosedShape, FlowShape } +import akka.stream.ClosedShape +import akka.stream.FlowShape import scala.concurrent.Await import scala.concurrent.duration._ -class GraphPartialSpec extends StreamSpec { - - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 16) - - implicit val materializer = ActorMaterializer(settings) +class GraphPartialSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + """) { "GraphDSL.partial" must { import GraphDSL.Implicits._ diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphPartitionSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphPartitionSpec.scala index f901222067..6dbfb3afa4 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphPartitionSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphPartitionSpec.scala @@ -4,21 +4,20 @@ package akka.stream.scaladsl -import akka.stream.testkit._ -import akka.stream.{ ActorMaterializer, ActorMaterializerSettings, ClosedShape, OverflowStrategy } -import akka.stream.testkit.scaladsl.StreamTestKit._ -import scala.concurrent.Await -import scala.concurrent.duration._ - import akka.stream.ActorAttributes import akka.stream.Supervision import akka.stream.testkit.Utils.TE +import akka.stream.testkit._ +import akka.stream.testkit.scaladsl.StreamTestKit._ +import akka.stream.ClosedShape +import akka.stream.OverflowStrategy -class GraphPartitionSpec extends StreamSpec { +import scala.concurrent.Await +import scala.concurrent.duration._ - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 16) - - implicit val materializer = ActorMaterializer(settings) +class GraphPartitionSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + """) { "A partition" must { import GraphDSL.Implicits._ diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphUnzipSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphUnzipSpec.scala index 2cfc21adee..9ddf4a4d29 100755 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphUnzipSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphUnzipSpec.scala @@ -4,16 +4,16 @@ package akka.stream.scaladsl -import scala.concurrent.duration._ -import akka.stream.{ ActorMaterializer, ActorMaterializerSettings, ClosedShape, OverflowStrategy } import akka.stream.testkit._ import akka.stream.testkit.scaladsl.StreamTestKit._ +import akka.stream.ClosedShape +import akka.stream.OverflowStrategy -class GraphUnzipSpec extends StreamSpec { +import scala.concurrent.duration._ - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 16) - - implicit val materializer = ActorMaterializer(settings) +class GraphUnzipSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + """) { "A unzip" must { import GraphDSL.Implicits._ diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphUnzipWithSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphUnzipWithSpec.scala index 376da8ef09..25a77dfdd8 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphUnzipWithSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphUnzipWithSpec.scala @@ -6,23 +6,21 @@ package akka.stream.scaladsl import akka.stream._ import akka.stream.testkit.TestSubscriber.Probe -import akka.stream.testkit.scaladsl.StreamTestKit._ import akka.stream.testkit._ +import akka.stream.testkit.scaladsl.StreamTestKit._ +import akka.testkit.EventFilter +import akka.util.unused import org.reactivestreams.Publisher import scala.concurrent.duration._ import scala.util.control.NoStackTrace -import akka.testkit.EventFilter -import akka.util.unused -class GraphUnzipWithSpec extends StreamSpec { +class GraphUnzipWithSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + """) { import GraphDSL.Implicits._ - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 16) - - implicit val materializer = ActorMaterializer(settings) - val TestException = new RuntimeException("test") with NoStackTrace type LeftOutput = Int @@ -147,56 +145,50 @@ class GraphUnzipWithSpec extends StreamSpec { } "work in the sad case" in { - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 1, maxSize = 1) - val mat = ActorMaterializer(settings) + val leftProbe = TestSubscriber.manualProbe[LeftOutput]() + val rightProbe = TestSubscriber.manualProbe[RightOutput]() - try { - val leftProbe = TestSubscriber.manualProbe[LeftOutput]() - val rightProbe = TestSubscriber.manualProbe[RightOutput]() + RunnableGraph + .fromGraph(GraphDSL.create() { implicit b => + val unzip = b.add(UnzipWith[Int, Int, String]((b: Int) => (1 / b, s"1 / $b"))) - RunnableGraph - .fromGraph(GraphDSL.create() { implicit b => - val unzip = b.add(UnzipWith[Int, Int, String]((b: Int) => (1 / b, s"1 / $b"))) + Source(-2 to 2) ~> unzip.in - Source(-2 to 2) ~> unzip.in + unzip.out0 ~> Sink.fromSubscriber(leftProbe) + unzip.out1 ~> Sink.fromSubscriber(rightProbe) - unzip.out0 ~> Sink.fromSubscriber(leftProbe) - unzip.out1 ~> Sink.fromSubscriber(rightProbe) + ClosedShape + }) + .withAttributes(Attributes.inputBuffer(1, 1)) + .run() - ClosedShape - }) - .run()(mat) + val leftSubscription = leftProbe.expectSubscription() + val rightSubscription = rightProbe.expectSubscription() - val leftSubscription = leftProbe.expectSubscription() - val rightSubscription = rightProbe.expectSubscription() - - def requestFromBoth(): Unit = { - leftSubscription.request(1) - rightSubscription.request(1) - } - - requestFromBoth() - leftProbe.expectNext(1 / -2) - rightProbe.expectNext("1 / -2") - - requestFromBoth() - leftProbe.expectNext(1 / -1) - rightProbe.expectNext("1 / -1") - - EventFilter[ArithmeticException](occurrences = 1).intercept { - requestFromBoth() - } - - leftProbe.expectError() match { - case a: java.lang.ArithmeticException => a.getMessage should be("/ by zero") - } - rightProbe.expectError() - - leftProbe.expectNoMessage(100.millis) - rightProbe.expectNoMessage(100.millis) - } finally { - mat.shutdown() + def requestFromBoth(): Unit = { + leftSubscription.request(1) + rightSubscription.request(1) } + + requestFromBoth() + leftProbe.expectNext(1 / -2) + rightProbe.expectNext("1 / -2") + + requestFromBoth() + leftProbe.expectNext(1 / -1) + rightProbe.expectNext("1 / -1") + + EventFilter[ArithmeticException](occurrences = 1).intercept { + requestFromBoth() + } + + leftProbe.expectError() match { + case a: java.lang.ArithmeticException => a.getMessage should be("/ by zero") + } + rightProbe.expectError() + + leftProbe.expectNoMessage(100.millis) + rightProbe.expectNoMessage(100.millis) } "unzipWith expanded Person.unapply (3 outputs)" in { diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphWireTapSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphWireTapSpec.scala index 12ac5473ef..f55fd49567 100755 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphWireTapSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/GraphWireTapSpec.scala @@ -4,16 +4,13 @@ package akka.stream.scaladsl -import akka.stream._ -import akka.stream.testkit.scaladsl.StreamTestKit._ import akka.stream.testkit._ +import akka.stream.testkit.scaladsl.StreamTestKit._ import akka.stream.testkit.scaladsl.TestSink -class GraphWireTapSpec extends StreamSpec { - - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 16) - - implicit val materializer = ActorMaterializer(settings) +class GraphWireTapSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + """) { "A wire tap" must { diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/HeadSinkSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/HeadSinkSpec.scala index 37ca22b7cf..f68668121c 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/HeadSinkSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/HeadSinkSpec.scala @@ -4,18 +4,18 @@ package akka.stream.scaladsl +import akka.stream.testkit._ +import akka.stream.testkit.scaladsl.StreamTestKit._ +import akka.stream.AbruptTerminationException +import akka.stream.ActorMaterializer + import scala.concurrent.Await import scala.concurrent.Future import scala.concurrent.duration._ -import akka.stream.{ AbruptTerminationException, ActorMaterializer, ActorMaterializerSettings } -import akka.stream.testkit._ -import akka.stream.testkit.scaladsl.StreamTestKit._ -class HeadSinkSpec extends StreamSpec with ScriptedTest { - - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 16) - - implicit val materializer = ActorMaterializer(settings) +class HeadSinkSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + """) with ScriptedTest { "A Flow with Sink.head" must { diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazyFlowSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazyFlowSpec.scala index 5e377f9e91..aa349bae1d 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazyFlowSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazyFlowSpec.scala @@ -5,21 +5,22 @@ package akka.stream.scaladsl import akka.NotUsed -import akka.stream._ -import akka.stream.testkit.{ StreamSpec, TestPublisher } import akka.stream.testkit.Utils._ import akka.stream.testkit.scaladsl.StreamTestKit._ -import akka.stream.testkit.scaladsl.{ TestSink, TestSource } -import akka.stream.testkit.{ StreamSpec, TestPublisher } +import akka.stream.testkit.scaladsl.TestSink +import akka.stream.testkit.scaladsl.TestSource +import akka.stream.testkit.StreamSpec +import akka.stream.testkit.TestPublisher import org.scalatest.concurrent.ScalaFutures import scala.concurrent.duration._ -import scala.concurrent.{ Future, Promise } +import scala.concurrent.Future +import scala.concurrent.Promise -class LazyFlowSpec extends StreamSpec { - - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 1, maxSize = 1) - implicit val materializer = ActorMaterializer(settings) +class LazyFlowSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 1 + akka.stream.materializer.max-input-buffer-size = 1 + """) { val ex = TE("") diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazySinkSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazySinkSpec.scala index 23e5e0512f..1b99c63236 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazySinkSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/LazySinkSpec.scala @@ -8,21 +8,25 @@ import java.util.concurrent.TimeoutException import akka.NotUsed import akka.stream._ -import akka.stream.stage.{ GraphStage, GraphStageLogic } +import akka.stream.stage.GraphStage +import akka.stream.stage.GraphStageLogic +import akka.stream.testkit.StreamSpec +import akka.stream.testkit.TestPublisher import akka.stream.testkit.TestSubscriber.Probe import akka.stream.testkit.Utils._ import akka.stream.testkit.scaladsl.StreamTestKit._ import akka.stream.testkit.scaladsl.TestSink -import akka.stream.testkit.{ StreamSpec, TestPublisher } import scala.collection.immutable +import scala.concurrent.Await +import scala.concurrent.Future +import scala.concurrent.Promise import scala.concurrent.duration._ -import scala.concurrent.{ Await, Future, Promise } -class LazySinkSpec extends StreamSpec { - - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 1, maxSize = 1) - implicit val materializer = ActorMaterializer(settings) +class LazySinkSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 1 + akka.stream.materializer.max-input-buffer-size = 1 + """) { val ex = TE("") diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SeqSinkSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SeqSinkSpec.scala index 04997abbd0..d467f698f2 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SeqSinkSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SeqSinkSpec.scala @@ -4,17 +4,18 @@ package akka.stream.scaladsl -import akka.stream.testkit.{ StreamSpec, TestPublisher } -import akka.stream.{ AbruptTerminationException, ActorMaterializer, ActorMaterializerSettings } +import akka.stream.testkit.StreamSpec +import akka.stream.testkit.TestPublisher +import akka.stream.AbruptTerminationException +import akka.stream.ActorMaterializer import scala.collection.immutable -import scala.concurrent.{ Await, Future } +import scala.concurrent.Await +import scala.concurrent.Future -class SeqSinkSpec extends StreamSpec { - - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 16) - - implicit val mat = ActorMaterializer(settings) +class SeqSinkSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + """) { "Sink.toSeq" must { "return a Seq[T] from a Source" in { 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 0e476e8b98..febf03c492 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 @@ -8,17 +8,16 @@ import java.util.stream.Collectors import akka.actor.ActorSystem import akka.stream._ -import akka.stream.impl.{ PhasedFusingActorMaterializer, StreamSupervisor } import akka.stream.impl.StreamSupervisor.Children +import akka.stream.impl.PhasedFusingActorMaterializer +import akka.stream.impl.StreamSupervisor import akka.stream.testkit.Utils._ -import akka.stream.testkit.scaladsl.StreamTestKit._ import akka.stream.testkit._ +import akka.stream.testkit.scaladsl.StreamTestKit._ import akka.stream.testkit.scaladsl.TestSource import akka.util.ByteString class SinkAsJavaStreamSpec extends StreamSpec(UnboundedMailboxConfig) { - val settings = ActorMaterializerSettings(system).withDispatcher("akka.actor.default-dispatcher") - implicit val materializer = ActorMaterializer(settings) "Java Stream Sink" must { diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SubscriberSinkSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SubscriberSinkSpec.scala index 39485651aa..d26b7883d6 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SubscriberSinkSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SubscriberSinkSpec.scala @@ -4,16 +4,12 @@ package akka.stream.scaladsl -import akka.stream.ActorMaterializer -import akka.stream.ActorMaterializerSettings import akka.stream.testkit._ import akka.stream.testkit.scaladsl.StreamTestKit._ -class SubscriberSinkSpec extends StreamSpec { - - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 16) - - implicit val materializer = ActorMaterializer(settings) +class SubscriberSinkSpec extends StreamSpec(""" + akka.stream.materializer.initial-input-buffer-size = 2 + """) { "A Flow with SubscriberSink" must { diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SubstreamSubscriptionTimeoutSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SubstreamSubscriptionTimeoutSpec.scala index 7693c9b24b..c2cb5b00ea 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SubstreamSubscriptionTimeoutSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SubstreamSubscriptionTimeoutSpec.scala @@ -4,36 +4,24 @@ package akka.stream.scaladsl -import akka.stream.{ ActorMaterializer, ActorMaterializerSettings } import akka.stream.impl.SubscriptionTimeoutException import akka.stream.testkit._ import akka.stream.testkit.scaladsl.StreamTestKit._ import scala.concurrent.Await -import scala.concurrent.duration._ -class SubstreamSubscriptionTimeoutSpec(conf: String) extends StreamSpec(conf) { +class SubstreamSubscriptionTimeoutSpec extends StreamSpec(""" + akka.stream.materializer { + initial-input-buffer-size = 2 + max-input-buffer-size = 2 + subscription-timeout { + mode = cancel + timeout = 300 ms + } + } +""") { import FlowGroupBySpec._ - - def this(subscriptionTimeout: FiniteDuration) { - this(s""" - |akka.stream.materializer { - | subscription-timeout { - | mode = cancel - | - | timeout = ${subscriptionTimeout.toMillis}ms - | } - |}""".stripMargin) - } - - def this() { - this(300.millis) - } - - val settings = ActorMaterializerSettings(system).withInputBuffer(initialSize = 2, maxSize = 2) - - implicit val dispatcher = system.dispatcher - implicit val materializer = ActorMaterializer(settings) + import system.dispatcher "groupBy and splitwhen" must { 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 a58e0125c7..f70e8321c7 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 @@ -8,12 +8,14 @@ import java.util.concurrent.atomic.AtomicInteger import akka.Done import akka.actor.ActorSystem +import akka.stream.ActorAttributes import akka.stream.impl.StreamSupervisor.Children import akka.stream.impl.{ PhasedFusingActorMaterializer, StreamSupervisor } import akka.stream.testkit.Utils._ import akka.stream.testkit.scaladsl.StreamTestKit._ import akka.stream.testkit.{ StreamSpec, TestSubscriber } -import akka.stream.{ ActorMaterializer, _ } +import akka.stream.ActorMaterializer +import akka.stream.Supervision import akka.testkit.{ TestLatch, TestProbe } import scala.concurrent.duration._ @@ -61,8 +63,6 @@ class UnfoldResourceAsyncSourceSpec extends StreamSpec(UnboundedMailboxConfig) { import UnfoldResourceAsyncSourceSpec._ - val settings = ActorMaterializerSettings(system).withDispatcher("akka.actor.default-dispatcher") - implicit val materializer = ActorMaterializer(settings) import system.dispatcher "Unfold Resource Async Source" must { 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 a462949cd5..523be44bb4 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 @@ -9,27 +9,27 @@ import java.nio.charset.StandardCharsets import java.nio.file.Files import java.util.concurrent.atomic.AtomicInteger -import akka.actor.ActorSystem +import akka.stream.ActorAttributes import akka.stream.ActorAttributes._ import akka.stream.Supervision._ +import akka.stream.SystemMaterializer +import akka.stream.impl.PhasedFusingActorMaterializer +import akka.stream.impl.StreamSupervisor import akka.stream.impl.StreamSupervisor.Children -import akka.stream.impl.{ PhasedFusingActorMaterializer, StreamSupervisor } +import akka.stream.testkit.StreamSpec +import akka.stream.testkit.TestSubscriber import akka.stream.testkit.Utils._ import akka.stream.testkit.scaladsl.StreamTestKit._ import akka.stream.testkit.scaladsl.TestSink -import akka.stream.testkit.{ StreamSpec, TestSubscriber } -import akka.stream.{ ActorMaterializer, _ } import akka.testkit.EventFilter import akka.util.ByteString -import com.google.common.jimfs.{ Configuration, Jimfs } +import com.google.common.jimfs.Configuration +import com.google.common.jimfs.Jimfs import scala.concurrent.duration._ class UnfoldResourceSourceSpec extends StreamSpec(UnboundedMailboxConfig) { - val settings = ActorMaterializerSettings(system).withDispatcher("akka.actor.default-dispatcher") - implicit val materializer = ActorMaterializer(settings) - private val fs = Jimfs.newFileSystem("UnfoldResourceSourceSpec", Configuration.unix()) private val manyLines = { @@ -148,24 +148,20 @@ class UnfoldResourceSourceSpec extends StreamSpec(UnboundedMailboxConfig) { } "use dedicated blocking-io-dispatcher by default" in assertAllStagesStopped { - val sys = ActorSystem("dispatcher-testing", UnboundedMailboxConfig) - val materializer = ActorMaterializer()(sys) - try { - val p = Source - .unfoldResource[String, BufferedReader]( - () => newBufferedReader(), - reader => Option(reader.readLine()), - reader => reader.close()) - .runWith(TestSink.probe)(materializer) + val p = Source + .unfoldResource[String, BufferedReader]( + () => newBufferedReader(), + reader => Option(reader.readLine()), + reader => reader.close()) + .runWith(TestSink.probe) - materializer - .asInstanceOf[PhasedFusingActorMaterializer] - .supervisor - .tell(StreamSupervisor.GetChildren, testActor) - val ref = expectMsgType[Children].children.find(_.path.toString contains "unfoldResourceSource").get - try assertDispatcher(ref, ActorAttributes.IODispatcher.dispatcher) - finally p.cancel() - } finally shutdown(sys) + SystemMaterializer(system).materializer + .asInstanceOf[PhasedFusingActorMaterializer] + .supervisor + .tell(StreamSupervisor.GetChildren, testActor) + val ref = expectMsgType[Children].children.find(_.path.toString contains "unfoldResourceSource").get + try assertDispatcher(ref, ActorAttributes.IODispatcher.dispatcher) + finally p.cancel() } "fail when create throws exception" in assertAllStagesStopped { diff --git a/akka-stream/src/main/mima-filters/2.5.x.backwards.excludes b/akka-stream/src/main/mima-filters/2.5.x.backwards.excludes index e7aa610441..318d3b61b1 100644 --- a/akka-stream/src/main/mima-filters/2.5.x.backwards.excludes +++ b/akka-stream/src/main/mima-filters/2.5.x.backwards.excludes @@ -159,3 +159,54 @@ ProblemFilters.exclude[Problem]("akka.stream.StreamRefMessages*") # #27266 changes to streams internals ProblemFilters.exclude[Problem]("akka.stream.impl.*") + +# #25559 simplifying materialization +ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.impl.ActorProcessorImpl.settings") +ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.impl.ActorProcessorImpl.this") +ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.impl.FanOut.settings") +ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.stream.impl.FanOut.this") +ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.impl.FanIn.settings") +ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.stream.impl.FanIn.this") +ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.stream.impl.Unzip.props") +ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.impl.FanoutProcessorImpl.this") +ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.stream.impl.PhasedFusingActorMaterializer.copy$default$6") +ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.stream.impl.PhasedFusingActorMaterializer.copy$default$5") +ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.impl.PhasedFusingActorMaterializer.copy") +ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.stream.impl.PhasedFusingActorMaterializer.copy$default$4") +ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.stream.impl.PhasedFusingActorMaterializer.copy$default$3") +ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.impl.PhasedFusingActorMaterializer.this") +ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.impl.PhasedFusingActorMaterializer.apply") +ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.stream.impl.Unzip.this") +ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.impl.FanoutProcessorImpl.props") +ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.stream.impl.StreamSupervisor.props") +ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.impl.GraphStageIsland.this") +ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.impl.io.ConnectionSourceStage.ioSettings") +ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.impl.io.ConnectionSourceStage.this") +ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.impl.io.TcpConnectionStage#Outbound.ioSettings") +ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.impl.io.TcpConnectionStage#Outbound.copy$default$5") +ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.impl.io.TcpConnectionStage#Outbound.copy") +ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.impl.io.TcpConnectionStage#Outbound.this") +ProblemFilters.exclude[MissingTypesProblem]("akka.stream.impl.io.TcpConnectionStage$Outbound$") +ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.impl.io.TcpConnectionStage#Outbound.apply") +ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.impl.io.OutgoingConnectionStage.this") +ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.stream.impl.io.IncomingConnectionStage.this") +ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.impl.io.IncomingConnectionStage.this") +ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.impl.io.TcpConnectionStage#Inbound.ioSettings") +ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.stream.impl.io.TcpConnectionStage#Inbound.copy") +ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.impl.io.TcpConnectionStage#Inbound.copy") +ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.stream.impl.io.TcpConnectionStage#Inbound.copy$default$3") +ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.impl.io.TcpConnectionStage#Inbound.copy$default$4") +ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.stream.impl.io.TcpConnectionStage#Inbound.this") +ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.impl.io.TcpConnectionStage#Inbound.this") +ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.impl.io.TcpConnectionStage#TcpRole.ioSettings") +ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.impl.io.TcpConnectionStage#TcpStreamLogic.this") +ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.stream.impl.io.TcpConnectionStage#Inbound.apply") +ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.impl.io.TcpConnectionStage#Inbound.apply") +ProblemFilters.exclude[MissingTypesProblem]("akka.stream.impl.io.TcpConnectionStage$Inbound$") +ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.impl.fusing.PrefixAndTail#PrefixAndTailLogic.this") +ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.impl.fusing.GraphInterpreterShell.this") +ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.stream.impl.ActorProcessorImpl.this") +ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.stream.impl.FanoutProcessorImpl.this") +ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.stream.impl.FanoutProcessorImpl.props") +ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.stream.impl.GraphStageIsland.this") +ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.stream.impl.fusing.GraphInterpreterShell.this") diff --git a/akka-stream/src/main/scala/akka/stream/ActorMaterializer.scala b/akka-stream/src/main/scala/akka/stream/ActorMaterializer.scala index f824b64ef5..9d9d6467e8 100644 --- a/akka-stream/src/main/scala/akka/stream/ActorMaterializer.scala +++ b/akka-stream/src/main/scala/akka/stream/ActorMaterializer.scala @@ -17,6 +17,7 @@ import com.typesafe.config.{ Config, ConfigFactory } import scala.concurrent.duration._ import akka.japi.function import akka.stream.stage.GraphStageLogic +import com.github.ghik.silencer.silent import scala.util.control.NoStackTrace @@ -35,6 +36,7 @@ object ActorMaterializer { * the processing steps. The default `namePrefix` is `"flow"`. The actor names are built up of * `namePrefix-flowNumber-flowStepNumber-stepName`. */ + @silent("deprecated") def apply(materializerSettings: Option[ActorMaterializerSettings] = None, namePrefix: Option[String] = None)( implicit context: ActorRefFactory): ActorMaterializer = { val system = actorSystemOf(context) @@ -59,21 +61,20 @@ object ActorMaterializer { implicit context: ActorRefFactory): ActorMaterializer = { val haveShutDown = new AtomicBoolean(false) val system = actorSystemOf(context) + val defaultAttributes = materializerSettings.toAttributes new PhasedFusingActorMaterializer( system, materializerSettings, + defaultAttributes, system.dispatchers, - actorOfStreamSupervisor(materializerSettings, context, haveShutDown), + actorOfStreamSupervisor(defaultAttributes, context, haveShutDown), haveShutDown, FlowNames(system).name.copy(namePrefix)) } - private def actorOfStreamSupervisor( - materializerSettings: ActorMaterializerSettings, - context: ActorRefFactory, - haveShutDown: AtomicBoolean) = { - val props = StreamSupervisor.props(materializerSettings, haveShutDown) + private def actorOfStreamSupervisor(attributes: Attributes, context: ActorRefFactory, haveShutDown: AtomicBoolean) = { + val props = StreamSupervisor.props(attributes, haveShutDown) context match { case s: ExtendedActorSystem => s.systemActorOf(props, StreamSupervisor.nextName()) case a: ActorContext => a.actorOf(props, StreamSupervisor.nextName()) @@ -103,11 +104,13 @@ object ActorMaterializer { namePrefix: String, system: ExtendedActorSystem): ActorMaterializer = { val haveShutDown = new AtomicBoolean(false) + val attributes = materializerSettings.toAttributes new PhasedFusingActorMaterializer( system, materializerSettings, + attributes, system.dispatchers, - system.systemActorOf(StreamSupervisor.props(materializerSettings, haveShutDown), StreamSupervisor.nextName()), + system.systemActorOf(StreamSupervisor.props(attributes, haveShutDown), StreamSupervisor.nextName()), haveShutDown, FlowNames(system).name.copy(namePrefix)) } @@ -138,6 +141,7 @@ object ActorMaterializer { * the processing steps. The default `namePrefix` is `"flow"`. The actor names are built up of * `namePrefix-flowNumber-flowStepNumber-stepName`. */ + @silent("deprecated") def create(context: ActorRefFactory, namePrefix: String): ActorMaterializer = { val system = actorSystemOf(context) val settings = ActorMaterializerSettings(system) @@ -206,6 +210,9 @@ private[akka] object ActorMaterializerHelper { */ abstract class ActorMaterializer extends Materializer with MaterializerLoggingProvider { + @deprecated( + "Use attributes to access settings from stages, see https://doc.akka.io/docs/akka/2.6/project/migration-guide-2.5.x-2.6.x.html", + "2.6.0") def settings: ActorMaterializerSettings /** @@ -271,11 +278,13 @@ object ActorMaterializerSettings { /** * Create [[ActorMaterializerSettings]] from individual settings (Scala). + * + * Prefer using either config for defaults or attributes for per-stream config. + * See migration guide for details https://doc.akka.io/docs/akka/2.6/project/migration-guide-2.5.x-2.6.x.html" */ - @Deprecated @deprecated( - "Create the settings using the apply(system) or apply(config) method, and then modify them using the .with methods.", - since = "2.5.10") + "Use config or attributes to configure the materializer. See migration guide for details https://doc.akka.io/docs/akka/2.6/project/migration-guide-2.5.x-2.6.x.html", + "2.6.0") def apply( initialInputBufferSize: Int, maxInputBufferSize: Int, @@ -308,13 +317,25 @@ object ActorMaterializerSettings { /** * Create [[ActorMaterializerSettings]] from the settings of an [[akka.actor.ActorSystem]] (Scala). + * + * Prefer using either config for defaults or attributes for per-stream config. + * See migration guide for details https://doc.akka.io/docs/akka/2.6/project/migration-guide-2.5.x-2.6.x.html" */ + @deprecated( + "Use config or attributes to configure the materializer. See migration guide for details https://doc.akka.io/docs/akka/2.6/project/migration-guide-2.5.x-2.6.x.html", + "2.6.0") def apply(system: ActorSystem): ActorMaterializerSettings = apply(system.settings.config.getConfig("akka.stream.materializer")) /** * Create [[ActorMaterializerSettings]] from a Config subsection (Scala). + * + * Prefer using either config for defaults or attributes for per-stream config. + * See migration guide for details https://doc.akka.io/docs/akka/2.6/project/migration-guide-2.5.x-2.6.x.html" */ + @deprecated( + "Use config or attributes to configure the materializer. See migration guide for details https://doc.akka.io/docs/akka/2.6/project/migration-guide-2.5.x-2.6.x.html", + "2.6.0") def apply(config: Config): ActorMaterializerSettings = new ActorMaterializerSettings( initialInputBufferSize = config.getInt("initial-input-buffer-size"), @@ -334,11 +355,13 @@ object ActorMaterializerSettings { /** * Create [[ActorMaterializerSettings]] from individual settings (Java). + * + * Prefer using either config for defaults or attributes for per-stream config. + * See migration guide for details https://doc.akka.io/docs/akka/2.6/project/migration-guide-2.5.x-2.6.x.html" */ - @Deprecated @deprecated( - "Create the settings using the create(system) or create(config) method, and then modify them using the .with methods.", - since = "2.5.10") + "Use config or attributes to configure the materializer. See migration guide for details https://doc.akka.io/docs/akka/2.6/project/migration-guide-2.5.x-2.6.x.html", + "2.6.0") def create( initialInputBufferSize: Int, maxInputBufferSize: Int, @@ -372,12 +395,21 @@ object ActorMaterializerSettings { /** * Create [[ActorMaterializerSettings]] from the settings of an [[akka.actor.ActorSystem]] (Java). */ + @deprecated( + "Use config or attributes to configure the materializer. See migration guide for details https://doc.akka.io/docs/akka/2.6/project/migration-guide-2.5.x-2.6.x.html", + "2.6.0") def create(system: ActorSystem): ActorMaterializerSettings = apply(system) /** * Create [[ActorMaterializerSettings]] from a Config subsection (Java). + * + * Prefer using either config for defaults or attributes for per-stream config. + * See migration guide for details https://doc.akka.io/docs/akka/2.6/project/migration-guide-2.5.x-2.6.x.html" */ + @deprecated( + "Use config or attributes to configure the materializer. See migration guide for details https://doc.akka.io/docs/akka/2.6/project/migration-guide-2.5.x-2.6.x.html", + "2.6.0") def create(config: Config): ActorMaterializerSettings = apply(config) @@ -389,6 +421,7 @@ object ActorMaterializerSettings { * * The constructor is not public API, use create or apply on the [[ActorMaterializerSettings]] companion instead. */ +@silent("deprecated") final class ActorMaterializerSettings @InternalApi private ( /* * Important note: `initialInputBufferSize`, `maxInputBufferSize`, `dispatcher` and @@ -396,19 +429,30 @@ final class ActorMaterializerSettings @InternalApi private ( * since these settings allow for overriding using [[Attributes]]. They must always be gotten from the effective * attributes. */ + @deprecated("Use attribute 'Attributes.InputBuffer' to read the concrete setting value", "2.6.0") val initialInputBufferSize: Int, + @deprecated("Use attribute 'Attributes.InputBuffer' to read the concrete setting value", "2.6.0") val maxInputBufferSize: Int, + @deprecated("Use attribute 'ActorAttributes.Dispatcher' to read the concrete setting value", "2.6.0") val dispatcher: String, + @deprecated("Use attribute 'ActorAttributes.SupervisionStrategy' to read the concrete setting value", "2.6.0") val supervisionDecider: Supervision.Decider, val subscriptionTimeoutSettings: StreamSubscriptionTimeoutSettings, + @deprecated("Use attribute 'ActorAttributes.DebugLogging' to read the concrete setting value", "2.6.0") val debugLogging: Boolean, + @deprecated("Use attribute 'ActorAttributes.OutputBurstLimit' to read the concrete setting value", "2.6.0") val outputBurstLimit: Int, + @deprecated("Use attribute 'ActorAttributes.FuzzingMode' to read the concrete setting value", "2.6.0") val fuzzingMode: Boolean, + @deprecated("No longer has any effect", "2.6.0") val autoFusing: Boolean, + @deprecated("Use attribute 'ActorAttributes.MaxFixedBufferSize' to read the concrete setting value", "2.6.0") val maxFixedBufferSize: Int, + @deprecated("Use attribute 'ActorAttributes.SyncProcessingLimit' to read the concrete setting value", "2.6.0") val syncProcessingLimit: Int, val ioSettings: IOSettings, val streamRefSettings: StreamRefSettings, + @deprecated("Use attribute 'ActorAttributes.BlockingIoDispatcher' to read the concrete setting value", "2.6.0") val blockingIoDispatcher: String) { require(initialInputBufferSize > 0, "initialInputBufferSize must be > 0") @@ -554,6 +598,7 @@ final class ActorMaterializerSettings @InternalApi private ( * FIXME: this is used for all kinds of buffers, not only the stream actor, some use initial some use max, * document and or fix if it should not be like that. Search for get[Attributes.InputBuffer] to see how it is used */ + @deprecated("Use attribute 'Attributes.InputBuffer' to change setting value", "2.6.0") def withInputBuffer(initialSize: Int, maxSize: Int): ActorMaterializerSettings = { if (initialSize == this.initialInputBufferSize && maxSize == this.maxInputBufferSize) this else copy(initialInputBufferSize = initialSize, maxInputBufferSize = maxSize) @@ -564,6 +609,7 @@ final class ActorMaterializerSettings @InternalApi private ( * with the [[ActorMaterializer]]. This can be overridden for individual parts of the * stream topology by using [[akka.stream.Attributes#dispatcher]]. */ + @deprecated("Use attribute 'ActorAttributes.Dispatcher' to change setting value", "2.6.0") def withDispatcher(dispatcher: String): ActorMaterializerSettings = { if (this.dispatcher == dispatcher) this else copy(dispatcher = dispatcher) @@ -577,6 +623,7 @@ final class ActorMaterializerSettings @InternalApi private ( * Note that supervision in streams are implemented on a per operator basis and is not supported * by every operator. */ + @deprecated("Use attribute 'ActorAttributes.supervisionStrategy' to change setting value", "2.6.0") def withSupervisionStrategy(decider: Supervision.Decider): ActorMaterializerSettings = { if (decider eq this.supervisionDecider) this else copy(supervisionDecider = decider) @@ -590,6 +637,7 @@ final class ActorMaterializerSettings @InternalApi private ( * Note that supervision in streams are implemented on a per operator basis and is not supported * by every operator. */ + @deprecated("Use attribute 'ActorAttributes.SupervisionStrategy' to change setting value", "2.6.0") def withSupervisionStrategy( decider: function.Function[Throwable, Supervision.Directive]): ActorMaterializerSettings = { import Supervision._ @@ -605,6 +653,7 @@ final class ActorMaterializerSettings @InternalApi private ( * Test utility: fuzzing mode means that GraphStage events are not processed * in FIFO order within a fused subgraph, but randomized. */ + @deprecated("Use attribute 'ActorAttributes.FuzzingMode' to change setting value", "2.6.0") def withFuzzing(enable: Boolean): ActorMaterializerSettings = if (enable == this.fuzzingMode) this else copy(fuzzingMode = enable) @@ -612,6 +661,7 @@ final class ActorMaterializerSettings @InternalApi private ( /** * Maximum number of elements emitted in batch if downstream signals large demand. */ + @deprecated("Use attribute 'ActorAttributes.OutputBurstLimit' to change setting value", "2.6.0") def withOutputBurstLimit(limit: Int): ActorMaterializerSettings = if (limit == this.outputBurstLimit) this else copy(outputBurstLimit = limit) @@ -619,6 +669,7 @@ final class ActorMaterializerSettings @InternalApi private ( /** * Limit for number of messages that can be processed synchronously in stream to substream communication */ + @deprecated("Use attribute 'ActorAttributes.SyncProcessingLimit' to change setting value", "2.6.0") def withSyncProcessingLimit(limit: Int): ActorMaterializerSettings = if (limit == this.syncProcessingLimit) this else copy(syncProcessingLimit = limit) @@ -626,6 +677,7 @@ final class ActorMaterializerSettings @InternalApi private ( /** * Enable to log all elements that are dropped due to failures (at DEBUG level). */ + @deprecated("Use attribute 'ActorAttributes.DebugLogging' to change setting value", "2.6.0") def withDebugLogging(enable: Boolean): ActorMaterializerSettings = if (enable == this.debugLogging) this else copy(debugLogging = enable) @@ -635,6 +687,7 @@ final class ActorMaterializerSettings @InternalApi private ( * This defaults to a large value because it is usually better to fail early when * system memory is not sufficient to hold the buffer. */ + @deprecated("Use attribute 'ActorAttributes.MaxFixedBufferSize' to change setting value", "2.6.0") def withMaxFixedBufferSize(size: Int): ActorMaterializerSettings = if (size == this.maxFixedBufferSize) this else copy(maxFixedBufferSize = size) @@ -656,6 +709,7 @@ final class ActorMaterializerSettings @InternalApi private ( if (streamRefSettings == this.streamRefSettings) this else copy(streamRefSettings = streamRefSettings) + @deprecated("Use attribute 'ActorAttributes.BlockingIoDispatcher' to change setting value", "2.6.0") def withBlockingIoDispatcher(newBlockingIoDispatcher: String): ActorMaterializerSettings = if (newBlockingIoDispatcher == blockingIoDispatcher) this else copy(blockingIoDispatcher = newBlockingIoDispatcher) @@ -682,6 +736,26 @@ final class ActorMaterializerSettings @InternalApi private ( case _ => false } + /** + * INTERNAL API + */ + @InternalApi + private[akka] def toAttributes: Attributes = + Attributes( + // these are the core stream/materializer settings, ad hoc handling of defaults for the stage specific ones + // for stream refs and io live with the respective stages + Attributes.InputBuffer(initialInputBufferSize, maxInputBufferSize) :: + ActorAttributes.Dispatcher(dispatcher) :: + ActorAttributes.SupervisionStrategy(supervisionDecider) :: + ActorAttributes.DebugLogging(debugLogging) :: + ActorAttributes + .StreamSubscriptionTimeout(subscriptionTimeoutSettings.timeout, subscriptionTimeoutSettings.mode) :: + ActorAttributes.OutputBurstLimit(outputBurstLimit) :: + ActorAttributes.FuzzingMode(fuzzingMode) :: + ActorAttributes.MaxFixedBufferSize(maxFixedBufferSize) :: + ActorAttributes.SyncProcessingLimit(syncProcessingLimit) :: + ActorAttributes.BlockingIoDispatcher(blockingIoDispatcher) :: Nil) + override def toString: String = s"ActorMaterializerSettings($initialInputBufferSize,$maxInputBufferSize," + s"$dispatcher,$supervisionDecider,$subscriptionTimeoutSettings,$debugLogging,$outputBurstLimit," + @@ -689,27 +763,48 @@ final class ActorMaterializerSettings @InternalApi private ( } object IOSettings { + @deprecated( + "Use setting 'akka.stream.materializer.io.tcp.write-buffer-size' or attribute TcpAttributes.writeBufferSize instead", + "2.6.0") def apply(system: ActorSystem): IOSettings = apply(system.settings.config.getConfig("akka.stream.materializer.io")) + @deprecated( + "Use setting 'akka.stream.materializer.io.tcp.write-buffer-size' or attribute TcpAttributes.writeBufferSize instead", + "2.6.0") def apply(config: Config): IOSettings = new IOSettings(tcpWriteBufferSize = math.min(Int.MaxValue, config.getBytes("tcp.write-buffer-size")).toInt) + @deprecated( + "Use setting 'akka.stream.materializer.io.tcp.write-buffer-size' or attribute TcpAttributes.writeBufferSize instead", + "2.6.0") def apply(tcpWriteBufferSize: Int): IOSettings = new IOSettings(tcpWriteBufferSize) /** Java API */ + @deprecated( + "Use setting 'akka.stream.materializer.io.tcp.write-buffer-size' or attribute TcpAttributes.writeBufferSize instead", + "2.6.0") def create(config: Config) = apply(config) /** Java API */ + @deprecated( + "Use setting 'akka.stream.materializer.io.tcp.write-buffer-size' or attribute TcpAttributes.writeBufferSize instead", + "2.6.0") def create(system: ActorSystem) = apply(system) /** Java API */ + @deprecated( + "Use setting 'akka.stream.materializer.io.tcp.write-buffer-size' or attribute TcpAttributes.writeBufferSize instead", + "2.6.0") def create(tcpWriteBufferSize: Int): IOSettings = apply(tcpWriteBufferSize) } -final class IOSettings private (val tcpWriteBufferSize: Int) { +@silent("deprecated") +final class IOSettings private ( + @deprecated("Use attribute 'TcpAttributes.TcpWriteBufferSize' to read the concrete setting value", "2.6.0") + val tcpWriteBufferSize: Int) { def withTcpWriteBufferSize(value: Int): IOSettings = copy(tcpWriteBufferSize = value) @@ -766,8 +861,13 @@ object StreamSubscriptionTimeoutSettings { * Leaked publishers and subscribers are cleaned up when they are not used within a given * deadline, configured by [[StreamSubscriptionTimeoutSettings]]. */ +@silent("deprecated") final class StreamSubscriptionTimeoutSettings( + @deprecated( + "Use attribute 'ActorAttributes.StreamSubscriptionTimeoutMode' to read the concrete setting value", + "2.6.0") val mode: StreamSubscriptionTimeoutTerminationMode, + @deprecated("Use attribute 'ActorAttributes.StreamSubscriptionTimeout' to read the concrete setting value", "2.6.0") val timeout: FiniteDuration) { override def equals(other: Any): Boolean = other match { case s: StreamSubscriptionTimeoutSettings => s.mode == mode && s.timeout == timeout diff --git a/akka-stream/src/main/scala/akka/stream/Attributes.scala b/akka-stream/src/main/scala/akka/stream/Attributes.scala index 1eb56e0f5c..500b25a60d 100644 --- a/akka-stream/src/main/scala/akka/stream/Attributes.scala +++ b/akka-stream/src/main/scala/akka/stream/Attributes.scala @@ -12,9 +12,12 @@ import scala.annotation.tailrec import scala.reflect.{ classTag, ClassTag } import akka.japi.function import java.net.URLEncoder +import java.time.Duration +import akka.annotation.DoNotInherit import akka.annotation.InternalApi import akka.stream.impl.TraversalBuilder +import akka.util.JavaDurationConverters._ import scala.compat.java8.OptionConverters._ import akka.util.{ ByteString, OptionVal } @@ -285,10 +288,25 @@ object Attributes { trait Attribute + /** + * Attributes that are always present (is defined with default values by the materializer) + * + * Not for user extension + */ + @DoNotInherit sealed trait MandatoryAttribute extends Attribute final case class Name(n: String) extends Attribute + + /** + * Each asynchronous piece of a materialized stream topology is executed by one Actor + * that manages an input buffer for all inlets of its shape. This attribute configures + * the initial and maximal input buffer in number of elements for each inlet. + * + * Use factory method [[Attributes#inputBuffer]] to create instances. + */ final case class InputBuffer(initial: Int, max: Int) extends MandatoryAttribute + final case class LogLevels(onElement: Logging.LogLevel, onFinish: Logging.LogLevel, onFailure: Logging.LogLevel) extends Attribute final case object AsyncBoundary extends Attribute @@ -349,7 +367,9 @@ object Attributes { else Attributes(Name(URLEncoder.encode(name, ByteString.UTF_8))) /** - * Specifies the initial and maximum size of the input buffer. + * Each asynchronous piece of a materialized stream topology is executed by one Actor + * that manages an input buffer for all inlets of its shape. This attribute configures + * the initial and maximal input buffer in number of elements for each inlet. */ def inputBuffer(initial: Int, max: Int): Attributes = Attributes(InputBuffer(initial, max)) @@ -403,6 +423,12 @@ object Attributes { */ object ActorAttributes { import Attributes._ + + /** + * Configures the dispatcher to be used by streams. + * + * Use factory method [[ActorAttributes#dispatcher]] to create instances. + */ final case class Dispatcher(dispatcher: String) extends MandatoryAttribute final case class SupervisionStrategy(decider: Supervision.Decider) extends MandatoryAttribute @@ -419,6 +445,8 @@ object ActorAttributes { * * Operators supporting supervision strategies explicitly document that they do so. If a operator does not document * support for these, it should be assumed it does not support supervision. + * + * For the Java API see [[#withSupervisionStrategy]] */ def supervisionStrategy(decider: Supervision.Decider): Attributes = Attributes(SupervisionStrategy(decider)) @@ -428,6 +456,8 @@ object ActorAttributes { * * Operators supporting supervision strategies explicitly document that they do so. If a operator does not document * support for these, it should be assumed it does not support supervision. + * + * For the Scala API see [[#supervisionStrategy]] */ def withSupervisionStrategy(decider: function.Function[Throwable, Supervision.Directive]): Attributes = ActorAttributes.supervisionStrategy(decider.apply) @@ -467,6 +497,103 @@ object ActorAttributes { onFailure: Logging.LogLevel = Logging.ErrorLevel) = Attributes(LogLevels(onElement, onFinish, onFailure)) + /** + * Enables additional low level troubleshooting logging at DEBUG log level + * + * Use factory method [[#debugLogging]] to create. + */ + final case class DebugLogging(enabled: Boolean) extends MandatoryAttribute + + /** + * Enables additional low level troubleshooting logging at DEBUG log level + */ + def debugLogging(enabled: Boolean): Attributes = + Attributes(DebugLogging(enabled)) + + /** + * Defines a timeout for stream subscription and what action to take when that hits. + * + * Use factory method `streamSubscriptionTimeout` to create. + */ + final case class StreamSubscriptionTimeout(timeout: FiniteDuration, mode: StreamSubscriptionTimeoutTerminationMode) + extends MandatoryAttribute + + /** + * Scala API: Defines a timeout for stream subscription and what action to take when that hits. + */ + def streamSubscriptionTimeout(timeout: FiniteDuration, mode: StreamSubscriptionTimeoutTerminationMode): Attributes = + Attributes(StreamSubscriptionTimeout(timeout, mode)) + + /** + * Java API: Defines a timeout for stream subscription and what action to take when that hits. + */ + def streamSubscriptionTimeout(timeout: Duration, mode: StreamSubscriptionTimeoutTerminationMode): Attributes = + streamSubscriptionTimeout(timeout.asScala, mode) + + /** + * Maximum number of elements emitted in batch if downstream signals large demand. + * + * Use factory method [[#outputBurstLimit]] to create. + */ + final case class OutputBurstLimit(limit: Int) extends MandatoryAttribute + + /** + * Maximum number of elements emitted in batch if downstream signals large demand. + */ + def outputBurstLimit(limit: Int): Attributes = + Attributes(OutputBurstLimit(limit)) + + /** + * Test utility: fuzzing mode means that GraphStage events are not processed + * in FIFO order within a fused subgraph, but randomized. + * + * Use factory method [[#fuzzingMode]] to create. + */ + final case class FuzzingMode(enabled: Boolean) extends MandatoryAttribute + + /** + * Test utility: fuzzing mode means that GraphStage events are not processed + * in FIFO order within a fused subgraph, but randomized. + */ + def fuzzingMode(enabled: Boolean): Attributes = + Attributes(FuzzingMode(enabled)) + + /** + * Configure the maximum buffer size for which a FixedSizeBuffer will be preallocated. + * This defaults to a large value because it is usually better to fail early when + * system memory is not sufficient to hold the buffer. + * + * Use factory method [[#maxFixedBufferSize]] to create. + */ + final case class MaxFixedBufferSize(size: Int) extends MandatoryAttribute + + /** + * Configure the maximum buffer size for which a FixedSizeBuffer will be preallocated. + * This defaults to a large value because it is usually better to fail early when + * system memory is not sufficient to hold the buffer. + */ + def maxFixedBufferSize(size: Int): Attributes = + Attributes(MaxFixedBufferSize(size: Int)) + + /** + * Limit for number of messages that can be processed synchronously in stream to substream communication. + * + * Use factory method [[#syncProcessingLimit]] to create. + */ + final case class SyncProcessingLimit(limit: Int) extends MandatoryAttribute + + /** + * Limit for number of messages that can be processed synchronously in stream to substream communication + */ + def syncProcessingLimit(limit: Int): Attributes = + Attributes(SyncProcessingLimit(limit)) + + /** + * FIXME Is this really needed anymore now that we have indirect dispatcher config? + */ + final case class BlockingIoDispatcher(dispatcher: String) extends MandatoryAttribute + def blockingIoDispatcher(dispatcher: String): Attributes = + Attributes(BlockingIoDispatcher(dispatcher)) } /** @@ -476,14 +603,57 @@ object ActorAttributes { object StreamRefAttributes { import Attributes._ - /** Attributes specific to stream refs. */ + /** Attributes specific to stream refs. + * + * Not for user extension. + */ + @DoNotInherit sealed trait StreamRefAttribute extends Attribute final case class SubscriptionTimeout(timeout: FiniteDuration) extends StreamRefAttribute + final case class BufferCapacity(capacity: Int) extends StreamRefAttribute { + require(capacity > 0, "Buffer capacity must be > 0") + } + final case class DemandRedeliveryInterval(timeout: FiniteDuration) extends StreamRefAttribute + final case class FinalTerminationSignalDeadline(timeout: FiniteDuration) extends StreamRefAttribute /** - * Specifies the subscription timeout within which the remote side MUST subscribe to the handed out stream reference. + * Scala API: Specifies the subscription timeout within which the remote side MUST subscribe to the handed out stream reference. */ def subscriptionTimeout(timeout: FiniteDuration): Attributes = Attributes(SubscriptionTimeout(timeout)) + /** + * Java API: Specifies the subscription timeout within which the remote side MUST subscribe to the handed out stream reference. + */ + def subscriptionTimeout(timeout: Duration): Attributes = subscriptionTimeout(timeout.asScala) + + /** + * Specifies the size of the buffer on the receiving side that is eagerly filled even without demand. + */ + def bufferCapacity(capacity: Int): Attributes = Attributes(BufferCapacity(capacity)) + + /** + * Scala API: If no new elements arrive within this timeout, demand is redelivered. + */ + def demandRedeliveryInterval(timeout: FiniteDuration): Attributes = + Attributes(DemandRedeliveryInterval(timeout)) + + /** + * Java API: If no new elements arrive within this timeout, demand is redelivered. + */ + def demandRedeliveryInterval(timeout: Duration): Attributes = + demandRedeliveryInterval(timeout.asScala) + + /** + * Scala API: The time between the Terminated signal being received and when the local SourceRef determines to fail itself + */ + def finalTerminationSignalDeadline(timeout: FiniteDuration): Attributes = + Attributes(FinalTerminationSignalDeadline(timeout)) + + /** + * Java API: The time between the Terminated signal being received and when the local SourceRef determines to fail itself + */ + def finalTerminationSignalDeadline(timeout: Duration): Attributes = + finalTerminationSignalDeadline(timeout.asScala) + } diff --git a/akka-stream/src/main/scala/akka/stream/StreamRefSettings.scala b/akka-stream/src/main/scala/akka/stream/StreamRefSettings.scala index 2eb2ca3d7e..c14dc306b0 100644 --- a/akka-stream/src/main/scala/akka/stream/StreamRefSettings.scala +++ b/akka-stream/src/main/scala/akka/stream/StreamRefSettings.scala @@ -9,24 +9,38 @@ import java.util.concurrent.TimeUnit import akka.actor.ActorSystem import akka.annotation.DoNotInherit import akka.stream.impl.streamref.StreamRefSettingsImpl +import com.github.ghik.silencer.silent import com.typesafe.config.Config import scala.concurrent.duration._ +@silent("deprecated") object StreamRefSettings { /** Java API */ + @deprecated( + "Use attributes on the Runnable graph or change the defaults in configuration, see migration guide for details https://doc.akka.io/docs/akka/2.6/project/migration-guide-2.5.x-2.6.x.html", + since = "2.6.0") def create(system: ActorSystem): StreamRefSettings = apply(system) /** Scala API */ + @deprecated( + "Use attributes on the Runnable graph or change the defaults in configuration, see migration guide for details https://doc.akka.io/docs/akka/2.6/project/migration-guide-2.5.x-2.6.x.html", + since = "2.6.0") def apply(system: ActorSystem): StreamRefSettings = { apply(system.settings.config.getConfig("akka.stream.materializer.stream-ref")) } /** Java API */ + @deprecated( + "Use attributes on the Runnable graph or change the defaults in configuration, see migration guide for details https://doc.akka.io/docs/akka/2.6/project/migration-guide-2.5.x-2.6.x.html", + since = "2.6.0") def create(c: Config): StreamRefSettings = apply(c) /** Scala API */ + @deprecated( + "Use attributes on the Runnable graph or change the defaults in configuration, see migration guide for details https://doc.akka.io/docs/akka/2.6/project/migration-guide-2.5.x-2.6.x.html", + since = "2.6.0") def apply(c: Config): StreamRefSettings = { StreamRefSettingsImpl( bufferCapacity = c.getInt("buffer-capacity"), @@ -41,10 +55,19 @@ object StreamRefSettings { * More detailed documentation about each of the settings is available in `reference.conf`. */ @DoNotInherit +@silent("deprecated") trait StreamRefSettings { + @deprecated("Use attribute 'StreamRefAttributes.BufferCapacity' to read the concrete setting value", "2.6.0") def bufferCapacity: Int + @deprecated( + "Use attribute 'StreamRefAttributes.DemandRedeliveryInterval' to read the concrete setting value", + "2.6.0") def demandRedeliveryInterval: FiniteDuration + @deprecated("Use attribute 'StreamRefAttributes.SubscriptionTimeout' to read the concrete setting value", "2.6.0") def subscriptionTimeout: FiniteDuration + @deprecated( + "Use attribute 'StreamRefAttributes.FinalTerminationSignalDeadline' to read the concrete setting value", + "2.6.0") def finalTerminationSignalDeadline: FiniteDuration // --- with... methods --- diff --git a/akka-stream/src/main/scala/akka/stream/SystemMaterializer.scala b/akka-stream/src/main/scala/akka/stream/SystemMaterializer.scala index 7c9e0ae9c2..e4f9412554 100644 --- a/akka-stream/src/main/scala/akka/stream/SystemMaterializer.scala +++ b/akka-stream/src/main/scala/akka/stream/SystemMaterializer.scala @@ -9,6 +9,7 @@ import akka.actor.ExtendedActorSystem import akka.actor.Extension import akka.actor.ExtensionId import akka.actor.ExtensionIdProvider +import com.github.ghik.silencer.silent /** * The system materializer is a default materializer to use for most cases running streams, it is a single instance @@ -26,6 +27,7 @@ object SystemMaterializer extends ExtensionId[SystemMaterializer] with Extension } final class SystemMaterializer(system: ExtendedActorSystem) extends Extension { + @silent("deprecated") val materializer = { val settings = ActorMaterializerSettings(system) ActorMaterializer.systemMaterializer(settings, "default", system) 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 4c5e6e5509..5e712793e8 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/ActorMaterializerImpl.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/ActorMaterializerImpl.scala @@ -49,7 +49,8 @@ import scala.concurrent.{ Await, ExecutionContextExecutor, Future } props.withDispatcher(context.effectiveAttributes.mandatoryAttribute[ActorAttributes.Dispatcher].dispatcher) case ActorAttributes.IODispatcher.dispatcher => // this one is actually not a dispatcher but a relative config key pointing containing the actual dispatcher name - props.withDispatcher(settings.blockingIoDispatcher) + val actual = context.effectiveAttributes.mandatoryAttribute[ActorAttributes.BlockingIoDispatcher].dispatcher + props.withDispatcher(actual) case _ => props } @@ -104,7 +105,7 @@ private[akka] class SubFusingActorMaterializerImpl( attributes: Attributes, materializer: PhasedFusingActorMaterializer, islandName: String): PhaseIsland[Any] = { - new GraphStageIsland(settings, attributes, materializer, islandName, OptionVal(registerShell)) + new GraphStageIsland(attributes, materializer, islandName, OptionVal(registerShell)) .asInstanceOf[PhaseIsland[Any]] } } @@ -173,8 +174,10 @@ private[akka] class SubFusingActorMaterializerImpl( * INTERNAL API */ @InternalApi private[akka] object StreamSupervisor { - def props(settings: ActorMaterializerSettings, haveShutDown: AtomicBoolean): Props = - Props(new StreamSupervisor(haveShutDown)).withDeploy(Deploy.local).withDispatcher(settings.dispatcher) + def props(attributes: Attributes, haveShutDown: AtomicBoolean): Props = + Props(new StreamSupervisor(haveShutDown)) + .withDeploy(Deploy.local) + .withDispatcher(attributes.mandatoryAttribute[ActorAttributes.Dispatcher].dispatcher) private[stream] val baseName = "StreamSupervisor" private val actorName = SeqActorName(baseName) def nextName(): String = actorName.next() diff --git a/akka-stream/src/main/scala/akka/stream/impl/ActorProcessor.scala b/akka-stream/src/main/scala/akka/stream/impl/ActorProcessor.scala index 8d720e9722..091fa78f42 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/ActorProcessor.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/ActorProcessor.scala @@ -6,11 +6,12 @@ package akka.stream.impl import akka.actor._ import akka.annotation.InternalApi -import akka.stream.{ AbruptTerminationException, ActorMaterializerSettings, Attributes } +import akka.stream.{ AbruptTerminationException, Attributes } import akka.stream.actor.ActorSubscriber.OnSubscribe import akka.stream.actor.ActorSubscriberMessage.{ OnComplete, OnError, OnNext } import org.reactivestreams.{ Processor, Subscriber, Subscription } import akka.event.Logging +import akka.stream.ActorAttributes import akka.util.unused /** @@ -257,13 +258,13 @@ private[akka] object ActorProcessorImpl { /** * INTERNAL API */ -@InternalApi private[akka] abstract class ActorProcessorImpl( - attributes: Attributes, - val settings: ActorMaterializerSettings) +@InternalApi private[akka] abstract class ActorProcessorImpl(attributes: Attributes) extends Actor with ActorLogging with Pump { + private val debugLoggingEnabled = attributes.mandatoryAttribute[ActorAttributes.DebugLogging].enabled + protected val primaryInputs: Inputs = { val initialInputBufferSize = attributes.mandatoryAttribute[Attributes.InputBuffer].initial new BatchingInputBuffer(initialInputBufferSize, this) { @@ -290,7 +291,7 @@ private[akka] object ActorProcessorImpl { protected def onError(e: Throwable): Unit = fail(e) protected def fail(e: Throwable): Unit = { - if (settings.debugLogging) + if (debugLoggingEnabled) log.debug("fail due to: {}", e.getMessage) primaryInputs.cancel() primaryOutputs.error(e) diff --git a/akka-stream/src/main/scala/akka/stream/impl/ActorRefSource.scala b/akka-stream/src/main/scala/akka/stream/impl/ActorRefSource.scala index c713d9f92a..3326d1bdfa 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/ActorRefSource.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/ActorRefSource.scala @@ -42,7 +42,7 @@ private object ActorRefSource { private val buffer: OptionVal[Buffer[T]] = if (maxBuffer != 0) - OptionVal(Buffer(maxBuffer, eagerMaterializer)) + OptionVal(Buffer(maxBuffer, inheritedAttributes)) else { OptionVal.None // for backwards compatibility with old actor publisher based implementation } diff --git a/akka-stream/src/main/scala/akka/stream/impl/Buffers.scala b/akka-stream/src/main/scala/akka/stream/impl/Buffers.scala index f5efaa18e7..3c78b8fd88 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/Buffers.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/Buffers.scala @@ -32,14 +32,8 @@ private[akka] object Buffer { val FixedQueueSize = 128 val FixedQueueMask = 127 - def apply[T](size: Int, settings: ActorMaterializerSettings): Buffer[T] = - apply(size, settings.maxFixedBufferSize) - - def apply[T](size: Int, materializer: Materializer): Buffer[T] = - materializer match { - case m: ActorMaterializer => apply(size, m.settings.maxFixedBufferSize) - case _ => apply(size, 1000000000) - } + def apply[T](size: Int, effectiveAttributes: Attributes): Buffer[T] = + apply(size, effectiveAttributes.mandatoryAttribute[ActorAttributes.MaxFixedBufferSize].size) def apply[T](size: Int, max: Int): Buffer[T] = if (size < FixedQueueSize || size < max) FixedSizeBuffer(size) diff --git a/akka-stream/src/main/scala/akka/stream/impl/FanIn.scala b/akka-stream/src/main/scala/akka/stream/impl/FanIn.scala index e967ada36d..037bfb6d31 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/FanIn.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/FanIn.scala @@ -6,7 +6,9 @@ package akka.stream.impl import akka.actor._ import akka.annotation.{ DoNotInherit, InternalApi } -import akka.stream.{ AbruptTerminationException, ActorMaterializerSettings } +import akka.stream.ActorAttributes +import akka.stream.Attributes +import akka.stream.AbruptTerminationException import akka.stream.actor.{ ActorSubscriber, ActorSubscriberMessage } import akka.util.unused import org.reactivestreams.{ Subscriber, Subscription } @@ -260,17 +262,21 @@ import org.reactivestreams.{ Subscriber, Subscription } /** * INTERNAL API */ -@DoNotInherit private[akka] class FanIn(val settings: ActorMaterializerSettings, val inputCount: Int) +@DoNotInherit private[akka] class FanIn(attributes: Attributes, val inputCount: Int) extends Actor with ActorLogging with Pump { import FanIn._ protected val primaryOutputs: Outputs = new SimpleOutputs(self, this) - protected val inputBunch = new InputBunch(inputCount, settings.maxInputBufferSize, this) { - override def onError(input: Int, e: Throwable): Unit = fail(e) - override def onCompleteWhenNoInput(): Unit = pumpFinished() + protected val inputBunch = { + val maxInputBufferSize = attributes.mandatoryAttribute[Attributes.InputBuffer].max + new InputBunch(inputCount, maxInputBufferSize, this) { + override def onError(input: Int, e: Throwable): Unit = fail(e) + override def onCompleteWhenNoInput(): Unit = pumpFinished() + } } + private val debugLoggingEnabled = attributes.mandatoryAttribute[ActorAttributes.DebugLogging].enabled override def pumpFinished(): Unit = { inputBunch.cancel() @@ -281,7 +287,7 @@ import org.reactivestreams.{ Subscriber, Subscription } override def pumpFailed(e: Throwable): Unit = fail(e) protected def fail(e: Throwable): Unit = { - if (settings.debugLogging) + if (debugLoggingEnabled) log.debug("fail due to: {}", e.getMessage) nextPhase(completedPhase) primaryOutputs.error(e) diff --git a/akka-stream/src/main/scala/akka/stream/impl/FanOut.scala b/akka-stream/src/main/scala/akka/stream/impl/FanOut.scala index c1c023c90b..05d7492541 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/FanOut.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/FanOut.scala @@ -4,11 +4,13 @@ package akka.stream.impl -import akka.stream.{ AbruptTerminationException, ActorMaterializerSettings } +import akka.stream.AbruptTerminationException import scala.collection.immutable import akka.actor._ import akka.annotation.{ DoNotInherit, InternalApi } +import akka.stream.ActorAttributes +import akka.stream.Attributes import akka.util.unused import org.reactivestreams.Subscription @@ -257,16 +259,20 @@ import org.reactivestreams.Subscription /** * INTERNAL API */ -@DoNotInherit private[akka] abstract class FanOut(val settings: ActorMaterializerSettings, val outputCount: Int) +@DoNotInherit private[akka] abstract class FanOut(attributes: Attributes, val outputCount: Int) extends Actor with ActorLogging with Pump { import FanOut._ protected val outputBunch = new OutputBunch(outputCount, self, this) - protected val primaryInputs: Inputs = new BatchingInputBuffer(settings.maxInputBufferSize, this) { - override def onError(e: Throwable): Unit = fail(e) + protected val primaryInputs: Inputs = { + val maxInputBufferSize = attributes.mandatoryAttribute[Attributes.InputBuffer].max + new BatchingInputBuffer(maxInputBufferSize, this) { + override def onError(e: Throwable): Unit = fail(e) + } } + private val debugLoggingEnabled = attributes.mandatoryAttribute[ActorAttributes.DebugLogging].enabled override def pumpFinished(): Unit = { primaryInputs.cancel() @@ -277,7 +283,7 @@ import org.reactivestreams.Subscription override def pumpFailed(e: Throwable): Unit = fail(e) protected def fail(e: Throwable): Unit = { - if (settings.debugLogging) + if (debugLoggingEnabled) log.debug("fail due to: {}", e.getMessage) primaryInputs.cancel() outputBunch.cancel(e) @@ -301,15 +307,14 @@ import org.reactivestreams.Subscription * INTERNAL API */ @InternalApi private[akka] object Unzip { - def props(settings: ActorMaterializerSettings): Props = - Props(new Unzip(settings)).withDeploy(Deploy.local) + def props(attributes: Attributes): Props = + Props(new Unzip(attributes)).withDeploy(Deploy.local) } /** * INTERNAL API */ -@InternalApi private[akka] class Unzip(_settings: ActorMaterializerSettings) - extends FanOut(_settings, outputCount = 2) { +@InternalApi private[akka] class Unzip(attributes: Attributes) extends FanOut(attributes, outputCount = 2) { outputBunch.markAllOutputs() initialPhase( diff --git a/akka-stream/src/main/scala/akka/stream/impl/FanoutProcessor.scala b/akka-stream/src/main/scala/akka/stream/impl/FanoutProcessor.scala index c3cd23cb11..2fe67c188e 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/FanoutProcessor.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/FanoutProcessor.scala @@ -9,7 +9,7 @@ import akka.actor.ActorRef import akka.actor.Deploy import akka.actor.Props import akka.annotation.InternalApi -import akka.stream.ActorMaterializerSettings +import akka.stream.ActorAttributes.StreamSubscriptionTimeout import akka.stream.Attributes import akka.stream.StreamSubscriptionTimeoutTerminationMode import org.reactivestreams.Subscriber @@ -111,20 +111,22 @@ import org.reactivestreams.Subscriber * INTERNAL API */ @InternalApi private[akka] object FanoutProcessorImpl { - def props(attributes: Attributes, actorMaterializerSettings: ActorMaterializerSettings): Props = - Props(new FanoutProcessorImpl(attributes, actorMaterializerSettings)).withDeploy(Deploy.local) + def props(attributes: Attributes): Props = + Props(new FanoutProcessorImpl(attributes)).withDeploy(Deploy.local) } /** * INTERNAL API */ -@InternalApi private[akka] class FanoutProcessorImpl(attributes: Attributes, _settings: ActorMaterializerSettings) - extends ActorProcessorImpl(attributes, _settings) { +@InternalApi private[akka] class FanoutProcessorImpl(attributes: Attributes) extends ActorProcessorImpl(attributes) { - if (settings.subscriptionTimeoutSettings.mode != StreamSubscriptionTimeoutTerminationMode.noop) { - import context.dispatcher - context.system.scheduler - .scheduleOnce(_settings.subscriptionTimeoutSettings.timeout, self, ActorProcessorImpl.SubscriptionTimeout) + val timeoutMode = { + val StreamSubscriptionTimeout(timeout, mode) = attributes.mandatoryAttribute[StreamSubscriptionTimeout] + if (mode != StreamSubscriptionTimeoutTerminationMode.noop) { + import context.dispatcher + context.system.scheduler.scheduleOnce(timeout, self, ActorProcessorImpl.SubscriptionTimeout) + } + mode } override val primaryOutputs: FanoutOutputs = { @@ -151,7 +153,7 @@ import org.reactivestreams.Subscriber case ActorProcessorImpl.SubscriptionTimeout => import StreamSubscriptionTimeoutTerminationMode._ if (!primaryOutputs.subscribed) { - settings.subscriptionTimeoutSettings.mode match { + timeoutMode match { case CancelTermination => primaryInputs.cancel() context.stop(self) diff --git a/akka-stream/src/main/scala/akka/stream/impl/PhasedFusingActorMaterializer.scala b/akka-stream/src/main/scala/akka/stream/impl/PhasedFusingActorMaterializer.scala index aafa688b69..6eca111c41 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/PhasedFusingActorMaterializer.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/PhasedFusingActorMaterializer.scala @@ -26,6 +26,7 @@ import scala.collection.immutable.Map import scala.concurrent.duration.FiniteDuration import scala.concurrent.ExecutionContextExecutor import akka.util.OptionVal +import com.github.ghik.silencer.silent /** * INTERNAL API @@ -40,7 +41,7 @@ import akka.util.OptionVal effectiveAttributes: Attributes, materializer: PhasedFusingActorMaterializer, islandName: String): PhaseIsland[Any] = - new GraphStageIsland(settings, effectiveAttributes, materializer, islandName, subflowFuser = OptionVal.None) + new GraphStageIsland(effectiveAttributes, materializer, islandName, subflowFuser = OptionVal.None) .asInstanceOf[PhaseIsland[Any]] } @@ -79,17 +80,20 @@ import akka.util.OptionVal }, GraphStageTag -> DefaultPhase) + @silent("deprecated") @InternalApi private[akka] def apply()(implicit context: ActorRefFactory): ActorMaterializer = { val haveShutDown = new AtomicBoolean(false) val system = actorSystemOf(context) val materializerSettings = ActorMaterializerSettings(system) + val defaultAttributes = materializerSettings.toAttributes val streamSupervisor = - context.actorOf(StreamSupervisor.props(materializerSettings, haveShutDown), StreamSupervisor.nextName()) + context.actorOf(StreamSupervisor.props(defaultAttributes, haveShutDown), StreamSupervisor.nextName()) PhasedFusingActorMaterializer( system, materializerSettings, + defaultAttributes, system.dispatchers, streamSupervisor, haveShutDown, @@ -369,6 +373,16 @@ private final case class SavedIslandData( @InternalApi private[akka] case class PhasedFusingActorMaterializer( system: ActorSystem, override val settings: ActorMaterializerSettings, + /** + * Default attributes for the materializer, based on the [[ActorMaterializerSettings]] and + * are always seen as least specific, so any attribute specified in the graph "wins" over these. + * In addition to that this also guarantees that the attributes `InputBuffer`, `SupervisionStrategy`, + * and `Dispatcher` is _always_ present in the attributes and can be accessed through `Attributes.mandatoryAttribute` + * + * When these attributes are needed later in the materialization process it is important that + * they are gotten through the attributes and not through the [[ActorMaterializerSettings]] + */ + defaultAttributes: Attributes, dispatchers: Dispatchers, supervisor: ActorRef, haveShutDown: AtomicBoolean, @@ -378,17 +392,8 @@ private final case class SavedIslandData( 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.") - } - if (!settings.autoFusing) { - _logger.warning( - "Deprecated setting auto-fusing set to false. Since Akka 2.5.0 it does not have any effect " + - "and streams are always fused.") - } + private val fuzzingWarningDisabled = + system.settings.config.hasPath("akka.stream.secret-test-fuzzing-warning-disable") override def shutdown(): Unit = if (haveShutDown.compareAndSet(false, true)) supervisor ! PoisonPill @@ -399,23 +404,9 @@ private final case class SavedIslandData( private[this] def createFlowName(): String = flowNames.next() - /** - * Default attributes for the materializer, based on the [[ActorMaterializerSettings]] and - * are always seen as least specific, so any attribute specified in the graph "wins" over these. - * In addition to that this also guarantees that the attributes `InputBuffer`, `SupervisionStrategy`, - * and `Dispatcher` is _always_ present in the attributes and can be accessed through `Attributes.mandatoryAttribute` - * - * When these attributes are needed later in the materialization process it is important that the - * they are gotten through the attributes and not through the [[ActorMaterializerSettings]] - */ - val defaultAttributes: Attributes = { - Attributes( - Attributes.InputBuffer(settings.initialInputBufferSize, settings.maxInputBufferSize) :: - ActorAttributes.SupervisionStrategy(settings.supervisionDecider) :: - ActorAttributes.Dispatcher(settings.dispatcher) :: Nil) - } - - override lazy val executionContext: ExecutionContextExecutor = dispatchers.lookup(settings.dispatcher) + // note that this will never be overridden on a per-graph-stage basis regardless of more specific attributes + override lazy val executionContext: ExecutionContextExecutor = + dispatchers.lookup(defaultAttributes.mandatoryAttribute[ActorAttributes.Dispatcher].dispatcher) override def scheduleWithFixedDelay( initialDelay: FiniteDuration, @@ -455,18 +446,27 @@ private final case class SavedIslandData( defaultPhase: Phase[Any], phases: Map[IslandTag, Phase[Any]]): Mat = { if (isShutdown) throw new IllegalStateException("Trying to materialize stream after materializer has been shutdown") + + // combine default attributes with top-level runnable/closed graph shape attributes so that per-stream + // attributes overriding defaults are used also for the top level interpreter etc. + val defaultAndGraphAttributes = defaultAttributes and graph.traversalBuilder.attributes + if (defaultAndGraphAttributes.mandatoryAttribute[ActorAttributes.FuzzingMode].enabled && !fuzzingWarningDisabled) { + _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.") + } + val islandTracking = new IslandTracking( phases, settings, - defaultAttributes, + defaultAndGraphAttributes, defaultPhase, this, islandNamePrefix = createFlowName() + "-") var current: Traversal = graph.traversalBuilder.traversal - val attributesStack = new java.util.ArrayDeque[Attributes](8) - attributesStack.addLast(defaultAttributes and graph.traversalBuilder.attributes) + attributesStack.addLast(defaultAndGraphAttributes) val traversalStack = new java.util.ArrayDeque[Traversal](16) traversalStack.addLast(current) @@ -653,7 +653,6 @@ private final case class SavedIslandData( * INTERNAL API */ @InternalApi private[akka] final class GraphStageIsland( - settings: ActorMaterializerSettings, effectiveAttributes: Attributes, materializer: PhasedFusingActorMaterializer, islandName: String, @@ -668,7 +667,7 @@ private final case class SavedIslandData( private var outConnections: List[Connection] = Nil private var fullIslandName: OptionVal[String] = OptionVal.None - val shell = new GraphInterpreterShell(connections = null, logics = null, settings, effectiveAttributes, materializer) + val shell = new GraphInterpreterShell(connections = null, logics = null, effectiveAttributes, materializer) override def name: String = "Fusing GraphStages phase" diff --git a/akka-stream/src/main/scala/akka/stream/impl/QueueSource.scala b/akka-stream/src/main/scala/akka/stream/impl/QueueSource.scala index 0902270639..da0a00dfe6 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/QueueSource.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/QueueSource.scala @@ -45,7 +45,7 @@ import scala.concurrent.{ Future, Promise } var terminating = false override def preStart(): Unit = { - if (maxBuffer > 0) buffer = Buffer(maxBuffer, materializer) + if (maxBuffer > 0) buffer = Buffer(maxBuffer, inheritedAttributes) } override def postStop(): Unit = { val exception = new StreamDetachedException() 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 ff4c5d3273..617be0ca16 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/Sinks.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/Sinks.scala @@ -6,15 +6,6 @@ package akka.stream.impl import java.util.function.BinaryOperator -import scala.annotation.unchecked.uncheckedVariance -import scala.collection.immutable -import scala.collection.mutable -import scala.concurrent.Future -import scala.concurrent.Promise -import scala.util.Failure -import scala.util.Success -import scala.util.Try -import scala.util.control.NonFatal import akka.NotUsed import akka.actor.ActorRef import akka.actor.Props @@ -22,6 +13,7 @@ import akka.annotation.DoNotInherit import akka.annotation.InternalApi import akka.dispatch.ExecutionContexts import akka.event.Logging +import akka.stream.ActorAttributes.StreamSubscriptionTimeout import akka.stream.Attributes.InputBuffer import akka.stream._ import akka.stream.impl.QueueSink.Output @@ -36,6 +28,16 @@ import akka.util.ccompat._ import org.reactivestreams.Publisher import org.reactivestreams.Subscriber +import scala.annotation.unchecked.uncheckedVariance +import scala.collection.immutable +import scala.collection.mutable +import scala.concurrent.Future +import scala.concurrent.Promise +import scala.util.Failure +import scala.util.Success +import scala.util.Try +import scala.util.control.NonFatal + /** * INTERNAL API */ @@ -93,10 +95,13 @@ import org.reactivestreams.Subscriber val proc = new VirtualPublisher[In] context.materializer match { case am: ActorMaterializer => - if (am.settings.subscriptionTimeoutSettings.mode != StreamSubscriptionTimeoutTerminationMode.noop) - am.scheduleOnce(am.settings.subscriptionTimeoutSettings.timeout, new Runnable { - def run(): Unit = proc.onSubscriptionTimeout(am) + val StreamSubscriptionTimeout(timeout, mode) = + context.effectiveAttributes.mandatoryAttribute[StreamSubscriptionTimeout] + if (mode != StreamSubscriptionTimeoutTerminationMode.noop) { + am.scheduleOnce(timeout, new Runnable { + def run(): Unit = proc.onSubscriptionTimeout(am, mode) }) + } case _ => // not possible to setup timeout } (proc, proc) @@ -116,9 +121,7 @@ import org.reactivestreams.Subscriber override def create(context: MaterializationContext): (Subscriber[In], Publisher[In]) = { val actorMaterializer = ActorMaterializerHelper.downcast(context.materializer) - val impl = actorMaterializer.actorOf( - context, - FanoutProcessorImpl.props(context.effectiveAttributes, actorMaterializer.settings)) + val impl = actorMaterializer.actorOf(context, FanoutProcessorImpl.props(context.effectiveAttributes)) val fanoutProcessor = new ActorProcessor[In, In](impl) // Resolve cyclic dependency with actor. This MUST be the first message no matter what. impl ! ExposedPublisher(fanoutProcessor.asInstanceOf[ActorPublisher[Any]]) @@ -356,7 +359,7 @@ import org.reactivestreams.Subscriber override def preStart(): Unit = { // Allocates one additional element to hold stream // closed/failure indicators - buffer = Buffer(maxBuffer + 1, materializer) + buffer = Buffer(maxBuffer + 1, inheritedAttributes) setKeepGoing(true) pull(in) } 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 078d461cf6..fff1797528 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/StreamLayout.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/StreamLayout.scala @@ -499,11 +499,11 @@ import scala.util.control.NonFatal // this is when the subscription timeout hits, implemented like this to // avoid allocating a separate object for that - def onSubscriptionTimeout(am: ActorMaterializer): Unit = { + def onSubscriptionTimeout(am: ActorMaterializer, mode: StreamSubscriptionTimeoutTerminationMode): Unit = { import StreamSubscriptionTimeoutTerminationMode._ get() match { case null | _: Publisher[_] => - am.settings.subscriptionTimeoutSettings.mode match { + mode match { case CancelTermination => subscribe(new CancellingSubscriber[T]) case WarnTermination => am.logger.warning("Subscription timeout for {}", this) diff --git a/akka-stream/src/main/scala/akka/stream/impl/StreamSubscriptionTimeout.scala b/akka-stream/src/main/scala/akka/stream/impl/StreamSubscriptionTimeout.scala index b6e41ca651..4c567c4795 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/StreamSubscriptionTimeout.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/StreamSubscriptionTimeout.scala @@ -8,6 +8,7 @@ import akka.actor._ import akka.annotation.InternalApi import akka.stream.StreamSubscriptionTimeoutTerminationMode.{ CancelTermination, NoopTermination, WarnTermination } import akka.stream.StreamSubscriptionTimeoutSettings +import com.github.ghik.silencer.silent import org.reactivestreams._ import scala.concurrent.duration.FiniteDuration @@ -56,6 +57,7 @@ import scala.util.control.NoStackTrace * * See `akka.stream.materializer.subscription-timeout` for configuration options. */ +@silent("deprecated") @InternalApi private[akka] trait StreamSubscriptionTimeoutSupport { this: Actor with ActorLogging => @@ -111,6 +113,7 @@ import scala.util.control.NoStackTrace /** * Called by the actor when a subscription has timed out. Expects the actual `Publisher` or `Processor` target. */ + @silent("deprecated") protected def subscriptionTimedOut(target: Publisher[_]): Unit = subscriptionTimeoutSettings.mode match { case NoopTermination => // ignore... case WarnTermination => warn(target, subscriptionTimeoutSettings.timeout) 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 c44348e9ba..d49d2d962f 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 @@ -10,16 +10,24 @@ import java.util.concurrent.atomic.AtomicReference import akka.Done import akka.actor._ -import akka.annotation.{ InternalApi, InternalStableApi } +import akka.annotation.InternalApi +import akka.annotation.InternalStableApi import akka.event.Logging import akka.stream._ import akka.stream.impl.ReactiveStreamsCompliance._ -import akka.stream.impl.fusing.GraphInterpreter.{ Connection, DownstreamBoundaryStageLogic, UpstreamBoundaryStageLogic } -import akka.stream.impl.{ SubFusingActorMaterializerImpl, _ } +import akka.stream.impl.fusing.GraphInterpreter.Connection +import akka.stream.impl.fusing.GraphInterpreter.DownstreamBoundaryStageLogic +import akka.stream.impl.fusing.GraphInterpreter.UpstreamBoundaryStageLogic +import akka.stream.impl.SubFusingActorMaterializerImpl +import akka.stream.impl._ import akka.stream.snapshot._ -import akka.stream.stage.{ GraphStageLogic, InHandler, OutHandler } +import akka.stream.stage.GraphStageLogic +import akka.stream.stage.InHandler +import akka.stream.stage.OutHandler import akka.util.OptionVal -import org.reactivestreams.{ Publisher, Subscriber, Subscription } +import org.reactivestreams.Publisher +import org.reactivestreams.Subscriber +import org.reactivestreams.Subscription import scala.annotation.tailrec import scala.collection.immutable @@ -464,8 +472,7 @@ import scala.util.control.NonFatal @InternalApi private[akka] final class GraphInterpreterShell( var connections: Array[Connection], var logics: Array[GraphStageLogic], - settings: ActorMaterializerSettings, - attributes: Attributes, + val attributes: Attributes, val mat: ExtendedActorMaterializer) { import ActorGraphInterpreter._ @@ -512,9 +519,11 @@ import scala.util.control.NonFatal override def execute(eventLimit: Int): Int = { if (waitingForShutdown) { subscribesPending = 0 + val subscriptionTimeout = attributes.mandatoryAttribute[ActorAttributes.StreamSubscriptionTimeout].timeout 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.")) + new TimeoutException( + "Streaming actor has been already stopped processing (normally), but not all of its " + + s"inputs or outputs have been subscribed in [${subscriptionTimeout}}]. Aborting actor now.")) } 0 } @@ -529,7 +538,7 @@ import scala.util.control.NonFatal if (currentInterpreter == null || (currentInterpreter.context ne self)) self ! asyncInput else enqueueToShortCircuit(asyncInput) - }, settings.fuzzingMode, self) + }, attributes.mandatoryAttribute[ActorAttributes.FuzzingMode].enabled, self) // TODO: really needed? private var subscribesPending = 0 @@ -618,7 +627,8 @@ import scala.util.control.NonFatal if (canShutDown) interpreterCompleted = true else { waitingForShutdown = true - mat.scheduleOnce(settings.subscriptionTimeoutSettings.timeout, new Runnable { + val subscriptionTimeout = attributes.mandatoryAttribute[ActorAttributes.StreamSubscriptionTimeout].timeout + mat.scheduleOnce(subscriptionTimeout, new Runnable { override def run(): Unit = self ! Abort(GraphInterpreterShell.this) }) } @@ -703,7 +713,7 @@ import scala.util.control.NonFatal } //this limits number of messages that can be processed synchronously during one actor receive. - private val eventLimit: Int = _initial.mat.settings.syncProcessingLimit + private val eventLimit: Int = _initial.attributes.mandatoryAttribute[ActorAttributes.SyncProcessingLimit].limit private var currentLimit: Int = eventLimit //this is a var in order to save the allocation when no short-circuiting actually happens private var shortCircuitBuffer: util.ArrayDeque[Any] = null 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 00a7078015..30ec20574a 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 @@ -211,7 +211,6 @@ import akka.stream.snapshot._ val onAsyncInput: (GraphStageLogic, Any, Promise[Done], (Any) => Unit) => Unit, val fuzzingMode: Boolean, val context: ActorRef) { - import GraphInterpreter._ private[this] val ChaseLimit = if (fuzzingMode) 0 else 16 diff --git a/akka-stream/src/main/scala/akka/stream/impl/fusing/Ops.scala b/akka-stream/src/main/scala/akka/stream/impl/fusing/Ops.scala index bac398fd1d..cca2df0129 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/fusing/Ops.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/fusing/Ops.scala @@ -907,7 +907,7 @@ private[stream] object Collect { new GraphStageLogic(shape) with InHandler with OutHandler with StageLogging { override protected def logSource: Class[_] = classOf[Buffer[_]] - private var buffer: BufferImpl[T] = _ + private val buffer: BufferImpl[T] = BufferImpl(size, inheritedAttributes) val enqueueAction: T => Unit = overflowStrategy match { @@ -966,7 +966,6 @@ private[stream] object Collect { } override def preStart(): Unit = { - buffer = BufferImpl(size, materializer) pull(in) } @@ -1253,7 +1252,7 @@ private[stream] object Collect { } }) - override def preStart(): Unit = buffer = BufferImpl(parallelism, materializer) + override def preStart(): Unit = buffer = BufferImpl(parallelism, inheritedAttributes) override def onPull(): Unit = pushNextIfPossible() @@ -1348,7 +1347,7 @@ private[stream] object Collect { private[this] def todo = inFlight + buffer.used - override def preStart(): Unit = buffer = BufferImpl(parallelism, materializer) + override def preStart(): Unit = buffer = BufferImpl(parallelism, inheritedAttributes) def futureCompleted(result: Try[Out]): Unit = { inFlight -= 1 @@ -1727,7 +1726,7 @@ private[stream] object Collect { var buffer: BufferImpl[(Long, T)] = _ // buffer has pairs timestamp with upstream element - override def preStart(): Unit = buffer = BufferImpl(size, materializer) + override def preStart(): Unit = buffer = BufferImpl(size, inheritedAttributes) val onPushWhenBufferFull: () => Unit = strategy match { case EmitEarly => 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 94b79400f9..742266615b 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 @@ -9,24 +9,24 @@ import java.util.concurrent.atomic.AtomicReference import akka.NotUsed import akka.annotation.InternalApi +import akka.stream.ActorAttributes.StreamSubscriptionTimeout import akka.stream.ActorAttributes.SupervisionStrategy import akka.stream._ +import akka.stream.actor.ActorSubscriberMessage import akka.stream.impl.Stages.DefaultAttributes import akka.stream.impl.SubscriptionTimeoutException -import akka.stream.stage._ +import akka.stream.impl.TraversalBuilder +import akka.stream.impl.fusing.GraphStages.SingleSource +import akka.stream.impl.{ Buffer => BufferImpl } import akka.stream.scaladsl._ -import akka.stream.actor.ActorSubscriberMessage +import akka.stream.stage._ import akka.util.OptionVal +import akka.util.ccompat.JavaConverters._ + +import scala.annotation.tailrec import scala.collection.immutable import scala.concurrent.duration.FiniteDuration import scala.util.control.NonFatal -import scala.annotation.tailrec - -import akka.stream.impl.{ Buffer => BufferImpl } -import akka.util.ccompat.JavaConverters._ - -import akka.stream.impl.TraversalBuilder -import akka.stream.impl.fusing.GraphStages.SingleSource /** * INTERNAL API @@ -48,7 +48,7 @@ import akka.stream.impl.fusing.GraphStages.SingleSource // SubSinkInlet[T] or SingleSource var queue: BufferImpl[AnyRef] = _ - override def preStart(): Unit = queue = BufferImpl(breadth, materializer) + override def preStart(): Unit = queue = BufferImpl(breadth, enclosingAttributes) def pushOut(): Unit = { queue.dequeue() match { @@ -144,7 +144,7 @@ import akka.stream.impl.fusing.GraphStages.SingleSource override def initialAttributes = DefaultAttributes.prefixAndTail - private final class PrefixAndTailLogic(_shape: Shape) + private final class PrefixAndTailLogic(_shape: Shape, inheritedAttributes: Attributes) extends TimerGraphStageLogic(_shape) with OutHandler with InHandler { @@ -159,10 +159,10 @@ import akka.stream.impl.fusing.GraphStages.SingleSource override protected def onTimer(timerKey: Any): Unit = { val materializer = ActorMaterializerHelper.downcast(interpreter.materializer) - val timeoutSettings = materializer.settings.subscriptionTimeoutSettings - val timeout = timeoutSettings.timeout + val StreamSubscriptionTimeout(timeout, mode) = + inheritedAttributes.mandatoryAttribute[ActorAttributes.StreamSubscriptionTimeout] - timeoutSettings.mode match { + mode match { case StreamSubscriptionTimeoutTerminationMode.CancelTermination => tailSource.timeout(timeout) if (tailSource.isClosed) completeStage() @@ -190,8 +190,7 @@ import akka.stream.impl.fusing.GraphStages.SingleSource } private def openSubstream(): Source[T, NotUsed] = { - val timeout = - ActorMaterializerHelper.downcast(interpreter.materializer).settings.subscriptionTimeoutSettings.timeout + val timeout = inheritedAttributes.mandatoryAttribute[ActorAttributes.StreamSubscriptionTimeout].timeout tailSource = new SubSourceOutlet[T]("TailSource") tailSource.setHandler(subHandler) setKeepGoing(true) @@ -244,7 +243,8 @@ import akka.stream.impl.fusing.GraphStages.SingleSource setHandlers(in, out, this) } - override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new PrefixAndTailLogic(shape) + override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = + new PrefixAndTailLogic(shape, inheritedAttributes) override def toString: String = s"PrefixAndTail($n)" } @@ -271,7 +271,8 @@ import akka.stream.impl.fusing.GraphStages.SingleSource private val closedSubstreams = if (allowClosedSubstreamRecreation) Collections.unmodifiableSet(Collections.emptySet[Any]) else new java.util.HashSet[Any]() - private var timeout: FiniteDuration = _ + private val timeout: FiniteDuration = + inheritedAttributes.mandatoryAttribute[ActorAttributes.StreamSubscriptionTimeout].timeout private var substreamWaitingToBePushed: Option[SubstreamSource] = None private var nextElementKey: K = null.asInstanceOf[K] private var nextElementValue: T = null.asInstanceOf[T] @@ -312,10 +313,6 @@ import akka.stream.impl.fusing.GraphStages.SingleSource private def needToPull: Boolean = !(hasBeenPulled(in) || isClosed(in) || hasNextElement || substreamWaitingToBePushed.nonEmpty) - override def preStart(): Unit = - timeout = - ActorMaterializerHelper.downcast(interpreter.materializer).settings.subscriptionTimeoutSettings.timeout - override def onPull(): Unit = { substreamWaitingToBePushed match { case Some(substreamSource) => @@ -491,15 +488,12 @@ import akka.stream.impl.fusing.GraphStages.SingleSource private val SubscriptionTimer = "SubstreamSubscriptionTimer" - private var timeout: FiniteDuration = _ + private val timeout: FiniteDuration = + inheritedAttributes.mandatoryAttribute[ActorAttributes.StreamSubscriptionTimeout].timeout private var substreamSource: SubSourceOutlet[T] = null private var substreamWaitingToBePushed = false private var substreamCancelled = false - override def preStart(): Unit = { - timeout = ActorMaterializerHelper.downcast(interpreter.materializer).settings.subscriptionTimeoutSettings.timeout - } - setHandler( out, new OutHandler { diff --git a/akka-stream/src/main/scala/akka/stream/impl/io/TcpStages.scala b/akka-stream/src/main/scala/akka/stream/impl/io/TcpStages.scala index d09cbfd624..acd3e9276d 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/io/TcpStages.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/io/TcpStages.scala @@ -19,9 +19,11 @@ import akka.stream._ import akka.stream.impl.ReactiveStreamsCompliance import akka.stream.impl.fusing.GraphStages.detacher import akka.stream.scaladsl.Tcp.{ OutgoingConnection, ServerBinding } +import akka.stream.scaladsl.TcpAttributes import akka.stream.scaladsl.{ BidiFlow, Flow, TcpIdleTimeoutException, Tcp => StreamTcp } import akka.stream.stage._ import akka.util.ByteString +import com.github.ghik.silencer.silent import scala.collection.immutable import scala.concurrent.duration.{ Duration, FiniteDuration } @@ -37,8 +39,7 @@ import scala.concurrent.{ Future, Promise } val options: immutable.Iterable[SocketOption], val halfClose: Boolean, val idleTimeout: Duration, - val bindShutdownTimeout: FiniteDuration, - val ioSettings: IOSettings) + val bindShutdownTimeout: FiniteDuration) extends GraphStageWithMaterializedValue[SourceShape[StreamTcp.IncomingConnection], Future[StreamTcp.ServerBinding]] { import ConnectionSourceStage._ @@ -46,9 +47,12 @@ import scala.concurrent.{ Future, Promise } override def initialAttributes = Attributes.name("ConnectionSource") val shape: SourceShape[StreamTcp.IncomingConnection] = SourceShape(out) - // TODO: Timeout on bind override def createLogicAndMaterializedValue( - inheritedAttributes: Attributes): (GraphStageLogic, Future[ServerBinding]) = { + inheritedAttributes: Attributes): (GraphStageLogic, Future[ServerBinding]) = + throw new UnsupportedOperationException("Not used") + + // TODO: Timeout on bind + override def createLogicAndMaterializedValue(inheritedAttributes: Attributes, eagerMaterialzer: Materializer) = { val bindingPromise = Promise[ServerBinding] val logic = new TimerGraphStageLogic(shape) { @@ -130,7 +134,6 @@ import scala.concurrent.{ Future, Promise } connection, connected.remoteAddress, halfClose, - ioSettings, () => connectionFlowsAwaitingInitialization.decrementAndGet())) .via(detacher[ByteString]) // must read ahead for proper completions @@ -189,18 +192,15 @@ private[stream] object ConnectionSourceStage { trait TcpRole { def halfClose: Boolean - def ioSettings: IOSettings } case class Outbound( manager: ActorRef, connectCmd: Connect, localAddressPromise: Promise[InetSocketAddress], - halfClose: Boolean, - ioSettings: IOSettings) + halfClose: Boolean) extends TcpRole - case class Inbound(connection: ActorRef, halfClose: Boolean, ioSettings: IOSettings, registerCallback: () => Unit) - extends TcpRole + case class Inbound(connection: ActorRef, halfClose: Boolean, registerCallback: () => Unit) extends TcpRole /* * This is a *non-detached* design, i.e. this does not prefetch itself any of the inputs. It relies on downstream @@ -212,7 +212,9 @@ private[stream] object ConnectionSourceStage { class TcpStreamLogic( val shape: FlowShape[ByteString, ByteString], val role: TcpRole, - remoteAddress: InetSocketAddress) + inheritedAttributes: Attributes, + remoteAddress: InetSocketAddress, + eagerMaterializer: Materializer) extends GraphStageLogic(shape) { implicit def self: ActorRef = stageActor.ref @@ -220,7 +222,13 @@ private[stream] object ConnectionSourceStage { private def bytesOut = shape.out private var connection: ActorRef = _ - private val writeBufferSize = role.ioSettings.tcpWriteBufferSize + @silent("deprecated") + private val writeBufferSize = inheritedAttributes + .get[TcpAttributes.TcpWriteBufferSize]( + TcpAttributes.TcpWriteBufferSize( + ActorMaterializerHelper.downcast(eagerMaterializer).settings.ioSettings.tcpWriteBufferSize)) + .size + private var writeBuffer = ByteString.empty private var writeInProgress = false private var connectionClosePending = false @@ -233,14 +241,14 @@ private[stream] object ConnectionSourceStage { override def preStart(): Unit = { setKeepGoing(true) role match { - case Inbound(conn, _, _, registerCallback) => + case Inbound(conn, _, registerCallback) => setHandler(bytesOut, readHandler) connection = conn getStageActor(connected).watch(connection) connection ! Register(self, keepOpenOnPeerClosed = true, useResumeWriting = false) registerCallback() pull(bytesIn) - case ob @ Outbound(manager, cmd, _, _, _) => + case ob @ Outbound(manager, cmd, _, _) => getStageActor(connecting(ob)).watch(manager) manager ! cmd } @@ -376,7 +384,7 @@ private[stream] object ConnectionSourceStage { } private def reportExceptionToPromise(ex: Throwable): Unit = role match { - case Outbound(_, _, localAddressPromise, _, _) => + case Outbound(_, _, localAddressPromise, _) => // Fail if has not been completed with an address earlier localAddressPromise.tryFailure(ex) case _ => // do nothing... @@ -395,7 +403,6 @@ private[stream] object ConnectionSourceStage { connection: ActorRef, remoteAddress: InetSocketAddress, halfClose: Boolean, - ioSettings: IOSettings, registerCallback: () => Unit) extends GraphStage[FlowShape[ByteString, ByteString]] { import TcpConnectionStage._ @@ -407,11 +414,21 @@ private[stream] object ConnectionSourceStage { override def initialAttributes = Attributes.name("IncomingConnection") val shape: FlowShape[ByteString, ByteString] = FlowShape(bytesIn, bytesOut) - override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = { + override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = + throw new UnsupportedOperationException("Not used") + + override def createLogicAndMaterializedValue(inheritedAttributes: Attributes, eagerMaterializer: Materializer) = { if (hasBeenCreated.get) throw new IllegalStateException("Cannot materialize an incoming connection Flow twice.") hasBeenCreated.set(true) - new TcpStreamLogic(shape, Inbound(connection, halfClose, ioSettings, registerCallback), remoteAddress) + ( + new TcpStreamLogic( + shape, + Inbound(connection, halfClose, registerCallback), + inheritedAttributes, + remoteAddress, + eagerMaterializer), + NotUsed) } override def toString = s"TCP-from($remoteAddress)" @@ -426,8 +443,7 @@ private[stream] object ConnectionSourceStage { localAddress: Option[InetSocketAddress] = None, options: immutable.Iterable[SocketOption] = Nil, halfClose: Boolean = true, - connectTimeout: Duration = Duration.Inf, - ioSettings: IOSettings) + connectTimeout: Duration = Duration.Inf) extends GraphStageWithMaterializedValue[FlowShape[ByteString, ByteString], Future[StreamTcp.OutgoingConnection]] { import TcpConnectionStage._ @@ -437,7 +453,12 @@ private[stream] object ConnectionSourceStage { val shape: FlowShape[ByteString, ByteString] = FlowShape(bytesIn, bytesOut) override def createLogicAndMaterializedValue( - inheritedAttributes: Attributes): (GraphStageLogic, Future[StreamTcp.OutgoingConnection]) = { + inheritedAttributes: Attributes): (GraphStageLogic, Future[OutgoingConnection]) = + throw new UnsupportedOperationException("Not used") + + override def createLogicAndMaterializedValue( + inheritedAttributes: Attributes, + eagerMaterializer: Materializer): (GraphStageLogic, Future[StreamTcp.OutgoingConnection]) = { // FIXME: A method like this would make soo much sense on Duration (i.e. toOption) val connTimeout = connectTimeout match { case x: FiniteDuration => Some(x) @@ -451,9 +472,10 @@ private[stream] object ConnectionSourceStage { manager, Connect(remoteAddress, localAddress, options, connTimeout, pullMode = true), localAddressPromise, - halfClose, - ioSettings), - remoteAddress) + halfClose), + inheritedAttributes, + remoteAddress, + eagerMaterializer) ( logic, diff --git a/akka-stream/src/main/scala/akka/stream/impl/streamref/SinkRefImpl.scala b/akka-stream/src/main/scala/akka/stream/impl/streamref/SinkRefImpl.scala index 6822cd4583..009c9baf89 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/streamref/SinkRefImpl.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/streamref/SinkRefImpl.scala @@ -13,6 +13,7 @@ import akka.stream._ import akka.stream.scaladsl.Sink import akka.stream.stage._ import akka.util.{ OptionVal, PrettyDuration } +import com.github.ghik.silencer.silent import scala.util.{ Failure, Success, Try } @@ -62,11 +63,13 @@ private[stream] final class SinkRefStageImpl[In] private[akka] (val initialPartn private[this] val streamRefsMaster = StreamRefsMaster(ActorMaterializerHelper.downcast(eagerMaterializer).system) // settings --- - import StreamRefAttributes._ - private[this] val settings = ActorMaterializerHelper.downcast(eagerMaterializer).settings.streamRefSettings - - private[this] val subscriptionTimeout = inheritedAttributes.get[StreamRefAttributes.SubscriptionTimeout]( - SubscriptionTimeout(settings.subscriptionTimeout)) + @silent("deprecated") // can't remove this settings access without breaking compat + private[this] val subscriptionTimeout = { + import StreamRefAttributes._ + val settings = ActorMaterializerHelper.downcast(eagerMaterializer).settings.streamRefSettings + inheritedAttributes.get[StreamRefAttributes.SubscriptionTimeout]( + SubscriptionTimeout(settings.subscriptionTimeout)) + } // end of settings --- override protected val stageActorName: String = streamRefsMaster.nextSinkRefStageName() diff --git a/akka-stream/src/main/scala/akka/stream/impl/streamref/SourceRefImpl.scala b/akka-stream/src/main/scala/akka/stream/impl/streamref/SourceRefImpl.scala index 7187fc1c14..619d5a86a1 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/streamref/SourceRefImpl.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/streamref/SourceRefImpl.scala @@ -14,6 +14,7 @@ import akka.stream.impl.FixedSizeBuffer import akka.stream.scaladsl.Source import akka.stream.stage._ import akka.util.{ OptionVal, PrettyDuration } +import com.github.ghik.silencer.silent /** INTERNAL API: Implementation class, not intended to be touched directly by end-users */ @InternalApi @@ -61,10 +62,29 @@ private[stream] final class SourceRefStageImpl[Out](val initialPartnerRef: Optio // settings --- import StreamRefAttributes._ + @silent("deprecated") // can't remove this settings access without breaking compat private[this] val settings = ActorMaterializerHelper.downcast(eagerMaterializer).settings.streamRefSettings + @silent("deprecated") // can't remove this settings access without breaking compat private[this] val subscriptionTimeout = inheritedAttributes.get[StreamRefAttributes.SubscriptionTimeout]( SubscriptionTimeout(settings.subscriptionTimeout)) + + @silent("deprecated") // can't remove this settings access without breaking compat + private[this] val bufferCapacity = inheritedAttributes + .get[StreamRefAttributes.BufferCapacity](StreamRefAttributes.BufferCapacity(settings.bufferCapacity)) + .capacity + + @silent("deprecated") // can't remove this settings access without breaking compat + private[this] val demandRedeliveryInterval = inheritedAttributes + .get[StreamRefAttributes.DemandRedeliveryInterval](DemandRedeliveryInterval(settings.demandRedeliveryInterval)) + .timeout + + @silent("deprecated") // can't remove this settings access without breaking compat + private[this] val finalTerminationSignalDeadline = + inheritedAttributes + .get[StreamRefAttributes.FinalTerminationSignalDeadline]( + FinalTerminationSignalDeadline(settings.finalTerminationSignalDeadline)) + .timeout // end of settings --- override protected val stageActorName: String = streamRefsMaster.nextSourceRefStageName() @@ -84,8 +104,7 @@ private[stream] final class SourceRefStageImpl[Out](val initialPartnerRef: Optio private var localCumulativeDemand: Long = 0L private var localRemainingRequested: Int = 0 - private var receiveBuffer - : FixedSizeBuffer.FixedSizeBuffer[Out] = _ // initialized in preStart since depends on settings + private val receiveBuffer = FixedSizeBuffer[Out](bufferCapacity) private var requestStrategy: RequestStrategy = _ // initialized in preStart since depends on receiveBuffer's size // end of demand management --- @@ -96,7 +115,6 @@ private[stream] final class SourceRefStageImpl[Out](val initialPartnerRef: Optio private def getPartnerRef = partnerRef.get override def preStart(): Unit = { - receiveBuffer = FixedSizeBuffer[Out](settings.bufferCapacity) requestStrategy = WatermarkRequestStrategy(highWatermark = receiveBuffer.capacity) log.debug("[{}] Allocated receiver: {}", stageActorName, self.ref) @@ -135,7 +153,7 @@ private[stream] final class SourceRefStageImpl[Out](val initialPartnerRef: Optio } def scheduleDemandRedelivery(): Unit = - scheduleOnce(DemandRedeliveryTimerKey, settings.demandRedeliveryInterval) + scheduleOnce(DemandRedeliveryTimerKey, demandRedeliveryInterval) override protected def onTimer(timerKey: Any): Unit = timerKey match { case SubscriptionTimeoutTimerKey => @@ -195,7 +213,7 @@ private[stream] final class SourceRefStageImpl[Out](val initialPartnerRef: Optio // we need to start a delayed shutdown in case we were network partitioned and the final signal complete/fail // will never reach us; so after the given timeout we need to forcefully terminate this side of the stream ref // the other (sending) side terminates by default once it gets a Terminated signal so no special handling is needed there. - scheduleOnce(TerminationDeadlineTimerKey, settings.finalTerminationSignalDeadline) + scheduleOnce(TerminationDeadlineTimerKey, finalTerminationSignalDeadline) case _ => // this should not have happened! It should be impossible that we watched some other actor diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/Tcp.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/Tcp.scala index 632f01b7f1..b4620b63c4 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl/Tcp.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl/Tcp.scala @@ -7,27 +7,35 @@ package akka.stream.scaladsl import java.net.InetSocketAddress import java.util.concurrent.TimeoutException -import javax.net.ssl.SSLContext -import javax.net.ssl.SSLEngine -import javax.net.ssl.SSLSession import akka.actor._ import akka.annotation.InternalApi import akka.io.Inet.SocketOption -import akka.io.{ IO, Tcp => IoTcp } +import akka.io.IO +import akka.io.{ Tcp => IoTcp } +import akka.stream.Attributes.Attribute import akka.stream.TLSProtocol.NegotiateNewSession import akka.stream._ import akka.stream.impl.fusing.GraphStages.detacher -import akka.stream.impl.io.{ ConnectionSourceStage, OutgoingConnectionStage, TcpIdleTimeout } -import akka.util.{ unused, ByteString } -import akka.{ Done, NotUsed } +import akka.stream.impl.io.ConnectionSourceStage +import akka.stream.impl.io.OutgoingConnectionStage +import akka.stream.impl.io.TcpIdleTimeout +import akka.util.ByteString +import akka.util.unused +import akka.util.JavaDurationConverters._ +import akka.Done +import akka.NotUsed import com.github.ghik.silencer.silent +import javax.net.ssl.SSLContext +import javax.net.ssl.SSLEngine +import javax.net.ssl.SSLSession import scala.collection.immutable import scala.concurrent.Future -import scala.concurrent.duration.{ Duration, FiniteDuration } +import scala.concurrent.duration._ +import scala.concurrent.duration.Duration +import scala.concurrent.duration.FiniteDuration import scala.util.Try import scala.util.control.NoStackTrace -import scala.collection.immutable object Tcp extends ExtensionId[Tcp] with ExtensionIdProvider { @@ -89,10 +97,9 @@ object Tcp extends ExtensionId[Tcp] with ExtensionIdProvider { final class Tcp(system: ExtendedActorSystem) extends akka.actor.Extension { import Tcp._ - private val settings = ActorMaterializerSettings(system) - // TODO maybe this should be a new setting, like `akka.stream.tcp.bind.timeout` / `shutdown-timeout` instead? - val bindShutdownTimeout = settings.subscriptionTimeoutSettings.timeout + val bindShutdownTimeout = + system.settings.config.getDuration("akka.stream.materializer.subscription-timeout.timeout").asScala /** * Creates a [[Tcp.ServerBinding]] instance which represents a prospective TCP server binding on the given `endpoint`. @@ -131,8 +138,7 @@ final class Tcp(system: ExtendedActorSystem) extends akka.actor.Extension { options.toList, halfClose, idleTimeout, - bindShutdownTimeout, - settings.ioSettings)) + bindShutdownTimeout)) /** * Creates a [[Tcp.ServerBinding]] instance which represents a prospective TCP server binding on the given `endpoint` @@ -210,8 +216,7 @@ final class Tcp(system: ExtendedActorSystem) extends akka.actor.Extension { localAddress, options.toList, halfClose, - connectTimeout, - settings.ioSettings)) + connectTimeout)) .via(detacher[ByteString]) // must read ahead for proper completions idleTimeout match { @@ -375,3 +380,11 @@ final class Tcp(system: ExtendedActorSystem) extends akka.actor.Extension { final class TcpIdleTimeoutException(msg: String, @unused timeout: Duration) extends TimeoutException(msg: String) with NoStackTrace // only used from a single stage + +object TcpAttributes { + final case class TcpWriteBufferSize(size: Int) extends Attribute { + require(size > 0) + } + def tcpWriteBufferSize(size: Int): Attributes = + Attributes(TcpWriteBufferSize(size)) +}