diff --git a/akka-actor-testkit-typed/src/main/scala/akka/actor/testkit/typed/internal/StubbedActorContext.scala b/akka-actor-testkit-typed/src/main/scala/akka/actor/testkit/typed/internal/StubbedActorContext.scala
index 537b41bb72..0e9d946100 100644
--- a/akka-actor-testkit-typed/src/main/scala/akka/actor/testkit/typed/internal/StubbedActorContext.scala
+++ b/akka-actor-testkit-typed/src/main/scala/akka/actor/testkit/typed/internal/StubbedActorContext.scala
@@ -182,6 +182,10 @@ private[akka] final class FunctionRef[-T](override val path: ActorPath, send: (T
private val loggingAdapter = new StubbedLogger
private var unhandled: List[T] = Nil
+ private[akka] def classicActorContext =
+ throw new UnsupportedOperationException(
+ "No classic ActorContext available with the stubbed actor context, to spawn materializers and run streams you will need a real actor")
+
override def children: Iterable[ActorRef[Nothing]] = _children.values.map(_.context.self)
def childrenNames: Iterable[String] = _children.keys
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/adapter/ActorContextAdapter.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/adapter/ActorContextAdapter.scala
index 3adff583e7..b12bc8c85a 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/internal/adapter/ActorContextAdapter.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/internal/adapter/ActorContextAdapter.scala
@@ -63,6 +63,7 @@ private[akka] object ActorContextAdapter {
final override val self = ActorRefAdapter(classicContext.self)
final override val system = ActorSystemAdapter(classicContext.system)
+ private[akka] def classicActorContext = classicContext
override def children: Iterable[ActorRef[Nothing]] = classicContext.children.map(ActorRefAdapter(_))
override def child(name: String): Option[ActorRef[Nothing]] = classicContext.child(name).map(ActorRefAdapter(_))
override def spawnAnonymous[U](behavior: Behavior[U], props: Props = Props.empty): ActorRef[U] =
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/ActorContext.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/ActorContext.scala
index fe4be0a595..b56453a096 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/ActorContext.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/javadsl/ActorContext.scala
@@ -12,6 +12,8 @@ import akka.actor.typed._
import java.util.Optional
import java.util.concurrent.CompletionStage
+import akka.actor.ClassicActorContextProvider
+
import scala.concurrent.ExecutionContextExecutor
/**
@@ -35,7 +37,7 @@ import scala.concurrent.ExecutionContextExecutor
* Not for user extension.
*/
@DoNotInherit
-trait ActorContext[T] extends TypedActorContext[T] {
+trait ActorContext[T] extends TypedActorContext[T] with ClassicActorContextProvider {
// this must be a pure interface, i.e. only abstract methods
/**
diff --git a/akka-actor-typed/src/main/scala/akka/actor/typed/scaladsl/ActorContext.scala b/akka-actor-typed/src/main/scala/akka/actor/typed/scaladsl/ActorContext.scala
index 14ef4fe3b8..52ab00f9b9 100644
--- a/akka-actor-typed/src/main/scala/akka/actor/typed/scaladsl/ActorContext.scala
+++ b/akka-actor-typed/src/main/scala/akka/actor/typed/scaladsl/ActorContext.scala
@@ -4,6 +4,7 @@
package akka.actor.typed.scaladsl
+import akka.actor.ClassicActorContextProvider
import akka.actor.typed._
import akka.annotation.DoNotInherit
import akka.util.Timeout
@@ -35,7 +36,7 @@ import akka.annotation.InternalApi
* Not for user extension.
*/
@DoNotInherit
-trait ActorContext[T] extends TypedActorContext[T] {
+trait ActorContext[T] extends TypedActorContext[T] with ClassicActorContextProvider {
/**
* Get the `javadsl` of this `ActorContext`.
diff --git a/akka-actor/src/main/mima-filters/2.5.x.backwards.excludes b/akka-actor/src/main/mima-filters/2.5.x.backwards.excludes
index 2f431524a2..d61bce31b5 100644
--- a/akka-actor/src/main/mima-filters/2.5.x.backwards.excludes
+++ b/akka-actor/src/main/mima-filters/2.5.x.backwards.excludes
@@ -85,4 +85,6 @@ ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.actor.DynamicAccess.c
# system wide materializer #25559
ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("akka.actor.ExtendedActorSystem.classicSystem")
-ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("akka.actor.ActorSystem.classicSystem")
\ No newline at end of file
+ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("akka.actor.ActorSystem.classicSystem")
+ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("akka.actor.AbstractActor#ActorContext.classicActorContext")
+ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("akka.actor.ActorContext.classicActorContext")
\ No newline at end of file
diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala
index 198694af8c..bb150d60ea 100644
--- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala
+++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala
@@ -44,7 +44,7 @@ import com.github.ghik.silencer.silent
*
* Where no name is given explicitly, one will be automatically generated.
*/
-trait ActorContext extends ActorRefFactory {
+trait ActorContext extends ActorRefFactory with ClassicActorContextProvider {
/**
* The ActorRef representing this actor
@@ -428,6 +428,8 @@ private[akka] class ActorCell(
protected final def lookupRoot = self
final def provider = system.provider
+ override final def classicActorContext: ActorContext = this
+
protected def uid: Int = self.path.uid
private[this] var _actor: Actor = _
def actor: Actor = _actor
diff --git a/akka-actor/src/main/scala/akka/actor/ClassicActorSystemProvider.scala b/akka-actor/src/main/scala/akka/actor/ClassicActorSystemProvider.scala
index 0c4dfaea13..0a2f978c3a 100644
--- a/akka-actor/src/main/scala/akka/actor/ClassicActorSystemProvider.scala
+++ b/akka-actor/src/main/scala/akka/actor/ClassicActorSystemProvider.scala
@@ -19,3 +19,16 @@ trait ClassicActorSystemProvider {
@InternalApi
private[akka] def classicSystem: ActorSystem
}
+
+/**
+ * Glue API introduced to allow minimal user effort integration between classic and typed for example for streams.
+ *
+ * Not for user extension.
+ */
+@DoNotInherit
+trait ClassicActorContextProvider {
+
+ /** INTERNAL API */
+ @InternalApi
+ private[akka] def classicActorContext: ActorContext
+}
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 d738424258..13ebaf9615 100644
--- a/akka-bench-jmh/src/main/scala/akka/stream/AskBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/stream/AskBenchmark.scala
@@ -86,7 +86,7 @@ class AskBenchmark {
private def awaitLatch(latch: CountDownLatch): Unit = {
if (!latch.await(30, TimeUnit.SECONDS)) {
- StreamTestKit.printDebugDump(ActorMaterializerHelper.downcast(SystemMaterializer(system).materializer).supervisor)
+ StreamTestKit.printDebugDump(SystemMaterializer(system).materializer.supervisor)
throw new RuntimeException("Latch didn't complete in time")
}
}
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 465df53e2f..520ba94452 100644
--- a/akka-bench-jmh/src/main/scala/akka/stream/FlatMapConcatBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/stream/FlatMapConcatBenchmark.scala
@@ -100,7 +100,7 @@ class FlatMapConcatBenchmark {
private def awaitLatch(latch: CountDownLatch): Unit = {
if (!latch.await(30, TimeUnit.SECONDS)) {
implicit val ec = system.dispatcher
- StreamTestKit.printDebugDump(ActorMaterializerHelper.downcast(SystemMaterializer(system).materializer).supervisor)
+ StreamTestKit.printDebugDump(SystemMaterializer(system).materializer.supervisor)
throw new RuntimeException("Latch didn't complete in time")
}
}
diff --git a/akka-bench-jmh/src/main/scala/akka/stream/FramingBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/stream/FramingBenchmark.scala
index d5825f1a89..6227fb6981 100644
--- a/akka-bench-jmh/src/main/scala/akka/stream/FramingBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/stream/FramingBenchmark.scala
@@ -4,14 +4,18 @@
package akka.stream
-import java.util.concurrent.{ Semaphore, TimeUnit }
+import java.util.concurrent.Semaphore
+import java.util.concurrent.TimeUnit
import akka.NotUsed
import akka.actor.ActorSystem
import akka.remote.artery.BenchTestSourceSameElement
-import akka.stream.scaladsl.{ Framing, Sink, Source }
+import akka.stream.scaladsl.Framing
+import akka.stream.scaladsl.Sink
+import akka.stream.scaladsl.Source
import akka.util.ByteString
-import com.typesafe.config.{ Config, ConfigFactory }
+import com.typesafe.config.Config
+import com.typesafe.config.ConfigFactory
import org.openjdk.jmh.annotations._
import scala.concurrent.Await
@@ -49,8 +53,6 @@ class FramingBenchmark {
implicit val system: ActorSystem = ActorSystem("test", config)
- var materializer: ActorMaterializer = _
-
// Safe to be benchmark scoped because the flows we construct in this bench are stateless
var flow: Source[ByteString, NotUsed] = _
@@ -62,7 +64,7 @@ class FramingBenchmark {
@Setup
def setup(): Unit = {
- materializer = ActorMaterializer()
+ SystemMaterializer(system).materializer
val frame = List.range(0, messageSize, 1).map(_ => Random.nextPrintableChar()).mkString + "\n"
val messageChunk = ByteString(List.range(0, framePerSeq, 1).map(_ => frame).mkString)
@@ -82,7 +84,7 @@ class FramingBenchmark {
def framing(): Unit = {
val lock = new Semaphore(1)
lock.acquire()
- flow.runWith(Sink.onComplete(_ => lock.release()))(materializer)
+ flow.runWith(Sink.onComplete(_ => lock.release()))
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 10eb3d6294..e11ddb607c 100644
--- a/akka-bench-jmh/src/main/scala/akka/stream/FusedGraphsBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/stream/FusedGraphsBenchmark.scala
@@ -101,7 +101,6 @@ class FusedGraphsBenchmark {
ConfigFactory.parseString(s"""
akka.stream.materializer.sync-processing-limit = ${Int.MaxValue}
"""))
- var materializer: ActorMaterializer = _
var testElements: Array[MutableElement] = _
var singleIdentity: RunnableGraph[CountDownLatch] = _
@@ -225,70 +224,70 @@ class FusedGraphsBenchmark {
@OperationsPerInvocation(100 * 1000)
def single_identity(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = {
FusedGraphsBenchmark.blackhole = blackhole
- singleIdentity.run()(materializer).await()
+ singleIdentity.run().await()
}
@Benchmark
@OperationsPerInvocation(100 * 1000)
def chain_of_identities(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = {
FusedGraphsBenchmark.blackhole = blackhole
- chainOfIdentities.run()(materializer).await()
+ chainOfIdentities.run().await()
}
@Benchmark
@OperationsPerInvocation(100 * 1000)
def single_map(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = {
FusedGraphsBenchmark.blackhole = blackhole
- singleMap.run()(materializer).await()
+ singleMap.run().await()
}
@Benchmark
@OperationsPerInvocation(100 * 1000)
def chain_of_maps(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = {
FusedGraphsBenchmark.blackhole = blackhole
- chainOfMaps.run()(materializer).await()
+ chainOfMaps.run().await()
}
@Benchmark
@OperationsPerInvocation(100 * 1000)
def repeat_take_map_and_fold(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = {
FusedGraphsBenchmark.blackhole = blackhole
- repeatTakeMapAndFold.run()(materializer).await()
+ repeatTakeMapAndFold.run().await()
}
@Benchmark
@OperationsPerInvocation(100 * 1000)
def single_buffer(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = {
FusedGraphsBenchmark.blackhole = blackhole
- singleBuffer.run()(materializer).await()
+ singleBuffer.run().await()
}
@Benchmark
@OperationsPerInvocation(100 * 1000)
def chain_of_buffers(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = {
FusedGraphsBenchmark.blackhole = blackhole
- chainOfBuffers.run()(materializer).await()
+ chainOfBuffers.run().await()
}
@Benchmark
@OperationsPerInvocation(100 * 1000)
def broadcast_zip(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = {
FusedGraphsBenchmark.blackhole = blackhole
- broadcastZip.run()(materializer).await()
+ broadcastZip.run().await()
}
@Benchmark
@OperationsPerInvocation(100 * 1000)
def balance_merge(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = {
FusedGraphsBenchmark.blackhole = blackhole
- balanceMerge.run()(materializer).await()
+ balanceMerge.run().await()
}
@Benchmark
@OperationsPerInvocation(100 * 1000)
def broadcast_zip_balance_merge(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = {
FusedGraphsBenchmark.blackhole = blackhole
- broadcastZipBalanceMerge.run()(materializer).await()
+ broadcastZipBalanceMerge.run().await()
}
@TearDown
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 f925cda01c..5b2e9b564a 100644
--- a/akka-bench-jmh/src/main/scala/akka/stream/MapAsyncBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/stream/MapAsyncBenchmark.scala
@@ -4,18 +4,21 @@
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.scaladsl._
import akka.stream.testkit.scaladsl.StreamTestKit
import com.typesafe.config.ConfigFactory
import org.openjdk.jmh.annotations._
-import scala.concurrent.{ Await, Future }
import scala.concurrent.duration._
+import scala.concurrent.Await
+import scala.concurrent.Future
object MapAsyncBenchmark {
final val OperationsPerInvocation = 100000
@@ -85,7 +88,7 @@ class MapAsyncBenchmark {
private def awaitLatch(latch: CountDownLatch): Unit = {
if (!latch.await(30, TimeUnit.SECONDS)) {
- StreamTestKit.printDebugDump(ActorMaterializerHelper.downcast(SystemMaterializer(system).materializer).supervisor)
+ StreamTestKit.printDebugDump(SystemMaterializer(system).materializer.supervisor)
throw new RuntimeException("Latch didn't complete in time")
}
}
diff --git a/akka-bench-jmh/src/main/scala/akka/stream/MaterializationBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/stream/MaterializationBenchmark.scala
index 609f017afb..7172e26fbc 100644
--- a/akka-bench-jmh/src/main/scala/akka/stream/MaterializationBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/stream/MaterializationBenchmark.scala
@@ -5,14 +5,16 @@
package akka.stream
import java.util.concurrent.TimeUnit
+
+import akka.Done
import akka.NotUsed
import akka.actor.ActorSystem
import akka.stream.scaladsl._
import org.openjdk.jmh.annotations._
+
import scala.concurrent.Await
-import scala.concurrent.duration._
import scala.concurrent.Future
-import akka.Done
+import scala.concurrent.duration._
object MaterializationBenchmark {
@@ -95,7 +97,6 @@ class MaterializationBenchmark {
import MaterializationBenchmark._
implicit val system = ActorSystem("MaterializationBenchmark")
- implicit val materializer = ActorMaterializer()
var flowWithMap: RunnableGraph[NotUsed] = _
var graphWithJunctionsGradual: RunnableGraph[NotUsed] = _
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 fc1a8ff385..67d323278e 100644
--- a/akka-bench-jmh/src/main/scala/akka/stream/PartitionHubBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/stream/PartitionHubBenchmark.scala
@@ -4,12 +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.remote.artery.{ BenchTestSource, FixedSizePartitionHub, LatchSink }
-import akka.stream.scaladsl.{ PartitionHub, _ }
+import akka.remote.artery.BenchTestSource
+import akka.remote.artery.FixedSizePartitionHub
+import akka.remote.artery.LatchSink
+import akka.stream.scaladsl.PartitionHub
+import akka.stream.scaladsl._
import akka.stream.testkit.scaladsl.StreamTestKit
import com.typesafe.config.ConfigFactory
import org.openjdk.jmh.annotations._
@@ -100,7 +104,7 @@ class PartitionHubBenchmark {
private def dumpMaterializer(): Unit = {
implicit val ec = system.dispatcher
- StreamTestKit.printDebugDump(ActorMaterializerHelper.downcast(SystemMaterializer(system).materializer).supervisor)
+ StreamTestKit.printDebugDump(SystemMaterializer(system).materializer.supervisor)
}
}
diff --git a/akka-bench-jmh/src/main/scala/akka/stream/SourceRefBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/stream/SourceRefBenchmark.scala
index f4c0099f39..45daca32f9 100644
--- a/akka-bench-jmh/src/main/scala/akka/stream/SourceRefBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/stream/SourceRefBenchmark.scala
@@ -4,7 +4,8 @@
package akka.stream
-import java.util.concurrent.{ Semaphore, TimeUnit }
+import java.util.concurrent.Semaphore
+import java.util.concurrent.TimeUnit
import akka.actor.ActorSystem
import akka.remote.artery.BenchTestSource
@@ -34,8 +35,6 @@ class SourceRefBenchmark {
implicit val system = ActorSystem("test", config)
- implicit val materializer: ActorMaterializer = ActorMaterializer()
-
final val successMarker = Success(1)
final val successFailure = Success(new Exception)
diff --git a/akka-bench-jmh/src/main/scala/akka/stream/impl/OutputStreamSourceStageBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/stream/impl/OutputStreamSourceStageBenchmark.scala
index ebeccc3a5b..b411cb9911 100644
--- a/akka-bench-jmh/src/main/scala/akka/stream/impl/OutputStreamSourceStageBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/stream/impl/OutputStreamSourceStageBenchmark.scala
@@ -6,17 +6,16 @@ package akka.stream.impl
import java.util.concurrent.TimeUnit
-import scala.concurrent.Await
-import scala.concurrent.duration._
-
import akka.actor.ActorSystem
-import akka.stream.ActorMaterializer
import akka.stream.scaladsl.Keep
import akka.stream.scaladsl.Sink
import akka.stream.scaladsl.StreamConverters
import org.openjdk.jmh.annotations.TearDown
import org.openjdk.jmh.annotations._
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
object OutputStreamSourceStageBenchmark {
final val WritesPerBench = 10000
}
@@ -26,7 +25,6 @@ object OutputStreamSourceStageBenchmark {
class OutputStreamSourceStageBenchmark {
import OutputStreamSourceStageBenchmark.WritesPerBench
implicit val system = ActorSystem("OutputStreamSourceStageBenchmark")
- implicit val materializer = ActorMaterializer()
private val bytes: Array[Byte] = Array.emptyByteArray
diff --git a/akka-bench-jmh/src/main/scala/akka/stream/io/FileSourcesBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/stream/io/FileSourcesBenchmark.scala
index 3783cc3641..9daefc26d8 100644
--- a/akka-bench-jmh/src/main/scala/akka/stream/io/FileSourcesBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/stream/io/FileSourcesBenchmark.scala
@@ -4,17 +4,23 @@
package akka.stream.io
-import java.nio.file.{ Files, Path }
+import java.nio.file.Files
+import java.nio.file.Path
import java.util.concurrent.TimeUnit
-import akka.{ Done, NotUsed }
+
import akka.actor.ActorSystem
-import akka.stream.{ ActorMaterializer, Attributes }
+import akka.stream.Attributes
+import akka.stream.IOResult
import akka.stream.scaladsl._
import akka.util.ByteString
+import akka.Done
+import akka.NotUsed
import org.openjdk.jmh.annotations._
+
import scala.concurrent.duration._
-import scala.concurrent.{ Await, Future, Promise }
-import akka.stream.IOResult
+import scala.concurrent.Await
+import scala.concurrent.Future
+import scala.concurrent.Promise
/**
* Benchmark (bufSize) Mode Cnt Score Error Units
@@ -26,7 +32,6 @@ import akka.stream.IOResult
class FileSourcesBenchmark {
implicit val system = ActorSystem("file-sources-benchmark")
- implicit val materializer = ActorMaterializer()
val file: Path = {
val line = ByteString("x" * 2048 + "\n")
diff --git a/akka-bench-jmh/src/main/scala/akka/stream/io/FileSourcesScaleBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/stream/io/FileSourcesScaleBenchmark.scala
index 0e2f08a634..46875f4535 100644
--- a/akka-bench-jmh/src/main/scala/akka/stream/io/FileSourcesScaleBenchmark.scala
+++ b/akka-bench-jmh/src/main/scala/akka/stream/io/FileSourcesScaleBenchmark.scala
@@ -4,17 +4,22 @@
package akka.stream.io
-import java.nio.file.{ Files, Path }
+import java.nio.file.Files
+import java.nio.file.Path
import java.util.concurrent.TimeUnit
import akka.actor.ActorSystem
+import akka.stream.IOResult
import akka.stream.scaladsl._
-import akka.stream.{ ActorMaterializer, IOResult }
import akka.util.ByteString
-import org.openjdk.jmh.annotations.{ BenchmarkMode, Scope, State, _ }
+import org.openjdk.jmh.annotations.BenchmarkMode
+import org.openjdk.jmh.annotations.Scope
+import org.openjdk.jmh.annotations.State
+import org.openjdk.jmh.annotations._
+import scala.concurrent.Await
+import scala.concurrent.Future
import scala.concurrent.duration._
-import scala.concurrent.{ Await, Future }
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.AverageTime))
@@ -30,7 +35,7 @@ class FileSourcesScaleBenchmark {
* FileSourcesScaleBenchmark.mapAsync 2048 avgt 10 0.899 ± 0.103 s/op
*/
implicit val system = ActorSystem("file-sources-benchmark")
- implicit val materializer = ActorMaterializer()
+
val FILES_NUMBER = 40
val files: Seq[Path] = {
val line = ByteString("x" * 2048 + "\n")
diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/StreamRefSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/StreamRefSpec.scala
index a872e95d7a..f8c7410c22 100644
--- a/akka-cluster/src/multi-jvm/scala/akka/cluster/StreamRefSpec.scala
+++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/StreamRefSpec.scala
@@ -4,11 +4,6 @@
package akka.cluster
-import scala.concurrent.Future
-import scala.concurrent.duration._
-import scala.util.Failure
-import scala.util.Success
-
import akka.Done
import akka.actor.Actor
import akka.actor.ActorIdentity
@@ -19,7 +14,7 @@ import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.remote.transport.ThrottlerTransportAdapter.Direction
import akka.serialization.jackson.CborSerializable
-import akka.stream.ActorMaterializer
+import akka.stream.Materializer
import akka.stream.RemoteStreamRefActorTerminatedException
import akka.stream.SinkRef
import akka.stream.SourceRef
@@ -32,6 +27,11 @@ import akka.stream.testkit.scaladsl.TestSink
import akka.testkit._
import com.typesafe.config.ConfigFactory
+import scala.concurrent.Future
+import scala.concurrent.duration._
+import scala.util.Failure
+import scala.util.Success
+
object StreamRefSpec extends MultiNodeConfig {
val first = role("first")
val second = role("second")
@@ -54,7 +54,7 @@ object StreamRefSpec extends MultiNodeConfig {
class DataSource(streamLifecycleProbe: ActorRef) extends Actor {
import context.dispatcher
- implicit val mat = ActorMaterializer()(context)
+ implicit val mat = Materializer(context)
def receive = {
case RequestLogs(streamId) =>
@@ -97,7 +97,7 @@ object StreamRefSpec extends MultiNodeConfig {
class DataReceiver(streamLifecycleProbe: ActorRef) extends Actor {
import context.dispatcher
- implicit val mat = ActorMaterializer()(context)
+ implicit val mat = Materializer(context)
def receive = {
case PrepareUpload(nodeId) =>
@@ -136,8 +136,6 @@ class StreamRefMultiJvmNode3 extends StreamRefSpec
abstract class StreamRefSpec extends MultiNodeSpec(StreamRefSpec) with MultiNodeClusterSpec with ImplicitSender {
import StreamRefSpec._
- private implicit val mat: ActorMaterializer = ActorMaterializer()
-
"A cluster with Stream Refs" must {
"join" taggedAs LongRunningTest in {
diff --git a/akka-cluster/src/test/scala/akka/cluster/ClusterSpec.scala b/akka-cluster/src/test/scala/akka/cluster/ClusterSpec.scala
index 24f30311df..d09e73bbb4 100644
--- a/akka-cluster/src/test/scala/akka/cluster/ClusterSpec.scala
+++ b/akka-cluster/src/test/scala/akka/cluster/ClusterSpec.scala
@@ -6,9 +6,6 @@ package akka.cluster
import java.lang.management.ManagementFactory
-import scala.concurrent.Await
-import scala.concurrent.duration._
-
import akka.actor.ActorSystem
import akka.actor.Address
import akka.actor.CoordinatedShutdown
@@ -17,7 +14,7 @@ import akka.actor.Props
import akka.cluster.ClusterEvent.MemberEvent
import akka.cluster.ClusterEvent._
import akka.cluster.InternalClusterAction._
-import akka.stream.ActorMaterializer
+import akka.stream.Materializer
import akka.stream.scaladsl.Sink
import akka.stream.scaladsl.Source
import akka.stream.scaladsl.StreamRefs
@@ -27,6 +24,9 @@ import akka.testkit.TestProbe
import com.typesafe.config.ConfigFactory
import javax.management.ObjectName
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
object ClusterSpec {
val config = """
akka.cluster {
@@ -223,7 +223,7 @@ class ClusterSpec extends AkkaSpec(ClusterSpec.config) with ImplicitSender {
probe.expectMsgType[CurrentClusterState]
Cluster(sys2).join(Cluster(sys2).selfAddress)
probe.expectMsgType[MemberUp]
- val mat = ActorMaterializer()(sys2)
+ val mat = Materializer(sys2)
val sink = StreamRefs.sinkRef[String]().to(Sink.ignore).run()(mat)
Source.tick(1.milli, 10.millis, "tick").to(sink).run()(mat)
diff --git a/akka-docs/src/main/paradox/additional/faq.md b/akka-docs/src/main/paradox/additional/faq.md
index 5e3646f3b5..4e9e04d033 100644
--- a/akka-docs/src/main/paradox/additional/faq.md
+++ b/akka-docs/src/main/paradox/additional/faq.md
@@ -24,7 +24,7 @@ Akka is also:
## Resources with Explicit Lifecycle
-Actors, ActorSystems, ActorMaterializers (for streams), all these types of objects bind
+Actors, ActorSystems, Materializers (for streams), all these types of objects bind
resources that must be released explicitly. The reason is that Actors are meant to have
a life of their own, existing independently of whether messages are currently en route
to them. Therefore you should always make sure that for every creation of such an object
@@ -39,7 +39,7 @@ Due to an ActorSystem’s explicit lifecycle the JVM will not exit until it is s
Therefore it is necessary to shutdown all ActorSystems within a running application or
Scala REPL session in order to allow these processes to terminate.
-Shutting down an ActorSystem will properly terminate all Actors and ActorMaterializers
+Shutting down an ActorSystem will properly terminate all Actors and Materializers
that were created within it.
## Actors in General
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 9a02f72fae..61d4473445 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
@@ -479,19 +479,27 @@ made before finalizing the APIs. Compared to Akka 2.5.x the source incompatible
* `ActorSource.actorRef` relying on `PartialFunction` has been replaced in the Java API with a variant more suitable to be called by Java.
* `toUntyped` has been renamed to `toClassic`.
-## Materializer changes
-### System global Materializer provided
+## Akka Stream changes
+
+### Materializer changes
A default materializer is now provided out of the box. For the Java API just pass `system` when running streams,
for Scala an implicit materializer is provided if there is an implicit `ActorSystem` available. This avoids leaking
materializers and simplifies most stream use cases somewhat.
-Having a default materializer available means that most, if not all, usages of Java `ActorMaterializer.create()`
-and Scala `implicit val materializer = ActorMaterializer()` should be removed.
+The `ActorMaterializer` factories has been deprecated and replaced with a few corresponding factories in `akka.stream.Materializer`.
+New factories with per-materializer settings has not been provided but should instead be done globally through config or per stream,
+see below for more details.
+
+Having a default materializer available means that most, if not all, usages of Java `ActorMaterializer.create()`
+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)
+When using streams from typed the same factories and methods for creating materializers and running streams as from classic can now be used with typed. The
+`akka.stream.typed.scaladsl.ActorMaterializer` and `akka.stream.typed.javadsl.ActorMaterializerFactory` that previously existed in the `akka-stream-typed` module has been removed.
+
### Materializer settings deprecated
The `ActorMaterializerSettings` class has been deprecated.
diff --git a/akka-docs/src/main/paradox/stream/operators/Sink/fromMaterializer.md b/akka-docs/src/main/paradox/stream/operators/Sink/fromMaterializer.md
new file mode 100644
index 0000000000..59d12c969a
--- /dev/null
+++ b/akka-docs/src/main/paradox/stream/operators/Sink/fromMaterializer.md
@@ -0,0 +1,17 @@
+# Sink.fromMaterializer
+
+Defer the creation of a `Sink` until materialization and access `Materializer` and `Attributes`
+
+@ref[Sink operators](../index.md#sink-operators)
+
+@@@ div { .group-scala }
+
+## Signature
+
+@@signature [Sink.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala) { #fromMaterializer }
+@@@
+
+## Description
+
+Typically used when access to materializer is needed to run a different stream during the construction of a sink.
+Can also be used to access the underlying `ActorSystem` from `Materializer`.
\ No newline at end of file
diff --git a/akka-docs/src/main/paradox/stream/operators/Sink/setup.md b/akka-docs/src/main/paradox/stream/operators/Sink/setup.md
index 855d08b022..5415b7f2d0 100644
--- a/akka-docs/src/main/paradox/stream/operators/Sink/setup.md
+++ b/akka-docs/src/main/paradox/stream/operators/Sink/setup.md
@@ -4,6 +4,12 @@ Defer the creation of a `Sink` until materialization and access `ActorMaterializ
@ref[Sink operators](../index.md#sink-operators)
+@@@ warning
+
+The `setup` operator has been deprecated, use @ref:[fromMaterializer](./fromMaterializer.md) instead.
+
+@@@
+
@@@ div { .group-scala }
## Signature
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/fromMaterializer.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/fromMaterializer.md
new file mode 100644
index 0000000000..abae21cb13
--- /dev/null
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/fromMaterializer.md
@@ -0,0 +1,19 @@
+# Source/Flow.fromMaterializer
+
+Defer the creation of a `Source/Flow` until materialization and access `Materializer` and `Attributes`
+
+@ref[Simple operators](../index.md#simple-operators)
+
+@@@ div { .group-scala }
+
+## Signature
+
+@@signature [Source.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala) { #fromMaterializer }
+@@signature [Flow.scala](/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala) { #fromMaterializer }
+
+@@@
+
+## Description
+
+Typically used when access to materializer is needed to run a different stream during the construction of a source/flow.
+Can also be used to access the underlying `ActorSystem` from `Materializer`.
\ No newline at end of file
diff --git a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/setup.md b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/setup.md
index b4421d07b2..82bce88eed 100644
--- a/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/setup.md
+++ b/akka-docs/src/main/paradox/stream/operators/Source-or-Flow/setup.md
@@ -1,9 +1,15 @@
# Source/Flow.setup
-Defer the creation of a `Source/Flow` until materialization and access `ActorMaterializer` and `Attributes`
+Defer the creation of a `Source/Flow` until materialization and access `Materializer` and `Attributes`
@ref[Simple operators](../index.md#simple-operators)
+@@@ warning
+
+The `setup` operator has been deprecated, use @ref:[fromMaterializer](./fromMaterializer.md) instead.
+
+@@@
+
@@@ div { .group-scala }
## Signature
diff --git a/akka-docs/src/main/paradox/stream/operators/index.md b/akka-docs/src/main/paradox/stream/operators/index.md
index b249346b16..0fb2d292e6 100644
--- a/akka-docs/src/main/paradox/stream/operators/index.md
+++ b/akka-docs/src/main/paradox/stream/operators/index.md
@@ -54,6 +54,7 @@ These built-in sinks are available from @scala[`akka.stream.scaladsl.Sink`] @jav
|Sink|@ref[foreach](Sink/foreach.md)|Invoke a given procedure for each element received.|
|Sink|@ref[foreachAsync](Sink/foreachAsync.md)|Invoke a given procedure asynchronously for each element received.|
|Sink|@ref[foreachParallel](Sink/foreachParallel.md)|Like `foreach` but allows up to `parallellism` procedure calls to happen in parallel.|
+|Sink|@ref[fromMaterializer](Sink/fromMaterializer.md)|Defer the creation of a `Sink` until materialization and access `Materializer` and `Attributes`|
|Sink|@ref[fromSubscriber](Sink/fromSubscriber.md)|Integration with Reactive Streams, wraps a `org.reactivestreams.Subscriber` as a sink.|
|Sink|@ref[head](Sink/head.md)|Materializes into a @scala[`Future`] @java[`CompletionStage`] which completes with the first value arriving, after this the stream is canceled.|
|Sink|@ref[headOption](Sink/headOption.md)|Materializes into a @scala[`Future[Option[T]]`] @java[`CompletionStage>`] which completes with the first value arriving wrapped in @scala[`Some`] @java[`Optional`], or @scala[a `None`] @java[an empty Optional] if the stream completes without any elements emitted.|
@@ -136,6 +137,7 @@ depending on being backpressured by downstream or not.
|Source/Flow|@ref[filterNot](Source-or-Flow/filterNot.md)|Filter the incoming elements using a predicate.|
|Source/Flow|@ref[fold](Source-or-Flow/fold.md)|Start with current value `zero` and then apply the current and next value to the given function. When upstream completes, the current value is emitted downstream.|
|Source/Flow|@ref[foldAsync](Source-or-Flow/foldAsync.md)|Just like `fold` but receives a function that results in a @scala[`Future`] @java[`CompletionStage`] to the next value.|
+|Source/Flow|@ref[fromMaterializer](Source-or-Flow/fromMaterializer.md)|Defer the creation of a `Source/Flow` until materialization and access `Materializer` and `Attributes`|
|Source/Flow|@ref[grouped](Source-or-Flow/grouped.md)|Accumulate incoming events until the specified number of elements have been accumulated and then pass the collection of elements downstream.|
|Source/Flow|@ref[intersperse](Source-or-Flow/intersperse.md)|Intersperse stream with provided element similar to `List.mkString`.|
|Flow|@ref[lazyInitAsync](Flow/lazyInitAsync.md)|Creates a real `Flow` upon receiving the first element by calling relevant `flowFactory` given as an argument.|
@@ -151,7 +153,7 @@ depending on being backpressured by downstream or not.
|Source/Flow|@ref[reduce](Source-or-Flow/reduce.md)|Start with first element and then apply the current and next value to the given function, when upstream complete the current value is emitted downstream.|
|Source/Flow|@ref[scan](Source-or-Flow/scan.md)|Emit its current value, which starts at `zero`, and then apply the current and next value to the given function, emitting the next current value.|
|Source/Flow|@ref[scanAsync](Source-or-Flow/scanAsync.md)|Just like `scan` but receives a function that results in a @scala[`Future`] @java[`CompletionStage`] to the next value.|
-|Source/Flow|@ref[setup](Source-or-Flow/setup.md)|Defer the creation of a `Source/Flow` until materialization and access `ActorMaterializer` and `Attributes`|
+|Source/Flow|@ref[setup](Source-or-Flow/setup.md)|Defer the creation of a `Source/Flow` until materialization and access `Materializer` and `Attributes`|
|Source/Flow|@ref[sliding](Source-or-Flow/sliding.md)|Provide a sliding window over the incoming stream and pass the windows as groups of elements downstream.|
|Source/Flow|@ref[statefulMapConcat](Source-or-Flow/statefulMapConcat.md)|Transform each element into zero or more elements that are individually passed downstream.|
|Source/Flow|@ref[take](Source-or-Flow/take.md)|Pass `n` incoming elements downstream and then complete|
@@ -315,6 +317,7 @@ For more background see the @ref[Error Handling in Streams](../stream-error.md)
* [fromPublisher](Source/fromPublisher.md)
* [fromIterator](Source/fromIterator.md)
* [cycle](Source/cycle.md)
+* [fromMaterializer](Source-or-Flow/fromMaterializer.md)
* [setup](Source-or-Flow/setup.md)
* [fromFuture](Source/fromFuture.md)
* [fromCompletionStage](Source/fromCompletionStage.md)
@@ -421,6 +424,7 @@ For more background see the @ref[Error Handling in Streams](../stream-error.md)
* [fromSinkAndSourceCoupled](Flow/fromSinkAndSourceCoupled.md)
* [lazyInitAsync](Flow/lazyInitAsync.md)
* [preMaterialize](Sink/preMaterialize.md)
+* [fromMaterializer](Sink/fromMaterializer.md)
* [setup](Sink/setup.md)
* [fromSubscriber](Sink/fromSubscriber.md)
* [cancelled](Sink/cancelled.md)
diff --git a/akka-docs/src/main/paradox/stream/stream-flows-and-basics.md b/akka-docs/src/main/paradox/stream/stream-flows-and-basics.md
index 122eea6218..d05cc0f0b6 100644
--- a/akka-docs/src/main/paradox/stream/stream-flows-and-basics.md
+++ b/akka-docs/src/main/paradox/stream/stream-flows-and-basics.md
@@ -269,7 +269,7 @@ well-known sinks, such as @scala[`runForeach(el => ...)`]@java[`runForeach(el ->
Materialization is performed synchronously on the materializing thread by an `ActorSystem` global `Materializer`.
The actual stream processing is handled by actors started up during the streams materialization,
which will be running on the thread pools they have been configured to run on - which defaults to the dispatcher set in
-`MaterializationSettings` while constructing the `ActorMaterializer`.
+the `ActorSystem` config or provided as attributes on the stream that is getting materialized.
@@@ note
@@ -386,9 +386,7 @@ The use cases that may require a custom instance of `Materializer` are:
* When wanting to change some specific default settings for a set of streams (FIXME we should phase this out)
* When all streams materialized in an actor should be tied to the Actor lifecycle and stop if the Actor stops or crashes
-Currently the `Materializer` has one concrete implementation, the `ActorMaterializer`.
-
-An important aspect of working with streams and actors is understanding an `ActorMaterializer`'s life-cycle.
+An important aspect of working with streams and actors is understanding a `Materializer`'s life-cycle.
The materializer is bound to the lifecycle of the `ActorRefFactory` it is created from, which in practice will
be either an `ActorSystem` or `ActorContext` (when the materializer is created within an `Actor`).
@@ -400,7 +398,7 @@ usual way to terminate streams, which is by cancelling/completing them. The stre
like this to prevent leaks, and in normal operations you should not rely on the mechanism and rather use `KillSwitch` or
normal completion signals to manage the lifecycles of your streams.
-If we look at the following example, where we create the `ActorMaterializer` within an `Actor`:
+If we look at the following example, where we create the `Materializer` within an `Actor`:
Scala
: @@snip [FlowDocSpec.scala](/akka-docs/src/test/scala/docs/stream/FlowDocSpec.scala) { #materializer-from-actor-context }
@@ -411,7 +409,7 @@ Java
In the above example we used the `ActorContext` to create the materializer. This binds its lifecycle to the surrounding `Actor`. In other words, while the stream we started there would under normal circumstances run forever, if we stop the Actor it would terminate the stream as well. We have *bound the stream's lifecycle to the surrounding actor's lifecycle*.
This is a very useful technique if the stream is closely related to the actor, e.g. when the actor represents a user or other entity, that we continuously query using the created stream -- and it would not make sense to keep the stream alive when the actor has terminated already. The streams termination will be signalled by an "Abrupt termination exception" signaled by the stream.
-You may also cause an `ActorMaterializer` to shut down by explicitly calling `shutdown()` on it, resulting in abruptly terminating all of the streams it has been running then.
+You may also cause a `Materializer` to shut down by explicitly calling `shutdown()` on it, resulting in abruptly terminating all of the streams it has been running then.
Sometimes, however, you may want to explicitly create a stream that will out-last the actor's life.
For example, you are using an Akka stream to push some large stream of data to an external service.
@@ -429,7 +427,7 @@ for example because of the materializer's settings etc.
@@@ warning
Do not create new actor materializers inside actors by passing the `context.system` to it.
-This will cause a new `ActorMaterializer` to be created and potentially leaked (unless you shut it down explicitly) for each such actor.
+This will cause a new `Materializer` to be created and potentially leaked (unless you shut it down explicitly) for each such actor.
It is instead recommended to either pass-in the Materializer or create one using the actor's `context`.
@@@
diff --git a/akka-docs/src/main/paradox/stream/stream-quickstart.md b/akka-docs/src/main/paradox/stream/stream-quickstart.md
index 17b75c66c3..2988fe9f25 100644
--- a/akka-docs/src/main/paradox/stream/stream-quickstart.md
+++ b/akka-docs/src/main/paradox/stream/stream-quickstart.md
@@ -416,7 +416,7 @@ has also a type parameter of @scala[`Future[Int]`]@java[`CompletionStage>`]. Next we call `run()` which uses the @scala[implicit] `ActorMaterializer`
+be `run()`, as indicated by its type: @scala[`RunnableGraph[Future[Int]]`]@java[`RunnableGraph>`]. Next we call `run()` which uses the @scala[implicit] `Materializer`
to materialize and run the Flow. The value returned by calling `run()` on a @scala[`RunnableGraph[T]`]@java[`RunnableGraph`] is of type `T`.
In our case this type is @scala[`Future[Int]`]@java[`CompletionStage`] which, when completed, will contain the total length of our `tweets` stream.
In case of the stream failing, this future would complete with a Failure.
diff --git a/akka-docs/src/test/java/jdocs/persistence/PersistenceQueryDocTest.java b/akka-docs/src/test/java/jdocs/persistence/PersistenceQueryDocTest.java
index 70cdc7a6bc..757f0bafb2 100644
--- a/akka-docs/src/test/java/jdocs/persistence/PersistenceQueryDocTest.java
+++ b/akka-docs/src/test/java/jdocs/persistence/PersistenceQueryDocTest.java
@@ -18,7 +18,6 @@ import com.typesafe.config.Config;
import akka.actor.*;
import akka.persistence.query.*;
-import akka.stream.ActorMaterializer;
import akka.stream.javadsl.Sink;
import akka.stream.javadsl.Source;
diff --git a/akka-docs/src/test/java/jdocs/persistence/query/MyEventsByTagSource.java b/akka-docs/src/test/java/jdocs/persistence/query/MyEventsByTagSource.java
index 143a8e2af6..3fabe6f6ba 100644
--- a/akka-docs/src/test/java/jdocs/persistence/query/MyEventsByTagSource.java
+++ b/akka-docs/src/test/java/jdocs/persistence/query/MyEventsByTagSource.java
@@ -64,7 +64,7 @@ public class MyEventsByTagSource extends GraphStage>
@Override
public GraphStageLogic createLogic(Attributes inheritedAttributes) {
return new TimerGraphStageLogic(shape()) {
- private ActorSystem system = ((ActorMaterializer) materializer()).system();
+ private ActorSystem system = materializer().system();
private long currentOffset = initialOffset;
private List buf = new LinkedList<>();
private final Serialization serialization = SerializationExtension.get(system);
diff --git a/akka-docs/src/test/java/jdocs/stream/FlowDocTest.java b/akka-docs/src/test/java/jdocs/stream/FlowDocTest.java
index 49a63de169..7cb6a964ca 100644
--- a/akka-docs/src/test/java/jdocs/stream/FlowDocTest.java
+++ b/akka-docs/src/test/java/jdocs/stream/FlowDocTest.java
@@ -295,7 +295,7 @@ public class FlowDocTest extends AbstractJavaTest {
// #materializer-from-actor-context
final class RunWithMyself extends AbstractActor {
- ActorMaterializer mat = ActorMaterializer.create(context());
+ Materializer mat = Materializer.create(context());
@Override
public void preStart() throws Exception {
diff --git a/akka-docs/src/test/java/jdocs/stream/HubDocTest.java b/akka-docs/src/test/java/jdocs/stream/HubDocTest.java
index 65638578cd..8c420210fd 100644
--- a/akka-docs/src/test/java/jdocs/stream/HubDocTest.java
+++ b/akka-docs/src/test/java/jdocs/stream/HubDocTest.java
@@ -62,7 +62,7 @@ public class HubDocTest extends AbstractJavaTest {
@Test
public void dynamicBroadcast() {
// Used to be able to clean up the running stream
- ActorMaterializer materializer = ActorMaterializer.create(system);
+ Materializer materializer = Materializer.create(system);
// #broadcast-hub
// A simple producer that publishes a new "message" every second
@@ -134,7 +134,7 @@ public class HubDocTest extends AbstractJavaTest {
@Test
public void dynamicPartition() {
// Used to be able to clean up the running stream
- ActorMaterializer materializer = ActorMaterializer.create(system);
+ Materializer materializer = Materializer.create(system);
// #partition-hub
// A simple producer that publishes a new "message-n" every second
@@ -182,7 +182,7 @@ public class HubDocTest extends AbstractJavaTest {
@Test
public void dynamicStatefulPartition() {
// Used to be able to clean up the running stream
- ActorMaterializer materializer = ActorMaterializer.create(system);
+ Materializer materializer = Materializer.create(system);
// #partition-hub-stateful
// A simple producer that publishes a new "message-n" every second
@@ -215,7 +215,7 @@ public class HubDocTest extends AbstractJavaTest {
@Test
public void dynamicFastestPartition() {
// Used to be able to clean up the running stream
- ActorMaterializer materializer = ActorMaterializer.create(system);
+ Materializer materializer = Materializer.create(system);
// #partition-hub-fastest
Source producer = Source.range(0, 100);
diff --git a/akka-docs/src/test/java/jdocs/stream/SubstreamDocTest.java b/akka-docs/src/test/java/jdocs/stream/SubstreamDocTest.java
index b2ebb8d692..672aace002 100644
--- a/akka-docs/src/test/java/jdocs/stream/SubstreamDocTest.java
+++ b/akka-docs/src/test/java/jdocs/stream/SubstreamDocTest.java
@@ -5,8 +5,6 @@
package jdocs.stream;
import akka.actor.ActorSystem;
-import akka.stream.ActorMaterializer;
-import akka.stream.Materializer;
import akka.stream.javadsl.Sink;
import akka.stream.javadsl.Source;
import akka.testkit.javadsl.TestKit;
@@ -16,8 +14,6 @@ import org.junit.BeforeClass;
import org.junit.Test;
import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
public class SubstreamDocTest extends AbstractJavaTest {
diff --git a/akka-docs/src/test/java/jdocs/stream/javadsl/cookbook/RecipeAdhocSourceTest.java b/akka-docs/src/test/java/jdocs/stream/javadsl/cookbook/RecipeAdhocSourceTest.java
index 4fe6142939..ac4784f398 100644
--- a/akka-docs/src/test/java/jdocs/stream/javadsl/cookbook/RecipeAdhocSourceTest.java
+++ b/akka-docs/src/test/java/jdocs/stream/javadsl/cookbook/RecipeAdhocSourceTest.java
@@ -8,8 +8,6 @@ import akka.Done;
import akka.actor.ActorSystem;
import akka.dispatch.Futures;
import akka.japi.pf.PFBuilder;
-import akka.stream.ActorMaterializer;
-import akka.stream.Materializer;
import akka.stream.javadsl.Keep;
import akka.stream.javadsl.Source;
import akka.stream.testkit.TestSubscriber;
diff --git a/akka-docs/src/test/scala/docs/persistence/PersistenceDocSpec.scala b/akka-docs/src/test/scala/docs/persistence/PersistenceDocSpec.scala
index 175832a8c2..ba58091f5c 100644
--- a/akka-docs/src/test/scala/docs/persistence/PersistenceDocSpec.scala
+++ b/akka-docs/src/test/scala/docs/persistence/PersistenceDocSpec.scala
@@ -5,10 +5,8 @@
package docs.persistence
import akka.actor._
-import akka.pattern.{ Backoff, BackoffOpts, BackoffSupervisor }
+import akka.pattern.{ BackoffOpts, BackoffSupervisor }
import akka.persistence._
-import akka.stream.ActorMaterializer
-import akka.stream.scaladsl.{ Flow, Sink, Source }
import scala.concurrent.duration._
import scala.language.postfixOps
diff --git a/akka-docs/src/test/scala/docs/persistence/query/MyEventsByTagSource.scala b/akka-docs/src/test/scala/docs/persistence/query/MyEventsByTagSource.scala
index 1491707242..6eaa947652 100644
--- a/akka-docs/src/test/scala/docs/persistence/query/MyEventsByTagSource.scala
+++ b/akka-docs/src/test/scala/docs/persistence/query/MyEventsByTagSource.scala
@@ -24,11 +24,7 @@ class MyEventsByTagSource(tag: String, offset: Long, refreshInterval: FiniteDura
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic =
new TimerGraphStageLogic(shape) with OutHandler {
- lazy val system = materializer match {
- case a: ActorMaterializer => a.system
- case _ =>
- throw new IllegalStateException("EventsByTagStage requires ActorMaterializer")
- }
+ lazy val system = materializer.system
private val Limit = 1000
private val connection: java.sql.Connection = ???
private var currentOffset = offset
diff --git a/akka-docs/src/test/scala/docs/stream/FlowDocSpec.scala b/akka-docs/src/test/scala/docs/stream/FlowDocSpec.scala
index fc7e9c7923..10a1ab4294 100644
--- a/akka-docs/src/test/scala/docs/stream/FlowDocSpec.scala
+++ b/akka-docs/src/test/scala/docs/stream/FlowDocSpec.scala
@@ -6,7 +6,6 @@ package docs.stream
import akka.NotUsed
import akka.actor.{ Actor, ActorSystem, Cancellable }
-import akka.stream.ActorMaterializer
import akka.stream.Materializer
import akka.stream.{ ClosedShape, FlowShape, OverflowStrategy }
import akka.stream.scaladsl._
@@ -245,7 +244,7 @@ object FlowDocSpec {
//#materializer-from-actor-context
final class RunWithMyself extends Actor {
- implicit val mat = ActorMaterializer()
+ implicit val mat = Materializer(context)
Source.maybe.runWith(Sink.onComplete {
case Success(done) => println(s"Completed: $done")
diff --git a/akka-docs/src/test/scala/docs/stream/FlowStreamRefsDocSpec.scala b/akka-docs/src/test/scala/docs/stream/FlowStreamRefsDocSpec.scala
index 2894a8148b..f0f1ca9f89 100644
--- a/akka-docs/src/test/scala/docs/stream/FlowStreamRefsDocSpec.scala
+++ b/akka-docs/src/test/scala/docs/stream/FlowStreamRefsDocSpec.scala
@@ -6,7 +6,6 @@ package docs.stream
import akka.NotUsed
import akka.actor.{ Actor, Props }
-import akka.stream.ActorMaterializer
import akka.stream.scaladsl._
import akka.testkit.AkkaSpec
import docs.CompileOnlySpec
@@ -22,7 +21,6 @@ class FlowStreamRefsDocSpec extends AkkaSpec with CompileOnlySpec {
case class LogsOffer(streamId: Int, sourceRef: SourceRef[String])
class DataSource extends Actor {
- implicit val mat = ActorMaterializer()(context)
def receive = {
case RequestLogs(streamId) =>
@@ -43,7 +41,6 @@ class FlowStreamRefsDocSpec extends AkkaSpec with CompileOnlySpec {
}
//#offer-source
- implicit val mat = ActorMaterializer()
//#offer-source-use
val sourceActor = system.actorOf(Props[DataSource], "dataSource")
@@ -60,7 +57,6 @@ class FlowStreamRefsDocSpec extends AkkaSpec with CompileOnlySpec {
"offer a sink ref" in compileOnlySpec {
//#offer-sink
- import akka.pattern.pipe
import akka.stream.SinkRef
case class PrepareUpload(id: String)
@@ -68,8 +64,6 @@ class FlowStreamRefsDocSpec extends AkkaSpec with CompileOnlySpec {
class DataReceiver extends Actor {
- implicit val mat = ActorMaterializer()(context)
-
def receive = {
case PrepareUpload(nodeId) =>
// obtain the source you want to offer:
diff --git a/akka-docs/src/test/scala/docs/stream/GraphCyclesSpec.scala b/akka-docs/src/test/scala/docs/stream/GraphCyclesSpec.scala
index 9ada92a825..4110e8f127 100644
--- a/akka-docs/src/test/scala/docs/stream/GraphCyclesSpec.scala
+++ b/akka-docs/src/test/scala/docs/stream/GraphCyclesSpec.scala
@@ -4,7 +4,7 @@
package docs.stream
-import akka.stream.{ ActorMaterializer, ClosedShape, OverflowStrategy }
+import akka.stream.{ ClosedShape, OverflowStrategy }
import akka.stream.scaladsl._
import akka.testkit.AkkaSpec
diff --git a/akka-docs/src/test/scala/docs/stream/TwitterStreamQuickstartDocSpec.scala b/akka-docs/src/test/scala/docs/stream/TwitterStreamQuickstartDocSpec.scala
index e06e2c0621..c70df3907a 100644
--- a/akka-docs/src/test/scala/docs/stream/TwitterStreamQuickstartDocSpec.scala
+++ b/akka-docs/src/test/scala/docs/stream/TwitterStreamQuickstartDocSpec.scala
@@ -22,7 +22,6 @@ object TwitterStreamQuickstartDocSpec {
//#fiddle_code
import akka.NotUsed
import akka.actor.ActorSystem
- import akka.stream.ActorMaterializer
import akka.stream.scaladsl._
//#model
diff --git a/akka-docs/src/test/scala/docs/stream/operators/SourceOperators.scala b/akka-docs/src/test/scala/docs/stream/operators/SourceOperators.scala
index 484b786f9d..29d1269f7c 100644
--- a/akka-docs/src/test/scala/docs/stream/operators/SourceOperators.scala
+++ b/akka-docs/src/test/scala/docs/stream/operators/SourceOperators.scala
@@ -13,7 +13,6 @@ object SourceOperators {
//#sourceFromFuture
import akka.actor.ActorSystem
- import akka.stream.ActorMaterializer
import akka.stream.scaladsl._
import akka.{ Done, NotUsed }
diff --git a/akka-docs/src/test/scala/docs/stream/operators/sourceorflow/Scan.scala b/akka-docs/src/test/scala/docs/stream/operators/sourceorflow/Scan.scala
index 065583e656..ccd7097bda 100644
--- a/akka-docs/src/test/scala/docs/stream/operators/sourceorflow/Scan.scala
+++ b/akka-docs/src/test/scala/docs/stream/operators/sourceorflow/Scan.scala
@@ -8,7 +8,6 @@ import akka.stream.scaladsl.Source
object Scan {
def scanExample(): Unit = {
import akka.actor.ActorSystem
- import akka.stream.ActorMaterializer
implicit val system: ActorSystem = ActorSystem()
diff --git a/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/AllPersistenceIdsStage.scala b/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/AllPersistenceIdsStage.scala
index 2e327d925a..dcb7dd50d1 100644
--- a/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/AllPersistenceIdsStage.scala
+++ b/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/AllPersistenceIdsStage.scala
@@ -4,31 +4,40 @@
package akka.persistence.query.journal.leveldb
+import akka.NotUsed
import akka.actor.ActorRef
import akka.annotation.InternalApi
import akka.persistence.Persistence
import akka.persistence.journal.leveldb.LeveldbJournal
-import akka.stream.{ ActorMaterializer, Attributes, Outlet, SourceShape }
-import akka.stream.stage.{ GraphStage, GraphStageLogic, OutHandler, TimerGraphStageLogicWithLogging }
+import akka.stream.Attributes
+import akka.stream.Materializer
+import akka.stream.Outlet
+import akka.stream.SourceShape
+import akka.stream.stage.GraphStage
+import akka.stream.stage.GraphStageLogic
+import akka.stream.stage.OutHandler
+import akka.stream.stage.TimerGraphStageLogicWithLogging
/**
* INTERNAL API
*/
@InternalApi
-final private[akka] class AllPersistenceIdsStage(
- liveQuery: Boolean,
- writeJournalPluginId: String,
- mat: ActorMaterializer)
+final private[akka] class AllPersistenceIdsStage(liveQuery: Boolean, writeJournalPluginId: String)
extends GraphStage[SourceShape[String]] {
val out: Outlet[String] = Outlet("AllPersistenceIds.out")
override def shape: SourceShape[String] = SourceShape(out)
- override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = {
- new TimerGraphStageLogicWithLogging(shape) with OutHandler with Buffer[String] {
+ override def createLogic(inheritedAttributes: Attributes): GraphStageLogic =
+ throw new UnsupportedOperationException("Not used")
+
+ override private[akka] def createLogicAndMaterializedValue(
+ inheritedAttributes: Attributes,
+ eagerMaterializer: Materializer): (GraphStageLogic, NotUsed) = {
+ val logic = new TimerGraphStageLogicWithLogging(shape) with OutHandler with Buffer[String] {
setHandler(out, this)
- val journal: ActorRef = Persistence(mat.system).journalFor(writeJournalPluginId)
+ val journal: ActorRef = Persistence(eagerMaterializer.system).journalFor(writeJournalPluginId)
var initialResponseReceived = false
override def preStart(): Unit = {
@@ -60,5 +69,7 @@ final private[akka] class AllPersistenceIdsStage(
}
}
+
+ (logic, NotUsed)
}
}
diff --git a/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/EventsByPersistenceIdStage.scala b/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/EventsByPersistenceIdStage.scala
index 2e607e5b77..82cebafe81 100644
--- a/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/EventsByPersistenceIdStage.scala
+++ b/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/EventsByPersistenceIdStage.scala
@@ -4,16 +4,27 @@
package akka.persistence.query.journal.leveldb
+import akka.NotUsed
import akka.actor.ActorRef
import akka.annotation.InternalApi
-import akka.persistence.JournalProtocol.{ RecoverySuccess, ReplayMessages, ReplayMessagesFailure, ReplayedMessage }
+import akka.persistence.JournalProtocol.RecoverySuccess
+import akka.persistence.JournalProtocol.ReplayMessages
+import akka.persistence.JournalProtocol.ReplayMessagesFailure
+import akka.persistence.JournalProtocol.ReplayedMessage
import akka.persistence.Persistence
import akka.persistence.journal.leveldb.LeveldbJournal
import akka.persistence.journal.leveldb.LeveldbJournal.EventAppended
-import akka.persistence.query.{ EventEnvelope, Sequence }
+import akka.persistence.query.EventEnvelope
+import akka.persistence.query.Sequence
import akka.persistence.query.journal.leveldb.EventsByPersistenceIdStage.Continue
-import akka.stream.{ ActorMaterializer, Attributes, Outlet, SourceShape }
-import akka.stream.stage.{ GraphStage, GraphStageLogic, OutHandler, TimerGraphStageLogicWithLogging }
+import akka.stream.Attributes
+import akka.stream.Materializer
+import akka.stream.Outlet
+import akka.stream.SourceShape
+import akka.stream.stage.GraphStage
+import akka.stream.stage.GraphStageLogic
+import akka.stream.stage.OutHandler
+import akka.stream.stage.TimerGraphStageLogicWithLogging
import scala.concurrent.duration.FiniteDuration
@@ -36,11 +47,17 @@ final private[akka] class EventsByPersistenceIdStage(
maxBufSize: Int,
writeJournalPluginId: String,
refreshInterval: Option[FiniteDuration],
- mat: ActorMaterializer)
+ mat: Materializer)
extends GraphStage[SourceShape[EventEnvelope]] {
val out: Outlet[EventEnvelope] = Outlet("EventsByPersistenceIdSource")
override def shape: SourceShape[EventEnvelope] = SourceShape(out)
- override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = {
+
+ override def createLogic(inheritedAttributes: Attributes): GraphStageLogic =
+ throw new UnsupportedOperationException("Not used")
+
+ override private[akka] def createLogicAndMaterializedValue(
+ inheritedAttributes: Attributes,
+ materializer: Materializer): (GraphStageLogic, NotUsed) = {
val logic = new TimerGraphStageLogicWithLogging(shape) with OutHandler with Buffer[EventEnvelope] {
val journal: ActorRef = Persistence(mat.system).journalFor(writeJournalPluginId)
var currSeqNo = fromSequenceNr
@@ -140,7 +157,7 @@ final private[akka] class EventsByPersistenceIdStage(
setHandler(out, this)
}
- logic
+ (logic, NotUsed)
}
}
diff --git a/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/EventsByTagStage.scala b/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/EventsByTagStage.scala
index 255720d08f..6d410a3bfb 100644
--- a/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/EventsByTagStage.scala
+++ b/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/EventsByTagStage.scala
@@ -4,20 +4,27 @@
package akka.persistence.query.journal.leveldb
+import akka.NotUsed
import akka.actor.ActorRef
import akka.annotation.InternalApi
-import akka.persistence.JournalProtocol.{ RecoverySuccess, ReplayMessagesFailure }
+import akka.persistence.JournalProtocol.RecoverySuccess
+import akka.persistence.JournalProtocol.ReplayMessagesFailure
import akka.persistence.Persistence
import akka.persistence.journal.leveldb.LeveldbJournal
-import akka.persistence.journal.leveldb.LeveldbJournal.{
- ReplayTaggedMessages,
- ReplayedTaggedMessage,
- TaggedEventAppended
-}
+import akka.persistence.journal.leveldb.LeveldbJournal.ReplayTaggedMessages
+import akka.persistence.journal.leveldb.LeveldbJournal.ReplayedTaggedMessage
+import akka.persistence.journal.leveldb.LeveldbJournal.TaggedEventAppended
import akka.persistence.query.journal.leveldb.EventsByTagStage.Continue
-import akka.persistence.query.{ EventEnvelope, Sequence }
-import akka.stream.stage.{ GraphStage, GraphStageLogic, OutHandler, TimerGraphStageLogicWithLogging }
-import akka.stream.{ ActorMaterializer, Attributes, Outlet, SourceShape }
+import akka.persistence.query.EventEnvelope
+import akka.persistence.query.Sequence
+import akka.stream.Materializer
+import akka.stream.stage.GraphStage
+import akka.stream.stage.GraphStageLogic
+import akka.stream.stage.OutHandler
+import akka.stream.stage.TimerGraphStageLogicWithLogging
+import akka.stream.Attributes
+import akka.stream.Outlet
+import akka.stream.SourceShape
import scala.concurrent.duration.FiniteDuration
@@ -38,7 +45,6 @@ final private[leveldb] class EventsByTagStage(
maxBufSize: Int,
initialTooOffset: Long,
writeJournalPluginId: String,
- mat: ActorMaterializer,
refreshInterval: Option[FiniteDuration])
extends GraphStage[SourceShape[EventEnvelope]] {
@@ -46,9 +52,15 @@ final private[leveldb] class EventsByTagStage(
override def shape: SourceShape[EventEnvelope] = SourceShape(out)
- override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = {
+ override def createLogic(inheritedAttributes: Attributes): GraphStageLogic =
+ throw new UnsupportedOperationException("Not used")
+
+ override private[akka] def createLogicAndMaterializedValue(
+ inheritedAttributes: Attributes,
+ eagerMaterializer: Materializer): (GraphStageLogic, NotUsed) = {
+
val logic = new TimerGraphStageLogicWithLogging(shape) with OutHandler with Buffer[EventEnvelope] {
- val journal: ActorRef = Persistence(mat.system).journalFor(writeJournalPluginId)
+ val journal: ActorRef = Persistence(eagerMaterializer.system).journalFor(writeJournalPluginId)
var currOffset: Long = fromOffset
var toOffset: Long = initialTooOffset
var stageActorRef: ActorRef = null
@@ -144,7 +156,7 @@ final private[leveldb] class EventsByTagStage(
setHandler(out, this)
}
- logic
+ (logic, NotUsed)
}
}
diff --git a/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/scaladsl/LeveldbReadJournal.scala b/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/scaladsl/LeveldbReadJournal.scala
index 75f6522327..c62cb2eb43 100644
--- a/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/scaladsl/LeveldbReadJournal.scala
+++ b/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/scaladsl/LeveldbReadJournal.scala
@@ -9,11 +9,18 @@ import java.net.URLEncoder
import akka.NotUsed
import akka.actor.ExtendedActorSystem
import akka.event.Logging
-import akka.persistence.query.journal.leveldb.{ AllPersistenceIdsStage, EventsByPersistenceIdStage, EventsByTagStage }
-import akka.persistence.query.scaladsl.{ ReadJournal, _ }
-import akka.persistence.query.{ EventEnvelope, NoOffset, Offset, Sequence }
+import akka.persistence.query.journal.leveldb.AllPersistenceIdsStage
+import akka.persistence.query.journal.leveldb.EventsByPersistenceIdStage
+import akka.persistence.query.journal.leveldb.EventsByTagStage
+import akka.persistence.query.scaladsl.ReadJournal
+import akka.persistence.query.scaladsl._
+import akka.persistence.query.EventEnvelope
+import akka.persistence.query.NoOffset
+import akka.persistence.query.Offset
+import akka.persistence.query.Sequence
import akka.stream.scaladsl.Source
-import akka.util.{ unused, ByteString }
+import akka.util.ByteString
+import akka.util.unused
import com.typesafe.config.Config
import scala.concurrent.duration._
@@ -63,32 +70,17 @@ class LeveldbReadJournal(@unused system: ExtendedActorSystem, config: Config)
* The stream is completed with failure if there is a failure in executing the query in the
* backend journal.
*/
- override def persistenceIds(): Source[String, NotUsed] = {
+ override def persistenceIds(): Source[String, NotUsed] =
// no polling for this query, the write journal will push all changes, i.e. no refreshInterval
- Source
- .setup { (mat, _) =>
- Source
- .fromGraph(new AllPersistenceIdsStage(liveQuery = true, writeJournalPluginId, mat))
- .named("allPersistenceIds")
- }
- .mapMaterializedValue(_ => NotUsed)
-
- }
+ Source.fromGraph(new AllPersistenceIdsStage(liveQuery = true, writeJournalPluginId)).named("allPersistenceIds")
/**
* Same type of query as [[#persistenceIds]] but the stream
* is completed immediately when it reaches the end of the "result set". Persistent
* actors that are created after the query is completed are not included in the stream.
*/
- override def currentPersistenceIds(): Source[String, NotUsed] = {
- Source
- .setup { (mat, _) =>
- Source
- .fromGraph(new AllPersistenceIdsStage(liveQuery = false, writeJournalPluginId, mat))
- .named("allPersistenceIds")
- }
- .mapMaterializedValue(_ => NotUsed)
- }
+ override def currentPersistenceIds(): Source[String, NotUsed] =
+ Source.fromGraph(new AllPersistenceIdsStage(liveQuery = false, writeJournalPluginId)).named("allPersistenceIds")
/**
* `eventsByPersistenceId` is used for retrieving events for a specific
@@ -121,7 +113,7 @@ class LeveldbReadJournal(@unused system: ExtendedActorSystem, config: Config)
fromSequenceNr: Long = 0L,
toSequenceNr: Long = Long.MaxValue): Source[EventEnvelope, NotUsed] = {
Source
- .setup { (mat, _) =>
+ .fromMaterializer { (mat, _) =>
Source
.fromGraph(
new EventsByPersistenceIdStage(
@@ -147,7 +139,7 @@ class LeveldbReadJournal(@unused system: ExtendedActorSystem, config: Config)
fromSequenceNr: Long = 0L,
toSequenceNr: Long = Long.MaxValue): Source[EventEnvelope, NotUsed] = {
Source
- .setup { (mat, _) =>
+ .fromMaterializer { (mat, _) =>
Source
.fromGraph(
new EventsByPersistenceIdStage(
@@ -207,20 +199,10 @@ class LeveldbReadJournal(@unused system: ExtendedActorSystem, config: Config)
offset match {
case seq: Sequence =>
Source
- .setup { (mat, _) =>
- Source
- .fromGraph(
- new EventsByTagStage(
- tag,
- seq.value,
- maxBufSize,
- Long.MaxValue,
- writeJournalPluginId,
- mat,
- refreshInterval))
- .named("eventsByTag-" + URLEncoder.encode(tag, ByteString.UTF_8))
- }
- .mapMaterializedValue(_ => NotUsed)
+ .fromGraph(
+ new EventsByTagStage(tag, seq.value, maxBufSize, Long.MaxValue, writeJournalPluginId, refreshInterval))
+ .named("eventsByTag-" + URLEncoder.encode(tag, ByteString.UTF_8))
+
case NoOffset => eventsByTag(tag, Sequence(0L)) //recursive
case _ =>
throw new IllegalArgumentException(
@@ -232,23 +214,17 @@ class LeveldbReadJournal(@unused system: ExtendedActorSystem, config: Config)
* is completed immediately when it reaches the end of the "result set". Events that are
* stored after the query is completed are not included in the event stream.
*/
- override def currentEventsByTag(tag: String, offset: Offset = Sequence(0L)): Source[EventEnvelope, NotUsed] = {
- Source
- .setup { (mat, _) =>
- offset match {
- case seq: Sequence =>
- Source
- .fromGraph(
- new EventsByTagStage(tag, seq.value, maxBufSize, Long.MaxValue, writeJournalPluginId, mat, None))
- .named("currentEventsByTag-" + URLEncoder.encode(tag, ByteString.UTF_8))
- case NoOffset => currentEventsByTag(tag, Sequence(0L))
- case _ =>
- throw new IllegalArgumentException(
- "LevelDB does not support " + Logging.simpleName(offset.getClass) + " offsets")
- }
- }
- .mapMaterializedValue(_ => NotUsed)
- }
+ override def currentEventsByTag(tag: String, offset: Offset = Sequence(0L)): Source[EventEnvelope, NotUsed] =
+ offset match {
+ case seq: Sequence =>
+ Source
+ .fromGraph(new EventsByTagStage(tag, seq.value, maxBufSize, Long.MaxValue, writeJournalPluginId, None))
+ .named("currentEventsByTag-" + URLEncoder.encode(tag, ByteString.UTF_8))
+ case NoOffset => currentEventsByTag(tag, Sequence(0L))
+ case _ =>
+ throw new IllegalArgumentException(
+ "LevelDB does not support " + Logging.simpleName(offset.getClass) + " offsets")
+ }
}
diff --git a/akka-persistence-query/src/test/scala/akka/persistence/query/journal/leveldb/AllPersistenceIdsSpec.scala b/akka-persistence-query/src/test/scala/akka/persistence/query/journal/leveldb/AllPersistenceIdsSpec.scala
index 346202adf9..3b7c181838 100644
--- a/akka-persistence-query/src/test/scala/akka/persistence/query/journal/leveldb/AllPersistenceIdsSpec.scala
+++ b/akka-persistence-query/src/test/scala/akka/persistence/query/journal/leveldb/AllPersistenceIdsSpec.scala
@@ -4,16 +4,15 @@
package akka.persistence.query.journal.leveldb
-import scala.concurrent.duration._
-
import akka.persistence.query.PersistenceQuery
import akka.persistence.query.journal.leveldb.scaladsl.LeveldbReadJournal
import akka.persistence.query.scaladsl.PersistenceIdsQuery
-import akka.stream.ActorMaterializer
import akka.stream.testkit.scaladsl.TestSink
import akka.testkit.AkkaSpec
import akka.testkit.ImplicitSender
+import scala.concurrent.duration._
+
object AllPersistenceIdsSpec {
val config = """
akka.loglevel = INFO
@@ -28,8 +27,6 @@ object AllPersistenceIdsSpec {
class AllPersistenceIdsSpec extends AkkaSpec(AllPersistenceIdsSpec.config) with Cleanup with ImplicitSender {
- implicit val mat = ActorMaterializer()(system)
-
val queries = PersistenceQuery(system).readJournalFor[LeveldbReadJournal](LeveldbReadJournal.Identifier)
"Leveldb query AllPersistenceIds" must {
diff --git a/akka-persistence-query/src/test/scala/akka/persistence/query/journal/leveldb/EventsByPersistenceIdSpec.scala b/akka-persistence-query/src/test/scala/akka/persistence/query/journal/leveldb/EventsByPersistenceIdSpec.scala
index 37d660fc57..9586143663 100644
--- a/akka-persistence-query/src/test/scala/akka/persistence/query/journal/leveldb/EventsByPersistenceIdSpec.scala
+++ b/akka-persistence-query/src/test/scala/akka/persistence/query/journal/leveldb/EventsByPersistenceIdSpec.scala
@@ -4,17 +4,16 @@
package akka.persistence.query.journal.leveldb
-import scala.concurrent.duration._
-
import akka.actor.ActorRef
import akka.persistence.query.PersistenceQuery
import akka.persistence.query.journal.leveldb.scaladsl.LeveldbReadJournal
import akka.persistence.query.scaladsl.EventsByTagQuery
-import akka.stream.ActorMaterializer
import akka.stream.testkit.scaladsl.TestSink
import akka.testkit.AkkaSpec
import akka.testkit.ImplicitSender
+import scala.concurrent.duration._
+
object EventsByPersistenceIdSpec {
val config = """
akka.loglevel = INFO
@@ -30,8 +29,6 @@ object EventsByPersistenceIdSpec {
class EventsByPersistenceIdSpec extends AkkaSpec(EventsByPersistenceIdSpec.config) with Cleanup with ImplicitSender {
- implicit val mat = ActorMaterializer()(system)
-
val queries = PersistenceQuery(system).readJournalFor[LeveldbReadJournal](LeveldbReadJournal.Identifier)
def setup(persistenceId: String): ActorRef = {
diff --git a/akka-persistence-query/src/test/scala/akka/persistence/query/journal/leveldb/EventsByTagSpec.scala b/akka-persistence-query/src/test/scala/akka/persistence/query/journal/leveldb/EventsByTagSpec.scala
index f717d90c1c..79c772c891 100644
--- a/akka-persistence-query/src/test/scala/akka/persistence/query/journal/leveldb/EventsByTagSpec.scala
+++ b/akka-persistence-query/src/test/scala/akka/persistence/query/journal/leveldb/EventsByTagSpec.scala
@@ -4,17 +4,19 @@
package akka.persistence.query.journal.leveldb
-import scala.concurrent.duration._
import akka.persistence.journal.Tagged
import akka.persistence.journal.WriteEventAdapter
-import akka.persistence.query.{ EventEnvelope, PersistenceQuery, Sequence }
+import akka.persistence.query.NoOffset
import akka.persistence.query.journal.leveldb.scaladsl.LeveldbReadJournal
import akka.persistence.query.scaladsl.EventsByTagQuery
-import akka.stream.ActorMaterializer
+import akka.persistence.query.EventEnvelope
+import akka.persistence.query.PersistenceQuery
+import akka.persistence.query.Sequence
import akka.stream.testkit.scaladsl.TestSink
import akka.testkit.AkkaSpec
import akka.testkit.ImplicitSender
-import akka.persistence.query.NoOffset
+
+import scala.concurrent.duration._
object EventsByTagSpec {
val config = s"""
@@ -59,8 +61,6 @@ class ColorTagger extends WriteEventAdapter {
class EventsByTagSpec extends AkkaSpec(EventsByTagSpec.config) with Cleanup with ImplicitSender {
- implicit val mat = ActorMaterializer()(system)
-
val queries = PersistenceQuery(system).readJournalFor[LeveldbReadJournal](LeveldbReadJournal.Identifier)
"Leveldb query EventsByTag" must {
diff --git a/akka-persistence-typed/src/test/java/akka/persistence/typed/javadsl/PersistentActorJavaDslTest.java b/akka-persistence-typed/src/test/java/akka/persistence/typed/javadsl/PersistentActorJavaDslTest.java
index 9e37112b5e..3946695872 100644
--- a/akka-persistence-typed/src/test/java/akka/persistence/typed/javadsl/PersistentActorJavaDslTest.java
+++ b/akka-persistence-typed/src/test/java/akka/persistence/typed/javadsl/PersistentActorJavaDslTest.java
@@ -19,7 +19,6 @@ import akka.persistence.query.journal.leveldb.javadsl.LeveldbReadJournal;
import akka.persistence.typed.*;
import akka.persistence.typed.scaladsl.EventSourcedBehaviorSpec;
import akka.serialization.jackson.CborSerializable;
-import akka.stream.ActorMaterializer;
import akka.stream.javadsl.Sink;
import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
import akka.actor.testkit.typed.javadsl.TestProbe;
@@ -54,9 +53,6 @@ public class PersistentActorJavaDslTest extends JUnitSuite {
PersistenceQuery.get(Adapter.toClassic(testKit.system()))
.getReadJournalFor(LeveldbReadJournal.class, LeveldbReadJournal.Identifier());
- private ActorMaterializer materializer =
- ActorMaterializer.create(Adapter.toClassic(testKit.system()));
-
interface Command extends CborSerializable {}
public enum Increment implements Command {
@@ -547,7 +543,7 @@ public class PersistentActorJavaDslTest extends JUnitSuite {
List events =
queries
.currentEventsByTag("tag1", NoOffset.getInstance())
- .runWith(Sink.seq(), materializer)
+ .runWith(Sink.seq(), testKit.system())
.toCompletableFuture()
.get();
assertEquals(
@@ -578,7 +574,7 @@ public class PersistentActorJavaDslTest extends JUnitSuite {
List events =
queries
.currentEventsByPersistenceId("transform", 0, Long.MAX_VALUE)
- .runWith(Sink.seq(), materializer)
+ .runWith(Sink.seq(), testKit.system())
.toCompletableFuture()
.get();
assertEquals(
diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorSpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorSpec.scala
index 7638396dc5..991367b2ee 100644
--- a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorSpec.scala
+++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorSpec.scala
@@ -41,7 +41,6 @@ import akka.persistence.typed.SnapshotMetadata
import akka.persistence.{ SnapshotMetadata => ClassicSnapshotMetadata }
import akka.persistence.{ SnapshotSelectionCriteria => ClassicSnapshotSelectionCriteria }
import akka.serialization.jackson.CborSerializable
-import akka.stream.ActorMaterializer
import akka.stream.scaladsl.Sink
import akka.testkit.EventFilter
import akka.testkit.TestEvent.Mute
@@ -289,7 +288,6 @@ class EventSourcedBehaviorSpec extends ScalaTestWithActorTestKit(EventSourcedBeh
import EventSourcedBehaviorSpec._
import akka.actor.typed.scaladsl.adapter._
- implicit val materializer = ActorMaterializer()(system.toClassic)
val queries: LeveldbReadJournal =
PersistenceQuery(system.toClassic).readJournalFor[LeveldbReadJournal](LeveldbReadJournal.Identifier)
diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedEventAdapterSpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedEventAdapterSpec.scala
index 11a854898b..83f9a0f7ed 100644
--- a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedEventAdapterSpec.scala
+++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedEventAdapterSpec.scala
@@ -19,7 +19,6 @@ import akka.persistence.typed.EventAdapter
import akka.persistence.typed.EventSeq
import akka.persistence.typed.PersistenceId
import akka.serialization.jackson.CborSerializable
-import akka.stream.ActorMaterializer
import akka.stream.scaladsl.Sink
import akka.testkit.EventFilter
import akka.testkit.JavaSerializable
@@ -104,7 +103,6 @@ class EventSourcedEventAdapterSpec
val pidCounter = new AtomicInteger(0)
private def nextPid(): PersistenceId = PersistenceId(s"c${pidCounter.incrementAndGet()})")
- implicit val materializer = ActorMaterializer()(system.toClassic)
val queries: LeveldbReadJournal =
PersistenceQuery(system.toClassic).readJournalFor[LeveldbReadJournal](LeveldbReadJournal.Identifier)
diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedSnapshotAdapterSpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedSnapshotAdapterSpec.scala
index 200fbd12a5..a8ac972b42 100644
--- a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedSnapshotAdapterSpec.scala
+++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedSnapshotAdapterSpec.scala
@@ -7,14 +7,16 @@ package akka.persistence.typed.scaladsl
import java.util.UUID
import java.util.concurrent.atomic.AtomicInteger
-import akka.actor.testkit.typed.scaladsl.{ ScalaTestWithActorTestKit, TestProbe }
+import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
+import akka.actor.testkit.typed.scaladsl.TestProbe
import akka.actor.typed.ActorRef
import akka.persistence.query.PersistenceQuery
import akka.persistence.query.journal.leveldb.scaladsl.LeveldbReadJournal
-import akka.persistence.typed.{ PersistenceId, SnapshotAdapter }
+import akka.persistence.typed.PersistenceId
+import akka.persistence.typed.SnapshotAdapter
import akka.serialization.jackson.CborSerializable
-import akka.stream.ActorMaterializer
-import com.typesafe.config.{ Config, ConfigFactory }
+import com.typesafe.config.Config
+import com.typesafe.config.ConfigFactory
import org.scalatest.WordSpecLike
object EventSourcedSnapshotAdapterSpec {
@@ -38,7 +40,7 @@ class EventSourcedSnapshotAdapterSpec
val pidCounter = new AtomicInteger(0)
private def nextPid(): PersistenceId = PersistenceId(s"c${pidCounter.incrementAndGet()})")
- implicit val materializer = ActorMaterializer()(system.toClassic)
+
val queries: LeveldbReadJournal =
PersistenceQuery(system.toClassic).readJournalFor[LeveldbReadJournal](LeveldbReadJournal.Identifier)
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/LatencySpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/LatencySpec.scala
index 9332e8e7dc..c6d36d701b 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/LatencySpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/LatencySpec.scala
@@ -14,7 +14,7 @@ import akka.remote.RemotingMultiNodeSpec
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.serialization.jackson.CborSerializable
-import akka.stream.{ ActorMaterializer, ThrottleMode }
+import akka.stream.ThrottleMode
import akka.stream.scaladsl.Source
import akka.testkit._
import com.typesafe.config.ConfigFactory
@@ -189,7 +189,6 @@ abstract class LatencySpec extends RemotingMultiNodeSpec(LatencySpec) {
var plots = LatencyPlots()
- lazy implicit val mat = ActorMaterializer()(system)
import system.dispatcher
override def initialParticipants = roles.size
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/aeron/AeronStreamConcistencySpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/aeron/AeronStreamConcistencySpec.scala
index d5ba148d6d..03038b938f 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/aeron/AeronStreamConcistencySpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/aeron/AeronStreamConcistencySpec.scala
@@ -8,9 +8,6 @@ package aeron
import java.io.File
import java.util.concurrent.atomic.AtomicInteger
-import scala.concurrent.Await
-import scala.concurrent.duration._
-
import akka.Done
import akka.actor.ExtendedActorSystem
import akka.actor.Props
@@ -18,7 +15,6 @@ import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.remote.testkit.STMultiNodeSpec
-import akka.stream.ActorMaterializer
import akka.stream.KillSwitches
import akka.stream.ThrottleMode
import akka.stream.scaladsl.Source
@@ -29,6 +25,9 @@ import io.aeron.Aeron
import io.aeron.driver.MediaDriver
import org.agrona.IoUtil
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
object AeronStreamConsistencySpec extends MultiNodeConfig {
val first = role("first")
val second = role("second")
@@ -73,7 +72,6 @@ abstract class AeronStreamConsistencySpec
val pool = new EnvelopeBufferPool(1024 * 1024, 128)
- lazy implicit val mat = ActorMaterializer()(system)
import system.dispatcher
override def initialParticipants = roles.size
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/aeron/AeronStreamLatencySpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/aeron/AeronStreamLatencySpec.scala
index 72ef1ea4c8..bd7a0dd6f1 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/aeron/AeronStreamLatencySpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/aeron/AeronStreamLatencySpec.scala
@@ -22,7 +22,6 @@ import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.remote.testkit.STMultiNodeSpec
-import akka.stream.ActorMaterializer
import akka.stream.KillSwitches
import akka.stream.ThrottleMode
import akka.stream.scaladsl.Flow
@@ -106,8 +105,6 @@ abstract class AeronStreamLatencySpec
r
}
- lazy implicit val mat = ActorMaterializer()(system)
-
override def initialParticipants = roles.size
def channel(roleName: RoleName) = {
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/aeron/AeronStreamMaxThroughputSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/aeron/AeronStreamMaxThroughputSpec.scala
index bc908e26f4..6ef1e53b28 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/aeron/AeronStreamMaxThroughputSpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/aeron/AeronStreamMaxThroughputSpec.scala
@@ -16,7 +16,6 @@ import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.remote.testkit.STMultiNodeSpec
-import akka.stream.ActorMaterializer
import akka.stream.scaladsl.Source
import akka.testkit._
import com.typesafe.config.ConfigFactory
@@ -103,7 +102,6 @@ abstract class AeronStreamMaxThroughputSpec
r
}
- lazy implicit val mat = ActorMaterializer()(system)
import system.dispatcher
def adjustedTotalMessages(n: Long): Long = (n * totalMessagesFactor).toLong
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 03d1a19a14..667f360fbf 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/SendQueueSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/SendQueueSpec.scala
@@ -51,7 +51,8 @@ object SendQueueSpec {
class SendQueueSpec extends AkkaSpec("""
akka.actor.serialize-messages = off
akka.stream.materializer.debug.fuzzing-mode = on
- """.stripMargin) with ImplicitSender {
+ akka.stream.secret-test-fuzzing-warning-disable = yep
+ """) with ImplicitSender {
import SendQueueSpec._
def sendToDeadLetters[T](pending: Vector[T]): Unit =
diff --git a/akka-stream-testkit/src/test/scala/akka/stream/testkit/ChainSetup.scala b/akka-stream-testkit/src/test/scala/akka/stream/testkit/ChainSetup.scala
index 791ee5ff5f..ef5be53763 100644
--- a/akka-stream-testkit/src/test/scala/akka/stream/testkit/ChainSetup.scala
+++ b/akka-stream-testkit/src/test/scala/akka/stream/testkit/ChainSetup.scala
@@ -5,29 +5,34 @@
package akka.stream.testkit
import akka.NotUsed
-import akka.actor.{ ActorRefFactory, ActorSystem }
-import akka.stream.ActorMaterializerSettings
-import akka.stream.scaladsl._
-import org.reactivestreams.Publisher
+import akka.actor.ActorRefFactory
+import akka.actor.ActorSystem
import akka.stream.ActorMaterializer
+import akka.stream.ActorMaterializerSettings
+import akka.stream.Materializer
+import akka.stream.scaladsl._
+import com.github.ghik.silencer.silent
+import org.reactivestreams.Publisher
class ChainSetup[In, Out, M](
stream: Flow[In, In, NotUsed] => Flow[In, Out, M],
val settings: ActorMaterializerSettings,
- materializer: ActorMaterializer,
- toPublisher: (Source[Out, _], ActorMaterializer) => Publisher[Out])(implicit val system: ActorSystem) {
+ materializer: Materializer,
+ toPublisher: (Source[Out, _], Materializer) => Publisher[Out])(implicit val system: ActorSystem) {
+ @silent("deprecated")
def this(
stream: Flow[In, In, NotUsed] => Flow[In, Out, M],
settings: ActorMaterializerSettings,
- toPublisher: (Source[Out, _], ActorMaterializer) => Publisher[Out])(implicit system: ActorSystem) =
+ toPublisher: (Source[Out, _], Materializer) => Publisher[Out])(implicit system: ActorSystem) =
this(stream, settings, ActorMaterializer(settings)(system), toPublisher)(system)
+ @silent("deprecated")
def this(
stream: Flow[In, In, NotUsed] => Flow[In, Out, M],
settings: ActorMaterializerSettings,
- materializerCreator: (ActorMaterializerSettings, ActorRefFactory) => ActorMaterializer,
- toPublisher: (Source[Out, _], ActorMaterializer) => Publisher[Out])(implicit system: ActorSystem) =
+ materializerCreator: (ActorMaterializerSettings, ActorRefFactory) => Materializer,
+ toPublisher: (Source[Out, _], Materializer) => Publisher[Out])(implicit system: ActorSystem) =
this(stream, settings, materializerCreator(settings, system), toPublisher)(system)
val upstream = TestPublisher.manualProbe[In]()
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 1e86eadd8c..b6eff7933c 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
@@ -4,12 +4,18 @@
package akka.stream.testkit
+import java.util.concurrent.ThreadLocalRandom
+
import akka.NotUsed
import akka.actor.ActorSystem
+import akka.stream.ActorMaterializerSettings
+import akka.stream.Materializer
+import akka.stream.SystemMaterializer
+import akka.stream.scaladsl.Flow
+import akka.stream.scaladsl.Sink
+import akka.stream.scaladsl.Source
import akka.stream.testkit.TestPublisher._
import akka.stream.testkit.TestSubscriber._
-import akka.stream.{ ActorMaterializer, ActorMaterializerSettings }
-import akka.stream.scaladsl.{ Flow, Sink, Source }
import org.reactivestreams.Publisher
import org.scalatest.Matchers
@@ -24,7 +30,7 @@ trait ScriptedTest extends Matchers {
class ScriptException(msg: String) extends RuntimeException(msg)
- def toPublisher[In, Out]: (Source[Out, _], ActorMaterializer) => Publisher[Out] =
+ def toPublisher[In, Out]: (Source[Out, _], Materializer) => Publisher[Out] =
(f, m) => f.runWith(Sink.asPublisher(false))(m)
object Script {
diff --git a/akka-stream-testkit/src/test/scala/akka/stream/testkit/StreamTestKitSpec.scala b/akka-stream-testkit/src/test/scala/akka/stream/testkit/StreamTestKitSpec.scala
index 82c672ca25..e615cdd828 100644
--- a/akka-stream-testkit/src/test/scala/akka/stream/testkit/StreamTestKitSpec.scala
+++ b/akka-stream-testkit/src/test/scala/akka/stream/testkit/StreamTestKitSpec.scala
@@ -4,19 +4,16 @@
package akka.stream.testkit
-import akka.stream._
import akka.stream.scaladsl.Source
import akka.stream.testkit.scaladsl.TestSink
-import scala.concurrent.duration._
-
import akka.testkit.AkkaSpec
import akka.testkit.EventFilter
import akka.testkit.TestEvent.Mute
import akka.testkit.TestEvent.UnMute
-class StreamTestKitSpec extends AkkaSpec {
+import scala.concurrent.duration._
- implicit val materializer = ActorMaterializer()
+class StreamTestKitSpec extends AkkaSpec {
val ex = new Exception("Boom!")
diff --git a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/MapTest.scala b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/MapTest.scala
index ab96166b6c..3ba2e3f2b3 100644
--- a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/MapTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/MapTest.scala
@@ -4,15 +4,12 @@
package akka.stream.tck
-import akka.stream.ActorMaterializer
import akka.stream.scaladsl.Flow
import org.reactivestreams.Processor
class MapTest extends AkkaIdentityProcessorVerification[Int] {
override def createIdentityProcessor(maxBufferSize: Int): Processor[Int, Int] = {
- implicit val materializer = ActorMaterializer()(system)
-
Flow[Int].map(elem => elem).named("identity").toProcessor.run()
}
diff --git a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/VirtualPublisherTest.scala b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/VirtualPublisherTest.scala
index 01d178db23..d841e317af 100644
--- a/akka-stream-tests-tck/src/test/scala/akka/stream/tck/VirtualPublisherTest.scala
+++ b/akka-stream-tests-tck/src/test/scala/akka/stream/tck/VirtualPublisherTest.scala
@@ -4,16 +4,13 @@
package akka.stream.tck
-import akka.stream.ActorMaterializer
+import akka.stream.impl.VirtualProcessor
import akka.stream.scaladsl.Flow
import org.reactivestreams.Processor
-import akka.stream.impl.VirtualProcessor
class VirtualProcessorTest extends AkkaIdentityProcessorVerification[Int] {
override def createIdentityProcessor(maxBufferSize: Int): Processor[Int, Int] = {
- implicit val materializer = ActorMaterializer()(system)
-
val identity = Flow[Int].map(elem => elem).named("identity").toProcessor.run()
val left, right = new VirtualProcessor[Int]
left.subscribe(identity)
diff --git a/akka-stream-tests/src/test/java/akka/stream/StreamAttributeDocTest.java b/akka-stream-tests/src/test/java/akka/stream/StreamAttributeDocTest.java
index f439490cb3..d159d6be3e 100644
--- a/akka-stream-tests/src/test/java/akka/stream/StreamAttributeDocTest.java
+++ b/akka-stream-tests/src/test/java/akka/stream/StreamAttributeDocTest.java
@@ -5,7 +5,6 @@
package akka.stream;
import akka.Done;
-import akka.NotUsed;
import akka.stream.javadsl.Keep;
import akka.stream.javadsl.RunnableGraph;
import akka.stream.javadsl.Sink;
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 dc5f3c81cc..2709c34c9b 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
@@ -15,20 +15,20 @@ 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
import akka.stream.testkit.TestPublisher
import akka.stream.testkit.TestSubscriber
+import akka.stream.testkit.Utils._
+import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.testkit.EventFilter
import akka.testkit.TestLatch
import org.reactivestreams.Publisher
import org.reactivestreams.Subscriber
import org.reactivestreams.Subscription
-import scala.concurrent.duration._
import scala.concurrent.Await
import scala.concurrent.Promise
+import scala.concurrent.duration._
class ActorGraphInterpreterSpec extends StreamSpec {
"ActorGraphInterpreter" must {
@@ -403,7 +403,7 @@ class ActorGraphInterpreterSpec extends StreamSpec {
}
"trigger postStop in all stages when abruptly terminated (and no upstream boundaries)" in {
- val mat = ActorMaterializer()
+ val mat = Materializer(system)
val gotStop = TestLatch(1)
object PostStopSnitchFlow extends SimpleLinearGraphStage[String] {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/GraphInterpreterSpecKit.scala b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/GraphInterpreterSpecKit.scala
index 9c26724785..bf8ea811ce 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/GraphInterpreterSpecKit.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/impl/fusing/GraphInterpreterSpecKit.scala
@@ -4,6 +4,10 @@
package akka.stream.impl.fusing
+import akka.actor.ActorSystem
+import akka.actor.Cancellable
+import akka.actor.Props
+import akka.annotation.InternalApi
import akka.event.Logging
import akka.stream.Supervision.Decider
import akka.stream._
@@ -19,6 +23,60 @@ import akka.stream.testkit.Utils.TE
import com.github.ghik.silencer.silent
import scala.collection.{ Map => SMap }
+import scala.concurrent.ExecutionContextExecutor
+import scala.concurrent.duration.FiniteDuration
+
+/**
+ * INTERNAL API
+ */
+@InternalApi
+private[akka] object NoMaterializer extends Materializer {
+ override def withNamePrefix(name: String): Materializer =
+ throw new UnsupportedOperationException("NoMaterializer cannot be named")
+ override def materialize[Mat](runnable: Graph[ClosedShape, Mat]): Mat =
+ throw new UnsupportedOperationException("NoMaterializer cannot materialize")
+ override def materialize[Mat](runnable: Graph[ClosedShape, Mat], defaultAttributes: Attributes): Mat =
+ throw new UnsupportedOperationException("NoMaterializer cannot materialize")
+
+ override def executionContext: ExecutionContextExecutor =
+ throw new UnsupportedOperationException("NoMaterializer does not provide an ExecutionContext")
+
+ def scheduleOnce(delay: FiniteDuration, task: Runnable): Cancellable =
+ throw new UnsupportedOperationException("NoMaterializer cannot schedule a single event")
+
+ def schedulePeriodically(initialDelay: FiniteDuration, interval: FiniteDuration, task: Runnable): Cancellable =
+ throw new UnsupportedOperationException("NoMaterializer cannot schedule a repeated event")
+
+ override def scheduleWithFixedDelay(
+ initialDelay: FiniteDuration,
+ delay: FiniteDuration,
+ task: Runnable): Cancellable =
+ throw new UnsupportedOperationException("NoMaterializer cannot scheduleWithFixedDelay")
+
+ override def scheduleAtFixedRate(
+ initialDelay: FiniteDuration,
+ interval: FiniteDuration,
+ task: Runnable): Cancellable =
+ throw new UnsupportedOperationException("NoMaterializer cannot scheduleAtFixedRate")
+
+ override def shutdown(): Unit = throw new UnsupportedOperationException("NoMaterializer cannot shutdown")
+
+ override def isShutdown: Boolean = throw new UnsupportedOperationException("NoMaterializer cannot shutdown")
+
+ override def system: ActorSystem =
+ throw new UnsupportedOperationException("NoMaterializer does not have an actorsystem")
+
+ override private[akka] def logger = throw new UnsupportedOperationException("NoMaterializer does not have a logger")
+
+ override private[akka] def supervisor =
+ throw new UnsupportedOperationException("NoMaterializer does not have a supervisor")
+
+ override private[akka] def actorOf(context: MaterializationContext, props: Props) =
+ throw new UnsupportedOperationException("NoMaterializer cannot spawn actors")
+
+ override def settings: ActorMaterializerSettings =
+ throw new UnsupportedOperationException("NoMaterializer does not have settings")
+}
@silent
object GraphInterpreterSpecKit {
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 e6b7edb4c8..1847f67494 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
@@ -4,27 +4,30 @@
package akka.stream.io
-import java.io.{ IOException, InputStream }
+import java.io.IOException
+import java.io.InputStream
+import java.util.concurrent.ThreadLocalRandom
import java.util.concurrent.TimeoutException
-import akka.actor.ActorSystem
-import akka.stream._
import akka.stream.Attributes.inputBuffer
+import akka.stream._
+import akka.stream.impl.PhasedFusingActorMaterializer
+import akka.stream.impl.StreamSupervisor
import akka.stream.impl.StreamSupervisor.Children
import akka.stream.impl.io.InputStreamSinkStage
-import akka.stream.impl.{ PhasedFusingActorMaterializer, StreamSupervisor }
-import akka.stream.scaladsl.{ Keep, Source, StreamConverters }
+import akka.stream.scaladsl.Keep
+import akka.stream.scaladsl.Source
+import akka.stream.scaladsl.StreamConverters
import akka.stream.testkit.Utils._
+import akka.stream.testkit._
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.stream.testkit.scaladsl.TestSource
-import akka.stream.testkit._
import akka.testkit.TestProbe
import akka.util.ByteString
+import scala.concurrent.Await
+import scala.concurrent.Future
import scala.concurrent.duration._
-import java.util.concurrent.ThreadLocalRandom
-
-import scala.concurrent.{ Await, Future }
import scala.util.control.NoStackTrace
class InputStreamSinkSpec extends StreamSpec(UnboundedMailboxConfig) {
@@ -211,17 +214,12 @@ class InputStreamSinkSpec extends StreamSpec(UnboundedMailboxConfig) {
}
"use dedicated default-blocking-io-dispatcher by default" in assertAllStagesStopped {
- val sys = ActorSystem("dispatcher-testing", UnboundedMailboxConfig)
- val materializer = ActorMaterializer()(sys)
- try {
- TestSource.probe[ByteString].runWith(StreamConverters.asInputStream())(materializer)
- materializer
- .asInstanceOf[PhasedFusingActorMaterializer]
- .supervisor
- .tell(StreamSupervisor.GetChildren, testActor)
- val ref = expectMsgType[Children].children.find(_.path.toString contains "inputStreamSink").get
- assertDispatcher(ref, ActorAttributes.IODispatcher.dispatcher)
- } finally shutdown(sys)
+ // use a separate materializer to ensure we know what child is our stream
+ implicit val materializer = Materializer(system)
+ TestSource.probe[ByteString].runWith(StreamConverters.asInputStream())
+ materializer.asInstanceOf[PhasedFusingActorMaterializer].supervisor.tell(StreamSupervisor.GetChildren, testActor)
+ val ref = expectMsgType[Children].children.find(_.path.toString contains "inputStreamSink").get
+ assertDispatcher(ref, ActorAttributes.IODispatcher.dispatcher)
}
"work when more bytes pulled from InputStream than available" in assertAllStagesStopped {
@@ -253,7 +251,7 @@ class InputStreamSinkSpec extends StreamSpec(UnboundedMailboxConfig) {
}
"throw from inputstream read if terminated abruptly" in {
- val mat = ActorMaterializer()
+ val mat = Materializer(system)
val probe = TestPublisher.probe[ByteString]()
val inputStream = Source.fromPublisher(probe).runWith(StreamConverters.asInputStream())(mat)
mat.shutdown()
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 f0ddb45e53..2e595d289a 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
@@ -180,7 +180,7 @@ class OutputStreamSourceSpec extends StreamSpec(UnboundedMailboxConfig) {
}
"not leave blocked threads when materializer shutdown" in {
- val materializer2 = ActorMaterializer()(system)
+ val materializer2 = Materializer(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/TcpSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/io/TcpSpec.scala
index 071a881efc..a85eeaec16 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
@@ -513,12 +513,11 @@ class TcpSpec extends StreamSpec("""
""").withFallback(system.settings.config))
try {
- val mat2 = ActorMaterializer.create(system2)
+ implicit val materializer = SystemMaterializer(system2).materializer
val serverAddress = temporaryServerAddress()
- val binding = Tcp(system2)
- .bindAndHandle(Flow[ByteString], serverAddress.getHostString, serverAddress.getPort)(mat2)
- .futureValue
+ val binding =
+ Tcp(system2).bindAndHandle(Flow[ByteString], serverAddress.getHostString, serverAddress.getPort).futureValue
val probe = TestProbe()
val testMsg = ByteString(0)
@@ -529,7 +528,7 @@ class TcpSpec extends StreamSpec("""
.via(Tcp(system2).outgoingConnection(serverAddress))
.runForeach { msg =>
probe.ref ! msg
- }(mat2)
+ }
// Ensure first that the actor is there
probe.expectMsg(testMsg)
@@ -811,12 +810,11 @@ class TcpSpec extends StreamSpec("""
"not thrown on unbind after system has been shut down" in {
val sys2 = ActorSystem("shutdown-test-system")
- val mat2 = ActorMaterializer()(sys2)
-
+ implicit val materializer = SystemMaterializer(sys2).materializer
try {
val address = temporaryServerAddress()
- val bindingFuture = Tcp().bindAndHandle(Flow[ByteString], address.getHostString, address.getPort)(mat2)
+ val bindingFuture = Tcp().bindAndHandle(Flow[ByteString], address.getHostString, address.getPort)
// Ensure server is running
bindingFuture.futureValue
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ActorRefBackpressureSinkSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ActorRefBackpressureSinkSpec.scala
index 5f83f342e8..35fdc93460 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ActorRefBackpressureSinkSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ActorRefBackpressureSinkSpec.scala
@@ -4,16 +4,19 @@
package akka.stream.scaladsl
-import scala.concurrent.duration._
-import akka.actor.{ Actor, ActorRef, Props, Status }
-import akka.stream.ActorMaterializer
+import akka.actor.Actor
+import akka.actor.ActorRef
+import akka.actor.Props
+import akka.actor.Status
import akka.stream.Attributes.inputBuffer
-import akka.stream.testkit.scaladsl.StreamTestKit._
+import akka.stream.Materializer
import akka.stream.testkit._
+import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.stream.testkit.scaladsl._
import akka.testkit.TestProbe
import scala.concurrent.Promise
+import scala.concurrent.duration._
object ActorRefBackpressureSinkSpec {
val initMessage = "start"
@@ -169,7 +172,7 @@ class ActorRefBackpressureSinkSpec extends StreamSpec {
}
"signal failure on abrupt termination" in {
- val mat = ActorMaterializer()
+ val mat = Materializer(system)
val probe = TestProbe()
val sink = Sink
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ActorRefSourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ActorRefSourceSpec.scala
index d88a79688c..7a226516c2 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ActorRefSourceSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/ActorRefSourceSpec.scala
@@ -5,25 +5,25 @@
package akka.stream.scaladsl
import akka.Done
-import akka.actor.{ PoisonPill, Status }
+import akka.actor.ActorRef
+import akka.actor.PoisonPill
+import akka.actor.Status
+import akka.stream._
import akka.stream.testkit.Utils._
import akka.stream.testkit._
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.stream.testkit.scaladsl._
-import akka.stream._
-import scala.concurrent.duration._
-
-import akka.actor.ActorRef
import org.reactivestreams.Publisher
+import scala.concurrent.duration._
+
class ActorRefSourceSpec extends StreamSpec {
"A ActorRefSource" must {
"emit received messages to the stream" in {
val s = TestSubscriber.manualProbe[Int]()
- val materializer2 = ActorMaterializer()
- val ref = Source.actorRef(10, OverflowStrategy.fail).to(Sink.fromSubscriber(s)).run()(materializer2)
+ val ref = Source.actorRef(10, OverflowStrategy.fail).to(Sink.fromSubscriber(s)).run()
val sub = s.expectSubscription()
sub.request(2)
ref ! 1
@@ -211,7 +211,7 @@ class ActorRefSourceSpec extends StreamSpec {
"be possible to run immediately, reproducer of #26714" in {
(1 to 100).foreach { _ =>
- val mat = ActorMaterializer()
+ val mat = Materializer(system)
val source: Source[String, ActorRef] = Source.actorRef[String](10000, OverflowStrategy.fail)
val (_: ActorRef, _: Publisher[String]) =
source.toMat(Sink.asPublisher(false))(Keep.both).run()(mat)
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 426646fb00..03873a0d7d 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,10 +4,10 @@
package akka.stream.scaladsl
+import akka.stream.AbruptTerminationException
+import akka.stream.Materializer
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
@@ -34,7 +34,7 @@ class CollectionSinkSpec extends StreamSpec("""
}
"fail the future on abrupt termination" in {
- val mat = ActorMaterializer()
+ val mat = Materializer(system)
val probe = TestPublisher.probe()
val future = Source.fromPublisher(probe).runWith(Sink.collection[Unit, Seq[Unit]])(mat)
mat.shutdown()
@@ -56,14 +56,6 @@ class CollectionSinkSpec extends StreamSpec("""
val result: immutable.Vector[Int] = Await.result(future, remainingOrDefault)
result should be(Vector.empty[Int])
}
-
- "fail the future on abrupt termination" in {
- val mat = ActorMaterializer()
- val probe = TestPublisher.probe()
- val future = Source.fromPublisher(probe).runWith(Sink.collection[Unit, Seq[Unit]])(mat)
- mat.shutdown()
- future.failed.futureValue shouldBe an[AbruptTerminationException]
- }
}
}
}
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLogSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLogSpec.scala
index 32c9c22052..ade733a9f7 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLogSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowLogSpec.scala
@@ -30,7 +30,7 @@ class FlowLogSpec extends StreamSpec("""
"A Log" must {
- val supervisorPath = ActorMaterializerHelper.downcast(SystemMaterializer(system).materializer).supervisor.path
+ val supervisorPath = SystemMaterializer(system).materializer.supervisor.path
val LogSrc = s"akka.stream.Log($supervisorPath)"
val LogClazz = classOf[Materializer]
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMonitorSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMonitorSpec.scala
index 381f8486ba..2320e12a34 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMonitorSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowMonitorSpec.scala
@@ -4,12 +4,12 @@
package akka.stream.scaladsl
+import akka.stream.FlowMonitorState
import akka.stream.FlowMonitorState._
+import akka.stream.Materializer
import akka.stream.testkit.StreamSpec
import akka.stream.testkit.scaladsl.TestSink
import akka.stream.testkit.scaladsl.TestSource
-import akka.stream.ActorMaterializer
-import akka.stream.FlowMonitorState
import scala.concurrent.duration._
@@ -69,7 +69,7 @@ class FlowMonitorSpec extends StreamSpec {
}
"return Failed when stream is abruptly terminated" in {
- val mat = ActorMaterializer()
+ val mat = Materializer(system)
val (_, monitor) = // notice that `monitor` is like a Keep.both
TestSource.probe[Any].monitor.to(Sink.ignore).run()(mat)
mat.shutdown()
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 208e4d0835..6c46b02425 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,15 +5,15 @@
package akka.stream.scaladsl
import akka.Done
-import akka.stream.ActorMaterializer
+import akka.stream.Materializer
import akka.stream.testkit._
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.testkit.TestProbe
import scala.concurrent.duration._
-import scala.util.control.NoStackTrace
import scala.util.Failure
import scala.util.Success
+import scala.util.control.NoStackTrace
class FlowOnCompleteSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 2
@@ -81,7 +81,7 @@ class FlowOnCompleteSpec extends StreamSpec("""
}
"yield error on abrupt termination" in {
- val mat = ActorMaterializer()
+ val mat = Materializer(system)
val onCompleteProbe = TestProbe()
val p = TestPublisher.manualProbe[Int]()
Source.fromPublisher(p).to(Sink.onComplete[Int](onCompleteProbe.ref ! _)).run()(mat)
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSpec.scala
index 5b91b440af..5d530815ac 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSpec.scala
@@ -45,10 +45,10 @@ class FlowSpec extends StreamSpec(ConfigFactory.parseString("akka.actor.debug.re
val identity: Flow[Any, Any, NotUsed] => Flow[Any, Any, NotUsed] = in => in.map(e => e)
val identity2: Flow[Any, Any, NotUsed] => Flow[Any, Any, NotUsed] = in => identity(in)
- val toPublisher: (Source[Any, _], ActorMaterializer) => Publisher[Any] =
+ val toPublisher: (Source[Any, _], Materializer) => Publisher[Any] =
(f, m) => f.runWith(Sink.asPublisher(false))(m)
- def toFanoutPublisher[In, Out](elasticity: Int): (Source[Out, _], ActorMaterializer) => Publisher[Out] =
+ def toFanoutPublisher[In, Out](elasticity: Int): (Source[Out, _], Materializer) => Publisher[Out] =
(f, m) => f.runWith(Sink.asPublisher(true).withAttributes(Attributes.inputBuffer(elasticity, elasticity)))(m)
def materializeIntoSubscriberAndPublisher[In, Out](flow: Flow[In, Out, _]): (Subscriber[In], Publisher[Out]) = {
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowWatchTerminationSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowWatchTerminationSpec.scala
index 2ead33158f..12963cf477 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowWatchTerminationSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowWatchTerminationSpec.scala
@@ -68,7 +68,7 @@ class FlowWatchTerminationSpec extends StreamSpec {
}
"fail future when stream abruptly terminated" in {
- val mat = ActorMaterializer()
+ val mat = Materializer(system)
val (_, future) = TestSource.probe[Int].watchTermination()(Keep.both).to(Sink.ignore).run()(mat)
mat.shutdown()
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowWithContextLogSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowWithContextLogSpec.scala
index 4013e1ea37..48813d72bd 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowWithContextLogSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowWithContextLogSpec.scala
@@ -24,7 +24,7 @@ class FlowWithContextLogSpec extends StreamSpec("""
"log() from FlowWithContextOps" must {
- val supervisorPath = ActorMaterializerHelper.downcast(SystemMaterializer(system).materializer).supervisor.path
+ val supervisorPath = (SystemMaterializer(system).materializer).supervisor.path
val LogSrc = s"akka.stream.Log($supervisorPath)"
val LogClazz = classOf[Materializer]
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FromMaterializationSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FromMaterializationSpec.scala
new file mode 100644
index 0000000000..d57a8e7461
--- /dev/null
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FromMaterializationSpec.scala
@@ -0,0 +1,230 @@
+/*
+ * Copyright (C) 2019 Lightbend Inc.
+ */
+
+package akka.stream.scaladsl
+
+import akka.NotUsed
+import akka.stream.testkit.StreamSpec
+
+class FromMaterializerSpec extends StreamSpec {
+
+ import system.dispatcher
+
+ "Source.fromMaterializer" should {
+
+ "expose materializer" in {
+ val source = Source.fromMaterializer { (mat, _) =>
+ Source.single(mat.isShutdown)
+ }
+
+ source.runWith(Sink.head).futureValue shouldBe false
+ }
+
+ "expose attributes" in {
+ val source = Source.fromMaterializer { (_, attr) =>
+ Source.single(attr.attributeList)
+ }
+
+ source.runWith(Sink.head).futureValue should not be empty
+ }
+
+ "propagate materialized value" in {
+ val source = Source.fromMaterializer { (_, _) =>
+ Source.maybe[NotUsed]
+ }
+
+ val (completion, element) = source.toMat(Sink.head)(Keep.both).run()
+ completion.futureValue.trySuccess(Some(NotUsed))
+ element.futureValue shouldBe NotUsed
+ }
+
+ "propagate attributes" in {
+ val source = Source
+ .fromMaterializer { (_, attr) =>
+ Source.single(attr.nameLifted)
+ }
+ .named("my-name")
+
+ source.runWith(Sink.head).futureValue shouldBe Some("my-name")
+ }
+
+ "propagate attributes when nested" in {
+ val source = Source
+ .fromMaterializer { (_, _) =>
+ Source.fromMaterializer { (_, attr) =>
+ Source.single(attr.nameLifted)
+ }
+ }
+ .named("my-name")
+
+ source.runWith(Sink.head).futureValue shouldBe Some("my-name")
+ }
+
+ "handle factory failure" in {
+ val error = new Error("boom")
+ val source = Source.fromMaterializer { (_, _) =>
+ throw error
+ }
+
+ val (materialized, completion) = source.toMat(Sink.head)(Keep.both).run()
+ materialized.failed.futureValue.getCause shouldBe error
+ completion.failed.futureValue.getCause shouldBe error
+ }
+
+ "handle materialization failure" in {
+ val error = new Error("boom")
+ val source = Source.fromMaterializer { (_, _) =>
+ Source.empty.mapMaterializedValue(_ => throw error)
+ }
+
+ val (materialized, completion) = source.toMat(Sink.head)(Keep.both).run()
+ materialized.failed.futureValue.getCause shouldBe error
+ completion.failed.futureValue.getCause shouldBe error
+ }
+
+ }
+
+ "Flow.fromMaterializer" should {
+
+ "expose materializer" in {
+ val flow = Flow.fromMaterializer { (mat, _) =>
+ Flow.fromSinkAndSource(Sink.ignore, Source.single(mat.isShutdown))
+ }
+
+ Source.empty.via(flow).runWith(Sink.head).futureValue shouldBe false
+ }
+
+ "expose attributes" in {
+ val flow = Flow.fromMaterializer { (_, attr) =>
+ Flow.fromSinkAndSource(Sink.ignore, Source.single(attr.attributeList))
+ }
+
+ Source.empty.via(flow).runWith(Sink.head).futureValue should not be empty
+ }
+
+ "propagate materialized value" in {
+ val flow = Flow.fromMaterializer { (_, _) =>
+ Flow.fromSinkAndSourceMat(Sink.ignore, Source.maybe[NotUsed])(Keep.right)
+ }
+
+ val (completion, element) = Source.empty.viaMat(flow)(Keep.right).toMat(Sink.head)(Keep.both).run()
+ completion.futureValue.trySuccess(Some(NotUsed))
+ element.futureValue shouldBe NotUsed
+ }
+
+ "propagate attributes" in {
+ val flow = Flow
+ .fromMaterializer { (_, attr) =>
+ Flow.fromSinkAndSource(Sink.ignore, Source.single(attr.nameLifted))
+ }
+ .named("my-name")
+
+ Source.empty.via(flow).runWith(Sink.head).futureValue shouldBe Some("my-name")
+ }
+
+ "propagate attributes when nested" in {
+ val flow = Flow
+ .fromMaterializer { (_, _) =>
+ Flow.fromMaterializer { (_, attr) =>
+ Flow.fromSinkAndSource(Sink.ignore, Source.single(attr.nameLifted))
+ }
+ }
+ .named("my-name")
+
+ Source.empty.via(flow).runWith(Sink.head).futureValue shouldBe Some("my-name")
+ }
+
+ "handle factory failure" in {
+ val error = new Error("boom")
+ val flow = Flow.fromMaterializer { (_, _) =>
+ throw error
+ }
+
+ val (materialized, completion) = Source.empty.viaMat(flow)(Keep.right).toMat(Sink.head)(Keep.both).run()
+ materialized.failed.futureValue.getCause shouldBe error
+ completion.failed.futureValue.getCause shouldBe error
+ }
+
+ "handle materialization failure" in {
+ val error = new Error("boom")
+ val flow = Flow.fromMaterializer { (_, _) =>
+ Flow[NotUsed].mapMaterializedValue(_ => throw error)
+ }
+
+ val (materialized, completion) = Source.empty.viaMat(flow)(Keep.right).toMat(Sink.head)(Keep.both).run()
+ materialized.failed.futureValue.getCause shouldBe error
+ completion.failed.futureValue.getCause shouldBe error
+ }
+
+ }
+
+ "Sink.fromMaterializer" should {
+
+ "expose materializer" in {
+ val sink = Sink.fromMaterializer { (mat, _) =>
+ Sink.fold(mat.isShutdown)(Keep.left)
+ }
+
+ Source.empty.runWith(sink).flatMap(identity).futureValue shouldBe false
+ }
+
+ "expose attributes" in {
+ val sink = Sink.fromMaterializer { (_, attr) =>
+ Sink.fold(attr.attributeList)(Keep.left)
+ }
+
+ Source.empty.runWith(sink).flatMap(identity).futureValue should not be empty
+ }
+
+ "propagate materialized value" in {
+ val sink = Sink.fromMaterializer { (_, _) =>
+ Sink.fold(NotUsed)(Keep.left)
+ }
+
+ Source.empty.runWith(sink).flatMap(identity).futureValue shouldBe NotUsed
+ }
+
+ "propagate attributes" in {
+ val sink = Sink
+ .fromMaterializer { (_, attr) =>
+ Sink.fold(attr.nameLifted)(Keep.left)
+ }
+ .named("my-name")
+
+ Source.empty.runWith(sink).flatMap(identity).futureValue shouldBe Some("my-name")
+ }
+
+ "propagate attributes when nested" in {
+ val sink = Sink
+ .fromMaterializer { (_, _) =>
+ Sink.fromMaterializer { (_, attr) =>
+ Sink.fold(attr.nameLifted)(Keep.left)
+ }
+ }
+ .named("my-name")
+
+ Source.empty.runWith(sink).flatMap(identity).flatMap(identity).futureValue shouldBe Some("my-name")
+ }
+
+ "handle factory failure" in {
+ val error = new Error("boom")
+ val sink = Sink.fromMaterializer { (_, _) =>
+ throw error
+ }
+
+ Source.empty.runWith(sink).failed.futureValue.getCause shouldBe error
+ }
+
+ "handle materialization failure" in {
+ val error = new Error("boom")
+ val sink = Sink.fromMaterializer { (_, _) =>
+ Sink.ignore.mapMaterializedValue(_ => throw error)
+ }
+
+ Source.empty.runWith(sink).failed.futureValue.getCause shouldBe error
+ }
+
+ }
+
+}
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 f68668121c..52bc7b08f2 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,10 +4,10 @@
package akka.stream.scaladsl
+import akka.stream.AbruptTerminationException
+import akka.stream.Materializer
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
@@ -81,7 +81,7 @@ class HeadSinkSpec extends StreamSpec("""
}
"fail on abrupt termination" in {
- val mat = ActorMaterializer()
+ val mat = Materializer(system)
val source = TestPublisher.probe()
val f = Source.fromPublisher(source).runWith(Sink.headOption)(mat)
mat.shutdown()
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/MaybeSourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/MaybeSourceSpec.scala
index 23a48971d1..90bef35106 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/MaybeSourceSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/MaybeSourceSpec.scala
@@ -4,11 +4,11 @@
package akka.stream.scaladsl
-import akka.stream.testkit.scaladsl.StreamTestKit._
+import akka.stream.AbruptStageTerminationException
+import akka.stream.Materializer
import akka.stream.testkit.StreamSpec
import akka.stream.testkit.TestSubscriber
-import akka.stream.AbruptStageTerminationException
-import akka.stream.ActorMaterializer
+import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.testkit.DefaultTimeout
import scala.concurrent.duration._
@@ -86,7 +86,7 @@ class MaybeSourceSpec extends StreamSpec with DefaultTimeout {
}
"complete materialized future when materializer is shutdown" in assertAllStagesStopped {
- val mat = ActorMaterializer()
+ val mat = Materializer(system)
val neverSource = Source.maybe[Int]
val pubSink = Sink.asPublisher[Int](false)
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/QueueSourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/QueueSourceSpec.scala
index e87dcdb038..75891ca4ab 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/QueueSourceSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/QueueSourceSpec.scala
@@ -9,9 +9,12 @@ import akka.actor.Status
import akka.pattern.pipe
import akka.stream._
import akka.stream.impl.QueueSource
+import akka.stream.testkit.GraphStageMessages
+import akka.stream.testkit.StreamSpec
+import akka.stream.testkit.TestSourceStage
+import akka.stream.testkit.TestSubscriber
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.stream.testkit.scaladsl.TestSink
-import akka.stream.testkit.{ GraphStageMessages, StreamSpec, TestSourceStage, TestSubscriber }
import akka.testkit.TestProbe
import org.scalatest.time.Span
@@ -19,7 +22,6 @@ import scala.concurrent._
import scala.concurrent.duration._
class QueueSourceSpec extends StreamSpec {
- implicit val materializer = ActorMaterializer()
implicit val ec = system.dispatcher
val pause = 300.millis
@@ -190,7 +192,7 @@ class QueueSourceSpec extends StreamSpec {
}
"complete watching future with failure if materializer shut down" in assertAllStagesStopped {
- val tempMap = ActorMaterializer()
+ val tempMap = Materializer(system)
val s = TestSubscriber.manualProbe[Int]()
val queue = Source.queue(1, OverflowStrategy.fail).to(Sink.fromSubscriber(s)).run()(tempMap)
queue.watchCompletion().pipeTo(testActor)
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 d467f698f2..8727bb0602 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,10 +4,10 @@
package akka.stream.scaladsl
+import akka.stream.AbruptTerminationException
+import akka.stream.Materializer
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
@@ -33,7 +33,7 @@ class SeqSinkSpec extends StreamSpec("""
}
"fail the future on abrupt termination" in {
- val mat = ActorMaterializer()
+ val mat = Materializer(system)
val probe = TestPublisher.probe()
val future: Future[immutable.Seq[Int]] =
Source.fromPublisher(probe).runWith(Sink.seq)(mat)
diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SetupSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SetupSpec.scala
index 9832c22690..29b98f40a4 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SetupSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SetupSpec.scala
@@ -6,7 +6,9 @@ package akka.stream.scaladsl
import akka.NotUsed
import akka.stream.testkit.StreamSpec
+import com.github.ghik.silencer.silent
+@silent("deprecated")
class SetupSpec extends StreamSpec {
import system.dispatcher
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 febf03c492..3a08666661 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
@@ -6,11 +6,10 @@ package akka.stream.scaladsl
import java.util.stream.Collectors
-import akka.actor.ActorSystem
import akka.stream._
-import akka.stream.impl.StreamSupervisor.Children
import akka.stream.impl.PhasedFusingActorMaterializer
import akka.stream.impl.StreamSupervisor
+import akka.stream.impl.StreamSupervisor.Children
import akka.stream.testkit.Utils._
import akka.stream.testkit._
import akka.stream.testkit.scaladsl.StreamTestKit._
@@ -50,37 +49,25 @@ class SinkAsJavaStreamSpec extends StreamSpec(UnboundedMailboxConfig) {
}
"allow overriding the dispatcher using Attributes" in assertAllStagesStopped {
- val sys = ActorSystem("dispatcher-testing", UnboundedMailboxConfig)
- val materializer = ActorMaterializer()(sys)
-
- try {
- TestSource
- .probe[ByteString]
- .runWith(
- StreamConverters.asJavaStream().addAttributes(ActorAttributes.dispatcher("akka.actor.default-dispatcher")))(
- materializer)
- materializer
- .asInstanceOf[PhasedFusingActorMaterializer]
- .supervisor
- .tell(StreamSupervisor.GetChildren, testActor)
- val ref = expectMsgType[Children].children.find(_.path.toString contains "asJavaStream").get
- assertDispatcher(ref, "akka.actor.default-dispatcher")
- } finally shutdown(sys)
+ val probe = TestSource
+ .probe[ByteString]
+ .to(StreamConverters.asJavaStream().addAttributes(ActorAttributes.dispatcher("akka.actor.default-dispatcher")))
+ .run()
+ SystemMaterializer(system).materializer
+ .asInstanceOf[PhasedFusingActorMaterializer]
+ .supervisor
+ .tell(StreamSupervisor.GetChildren, testActor)
+ val ref = expectMsgType[Children].children.find(_.path.toString contains "asJavaStream").get
+ assertDispatcher(ref, "akka.actor.default-dispatcher")
+ probe.sendComplete()
}
"work in separate IO dispatcher" in assertAllStagesStopped {
- val sys = ActorSystem("dispatcher-testing", UnboundedMailboxConfig)
- val materializer = ActorMaterializer()(sys)
-
- try {
- TestSource.probe[ByteString].runWith(StreamConverters.asJavaStream())(materializer)
- materializer
- .asInstanceOf[PhasedFusingActorMaterializer]
- .supervisor
- .tell(StreamSupervisor.GetChildren, testActor)
- val ref = expectMsgType[Children].children.find(_.path.toString contains "asJavaStream").get
- assertDispatcher(ref, ActorAttributes.IODispatcher.dispatcher)
- } finally shutdown(sys)
+ val materializer = Materializer.create(system)
+ TestSource.probe[ByteString].runWith(StreamConverters.asJavaStream())(materializer)
+ materializer.asInstanceOf[PhasedFusingActorMaterializer].supervisor.tell(StreamSupervisor.GetChildren, testActor)
+ val ref = expectMsgType[Children].children.find(_.path.toString contains "asJavaStream").get
+ assertDispatcher(ref, ActorAttributes.IODispatcher.dispatcher)
}
}
}
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 f70e8321c7..c14ac7bb82 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
@@ -7,19 +7,24 @@ package akka.stream.scaladsl
import java.util.concurrent.atomic.AtomicInteger
import akka.Done
-import akka.actor.ActorSystem
import akka.stream.ActorAttributes
+import akka.stream.Materializer
+import akka.stream.Supervision
import akka.stream.impl.StreamSupervisor.Children
-import akka.stream.impl.{ PhasedFusingActorMaterializer, StreamSupervisor }
+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.{ StreamSpec, TestSubscriber }
-import akka.stream.ActorMaterializer
-import akka.stream.Supervision
-import akka.testkit.{ TestLatch, TestProbe }
+import akka.stream.testkit.StreamSpec
+import akka.stream.testkit.TestSubscriber
+import akka.testkit.TestLatch
+import akka.testkit.TestProbe
import scala.concurrent.duration._
-import scala.concurrent.{ Await, ExecutionContext, Future, Promise }
+import scala.concurrent.Await
+import scala.concurrent.ExecutionContext
+import scala.concurrent.Future
+import scala.concurrent.Promise
object UnfoldResourceAsyncSourceSpec {
@@ -62,7 +67,6 @@ object UnfoldResourceAsyncSourceSpec {
class UnfoldResourceAsyncSourceSpec extends StreamSpec(UnboundedMailboxConfig) {
import UnfoldResourceAsyncSourceSpec._
-
import system.dispatcher
"Unfold Resource Async Source" must {
@@ -310,29 +314,25 @@ class UnfoldResourceAsyncSourceSpec extends StreamSpec(UnboundedMailboxConfig) {
}
"use dedicated blocking-io-dispatcher by default" in assertAllStagesStopped {
- val sys = ActorSystem("dispatcher-testing", UnboundedMailboxConfig)
- val materializer = ActorMaterializer()(sys)
- try {
- Source
- .unfoldResourceAsync[String, Unit](
- () => Promise[Unit].future, // never complete
- _ => ???,
- _ => ???)
- .runWith(Sink.ignore)(materializer)
+ // use a separate materializer to ensure we know what child is our stream
+ implicit val materializer = Materializer(system)
- materializer
- .asInstanceOf[PhasedFusingActorMaterializer]
- .supervisor
- .tell(StreamSupervisor.GetChildren, testActor)
- val ref = expectMsgType[Children].children.find(_.path.toString contains "unfoldResourceSourceAsync").get
- assertDispatcher(ref, ActorAttributes.IODispatcher.dispatcher)
- } finally shutdown(sys)
+ Source
+ .unfoldResourceAsync[String, Unit](
+ () => Promise[Unit].future, // never complete
+ _ => ???,
+ _ => ???)
+ .runWith(Sink.ignore)
+
+ materializer.asInstanceOf[PhasedFusingActorMaterializer].supervisor.tell(StreamSupervisor.GetChildren, testActor)
+ val ref = expectMsgType[Children].children.find(_.path.toString contains "unfoldResourceSourceAsync").get
+ assertDispatcher(ref, ActorAttributes.IODispatcher.dispatcher)
}
"close resource when stream is abruptly terminated" in {
import system.dispatcher
val closeLatch = TestLatch(1)
- val mat = ActorMaterializer()
+ val mat = Materializer(system)
val p = Source
.unfoldResourceAsync[String, Unit](
() => Future.successful(()),
diff --git a/akka-stream-tests/src/test/scala/akka/stream/snapshot/MaterializerStateSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/snapshot/MaterializerStateSpec.scala
index 0f4ce7723a..61d66d7128 100644
--- a/akka-stream-tests/src/test/scala/akka/stream/snapshot/MaterializerStateSpec.scala
+++ b/akka-stream-tests/src/test/scala/akka/stream/snapshot/MaterializerStateSpec.scala
@@ -4,10 +4,17 @@
package akka.stream.snapshot
-import akka.stream.{ ActorMaterializer, FlowShape }
-import akka.stream.scaladsl.{ Flow, GraphDSL, Keep, Merge, Partition, Sink, Source }
-import akka.stream.testkit.scaladsl.TestSink
+import akka.stream.FlowShape
+import akka.stream.Materializer
+import akka.stream.scaladsl.Flow
+import akka.stream.scaladsl.GraphDSL
+import akka.stream.scaladsl.Keep
+import akka.stream.scaladsl.Merge
+import akka.stream.scaladsl.Partition
+import akka.stream.scaladsl.Sink
+import akka.stream.scaladsl.Source
import akka.stream.testkit.StreamSpec
+import akka.stream.testkit.scaladsl.TestSink
import scala.concurrent.Promise
@@ -16,7 +23,7 @@ class MaterializerStateSpec extends StreamSpec {
"The MaterializerSnapshotting" must {
"snapshot a running stream" in {
- implicit val mat = ActorMaterializer()
+ implicit val mat = Materializer(system)
try {
Source.maybe[Int].map(_.toString).zipWithIndex.runWith(Sink.seq)
@@ -47,7 +54,7 @@ class MaterializerStateSpec extends StreamSpec {
}
"snapshot a stream that has a stopped stage" in {
- implicit val mat = ActorMaterializer()
+ implicit val mat = Materializer(system)
try {
val probe = TestSink.probe[String](system)
val out = Source
@@ -68,7 +75,7 @@ class MaterializerStateSpec extends StreamSpec {
}
"snapshot a more complicated graph" in {
- implicit val mat = ActorMaterializer()
+ implicit val mat = Materializer(system)
try {
// snapshot before anything is running
MaterializerState.streamSnapshots(mat).futureValue
diff --git a/akka-stream-typed/src/main/scala/akka/stream/typed/javadsl/ActorMaterializerFactory.scala b/akka-stream-typed/src/main/scala/akka/stream/typed/javadsl/ActorMaterializerFactory.scala
deleted file mode 100644
index 42b9f1dadc..0000000000
--- a/akka-stream-typed/src/main/scala/akka/stream/typed/javadsl/ActorMaterializerFactory.scala
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * Copyright (C) 2018-2019 Lightbend Inc.
- */
-
-package akka.stream.typed.javadsl
-
-import akka.actor.typed.ActorSystem
-import akka.actor.typed.javadsl.{ ActorContext, Adapter }
-import akka.stream.ActorMaterializerSettings
-
-object ActorMaterializerFactory {
- import akka.actor.typed.scaladsl.adapter._
-
- /**
- * Creates an `ActorMaterializer` which will execute every step of a transformation
- * pipeline within its own [[akka.actor.Actor]]. The required [[akka.actor.typed.ActorSystem]]
- * will be used to create these actors, therefore it is *forbidden* to pass this object
- * to another actor if the factory is an ActorContext.
- *
- * Defaults the actor name prefix used to name actors running the processing steps to `"flow"`.
- * The actor names are built up of `namePrefix-flowNumber-flowStepNumber-stepName`.
- */
- def create[T](actorSystem: ActorSystem[T]): akka.stream.ActorMaterializer =
- akka.stream.ActorMaterializer.create(actorSystem.toClassic)
-
- /**
- * Creates an `ActorMaterializer` which will execute every step of a transformation
- * pipeline within its own [[akka.actor.Actor]]. The required [[akka.actor.typed.ActorSystem]]
- * will be used to create one actor that in turn creates actors for the transformation steps.
- */
- def create[T](settings: ActorMaterializerSettings, actorSystem: ActorSystem[T]): akka.stream.ActorMaterializer =
- akka.stream.ActorMaterializer.create(settings, actorSystem.toClassic)
-
- /**
- * Creates an `ActorMaterializer` which will execute every step of a transformation
- * pipeline within its own [[akka.actor.Actor]]. The required [[akka.actor.typed.ActorSystem]]
- * will be used to create these actors, therefore it is *forbidden* to pass this object
- * to another actor if the factory is an ActorContext.
- *
- * The `namePrefix` is used as the first part of the names of the actors running
- * the processing steps. The default `namePrefix` is `"flow"`. The actor names are built up of
- * `namePrefix-flowNumber-flowStepNumber-stepName`.
- */
- def create[T](
- settings: ActorMaterializerSettings,
- namePrefix: String,
- actorSystem: ActorSystem[T]): akka.stream.ActorMaterializer =
- akka.stream.ActorMaterializer.create(settings, actorSystem.toClassic, namePrefix)
-
- /**
- * Creates an `ActorMaterializer` which will execute every step of a transformation
- * pipeline within its own [[akka.actor.Actor]]. The lifecycle of the materialized streams
- * will be bound to the lifecycle of the provided [[akka.actor.typed.javadsl.ActorContext]]
- *
- * Defaults the actor name prefix used to name actors running the processing steps to `"flow"`.
- * The actor names are built up of `namePrefix-flowNumber-flowStepNumber-stepName`.
- */
- def create[T](ctx: ActorContext[T]): akka.stream.ActorMaterializer =
- akka.stream.ActorMaterializer.create(Adapter.toClassic(ctx))
-
- /**
- * Creates an `ActorMaterializer` which will execute every step of a transformation
- * pipeline within its own [[akka.actor.Actor]]. The lifecycle of the materialized streams
- * will be bound to the lifecycle of the provided [[akka.actor.typed.javadsl.ActorContext]]
- */
- def create[T](settings: ActorMaterializerSettings, ctx: ActorContext[T]): akka.stream.ActorMaterializer =
- akka.stream.ActorMaterializer.create(settings, Adapter.toClassic(ctx))
-
- /**
- * Creates an `ActorMaterializer` which will execute every step of a transformation
- * pipeline within its own [[akka.actor.Actor]]. The lifecycle of the materialized streams
- * will be bound to the lifecycle of the provided [[akka.actor.typed.javadsl.ActorContext]]
- *
- * The `namePrefix` is used as the first part of the names of the actors running
- * the processing steps. The default `namePrefix` is `"flow"`. The actor names are built up of
- * `namePrefix-flowNumber-flowStepNumber-stepName`.
- */
- def create[T](
- settings: ActorMaterializerSettings,
- namePrefix: String,
- ctx: ActorContext[T]): akka.stream.ActorMaterializer =
- akka.stream.ActorMaterializer.create(settings, Adapter.toClassic(ctx), namePrefix)
-}
diff --git a/akka-stream-typed/src/main/scala/akka/stream/typed/package.scala b/akka-stream-typed/src/main/scala/akka/stream/typed/package.scala
deleted file mode 100644
index 0690a9c776..0000000000
--- a/akka-stream-typed/src/main/scala/akka/stream/typed/package.scala
+++ /dev/null
@@ -1,9 +0,0 @@
-/*
- * Copyright (C) 2018-2019 Lightbend Inc.
- */
-
-package akka.stream.typed
-
-package object scaladsl {
- type ActorMaterializer = akka.stream.ActorMaterializer
-}
diff --git a/akka-stream-typed/src/main/scala/akka/stream/typed/scaladsl/ActorMaterializer.scala b/akka-stream-typed/src/main/scala/akka/stream/typed/scaladsl/ActorMaterializer.scala
deleted file mode 100644
index cd5326beeb..0000000000
--- a/akka-stream-typed/src/main/scala/akka/stream/typed/scaladsl/ActorMaterializer.scala
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Copyright (C) 2018-2019 Lightbend Inc.
- */
-
-package akka.stream.typed.scaladsl
-
-import akka.actor.typed.ActorSystem
-import akka.actor.typed.scaladsl.ActorContext
-import akka.stream.ActorMaterializerSettings
-
-object ActorMaterializer {
- import akka.actor.typed.scaladsl.adapter._
-
- /**
- * Creates an `ActorMaterializer` which will execute every step of a transformation
- * pipeline within its own [[akka.actor.Actor]]. The required [[akka.actor.typed.ActorSystem]]
- * will be used to create one actor that in turn creates actors for the transformation steps.
- *
- * The materializer's [[akka.stream.ActorMaterializerSettings]] will be obtained from the
- * configuration of the `context`'s underlying [[akka.actor.typed.ActorSystem]].
- *
- * The `namePrefix` is used as the first part of the names of the actors running
- * the processing steps. The default `namePrefix` is `"flow"`. The actor names are built up of
- * `namePrefix-flowNumber-flowStepNumber-stepName`.
- */
- def apply[T](materializerSettings: Option[ActorMaterializerSettings] = None, namePrefix: Option[String] = None)(
- implicit actorSystem: ActorSystem[T]): ActorMaterializer =
- akka.stream.ActorMaterializer(materializerSettings, namePrefix)(actorSystem.toClassic)
-
- /**
- * Creates an `ActorMaterializer` which will execute every step of a transformation
- * pipeline within its own [[akka.actor.Actor]]. The lifecycle of the materialized streams
- * will be bound to the lifecycle of the provided [[akka.actor.typed.scaladsl.ActorContext]]
- *
- * The materializer's [[akka.stream.ActorMaterializerSettings]] will be obtained from the
- * configuration of the `context`'s underlying [[akka.actor.typed.ActorSystem]].
- *
- * The `namePrefix` is used as the first part of the names of the actors running
- * the processing steps. The default `namePrefix` is `"flow"`. The actor names are built up of
- * `namePrefix-flowNumber-flowStepNumber-stepName`.
- */
- def boundToActor[T](
- ctx: ActorContext[T],
- materializerSettings: Option[ActorMaterializerSettings] = None,
- namePrefix: Option[String] = None): ActorMaterializer =
- akka.stream.ActorMaterializer(materializerSettings, namePrefix)(ctx.toClassic)
-
-}
diff --git a/akka-stream-typed/src/test/java/akka/stream/typed/javadsl/ActorFlowCompileTest.java b/akka-stream-typed/src/test/java/akka/stream/typed/javadsl/ActorFlowCompileTest.java
index 5f2839fa26..d859141915 100644
--- a/akka-stream-typed/src/test/java/akka/stream/typed/javadsl/ActorFlowCompileTest.java
+++ b/akka-stream-typed/src/test/java/akka/stream/typed/javadsl/ActorFlowCompileTest.java
@@ -29,7 +29,6 @@ public class ActorFlowCompileTest {
{
final ActorSystem system = null;
- final ActorMaterializer mat = ActorMaterializerFactory.create(system);
}
static
diff --git a/akka-stream-typed/src/test/java/akka/stream/typed/javadsl/ActorSourceSinkCompileTest.java b/akka-stream-typed/src/test/java/akka/stream/typed/javadsl/ActorSourceSinkCompileTest.java
index e1d7138940..260770c677 100644
--- a/akka-stream-typed/src/test/java/akka/stream/typed/javadsl/ActorSourceSinkCompileTest.java
+++ b/akka-stream-typed/src/test/java/akka/stream/typed/javadsl/ActorSourceSinkCompileTest.java
@@ -30,7 +30,6 @@ public class ActorSourceSinkCompileTest {
{
final ActorSystem system = null;
- final ActorMaterializer mat = ActorMaterializerFactory.create(system);
}
{
diff --git a/akka-stream-typed/src/test/java/akka/stream/typed/javadsl/CustomGuardianAndMaterializerTest.java b/akka-stream-typed/src/test/java/akka/stream/typed/javadsl/CustomGuardianAndMaterializerTest.java
new file mode 100644
index 0000000000..d6152fb2f4
--- /dev/null
+++ b/akka-stream-typed/src/test/java/akka/stream/typed/javadsl/CustomGuardianAndMaterializerTest.java
@@ -0,0 +1,76 @@
+/*
+ * Copyright (C) 2009-2019 Lightbend Inc.
+ */
+
+package akka.stream.typed.javadsl;
+
+import akka.Done;
+import akka.actor.AbstractActor;
+import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
+import akka.actor.testkit.typed.javadsl.TestProbe;
+import akka.actor.typed.ActorRef;
+import akka.actor.typed.Behavior;
+import akka.actor.typed.TimerSpec;
+import akka.actor.typed.javadsl.Behaviors;
+import akka.stream.AbruptStageTerminationException;
+import akka.stream.AbruptTerminationException;
+import akka.stream.Materializer;
+import akka.stream.javadsl.Sink;
+import akka.stream.javadsl.Source;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.scalatest.junit.JUnitSuite;
+
+import java.util.concurrent.CompletionStage;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertEquals;
+
+public class CustomGuardianAndMaterializerTest extends JUnitSuite {
+
+ @ClassRule public static final TestKitJunitResource testKit = new TestKitJunitResource();
+
+ @Test
+ public void useSystemWideMaterialiser() throws Exception {
+ CompletionStage result = Source.single("hello").runWith(Sink.head(), testKit.system());
+
+ assertEquals("hello", result.toCompletableFuture().get(3, TimeUnit.SECONDS));
+ }
+
+ @Test
+ public void createCustomSystemLevelMaterialiser() throws Exception {
+ Materializer materializer = Materializer.create(testKit.system());
+
+ CompletionStage result = Source.single("hello").runWith(Sink.head(), materializer);
+
+ assertEquals("hello", result.toCompletableFuture().get(3, TimeUnit.SECONDS));
+ }
+
+ private static Behavior actorStreamBehavior(ActorRef