fixes #20414 Allow different ActorMaterializer subtypes
This commit is contained in:
parent
47c1b5b9ad
commit
049b95077f
21 changed files with 114 additions and 69 deletions
|
|
@ -609,7 +609,7 @@ private[http] object HttpServerBluePrint {
|
|||
})
|
||||
|
||||
private var activeTimers = 0
|
||||
private def timeout = ActorMaterializer.downcast(materializer).settings.subscriptionTimeoutSettings.timeout
|
||||
private def timeout = ActorMaterializerHelper.downcast(materializer).settings.subscriptionTimeoutSettings.timeout
|
||||
private def addTimeout(s: SubscriptionTimeout): Unit = {
|
||||
if (activeTimers == 0) setKeepGoing(true)
|
||||
activeTimers += 1
|
||||
|
|
|
|||
|
|
@ -4,11 +4,11 @@
|
|||
|
||||
package akka.http.scaladsl.server
|
||||
|
||||
import scala.concurrent.{ Future, ExecutionContextExecutor }
|
||||
import akka.stream.{ ActorMaterializer, Materializer }
|
||||
import scala.concurrent.{ExecutionContextExecutor, Future}
|
||||
import akka.stream.{ActorMaterializer, ActorMaterializerHelper, Materializer}
|
||||
import akka.event.LoggingAdapter
|
||||
import akka.http.scaladsl.settings.{ RoutingSettings, ParserSettings }
|
||||
import akka.http.scaladsl.marshalling.{ Marshal, ToResponseMarshallable }
|
||||
import akka.http.scaladsl.settings.{ParserSettings, RoutingSettings}
|
||||
import akka.http.scaladsl.marshalling.{Marshal, ToResponseMarshallable}
|
||||
import akka.http.scaladsl.model._
|
||||
import akka.http.scaladsl.util.FastFuture
|
||||
import akka.http.scaladsl.util.FastFuture._
|
||||
|
|
@ -29,7 +29,7 @@ private[http] class RequestContextImpl(
|
|||
this(request, request.uri.path, ec, materializer, log, settings, parserSettings)
|
||||
|
||||
def this(request: HttpRequest, log: LoggingAdapter, settings: RoutingSettings)(implicit ec: ExecutionContextExecutor, materializer: Materializer) =
|
||||
this(request, request.uri.path, ec, materializer, log, settings, ParserSettings(ActorMaterializer.downcast(materializer).system))
|
||||
this(request, request.uri.path, ec, materializer, log, settings, ParserSettings(ActorMaterializerHelper.downcast(materializer).system))
|
||||
|
||||
def reconfigure(executionContext: ExecutionContextExecutor, materializer: Materializer, log: LoggingAdapter, settings: RoutingSettings): RequestContext =
|
||||
copy(executionContext = executionContext, materializer = materializer, log = log, routingSettings = settings)
|
||||
|
|
|
|||
|
|
@ -5,12 +5,12 @@
|
|||
package akka.http.scaladsl.server
|
||||
|
||||
import akka.NotUsed
|
||||
import akka.http.scaladsl.settings.{ RoutingSettings, ParserSettings }
|
||||
import akka.stream.{ ActorMaterializer, Materializer }
|
||||
import akka.http.scaladsl.settings.{ParserSettings, RoutingSettings}
|
||||
import akka.stream.{ActorMaterializer, ActorMaterializerHelper, Materializer}
|
||||
|
||||
import scala.concurrent.{ ExecutionContextExecutor, Future }
|
||||
import scala.concurrent.{ExecutionContextExecutor, Future}
|
||||
import akka.stream.scaladsl.Flow
|
||||
import akka.http.scaladsl.model.{ HttpRequest, HttpResponse }
|
||||
import akka.http.scaladsl.model.{HttpRequest, HttpResponse}
|
||||
import akka.http.scaladsl.util.FastFuture._
|
||||
|
||||
object Route {
|
||||
|
|
@ -66,7 +66,7 @@ object Route {
|
|||
|
||||
{
|
||||
implicit val executionContext = effectiveEC // overrides parameter
|
||||
val effectiveParserSettings = if (parserSettings ne null) parserSettings else ParserSettings(ActorMaterializer.downcast(materializer).system)
|
||||
val effectiveParserSettings = if (parserSettings ne null) parserSettings else ParserSettings(ActorMaterializerHelper.downcast(materializer).system)
|
||||
|
||||
val sealedRoute = seal(route)
|
||||
request ⇒
|
||||
|
|
|
|||
|
|
@ -9,8 +9,8 @@ import akka.http.scaladsl.settings.ParserSettings
|
|||
import scala.collection.immutable
|
||||
import scala.collection.immutable.VectorBuilder
|
||||
import akka.util.ByteString
|
||||
import akka.event.{ NoLogging, LoggingAdapter }
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.event.{LoggingAdapter, NoLogging}
|
||||
import akka.stream.{ActorMaterializer, ActorMaterializerHelper}
|
||||
import akka.stream.impl.fusing.IteratorInterpreter
|
||||
import akka.stream.scaladsl._
|
||||
import akka.http.impl.engine.parsing.BodyPartParser
|
||||
|
|
@ -75,7 +75,7 @@ trait MultipartUnmarshallers {
|
|||
FastFuture.failed(new RuntimeException("Content-Type with a multipart media type must have a 'boundary' parameter"))
|
||||
case Some(boundary) ⇒
|
||||
import BodyPartParser._
|
||||
val effectiveParserSettings = Option(parserSettings).getOrElse(ParserSettings(ActorMaterializer.downcast(mat).system))
|
||||
val effectiveParserSettings = Option(parserSettings).getOrElse(ParserSettings(ActorMaterializerHelper.downcast(mat).system))
|
||||
val parser = new BodyPartParser(defaultContentType, boundary, log, effectiveParserSettings)
|
||||
FastFuture.successful {
|
||||
entity match {
|
||||
|
|
|
|||
|
|
@ -3,10 +3,10 @@
|
|||
*/
|
||||
package akka.stream.impl
|
||||
|
||||
import akka.stream._
|
||||
import akka.stream.scaladsl._
|
||||
import akka.testkit.AkkaSpec
|
||||
import org.reactivestreams.{ Subscription, Subscriber, Publisher }
|
||||
import akka.stream._
|
||||
import org.reactivestreams.{ Publisher, Subscriber, Subscription }
|
||||
|
||||
import scala.concurrent.duration._
|
||||
|
||||
|
|
|
|||
|
|
@ -9,9 +9,9 @@ import akka.stream.ActorAttributes._
|
|||
import akka.stream.Attributes.LogLevels
|
||||
import akka.stream.Supervision._
|
||||
import akka.stream.testkit.ScriptedTest
|
||||
import akka.stream.javadsl
|
||||
import akka.stream.{ ActorMaterializer, Materializer, Attributes }
|
||||
import akka.stream._
|
||||
import akka.testkit.TestProbe
|
||||
|
||||
import scala.concurrent.duration._
|
||||
import scala.concurrent.Await
|
||||
import scala.util.control.NoStackTrace
|
||||
|
|
@ -29,7 +29,7 @@ class FlowLogSpec extends AkkaSpec("akka.loglevel = DEBUG") with ScriptedTest {
|
|||
|
||||
"A Log" must {
|
||||
|
||||
val supervisorPath = ActorMaterializer.downcast(mat).supervisor.path
|
||||
val supervisorPath = ActorMaterializerHelper.downcast(mat).supervisor.path
|
||||
val LogSrc = s"akka.stream.Log($supervisorPath)"
|
||||
val LogClazz = classOf[Materializer]
|
||||
|
||||
|
|
|
|||
|
|
@ -5,7 +5,7 @@ package akka.stream
|
|||
|
||||
import java.util.Locale
|
||||
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.event.LoggingAdapter
|
||||
|
|
@ -16,6 +16,7 @@ import com.typesafe.config.Config
|
|||
|
||||
import scala.concurrent.duration._
|
||||
import akka.japi.function
|
||||
import akka.stream.impl.fusing.GraphInterpreterShell
|
||||
|
||||
import scala.util.control.NoStackTrace
|
||||
|
||||
|
|
@ -126,6 +127,12 @@ object ActorMaterializer {
|
|||
system
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] object ActorMaterializerHelper {
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
|
|
@ -163,21 +170,23 @@ abstract class ActorMaterializer extends Materializer {
|
|||
def isShutdown: Boolean
|
||||
|
||||
/**
|
||||
* INTERNAL API: this might become public later
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] def actorOf(context: MaterializationContext, props: Props): ActorRef
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] def system: ActorSystem
|
||||
def system: ActorSystem
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] def logger: LoggingAdapter
|
||||
|
||||
/** INTERNAL API */
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] def supervisor: ActorRef
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -73,11 +73,10 @@ private[akka] object NoMaterializer extends Materializer {
|
|||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API: this might become public later
|
||||
*
|
||||
* Context parameter to the `create` methods of sources and sinks.
|
||||
*/
|
||||
private[akka] case class MaterializationContext(
|
||||
case class MaterializationContext(
|
||||
materializer: Materializer,
|
||||
effectiveAttributes: Attributes,
|
||||
stageName: String)
|
||||
|
|
|
|||
|
|
@ -3,19 +3,21 @@
|
|||
*/
|
||||
package akka.stream.impl
|
||||
|
||||
import java.util.concurrent.atomic.{ AtomicBoolean }
|
||||
import java.util.concurrent.atomic.AtomicBoolean
|
||||
import java.{ util ⇒ ju }
|
||||
|
||||
import akka.NotUsed
|
||||
import akka.actor._
|
||||
import akka.event.Logging
|
||||
import akka.event.{ Logging, LoggingAdapter }
|
||||
import akka.dispatch.Dispatchers
|
||||
import akka.pattern.ask
|
||||
import akka.stream._
|
||||
import akka.stream.impl.StreamLayout.{ Module, AtomicModule }
|
||||
import akka.stream.impl.StreamLayout.{ AtomicModule, Module }
|
||||
import akka.stream.impl.fusing.{ ActorGraphInterpreter, GraphModule }
|
||||
import akka.stream.impl.io.TLSActor
|
||||
import akka.stream.impl.io.TlsModule
|
||||
import org.reactivestreams._
|
||||
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
import scala.concurrent.{ Await, ExecutionContextExecutor }
|
||||
import akka.stream.impl.fusing.GraphStageModule
|
||||
|
|
@ -23,6 +25,37 @@ import akka.stream.impl.fusing.GraphInterpreter.GraphAssembly
|
|||
import akka.stream.impl.fusing.Fusing
|
||||
import akka.stream.impl.fusing.GraphInterpreterShell
|
||||
|
||||
/**
|
||||
* ExtendedActorMaterializer used by subtypes which materializer using GraphInterpreterShell
|
||||
*/
|
||||
abstract class ExtendedActorMaterializer extends ActorMaterializer {
|
||||
|
||||
override def withNamePrefix(name: String): ExtendedActorMaterializer
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
def materialize[Mat](
|
||||
_runnableGraph: Graph[ClosedShape, Mat],
|
||||
subflowFuser: GraphInterpreterShell ⇒ ActorRef): Mat
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
def actorOf(context: MaterializationContext, props: Props): ActorRef
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
override def logger: LoggingAdapter
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
override def supervisor: ActorRef
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
|
|
@ -32,7 +65,7 @@ private[akka] case class ActorMaterializerImpl(
|
|||
dispatchers: Dispatchers,
|
||||
supervisor: ActorRef,
|
||||
haveShutDown: AtomicBoolean,
|
||||
flowNames: SeqActorName) extends ActorMaterializer {
|
||||
flowNames: SeqActorName) extends ExtendedActorMaterializer {
|
||||
import akka.stream.impl.Stages._
|
||||
private val _logger = Logging.getLogger(system, this)
|
||||
override def logger = _logger
|
||||
|
|
@ -79,7 +112,7 @@ private[akka] case class ActorMaterializerImpl(
|
|||
override def materialize[Mat](_runnableGraph: Graph[ClosedShape, Mat]): Mat =
|
||||
materialize(_runnableGraph, null)
|
||||
|
||||
private[stream] def materialize[Mat](
|
||||
override def materialize[Mat](
|
||||
_runnableGraph: Graph[ClosedShape, Mat],
|
||||
subflowFuser: GraphInterpreterShell ⇒ ActorRef): Mat = {
|
||||
val runnableGraph =
|
||||
|
|
@ -213,7 +246,7 @@ private[akka] case class ActorMaterializerImpl(
|
|||
|
||||
}
|
||||
|
||||
private[akka] class SubFusingActorMaterializerImpl(val delegate: ActorMaterializerImpl, registerShell: GraphInterpreterShell ⇒ ActorRef) extends Materializer {
|
||||
private[akka] class SubFusingActorMaterializerImpl(val delegate: ExtendedActorMaterializer, registerShell: GraphInterpreterShell ⇒ ActorRef) extends Materializer {
|
||||
override def executionContext: ExecutionContextExecutor = delegate.executionContext
|
||||
|
||||
override def materialize[Mat](runnable: Graph[ClosedShape, Mat]): Mat = delegate.materialize(runnable, registerShell)
|
||||
|
|
@ -223,7 +256,7 @@ private[akka] class SubFusingActorMaterializerImpl(val delegate: ActorMaterializ
|
|||
override def schedulePeriodically(initialDelay: FiniteDuration, interval: FiniteDuration, task: Runnable): Cancellable =
|
||||
delegate.schedulePeriodically(initialDelay, interval, task)
|
||||
|
||||
def withNamePrefix(name: String): SubFusingActorMaterializerImpl =
|
||||
override def withNamePrefix(name: String): SubFusingActorMaterializerImpl =
|
||||
new SubFusingActorMaterializerImpl(delegate.withNamePrefix(name), registerShell)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -94,7 +94,7 @@ private[akka] final class MaybeSource[Out](val attributes: Attributes, shape: So
|
|||
private[akka] final class ActorPublisherSource[Out](props: Props, val attributes: Attributes, shape: SourceShape[Out]) extends SourceModule[Out, ActorRef](shape) {
|
||||
|
||||
override def create(context: MaterializationContext) = {
|
||||
val publisherRef = ActorMaterializer.downcast(context.materializer).actorOf(context, props)
|
||||
val publisherRef = ActorMaterializerHelper.downcast(context.materializer).actorOf(context, props)
|
||||
(akka.stream.actor.ActorPublisher[Out](publisherRef), publisherRef)
|
||||
}
|
||||
|
||||
|
|
@ -113,7 +113,7 @@ private[akka] final class ActorRefSource[Out](
|
|||
override protected def label: String = s"ActorRefSource($bufferSize, $overflowStrategy)"
|
||||
|
||||
override def create(context: MaterializationContext) = {
|
||||
val mat = ActorMaterializer.downcast(context.materializer)
|
||||
val mat = ActorMaterializerHelper.downcast(context.materializer)
|
||||
val ref = mat.actorOf(context, ActorRefSourceActor.props(bufferSize, overflowStrategy, mat.settings))
|
||||
(akka.stream.actor.ActorPublisher[Out](ref), ref)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -99,7 +99,7 @@ private[akka] final class FanoutPublisherSink[In](
|
|||
extends SinkModule[In, Publisher[In]](shape) {
|
||||
|
||||
override def create(context: MaterializationContext): (Subscriber[In], Publisher[In]) = {
|
||||
val actorMaterializer = ActorMaterializer.downcast(context.materializer)
|
||||
val actorMaterializer = ActorMaterializerHelper.downcast(context.materializer)
|
||||
val impl = actorMaterializer.actorOf(
|
||||
context,
|
||||
FanoutProcessorImpl.props(actorMaterializer.effectiveSettings(attributes)))
|
||||
|
|
@ -124,7 +124,7 @@ private[akka] final class FanoutPublisherSink[In](
|
|||
private[akka] final class SinkholeSink(val attributes: Attributes, shape: SinkShape[Any]) extends SinkModule[Any, Future[Done]](shape) {
|
||||
|
||||
override def create(context: MaterializationContext) = {
|
||||
val effectiveSettings = ActorMaterializer.downcast(context.materializer).effectiveSettings(context.effectiveAttributes)
|
||||
val effectiveSettings = ActorMaterializerHelper.downcast(context.materializer).effectiveSettings(context.effectiveAttributes)
|
||||
val p = Promise[Done]()
|
||||
(new SinkholeSubscriber[Any](p), p.future)
|
||||
}
|
||||
|
|
@ -163,7 +163,7 @@ private[akka] final class CancelSink(val attributes: Attributes, shape: SinkShap
|
|||
private[akka] final class ActorSubscriberSink[In](props: Props, val attributes: Attributes, shape: SinkShape[In]) extends SinkModule[In, ActorRef](shape) {
|
||||
|
||||
override def create(context: MaterializationContext) = {
|
||||
val subscriberRef = ActorMaterializer.downcast(context.materializer).actorOf(context, props)
|
||||
val subscriberRef = ActorMaterializerHelper.downcast(context.materializer).actorOf(context, props)
|
||||
(akka.stream.actor.ActorSubscriber[In](subscriberRef), subscriberRef)
|
||||
}
|
||||
|
||||
|
|
@ -179,7 +179,7 @@ private[akka] final class ActorRefSink[In](ref: ActorRef, onCompleteMessage: Any
|
|||
shape: SinkShape[In]) extends SinkModule[In, NotUsed](shape) {
|
||||
|
||||
override def create(context: MaterializationContext) = {
|
||||
val actorMaterializer = ActorMaterializer.downcast(context.materializer)
|
||||
val actorMaterializer = ActorMaterializerHelper.downcast(context.materializer)
|
||||
val effectiveSettings = actorMaterializer.effectiveSettings(context.effectiveAttributes)
|
||||
val subscriberRef = actorMaterializer.actorOf(
|
||||
context,
|
||||
|
|
|
|||
|
|
@ -801,7 +801,7 @@ private[impl] class VirtualPublisher[T] extends AtomicReference[AnyRef] with Pub
|
|||
/**
|
||||
* INERNAL API
|
||||
*/
|
||||
private[stream] object MaterializerSession {
|
||||
object MaterializerSession {
|
||||
class MaterializationPanic(cause: Throwable) extends RuntimeException("Materialization aborted.", cause) with NoStackTrace
|
||||
|
||||
final val Debug = false
|
||||
|
|
@ -810,7 +810,7 @@ private[stream] object MaterializerSession {
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[stream] abstract class MaterializerSession(val topLevel: StreamLayout.Module, val initialAttributes: Attributes) {
|
||||
abstract class MaterializerSession(val topLevel: StreamLayout.Module, val initialAttributes: Attributes) {
|
||||
import StreamLayout._
|
||||
|
||||
// the contained maps store either Subscriber[Any] or VirtualPublisher, but the type system cannot express that
|
||||
|
|
@ -839,7 +839,7 @@ private[stream] abstract class MaterializerSession(val topLevel: StreamLayout.Mo
|
|||
// Enters a copied module and establishes a scope that prevents internals to leak out and interfere with copies
|
||||
// of the same module.
|
||||
// We don't store the enclosing CopiedModule itself as state since we don't use it anywhere else than exit and enter
|
||||
private def enterScope(enclosing: CopiedModule): Unit = {
|
||||
protected def enterScope(enclosing: CopiedModule): Unit = {
|
||||
if (MaterializerSession.Debug) println(f"entering scope [${System.identityHashCode(enclosing)}%08x]")
|
||||
subscribersStack ::= new ju.HashMap
|
||||
publishersStack ::= new ju.HashMap
|
||||
|
|
@ -851,7 +851,7 @@ private[stream] abstract class MaterializerSession(val topLevel: StreamLayout.Mo
|
|||
// them to the copied ports instead of the original ones (since there might be multiple copies of the same module
|
||||
// leading to port identity collisions)
|
||||
// We don't store the enclosing CopiedModule itself as state since we don't use it anywhere else than exit and enter
|
||||
private def exitScope(enclosing: CopiedModule): Unit = {
|
||||
protected def exitScope(enclosing: CopiedModule): Unit = {
|
||||
if (MaterializerSession.Debug) println(f"exiting scope [${System.identityHashCode(enclosing)}%08x]")
|
||||
val scopeSubscribers = subscribers
|
||||
val scopePublishers = publishers
|
||||
|
|
@ -969,7 +969,7 @@ private[stream] abstract class MaterializerSession(val topLevel: StreamLayout.Mo
|
|||
ret
|
||||
}
|
||||
|
||||
final protected def assignPort(in: InPort, subscriberOrVirtual: AnyRef): Unit = {
|
||||
protected def assignPort(in: InPort, subscriberOrVirtual: AnyRef): Unit = {
|
||||
subscribers.put(in, subscriberOrVirtual)
|
||||
|
||||
currentLayout.upstreams.get(in) match {
|
||||
|
|
@ -981,7 +981,7 @@ private[stream] abstract class MaterializerSession(val topLevel: StreamLayout.Mo
|
|||
}
|
||||
}
|
||||
|
||||
final protected def assignPort(out: OutPort, publisher: Publisher[Any]): Unit = {
|
||||
protected def assignPort(out: OutPort, publisher: Publisher[Any]): Unit = {
|
||||
publishers.put(out, publisher)
|
||||
|
||||
currentLayout.downstreams.get(out) match {
|
||||
|
|
|
|||
|
|
@ -5,20 +5,19 @@ package akka.stream.impl.fusing
|
|||
|
||||
import java.util
|
||||
import java.util.concurrent.TimeoutException
|
||||
|
||||
import akka.actor._
|
||||
import akka.event.Logging
|
||||
import akka.stream._
|
||||
import akka.stream.impl._
|
||||
import akka.stream.impl.ReactiveStreamsCompliance._
|
||||
import akka.stream.impl.StreamLayout.{ CompositeModule, CopiedModule, Module, AtomicModule }
|
||||
import akka.stream.impl.fusing.GraphInterpreter.{ DownstreamBoundaryStageLogic, UpstreamBoundaryStageLogic, GraphAssembly }
|
||||
import akka.stream.impl.StreamLayout.{ AtomicModule, CompositeModule, CopiedModule, Module }
|
||||
import akka.stream.impl.{ SubFusingActorMaterializerImpl, _ }
|
||||
import akka.stream.impl.fusing.GraphInterpreter.{ DownstreamBoundaryStageLogic, GraphAssembly, UpstreamBoundaryStageLogic }
|
||||
import akka.stream.stage.{ GraphStageLogic, InHandler, OutHandler }
|
||||
import org.reactivestreams.{ Subscriber, Subscription }
|
||||
import scala.concurrent.forkjoin.ThreadLocalRandom
|
||||
import scala.util.control.NonFatal
|
||||
import akka.stream.impl.ActorMaterializerImpl
|
||||
import akka.stream.impl.SubFusingActorMaterializerImpl
|
||||
|
||||
import scala.annotation.tailrec
|
||||
import scala.util.control.NonFatal
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
@ -307,14 +306,14 @@ private[stream] object ActorGraphInterpreter {
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[stream] final class GraphInterpreterShell(
|
||||
final class GraphInterpreterShell(
|
||||
assembly: GraphAssembly,
|
||||
inHandlers: Array[InHandler],
|
||||
outHandlers: Array[OutHandler],
|
||||
logics: Array[GraphStageLogic],
|
||||
shape: Shape,
|
||||
settings: ActorMaterializerSettings,
|
||||
val mat: ActorMaterializerImpl) {
|
||||
val mat: ExtendedActorMaterializer) {
|
||||
|
||||
import ActorGraphInterpreter._
|
||||
|
||||
|
|
@ -643,4 +642,4 @@ private[stream] class ActorGraphInterpreter(_initial: GraphInterpreterShell) ext
|
|||
activeInterpreters = Set.empty[GraphInterpreterShell]
|
||||
newShells.foreach(s ⇒ if (tryInit(s)) s.tryAbort(ex))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -267,7 +267,7 @@ object GraphStages {
|
|||
*
|
||||
* This source is not reusable, it is only created internally.
|
||||
*/
|
||||
private[stream] final class MaterializedValueSource[T](val computation: MaterializedValueNode, val out: Outlet[T]) extends GraphStage[SourceShape[T]] {
|
||||
final class MaterializedValueSource[T](val computation: MaterializedValueNode, val out: Outlet[T]) extends GraphStage[SourceShape[T]] {
|
||||
def this(computation: MaterializedValueNode) = this(computation, Outlet[T]("matValue"))
|
||||
override def initialAttributes: Attributes = DefaultAttributes.materializedValueSource
|
||||
override val shape = SourceShape(out)
|
||||
|
|
|
|||
|
|
@ -922,7 +922,7 @@ private[akka] final case class Log[T](name: String, extract: T ⇒ Any,
|
|||
log = logAdapter match {
|
||||
case Some(l) ⇒ l
|
||||
case _ ⇒
|
||||
val mat = try ActorMaterializer.downcast(ctx.materializer)
|
||||
val mat = try ActorMaterializerHelper.downcast(ctx.materializer)
|
||||
catch {
|
||||
case ex: Exception ⇒
|
||||
throw new RuntimeException("Log stage can only provide LoggingAdapter when used with ActorMaterializer! " +
|
||||
|
|
@ -984,7 +984,7 @@ private[akka] object Log {
|
|||
override def getClazz(t: LifecycleContext): Class[_] = classOf[Materializer]
|
||||
|
||||
override def genString(t: LifecycleContext): String = {
|
||||
try s"$DefaultLoggerName(${ActorMaterializer.downcast(t.materializer).supervisor.path})"
|
||||
try s"$DefaultLoggerName(${ActorMaterializerHelper.downcast(t.materializer).supervisor.path})"
|
||||
catch {
|
||||
case ex: Exception ⇒ LogSource.fromString.genString(DefaultLoggerName)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -121,7 +121,7 @@ final class PrefixAndTail[T](n: Int) extends GraphStage[FlowShape[T, (immutable.
|
|||
private val SubscriptionTimer = "SubstreamSubscriptionTimer"
|
||||
|
||||
override protected def onTimer(timerKey: Any): Unit = {
|
||||
val materializer = ActorMaterializer.downcast(interpreter.materializer)
|
||||
val materializer = ActorMaterializerHelper.downcast(interpreter.materializer)
|
||||
val timeoutSettings = materializer.settings.subscriptionTimeoutSettings
|
||||
val timeout = timeoutSettings.timeout
|
||||
|
||||
|
|
@ -150,7 +150,7 @@ final class PrefixAndTail[T](n: Int) extends GraphStage[FlowShape[T, (immutable.
|
|||
}
|
||||
|
||||
private def openSubstream(): Source[T, NotUsed] = {
|
||||
val timeout = ActorMaterializer.downcast(interpreter.materializer).settings.subscriptionTimeoutSettings.timeout
|
||||
val timeout = ActorMaterializerHelper.downcast(interpreter.materializer).settings.subscriptionTimeoutSettings.timeout
|
||||
tailSource = new SubSourceOutlet[T]("TailSource")
|
||||
tailSource.setHandler(subHandler)
|
||||
setKeepGoing(true)
|
||||
|
|
@ -254,7 +254,7 @@ final class GroupBy[T, K](maxSubstreams: Int, keyFor: T ⇒ K) extends GraphStag
|
|||
private def needToPull: Boolean = !(hasBeenPulled(in) || isClosed(in) || hasNextElement)
|
||||
|
||||
override def preStart(): Unit =
|
||||
timeout = ActorMaterializer.downcast(interpreter.materializer).settings.subscriptionTimeoutSettings.timeout
|
||||
timeout = ActorMaterializerHelper.downcast(interpreter.materializer).settings.subscriptionTimeoutSettings.timeout
|
||||
|
||||
override def onPull(): Unit = {
|
||||
substreamWaitingToBePushed match {
|
||||
|
|
@ -424,7 +424,7 @@ final class Split[T](decision: Split.SplitDecision, p: T ⇒ Boolean, substreamC
|
|||
private var substreamCancelled = false
|
||||
|
||||
override def preStart(): Unit = {
|
||||
timeout = ActorMaterializer.downcast(interpreter.materializer).settings.subscriptionTimeoutSettings.timeout
|
||||
timeout = ActorMaterializerHelper.downcast(interpreter.materializer).settings.subscriptionTimeoutSettings.timeout
|
||||
}
|
||||
|
||||
setHandler(out, new OutHandler {
|
||||
|
|
|
|||
|
|
@ -5,13 +5,14 @@ package akka.stream.impl.io
|
|||
|
||||
import java.io.OutputStream
|
||||
import java.nio.file.{ Path, StandardOpenOption }
|
||||
import akka.stream.IOResult
|
||||
|
||||
import akka.stream._
|
||||
import akka.stream.impl.SinkModule
|
||||
import akka.stream.impl.StreamLayout.Module
|
||||
import akka.stream.impl.Stages.DefaultAttributes.IODispatcher
|
||||
import akka.stream.{ ActorMaterializer, MaterializationContext, Attributes, SinkShape }
|
||||
import akka.stream.ActorAttributes.Dispatcher
|
||||
import akka.util.ByteString
|
||||
|
||||
import scala.concurrent.{ Future, Promise }
|
||||
|
||||
/**
|
||||
|
|
@ -25,7 +26,7 @@ private[akka] final class FileSink(f: Path, options: Set[StandardOpenOption], va
|
|||
override protected def label: String = s"FileSink($f, $options)"
|
||||
|
||||
override def create(context: MaterializationContext) = {
|
||||
val materializer = ActorMaterializer.downcast(context.materializer)
|
||||
val materializer = ActorMaterializerHelper.downcast(context.materializer)
|
||||
val settings = materializer.effectiveSettings(context.effectiveAttributes)
|
||||
|
||||
val ioResultPromise = Promise[IOResult]()
|
||||
|
|
@ -51,7 +52,7 @@ private[akka] final class OutputStreamSink(createOutput: () ⇒ OutputStream, va
|
|||
extends SinkModule[ByteString, Future[IOResult]](shape) {
|
||||
|
||||
override def create(context: MaterializationContext) = {
|
||||
val materializer = ActorMaterializer.downcast(context.materializer)
|
||||
val materializer = ActorMaterializerHelper.downcast(context.materializer)
|
||||
val settings = materializer.effectiveSettings(context.effectiveAttributes)
|
||||
val ioResultPromise = Promise[IOResult]()
|
||||
|
||||
|
|
|
|||
|
|
@ -25,7 +25,7 @@ private[akka] final class FileSource(f: Path, chunkSize: Int, val attributes: At
|
|||
require(chunkSize > 0, "chunkSize must be greater than 0")
|
||||
override def create(context: MaterializationContext) = {
|
||||
// FIXME rewrite to be based on GraphStage rather than dangerous downcasts
|
||||
val materializer = ActorMaterializer.downcast(context.materializer)
|
||||
val materializer = ActorMaterializerHelper.downcast(context.materializer)
|
||||
val settings = materializer.effectiveSettings(context.effectiveAttributes)
|
||||
|
||||
val ioResultPromise = Promise[IOResult]()
|
||||
|
|
@ -53,7 +53,7 @@ private[akka] final class FileSource(f: Path, chunkSize: Int, val attributes: At
|
|||
private[akka] final class InputStreamSource(createInputStream: () ⇒ InputStream, chunkSize: Int, val attributes: Attributes, shape: SourceShape[ByteString])
|
||||
extends SourceModule[ByteString, Future[IOResult]](shape) {
|
||||
override def create(context: MaterializationContext) = {
|
||||
val materializer = ActorMaterializer.downcast(context.materializer)
|
||||
val materializer = ActorMaterializerHelper.downcast(context.materializer)
|
||||
val ioResultPromise = Promise[IOResult]()
|
||||
|
||||
val pub = try {
|
||||
|
|
|
|||
|
|
@ -21,6 +21,7 @@ import akka.stream.impl.Stages.DefaultAttributes.IODispatcher
|
|||
import akka.stream.ActorAttributes.Dispatcher
|
||||
import scala.concurrent.ExecutionContext
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.stream.ActorMaterializerHelper
|
||||
|
||||
private[stream] object OutputStreamSourceStage {
|
||||
sealed trait AdapterToStageMessage
|
||||
|
|
@ -112,7 +113,7 @@ final private[stream] class OutputStreamSourceStage(writeTimeout: FiniteDuration
|
|||
}
|
||||
|
||||
override def preStart(): Unit = {
|
||||
dispatcher = ActorMaterializer.downcast(materializer).system.dispatchers.lookup(dispatcherId)
|
||||
dispatcher = ActorMaterializerHelper.downcast(materializer).system.dispatchers.lookup(dispatcherId)
|
||||
super.preStart()
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -900,7 +900,7 @@ abstract class GraphStageLogic private[stream] (val inCount: Int, val outCount:
|
|||
final protected def getStageActor(receive: ((ActorRef, Any)) ⇒ Unit): StageActor = {
|
||||
_stageActor match {
|
||||
case null ⇒
|
||||
val actorMaterializer = ActorMaterializer.downcast(interpreter.materializer)
|
||||
val actorMaterializer = ActorMaterializerHelper.downcast(interpreter.materializer)
|
||||
_stageActor = new StageActor(actorMaterializer, getAsyncCallback, receive)
|
||||
_stageActor
|
||||
case existing ⇒
|
||||
|
|
|
|||
|
|
@ -873,6 +873,9 @@ object MiMa extends AutoPlugin {
|
|||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.scaladsl.Framing#LengthFieldFramingStage.onUpstreamFinish"),
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.scaladsl.Framing#LengthFieldFramingStage.onPull"),
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.scaladsl.Framing#LengthFieldFramingStage.postStop"),
|
||||
|
||||
// #20414 Allow different ActorMaterializer subtypes
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.ActorMaterializer.downcast"),
|
||||
|
||||
// #20531 adding refuseUid to Gated
|
||||
FilterAnyProblem("akka.remote.EndpointManager$Gated")
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue