Deprecate actor materializer (#27538)

This commit is contained in:
Johan Andrén 2019-09-05 16:08:37 +02:00 committed by GitHub
parent 8765a4fbe6
commit b68d67008a
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
118 changed files with 1233 additions and 731 deletions

View file

@ -182,6 +182,10 @@ private[akka] final class FunctionRef[-T](override val path: ActorPath, send: (T
private val loggingAdapter = new StubbedLogger private val loggingAdapter = new StubbedLogger
private var unhandled: List[T] = Nil 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) override def children: Iterable[ActorRef[Nothing]] = _children.values.map(_.context.self)
def childrenNames: Iterable[String] = _children.keys def childrenNames: Iterable[String] = _children.keys

View file

@ -63,6 +63,7 @@ private[akka] object ActorContextAdapter {
final override val self = ActorRefAdapter(classicContext.self) final override val self = ActorRefAdapter(classicContext.self)
final override val system = ActorSystemAdapter(classicContext.system) final override val system = ActorSystemAdapter(classicContext.system)
private[akka] def classicActorContext = classicContext
override def children: Iterable[ActorRef[Nothing]] = classicContext.children.map(ActorRefAdapter(_)) 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 child(name: String): Option[ActorRef[Nothing]] = classicContext.child(name).map(ActorRefAdapter(_))
override def spawnAnonymous[U](behavior: Behavior[U], props: Props = Props.empty): ActorRef[U] = override def spawnAnonymous[U](behavior: Behavior[U], props: Props = Props.empty): ActorRef[U] =

View file

@ -12,6 +12,8 @@ import akka.actor.typed._
import java.util.Optional import java.util.Optional
import java.util.concurrent.CompletionStage import java.util.concurrent.CompletionStage
import akka.actor.ClassicActorContextProvider
import scala.concurrent.ExecutionContextExecutor import scala.concurrent.ExecutionContextExecutor
/** /**
@ -35,7 +37,7 @@ import scala.concurrent.ExecutionContextExecutor
* Not for user extension. * Not for user extension.
*/ */
@DoNotInherit @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 // this must be a pure interface, i.e. only abstract methods
/** /**

View file

@ -4,6 +4,7 @@
package akka.actor.typed.scaladsl package akka.actor.typed.scaladsl
import akka.actor.ClassicActorContextProvider
import akka.actor.typed._ import akka.actor.typed._
import akka.annotation.DoNotInherit import akka.annotation.DoNotInherit
import akka.util.Timeout import akka.util.Timeout
@ -35,7 +36,7 @@ import akka.annotation.InternalApi
* Not for user extension. * Not for user extension.
*/ */
@DoNotInherit @DoNotInherit
trait ActorContext[T] extends TypedActorContext[T] { trait ActorContext[T] extends TypedActorContext[T] with ClassicActorContextProvider {
/** /**
* Get the `javadsl` of this `ActorContext`. * Get the `javadsl` of this `ActorContext`.

View file

@ -85,4 +85,6 @@ ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.actor.DynamicAccess.c
# system wide materializer #25559 # system wide materializer #25559
ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("akka.actor.ExtendedActorSystem.classicSystem") ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("akka.actor.ExtendedActorSystem.classicSystem")
ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("akka.actor.ActorSystem.classicSystem") ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("akka.actor.ActorSystem.classicSystem")
ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("akka.actor.AbstractActor#ActorContext.classicActorContext")
ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("akka.actor.ActorContext.classicActorContext")

View file

@ -44,7 +44,7 @@ import com.github.ghik.silencer.silent
* *
* Where no name is given explicitly, one will be automatically generated. * 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 * The ActorRef representing this actor
@ -428,6 +428,8 @@ private[akka] class ActorCell(
protected final def lookupRoot = self protected final def lookupRoot = self
final def provider = system.provider final def provider = system.provider
override final def classicActorContext: ActorContext = this
protected def uid: Int = self.path.uid protected def uid: Int = self.path.uid
private[this] var _actor: Actor = _ private[this] var _actor: Actor = _
def actor: Actor = _actor def actor: Actor = _actor

View file

@ -19,3 +19,16 @@ trait ClassicActorSystemProvider {
@InternalApi @InternalApi
private[akka] def classicSystem: ActorSystem 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
}

View file

@ -86,7 +86,7 @@ class AskBenchmark {
private def awaitLatch(latch: CountDownLatch): Unit = { private def awaitLatch(latch: CountDownLatch): Unit = {
if (!latch.await(30, TimeUnit.SECONDS)) { 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") throw new RuntimeException("Latch didn't complete in time")
} }
} }

View file

@ -100,7 +100,7 @@ class FlatMapConcatBenchmark {
private def awaitLatch(latch: CountDownLatch): Unit = { private def awaitLatch(latch: CountDownLatch): Unit = {
if (!latch.await(30, TimeUnit.SECONDS)) { if (!latch.await(30, TimeUnit.SECONDS)) {
implicit val ec = system.dispatcher 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") throw new RuntimeException("Latch didn't complete in time")
} }
} }

View file

@ -4,14 +4,18 @@
package akka.stream package akka.stream
import java.util.concurrent.{ Semaphore, TimeUnit } import java.util.concurrent.Semaphore
import java.util.concurrent.TimeUnit
import akka.NotUsed import akka.NotUsed
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.remote.artery.BenchTestSourceSameElement 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 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 org.openjdk.jmh.annotations._
import scala.concurrent.Await import scala.concurrent.Await
@ -49,8 +53,6 @@ class FramingBenchmark {
implicit val system: ActorSystem = ActorSystem("test", config) 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 // Safe to be benchmark scoped because the flows we construct in this bench are stateless
var flow: Source[ByteString, NotUsed] = _ var flow: Source[ByteString, NotUsed] = _
@ -62,7 +64,7 @@ class FramingBenchmark {
@Setup @Setup
def setup(): Unit = { def setup(): Unit = {
materializer = ActorMaterializer() SystemMaterializer(system).materializer
val frame = List.range(0, messageSize, 1).map(_ => Random.nextPrintableChar()).mkString + "\n" val frame = List.range(0, messageSize, 1).map(_ => Random.nextPrintableChar()).mkString + "\n"
val messageChunk = ByteString(List.range(0, framePerSeq, 1).map(_ => frame).mkString) val messageChunk = ByteString(List.range(0, framePerSeq, 1).map(_ => frame).mkString)
@ -82,7 +84,7 @@ class FramingBenchmark {
def framing(): Unit = { def framing(): Unit = {
val lock = new Semaphore(1) val lock = new Semaphore(1)
lock.acquire() lock.acquire()
flow.runWith(Sink.onComplete(_ => lock.release()))(materializer) flow.runWith(Sink.onComplete(_ => lock.release()))
lock.acquire() lock.acquire()
} }

View file

@ -101,7 +101,6 @@ class FusedGraphsBenchmark {
ConfigFactory.parseString(s""" ConfigFactory.parseString(s"""
akka.stream.materializer.sync-processing-limit = ${Int.MaxValue} akka.stream.materializer.sync-processing-limit = ${Int.MaxValue}
""")) """))
var materializer: ActorMaterializer = _
var testElements: Array[MutableElement] = _ var testElements: Array[MutableElement] = _
var singleIdentity: RunnableGraph[CountDownLatch] = _ var singleIdentity: RunnableGraph[CountDownLatch] = _
@ -225,70 +224,70 @@ class FusedGraphsBenchmark {
@OperationsPerInvocation(100 * 1000) @OperationsPerInvocation(100 * 1000)
def single_identity(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = { def single_identity(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = {
FusedGraphsBenchmark.blackhole = blackhole FusedGraphsBenchmark.blackhole = blackhole
singleIdentity.run()(materializer).await() singleIdentity.run().await()
} }
@Benchmark @Benchmark
@OperationsPerInvocation(100 * 1000) @OperationsPerInvocation(100 * 1000)
def chain_of_identities(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = { def chain_of_identities(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = {
FusedGraphsBenchmark.blackhole = blackhole FusedGraphsBenchmark.blackhole = blackhole
chainOfIdentities.run()(materializer).await() chainOfIdentities.run().await()
} }
@Benchmark @Benchmark
@OperationsPerInvocation(100 * 1000) @OperationsPerInvocation(100 * 1000)
def single_map(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = { def single_map(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = {
FusedGraphsBenchmark.blackhole = blackhole FusedGraphsBenchmark.blackhole = blackhole
singleMap.run()(materializer).await() singleMap.run().await()
} }
@Benchmark @Benchmark
@OperationsPerInvocation(100 * 1000) @OperationsPerInvocation(100 * 1000)
def chain_of_maps(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = { def chain_of_maps(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = {
FusedGraphsBenchmark.blackhole = blackhole FusedGraphsBenchmark.blackhole = blackhole
chainOfMaps.run()(materializer).await() chainOfMaps.run().await()
} }
@Benchmark @Benchmark
@OperationsPerInvocation(100 * 1000) @OperationsPerInvocation(100 * 1000)
def repeat_take_map_and_fold(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = { def repeat_take_map_and_fold(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = {
FusedGraphsBenchmark.blackhole = blackhole FusedGraphsBenchmark.blackhole = blackhole
repeatTakeMapAndFold.run()(materializer).await() repeatTakeMapAndFold.run().await()
} }
@Benchmark @Benchmark
@OperationsPerInvocation(100 * 1000) @OperationsPerInvocation(100 * 1000)
def single_buffer(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = { def single_buffer(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = {
FusedGraphsBenchmark.blackhole = blackhole FusedGraphsBenchmark.blackhole = blackhole
singleBuffer.run()(materializer).await() singleBuffer.run().await()
} }
@Benchmark @Benchmark
@OperationsPerInvocation(100 * 1000) @OperationsPerInvocation(100 * 1000)
def chain_of_buffers(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = { def chain_of_buffers(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = {
FusedGraphsBenchmark.blackhole = blackhole FusedGraphsBenchmark.blackhole = blackhole
chainOfBuffers.run()(materializer).await() chainOfBuffers.run().await()
} }
@Benchmark @Benchmark
@OperationsPerInvocation(100 * 1000) @OperationsPerInvocation(100 * 1000)
def broadcast_zip(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = { def broadcast_zip(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = {
FusedGraphsBenchmark.blackhole = blackhole FusedGraphsBenchmark.blackhole = blackhole
broadcastZip.run()(materializer).await() broadcastZip.run().await()
} }
@Benchmark @Benchmark
@OperationsPerInvocation(100 * 1000) @OperationsPerInvocation(100 * 1000)
def balance_merge(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = { def balance_merge(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = {
FusedGraphsBenchmark.blackhole = blackhole FusedGraphsBenchmark.blackhole = blackhole
balanceMerge.run()(materializer).await() balanceMerge.run().await()
} }
@Benchmark @Benchmark
@OperationsPerInvocation(100 * 1000) @OperationsPerInvocation(100 * 1000)
def broadcast_zip_balance_merge(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = { def broadcast_zip_balance_merge(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = {
FusedGraphsBenchmark.blackhole = blackhole FusedGraphsBenchmark.blackhole = blackhole
broadcastZipBalanceMerge.run()(materializer).await() broadcastZipBalanceMerge.run().await()
} }
@TearDown @TearDown

View file

@ -4,18 +4,21 @@
package akka.stream package akka.stream
import java.util.concurrent.{ CountDownLatch, TimeUnit } import java.util.concurrent.CountDownLatch
import java.util.concurrent.TimeUnit
import akka.NotUsed import akka.NotUsed
import akka.actor.ActorSystem 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.scaladsl._
import akka.stream.testkit.scaladsl.StreamTestKit import akka.stream.testkit.scaladsl.StreamTestKit
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import org.openjdk.jmh.annotations._ import org.openjdk.jmh.annotations._
import scala.concurrent.{ Await, Future }
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.concurrent.Await
import scala.concurrent.Future
object MapAsyncBenchmark { object MapAsyncBenchmark {
final val OperationsPerInvocation = 100000 final val OperationsPerInvocation = 100000
@ -85,7 +88,7 @@ class MapAsyncBenchmark {
private def awaitLatch(latch: CountDownLatch): Unit = { private def awaitLatch(latch: CountDownLatch): Unit = {
if (!latch.await(30, TimeUnit.SECONDS)) { 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") throw new RuntimeException("Latch didn't complete in time")
} }
} }

View file

@ -5,14 +5,16 @@
package akka.stream package akka.stream
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
import akka.Done
import akka.NotUsed import akka.NotUsed
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import org.openjdk.jmh.annotations._ import org.openjdk.jmh.annotations._
import scala.concurrent.Await import scala.concurrent.Await
import scala.concurrent.duration._
import scala.concurrent.Future import scala.concurrent.Future
import akka.Done import scala.concurrent.duration._
object MaterializationBenchmark { object MaterializationBenchmark {
@ -95,7 +97,6 @@ class MaterializationBenchmark {
import MaterializationBenchmark._ import MaterializationBenchmark._
implicit val system = ActorSystem("MaterializationBenchmark") implicit val system = ActorSystem("MaterializationBenchmark")
implicit val materializer = ActorMaterializer()
var flowWithMap: RunnableGraph[NotUsed] = _ var flowWithMap: RunnableGraph[NotUsed] = _
var graphWithJunctionsGradual: RunnableGraph[NotUsed] = _ var graphWithJunctionsGradual: RunnableGraph[NotUsed] = _

View file

@ -4,12 +4,16 @@
package akka.stream package akka.stream
import java.util.concurrent.{ CountDownLatch, TimeUnit } import java.util.concurrent.CountDownLatch
import java.util.concurrent.TimeUnit
import akka.NotUsed import akka.NotUsed
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.remote.artery.{ BenchTestSource, FixedSizePartitionHub, LatchSink } import akka.remote.artery.BenchTestSource
import akka.stream.scaladsl.{ PartitionHub, _ } 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 akka.stream.testkit.scaladsl.StreamTestKit
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import org.openjdk.jmh.annotations._ import org.openjdk.jmh.annotations._
@ -100,7 +104,7 @@ class PartitionHubBenchmark {
private def dumpMaterializer(): Unit = { private def dumpMaterializer(): Unit = {
implicit val ec = system.dispatcher implicit val ec = system.dispatcher
StreamTestKit.printDebugDump(ActorMaterializerHelper.downcast(SystemMaterializer(system).materializer).supervisor) StreamTestKit.printDebugDump(SystemMaterializer(system).materializer.supervisor)
} }
} }

View file

@ -4,7 +4,8 @@
package akka.stream 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.actor.ActorSystem
import akka.remote.artery.BenchTestSource import akka.remote.artery.BenchTestSource
@ -34,8 +35,6 @@ class SourceRefBenchmark {
implicit val system = ActorSystem("test", config) implicit val system = ActorSystem("test", config)
implicit val materializer: ActorMaterializer = ActorMaterializer()
final val successMarker = Success(1) final val successMarker = Success(1)
final val successFailure = Success(new Exception) final val successFailure = Success(new Exception)

View file

@ -6,17 +6,16 @@ package akka.stream.impl
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
import scala.concurrent.Await
import scala.concurrent.duration._
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.stream.ActorMaterializer
import akka.stream.scaladsl.Keep import akka.stream.scaladsl.Keep
import akka.stream.scaladsl.Sink import akka.stream.scaladsl.Sink
import akka.stream.scaladsl.StreamConverters import akka.stream.scaladsl.StreamConverters
import org.openjdk.jmh.annotations.TearDown import org.openjdk.jmh.annotations.TearDown
import org.openjdk.jmh.annotations._ import org.openjdk.jmh.annotations._
import scala.concurrent.Await
import scala.concurrent.duration._
object OutputStreamSourceStageBenchmark { object OutputStreamSourceStageBenchmark {
final val WritesPerBench = 10000 final val WritesPerBench = 10000
} }
@ -26,7 +25,6 @@ object OutputStreamSourceStageBenchmark {
class OutputStreamSourceStageBenchmark { class OutputStreamSourceStageBenchmark {
import OutputStreamSourceStageBenchmark.WritesPerBench import OutputStreamSourceStageBenchmark.WritesPerBench
implicit val system = ActorSystem("OutputStreamSourceStageBenchmark") implicit val system = ActorSystem("OutputStreamSourceStageBenchmark")
implicit val materializer = ActorMaterializer()
private val bytes: Array[Byte] = Array.emptyByteArray private val bytes: Array[Byte] = Array.emptyByteArray

View file

@ -4,17 +4,23 @@
package akka.stream.io 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 java.util.concurrent.TimeUnit
import akka.{ Done, NotUsed }
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.stream.{ ActorMaterializer, Attributes } import akka.stream.Attributes
import akka.stream.IOResult
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.util.ByteString import akka.util.ByteString
import akka.Done
import akka.NotUsed
import org.openjdk.jmh.annotations._ import org.openjdk.jmh.annotations._
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.concurrent.{ Await, Future, Promise } import scala.concurrent.Await
import akka.stream.IOResult import scala.concurrent.Future
import scala.concurrent.Promise
/** /**
* Benchmark (bufSize) Mode Cnt Score Error Units * Benchmark (bufSize) Mode Cnt Score Error Units
@ -26,7 +32,6 @@ import akka.stream.IOResult
class FileSourcesBenchmark { class FileSourcesBenchmark {
implicit val system = ActorSystem("file-sources-benchmark") implicit val system = ActorSystem("file-sources-benchmark")
implicit val materializer = ActorMaterializer()
val file: Path = { val file: Path = {
val line = ByteString("x" * 2048 + "\n") val line = ByteString("x" * 2048 + "\n")

View file

@ -4,17 +4,22 @@
package akka.stream.io 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 java.util.concurrent.TimeUnit
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.stream.IOResult
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.stream.{ ActorMaterializer, IOResult }
import akka.util.ByteString 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.duration._
import scala.concurrent.{ Await, Future }
@State(Scope.Benchmark) @State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.AverageTime)) @BenchmarkMode(Array(Mode.AverageTime))
@ -30,7 +35,7 @@ class FileSourcesScaleBenchmark {
* FileSourcesScaleBenchmark.mapAsync 2048 avgt 10 0.899 ± 0.103 s/op * FileSourcesScaleBenchmark.mapAsync 2048 avgt 10 0.899 ± 0.103 s/op
*/ */
implicit val system = ActorSystem("file-sources-benchmark") implicit val system = ActorSystem("file-sources-benchmark")
implicit val materializer = ActorMaterializer()
val FILES_NUMBER = 40 val FILES_NUMBER = 40
val files: Seq[Path] = { val files: Seq[Path] = {
val line = ByteString("x" * 2048 + "\n") val line = ByteString("x" * 2048 + "\n")

View file

@ -4,11 +4,6 @@
package akka.cluster package akka.cluster
import scala.concurrent.Future
import scala.concurrent.duration._
import scala.util.Failure
import scala.util.Success
import akka.Done import akka.Done
import akka.actor.Actor import akka.actor.Actor
import akka.actor.ActorIdentity import akka.actor.ActorIdentity
@ -19,7 +14,7 @@ import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec import akka.remote.testkit.MultiNodeSpec
import akka.remote.transport.ThrottlerTransportAdapter.Direction import akka.remote.transport.ThrottlerTransportAdapter.Direction
import akka.serialization.jackson.CborSerializable import akka.serialization.jackson.CborSerializable
import akka.stream.ActorMaterializer import akka.stream.Materializer
import akka.stream.RemoteStreamRefActorTerminatedException import akka.stream.RemoteStreamRefActorTerminatedException
import akka.stream.SinkRef import akka.stream.SinkRef
import akka.stream.SourceRef import akka.stream.SourceRef
@ -32,6 +27,11 @@ import akka.stream.testkit.scaladsl.TestSink
import akka.testkit._ import akka.testkit._
import com.typesafe.config.ConfigFactory 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 { object StreamRefSpec extends MultiNodeConfig {
val first = role("first") val first = role("first")
val second = role("second") val second = role("second")
@ -54,7 +54,7 @@ object StreamRefSpec extends MultiNodeConfig {
class DataSource(streamLifecycleProbe: ActorRef) extends Actor { class DataSource(streamLifecycleProbe: ActorRef) extends Actor {
import context.dispatcher import context.dispatcher
implicit val mat = ActorMaterializer()(context) implicit val mat = Materializer(context)
def receive = { def receive = {
case RequestLogs(streamId) => case RequestLogs(streamId) =>
@ -97,7 +97,7 @@ object StreamRefSpec extends MultiNodeConfig {
class DataReceiver(streamLifecycleProbe: ActorRef) extends Actor { class DataReceiver(streamLifecycleProbe: ActorRef) extends Actor {
import context.dispatcher import context.dispatcher
implicit val mat = ActorMaterializer()(context) implicit val mat = Materializer(context)
def receive = { def receive = {
case PrepareUpload(nodeId) => case PrepareUpload(nodeId) =>
@ -136,8 +136,6 @@ class StreamRefMultiJvmNode3 extends StreamRefSpec
abstract class StreamRefSpec extends MultiNodeSpec(StreamRefSpec) with MultiNodeClusterSpec with ImplicitSender { abstract class StreamRefSpec extends MultiNodeSpec(StreamRefSpec) with MultiNodeClusterSpec with ImplicitSender {
import StreamRefSpec._ import StreamRefSpec._
private implicit val mat: ActorMaterializer = ActorMaterializer()
"A cluster with Stream Refs" must { "A cluster with Stream Refs" must {
"join" taggedAs LongRunningTest in { "join" taggedAs LongRunningTest in {

View file

@ -6,9 +6,6 @@ package akka.cluster
import java.lang.management.ManagementFactory import java.lang.management.ManagementFactory
import scala.concurrent.Await
import scala.concurrent.duration._
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.actor.Address import akka.actor.Address
import akka.actor.CoordinatedShutdown import akka.actor.CoordinatedShutdown
@ -17,7 +14,7 @@ import akka.actor.Props
import akka.cluster.ClusterEvent.MemberEvent import akka.cluster.ClusterEvent.MemberEvent
import akka.cluster.ClusterEvent._ import akka.cluster.ClusterEvent._
import akka.cluster.InternalClusterAction._ import akka.cluster.InternalClusterAction._
import akka.stream.ActorMaterializer import akka.stream.Materializer
import akka.stream.scaladsl.Sink import akka.stream.scaladsl.Sink
import akka.stream.scaladsl.Source import akka.stream.scaladsl.Source
import akka.stream.scaladsl.StreamRefs import akka.stream.scaladsl.StreamRefs
@ -27,6 +24,9 @@ import akka.testkit.TestProbe
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import javax.management.ObjectName import javax.management.ObjectName
import scala.concurrent.Await
import scala.concurrent.duration._
object ClusterSpec { object ClusterSpec {
val config = """ val config = """
akka.cluster { akka.cluster {
@ -223,7 +223,7 @@ class ClusterSpec extends AkkaSpec(ClusterSpec.config) with ImplicitSender {
probe.expectMsgType[CurrentClusterState] probe.expectMsgType[CurrentClusterState]
Cluster(sys2).join(Cluster(sys2).selfAddress) Cluster(sys2).join(Cluster(sys2).selfAddress)
probe.expectMsgType[MemberUp] probe.expectMsgType[MemberUp]
val mat = ActorMaterializer()(sys2) val mat = Materializer(sys2)
val sink = StreamRefs.sinkRef[String]().to(Sink.ignore).run()(mat) val sink = StreamRefs.sinkRef[String]().to(Sink.ignore).run()(mat)
Source.tick(1.milli, 10.millis, "tick").to(sink).run()(mat) Source.tick(1.milli, 10.millis, "tick").to(sink).run()(mat)

View file

@ -24,7 +24,7 @@ Akka is also:
## Resources with Explicit Lifecycle ## 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 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 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 to them. Therefore you should always make sure that for every creation of such an object
@ -39,7 +39,7 @@ Due to an ActorSystems explicit lifecycle the JVM will not exit until it is s
Therefore it is necessary to shutdown all ActorSystems within a running application or Therefore it is necessary to shutdown all ActorSystems within a running application or
Scala REPL session in order to allow these processes to terminate. 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. that were created within it.
## Actors in General ## Actors in General

View file

@ -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. * `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`. * `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, 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 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. materializers and simplifies most stream use cases somewhat.
Having a default materializer available means that most, if not all, usages of Java `ActorMaterializer.create()` The `ActorMaterializer` factories has been deprecated and replaced with a few corresponding factories in `akka.stream.Materializer`.
and Scala `implicit val materializer = ActorMaterializer()` should be removed. 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) 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 ### Materializer settings deprecated
The `ActorMaterializerSettings` class has been deprecated. The `ActorMaterializerSettings` class has been deprecated.

View file

@ -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`.

View file

@ -4,6 +4,12 @@ Defer the creation of a `Sink` until materialization and access `ActorMaterializ
@ref[Sink operators](../index.md#sink-operators) @ref[Sink operators](../index.md#sink-operators)
@@@ warning
The `setup` operator has been deprecated, use @ref:[fromMaterializer](./fromMaterializer.md) instead.
@@@
@@@ div { .group-scala } @@@ div { .group-scala }
## Signature ## Signature

View file

@ -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`.

View file

@ -1,9 +1,15 @@
# Source/Flow.setup # 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) @ref[Simple operators](../index.md#simple-operators)
@@@ warning
The `setup` operator has been deprecated, use @ref:[fromMaterializer](./fromMaterializer.md) instead.
@@@
@@@ div { .group-scala } @@@ div { .group-scala }
## Signature ## Signature

View file

@ -54,6 +54,7 @@ These built-in sinks are available from @scala[`akka.stream.scaladsl.Sink`] @jav
|Sink|<a name="foreach"></a>@ref[foreach](Sink/foreach.md)|Invoke a given procedure for each element received.| |Sink|<a name="foreach"></a>@ref[foreach](Sink/foreach.md)|Invoke a given procedure for each element received.|
|Sink|<a name="foreachasync"></a>@ref[foreachAsync](Sink/foreachAsync.md)|Invoke a given procedure asynchronously for each element received.| |Sink|<a name="foreachasync"></a>@ref[foreachAsync](Sink/foreachAsync.md)|Invoke a given procedure asynchronously for each element received.|
|Sink|<a name="foreachparallel"></a>@ref[foreachParallel](Sink/foreachParallel.md)|Like `foreach` but allows up to `parallellism` procedure calls to happen in parallel.| |Sink|<a name="foreachparallel"></a>@ref[foreachParallel](Sink/foreachParallel.md)|Like `foreach` but allows up to `parallellism` procedure calls to happen in parallel.|
|Sink|<a name="frommaterializer"></a>@ref[fromMaterializer](Sink/fromMaterializer.md)|Defer the creation of a `Sink` until materialization and access `Materializer` and `Attributes`|
|Sink|<a name="fromsubscriber"></a>@ref[fromSubscriber](Sink/fromSubscriber.md)|Integration with Reactive Streams, wraps a `org.reactivestreams.Subscriber` as a sink.| |Sink|<a name="fromsubscriber"></a>@ref[fromSubscriber](Sink/fromSubscriber.md)|Integration with Reactive Streams, wraps a `org.reactivestreams.Subscriber` as a sink.|
|Sink|<a name="head"></a>@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|<a name="head"></a>@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|<a name="headoption"></a>@ref[headOption](Sink/headOption.md)|Materializes into a @scala[`Future[Option[T]]`] @java[`CompletionStage<Optional<T>>`] 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.| |Sink|<a name="headoption"></a>@ref[headOption](Sink/headOption.md)|Materializes into a @scala[`Future[Option[T]]`] @java[`CompletionStage<Optional<T>>`] 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|<a name="filternot"></a>@ref[filterNot](Source-or-Flow/filterNot.md)|Filter the incoming elements using a predicate.| |Source/Flow|<a name="filternot"></a>@ref[filterNot](Source-or-Flow/filterNot.md)|Filter the incoming elements using a predicate.|
|Source/Flow|<a name="fold"></a>@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|<a name="fold"></a>@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|<a name="foldasync"></a>@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|<a name="foldasync"></a>@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|<a name="frommaterializer"></a>@ref[fromMaterializer](Source-or-Flow/fromMaterializer.md)|Defer the creation of a `Source/Flow` until materialization and access `Materializer` and `Attributes`|
|Source/Flow|<a name="grouped"></a>@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|<a name="grouped"></a>@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|<a name="intersperse"></a>@ref[intersperse](Source-or-Flow/intersperse.md)|Intersperse stream with provided element similar to `List.mkString`.| |Source/Flow|<a name="intersperse"></a>@ref[intersperse](Source-or-Flow/intersperse.md)|Intersperse stream with provided element similar to `List.mkString`.|
|Flow|<a name="lazyinitasync"></a>@ref[lazyInitAsync](Flow/lazyInitAsync.md)|Creates a real `Flow` upon receiving the first element by calling relevant `flowFactory` given as an argument.| |Flow|<a name="lazyinitasync"></a>@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|<a name="reduce"></a>@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|<a name="reduce"></a>@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|<a name="scan"></a>@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|<a name="scan"></a>@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|<a name="scanasync"></a>@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|<a name="scanasync"></a>@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|<a name="setup"></a>@ref[setup](Source-or-Flow/setup.md)|Defer the creation of a `Source/Flow` until materialization and access `ActorMaterializer` and `Attributes`| |Source/Flow|<a name="setup"></a>@ref[setup](Source-or-Flow/setup.md)|Defer the creation of a `Source/Flow` until materialization and access `Materializer` and `Attributes`|
|Source/Flow|<a name="sliding"></a>@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|<a name="sliding"></a>@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|<a name="statefulmapconcat"></a>@ref[statefulMapConcat](Source-or-Flow/statefulMapConcat.md)|Transform each element into zero or more elements that are individually passed downstream.| |Source/Flow|<a name="statefulmapconcat"></a>@ref[statefulMapConcat](Source-or-Flow/statefulMapConcat.md)|Transform each element into zero or more elements that are individually passed downstream.|
|Source/Flow|<a name="take"></a>@ref[take](Source-or-Flow/take.md)|Pass `n` incoming elements downstream and then complete| |Source/Flow|<a name="take"></a>@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) * [fromPublisher](Source/fromPublisher.md)
* [fromIterator](Source/fromIterator.md) * [fromIterator](Source/fromIterator.md)
* [cycle](Source/cycle.md) * [cycle](Source/cycle.md)
* [fromMaterializer](Source-or-Flow/fromMaterializer.md)
* [setup](Source-or-Flow/setup.md) * [setup](Source-or-Flow/setup.md)
* [fromFuture](Source/fromFuture.md) * [fromFuture](Source/fromFuture.md)
* [fromCompletionStage](Source/fromCompletionStage.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) * [fromSinkAndSourceCoupled](Flow/fromSinkAndSourceCoupled.md)
* [lazyInitAsync](Flow/lazyInitAsync.md) * [lazyInitAsync](Flow/lazyInitAsync.md)
* [preMaterialize](Sink/preMaterialize.md) * [preMaterialize](Sink/preMaterialize.md)
* [fromMaterializer](Sink/fromMaterializer.md)
* [setup](Sink/setup.md) * [setup](Sink/setup.md)
* [fromSubscriber](Sink/fromSubscriber.md) * [fromSubscriber](Sink/fromSubscriber.md)
* [cancelled](Sink/cancelled.md) * [cancelled](Sink/cancelled.md)

View file

@ -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`. 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, 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 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 @@@ 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 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 * 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 a `Materializer`'s life-cycle.
An important aspect of working with streams and actors is understanding an `ActorMaterializer`'s life-cycle.
The materializer is bound to the lifecycle of the `ActorRefFactory` it is created from, which in practice will 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`). 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 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. 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 Scala
: @@snip [FlowDocSpec.scala](/akka-docs/src/test/scala/docs/stream/FlowDocSpec.scala) { #materializer-from-actor-context } : @@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*. 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. 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. 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. 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 @@@ warning
Do not create new actor materializers inside actors by passing the `context.system` to it. 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`. It is instead recommended to either pass-in the Materializer or create one using the actor's `context`.
@@@ @@@

View file

@ -416,7 +416,7 @@ has also a type parameter of @scala[`Future[Int]`]@java[`CompletionStage<Integer
This step does *not* yet materialize the This step does *not* yet materialize the
processing pipeline, it merely prepares the description of the Flow, which is now connected to a Sink, and therefore can processing pipeline, it merely prepares the description of the Flow, which is now connected to a Sink, and therefore can
be `run()`, as indicated by its type: @scala[`RunnableGraph[Future[Int]]`]@java[`RunnableGraph<CompletionStage<Integer>>`]. Next we call `run()` which uses the @scala[implicit] `ActorMaterializer` be `run()`, as indicated by its type: @scala[`RunnableGraph[Future[Int]]`]@java[`RunnableGraph<CompletionStage<Integer>>`]. 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<T>`] is of type `T`. to materialize and run the Flow. The value returned by calling `run()` on a @scala[`RunnableGraph[T]`]@java[`RunnableGraph<T>`] is of type `T`.
In our case this type is @scala[`Future[Int]`]@java[`CompletionStage<Integer>`] which, when completed, will contain the total length of our `tweets` stream. In our case this type is @scala[`Future[Int]`]@java[`CompletionStage<Integer>`] 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. In case of the stream failing, this future would complete with a Failure.

View file

@ -18,7 +18,6 @@ import com.typesafe.config.Config;
import akka.actor.*; import akka.actor.*;
import akka.persistence.query.*; import akka.persistence.query.*;
import akka.stream.ActorMaterializer;
import akka.stream.javadsl.Sink; import akka.stream.javadsl.Sink;
import akka.stream.javadsl.Source; import akka.stream.javadsl.Source;

View file

@ -64,7 +64,7 @@ public class MyEventsByTagSource extends GraphStage<SourceShape<EventEnvelope>>
@Override @Override
public GraphStageLogic createLogic(Attributes inheritedAttributes) { public GraphStageLogic createLogic(Attributes inheritedAttributes) {
return new TimerGraphStageLogic(shape()) { return new TimerGraphStageLogic(shape()) {
private ActorSystem system = ((ActorMaterializer) materializer()).system(); private ActorSystem system = materializer().system();
private long currentOffset = initialOffset; private long currentOffset = initialOffset;
private List<EventEnvelope> buf = new LinkedList<>(); private List<EventEnvelope> buf = new LinkedList<>();
private final Serialization serialization = SerializationExtension.get(system); private final Serialization serialization = SerializationExtension.get(system);

View file

@ -295,7 +295,7 @@ public class FlowDocTest extends AbstractJavaTest {
// #materializer-from-actor-context // #materializer-from-actor-context
final class RunWithMyself extends AbstractActor { final class RunWithMyself extends AbstractActor {
ActorMaterializer mat = ActorMaterializer.create(context()); Materializer mat = Materializer.create(context());
@Override @Override
public void preStart() throws Exception { public void preStart() throws Exception {

View file

@ -62,7 +62,7 @@ public class HubDocTest extends AbstractJavaTest {
@Test @Test
public void dynamicBroadcast() { public void dynamicBroadcast() {
// Used to be able to clean up the running stream // Used to be able to clean up the running stream
ActorMaterializer materializer = ActorMaterializer.create(system); Materializer materializer = Materializer.create(system);
// #broadcast-hub // #broadcast-hub
// A simple producer that publishes a new "message" every second // A simple producer that publishes a new "message" every second
@ -134,7 +134,7 @@ public class HubDocTest extends AbstractJavaTest {
@Test @Test
public void dynamicPartition() { public void dynamicPartition() {
// Used to be able to clean up the running stream // Used to be able to clean up the running stream
ActorMaterializer materializer = ActorMaterializer.create(system); Materializer materializer = Materializer.create(system);
// #partition-hub // #partition-hub
// A simple producer that publishes a new "message-n" every second // A simple producer that publishes a new "message-n" every second
@ -182,7 +182,7 @@ public class HubDocTest extends AbstractJavaTest {
@Test @Test
public void dynamicStatefulPartition() { public void dynamicStatefulPartition() {
// Used to be able to clean up the running stream // Used to be able to clean up the running stream
ActorMaterializer materializer = ActorMaterializer.create(system); Materializer materializer = Materializer.create(system);
// #partition-hub-stateful // #partition-hub-stateful
// A simple producer that publishes a new "message-n" every second // A simple producer that publishes a new "message-n" every second
@ -215,7 +215,7 @@ public class HubDocTest extends AbstractJavaTest {
@Test @Test
public void dynamicFastestPartition() { public void dynamicFastestPartition() {
// Used to be able to clean up the running stream // Used to be able to clean up the running stream
ActorMaterializer materializer = ActorMaterializer.create(system); Materializer materializer = Materializer.create(system);
// #partition-hub-fastest // #partition-hub-fastest
Source<Integer, NotUsed> producer = Source.range(0, 100); Source<Integer, NotUsed> producer = Source.range(0, 100);

View file

@ -5,8 +5,6 @@
package jdocs.stream; package jdocs.stream;
import akka.actor.ActorSystem; import akka.actor.ActorSystem;
import akka.stream.ActorMaterializer;
import akka.stream.Materializer;
import akka.stream.javadsl.Sink; import akka.stream.javadsl.Sink;
import akka.stream.javadsl.Source; import akka.stream.javadsl.Source;
import akka.testkit.javadsl.TestKit; import akka.testkit.javadsl.TestKit;
@ -16,8 +14,6 @@ import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.List;
public class SubstreamDocTest extends AbstractJavaTest { public class SubstreamDocTest extends AbstractJavaTest {

View file

@ -8,8 +8,6 @@ import akka.Done;
import akka.actor.ActorSystem; import akka.actor.ActorSystem;
import akka.dispatch.Futures; import akka.dispatch.Futures;
import akka.japi.pf.PFBuilder; import akka.japi.pf.PFBuilder;
import akka.stream.ActorMaterializer;
import akka.stream.Materializer;
import akka.stream.javadsl.Keep; import akka.stream.javadsl.Keep;
import akka.stream.javadsl.Source; import akka.stream.javadsl.Source;
import akka.stream.testkit.TestSubscriber; import akka.stream.testkit.TestSubscriber;

View file

@ -5,10 +5,8 @@
package docs.persistence package docs.persistence
import akka.actor._ import akka.actor._
import akka.pattern.{ Backoff, BackoffOpts, BackoffSupervisor } import akka.pattern.{ BackoffOpts, BackoffSupervisor }
import akka.persistence._ import akka.persistence._
import akka.stream.ActorMaterializer
import akka.stream.scaladsl.{ Flow, Sink, Source }
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.language.postfixOps import scala.language.postfixOps

View file

@ -24,11 +24,7 @@ class MyEventsByTagSource(tag: String, offset: Long, refreshInterval: FiniteDura
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = override def createLogic(inheritedAttributes: Attributes): GraphStageLogic =
new TimerGraphStageLogic(shape) with OutHandler { new TimerGraphStageLogic(shape) with OutHandler {
lazy val system = materializer match { lazy val system = materializer.system
case a: ActorMaterializer => a.system
case _ =>
throw new IllegalStateException("EventsByTagStage requires ActorMaterializer")
}
private val Limit = 1000 private val Limit = 1000
private val connection: java.sql.Connection = ??? private val connection: java.sql.Connection = ???
private var currentOffset = offset private var currentOffset = offset

View file

@ -6,7 +6,6 @@ package docs.stream
import akka.NotUsed import akka.NotUsed
import akka.actor.{ Actor, ActorSystem, Cancellable } import akka.actor.{ Actor, ActorSystem, Cancellable }
import akka.stream.ActorMaterializer
import akka.stream.Materializer import akka.stream.Materializer
import akka.stream.{ ClosedShape, FlowShape, OverflowStrategy } import akka.stream.{ ClosedShape, FlowShape, OverflowStrategy }
import akka.stream.scaladsl._ import akka.stream.scaladsl._
@ -245,7 +244,7 @@ object FlowDocSpec {
//#materializer-from-actor-context //#materializer-from-actor-context
final class RunWithMyself extends Actor { final class RunWithMyself extends Actor {
implicit val mat = ActorMaterializer() implicit val mat = Materializer(context)
Source.maybe.runWith(Sink.onComplete { Source.maybe.runWith(Sink.onComplete {
case Success(done) => println(s"Completed: $done") case Success(done) => println(s"Completed: $done")

View file

@ -6,7 +6,6 @@ package docs.stream
import akka.NotUsed import akka.NotUsed
import akka.actor.{ Actor, Props } import akka.actor.{ Actor, Props }
import akka.stream.ActorMaterializer
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import docs.CompileOnlySpec import docs.CompileOnlySpec
@ -22,7 +21,6 @@ class FlowStreamRefsDocSpec extends AkkaSpec with CompileOnlySpec {
case class LogsOffer(streamId: Int, sourceRef: SourceRef[String]) case class LogsOffer(streamId: Int, sourceRef: SourceRef[String])
class DataSource extends Actor { class DataSource extends Actor {
implicit val mat = ActorMaterializer()(context)
def receive = { def receive = {
case RequestLogs(streamId) => case RequestLogs(streamId) =>
@ -43,7 +41,6 @@ class FlowStreamRefsDocSpec extends AkkaSpec with CompileOnlySpec {
} }
//#offer-source //#offer-source
implicit val mat = ActorMaterializer()
//#offer-source-use //#offer-source-use
val sourceActor = system.actorOf(Props[DataSource], "dataSource") val sourceActor = system.actorOf(Props[DataSource], "dataSource")
@ -60,7 +57,6 @@ class FlowStreamRefsDocSpec extends AkkaSpec with CompileOnlySpec {
"offer a sink ref" in compileOnlySpec { "offer a sink ref" in compileOnlySpec {
//#offer-sink //#offer-sink
import akka.pattern.pipe
import akka.stream.SinkRef import akka.stream.SinkRef
case class PrepareUpload(id: String) case class PrepareUpload(id: String)
@ -68,8 +64,6 @@ class FlowStreamRefsDocSpec extends AkkaSpec with CompileOnlySpec {
class DataReceiver extends Actor { class DataReceiver extends Actor {
implicit val mat = ActorMaterializer()(context)
def receive = { def receive = {
case PrepareUpload(nodeId) => case PrepareUpload(nodeId) =>
// obtain the source you want to offer: // obtain the source you want to offer:

View file

@ -4,7 +4,7 @@
package docs.stream package docs.stream
import akka.stream.{ ActorMaterializer, ClosedShape, OverflowStrategy } import akka.stream.{ ClosedShape, OverflowStrategy }
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec

View file

@ -22,7 +22,6 @@ object TwitterStreamQuickstartDocSpec {
//#fiddle_code //#fiddle_code
import akka.NotUsed import akka.NotUsed
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.stream.ActorMaterializer
import akka.stream.scaladsl._ import akka.stream.scaladsl._
//#model //#model

View file

@ -13,7 +13,6 @@ object SourceOperators {
//#sourceFromFuture //#sourceFromFuture
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.stream.ActorMaterializer
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.{ Done, NotUsed } import akka.{ Done, NotUsed }

View file

@ -8,7 +8,6 @@ import akka.stream.scaladsl.Source
object Scan { object Scan {
def scanExample(): Unit = { def scanExample(): Unit = {
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.stream.ActorMaterializer
implicit val system: ActorSystem = ActorSystem() implicit val system: ActorSystem = ActorSystem()

View file

@ -4,31 +4,40 @@
package akka.persistence.query.journal.leveldb package akka.persistence.query.journal.leveldb
import akka.NotUsed
import akka.actor.ActorRef import akka.actor.ActorRef
import akka.annotation.InternalApi import akka.annotation.InternalApi
import akka.persistence.Persistence import akka.persistence.Persistence
import akka.persistence.journal.leveldb.LeveldbJournal import akka.persistence.journal.leveldb.LeveldbJournal
import akka.stream.{ ActorMaterializer, Attributes, Outlet, SourceShape } import akka.stream.Attributes
import akka.stream.stage.{ GraphStage, GraphStageLogic, OutHandler, TimerGraphStageLogicWithLogging } 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 * INTERNAL API
*/ */
@InternalApi @InternalApi
final private[akka] class AllPersistenceIdsStage( final private[akka] class AllPersistenceIdsStage(liveQuery: Boolean, writeJournalPluginId: String)
liveQuery: Boolean,
writeJournalPluginId: String,
mat: ActorMaterializer)
extends GraphStage[SourceShape[String]] { extends GraphStage[SourceShape[String]] {
val out: Outlet[String] = Outlet("AllPersistenceIds.out") val out: Outlet[String] = Outlet("AllPersistenceIds.out")
override def shape: SourceShape[String] = SourceShape(out) override def shape: SourceShape[String] = SourceShape(out)
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = { override def createLogic(inheritedAttributes: Attributes): GraphStageLogic =
new TimerGraphStageLogicWithLogging(shape) with OutHandler with Buffer[String] { 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) setHandler(out, this)
val journal: ActorRef = Persistence(mat.system).journalFor(writeJournalPluginId) val journal: ActorRef = Persistence(eagerMaterializer.system).journalFor(writeJournalPluginId)
var initialResponseReceived = false var initialResponseReceived = false
override def preStart(): Unit = { override def preStart(): Unit = {
@ -60,5 +69,7 @@ final private[akka] class AllPersistenceIdsStage(
} }
} }
(logic, NotUsed)
} }
} }

View file

@ -4,16 +4,27 @@
package akka.persistence.query.journal.leveldb package akka.persistence.query.journal.leveldb
import akka.NotUsed
import akka.actor.ActorRef import akka.actor.ActorRef
import akka.annotation.InternalApi 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.Persistence
import akka.persistence.journal.leveldb.LeveldbJournal import akka.persistence.journal.leveldb.LeveldbJournal
import akka.persistence.journal.leveldb.LeveldbJournal.EventAppended 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.persistence.query.journal.leveldb.EventsByPersistenceIdStage.Continue
import akka.stream.{ ActorMaterializer, Attributes, Outlet, SourceShape } import akka.stream.Attributes
import akka.stream.stage.{ GraphStage, GraphStageLogic, OutHandler, TimerGraphStageLogicWithLogging } 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 import scala.concurrent.duration.FiniteDuration
@ -36,11 +47,17 @@ final private[akka] class EventsByPersistenceIdStage(
maxBufSize: Int, maxBufSize: Int,
writeJournalPluginId: String, writeJournalPluginId: String,
refreshInterval: Option[FiniteDuration], refreshInterval: Option[FiniteDuration],
mat: ActorMaterializer) mat: Materializer)
extends GraphStage[SourceShape[EventEnvelope]] { extends GraphStage[SourceShape[EventEnvelope]] {
val out: Outlet[EventEnvelope] = Outlet("EventsByPersistenceIdSource") val out: Outlet[EventEnvelope] = Outlet("EventsByPersistenceIdSource")
override def shape: SourceShape[EventEnvelope] = SourceShape(out) 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 logic = new TimerGraphStageLogicWithLogging(shape) with OutHandler with Buffer[EventEnvelope] {
val journal: ActorRef = Persistence(mat.system).journalFor(writeJournalPluginId) val journal: ActorRef = Persistence(mat.system).journalFor(writeJournalPluginId)
var currSeqNo = fromSequenceNr var currSeqNo = fromSequenceNr
@ -140,7 +157,7 @@ final private[akka] class EventsByPersistenceIdStage(
setHandler(out, this) setHandler(out, this)
} }
logic (logic, NotUsed)
} }
} }

View file

@ -4,20 +4,27 @@
package akka.persistence.query.journal.leveldb package akka.persistence.query.journal.leveldb
import akka.NotUsed
import akka.actor.ActorRef import akka.actor.ActorRef
import akka.annotation.InternalApi 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.Persistence
import akka.persistence.journal.leveldb.LeveldbJournal import akka.persistence.journal.leveldb.LeveldbJournal
import akka.persistence.journal.leveldb.LeveldbJournal.{ import akka.persistence.journal.leveldb.LeveldbJournal.ReplayTaggedMessages
ReplayTaggedMessages, import akka.persistence.journal.leveldb.LeveldbJournal.ReplayedTaggedMessage
ReplayedTaggedMessage, import akka.persistence.journal.leveldb.LeveldbJournal.TaggedEventAppended
TaggedEventAppended
}
import akka.persistence.query.journal.leveldb.EventsByTagStage.Continue import akka.persistence.query.journal.leveldb.EventsByTagStage.Continue
import akka.persistence.query.{ EventEnvelope, Sequence } import akka.persistence.query.EventEnvelope
import akka.stream.stage.{ GraphStage, GraphStageLogic, OutHandler, TimerGraphStageLogicWithLogging } import akka.persistence.query.Sequence
import akka.stream.{ ActorMaterializer, Attributes, Outlet, SourceShape } 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 import scala.concurrent.duration.FiniteDuration
@ -38,7 +45,6 @@ final private[leveldb] class EventsByTagStage(
maxBufSize: Int, maxBufSize: Int,
initialTooOffset: Long, initialTooOffset: Long,
writeJournalPluginId: String, writeJournalPluginId: String,
mat: ActorMaterializer,
refreshInterval: Option[FiniteDuration]) refreshInterval: Option[FiniteDuration])
extends GraphStage[SourceShape[EventEnvelope]] { extends GraphStage[SourceShape[EventEnvelope]] {
@ -46,9 +52,15 @@ final private[leveldb] class EventsByTagStage(
override def shape: SourceShape[EventEnvelope] = SourceShape(out) 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 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 currOffset: Long = fromOffset
var toOffset: Long = initialTooOffset var toOffset: Long = initialTooOffset
var stageActorRef: ActorRef = null var stageActorRef: ActorRef = null
@ -144,7 +156,7 @@ final private[leveldb] class EventsByTagStage(
setHandler(out, this) setHandler(out, this)
} }
logic (logic, NotUsed)
} }
} }

View file

@ -9,11 +9,18 @@ import java.net.URLEncoder
import akka.NotUsed import akka.NotUsed
import akka.actor.ExtendedActorSystem import akka.actor.ExtendedActorSystem
import akka.event.Logging import akka.event.Logging
import akka.persistence.query.journal.leveldb.{ AllPersistenceIdsStage, EventsByPersistenceIdStage, EventsByTagStage } import akka.persistence.query.journal.leveldb.AllPersistenceIdsStage
import akka.persistence.query.scaladsl.{ ReadJournal, _ } import akka.persistence.query.journal.leveldb.EventsByPersistenceIdStage
import akka.persistence.query.{ EventEnvelope, NoOffset, Offset, Sequence } 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.stream.scaladsl.Source
import akka.util.{ unused, ByteString } import akka.util.ByteString
import akka.util.unused
import com.typesafe.config.Config import com.typesafe.config.Config
import scala.concurrent.duration._ 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 * The stream is completed with failure if there is a failure in executing the query in the
* backend journal. * 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 // no polling for this query, the write journal will push all changes, i.e. no refreshInterval
Source Source.fromGraph(new AllPersistenceIdsStage(liveQuery = true, writeJournalPluginId)).named("allPersistenceIds")
.setup { (mat, _) =>
Source
.fromGraph(new AllPersistenceIdsStage(liveQuery = true, writeJournalPluginId, mat))
.named("allPersistenceIds")
}
.mapMaterializedValue(_ => NotUsed)
}
/** /**
* Same type of query as [[#persistenceIds]] but the stream * Same type of query as [[#persistenceIds]] but the stream
* is completed immediately when it reaches the end of the "result set". Persistent * 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. * actors that are created after the query is completed are not included in the stream.
*/ */
override def currentPersistenceIds(): Source[String, NotUsed] = { override def currentPersistenceIds(): Source[String, NotUsed] =
Source Source.fromGraph(new AllPersistenceIdsStage(liveQuery = false, writeJournalPluginId)).named("allPersistenceIds")
.setup { (mat, _) =>
Source
.fromGraph(new AllPersistenceIdsStage(liveQuery = false, writeJournalPluginId, mat))
.named("allPersistenceIds")
}
.mapMaterializedValue(_ => NotUsed)
}
/** /**
* `eventsByPersistenceId` is used for retrieving events for a specific * `eventsByPersistenceId` is used for retrieving events for a specific
@ -121,7 +113,7 @@ class LeveldbReadJournal(@unused system: ExtendedActorSystem, config: Config)
fromSequenceNr: Long = 0L, fromSequenceNr: Long = 0L,
toSequenceNr: Long = Long.MaxValue): Source[EventEnvelope, NotUsed] = { toSequenceNr: Long = Long.MaxValue): Source[EventEnvelope, NotUsed] = {
Source Source
.setup { (mat, _) => .fromMaterializer { (mat, _) =>
Source Source
.fromGraph( .fromGraph(
new EventsByPersistenceIdStage( new EventsByPersistenceIdStage(
@ -147,7 +139,7 @@ class LeveldbReadJournal(@unused system: ExtendedActorSystem, config: Config)
fromSequenceNr: Long = 0L, fromSequenceNr: Long = 0L,
toSequenceNr: Long = Long.MaxValue): Source[EventEnvelope, NotUsed] = { toSequenceNr: Long = Long.MaxValue): Source[EventEnvelope, NotUsed] = {
Source Source
.setup { (mat, _) => .fromMaterializer { (mat, _) =>
Source Source
.fromGraph( .fromGraph(
new EventsByPersistenceIdStage( new EventsByPersistenceIdStage(
@ -207,20 +199,10 @@ class LeveldbReadJournal(@unused system: ExtendedActorSystem, config: Config)
offset match { offset match {
case seq: Sequence => case seq: Sequence =>
Source Source
.setup { (mat, _) => .fromGraph(
Source new EventsByTagStage(tag, seq.value, maxBufSize, Long.MaxValue, writeJournalPluginId, refreshInterval))
.fromGraph( .named("eventsByTag-" + URLEncoder.encode(tag, ByteString.UTF_8))
new EventsByTagStage(
tag,
seq.value,
maxBufSize,
Long.MaxValue,
writeJournalPluginId,
mat,
refreshInterval))
.named("eventsByTag-" + URLEncoder.encode(tag, ByteString.UTF_8))
}
.mapMaterializedValue(_ => NotUsed)
case NoOffset => eventsByTag(tag, Sequence(0L)) //recursive case NoOffset => eventsByTag(tag, Sequence(0L)) //recursive
case _ => case _ =>
throw new IllegalArgumentException( 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 * 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. * 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] = { override def currentEventsByTag(tag: String, offset: Offset = Sequence(0L)): Source[EventEnvelope, NotUsed] =
Source offset match {
.setup { (mat, _) => case seq: Sequence =>
offset match { Source
case seq: Sequence => .fromGraph(new EventsByTagStage(tag, seq.value, maxBufSize, Long.MaxValue, writeJournalPluginId, None))
Source .named("currentEventsByTag-" + URLEncoder.encode(tag, ByteString.UTF_8))
.fromGraph( case NoOffset => currentEventsByTag(tag, Sequence(0L))
new EventsByTagStage(tag, seq.value, maxBufSize, Long.MaxValue, writeJournalPluginId, mat, None)) case _ =>
.named("currentEventsByTag-" + URLEncoder.encode(tag, ByteString.UTF_8)) throw new IllegalArgumentException(
case NoOffset => currentEventsByTag(tag, Sequence(0L)) "LevelDB does not support " + Logging.simpleName(offset.getClass) + " offsets")
case _ => }
throw new IllegalArgumentException(
"LevelDB does not support " + Logging.simpleName(offset.getClass) + " offsets")
}
}
.mapMaterializedValue(_ => NotUsed)
}
} }

View file

@ -4,16 +4,15 @@
package akka.persistence.query.journal.leveldb package akka.persistence.query.journal.leveldb
import scala.concurrent.duration._
import akka.persistence.query.PersistenceQuery import akka.persistence.query.PersistenceQuery
import akka.persistence.query.journal.leveldb.scaladsl.LeveldbReadJournal import akka.persistence.query.journal.leveldb.scaladsl.LeveldbReadJournal
import akka.persistence.query.scaladsl.PersistenceIdsQuery import akka.persistence.query.scaladsl.PersistenceIdsQuery
import akka.stream.ActorMaterializer
import akka.stream.testkit.scaladsl.TestSink import akka.stream.testkit.scaladsl.TestSink
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.testkit.ImplicitSender import akka.testkit.ImplicitSender
import scala.concurrent.duration._
object AllPersistenceIdsSpec { object AllPersistenceIdsSpec {
val config = """ val config = """
akka.loglevel = INFO akka.loglevel = INFO
@ -28,8 +27,6 @@ object AllPersistenceIdsSpec {
class AllPersistenceIdsSpec extends AkkaSpec(AllPersistenceIdsSpec.config) with Cleanup with ImplicitSender { class AllPersistenceIdsSpec extends AkkaSpec(AllPersistenceIdsSpec.config) with Cleanup with ImplicitSender {
implicit val mat = ActorMaterializer()(system)
val queries = PersistenceQuery(system).readJournalFor[LeveldbReadJournal](LeveldbReadJournal.Identifier) val queries = PersistenceQuery(system).readJournalFor[LeveldbReadJournal](LeveldbReadJournal.Identifier)
"Leveldb query AllPersistenceIds" must { "Leveldb query AllPersistenceIds" must {

View file

@ -4,17 +4,16 @@
package akka.persistence.query.journal.leveldb package akka.persistence.query.journal.leveldb
import scala.concurrent.duration._
import akka.actor.ActorRef import akka.actor.ActorRef
import akka.persistence.query.PersistenceQuery import akka.persistence.query.PersistenceQuery
import akka.persistence.query.journal.leveldb.scaladsl.LeveldbReadJournal import akka.persistence.query.journal.leveldb.scaladsl.LeveldbReadJournal
import akka.persistence.query.scaladsl.EventsByTagQuery import akka.persistence.query.scaladsl.EventsByTagQuery
import akka.stream.ActorMaterializer
import akka.stream.testkit.scaladsl.TestSink import akka.stream.testkit.scaladsl.TestSink
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.testkit.ImplicitSender import akka.testkit.ImplicitSender
import scala.concurrent.duration._
object EventsByPersistenceIdSpec { object EventsByPersistenceIdSpec {
val config = """ val config = """
akka.loglevel = INFO akka.loglevel = INFO
@ -30,8 +29,6 @@ object EventsByPersistenceIdSpec {
class EventsByPersistenceIdSpec extends AkkaSpec(EventsByPersistenceIdSpec.config) with Cleanup with ImplicitSender { class EventsByPersistenceIdSpec extends AkkaSpec(EventsByPersistenceIdSpec.config) with Cleanup with ImplicitSender {
implicit val mat = ActorMaterializer()(system)
val queries = PersistenceQuery(system).readJournalFor[LeveldbReadJournal](LeveldbReadJournal.Identifier) val queries = PersistenceQuery(system).readJournalFor[LeveldbReadJournal](LeveldbReadJournal.Identifier)
def setup(persistenceId: String): ActorRef = { def setup(persistenceId: String): ActorRef = {

View file

@ -4,17 +4,19 @@
package akka.persistence.query.journal.leveldb package akka.persistence.query.journal.leveldb
import scala.concurrent.duration._
import akka.persistence.journal.Tagged import akka.persistence.journal.Tagged
import akka.persistence.journal.WriteEventAdapter 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.journal.leveldb.scaladsl.LeveldbReadJournal
import akka.persistence.query.scaladsl.EventsByTagQuery 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.stream.testkit.scaladsl.TestSink
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.testkit.ImplicitSender import akka.testkit.ImplicitSender
import akka.persistence.query.NoOffset
import scala.concurrent.duration._
object EventsByTagSpec { object EventsByTagSpec {
val config = s""" val config = s"""
@ -59,8 +61,6 @@ class ColorTagger extends WriteEventAdapter {
class EventsByTagSpec extends AkkaSpec(EventsByTagSpec.config) with Cleanup with ImplicitSender { class EventsByTagSpec extends AkkaSpec(EventsByTagSpec.config) with Cleanup with ImplicitSender {
implicit val mat = ActorMaterializer()(system)
val queries = PersistenceQuery(system).readJournalFor[LeveldbReadJournal](LeveldbReadJournal.Identifier) val queries = PersistenceQuery(system).readJournalFor[LeveldbReadJournal](LeveldbReadJournal.Identifier)
"Leveldb query EventsByTag" must { "Leveldb query EventsByTag" must {

View file

@ -19,7 +19,6 @@ import akka.persistence.query.journal.leveldb.javadsl.LeveldbReadJournal;
import akka.persistence.typed.*; import akka.persistence.typed.*;
import akka.persistence.typed.scaladsl.EventSourcedBehaviorSpec; import akka.persistence.typed.scaladsl.EventSourcedBehaviorSpec;
import akka.serialization.jackson.CborSerializable; import akka.serialization.jackson.CborSerializable;
import akka.stream.ActorMaterializer;
import akka.stream.javadsl.Sink; import akka.stream.javadsl.Sink;
import akka.actor.testkit.typed.javadsl.TestKitJunitResource; import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
import akka.actor.testkit.typed.javadsl.TestProbe; import akka.actor.testkit.typed.javadsl.TestProbe;
@ -54,9 +53,6 @@ public class PersistentActorJavaDslTest extends JUnitSuite {
PersistenceQuery.get(Adapter.toClassic(testKit.system())) PersistenceQuery.get(Adapter.toClassic(testKit.system()))
.getReadJournalFor(LeveldbReadJournal.class, LeveldbReadJournal.Identifier()); .getReadJournalFor(LeveldbReadJournal.class, LeveldbReadJournal.Identifier());
private ActorMaterializer materializer =
ActorMaterializer.create(Adapter.toClassic(testKit.system()));
interface Command extends CborSerializable {} interface Command extends CborSerializable {}
public enum Increment implements Command { public enum Increment implements Command {
@ -547,7 +543,7 @@ public class PersistentActorJavaDslTest extends JUnitSuite {
List<EventEnvelope> events = List<EventEnvelope> events =
queries queries
.currentEventsByTag("tag1", NoOffset.getInstance()) .currentEventsByTag("tag1", NoOffset.getInstance())
.runWith(Sink.seq(), materializer) .runWith(Sink.seq(), testKit.system())
.toCompletableFuture() .toCompletableFuture()
.get(); .get();
assertEquals( assertEquals(
@ -578,7 +574,7 @@ public class PersistentActorJavaDslTest extends JUnitSuite {
List<EventEnvelope> events = List<EventEnvelope> events =
queries queries
.currentEventsByPersistenceId("transform", 0, Long.MAX_VALUE) .currentEventsByPersistenceId("transform", 0, Long.MAX_VALUE)
.runWith(Sink.seq(), materializer) .runWith(Sink.seq(), testKit.system())
.toCompletableFuture() .toCompletableFuture()
.get(); .get();
assertEquals( assertEquals(

View file

@ -41,7 +41,6 @@ import akka.persistence.typed.SnapshotMetadata
import akka.persistence.{ SnapshotMetadata => ClassicSnapshotMetadata } import akka.persistence.{ SnapshotMetadata => ClassicSnapshotMetadata }
import akka.persistence.{ SnapshotSelectionCriteria => ClassicSnapshotSelectionCriteria } import akka.persistence.{ SnapshotSelectionCriteria => ClassicSnapshotSelectionCriteria }
import akka.serialization.jackson.CborSerializable import akka.serialization.jackson.CborSerializable
import akka.stream.ActorMaterializer
import akka.stream.scaladsl.Sink import akka.stream.scaladsl.Sink
import akka.testkit.EventFilter import akka.testkit.EventFilter
import akka.testkit.TestEvent.Mute import akka.testkit.TestEvent.Mute
@ -289,7 +288,6 @@ class EventSourcedBehaviorSpec extends ScalaTestWithActorTestKit(EventSourcedBeh
import EventSourcedBehaviorSpec._ import EventSourcedBehaviorSpec._
import akka.actor.typed.scaladsl.adapter._ import akka.actor.typed.scaladsl.adapter._
implicit val materializer = ActorMaterializer()(system.toClassic)
val queries: LeveldbReadJournal = val queries: LeveldbReadJournal =
PersistenceQuery(system.toClassic).readJournalFor[LeveldbReadJournal](LeveldbReadJournal.Identifier) PersistenceQuery(system.toClassic).readJournalFor[LeveldbReadJournal](LeveldbReadJournal.Identifier)

View file

@ -19,7 +19,6 @@ import akka.persistence.typed.EventAdapter
import akka.persistence.typed.EventSeq import akka.persistence.typed.EventSeq
import akka.persistence.typed.PersistenceId import akka.persistence.typed.PersistenceId
import akka.serialization.jackson.CborSerializable import akka.serialization.jackson.CborSerializable
import akka.stream.ActorMaterializer
import akka.stream.scaladsl.Sink import akka.stream.scaladsl.Sink
import akka.testkit.EventFilter import akka.testkit.EventFilter
import akka.testkit.JavaSerializable import akka.testkit.JavaSerializable
@ -104,7 +103,6 @@ class EventSourcedEventAdapterSpec
val pidCounter = new AtomicInteger(0) val pidCounter = new AtomicInteger(0)
private def nextPid(): PersistenceId = PersistenceId(s"c${pidCounter.incrementAndGet()})") private def nextPid(): PersistenceId = PersistenceId(s"c${pidCounter.incrementAndGet()})")
implicit val materializer = ActorMaterializer()(system.toClassic)
val queries: LeveldbReadJournal = val queries: LeveldbReadJournal =
PersistenceQuery(system.toClassic).readJournalFor[LeveldbReadJournal](LeveldbReadJournal.Identifier) PersistenceQuery(system.toClassic).readJournalFor[LeveldbReadJournal](LeveldbReadJournal.Identifier)

View file

@ -7,14 +7,16 @@ package akka.persistence.typed.scaladsl
import java.util.UUID import java.util.UUID
import java.util.concurrent.atomic.AtomicInteger 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.actor.typed.ActorRef
import akka.persistence.query.PersistenceQuery import akka.persistence.query.PersistenceQuery
import akka.persistence.query.journal.leveldb.scaladsl.LeveldbReadJournal 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.serialization.jackson.CborSerializable
import akka.stream.ActorMaterializer import com.typesafe.config.Config
import com.typesafe.config.{ Config, ConfigFactory } import com.typesafe.config.ConfigFactory
import org.scalatest.WordSpecLike import org.scalatest.WordSpecLike
object EventSourcedSnapshotAdapterSpec { object EventSourcedSnapshotAdapterSpec {
@ -38,7 +40,7 @@ class EventSourcedSnapshotAdapterSpec
val pidCounter = new AtomicInteger(0) val pidCounter = new AtomicInteger(0)
private def nextPid(): PersistenceId = PersistenceId(s"c${pidCounter.incrementAndGet()})") private def nextPid(): PersistenceId = PersistenceId(s"c${pidCounter.incrementAndGet()})")
implicit val materializer = ActorMaterializer()(system.toClassic)
val queries: LeveldbReadJournal = val queries: LeveldbReadJournal =
PersistenceQuery(system.toClassic).readJournalFor[LeveldbReadJournal](LeveldbReadJournal.Identifier) PersistenceQuery(system.toClassic).readJournalFor[LeveldbReadJournal](LeveldbReadJournal.Identifier)

View file

@ -14,7 +14,7 @@ import akka.remote.RemotingMultiNodeSpec
import akka.remote.testconductor.RoleName import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig import akka.remote.testkit.MultiNodeConfig
import akka.serialization.jackson.CborSerializable import akka.serialization.jackson.CborSerializable
import akka.stream.{ ActorMaterializer, ThrottleMode } import akka.stream.ThrottleMode
import akka.stream.scaladsl.Source import akka.stream.scaladsl.Source
import akka.testkit._ import akka.testkit._
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
@ -189,7 +189,6 @@ abstract class LatencySpec extends RemotingMultiNodeSpec(LatencySpec) {
var plots = LatencyPlots() var plots = LatencyPlots()
lazy implicit val mat = ActorMaterializer()(system)
import system.dispatcher import system.dispatcher
override def initialParticipants = roles.size override def initialParticipants = roles.size

View file

@ -8,9 +8,6 @@ package aeron
import java.io.File import java.io.File
import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.atomic.AtomicInteger
import scala.concurrent.Await
import scala.concurrent.duration._
import akka.Done import akka.Done
import akka.actor.ExtendedActorSystem import akka.actor.ExtendedActorSystem
import akka.actor.Props import akka.actor.Props
@ -18,7 +15,6 @@ import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec import akka.remote.testkit.MultiNodeSpec
import akka.remote.testkit.STMultiNodeSpec import akka.remote.testkit.STMultiNodeSpec
import akka.stream.ActorMaterializer
import akka.stream.KillSwitches import akka.stream.KillSwitches
import akka.stream.ThrottleMode import akka.stream.ThrottleMode
import akka.stream.scaladsl.Source import akka.stream.scaladsl.Source
@ -29,6 +25,9 @@ import io.aeron.Aeron
import io.aeron.driver.MediaDriver import io.aeron.driver.MediaDriver
import org.agrona.IoUtil import org.agrona.IoUtil
import scala.concurrent.Await
import scala.concurrent.duration._
object AeronStreamConsistencySpec extends MultiNodeConfig { object AeronStreamConsistencySpec extends MultiNodeConfig {
val first = role("first") val first = role("first")
val second = role("second") val second = role("second")
@ -73,7 +72,6 @@ abstract class AeronStreamConsistencySpec
val pool = new EnvelopeBufferPool(1024 * 1024, 128) val pool = new EnvelopeBufferPool(1024 * 1024, 128)
lazy implicit val mat = ActorMaterializer()(system)
import system.dispatcher import system.dispatcher
override def initialParticipants = roles.size override def initialParticipants = roles.size

View file

@ -22,7 +22,6 @@ import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec import akka.remote.testkit.MultiNodeSpec
import akka.remote.testkit.STMultiNodeSpec import akka.remote.testkit.STMultiNodeSpec
import akka.stream.ActorMaterializer
import akka.stream.KillSwitches import akka.stream.KillSwitches
import akka.stream.ThrottleMode import akka.stream.ThrottleMode
import akka.stream.scaladsl.Flow import akka.stream.scaladsl.Flow
@ -106,8 +105,6 @@ abstract class AeronStreamLatencySpec
r r
} }
lazy implicit val mat = ActorMaterializer()(system)
override def initialParticipants = roles.size override def initialParticipants = roles.size
def channel(roleName: RoleName) = { def channel(roleName: RoleName) = {

View file

@ -16,7 +16,6 @@ import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec import akka.remote.testkit.MultiNodeSpec
import akka.remote.testkit.STMultiNodeSpec import akka.remote.testkit.STMultiNodeSpec
import akka.stream.ActorMaterializer
import akka.stream.scaladsl.Source import akka.stream.scaladsl.Source
import akka.testkit._ import akka.testkit._
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
@ -103,7 +102,6 @@ abstract class AeronStreamMaxThroughputSpec
r r
} }
lazy implicit val mat = ActorMaterializer()(system)
import system.dispatcher import system.dispatcher
def adjustedTotalMessages(n: Long): Long = (n * totalMessagesFactor).toLong def adjustedTotalMessages(n: Long): Long = (n * totalMessagesFactor).toLong

View file

@ -51,7 +51,8 @@ object SendQueueSpec {
class SendQueueSpec extends AkkaSpec(""" class SendQueueSpec extends AkkaSpec("""
akka.actor.serialize-messages = off akka.actor.serialize-messages = off
akka.stream.materializer.debug.fuzzing-mode = on akka.stream.materializer.debug.fuzzing-mode = on
""".stripMargin) with ImplicitSender { akka.stream.secret-test-fuzzing-warning-disable = yep
""") with ImplicitSender {
import SendQueueSpec._ import SendQueueSpec._
def sendToDeadLetters[T](pending: Vector[T]): Unit = def sendToDeadLetters[T](pending: Vector[T]): Unit =

View file

@ -5,29 +5,34 @@
package akka.stream.testkit package akka.stream.testkit
import akka.NotUsed import akka.NotUsed
import akka.actor.{ ActorRefFactory, ActorSystem } import akka.actor.ActorRefFactory
import akka.stream.ActorMaterializerSettings import akka.actor.ActorSystem
import akka.stream.scaladsl._
import org.reactivestreams.Publisher
import akka.stream.ActorMaterializer 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]( class ChainSetup[In, Out, M](
stream: Flow[In, In, NotUsed] => Flow[In, Out, M], stream: Flow[In, In, NotUsed] => Flow[In, Out, M],
val settings: ActorMaterializerSettings, val settings: ActorMaterializerSettings,
materializer: ActorMaterializer, materializer: Materializer,
toPublisher: (Source[Out, _], ActorMaterializer) => Publisher[Out])(implicit val system: ActorSystem) { toPublisher: (Source[Out, _], Materializer) => Publisher[Out])(implicit val system: ActorSystem) {
@silent("deprecated")
def this( def this(
stream: Flow[In, In, NotUsed] => Flow[In, Out, M], stream: Flow[In, In, NotUsed] => Flow[In, Out, M],
settings: ActorMaterializerSettings, 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) this(stream, settings, ActorMaterializer(settings)(system), toPublisher)(system)
@silent("deprecated")
def this( def this(
stream: Flow[In, In, NotUsed] => Flow[In, Out, M], stream: Flow[In, In, NotUsed] => Flow[In, Out, M],
settings: ActorMaterializerSettings, settings: ActorMaterializerSettings,
materializerCreator: (ActorMaterializerSettings, ActorRefFactory) => ActorMaterializer, materializerCreator: (ActorMaterializerSettings, ActorRefFactory) => Materializer,
toPublisher: (Source[Out, _], ActorMaterializer) => Publisher[Out])(implicit system: ActorSystem) = toPublisher: (Source[Out, _], Materializer) => Publisher[Out])(implicit system: ActorSystem) =
this(stream, settings, materializerCreator(settings, system), toPublisher)(system) this(stream, settings, materializerCreator(settings, system), toPublisher)(system)
val upstream = TestPublisher.manualProbe[In]() val upstream = TestPublisher.manualProbe[In]()

View file

@ -4,12 +4,18 @@
package akka.stream.testkit package akka.stream.testkit
import java.util.concurrent.ThreadLocalRandom
import akka.NotUsed import akka.NotUsed
import akka.actor.ActorSystem 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.TestPublisher._
import akka.stream.testkit.TestSubscriber._ import akka.stream.testkit.TestSubscriber._
import akka.stream.{ ActorMaterializer, ActorMaterializerSettings }
import akka.stream.scaladsl.{ Flow, Sink, Source }
import org.reactivestreams.Publisher import org.reactivestreams.Publisher
import org.scalatest.Matchers import org.scalatest.Matchers
@ -24,7 +30,7 @@ trait ScriptedTest extends Matchers {
class ScriptException(msg: String) extends RuntimeException(msg) 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) (f, m) => f.runWith(Sink.asPublisher(false))(m)
object Script { object Script {

View file

@ -4,19 +4,16 @@
package akka.stream.testkit package akka.stream.testkit
import akka.stream._
import akka.stream.scaladsl.Source import akka.stream.scaladsl.Source
import akka.stream.testkit.scaladsl.TestSink import akka.stream.testkit.scaladsl.TestSink
import scala.concurrent.duration._
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.testkit.EventFilter import akka.testkit.EventFilter
import akka.testkit.TestEvent.Mute import akka.testkit.TestEvent.Mute
import akka.testkit.TestEvent.UnMute 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!") val ex = new Exception("Boom!")

View file

@ -4,15 +4,12 @@
package akka.stream.tck package akka.stream.tck
import akka.stream.ActorMaterializer
import akka.stream.scaladsl.Flow import akka.stream.scaladsl.Flow
import org.reactivestreams.Processor import org.reactivestreams.Processor
class MapTest extends AkkaIdentityProcessorVerification[Int] { class MapTest extends AkkaIdentityProcessorVerification[Int] {
override def createIdentityProcessor(maxBufferSize: Int): Processor[Int, Int] = { override def createIdentityProcessor(maxBufferSize: Int): Processor[Int, Int] = {
implicit val materializer = ActorMaterializer()(system)
Flow[Int].map(elem => elem).named("identity").toProcessor.run() Flow[Int].map(elem => elem).named("identity").toProcessor.run()
} }

View file

@ -4,16 +4,13 @@
package akka.stream.tck package akka.stream.tck
import akka.stream.ActorMaterializer import akka.stream.impl.VirtualProcessor
import akka.stream.scaladsl.Flow import akka.stream.scaladsl.Flow
import org.reactivestreams.Processor import org.reactivestreams.Processor
import akka.stream.impl.VirtualProcessor
class VirtualProcessorTest extends AkkaIdentityProcessorVerification[Int] { class VirtualProcessorTest extends AkkaIdentityProcessorVerification[Int] {
override def createIdentityProcessor(maxBufferSize: Int): Processor[Int, 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 identity = Flow[Int].map(elem => elem).named("identity").toProcessor.run()
val left, right = new VirtualProcessor[Int] val left, right = new VirtualProcessor[Int]
left.subscribe(identity) left.subscribe(identity)

View file

@ -5,7 +5,6 @@
package akka.stream; package akka.stream;
import akka.Done; import akka.Done;
import akka.NotUsed;
import akka.stream.javadsl.Keep; import akka.stream.javadsl.Keep;
import akka.stream.javadsl.RunnableGraph; import akka.stream.javadsl.RunnableGraph;
import akka.stream.javadsl.Sink; import akka.stream.javadsl.Sink;

View file

@ -15,20 +15,20 @@ import akka.stream.stage.GraphStage
import akka.stream.stage.GraphStageLogic import akka.stream.stage.GraphStageLogic
import akka.stream.stage.InHandler import akka.stream.stage.InHandler
import akka.stream.stage.OutHandler 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.StreamSpec
import akka.stream.testkit.TestPublisher import akka.stream.testkit.TestPublisher
import akka.stream.testkit.TestSubscriber import akka.stream.testkit.TestSubscriber
import akka.stream.testkit.Utils._
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.testkit.EventFilter import akka.testkit.EventFilter
import akka.testkit.TestLatch import akka.testkit.TestLatch
import org.reactivestreams.Publisher import org.reactivestreams.Publisher
import org.reactivestreams.Subscriber import org.reactivestreams.Subscriber
import org.reactivestreams.Subscription import org.reactivestreams.Subscription
import scala.concurrent.duration._
import scala.concurrent.Await import scala.concurrent.Await
import scala.concurrent.Promise import scala.concurrent.Promise
import scala.concurrent.duration._
class ActorGraphInterpreterSpec extends StreamSpec { class ActorGraphInterpreterSpec extends StreamSpec {
"ActorGraphInterpreter" must { "ActorGraphInterpreter" must {
@ -403,7 +403,7 @@ class ActorGraphInterpreterSpec extends StreamSpec {
} }
"trigger postStop in all stages when abruptly terminated (and no upstream boundaries)" in { "trigger postStop in all stages when abruptly terminated (and no upstream boundaries)" in {
val mat = ActorMaterializer() val mat = Materializer(system)
val gotStop = TestLatch(1) val gotStop = TestLatch(1)
object PostStopSnitchFlow extends SimpleLinearGraphStage[String] { object PostStopSnitchFlow extends SimpleLinearGraphStage[String] {

View file

@ -4,6 +4,10 @@
package akka.stream.impl.fusing 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.event.Logging
import akka.stream.Supervision.Decider import akka.stream.Supervision.Decider
import akka.stream._ import akka.stream._
@ -19,6 +23,60 @@ import akka.stream.testkit.Utils.TE
import com.github.ghik.silencer.silent import com.github.ghik.silencer.silent
import scala.collection.{ Map => SMap } 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 @silent
object GraphInterpreterSpecKit { object GraphInterpreterSpecKit {

View file

@ -4,27 +4,30 @@
package akka.stream.io 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 java.util.concurrent.TimeoutException
import akka.actor.ActorSystem
import akka.stream._
import akka.stream.Attributes.inputBuffer 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.StreamSupervisor.Children
import akka.stream.impl.io.InputStreamSinkStage import akka.stream.impl.io.InputStreamSinkStage
import akka.stream.impl.{ PhasedFusingActorMaterializer, StreamSupervisor } import akka.stream.scaladsl.Keep
import akka.stream.scaladsl.{ Keep, Source, StreamConverters } import akka.stream.scaladsl.Source
import akka.stream.scaladsl.StreamConverters
import akka.stream.testkit.Utils._ import akka.stream.testkit.Utils._
import akka.stream.testkit._
import akka.stream.testkit.scaladsl.StreamTestKit._ import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.stream.testkit.scaladsl.TestSource import akka.stream.testkit.scaladsl.TestSource
import akka.stream.testkit._
import akka.testkit.TestProbe import akka.testkit.TestProbe
import akka.util.ByteString import akka.util.ByteString
import scala.concurrent.Await
import scala.concurrent.Future
import scala.concurrent.duration._ import scala.concurrent.duration._
import java.util.concurrent.ThreadLocalRandom
import scala.concurrent.{ Await, Future }
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace
class InputStreamSinkSpec extends StreamSpec(UnboundedMailboxConfig) { class InputStreamSinkSpec extends StreamSpec(UnboundedMailboxConfig) {
@ -211,17 +214,12 @@ class InputStreamSinkSpec extends StreamSpec(UnboundedMailboxConfig) {
} }
"use dedicated default-blocking-io-dispatcher by default" in assertAllStagesStopped { "use dedicated default-blocking-io-dispatcher by default" in assertAllStagesStopped {
val sys = ActorSystem("dispatcher-testing", UnboundedMailboxConfig) // use a separate materializer to ensure we know what child is our stream
val materializer = ActorMaterializer()(sys) implicit val materializer = Materializer(system)
try { TestSource.probe[ByteString].runWith(StreamConverters.asInputStream())
TestSource.probe[ByteString].runWith(StreamConverters.asInputStream())(materializer) materializer.asInstanceOf[PhasedFusingActorMaterializer].supervisor.tell(StreamSupervisor.GetChildren, testActor)
materializer val ref = expectMsgType[Children].children.find(_.path.toString contains "inputStreamSink").get
.asInstanceOf[PhasedFusingActorMaterializer] assertDispatcher(ref, ActorAttributes.IODispatcher.dispatcher)
.supervisor
.tell(StreamSupervisor.GetChildren, testActor)
val ref = expectMsgType[Children].children.find(_.path.toString contains "inputStreamSink").get
assertDispatcher(ref, ActorAttributes.IODispatcher.dispatcher)
} finally shutdown(sys)
} }
"work when more bytes pulled from InputStream than available" in assertAllStagesStopped { "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 { "throw from inputstream read if terminated abruptly" in {
val mat = ActorMaterializer() val mat = Materializer(system)
val probe = TestPublisher.probe[ByteString]() val probe = TestPublisher.probe[ByteString]()
val inputStream = Source.fromPublisher(probe).runWith(StreamConverters.asInputStream())(mat) val inputStream = Source.fromPublisher(probe).runWith(StreamConverters.asInputStream())(mat)
mat.shutdown() mat.shutdown()

View file

@ -180,7 +180,7 @@ class OutputStreamSourceSpec extends StreamSpec(UnboundedMailboxConfig) {
} }
"not leave blocked threads when materializer shutdown" in { "not leave blocked threads when materializer shutdown" in {
val materializer2 = ActorMaterializer()(system) val materializer2 = Materializer(system)
val (_, probe) = val (_, probe) =
StreamConverters.asOutputStream(timeout).toMat(TestSink.probe[ByteString])(Keep.both).run()(materializer2) StreamConverters.asOutputStream(timeout).toMat(TestSink.probe[ByteString])(Keep.both).run()(materializer2)

View file

@ -513,12 +513,11 @@ class TcpSpec extends StreamSpec("""
""").withFallback(system.settings.config)) """).withFallback(system.settings.config))
try { try {
val mat2 = ActorMaterializer.create(system2) implicit val materializer = SystemMaterializer(system2).materializer
val serverAddress = temporaryServerAddress() val serverAddress = temporaryServerAddress()
val binding = Tcp(system2) val binding =
.bindAndHandle(Flow[ByteString], serverAddress.getHostString, serverAddress.getPort)(mat2) Tcp(system2).bindAndHandle(Flow[ByteString], serverAddress.getHostString, serverAddress.getPort).futureValue
.futureValue
val probe = TestProbe() val probe = TestProbe()
val testMsg = ByteString(0) val testMsg = ByteString(0)
@ -529,7 +528,7 @@ class TcpSpec extends StreamSpec("""
.via(Tcp(system2).outgoingConnection(serverAddress)) .via(Tcp(system2).outgoingConnection(serverAddress))
.runForeach { msg => .runForeach { msg =>
probe.ref ! msg probe.ref ! msg
}(mat2) }
// Ensure first that the actor is there // Ensure first that the actor is there
probe.expectMsg(testMsg) probe.expectMsg(testMsg)
@ -811,12 +810,11 @@ class TcpSpec extends StreamSpec("""
"not thrown on unbind after system has been shut down" in { "not thrown on unbind after system has been shut down" in {
val sys2 = ActorSystem("shutdown-test-system") val sys2 = ActorSystem("shutdown-test-system")
val mat2 = ActorMaterializer()(sys2) implicit val materializer = SystemMaterializer(sys2).materializer
try { try {
val address = temporaryServerAddress() 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 // Ensure server is running
bindingFuture.futureValue bindingFuture.futureValue

View file

@ -4,16 +4,19 @@
package akka.stream.scaladsl package akka.stream.scaladsl
import scala.concurrent.duration._ import akka.actor.Actor
import akka.actor.{ Actor, ActorRef, Props, Status } import akka.actor.ActorRef
import akka.stream.ActorMaterializer import akka.actor.Props
import akka.actor.Status
import akka.stream.Attributes.inputBuffer import akka.stream.Attributes.inputBuffer
import akka.stream.testkit.scaladsl.StreamTestKit._ import akka.stream.Materializer
import akka.stream.testkit._ import akka.stream.testkit._
import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.stream.testkit.scaladsl._ import akka.stream.testkit.scaladsl._
import akka.testkit.TestProbe import akka.testkit.TestProbe
import scala.concurrent.Promise import scala.concurrent.Promise
import scala.concurrent.duration._
object ActorRefBackpressureSinkSpec { object ActorRefBackpressureSinkSpec {
val initMessage = "start" val initMessage = "start"
@ -169,7 +172,7 @@ class ActorRefBackpressureSinkSpec extends StreamSpec {
} }
"signal failure on abrupt termination" in { "signal failure on abrupt termination" in {
val mat = ActorMaterializer() val mat = Materializer(system)
val probe = TestProbe() val probe = TestProbe()
val sink = Sink val sink = Sink

View file

@ -5,25 +5,25 @@
package akka.stream.scaladsl package akka.stream.scaladsl
import akka.Done 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.Utils._
import akka.stream.testkit._ import akka.stream.testkit._
import akka.stream.testkit.scaladsl.StreamTestKit._ import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.stream.testkit.scaladsl._ import akka.stream.testkit.scaladsl._
import akka.stream._
import scala.concurrent.duration._
import akka.actor.ActorRef
import org.reactivestreams.Publisher import org.reactivestreams.Publisher
import scala.concurrent.duration._
class ActorRefSourceSpec extends StreamSpec { class ActorRefSourceSpec extends StreamSpec {
"A ActorRefSource" must { "A ActorRefSource" must {
"emit received messages to the stream" in { "emit received messages to the stream" in {
val s = TestSubscriber.manualProbe[Int]() val s = TestSubscriber.manualProbe[Int]()
val materializer2 = ActorMaterializer() val ref = Source.actorRef(10, OverflowStrategy.fail).to(Sink.fromSubscriber(s)).run()
val ref = Source.actorRef(10, OverflowStrategy.fail).to(Sink.fromSubscriber(s)).run()(materializer2)
val sub = s.expectSubscription() val sub = s.expectSubscription()
sub.request(2) sub.request(2)
ref ! 1 ref ! 1
@ -211,7 +211,7 @@ class ActorRefSourceSpec extends StreamSpec {
"be possible to run immediately, reproducer of #26714" in { "be possible to run immediately, reproducer of #26714" in {
(1 to 100).foreach { _ => (1 to 100).foreach { _ =>
val mat = ActorMaterializer() val mat = Materializer(system)
val source: Source[String, ActorRef] = Source.actorRef[String](10000, OverflowStrategy.fail) val source: Source[String, ActorRef] = Source.actorRef[String](10000, OverflowStrategy.fail)
val (_: ActorRef, _: Publisher[String]) = val (_: ActorRef, _: Publisher[String]) =
source.toMat(Sink.asPublisher(false))(Keep.both).run()(mat) source.toMat(Sink.asPublisher(false))(Keep.both).run()(mat)

View file

@ -4,10 +4,10 @@
package akka.stream.scaladsl package akka.stream.scaladsl
import akka.stream.AbruptTerminationException
import akka.stream.Materializer
import akka.stream.testkit.StreamSpec import akka.stream.testkit.StreamSpec
import akka.stream.testkit.TestPublisher import akka.stream.testkit.TestPublisher
import akka.stream.AbruptTerminationException
import akka.stream.ActorMaterializer
import scala.collection.immutable import scala.collection.immutable
import scala.concurrent.Await import scala.concurrent.Await
@ -34,7 +34,7 @@ class CollectionSinkSpec extends StreamSpec("""
} }
"fail the future on abrupt termination" in { "fail the future on abrupt termination" in {
val mat = ActorMaterializer() val mat = Materializer(system)
val probe = TestPublisher.probe() val probe = TestPublisher.probe()
val future = Source.fromPublisher(probe).runWith(Sink.collection[Unit, Seq[Unit]])(mat) val future = Source.fromPublisher(probe).runWith(Sink.collection[Unit, Seq[Unit]])(mat)
mat.shutdown() mat.shutdown()
@ -56,14 +56,6 @@ class CollectionSinkSpec extends StreamSpec("""
val result: immutable.Vector[Int] = Await.result(future, remainingOrDefault) val result: immutable.Vector[Int] = Await.result(future, remainingOrDefault)
result should be(Vector.empty[Int]) 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]
}
} }
} }
} }

View file

@ -30,7 +30,7 @@ class FlowLogSpec extends StreamSpec("""
"A Log" must { "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 LogSrc = s"akka.stream.Log($supervisorPath)"
val LogClazz = classOf[Materializer] val LogClazz = classOf[Materializer]

View file

@ -4,12 +4,12 @@
package akka.stream.scaladsl package akka.stream.scaladsl
import akka.stream.FlowMonitorState
import akka.stream.FlowMonitorState._ import akka.stream.FlowMonitorState._
import akka.stream.Materializer
import akka.stream.testkit.StreamSpec import akka.stream.testkit.StreamSpec
import akka.stream.testkit.scaladsl.TestSink import akka.stream.testkit.scaladsl.TestSink
import akka.stream.testkit.scaladsl.TestSource import akka.stream.testkit.scaladsl.TestSource
import akka.stream.ActorMaterializer
import akka.stream.FlowMonitorState
import scala.concurrent.duration._ import scala.concurrent.duration._
@ -69,7 +69,7 @@ class FlowMonitorSpec extends StreamSpec {
} }
"return Failed when stream is abruptly terminated" in { "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 val (_, monitor) = // notice that `monitor` is like a Keep.both
TestSource.probe[Any].monitor.to(Sink.ignore).run()(mat) TestSource.probe[Any].monitor.to(Sink.ignore).run()(mat)
mat.shutdown() mat.shutdown()

View file

@ -5,15 +5,15 @@
package akka.stream.scaladsl package akka.stream.scaladsl
import akka.Done import akka.Done
import akka.stream.ActorMaterializer import akka.stream.Materializer
import akka.stream.testkit._ import akka.stream.testkit._
import akka.stream.testkit.scaladsl.StreamTestKit._ import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.testkit.TestProbe import akka.testkit.TestProbe
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.util.control.NoStackTrace
import scala.util.Failure import scala.util.Failure
import scala.util.Success import scala.util.Success
import scala.util.control.NoStackTrace
class FlowOnCompleteSpec extends StreamSpec(""" class FlowOnCompleteSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 2 akka.stream.materializer.initial-input-buffer-size = 2
@ -81,7 +81,7 @@ class FlowOnCompleteSpec extends StreamSpec("""
} }
"yield error on abrupt termination" in { "yield error on abrupt termination" in {
val mat = ActorMaterializer() val mat = Materializer(system)
val onCompleteProbe = TestProbe() val onCompleteProbe = TestProbe()
val p = TestPublisher.manualProbe[Int]() val p = TestPublisher.manualProbe[Int]()
Source.fromPublisher(p).to(Sink.onComplete[Int](onCompleteProbe.ref ! _)).run()(mat) Source.fromPublisher(p).to(Sink.onComplete[Int](onCompleteProbe.ref ! _)).run()(mat)

View file

@ -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 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 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) (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) (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]) = { def materializeIntoSubscriberAndPublisher[In, Out](flow: Flow[In, Out, _]): (Subscriber[In], Publisher[Out]) = {

View file

@ -68,7 +68,7 @@ class FlowWatchTerminationSpec extends StreamSpec {
} }
"fail future when stream abruptly terminated" in { "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) val (_, future) = TestSource.probe[Int].watchTermination()(Keep.both).to(Sink.ignore).run()(mat)
mat.shutdown() mat.shutdown()

View file

@ -24,7 +24,7 @@ class FlowWithContextLogSpec extends StreamSpec("""
"log() from FlowWithContextOps" must { "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 LogSrc = s"akka.stream.Log($supervisorPath)"
val LogClazz = classOf[Materializer] val LogClazz = classOf[Materializer]

View file

@ -0,0 +1,230 @@
/*
* Copyright (C) 2019 Lightbend Inc. <https://www.lightbend.com>
*/
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
}
}
}

View file

@ -4,10 +4,10 @@
package akka.stream.scaladsl package akka.stream.scaladsl
import akka.stream.AbruptTerminationException
import akka.stream.Materializer
import akka.stream.testkit._ import akka.stream.testkit._
import akka.stream.testkit.scaladsl.StreamTestKit._ import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.stream.AbruptTerminationException
import akka.stream.ActorMaterializer
import scala.concurrent.Await import scala.concurrent.Await
import scala.concurrent.Future import scala.concurrent.Future
@ -81,7 +81,7 @@ class HeadSinkSpec extends StreamSpec("""
} }
"fail on abrupt termination" in { "fail on abrupt termination" in {
val mat = ActorMaterializer() val mat = Materializer(system)
val source = TestPublisher.probe() val source = TestPublisher.probe()
val f = Source.fromPublisher(source).runWith(Sink.headOption)(mat) val f = Source.fromPublisher(source).runWith(Sink.headOption)(mat)
mat.shutdown() mat.shutdown()

View file

@ -4,11 +4,11 @@
package akka.stream.scaladsl 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.StreamSpec
import akka.stream.testkit.TestSubscriber import akka.stream.testkit.TestSubscriber
import akka.stream.AbruptStageTerminationException import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.stream.ActorMaterializer
import akka.testkit.DefaultTimeout import akka.testkit.DefaultTimeout
import scala.concurrent.duration._ import scala.concurrent.duration._
@ -86,7 +86,7 @@ class MaybeSourceSpec extends StreamSpec with DefaultTimeout {
} }
"complete materialized future when materializer is shutdown" in assertAllStagesStopped { "complete materialized future when materializer is shutdown" in assertAllStagesStopped {
val mat = ActorMaterializer() val mat = Materializer(system)
val neverSource = Source.maybe[Int] val neverSource = Source.maybe[Int]
val pubSink = Sink.asPublisher[Int](false) val pubSink = Sink.asPublisher[Int](false)

View file

@ -9,9 +9,12 @@ import akka.actor.Status
import akka.pattern.pipe import akka.pattern.pipe
import akka.stream._ import akka.stream._
import akka.stream.impl.QueueSource 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.StreamTestKit._
import akka.stream.testkit.scaladsl.TestSink import akka.stream.testkit.scaladsl.TestSink
import akka.stream.testkit.{ GraphStageMessages, StreamSpec, TestSourceStage, TestSubscriber }
import akka.testkit.TestProbe import akka.testkit.TestProbe
import org.scalatest.time.Span import org.scalatest.time.Span
@ -19,7 +22,6 @@ import scala.concurrent._
import scala.concurrent.duration._ import scala.concurrent.duration._
class QueueSourceSpec extends StreamSpec { class QueueSourceSpec extends StreamSpec {
implicit val materializer = ActorMaterializer()
implicit val ec = system.dispatcher implicit val ec = system.dispatcher
val pause = 300.millis val pause = 300.millis
@ -190,7 +192,7 @@ class QueueSourceSpec extends StreamSpec {
} }
"complete watching future with failure if materializer shut down" in assertAllStagesStopped { "complete watching future with failure if materializer shut down" in assertAllStagesStopped {
val tempMap = ActorMaterializer() val tempMap = Materializer(system)
val s = TestSubscriber.manualProbe[Int]() val s = TestSubscriber.manualProbe[Int]()
val queue = Source.queue(1, OverflowStrategy.fail).to(Sink.fromSubscriber(s)).run()(tempMap) val queue = Source.queue(1, OverflowStrategy.fail).to(Sink.fromSubscriber(s)).run()(tempMap)
queue.watchCompletion().pipeTo(testActor) queue.watchCompletion().pipeTo(testActor)

View file

@ -4,10 +4,10 @@
package akka.stream.scaladsl package akka.stream.scaladsl
import akka.stream.AbruptTerminationException
import akka.stream.Materializer
import akka.stream.testkit.StreamSpec import akka.stream.testkit.StreamSpec
import akka.stream.testkit.TestPublisher import akka.stream.testkit.TestPublisher
import akka.stream.AbruptTerminationException
import akka.stream.ActorMaterializer
import scala.collection.immutable import scala.collection.immutable
import scala.concurrent.Await import scala.concurrent.Await
@ -33,7 +33,7 @@ class SeqSinkSpec extends StreamSpec("""
} }
"fail the future on abrupt termination" in { "fail the future on abrupt termination" in {
val mat = ActorMaterializer() val mat = Materializer(system)
val probe = TestPublisher.probe() val probe = TestPublisher.probe()
val future: Future[immutable.Seq[Int]] = val future: Future[immutable.Seq[Int]] =
Source.fromPublisher(probe).runWith(Sink.seq)(mat) Source.fromPublisher(probe).runWith(Sink.seq)(mat)

View file

@ -6,7 +6,9 @@ package akka.stream.scaladsl
import akka.NotUsed import akka.NotUsed
import akka.stream.testkit.StreamSpec import akka.stream.testkit.StreamSpec
import com.github.ghik.silencer.silent
@silent("deprecated")
class SetupSpec extends StreamSpec { class SetupSpec extends StreamSpec {
import system.dispatcher import system.dispatcher

View file

@ -6,11 +6,10 @@ package akka.stream.scaladsl
import java.util.stream.Collectors import java.util.stream.Collectors
import akka.actor.ActorSystem
import akka.stream._ import akka.stream._
import akka.stream.impl.StreamSupervisor.Children
import akka.stream.impl.PhasedFusingActorMaterializer import akka.stream.impl.PhasedFusingActorMaterializer
import akka.stream.impl.StreamSupervisor import akka.stream.impl.StreamSupervisor
import akka.stream.impl.StreamSupervisor.Children
import akka.stream.testkit.Utils._ import akka.stream.testkit.Utils._
import akka.stream.testkit._ import akka.stream.testkit._
import akka.stream.testkit.scaladsl.StreamTestKit._ import akka.stream.testkit.scaladsl.StreamTestKit._
@ -50,37 +49,25 @@ class SinkAsJavaStreamSpec extends StreamSpec(UnboundedMailboxConfig) {
} }
"allow overriding the dispatcher using Attributes" in assertAllStagesStopped { "allow overriding the dispatcher using Attributes" in assertAllStagesStopped {
val sys = ActorSystem("dispatcher-testing", UnboundedMailboxConfig) val probe = TestSource
val materializer = ActorMaterializer()(sys) .probe[ByteString]
.to(StreamConverters.asJavaStream().addAttributes(ActorAttributes.dispatcher("akka.actor.default-dispatcher")))
try { .run()
TestSource SystemMaterializer(system).materializer
.probe[ByteString] .asInstanceOf[PhasedFusingActorMaterializer]
.runWith( .supervisor
StreamConverters.asJavaStream().addAttributes(ActorAttributes.dispatcher("akka.actor.default-dispatcher")))( .tell(StreamSupervisor.GetChildren, testActor)
materializer) val ref = expectMsgType[Children].children.find(_.path.toString contains "asJavaStream").get
materializer assertDispatcher(ref, "akka.actor.default-dispatcher")
.asInstanceOf[PhasedFusingActorMaterializer] probe.sendComplete()
.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)
} }
"work in separate IO dispatcher" in assertAllStagesStopped { "work in separate IO dispatcher" in assertAllStagesStopped {
val sys = ActorSystem("dispatcher-testing", UnboundedMailboxConfig) val materializer = Materializer.create(system)
val materializer = ActorMaterializer()(sys) TestSource.probe[ByteString].runWith(StreamConverters.asJavaStream())(materializer)
materializer.asInstanceOf[PhasedFusingActorMaterializer].supervisor.tell(StreamSupervisor.GetChildren, testActor)
try { val ref = expectMsgType[Children].children.find(_.path.toString contains "asJavaStream").get
TestSource.probe[ByteString].runWith(StreamConverters.asJavaStream())(materializer) assertDispatcher(ref, ActorAttributes.IODispatcher.dispatcher)
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)
} }
} }
} }

View file

@ -7,19 +7,24 @@ package akka.stream.scaladsl
import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.atomic.AtomicInteger
import akka.Done import akka.Done
import akka.actor.ActorSystem
import akka.stream.ActorAttributes import akka.stream.ActorAttributes
import akka.stream.Materializer
import akka.stream.Supervision
import akka.stream.impl.StreamSupervisor.Children 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.Utils._
import akka.stream.testkit.scaladsl.StreamTestKit._ import akka.stream.testkit.scaladsl.StreamTestKit._
import akka.stream.testkit.{ StreamSpec, TestSubscriber } import akka.stream.testkit.StreamSpec
import akka.stream.ActorMaterializer import akka.stream.testkit.TestSubscriber
import akka.stream.Supervision import akka.testkit.TestLatch
import akka.testkit.{ TestLatch, TestProbe } import akka.testkit.TestProbe
import scala.concurrent.duration._ 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 { object UnfoldResourceAsyncSourceSpec {
@ -62,7 +67,6 @@ object UnfoldResourceAsyncSourceSpec {
class UnfoldResourceAsyncSourceSpec extends StreamSpec(UnboundedMailboxConfig) { class UnfoldResourceAsyncSourceSpec extends StreamSpec(UnboundedMailboxConfig) {
import UnfoldResourceAsyncSourceSpec._ import UnfoldResourceAsyncSourceSpec._
import system.dispatcher import system.dispatcher
"Unfold Resource Async Source" must { "Unfold Resource Async Source" must {
@ -310,29 +314,25 @@ class UnfoldResourceAsyncSourceSpec extends StreamSpec(UnboundedMailboxConfig) {
} }
"use dedicated blocking-io-dispatcher by default" in assertAllStagesStopped { "use dedicated blocking-io-dispatcher by default" in assertAllStagesStopped {
val sys = ActorSystem("dispatcher-testing", UnboundedMailboxConfig) // use a separate materializer to ensure we know what child is our stream
val materializer = ActorMaterializer()(sys) implicit val materializer = Materializer(system)
try {
Source
.unfoldResourceAsync[String, Unit](
() => Promise[Unit].future, // never complete
_ => ???,
_ => ???)
.runWith(Sink.ignore)(materializer)
materializer Source
.asInstanceOf[PhasedFusingActorMaterializer] .unfoldResourceAsync[String, Unit](
.supervisor () => Promise[Unit].future, // never complete
.tell(StreamSupervisor.GetChildren, testActor) _ => ???,
val ref = expectMsgType[Children].children.find(_.path.toString contains "unfoldResourceSourceAsync").get _ => ???)
assertDispatcher(ref, ActorAttributes.IODispatcher.dispatcher) .runWith(Sink.ignore)
} finally shutdown(sys)
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 { "close resource when stream is abruptly terminated" in {
import system.dispatcher import system.dispatcher
val closeLatch = TestLatch(1) val closeLatch = TestLatch(1)
val mat = ActorMaterializer() val mat = Materializer(system)
val p = Source val p = Source
.unfoldResourceAsync[String, Unit]( .unfoldResourceAsync[String, Unit](
() => Future.successful(()), () => Future.successful(()),

View file

@ -4,10 +4,17 @@
package akka.stream.snapshot package akka.stream.snapshot
import akka.stream.{ ActorMaterializer, FlowShape } import akka.stream.FlowShape
import akka.stream.scaladsl.{ Flow, GraphDSL, Keep, Merge, Partition, Sink, Source } import akka.stream.Materializer
import akka.stream.testkit.scaladsl.TestSink 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.StreamSpec
import akka.stream.testkit.scaladsl.TestSink
import scala.concurrent.Promise import scala.concurrent.Promise
@ -16,7 +23,7 @@ class MaterializerStateSpec extends StreamSpec {
"The MaterializerSnapshotting" must { "The MaterializerSnapshotting" must {
"snapshot a running stream" in { "snapshot a running stream" in {
implicit val mat = ActorMaterializer() implicit val mat = Materializer(system)
try { try {
Source.maybe[Int].map(_.toString).zipWithIndex.runWith(Sink.seq) 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 { "snapshot a stream that has a stopped stage" in {
implicit val mat = ActorMaterializer() implicit val mat = Materializer(system)
try { try {
val probe = TestSink.probe[String](system) val probe = TestSink.probe[String](system)
val out = Source val out = Source
@ -68,7 +75,7 @@ class MaterializerStateSpec extends StreamSpec {
} }
"snapshot a more complicated graph" in { "snapshot a more complicated graph" in {
implicit val mat = ActorMaterializer() implicit val mat = Materializer(system)
try { try {
// snapshot before anything is running // snapshot before anything is running
MaterializerState.streamSnapshots(mat).futureValue MaterializerState.streamSnapshots(mat).futureValue

View file

@ -1,83 +0,0 @@
/*
* Copyright (C) 2018-2019 Lightbend Inc. <https://www.lightbend.com>
*/
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)
}

View file

@ -1,9 +0,0 @@
/*
* Copyright (C) 2018-2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.stream.typed
package object scaladsl {
type ActorMaterializer = akka.stream.ActorMaterializer
}

View file

@ -1,48 +0,0 @@
/*
* Copyright (C) 2018-2019 Lightbend Inc. <https://www.lightbend.com>
*/
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)
}

View file

@ -29,7 +29,6 @@ public class ActorFlowCompileTest {
{ {
final ActorSystem<String> system = null; final ActorSystem<String> system = null;
final ActorMaterializer mat = ActorMaterializerFactory.create(system);
} }
static static

View file

@ -30,7 +30,6 @@ public class ActorSourceSinkCompileTest {
{ {
final ActorSystem<String> system = null; final ActorSystem<String> system = null;
final ActorMaterializer mat = ActorMaterializerFactory.create(system);
} }
{ {

View file

@ -0,0 +1,76 @@
/*
* Copyright (C) 2009-2019 Lightbend Inc. <https://www.lightbend.com>
*/
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<String> 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<String> result = Source.single("hello").runWith(Sink.head(), materializer);
assertEquals("hello", result.toCompletableFuture().get(3, TimeUnit.SECONDS));
}
private static Behavior<String> actorStreamBehavior(ActorRef<Object> probe) {
return Behaviors.setup(
(context) -> {
Materializer materializer = Materializer.create(context);
CompletionStage<Done> done = Source.repeat("hello").runWith(Sink.ignore(), materializer);
done.whenComplete(
(success, failure) -> {
if (success != null) probe.tell(success);
else probe.tell(failure);
});
return Behaviors.receive(String.class)
.onMessageEquals("stop", () -> Behaviors.stopped())
.build();
});
}
@Test
public void createCustomActorLevelMaterializer() throws Exception {
TestProbe<Object> probe = testKit.createTestProbe();
ActorRef<String> actor = testKit.spawn(actorStreamBehavior(probe.getRef()));
actor.tell("stop");
probe.expectMessageClass(AbruptStageTerminationException.class);
}
}

View file

@ -25,8 +25,6 @@ object ActorSourceSinkSpec {
class ActorSourceSinkSpec extends ScalaTestWithActorTestKit with WordSpecLike { class ActorSourceSinkSpec extends ScalaTestWithActorTestKit with WordSpecLike {
import ActorSourceSinkSpec._ import ActorSourceSinkSpec._
implicit val mat = ActorMaterializer()
"ActorSink" should { "ActorSink" should {
"accept messages" in { "accept messages" in {

View file

@ -1,44 +0,0 @@
/*
* Copyright (C) 2017-2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.stream.typed.scaladsl
import akka.Done
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.typed.scaladsl.Behaviors
import akka.stream.AbruptStageTerminationException
import akka.stream.scaladsl.{ Sink, Source }
import org.scalatest.WordSpecLike
import scala.concurrent.Future
class CustomGuardianAndMaterializerSpec extends ScalaTestWithActorTestKit with WordSpecLike {
implicit val mat = ActorMaterializer()
"ActorMaterializer" should {
"work with typed ActorSystem with custom guardian" in {
val it: Future[String] = Source.single("hello").runWith(Sink.head)
it.futureValue should ===("hello")
}
"should kill streams with bound actor context" in {
var doneF: Future[Done] = null
val behavior =
Behaviors.setup[String] { ctx =>
implicit val mat: ActorMaterializer = ActorMaterializer.boundToActor(ctx)
doneF = Source.repeat("hello").runWith(Sink.ignore)
Behaviors.receiveMessage[String](_ => Behaviors.stopped)
}
val actorRef = spawn(behavior)
actorRef ! "kill"
eventually(doneF.failed.futureValue shouldBe an[AbruptStageTerminationException])
}
}
}

View file

@ -0,0 +1,68 @@
/*
* Copyright (C) 2017-2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.stream.typed.scaladsl
import akka.Done
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.typed.scaladsl.Behaviors
import akka.stream.AbruptStageTerminationException
import akka.stream.Materializer
import akka.stream.scaladsl.Sink
import akka.stream.scaladsl.Source
import org.scalatest.WordSpecLike
import scala.concurrent.Future
import scala.util.Success
class MaterializerForTypedSpec extends ScalaTestWithActorTestKit with WordSpecLike {
"Materialization in typed" should {
"use system materializer by default" in {
val it: Future[String] = Source.single("hello").runWith(Sink.head)
it.futureValue should ===("hello")
}
"allow for custom instances for special cases" in {
val customMaterializer = Materializer(system)
val it: Future[String] = Source.single("hello").runWith(Sink.head)(customMaterializer)
it.futureValue should ===("hello")
}
"allow for actor context bound instances" in {
val probe = testKit.createTestProbe[Any]()
val actor = testKit.spawn(Behaviors.setup[String] { context =>
val materializerForActor = Materializer(context)
Behaviors.receiveMessage[String] {
case "run" =>
val f = Source.single("hello").runWith(Sink.head)(materializerForActor)
f.onComplete(probe.ref ! _)(system.executionContext)
Behaviors.same
}
})
actor ! "run"
probe.expectMessage(Success("hello"))
}
"should kill streams with bound actor context" in {
var doneF: Future[Done] = null
val behavior =
Behaviors.setup[String] { ctx =>
implicit val mat: Materializer = Materializer(ctx)
doneF = Source.repeat("hello").runWith(Sink.ignore)
Behaviors.receiveMessage[String](_ => Behaviors.stopped)
}
val actorRef = spawn(behavior)
actorRef ! "kill"
eventually(doneF.failed.futureValue shouldBe an[AbruptStageTerminationException])
}
}
}

View file

@ -161,6 +161,7 @@ ProblemFilters.exclude[Problem]("akka.stream.StreamRefMessages*")
ProblemFilters.exclude[Problem]("akka.stream.impl.*") ProblemFilters.exclude[Problem]("akka.stream.impl.*")
# #25559 simplifying materialization # #25559 simplifying materialization
# internals
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.impl.ActorProcessorImpl.settings") ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.impl.ActorProcessorImpl.settings")
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.impl.ActorProcessorImpl.this") ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.impl.ActorProcessorImpl.this")
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.impl.FanOut.settings") ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.impl.FanOut.settings")
@ -210,3 +211,17 @@ ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.stream.impl.FanoutProc
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.stream.impl.FanoutProcessorImpl.props") ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.stream.impl.FanoutProcessorImpl.props")
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.stream.impl.GraphStageIsland.this") ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.stream.impl.GraphStageIsland.this")
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.stream.impl.fusing.GraphInterpreterShell.this") ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.stream.impl.fusing.GraphInterpreterShell.this")
# internal and only used in one test
ProblemFilters.exclude[MissingClassProblem]("akka.stream.NoMaterializer$")
ProblemFilters.exclude[MissingClassProblem]("akka.stream.NoMaterializer")
# Materializer not for user extension
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.stream.Materializer.supervisor")
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.stream.Materializer.logger")
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.stream.Materializer.settings")
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.stream.Materializer.actorOf")
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.stream.Materializer.isShutdown")
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.stream.Materializer.system")
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.stream.Materializer.shutdown")
# StageActor was never meant to be constructed by users
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.stage.GraphStageLogic#StageActor.this")
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.stream.stage.GraphStageLogic#StageActor.this")

View file

@ -7,18 +7,23 @@ package akka.stream
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.atomic.AtomicBoolean
import akka.actor.{ ActorContext, ActorRef, ActorRefFactory, ActorSystem, ExtendedActorSystem, Props } import akka.actor.ActorContext
import akka.actor.ActorRef
import akka.actor.ActorRefFactory
import akka.actor.ActorSystem
import akka.actor.ExtendedActorSystem
import akka.actor.Props
import akka.annotation.InternalApi import akka.annotation.InternalApi
import akka.event.LoggingAdapter import akka.event.LoggingAdapter
import akka.util.Helpers.toRootLowerCase import akka.japi.function
import akka.stream.impl._ import akka.stream.impl._
import com.typesafe.config.{ Config, ConfigFactory } import akka.stream.stage.GraphStageLogic
import akka.util.Helpers.toRootLowerCase
import com.github.ghik.silencer.silent
import com.typesafe.config.Config
import com.typesafe.config.ConfigFactory
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.japi.function
import akka.stream.stage.GraphStageLogic
import com.github.ghik.silencer.silent
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace
object ActorMaterializer { object ActorMaterializer {
@ -36,7 +41,9 @@ object ActorMaterializer {
* the processing steps. The default `namePrefix` is `"flow"`. The actor names are built up of * the processing steps. The default `namePrefix` is `"flow"`. The actor names are built up of
* `namePrefix-flowNumber-flowStepNumber-stepName`. * `namePrefix-flowNumber-flowStepNumber-stepName`.
*/ */
@silent("deprecated") @deprecated(
"Use the system wide materializer with stream attributes or configuration settings to change defaults",
"2.6.0")
def apply(materializerSettings: Option[ActorMaterializerSettings] = None, namePrefix: Option[String] = None)( def apply(materializerSettings: Option[ActorMaterializerSettings] = None, namePrefix: Option[String] = None)(
implicit context: ActorRefFactory): ActorMaterializer = { implicit context: ActorRefFactory): ActorMaterializer = {
val system = actorSystemOf(context) val system = actorSystemOf(context)
@ -57,6 +64,9 @@ object ActorMaterializer {
* the processing steps. The default `namePrefix` is `"flow"`. The actor names are built up of * the processing steps. The default `namePrefix` is `"flow"`. The actor names are built up of
* `namePrefix-flowNumber-flowStepNumber-stepName`. * `namePrefix-flowNumber-flowStepNumber-stepName`.
*/ */
@deprecated(
"Use the system wide materializer with stream attributes or configuration settings to change defaults",
"2.6.0")
def apply(materializerSettings: ActorMaterializerSettings, namePrefix: String)( def apply(materializerSettings: ActorMaterializerSettings, namePrefix: String)(
implicit context: ActorRefFactory): ActorMaterializer = { implicit context: ActorRefFactory): ActorMaterializer = {
val haveShutDown = new AtomicBoolean(false) val haveShutDown = new AtomicBoolean(false)
@ -93,6 +103,9 @@ object ActorMaterializer {
* the processing steps. The default `namePrefix` is `"flow"`. The actor names are built up of * the processing steps. The default `namePrefix` is `"flow"`. The actor names are built up of
* `namePrefix-flowNumber-flowStepNumber-stepName`. * `namePrefix-flowNumber-flowStepNumber-stepName`.
*/ */
@deprecated(
"Use the system wide materializer or Materializer.apply(actorContext) with stream attributes or configuration settings to change defaults",
"2.6.0")
def apply(materializerSettings: ActorMaterializerSettings)(implicit context: ActorRefFactory): ActorMaterializer = def apply(materializerSettings: ActorMaterializerSettings)(implicit context: ActorRefFactory): ActorMaterializer =
apply(Some(materializerSettings), None) apply(Some(materializerSettings), None)
@ -126,6 +139,9 @@ object ActorMaterializer {
* Defaults the actor name prefix used to name actors running the processing steps to `"flow"`. * 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`. * The actor names are built up of `namePrefix-flowNumber-flowStepNumber-stepName`.
*/ */
@deprecated(
"Use the system wide materializer or Materializer.create(actorContext) with stream attributes or configuration settings to change defaults",
"2.6.0")
def create(context: ActorRefFactory): ActorMaterializer = def create(context: ActorRefFactory): ActorMaterializer =
apply()(context) apply()(context)
@ -141,7 +157,9 @@ object ActorMaterializer {
* the processing steps. The default `namePrefix` is `"flow"`. The actor names are built up of * the processing steps. The default `namePrefix` is `"flow"`. The actor names are built up of
* `namePrefix-flowNumber-flowStepNumber-stepName`. * `namePrefix-flowNumber-flowStepNumber-stepName`.
*/ */
@silent("deprecated") @deprecated(
"Use the system wide materializer or Materializer.create(actorContext) with stream attributes or configuration settings to change defaults",
"2.6.0")
def create(context: ActorRefFactory, namePrefix: String): ActorMaterializer = { def create(context: ActorRefFactory, namePrefix: String): ActorMaterializer = {
val system = actorSystemOf(context) val system = actorSystemOf(context)
val settings = ActorMaterializerSettings(system) val settings = ActorMaterializerSettings(system)
@ -155,6 +173,9 @@ object ActorMaterializer {
* (which can be either an [[akka.actor.ActorSystem]] or an [[akka.actor.ActorContext]]) * (which can be either an [[akka.actor.ActorSystem]] or an [[akka.actor.ActorContext]])
* will be used to create one actor that in turn creates actors for the transformation steps. * will be used to create one actor that in turn creates actors for the transformation steps.
*/ */
@deprecated(
"Use the system wide materializer or Materializer.create(actorContext) with stream attributes or configuration settings to change defaults",
"2.6.0")
def create(settings: ActorMaterializerSettings, context: ActorRefFactory): ActorMaterializer = def create(settings: ActorMaterializerSettings, context: ActorRefFactory): ActorMaterializer =
apply(Option(settings), None)(context) apply(Option(settings), None)(context)
@ -170,6 +191,9 @@ object ActorMaterializer {
* the processing steps. The default `namePrefix` is `"flow"`. The actor names are built up of * the processing steps. The default `namePrefix` is `"flow"`. The actor names are built up of
* `namePrefix-flowNumber-flowStepNumber-stepName`. * `namePrefix-flowNumber-flowStepNumber-stepName`.
*/ */
@deprecated(
"Use the system wide materializer or Materializer.create(actorContext) with stream attributes or configuration settings to change defaults",
"2.6.0")
def create(settings: ActorMaterializerSettings, context: ActorRefFactory, namePrefix: String): ActorMaterializer = def create(settings: ActorMaterializerSettings, context: ActorRefFactory, namePrefix: String): ActorMaterializer =
apply(Option(settings), Option(namePrefix))(context) apply(Option(settings), Option(namePrefix))(context)
@ -195,8 +219,9 @@ private[akka] object ActorMaterializerHelper {
/** /**
* INTERNAL API * INTERNAL API
*/ */
@deprecated("The Materializer now has all methods the ActorMaterializer used to have", "2.6.0")
private[akka] def downcast(materializer: Materializer): ActorMaterializer = private[akka] def downcast(materializer: Materializer): ActorMaterializer =
materializer match { //FIXME this method is going to cause trouble for other Materializer implementations materializer match {
case m: ActorMaterializer => m case m: ActorMaterializer => m
case _ => case _ =>
throw new IllegalArgumentException( throw new IllegalArgumentException(
@ -208,6 +233,7 @@ private[akka] object ActorMaterializerHelper {
/** /**
* An ActorMaterializer takes a stream blueprint and turns it into a running stream. * An ActorMaterializer takes a stream blueprint and turns it into a running stream.
*/ */
@deprecated("The Materializer now has all methods the ActorMaterializer used to have", "2.6.0")
abstract class ActorMaterializer extends Materializer with MaterializerLoggingProvider { abstract class ActorMaterializer extends Materializer with MaterializerLoggingProvider {
@deprecated( @deprecated(

View file

@ -28,7 +28,7 @@ import scala.concurrent.duration.FiniteDuration
* Holds attributes which can be used to alter [[akka.stream.scaladsl.Flow]] / [[akka.stream.javadsl.Flow]] * Holds attributes which can be used to alter [[akka.stream.scaladsl.Flow]] / [[akka.stream.javadsl.Flow]]
* or [[akka.stream.scaladsl.GraphDSL]] / [[akka.stream.javadsl.GraphDSL]] materialization. * or [[akka.stream.scaladsl.GraphDSL]] / [[akka.stream.javadsl.GraphDSL]] materialization.
* *
* Note that more attributes for the [[ActorMaterializer]] are defined in [[ActorAttributes]]. * Note that more attributes for the [[Materializer]] are defined in [[ActorAttributes]].
* *
* The ``attributeList`` is ordered with the most specific attribute first, least specific last. * The ``attributeList`` is ordered with the most specific attribute first, least specific last.
* Note that the order was the opposite in Akka 2.4.x. * Note that the order was the opposite in Akka 2.4.x.
@ -282,7 +282,7 @@ final case class Attributes(attributeList: List[Attributes.Attribute] = Nil) {
} }
/** /**
* Note that more attributes for the [[ActorMaterializer]] are defined in [[ActorAttributes]]. * Note that more attributes for the [[Materializer]] are defined in [[ActorAttributes]].
*/ */
object Attributes { object Attributes {
@ -418,7 +418,7 @@ object Attributes {
} }
/** /**
* Attributes for the [[ActorMaterializer]]. * Attributes for the [[Materializer]].
* Note that more attributes defined in [[Attributes]]. * Note that more attributes defined in [[Attributes]].
*/ */
object ActorAttributes { object ActorAttributes {

Some files were not shown because too many files have changed in this diff Show more