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:
Johan Andrén 2017-11-22 13:51:24 +01:00 committed by Konrad `ktoso` Malawski
parent 4402c73a53
commit 4dde0742d0
20 changed files with 369 additions and 249 deletions

View file

@ -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)