2017-03-08 13:10:36 +01:00
|
|
|
/**
|
|
|
|
|
* Copyright (C) 2015-2017 Lightbend Inc. <http://www.lightbend.com>
|
|
|
|
|
*/
|
2016-07-27 13:29:23 +02:00
|
|
|
package akka.stream.impl
|
|
|
|
|
|
2017-03-07 10:55:46 +01:00
|
|
|
import java.util
|
2016-07-27 13:29:23 +02:00
|
|
|
import java.util.ArrayList
|
|
|
|
|
import java.util.concurrent.atomic.AtomicBoolean
|
|
|
|
|
|
|
|
|
|
import akka.NotUsed
|
|
|
|
|
import akka.actor.{ ActorContext, ActorRef, ActorRefFactory, ActorSystem, Cancellable, Deploy, ExtendedActorSystem, PoisonPill, Props }
|
|
|
|
|
import akka.dispatch.Dispatchers
|
|
|
|
|
import akka.event.{ Logging, LoggingAdapter }
|
2017-03-02 13:44:19 +01:00
|
|
|
import akka.stream.Attributes.InputBuffer
|
2016-07-27 13:29:23 +02:00
|
|
|
import akka.stream._
|
|
|
|
|
import akka.stream.impl.StreamLayout.AtomicModule
|
|
|
|
|
import akka.stream.impl.fusing.ActorGraphInterpreter.{ ActorOutputBoundary, BatchingActorInputBoundary }
|
|
|
|
|
import akka.stream.impl.fusing.GraphInterpreter.Connection
|
|
|
|
|
import akka.stream.impl.fusing._
|
2017-03-07 12:39:15 +01:00
|
|
|
import akka.stream.impl.io.{ TLSActor, TlsModule }
|
2016-07-27 13:29:23 +02:00
|
|
|
import akka.stream.stage.{ GraphStageLogic, InHandler, OutHandler }
|
2017-03-09 17:04:46 +01:00
|
|
|
import akka.util.OptionVal
|
2017-03-03 10:49:34 +01:00
|
|
|
import org.reactivestreams.{ Processor, Publisher, Subscriber, Subscription }
|
2016-07-27 13:29:23 +02:00
|
|
|
|
|
|
|
|
import scala.collection.immutable.Map
|
|
|
|
|
import scala.concurrent.duration.FiniteDuration
|
2017-03-07 12:39:15 +01:00
|
|
|
import scala.concurrent.ExecutionContextExecutor
|
2017-03-07 19:40:50 +01:00
|
|
|
import scala.annotation.tailrec
|
|
|
|
|
import akka.stream.impl.fusing.GraphInterpreter.DownstreamBoundaryStageLogic
|
|
|
|
|
import akka.stream.impl.fusing.GraphInterpreter.UpstreamBoundaryStageLogic
|
2016-07-27 13:29:23 +02:00
|
|
|
|
|
|
|
|
object PhasedFusingActorMaterializer {
|
|
|
|
|
|
|
|
|
|
val Debug = false
|
|
|
|
|
|
|
|
|
|
val DefaultPhase: Phase[Any] = new Phase[Any] {
|
2017-03-08 13:10:36 +01:00
|
|
|
override def apply(settings: ActorMaterializerSettings, materializer: PhasedFusingActorMaterializer, islandName: String): PhaseIsland[Any] =
|
|
|
|
|
new GraphStageIsland(settings, materializer, islandName, subflowFuser = None).asInstanceOf[PhaseIsland[Any]]
|
2016-07-27 13:29:23 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
val DefaultPhases: Map[IslandTag, Phase[Any]] = Map[IslandTag, Phase[Any]](
|
|
|
|
|
SinkModuleIslandTag → new Phase[Any] {
|
2017-03-07 10:43:18 +01:00
|
|
|
override def apply(settings: ActorMaterializerSettings, materializer: PhasedFusingActorMaterializer,
|
|
|
|
|
islandName: String): PhaseIsland[Any] =
|
2017-03-08 13:10:36 +01:00
|
|
|
new SinkModulePhase(materializer, islandName).asInstanceOf[PhaseIsland[Any]]
|
2016-07-27 13:29:23 +02:00
|
|
|
},
|
|
|
|
|
SourceModuleIslandTag → new Phase[Any] {
|
2017-03-07 10:43:18 +01:00
|
|
|
override def apply(settings: ActorMaterializerSettings, materializer: PhasedFusingActorMaterializer,
|
|
|
|
|
islandName: String): PhaseIsland[Any] =
|
|
|
|
|
new SourceModulePhase(materializer, islandName).asInstanceOf[PhaseIsland[Any]]
|
2016-07-27 13:29:23 +02:00
|
|
|
},
|
2017-03-03 10:49:34 +01:00
|
|
|
ProcessorModuleIslandTag → new Phase[Any] {
|
2017-03-07 10:43:18 +01:00
|
|
|
override def apply(settings: ActorMaterializerSettings, materializer: PhasedFusingActorMaterializer,
|
|
|
|
|
islandName: String): PhaseIsland[Any] =
|
|
|
|
|
new ProcessorModulePhase(materializer, islandName).asInstanceOf[PhaseIsland[Any]]
|
2017-03-03 10:49:34 +01:00
|
|
|
},
|
2017-03-07 12:39:15 +01:00
|
|
|
TlsModuleIslandTag → new Phase[Any] {
|
|
|
|
|
def apply(settings: ActorMaterializerSettings, materializer: PhasedFusingActorMaterializer, islandName: String): PhaseIsland[Any] =
|
|
|
|
|
new TlsModulePhase(settings, materializer, islandName).asInstanceOf[PhaseIsland[Any]]
|
|
|
|
|
},
|
2017-03-07 10:43:18 +01:00
|
|
|
GraphStageTag → DefaultPhase)
|
2016-07-27 13:29:23 +02:00
|
|
|
|
|
|
|
|
def apply(settings: ActorMaterializerSettings)(implicit context: ActorRefFactory): ActorMaterializer = {
|
|
|
|
|
val haveShutDown = new AtomicBoolean(false)
|
|
|
|
|
val system = actorSystemOf(context)
|
|
|
|
|
val materializerSettings = ActorMaterializerSettings(system)
|
|
|
|
|
|
2017-03-07 10:43:18 +01:00
|
|
|
val streamSupervisor = context.actorOf(StreamSupervisor.props(materializerSettings, haveShutDown)
|
|
|
|
|
.withDispatcher(materializerSettings.dispatcher), StreamSupervisor.nextName())
|
|
|
|
|
|
2016-07-27 13:29:23 +02:00
|
|
|
PhasedFusingActorMaterializer(
|
|
|
|
|
system,
|
|
|
|
|
materializerSettings,
|
|
|
|
|
system.dispatchers,
|
2017-03-07 10:43:18 +01:00
|
|
|
streamSupervisor,
|
2016-07-27 13:29:23 +02:00
|
|
|
haveShutDown,
|
|
|
|
|
FlowNames(system).name.copy("flow"))
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private def actorSystemOf(context: ActorRefFactory): ActorSystem = {
|
|
|
|
|
val system = context match {
|
|
|
|
|
case s: ExtendedActorSystem ⇒ s
|
|
|
|
|
case c: ActorContext ⇒ c.system
|
|
|
|
|
case null ⇒ throw new IllegalArgumentException("ActorRefFactory context must be defined")
|
|
|
|
|
case _ ⇒
|
|
|
|
|
throw new IllegalArgumentException(s"ActorRefFactory context must be an ActorSystem or ActorContext, got [${context.getClass.getName}]")
|
|
|
|
|
}
|
|
|
|
|
system
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
2017-03-07 14:26:18 +01:00
|
|
|
private final case class SegmentInfo(
|
2016-07-27 13:29:23 +02:00
|
|
|
globalislandOffset: Int, // The island to which the segment belongs
|
|
|
|
|
length: Int, // How many slots are contained by the segment
|
|
|
|
|
globalBaseOffset: Int, // The global slot where this segment starts
|
|
|
|
|
relativeBaseOffset: Int, // the local offset of the slot where this segment starts
|
|
|
|
|
phase: PhaseIsland[Any]) {
|
|
|
|
|
|
|
|
|
|
override def toString: String =
|
|
|
|
|
s"""
|
|
|
|
|
| Segment
|
|
|
|
|
| globalislandOffset = $globalislandOffset
|
|
|
|
|
| length = $length
|
|
|
|
|
| globalBaseOffset = $globalBaseOffset
|
|
|
|
|
| relativeBaseOffset = $relativeBaseOffset
|
|
|
|
|
| phase = $phase
|
|
|
|
|
""".stripMargin
|
|
|
|
|
}
|
|
|
|
|
|
2017-03-07 14:26:18 +01:00
|
|
|
private final case class ForwardWire(
|
2016-07-27 13:29:23 +02:00
|
|
|
islandGlobalOffset: Int,
|
|
|
|
|
from: OutPort,
|
|
|
|
|
toGlobalOffset: Int,
|
|
|
|
|
outStage: Any,
|
|
|
|
|
phase: PhaseIsland[Any]) {
|
|
|
|
|
|
|
|
|
|
override def toString: String = s"ForwardWire(islandId = $islandGlobalOffset, from = $from, toGlobal = $toGlobalOffset, phase = $phase)"
|
|
|
|
|
}
|
|
|
|
|
|
2017-03-08 10:30:00 +01:00
|
|
|
private final case class SavedIslandData(islandGlobalOffset: Int, lastVisitedOffset: Int, skippedSlots: Int, phase: PhaseIsland[Any])
|
2017-03-07 14:26:18 +01:00
|
|
|
|
2016-07-27 13:29:23 +02:00
|
|
|
class IslandTracking(
|
|
|
|
|
val phases: Map[IslandTag, Phase[Any]],
|
|
|
|
|
val settings: ActorMaterializerSettings,
|
|
|
|
|
defaultPhase: Phase[Any],
|
2017-03-07 10:43:18 +01:00
|
|
|
val materializer: PhasedFusingActorMaterializer,
|
|
|
|
|
islandNamePrefix: String) {
|
2016-07-27 13:29:23 +02:00
|
|
|
|
|
|
|
|
import PhasedFusingActorMaterializer.Debug
|
|
|
|
|
|
2017-03-07 12:43:27 +01:00
|
|
|
private var islandNameCounter = 0
|
|
|
|
|
private def nextIslandName(): String = {
|
|
|
|
|
val s = islandNamePrefix + islandNameCounter
|
|
|
|
|
islandNameCounter += 1
|
2017-03-07 10:43:18 +01:00
|
|
|
s
|
|
|
|
|
}
|
|
|
|
|
|
2016-07-27 13:29:23 +02:00
|
|
|
private var currentGlobalOffset = 0
|
|
|
|
|
private var currentSegmentGlobalOffset = 0
|
|
|
|
|
private var currentIslandGlobalOffset = 0
|
|
|
|
|
// The number of slots that belong to segments of other islands encountered so far, from the
|
|
|
|
|
// beginning of the island
|
|
|
|
|
private var currentIslandSkippetSlots = 0
|
|
|
|
|
|
|
|
|
|
private var segments: java.util.ArrayList[SegmentInfo] = null
|
2017-03-07 10:55:46 +01:00
|
|
|
private var activePhases: java.util.ArrayList[PhaseIsland[Any]] = null
|
2016-07-27 13:29:23 +02:00
|
|
|
private var forwardWires: java.util.ArrayList[ForwardWire] = null
|
2017-03-07 14:26:18 +01:00
|
|
|
private var islandStateStack: java.util.ArrayList[SavedIslandData] = null
|
2016-07-27 13:29:23 +02:00
|
|
|
|
2017-03-07 12:43:27 +01:00
|
|
|
private var currentPhase: PhaseIsland[Any] = defaultPhase.apply(settings, materializer, nextIslandName())
|
2016-07-27 13:29:23 +02:00
|
|
|
|
|
|
|
|
def getCurrentPhase: PhaseIsland[Any] = currentPhase
|
|
|
|
|
def getCurrentOffset: Int = currentGlobalOffset
|
|
|
|
|
|
2017-03-08 10:30:00 +01:00
|
|
|
private def completeSegment(): Unit = {
|
2016-07-27 13:29:23 +02:00
|
|
|
val length = currentGlobalOffset - currentSegmentGlobalOffset
|
|
|
|
|
|
2017-03-07 14:26:18 +01:00
|
|
|
if (activePhases eq null) {
|
|
|
|
|
activePhases = new util.ArrayList[PhaseIsland[Any]](8)
|
|
|
|
|
islandStateStack = new java.util.ArrayList[SavedIslandData](4)
|
|
|
|
|
}
|
2017-03-07 10:55:46 +01:00
|
|
|
|
2016-07-27 13:29:23 +02:00
|
|
|
if (length > 0) {
|
|
|
|
|
// We just finished a segment by entering an island.
|
|
|
|
|
val previousSegment = SegmentInfo(
|
|
|
|
|
globalislandOffset = currentIslandGlobalOffset,
|
|
|
|
|
length = currentGlobalOffset - currentSegmentGlobalOffset,
|
|
|
|
|
globalBaseOffset = currentSegmentGlobalOffset,
|
|
|
|
|
relativeBaseOffset = currentSegmentGlobalOffset - currentIslandGlobalOffset - currentIslandSkippetSlots,
|
2017-03-07 10:43:18 +01:00
|
|
|
currentPhase)
|
2016-07-27 13:29:23 +02:00
|
|
|
|
|
|
|
|
// Segment tracking is by demand, we only allocate this list if it is used.
|
|
|
|
|
// If there are no islands, then there is no need to track segments
|
|
|
|
|
if (segments eq null) segments = new java.util.ArrayList[SegmentInfo](8)
|
|
|
|
|
segments.add(previousSegment)
|
|
|
|
|
if (Debug) println(s"Completed segment $previousSegment")
|
|
|
|
|
} else {
|
|
|
|
|
if (Debug) println(s"Skipped zero length segment")
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2017-03-07 14:26:18 +01:00
|
|
|
def enterIsland(tag: IslandTag, attributes: Attributes): Unit = {
|
2016-07-27 13:29:23 +02:00
|
|
|
completeSegment()
|
|
|
|
|
val previousPhase = currentPhase
|
|
|
|
|
val previousIslandOffset = currentIslandGlobalOffset
|
2017-03-08 10:30:00 +01:00
|
|
|
islandStateStack.add(SavedIslandData(previousIslandOffset, currentGlobalOffset, currentIslandSkippetSlots, previousPhase))
|
2016-07-27 13:29:23 +02:00
|
|
|
|
2017-03-07 10:43:18 +01:00
|
|
|
val effectiveSettings = materializer.effectiveSettings(attributes)
|
2017-03-07 12:43:27 +01:00
|
|
|
currentPhase = phases(tag)(effectiveSettings, materializer, nextIslandName())
|
2017-03-07 10:55:46 +01:00
|
|
|
activePhases.add(currentPhase)
|
2016-07-27 13:29:23 +02:00
|
|
|
|
|
|
|
|
// Resolve the phase to be used to materialize this island
|
|
|
|
|
currentIslandGlobalOffset = currentGlobalOffset
|
|
|
|
|
|
|
|
|
|
// The base offset of this segment is the current global offset
|
|
|
|
|
currentSegmentGlobalOffset = currentGlobalOffset
|
2017-03-08 10:30:00 +01:00
|
|
|
currentIslandSkippetSlots = 0
|
|
|
|
|
if (Debug) println(s"Entering island starting at offset = $currentIslandGlobalOffset phase = $currentPhase")
|
2016-07-27 13:29:23 +02:00
|
|
|
}
|
|
|
|
|
|
2017-03-07 14:26:18 +01:00
|
|
|
def exitIsland(): Unit = {
|
|
|
|
|
val parentIsland = islandStateStack.remove(islandStateStack.size() - 1)
|
2016-07-27 13:29:23 +02:00
|
|
|
val previousSegmentLength = completeSegment()
|
|
|
|
|
|
|
|
|
|
// We start a new segment
|
|
|
|
|
currentSegmentGlobalOffset = currentGlobalOffset
|
|
|
|
|
|
|
|
|
|
// We restore data for the island
|
2017-03-07 14:26:18 +01:00
|
|
|
currentIslandGlobalOffset = parentIsland.islandGlobalOffset
|
|
|
|
|
currentPhase = parentIsland.phase
|
2017-03-08 10:30:00 +01:00
|
|
|
currentIslandSkippetSlots = parentIsland.skippedSlots + (currentGlobalOffset - parentIsland.lastVisitedOffset)
|
2016-07-27 13:29:23 +02:00
|
|
|
|
|
|
|
|
if (Debug) println(s"Exited to island starting at offset = $currentIslandGlobalOffset phase = $currentPhase")
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def wireIn(in: InPort, logic: Any): Unit = {
|
|
|
|
|
// The slot for this InPort always belong to the current segment, so resolving its local
|
|
|
|
|
// offset/slot is simple
|
|
|
|
|
val localInSlot = currentGlobalOffset - currentIslandGlobalOffset - currentIslandSkippetSlots
|
|
|
|
|
if (Debug) println(s" wiring port $in inOffs absolute = $currentGlobalOffset local = $localInSlot")
|
|
|
|
|
|
|
|
|
|
// Assign the logic belonging to the current port to its calculated local slot in the island
|
|
|
|
|
currentPhase.assignPort(in, localInSlot, logic)
|
|
|
|
|
|
|
|
|
|
// Check if there was any forward wiring that has this offset/slot as its target
|
|
|
|
|
// First try to find such wiring
|
|
|
|
|
var forwardWire: ForwardWire = null
|
|
|
|
|
if ((forwardWires ne null) && !forwardWires.isEmpty) {
|
|
|
|
|
var i = 0
|
|
|
|
|
while (i < forwardWires.size()) {
|
|
|
|
|
forwardWire = forwardWires.get(i)
|
|
|
|
|
if (forwardWire.toGlobalOffset == currentGlobalOffset) {
|
|
|
|
|
if (Debug) println(s" there is a forward wire to this slot $forwardWire")
|
|
|
|
|
forwardWires.remove(i)
|
|
|
|
|
i = Int.MaxValue // Exit the loop
|
|
|
|
|
} else {
|
|
|
|
|
forwardWire = null // Didn't found it yet
|
|
|
|
|
i += 1
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// If there is a forward wiring we need to resolve it
|
|
|
|
|
if (forwardWire ne null) {
|
|
|
|
|
// The forward wire ends up in the same island
|
|
|
|
|
if (forwardWire.phase eq currentPhase) {
|
|
|
|
|
if (Debug) println(s" in-island forward wiring from port ${forwardWire.from} wired to local slot = $localInSlot")
|
|
|
|
|
forwardWire.phase.assignPort(forwardWire.from, localInSlot, forwardWire.outStage)
|
|
|
|
|
} else {
|
|
|
|
|
if (Debug) println(s" cross island forward wiring from port ${forwardWire.from} wired to local slot = $localInSlot")
|
|
|
|
|
val publisher = forwardWire.phase.createPublisher(forwardWire.from, forwardWire.outStage)
|
|
|
|
|
currentPhase.takePublisher(localInSlot, publisher)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
currentGlobalOffset += 1
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def wireOut(out: OutPort, absoluteOffset: Int, logic: Any): Unit = {
|
|
|
|
|
if (Debug) println(s" wiring $out to absolute = $absoluteOffset")
|
|
|
|
|
|
|
|
|
|
// First check if we are wiring backwards. This is important since we can only do resolution for backward wires.
|
|
|
|
|
// In other cases we need to record the forward wire and resolve it later once its target inSlot has been visited.
|
|
|
|
|
if (absoluteOffset < currentGlobalOffset) {
|
|
|
|
|
if (Debug) println(" backward wiring")
|
|
|
|
|
|
|
|
|
|
if (absoluteOffset >= currentSegmentGlobalOffset) {
|
|
|
|
|
// Wiring is in the same segment, no complex lookup needed
|
|
|
|
|
val localInSlot = absoluteOffset - currentIslandGlobalOffset - currentIslandSkippetSlots
|
|
|
|
|
if (Debug) println(s" in-segment wiring to local ($absoluteOffset - $currentIslandGlobalOffset - $currentIslandSkippetSlots) = $localInSlot")
|
|
|
|
|
currentPhase.assignPort(out, localInSlot, logic)
|
|
|
|
|
} else {
|
|
|
|
|
// Wiring is cross-segment, but we don't know if it is cross-island or not yet
|
|
|
|
|
// We must find the segment to which this slot belongs first
|
|
|
|
|
var i = segments.size() - 1
|
|
|
|
|
var targetSegment: SegmentInfo = segments.get(i)
|
|
|
|
|
// Skip segments that have a higher offset than our slot, until we find the containing segment
|
|
|
|
|
while (i > 0 && targetSegment.globalBaseOffset > absoluteOffset) {
|
|
|
|
|
i -= 1
|
|
|
|
|
targetSegment = segments.get(i)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Independently of the target island the local slot for the target island is calculated the same:
|
|
|
|
|
// - Calculate the relative offset of the local slot in the segment
|
|
|
|
|
// - calculate the island relative offset by adding the island relative base offset of the segment
|
|
|
|
|
val distanceFromSegmentStart = absoluteOffset - targetSegment.globalBaseOffset
|
|
|
|
|
val localInSlot = distanceFromSegmentStart + targetSegment.relativeBaseOffset
|
|
|
|
|
|
|
|
|
|
if (targetSegment.phase eq currentPhase) {
|
|
|
|
|
if (Debug) println(s" cross-segment, in-island wiring to local slot $localInSlot")
|
|
|
|
|
currentPhase.assignPort(out, localInSlot, logic)
|
|
|
|
|
} else {
|
|
|
|
|
if (Debug) println(s" cross-island wiring to local slot $localInSlot in target island")
|
|
|
|
|
val publisher = currentPhase.createPublisher(out, logic)
|
|
|
|
|
targetSegment.phase.takePublisher(localInSlot, publisher)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
} else {
|
|
|
|
|
// We need to record the forward wiring so we can resolve it later
|
|
|
|
|
|
|
|
|
|
// The forward wire tracking data structure is only allocated when needed. Many graphs have no forward wires
|
|
|
|
|
// even though it might have islands.
|
|
|
|
|
if (forwardWires eq null) {
|
|
|
|
|
forwardWires = new java.util.ArrayList[ForwardWire](8)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
val forwardWire = ForwardWire(
|
|
|
|
|
islandGlobalOffset = currentIslandGlobalOffset,
|
|
|
|
|
from = out,
|
|
|
|
|
toGlobalOffset = absoluteOffset,
|
|
|
|
|
logic,
|
2017-03-07 10:43:18 +01:00
|
|
|
currentPhase)
|
2016-07-27 13:29:23 +02:00
|
|
|
|
|
|
|
|
if (Debug) println(s" wiring is forward, recording $forwardWire")
|
|
|
|
|
forwardWires.add(forwardWire)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
2017-03-07 10:55:46 +01:00
|
|
|
def allNestedIslandsReady(): Unit = {
|
|
|
|
|
if (activePhases ne null) {
|
|
|
|
|
var i = 0
|
|
|
|
|
while (i < activePhases.size()) {
|
|
|
|
|
activePhases.get(i).onIslandReady()
|
|
|
|
|
i += 1
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2016-07-27 13:29:23 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
case class PhasedFusingActorMaterializer(
|
|
|
|
|
system: ActorSystem,
|
|
|
|
|
override val settings: ActorMaterializerSettings,
|
|
|
|
|
dispatchers: Dispatchers,
|
|
|
|
|
supervisor: ActorRef,
|
|
|
|
|
haveShutDown: AtomicBoolean,
|
2017-03-07 10:43:18 +01:00
|
|
|
flowNames: SeqActorName) extends ExtendedActorMaterializer {
|
2016-07-27 13:29:23 +02:00
|
|
|
import PhasedFusingActorMaterializer._
|
|
|
|
|
|
|
|
|
|
private val _logger = Logging.getLogger(system, this)
|
|
|
|
|
override def logger: LoggingAdapter = _logger
|
|
|
|
|
|
|
|
|
|
if (settings.fuzzingMode && !system.settings.config.hasPath("akka.stream.secret-test-fuzzing-warning-disable")) {
|
|
|
|
|
_logger.warning("Fuzzing mode is enabled on this system. If you see this warning on your production system then " +
|
|
|
|
|
"set akka.stream.materializer.debug.fuzzing-mode to off.")
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def shutdown(): Unit =
|
|
|
|
|
if (haveShutDown.compareAndSet(false, true)) supervisor ! PoisonPill
|
|
|
|
|
|
|
|
|
|
override def isShutdown: Boolean = haveShutDown.get()
|
|
|
|
|
|
|
|
|
|
override def withNamePrefix(name: String): PhasedFusingActorMaterializer = this.copy(flowNames = flowNames.copy(name))
|
|
|
|
|
|
|
|
|
|
private[this] def createFlowName(): String = flowNames.next()
|
|
|
|
|
|
|
|
|
|
private val defaultInitialAttributes = Attributes(
|
|
|
|
|
Attributes.InputBuffer(settings.initialInputBufferSize, settings.maxInputBufferSize) ::
|
|
|
|
|
ActorAttributes.Dispatcher(settings.dispatcher) ::
|
|
|
|
|
ActorAttributes.SupervisionStrategy(settings.supervisionDecider) ::
|
|
|
|
|
Nil)
|
|
|
|
|
|
|
|
|
|
override def effectiveSettings(opAttr: Attributes): ActorMaterializerSettings = {
|
|
|
|
|
import ActorAttributes._
|
|
|
|
|
import Attributes._
|
|
|
|
|
opAttr.attributeList.foldLeft(settings) { (s, attr) ⇒
|
|
|
|
|
attr match {
|
|
|
|
|
case InputBuffer(initial, max) ⇒ s.withInputBuffer(initial, max)
|
|
|
|
|
case Dispatcher(dispatcher) ⇒ s.withDispatcher(dispatcher)
|
|
|
|
|
case SupervisionStrategy(decider) ⇒ s.withSupervisionStrategy(decider)
|
|
|
|
|
case _ ⇒ s
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2017-03-08 13:10:36 +01:00
|
|
|
override lazy val executionContext: ExecutionContextExecutor = dispatchers.lookup(settings.dispatcher match {
|
|
|
|
|
case Deploy.NoDispatcherGiven ⇒ Dispatchers.DefaultDispatcherId
|
|
|
|
|
case other ⇒ other
|
|
|
|
|
})
|
|
|
|
|
|
2016-07-27 13:29:23 +02:00
|
|
|
override def schedulePeriodically(initialDelay: FiniteDuration, interval: FiniteDuration, task: Runnable): Cancellable =
|
|
|
|
|
system.scheduler.schedule(initialDelay, interval, task)(executionContext)
|
|
|
|
|
|
|
|
|
|
override def scheduleOnce(delay: FiniteDuration, task: Runnable): Cancellable =
|
|
|
|
|
system.scheduler.scheduleOnce(delay, task)(executionContext)
|
|
|
|
|
|
|
|
|
|
override def materialize[Mat](_runnableGraph: Graph[ClosedShape, Mat]): Mat =
|
2017-03-08 13:10:36 +01:00
|
|
|
materialize(_runnableGraph, defaultInitialAttributes)
|
2016-07-27 13:29:23 +02:00
|
|
|
|
|
|
|
|
override def materialize[Mat](
|
|
|
|
|
_runnableGraph: Graph[ClosedShape, Mat],
|
2017-03-08 13:10:36 +01:00
|
|
|
initialAttributes: Attributes): Mat =
|
2016-07-27 13:29:23 +02:00
|
|
|
materialize(
|
|
|
|
|
_runnableGraph,
|
|
|
|
|
initialAttributes,
|
|
|
|
|
PhasedFusingActorMaterializer.DefaultPhase,
|
2017-03-07 10:43:18 +01:00
|
|
|
PhasedFusingActorMaterializer.DefaultPhases)
|
2016-07-27 13:29:23 +02:00
|
|
|
|
2017-03-08 13:10:36 +01:00
|
|
|
override def materialize[Mat](
|
2016-07-27 13:29:23 +02:00
|
|
|
graph: Graph[ClosedShape, Mat],
|
|
|
|
|
initialAttributes: Attributes,
|
|
|
|
|
defaultPhase: Phase[Any],
|
2017-03-07 10:43:18 +01:00
|
|
|
phases: Map[IslandTag, Phase[Any]]): Mat = {
|
|
|
|
|
val islandTracking = new IslandTracking(phases, settings, defaultPhase, this, islandNamePrefix = createFlowName() + "-")
|
2016-07-27 13:29:23 +02:00
|
|
|
|
|
|
|
|
var current: Traversal = graph.traversalBuilder.traversal
|
|
|
|
|
|
|
|
|
|
val attributesStack = new java.util.ArrayDeque[Attributes](8)
|
2017-03-07 14:26:18 +01:00
|
|
|
attributesStack.addLast(initialAttributes and graph.traversalBuilder.attributes)
|
2016-07-27 13:29:23 +02:00
|
|
|
|
|
|
|
|
// TODO: No longer need for a stack
|
|
|
|
|
val traversalStack = new java.util.ArrayDeque[Traversal](16)
|
|
|
|
|
traversalStack.addLast(current)
|
|
|
|
|
|
|
|
|
|
val matValueStack = new java.util.ArrayDeque[Any](8)
|
|
|
|
|
|
|
|
|
|
if (Debug) {
|
|
|
|
|
println(s"--- Materializing layout:")
|
|
|
|
|
TraversalBuilder.printTraversal(current)
|
|
|
|
|
println(s"--- Start materialization")
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Due to how Concat works, we need a stack. This probably can be optimized for the most common cases.
|
|
|
|
|
while (!traversalStack.isEmpty) {
|
|
|
|
|
current = traversalStack.removeLast()
|
|
|
|
|
|
|
|
|
|
while (current ne EmptyTraversal) {
|
|
|
|
|
var nextStep: Traversal = EmptyTraversal
|
|
|
|
|
current match {
|
|
|
|
|
case MaterializeAtomic(mod, outToSlot) ⇒
|
|
|
|
|
if (Debug) println(s"materializing module: $mod")
|
|
|
|
|
val matAndStage = islandTracking.getCurrentPhase.materializeAtomic(mod, attributesStack.getLast)
|
|
|
|
|
val logic = matAndStage._1
|
|
|
|
|
val matValue = matAndStage._2
|
|
|
|
|
if (Debug) println(s" materialized value is $matValue")
|
|
|
|
|
matValueStack.addLast(matValue)
|
|
|
|
|
|
|
|
|
|
val ins = mod.shape.inlets.iterator
|
|
|
|
|
val stageGlobalOffset = islandTracking.getCurrentOffset
|
|
|
|
|
|
|
|
|
|
while (ins.hasNext) {
|
|
|
|
|
val in = ins.next()
|
|
|
|
|
islandTracking.wireIn(in, logic)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
val outs = mod.shape.outlets.iterator
|
|
|
|
|
while (outs.hasNext) {
|
|
|
|
|
val out = outs.next()
|
|
|
|
|
val absoluteTargetSlot = stageGlobalOffset + outToSlot(out.id)
|
|
|
|
|
if (Debug) println(s" wiring offset: ${outToSlot.mkString("[", ",", "]")}")
|
|
|
|
|
islandTracking.wireOut(out, absoluteTargetSlot, logic)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if (Debug) println(s"PUSH: $matValue => $matValueStack")
|
|
|
|
|
|
|
|
|
|
case Concat(first, next) ⇒
|
|
|
|
|
if (next ne EmptyTraversal) traversalStack.add(next)
|
|
|
|
|
nextStep = first
|
|
|
|
|
case Pop ⇒
|
|
|
|
|
val popped = matValueStack.removeLast()
|
|
|
|
|
if (Debug) println(s"POP: $popped => $matValueStack")
|
|
|
|
|
case PushNotUsed ⇒
|
|
|
|
|
matValueStack.addLast(NotUsed)
|
|
|
|
|
if (Debug) println(s"PUSH: NotUsed => $matValueStack")
|
|
|
|
|
case Transform(f) ⇒
|
|
|
|
|
val prev = matValueStack.removeLast()
|
|
|
|
|
val result = f(prev)
|
|
|
|
|
matValueStack.addLast(result)
|
|
|
|
|
if (Debug) println(s"TRFM: $matValueStack")
|
|
|
|
|
case Compose(f) ⇒
|
|
|
|
|
val second = matValueStack.removeLast()
|
|
|
|
|
val first = matValueStack.removeLast()
|
|
|
|
|
val result = f(first, second)
|
|
|
|
|
matValueStack.addLast(result)
|
|
|
|
|
if (Debug) println(s"COMP: $matValueStack")
|
|
|
|
|
case PushAttributes(attr) ⇒
|
|
|
|
|
attributesStack.addLast(attributesStack.getLast and attr)
|
|
|
|
|
if (Debug) println(s"ATTR PUSH: $attr")
|
|
|
|
|
case PopAttributes ⇒
|
|
|
|
|
attributesStack.removeLast()
|
|
|
|
|
if (Debug) println(s"ATTR POP")
|
2017-03-07 14:26:18 +01:00
|
|
|
case EnterIsland(tag) ⇒
|
|
|
|
|
islandTracking.enterIsland(tag, attributesStack.getLast)
|
|
|
|
|
case ExitIsland ⇒
|
|
|
|
|
islandTracking.exitIsland()
|
2016-07-27 13:29:23 +02:00
|
|
|
case _ ⇒
|
|
|
|
|
}
|
|
|
|
|
current = nextStep
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
islandTracking.getCurrentPhase.onIslandReady()
|
2017-03-07 10:55:46 +01:00
|
|
|
islandTracking.allNestedIslandsReady()
|
2016-07-27 13:29:23 +02:00
|
|
|
|
|
|
|
|
if (Debug) println("--- Finished materialization")
|
|
|
|
|
matValueStack.peekLast().asInstanceOf[Mat]
|
|
|
|
|
}
|
|
|
|
|
|
2017-03-08 13:10:36 +01:00
|
|
|
override def makeLogger(logSource: Class[_]): LoggingAdapter =
|
|
|
|
|
Logging(system, logSource)
|
|
|
|
|
|
2016-07-27 13:29:23 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
trait IslandTag
|
|
|
|
|
|
|
|
|
|
trait Phase[M] {
|
2017-03-07 10:43:18 +01:00
|
|
|
def apply(
|
|
|
|
|
effectiveSettings: ActorMaterializerSettings,
|
|
|
|
|
materializer: PhasedFusingActorMaterializer,
|
|
|
|
|
islandName: String): PhaseIsland[M]
|
2016-07-27 13:29:23 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
trait PhaseIsland[M] {
|
|
|
|
|
|
|
|
|
|
def name: String
|
|
|
|
|
|
|
|
|
|
def materializeAtomic(mod: AtomicModule[Shape, Any], attributes: Attributes): (M, Any)
|
|
|
|
|
|
|
|
|
|
def assignPort(in: InPort, slot: Int, logic: M): Unit
|
|
|
|
|
|
|
|
|
|
def assignPort(out: OutPort, slot: Int, logic: M): Unit
|
|
|
|
|
|
|
|
|
|
def createPublisher(out: OutPort, logic: M): Publisher[Any]
|
|
|
|
|
|
|
|
|
|
def takePublisher(slot: Int, publisher: Publisher[Any]): Unit
|
|
|
|
|
|
|
|
|
|
def onIslandReady(): Unit
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
object GraphStageTag extends IslandTag
|
|
|
|
|
|
|
|
|
|
final class GraphStageIsland(
|
2017-03-07 10:43:18 +01:00
|
|
|
effectiveSettings: ActorMaterializerSettings,
|
|
|
|
|
materializer: PhasedFusingActorMaterializer,
|
2017-03-08 13:10:36 +01:00
|
|
|
islandName: String,
|
|
|
|
|
subflowFuser: Option[GraphInterpreterShell ⇒ ActorRef]) extends PhaseIsland[GraphStageLogic] {
|
2016-07-27 13:29:23 +02:00
|
|
|
// TODO: remove these
|
|
|
|
|
private val logicArrayType = Array.empty[GraphStageLogic]
|
|
|
|
|
private[this] val logics = new ArrayList[GraphStageLogic](64)
|
|
|
|
|
// TODO: Resize
|
|
|
|
|
private val connections = Array.ofDim[Connection](64)
|
|
|
|
|
private var maxConnections = 0
|
|
|
|
|
private var outConnections: List[Connection] = Nil
|
|
|
|
|
|
|
|
|
|
val shell = new GraphInterpreterShell(
|
|
|
|
|
connections = null,
|
|
|
|
|
logics = null,
|
2017-03-07 10:43:18 +01:00
|
|
|
effectiveSettings,
|
2016-07-27 13:29:23 +02:00
|
|
|
materializer)
|
|
|
|
|
|
|
|
|
|
override def name: String = "Fusing GraphStages phase"
|
|
|
|
|
|
|
|
|
|
override def materializeAtomic(mod: AtomicModule[Shape, Any], attributes: Attributes): (GraphStageLogic, Any) = {
|
|
|
|
|
// TODO: bail on unknown types
|
|
|
|
|
val stageModule = mod.asInstanceOf[GraphStageModule[Shape, Any]]
|
2017-03-09 17:04:46 +01:00
|
|
|
val stage = stageModule.stage
|
|
|
|
|
val matAndLogic = stage.createLogicAndMaterializedValue(attributes)
|
2016-07-27 13:29:23 +02:00
|
|
|
val logic = matAndLogic._1
|
2017-03-09 17:04:46 +01:00
|
|
|
logic.originalStage = OptionVal.Some(stage)
|
2017-03-02 13:44:19 +01:00
|
|
|
logic.attributes = attributes
|
2016-07-27 13:29:23 +02:00
|
|
|
logics.add(logic)
|
|
|
|
|
logic.stageId = logics.size() - 1
|
|
|
|
|
matAndLogic
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def conn(slot: Int): Connection = {
|
|
|
|
|
maxConnections = math.max(slot, maxConnections)
|
|
|
|
|
val c = connections(slot)
|
|
|
|
|
if (c ne null) c
|
|
|
|
|
else {
|
|
|
|
|
val c2 = new Connection(0, 0, null, 0, null, null, null)
|
|
|
|
|
connections(slot) = c2
|
|
|
|
|
c2
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def outConn(): Connection = {
|
|
|
|
|
val connection = new Connection(0, 0, null, 0, null, null, null)
|
|
|
|
|
outConnections ::= connection
|
|
|
|
|
connection
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def assignPort(in: InPort, slot: Int, logic: GraphStageLogic): Unit = {
|
|
|
|
|
val connection = conn(slot)
|
|
|
|
|
connection.inOwner = logic
|
|
|
|
|
connection.id = slot
|
|
|
|
|
connection.inOwnerId = logic.stageId
|
|
|
|
|
connection.inHandler = logic.handlers(in.id).asInstanceOf[InHandler]
|
|
|
|
|
logic.portToConn(in.id) = connection
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def assignPort(out: OutPort, slot: Int, logic: GraphStageLogic): Unit = {
|
|
|
|
|
val connection = conn(slot)
|
|
|
|
|
connection.outOwner = logic
|
|
|
|
|
connection.id = slot
|
|
|
|
|
connection.outOwnerId = logic.stageId
|
|
|
|
|
connection.outHandler = logic.handlers(logic.inCount + out.id).asInstanceOf[OutHandler]
|
|
|
|
|
logic.portToConn(logic.inCount + out.id) = connection
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def createPublisher(out: OutPort, logic: GraphStageLogic): Publisher[Any] = {
|
|
|
|
|
val boundary = new ActorOutputBoundary(shell, out.toString)
|
|
|
|
|
logics.add(boundary)
|
|
|
|
|
boundary.stageId = logics.size() - 1
|
|
|
|
|
|
|
|
|
|
val connection = outConn()
|
|
|
|
|
boundary.portToConn(boundary.in.id) = connection
|
|
|
|
|
connection.inHandler = boundary.handlers(0).asInstanceOf[InHandler]
|
|
|
|
|
connection.inOwner = boundary
|
|
|
|
|
connection.inOwnerId = boundary.stageId
|
|
|
|
|
|
|
|
|
|
connection.outOwner = logic
|
|
|
|
|
connection.id = -1 // Will be filled later
|
|
|
|
|
connection.outOwnerId = logic.stageId
|
|
|
|
|
connection.outHandler = logic.handlers(logic.inCount + out.id).asInstanceOf[OutHandler]
|
|
|
|
|
logic.portToConn(logic.inCount + out.id) = connection
|
|
|
|
|
|
|
|
|
|
boundary.publisher
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def takePublisher(slot: Int, publisher: Publisher[Any]): Unit = {
|
|
|
|
|
val connection = conn(slot)
|
|
|
|
|
// TODO: proper input port debug string (currently prints the stage)
|
2017-03-02 13:44:19 +01:00
|
|
|
val bufferSize = connection.inOwner.attributes.get[InputBuffer].get.max
|
|
|
|
|
val boundary =
|
|
|
|
|
new BatchingActorInputBoundary(bufferSize, shell, publisher, connection.inOwner.toString)
|
2016-07-27 13:29:23 +02:00
|
|
|
logics.add(boundary)
|
|
|
|
|
boundary.stageId = logics.size() - 1
|
|
|
|
|
|
|
|
|
|
boundary.portToConn(boundary.out.id + boundary.inCount) = connection
|
|
|
|
|
connection.outHandler = boundary.handlers(0).asInstanceOf[OutHandler]
|
|
|
|
|
connection.outOwner = boundary
|
|
|
|
|
connection.outOwnerId = boundary.stageId
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def onIslandReady(): Unit = {
|
|
|
|
|
|
|
|
|
|
val totalConnections = maxConnections + outConnections.size + 1
|
|
|
|
|
val finalConnections = java.util.Arrays.copyOf(connections, totalConnections)
|
|
|
|
|
|
|
|
|
|
var i = maxConnections + 1
|
|
|
|
|
var outConns = outConnections
|
|
|
|
|
while (i < totalConnections) {
|
|
|
|
|
val conn = outConns.head
|
|
|
|
|
outConns = outConns.tail
|
|
|
|
|
finalConnections(i) = conn
|
|
|
|
|
conn.id = i
|
|
|
|
|
i += 1
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
shell.connections = finalConnections
|
|
|
|
|
shell.logics = logics.toArray(logicArrayType)
|
|
|
|
|
|
2017-03-08 13:10:36 +01:00
|
|
|
// TODO make OptionVal
|
|
|
|
|
subflowFuser match {
|
|
|
|
|
case Some(fuseIntoExistingInterperter) ⇒
|
|
|
|
|
fuseIntoExistingInterperter(shell)
|
2016-07-27 13:29:23 +02:00
|
|
|
|
2017-03-08 13:10:36 +01:00
|
|
|
case _ ⇒
|
|
|
|
|
val props = ActorGraphInterpreter.props(shell)
|
|
|
|
|
.withDispatcher(effectiveSettings.dispatcher)
|
2017-03-07 19:40:50 +01:00
|
|
|
materializer.actorOf(props, fullIslandName)
|
|
|
|
|
}
|
|
|
|
|
}
|
2017-03-08 13:10:36 +01:00
|
|
|
|
2017-03-07 19:40:50 +01:00
|
|
|
private def fullIslandName: String = {
|
|
|
|
|
@tailrec def findUsefulName(i: Int): String = {
|
|
|
|
|
if (i == logics.size) islandName
|
|
|
|
|
else logics.get(i) match {
|
|
|
|
|
case _: DownstreamBoundaryStageLogic[_] | _: UpstreamBoundaryStageLogic[_] ⇒
|
|
|
|
|
findUsefulName(i + 1)
|
|
|
|
|
case _ ⇒
|
|
|
|
|
islandName + "-" + logics.get(i).attributes.nameOrDefault()
|
|
|
|
|
}
|
2017-03-08 13:10:36 +01:00
|
|
|
}
|
2017-03-07 19:40:50 +01:00
|
|
|
findUsefulName(0)
|
2016-07-27 13:29:23 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def toString: String = "GraphStagePhase"
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
object SourceModuleIslandTag extends IslandTag
|
|
|
|
|
|
2017-03-07 10:43:18 +01:00
|
|
|
final class SourceModulePhase(
|
|
|
|
|
materializer: PhasedFusingActorMaterializer,
|
|
|
|
|
islandName: String) extends PhaseIsland[Publisher[Any]] {
|
2016-07-27 13:29:23 +02:00
|
|
|
override def name: String = s"SourceModule phase"
|
|
|
|
|
|
|
|
|
|
override def materializeAtomic(mod: AtomicModule[Shape, Any], attributes: Attributes): (Publisher[Any], Any) = {
|
2017-03-07 10:43:18 +01:00
|
|
|
mod.asInstanceOf[SourceModule[Any, Any]].create(MaterializationContext(materializer, attributes,
|
|
|
|
|
islandName + "-" + attributes.nameOrDefault()))
|
2016-07-27 13:29:23 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def assignPort(in: InPort, slot: Int, logic: Publisher[Any]): Unit = ()
|
|
|
|
|
|
|
|
|
|
override def assignPort(out: OutPort, slot: Int, logic: Publisher[Any]): Unit = ()
|
|
|
|
|
|
|
|
|
|
override def createPublisher(out: OutPort, logic: Publisher[Any]): Publisher[Any] = logic
|
|
|
|
|
|
|
|
|
|
override def takePublisher(slot: Int, publisher: Publisher[Any]): Unit =
|
|
|
|
|
throw new UnsupportedOperationException("A Source cannot take a Publisher")
|
|
|
|
|
|
|
|
|
|
override def onIslandReady(): Unit = ()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
object SinkModuleIslandTag extends IslandTag
|
|
|
|
|
|
2017-03-07 10:43:18 +01:00
|
|
|
final class SinkModulePhase(materializer: PhasedFusingActorMaterializer, islandName: String)
|
|
|
|
|
extends PhaseIsland[AnyRef] {
|
2016-07-27 13:29:23 +02:00
|
|
|
override def name: String = s"SourceModule phase"
|
|
|
|
|
var subscriberOrVirtualPublisher: AnyRef = _
|
|
|
|
|
|
|
|
|
|
override def materializeAtomic(mod: AtomicModule[Shape, Any], attributes: Attributes): (AnyRef, Any) = {
|
|
|
|
|
val subAndMat =
|
2017-03-07 10:43:18 +01:00
|
|
|
mod.asInstanceOf[SinkModule[Any, Any]].create(MaterializationContext(materializer, attributes,
|
|
|
|
|
islandName + "-" + attributes.nameOrDefault()))
|
2016-07-27 13:29:23 +02:00
|
|
|
|
|
|
|
|
subscriberOrVirtualPublisher = subAndMat._1
|
|
|
|
|
(subscriberOrVirtualPublisher, subAndMat._2)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def assignPort(in: InPort, slot: Int, logic: AnyRef): Unit = ()
|
|
|
|
|
|
|
|
|
|
override def assignPort(out: OutPort, slot: Int, logic: AnyRef): Unit = ()
|
|
|
|
|
|
|
|
|
|
override def createPublisher(out: OutPort, logic: AnyRef): Publisher[Any] = {
|
|
|
|
|
throw new UnsupportedOperationException("A Sink cannot create a Publisher")
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def takePublisher(slot: Int, publisher: Publisher[Any]): Unit = {
|
|
|
|
|
subscriberOrVirtualPublisher match {
|
2017-03-07 19:40:50 +01:00
|
|
|
case v: VirtualPublisher[_] ⇒ v.registerPublisher(publisher)
|
|
|
|
|
case s: Subscriber[Any] @unchecked ⇒ publisher.subscribe(s)
|
2016-07-27 13:29:23 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def onIslandReady(): Unit = ()
|
|
|
|
|
}
|
2017-03-03 10:49:34 +01:00
|
|
|
|
|
|
|
|
object ProcessorModuleIslandTag extends IslandTag
|
|
|
|
|
|
2017-03-07 10:43:18 +01:00
|
|
|
final class ProcessorModulePhase(materializer: PhasedFusingActorMaterializer, islandName: String)
|
|
|
|
|
extends PhaseIsland[Processor[Any, Any]] {
|
2017-03-03 10:49:34 +01:00
|
|
|
override def name: String = "ProcessorModulePhase"
|
|
|
|
|
private[this] var processor: Processor[Any, Any] = _
|
|
|
|
|
|
|
|
|
|
override def materializeAtomic(mod: AtomicModule[Shape, Any], attributes: Attributes): (Processor[Any, Any], Any) = {
|
|
|
|
|
val procAndMat = mod.asInstanceOf[ProcessorModule[Any, Any, Any]].createProcessor()
|
|
|
|
|
processor = procAndMat._1
|
|
|
|
|
procAndMat
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def assignPort(in: InPort, slot: Int, logic: Processor[Any, Any]): Unit = ()
|
|
|
|
|
override def assignPort(out: OutPort, slot: Int, logic: Processor[Any, Any]): Unit = ()
|
|
|
|
|
|
|
|
|
|
override def createPublisher(out: OutPort, logic: Processor[Any, Any]): Publisher[Any] = logic
|
|
|
|
|
override def takePublisher(slot: Int, publisher: Publisher[Any]): Unit = publisher.subscribe(processor)
|
|
|
|
|
|
|
|
|
|
override def onIslandReady(): Unit = ()
|
2017-03-07 10:43:18 +01:00
|
|
|
}
|
2017-03-07 12:39:15 +01:00
|
|
|
|
|
|
|
|
object TlsModuleIslandTag extends IslandTag
|
|
|
|
|
|
|
|
|
|
final class TlsModulePhase(settings: ActorMaterializerSettings, materializer: PhasedFusingActorMaterializer, islandName: String) extends PhaseIsland[NotUsed] {
|
|
|
|
|
def name: String = "TlsModulePhase"
|
|
|
|
|
|
|
|
|
|
var tlsActor: ActorRef = _
|
|
|
|
|
var publishers: Vector[ActorPublisher[Any]] = _
|
|
|
|
|
|
|
|
|
|
def materializeAtomic(mod: AtomicModule[Shape, Any], attributes: Attributes): (NotUsed, Any) = {
|
|
|
|
|
val tls = mod.asInstanceOf[TlsModule]
|
|
|
|
|
|
2017-03-08 10:30:00 +01:00
|
|
|
val props =
|
|
|
|
|
TLSActor.props(settings, tls.createSSLEngine, tls.verifySession, tls.closing).withDispatcher(settings.dispatcher)
|
2017-03-07 12:39:15 +01:00
|
|
|
tlsActor = materializer.actorOf(props, islandName)
|
|
|
|
|
def factory(id: Int) = new ActorPublisher[Any](tlsActor) {
|
|
|
|
|
override val wakeUpMsg = FanOut.SubstreamSubscribePending(id)
|
|
|
|
|
}
|
|
|
|
|
publishers = Vector.tabulate(2)(factory)
|
|
|
|
|
tlsActor ! FanOut.ExposedPublishers(publishers)
|
|
|
|
|
(NotUsed, NotUsed)
|
|
|
|
|
}
|
|
|
|
|
def assignPort(in: InPort, slot: Int, logic: NotUsed): Unit = ()
|
|
|
|
|
def assignPort(out: OutPort, slot: Int, logic: NotUsed): Unit = ()
|
|
|
|
|
|
|
|
|
|
def createPublisher(out: OutPort, logic: NotUsed): Publisher[Any] =
|
|
|
|
|
publishers(out.id)
|
|
|
|
|
|
|
|
|
|
def takePublisher(slot: Int, publisher: Publisher[Any]): Unit =
|
2017-03-08 10:30:00 +01:00
|
|
|
publisher.subscribe(FanIn.SubInput[Any](tlsActor, 1 - slot))
|
2017-03-07 12:39:15 +01:00
|
|
|
|
|
|
|
|
def onIslandReady(): Unit = ()
|
|
|
|
|
}
|