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