Add @InternalApi annotations and private markers on internal apis (#22563)
* #22506 Mark materializer extension as @InternalApi * #22506 Added missing annotations on internal apis in the fusing package * #22506 Missing annotations in io package * #22506 Add internal api annotations in impl package * #22563 more hiding of the classes in the impl package * #22563 Formatting fixes * #22506 Fix private access in stream tcl tests
This commit is contained in:
parent
6434cbe868
commit
2a9c0370e0
60 changed files with 568 additions and 316 deletions
|
|
@ -9,6 +9,7 @@ import java.util.concurrent.atomic.AtomicBoolean
|
|||
|
||||
import akka.NotUsed
|
||||
import akka.actor.{ ActorContext, ActorRef, ActorRefFactory, ActorSystem, Cancellable, Deploy, ExtendedActorSystem, PoisonPill }
|
||||
import akka.annotation.{ DoNotInherit, InternalApi }
|
||||
import akka.dispatch.Dispatchers
|
||||
import akka.event.{ Logging, LoggingAdapter }
|
||||
import akka.stream.Attributes.InputBuffer
|
||||
|
|
@ -19,18 +20,18 @@ import akka.stream.impl.fusing.GraphInterpreter.Connection
|
|||
import akka.stream.impl.fusing._
|
||||
import akka.stream.impl.io.{ TLSActor, TlsModule }
|
||||
import akka.stream.stage.{ GraphStageLogic, InHandler, OutHandler }
|
||||
import akka.util.OptionVal
|
||||
import org.reactivestreams.{ Processor, Publisher, Subscriber, Subscription }
|
||||
import org.reactivestreams.{ Processor, Publisher, Subscriber }
|
||||
|
||||
import scala.collection.immutable.Map
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
import scala.concurrent.ExecutionContextExecutor
|
||||
import scala.annotation.tailrec
|
||||
import akka.stream.impl.fusing.GraphInterpreter.DownstreamBoundaryStageLogic
|
||||
import akka.stream.impl.fusing.GraphInterpreter.UpstreamBoundaryStageLogic
|
||||
import akka.util.OptionVal
|
||||
|
||||
object PhasedFusingActorMaterializer {
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi private[akka] object PhasedFusingActorMaterializer {
|
||||
|
||||
val Debug = false
|
||||
|
||||
|
|
@ -61,7 +62,7 @@ object PhasedFusingActorMaterializer {
|
|||
},
|
||||
GraphStageTag → DefaultPhase)
|
||||
|
||||
def apply(settings: ActorMaterializerSettings)(implicit context: ActorRefFactory): ActorMaterializer = {
|
||||
@InternalApi private[akka] def apply(settings: ActorMaterializerSettings)(implicit context: ActorRefFactory): ActorMaterializer = {
|
||||
val haveShutDown = new AtomicBoolean(false)
|
||||
val system = actorSystemOf(context)
|
||||
val materializerSettings = ActorMaterializerSettings(system)
|
||||
|
|
@ -121,7 +122,7 @@ private final case class ForwardWire(
|
|||
|
||||
private final case class SavedIslandData(islandGlobalOffset: Int, lastVisitedOffset: Int, skippedSlots: Int, phase: PhaseIsland[Any])
|
||||
|
||||
class IslandTracking(
|
||||
@InternalApi private[akka] class IslandTracking(
|
||||
val phases: Map[IslandTag, Phase[Any]],
|
||||
val settings: ActorMaterializerSettings,
|
||||
defaultPhase: Phase[Any],
|
||||
|
|
@ -151,8 +152,8 @@ class IslandTracking(
|
|||
|
||||
private var currentPhase: PhaseIsland[Any] = defaultPhase.apply(settings, materializer, nextIslandName())
|
||||
|
||||
def getCurrentPhase: PhaseIsland[Any] = currentPhase
|
||||
def getCurrentOffset: Int = currentGlobalOffset
|
||||
@InternalApi private[akka] def getCurrentPhase: PhaseIsland[Any] = currentPhase
|
||||
@InternalApi private[akka] def getCurrentOffset: Int = currentGlobalOffset
|
||||
|
||||
private def completeSegment(): Unit = {
|
||||
val length = currentGlobalOffset - currentSegmentGlobalOffset
|
||||
|
|
@ -181,7 +182,7 @@ class IslandTracking(
|
|||
}
|
||||
}
|
||||
|
||||
def enterIsland(tag: IslandTag, attributes: Attributes): Unit = {
|
||||
@InternalApi private[akka] def enterIsland(tag: IslandTag, attributes: Attributes): Unit = {
|
||||
completeSegment()
|
||||
val previousPhase = currentPhase
|
||||
val previousIslandOffset = currentIslandGlobalOffset
|
||||
|
|
@ -200,7 +201,7 @@ class IslandTracking(
|
|||
if (Debug) println(s"Entering island starting at offset = $currentIslandGlobalOffset phase = $currentPhase")
|
||||
}
|
||||
|
||||
def exitIsland(): Unit = {
|
||||
@InternalApi private[akka] def exitIsland(): Unit = {
|
||||
val parentIsland = islandStateStack.remove(islandStateStack.size() - 1)
|
||||
val previousSegmentLength = completeSegment()
|
||||
|
||||
|
|
@ -215,7 +216,7 @@ class IslandTracking(
|
|||
if (Debug) println(s"Exited to island starting at offset = $currentIslandGlobalOffset phase = $currentPhase")
|
||||
}
|
||||
|
||||
def wireIn(in: InPort, logic: Any): Unit = {
|
||||
@InternalApi private[akka] 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
|
||||
|
|
@ -258,7 +259,7 @@ class IslandTracking(
|
|||
currentGlobalOffset += 1
|
||||
}
|
||||
|
||||
def wireOut(out: OutPort, absoluteOffset: Int, logic: Any): Unit = {
|
||||
@InternalApi private[akka] 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.
|
||||
|
|
@ -319,7 +320,7 @@ class IslandTracking(
|
|||
|
||||
}
|
||||
|
||||
def allNestedIslandsReady(): Unit = {
|
||||
@InternalApi private[akka] def allNestedIslandsReady(): Unit = {
|
||||
if (activePhases ne null) {
|
||||
var i = 0
|
||||
while (i < activePhases.size()) {
|
||||
|
|
@ -331,7 +332,10 @@ class IslandTracking(
|
|||
|
||||
}
|
||||
|
||||
case class PhasedFusingActorMaterializer(
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi private[akka] case class PhasedFusingActorMaterializer(
|
||||
system: ActorSystem,
|
||||
override val settings: ActorMaterializerSettings,
|
||||
dispatchers: Dispatchers,
|
||||
|
|
@ -547,16 +551,25 @@ case class PhasedFusingActorMaterializer(
|
|||
|
||||
}
|
||||
|
||||
trait IslandTag
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@DoNotInherit private[akka] trait IslandTag
|
||||
|
||||
trait Phase[M] {
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@DoNotInherit private[akka] trait Phase[M] {
|
||||
def apply(
|
||||
effectiveSettings: ActorMaterializerSettings,
|
||||
materializer: PhasedFusingActorMaterializer,
|
||||
islandName: String): PhaseIsland[M]
|
||||
}
|
||||
|
||||
trait PhaseIsland[M] {
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@DoNotInherit private[akka] trait PhaseIsland[M] {
|
||||
|
||||
def name: String
|
||||
|
||||
|
|
@ -574,9 +587,15 @@ trait PhaseIsland[M] {
|
|||
|
||||
}
|
||||
|
||||
object GraphStageTag extends IslandTag
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi private[akka] object GraphStageTag extends IslandTag
|
||||
|
||||
final class GraphStageIsland(
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi private[akka] final class GraphStageIsland(
|
||||
effectiveSettings: ActorMaterializerSettings,
|
||||
materializer: PhasedFusingActorMaterializer,
|
||||
islandName: String,
|
||||
|
|
@ -739,9 +758,15 @@ final class GraphStageIsland(
|
|||
override def toString: String = "GraphStagePhase"
|
||||
}
|
||||
|
||||
object SourceModuleIslandTag extends IslandTag
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi private[akka] object SourceModuleIslandTag extends IslandTag
|
||||
|
||||
final class SourceModulePhase(
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi private[akka] final class SourceModulePhase(
|
||||
materializer: PhasedFusingActorMaterializer,
|
||||
islandName: String) extends PhaseIsland[Publisher[Any]] {
|
||||
override def name: String = s"SourceModule phase"
|
||||
|
|
@ -763,9 +788,15 @@ final class SourceModulePhase(
|
|||
override def onIslandReady(): Unit = ()
|
||||
}
|
||||
|
||||
object SinkModuleIslandTag extends IslandTag
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi private[akka] object SinkModuleIslandTag extends IslandTag
|
||||
|
||||
final class SinkModulePhase(materializer: PhasedFusingActorMaterializer, islandName: String)
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi private[akka] final class SinkModulePhase(materializer: PhasedFusingActorMaterializer, islandName: String)
|
||||
extends PhaseIsland[AnyRef] {
|
||||
override def name: String = s"SourceModule phase"
|
||||
var subscriberOrVirtualPublisher: AnyRef = _
|
||||
|
|
@ -797,9 +828,15 @@ final class SinkModulePhase(materializer: PhasedFusingActorMaterializer, islandN
|
|||
override def onIslandReady(): Unit = ()
|
||||
}
|
||||
|
||||
object ProcessorModuleIslandTag extends IslandTag
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi private[akka] object ProcessorModuleIslandTag extends IslandTag
|
||||
|
||||
final class ProcessorModulePhase(materializer: PhasedFusingActorMaterializer, islandName: String)
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi private[akka] final class ProcessorModulePhase(materializer: PhasedFusingActorMaterializer, islandName: String)
|
||||
extends PhaseIsland[Processor[Any, Any]] {
|
||||
override def name: String = "ProcessorModulePhase"
|
||||
private[this] var processor: Processor[Any, Any] = _
|
||||
|
|
@ -819,9 +856,15 @@ final class ProcessorModulePhase(materializer: PhasedFusingActorMaterializer, is
|
|||
override def onIslandReady(): Unit = ()
|
||||
}
|
||||
|
||||
object TlsModuleIslandTag extends IslandTag
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi private[akka] object TlsModuleIslandTag extends IslandTag
|
||||
|
||||
final class TlsModulePhase(settings: ActorMaterializerSettings, materializer: PhasedFusingActorMaterializer, islandName: String) extends PhaseIsland[NotUsed] {
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi private[akka] final class TlsModulePhase(settings: ActorMaterializerSettings, materializer: PhasedFusingActorMaterializer, islandName: String) extends PhaseIsland[NotUsed] {
|
||||
def name: String = "TlsModulePhase"
|
||||
|
||||
var tlsActor: ActorRef = _
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue