Deprecate actor materializer (#27538)
This commit is contained in:
parent
8765a4fbe6
commit
b68d67008a
118 changed files with 1233 additions and 731 deletions
|
|
@ -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
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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] =
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -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`.
|
||||||
|
|
|
||||||
|
|
@ -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")
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
}
|
||||||
|
|
|
||||||
|
|
@ -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")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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] = _
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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")
|
||||||
|
|
|
||||||
|
|
@ -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")
|
||||||
|
|
|
||||||
|
|
@ -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 {
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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 ActorSystem’s explicit lifecycle the JVM will not exit until it is s
|
||||||
Therefore it is necessary to shutdown all ActorSystems within a running application or
|
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
|
||||||
|
|
|
||||||
|
|
@ -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.
|
||||||
|
|
|
||||||
|
|
@ -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`.
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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`.
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
|
|
|
||||||
|
|
@ -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`.
|
||||||
|
|
||||||
@@@
|
@@@
|
||||||
|
|
|
||||||
|
|
@ -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.
|
||||||
|
|
|
||||||
|
|
@ -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;
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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);
|
||||||
|
|
|
||||||
|
|
@ -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 {
|
||||||
|
|
|
||||||
|
|
@ -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);
|
||||||
|
|
|
||||||
|
|
@ -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 {
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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;
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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")
|
||||||
|
|
|
||||||
|
|
@ -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:
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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 }
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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()
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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)
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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 {
|
||||||
|
|
|
||||||
|
|
@ -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 = {
|
||||||
|
|
|
||||||
|
|
@ -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 {
|
||||||
|
|
|
||||||
|
|
@ -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(
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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) = {
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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 =
|
||||||
|
|
|
||||||
|
|
@ -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]()
|
||||||
|
|
|
||||||
|
|
@ -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 {
|
||||||
|
|
|
||||||
|
|
@ -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!")
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
|
|
|
||||||
|
|
@ -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;
|
||||||
|
|
|
||||||
|
|
@ -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] {
|
||||||
|
|
|
||||||
|
|
@ -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 {
|
||||||
|
|
|
||||||
|
|
@ -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()
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
|
|
|
||||||
|
|
@ -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]
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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]
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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()
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
|
|
|
||||||
|
|
@ -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]) = {
|
||||||
|
|
|
||||||
|
|
@ -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()
|
||||||
|
|
|
||||||
|
|
@ -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]
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
@ -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()
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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)
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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(()),
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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)
|
|
||||||
}
|
|
||||||
|
|
@ -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
|
|
||||||
}
|
|
||||||
|
|
@ -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)
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -30,7 +30,6 @@ public class ActorSourceSinkCompileTest {
|
||||||
|
|
||||||
{
|
{
|
||||||
final ActorSystem<String> system = null;
|
final ActorSystem<String> system = null;
|
||||||
final ActorMaterializer mat = ActorMaterializerFactory.create(system);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
{
|
{
|
||||||
|
|
|
||||||
|
|
@ -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);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -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 {
|
||||||
|
|
|
||||||
|
|
@ -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])
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
@ -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])
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -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")
|
||||||
|
|
@ -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(
|
||||||
|
|
|
||||||
|
|
@ -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
Loading…
Add table
Add a link
Reference in a new issue