Untangle settings from attributes in materializer #24063
Instead of renaming - deprecating and documenting Renaming defaultInitialAttributes to defaultAttributes and adding docs Tests passing More work on using attributes for the attribute settings in ActorMaterializerSettings Mark some attributes as mandatory and provide a smoother access w/o allocation for those Another warning notice MiMa filters Part of the review adressed Incorrect rebase fixed + formatting Review feedback adressed Formatting ...
This commit is contained in:
parent
4402c73a53
commit
4dde0742d0
20 changed files with 369 additions and 249 deletions
|
|
@ -36,29 +36,34 @@ import akka.util.OptionVal
|
|||
val Debug = false
|
||||
|
||||
val DefaultPhase: Phase[Any] = new Phase[Any] {
|
||||
override def apply(settings: ActorMaterializerSettings, materializer: PhasedFusingActorMaterializer, islandName: String): PhaseIsland[Any] =
|
||||
new GraphStageIsland(settings, materializer, islandName, subflowFuser = OptionVal.None).asInstanceOf[PhaseIsland[Any]]
|
||||
override def apply(settings: ActorMaterializerSettings, effectiveAttributes: Attributes,
|
||||
materializer: PhasedFusingActorMaterializer, islandName: String): PhaseIsland[Any] =
|
||||
new GraphStageIsland(settings, effectiveAttributes, materializer, islandName, subflowFuser = OptionVal.None).asInstanceOf[PhaseIsland[Any]]
|
||||
}
|
||||
|
||||
val DefaultPhases: Map[IslandTag, Phase[Any]] = Map[IslandTag, Phase[Any]](
|
||||
SinkModuleIslandTag → new Phase[Any] {
|
||||
override def apply(settings: ActorMaterializerSettings, materializer: PhasedFusingActorMaterializer,
|
||||
islandName: String): PhaseIsland[Any] =
|
||||
override def apply(settings: ActorMaterializerSettings, effectiveAttributes: Attributes,
|
||||
materializer: PhasedFusingActorMaterializer,
|
||||
islandName: String): PhaseIsland[Any] =
|
||||
new SinkModulePhase(materializer, islandName).asInstanceOf[PhaseIsland[Any]]
|
||||
},
|
||||
SourceModuleIslandTag → new Phase[Any] {
|
||||
override def apply(settings: ActorMaterializerSettings, materializer: PhasedFusingActorMaterializer,
|
||||
islandName: String): PhaseIsland[Any] =
|
||||
override def apply(settings: ActorMaterializerSettings, effectiveAttributes: Attributes,
|
||||
materializer: PhasedFusingActorMaterializer,
|
||||
islandName: String): PhaseIsland[Any] =
|
||||
new SourceModulePhase(materializer, islandName).asInstanceOf[PhaseIsland[Any]]
|
||||
},
|
||||
ProcessorModuleIslandTag → new Phase[Any] {
|
||||
override def apply(settings: ActorMaterializerSettings, materializer: PhasedFusingActorMaterializer,
|
||||
islandName: String): PhaseIsland[Any] =
|
||||
override def apply(settings: ActorMaterializerSettings, effectiveAttributes: Attributes,
|
||||
materializer: PhasedFusingActorMaterializer,
|
||||
islandName: String): PhaseIsland[Any] =
|
||||
new ProcessorModulePhase(materializer, islandName).asInstanceOf[PhaseIsland[Any]]
|
||||
},
|
||||
TlsModuleIslandTag → new Phase[Any] {
|
||||
def apply(settings: ActorMaterializerSettings, materializer: PhasedFusingActorMaterializer, islandName: String): PhaseIsland[Any] =
|
||||
new TlsModulePhase(settings, materializer, islandName).asInstanceOf[PhaseIsland[Any]]
|
||||
def apply(settings: ActorMaterializerSettings, effectiveAttributes: Attributes,
|
||||
materializer: PhasedFusingActorMaterializer, islandName: String): PhaseIsland[Any] =
|
||||
new TlsModulePhase(effectiveAttributes, materializer, islandName).asInstanceOf[PhaseIsland[Any]]
|
||||
},
|
||||
GraphStageTag → DefaultPhase)
|
||||
|
||||
|
|
@ -125,6 +130,7 @@ private final case class SavedIslandData(islandGlobalOffset: Int, lastVisitedOff
|
|||
@InternalApi private[akka] class IslandTracking(
|
||||
val phases: Map[IslandTag, Phase[Any]],
|
||||
val settings: ActorMaterializerSettings,
|
||||
attributes: Attributes,
|
||||
defaultPhase: Phase[Any],
|
||||
val materializer: PhasedFusingActorMaterializer,
|
||||
islandNamePrefix: String) {
|
||||
|
|
@ -150,7 +156,7 @@ private final case class SavedIslandData(islandGlobalOffset: Int, lastVisitedOff
|
|||
private var forwardWires: java.util.ArrayList[ForwardWire] = null
|
||||
private var islandStateStack: java.util.ArrayList[SavedIslandData] = null
|
||||
|
||||
private var currentPhase: PhaseIsland[Any] = defaultPhase.apply(settings, materializer, nextIslandName())
|
||||
private var currentPhase: PhaseIsland[Any] = defaultPhase.apply(settings, attributes, materializer, nextIslandName())
|
||||
|
||||
@InternalApi private[akka] def getCurrentPhase: PhaseIsland[Any] = currentPhase
|
||||
@InternalApi private[akka] def getCurrentOffset: Int = currentGlobalOffset
|
||||
|
|
@ -188,8 +194,7 @@ private final case class SavedIslandData(islandGlobalOffset: Int, lastVisitedOff
|
|||
val previousIslandOffset = currentIslandGlobalOffset
|
||||
islandStateStack.add(SavedIslandData(previousIslandOffset, currentGlobalOffset, currentIslandSkippetSlots, previousPhase))
|
||||
|
||||
val effectiveSettings = materializer.effectiveSettings(attributes)
|
||||
currentPhase = phases(tag)(effectiveSettings, materializer, nextIslandName())
|
||||
currentPhase = phases(tag)(settings, attributes, materializer, nextIslandName())
|
||||
activePhases.add(currentPhase)
|
||||
|
||||
// Resolve the phase to be used to materialize this island
|
||||
|
|
@ -368,36 +373,23 @@ private final case class SavedIslandData(islandGlobalOffset: Int, lastVisitedOff
|
|||
|
||||
private[this] def createFlowName(): String = flowNames.next()
|
||||
|
||||
/** INTERNAL API */
|
||||
private[akka] val defaultInitialAttributes = {
|
||||
val a = Attributes(
|
||||
/**
|
||||
* Default attributes for the materializer, based on the [[ActorMaterializerSettings]] and
|
||||
* are always seen as least specific, so any attribute specified in the graph "wins" over these.
|
||||
* In addition to that this also guarantees that the attributes `InputBuffer`, `SupervisionStrategy`,
|
||||
* and `Dispatcher` is _always_ present in the attributes.
|
||||
*
|
||||
* When these attributes are needed later in the materialization process it is important that the
|
||||
* they are gotten through the attributes and not through the [[ActorMaterializerSettings]]
|
||||
*/
|
||||
val defaultAttributes = {
|
||||
Attributes(
|
||||
Attributes.InputBuffer(settings.initialInputBufferSize, settings.maxInputBufferSize) ::
|
||||
ActorAttributes.SupervisionStrategy(settings.supervisionDecider) ::
|
||||
Nil)
|
||||
if (settings.dispatcher == Deploy.NoDispatcherGiven) a
|
||||
else a and ActorAttributes.dispatcher(settings.dispatcher)
|
||||
}
|
||||
|
||||
override def effectiveSettings(opAttr: Attributes): ActorMaterializerSettings = {
|
||||
import ActorAttributes._
|
||||
import Attributes._
|
||||
@tailrec def applyAttributes(attrs: List[Attribute], s: ActorMaterializerSettings,
|
||||
inputBufferDone: Boolean, dispatcherDone: Boolean, supervisorDone: Boolean): ActorMaterializerSettings = {
|
||||
attrs match {
|
||||
case InputBuffer(initial, max) :: tail if !inputBufferDone ⇒
|
||||
applyAttributes(tail, s.withInputBuffer(initial, max), inputBufferDone = true, dispatcherDone, supervisorDone)
|
||||
case Dispatcher(dispatcher) :: tail if !dispatcherDone ⇒
|
||||
applyAttributes(tail, s.withDispatcher(dispatcher), inputBufferDone, dispatcherDone = true, supervisorDone)
|
||||
case SupervisionStrategy(decider) :: tail if !supervisorDone ⇒
|
||||
applyAttributes(tail, s.withSupervisionStrategy(decider), inputBufferDone, dispatcherDone, supervisorDone = true)
|
||||
case _ if inputBufferDone || dispatcherDone || supervisorDone ⇒ s
|
||||
case _ :: tail ⇒
|
||||
applyAttributes(tail, s, inputBufferDone, dispatcherDone, supervisorDone)
|
||||
case Nil ⇒
|
||||
s
|
||||
}
|
||||
}
|
||||
applyAttributes(opAttr.attributeList, settings, false, false, false)
|
||||
ActorAttributes.Dispatcher(
|
||||
if (settings.dispatcher == Deploy.NoDispatcherGiven) Dispatchers.DefaultDispatcherId
|
||||
else settings.dispatcher
|
||||
) :: Nil)
|
||||
}
|
||||
|
||||
override lazy val executionContext: ExecutionContextExecutor = dispatchers.lookup(settings.dispatcher match {
|
||||
|
|
@ -412,28 +404,28 @@ private final case class SavedIslandData(islandGlobalOffset: Int, lastVisitedOff
|
|||
system.scheduler.scheduleOnce(delay, task)(executionContext)
|
||||
|
||||
override def materialize[Mat](_runnableGraph: Graph[ClosedShape, Mat]): Mat =
|
||||
materialize(_runnableGraph, defaultInitialAttributes)
|
||||
materialize(_runnableGraph, defaultAttributes)
|
||||
|
||||
override def materialize[Mat](
|
||||
_runnableGraph: Graph[ClosedShape, Mat],
|
||||
initialAttributes: Attributes): Mat =
|
||||
defaultAttributes: Attributes): Mat =
|
||||
materialize(
|
||||
_runnableGraph,
|
||||
initialAttributes,
|
||||
defaultAttributes,
|
||||
PhasedFusingActorMaterializer.DefaultPhase,
|
||||
PhasedFusingActorMaterializer.DefaultPhases)
|
||||
|
||||
override def materialize[Mat](
|
||||
graph: Graph[ClosedShape, Mat],
|
||||
initialAttributes: Attributes,
|
||||
defaultAttributes: Attributes,
|
||||
defaultPhase: Phase[Any],
|
||||
phases: Map[IslandTag, Phase[Any]]): Mat = {
|
||||
val islandTracking = new IslandTracking(phases, settings, defaultPhase, this, islandNamePrefix = createFlowName() + "-")
|
||||
val islandTracking = new IslandTracking(phases, settings, defaultAttributes, defaultPhase, this, islandNamePrefix = createFlowName() + "-")
|
||||
|
||||
var current: Traversal = graph.traversalBuilder.traversal
|
||||
|
||||
val attributesStack = new java.util.ArrayDeque[Attributes](8)
|
||||
attributesStack.addLast(initialAttributes and graph.traversalBuilder.attributes)
|
||||
attributesStack.addLast(defaultAttributes and graph.traversalBuilder.attributes)
|
||||
|
||||
val traversalStack = new java.util.ArrayDeque[Traversal](16)
|
||||
traversalStack.addLast(current)
|
||||
|
|
@ -564,9 +556,10 @@ private final case class SavedIslandData(islandGlobalOffset: Int, lastVisitedOff
|
|||
*/
|
||||
@DoNotInherit private[akka] trait Phase[M] {
|
||||
def apply(
|
||||
effectiveSettings: ActorMaterializerSettings,
|
||||
materializer: PhasedFusingActorMaterializer,
|
||||
islandName: String): PhaseIsland[M]
|
||||
settings: ActorMaterializerSettings,
|
||||
effectiveAttributes: Attributes,
|
||||
materializer: PhasedFusingActorMaterializer,
|
||||
islandName: String): PhaseIsland[M]
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -599,10 +592,11 @@ private final case class SavedIslandData(islandGlobalOffset: Int, lastVisitedOff
|
|||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi private[akka] final class GraphStageIsland(
|
||||
effectiveSettings: ActorMaterializerSettings,
|
||||
materializer: PhasedFusingActorMaterializer,
|
||||
islandName: String,
|
||||
subflowFuser: OptionVal[GraphInterpreterShell ⇒ ActorRef]) extends PhaseIsland[GraphStageLogic] {
|
||||
settings: ActorMaterializerSettings,
|
||||
effectiveAttributes: Attributes,
|
||||
materializer: PhasedFusingActorMaterializer,
|
||||
islandName: String,
|
||||
subflowFuser: OptionVal[GraphInterpreterShell ⇒ ActorRef]) extends PhaseIsland[GraphStageLogic] {
|
||||
// TODO: remove these
|
||||
private val logicArrayType = Array.empty[GraphStageLogic]
|
||||
private[this] val logics = new ArrayList[GraphStageLogic](16)
|
||||
|
|
@ -615,7 +609,8 @@ private final case class SavedIslandData(islandGlobalOffset: Int, lastVisitedOff
|
|||
val shell = new GraphInterpreterShell(
|
||||
connections = null,
|
||||
logics = null,
|
||||
effectiveSettings,
|
||||
settings,
|
||||
effectiveAttributes,
|
||||
materializer)
|
||||
|
||||
override def name: String = "Fusing GraphStages phase"
|
||||
|
|
@ -697,7 +692,7 @@ private final case class SavedIslandData(islandGlobalOffset: Int, lastVisitedOff
|
|||
override def takePublisher(slot: Int, publisher: Publisher[Any]): Unit = {
|
||||
val connection = conn(slot)
|
||||
// TODO: proper input port debug string (currently prints the stage)
|
||||
val bufferSize = connection.inOwner.attributes.get[InputBuffer].get.max
|
||||
val bufferSize = connection.inOwner.attributes.mandatoryAttribute[InputBuffer].max
|
||||
val boundary =
|
||||
new BatchingActorInputBoundary(bufferSize, shell, publisher, connection.inOwner.toString)
|
||||
logics.add(boundary)
|
||||
|
|
@ -734,7 +729,7 @@ private final case class SavedIslandData(islandGlobalOffset: Int, lastVisitedOff
|
|||
|
||||
case _ ⇒
|
||||
val props = ActorGraphInterpreter.props(shell)
|
||||
.withDispatcher(effectiveSettings.dispatcher)
|
||||
.withDispatcher(effectiveAttributes.mandatoryAttribute[ActorAttributes.Dispatcher].dispatcher)
|
||||
val actorName = fullIslandName match {
|
||||
case OptionVal.Some(n) ⇒ n
|
||||
case OptionVal.None ⇒ islandName
|
||||
|
|
@ -867,7 +862,7 @@ private final case class SavedIslandData(islandGlobalOffset: Int, lastVisitedOff
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi private[akka] final class TlsModulePhase(settings: ActorMaterializerSettings, materializer: PhasedFusingActorMaterializer, islandName: String) extends PhaseIsland[NotUsed] {
|
||||
@InternalApi private[akka] final class TlsModulePhase(attributes: Attributes, materializer: PhasedFusingActorMaterializer, islandName: String) extends PhaseIsland[NotUsed] {
|
||||
def name: String = "TlsModulePhase"
|
||||
|
||||
var tlsActor: ActorRef = _
|
||||
|
|
@ -876,8 +871,11 @@ private final case class SavedIslandData(islandGlobalOffset: Int, lastVisitedOff
|
|||
def materializeAtomic(mod: AtomicModule[Shape, Any], attributes: Attributes): (NotUsed, Any) = {
|
||||
val tls = mod.asInstanceOf[TlsModule]
|
||||
|
||||
val dispatcher = attributes.mandatoryAttribute[ActorAttributes.Dispatcher].dispatcher
|
||||
val maxInputBuffer = attributes.mandatoryAttribute[Attributes.InputBuffer].max
|
||||
|
||||
val props =
|
||||
TLSActor.props(settings, tls.createSSLEngine, tls.verifySession, tls.closing).withDispatcher(settings.dispatcher)
|
||||
TLSActor.props(maxInputBuffer, tls.createSSLEngine, tls.verifySession, tls.closing).withDispatcher(dispatcher)
|
||||
tlsActor = materializer.actorOf(props, islandName)
|
||||
def factory(id: Int) = new ActorPublisher[Any](tlsActor) {
|
||||
override val wakeUpMsg = FanOut.SubstreamSubscribePending(id)
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue