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
|
|
@ -8,7 +8,7 @@ import akka.stream.ActorAttributes.SupervisionStrategy
|
|||
import akka.stream.Supervision.{ Stop, stoppingDecider }
|
||||
import akka.stream.impl.QueueSink.{ Output, Pull }
|
||||
import akka.stream.impl.fusing.GraphInterpreter
|
||||
import akka.{ Done, NotUsed }
|
||||
import akka.{ Done, NotUsed, annotation }
|
||||
import akka.actor.{ Actor, ActorRef, Props }
|
||||
import akka.stream.Attributes.InputBuffer
|
||||
import akka.stream._
|
||||
|
|
@ -36,13 +36,14 @@ import scala.compat.java8.FutureConverters._
|
|||
import scala.compat.java8.OptionConverters._
|
||||
import java.util.Optional
|
||||
|
||||
import akka.annotation.{ DoNotInherit, InternalApi }
|
||||
import akka.event.Logging
|
||||
import akka.util.OptionVal
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
abstract class SinkModule[-In, Mat](val shape: SinkShape[In]) extends AtomicModule[SinkShape[In], Mat] {
|
||||
@DoNotInherit private[akka] abstract class SinkModule[-In, Mat](val shape: SinkShape[In]) extends AtomicModule[SinkShape[In], Mat] {
|
||||
|
||||
/**
|
||||
* Create the Subscriber or VirtualPublisher that consumes the incoming
|
||||
|
|
@ -82,7 +83,7 @@ abstract class SinkModule[-In, Mat](val shape: SinkShape[In]) extends AtomicModu
|
|||
* elements to fill the internal buffers it will assert back-pressure until
|
||||
* a subscriber connects and creates demand for elements to be emitted.
|
||||
*/
|
||||
private[akka] class PublisherSink[In](val attributes: Attributes, shape: SinkShape[In]) extends SinkModule[In, Publisher[In]](shape) {
|
||||
@InternalApi private[akka] class PublisherSink[In](val attributes: Attributes, shape: SinkShape[In]) extends SinkModule[In, Publisher[In]](shape) {
|
||||
|
||||
/*
|
||||
* This method is the reason why SinkModule.create may return something that is
|
||||
|
|
@ -101,7 +102,7 @@ private[akka] class PublisherSink[In](val attributes: Attributes, shape: SinkSha
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] final class FanoutPublisherSink[In](
|
||||
@InternalApi private[akka] final class FanoutPublisherSink[In](
|
||||
val attributes: Attributes,
|
||||
shape: SinkShape[In])
|
||||
extends SinkModule[In, Publisher[In]](shape) {
|
||||
|
|
@ -128,7 +129,7 @@ private[akka] final class FanoutPublisherSink[In](
|
|||
* INTERNAL API
|
||||
* Attaches a subscriber to this stream.
|
||||
*/
|
||||
final class SubscriberSink[In](subscriber: Subscriber[In], val attributes: Attributes, shape: SinkShape[In]) extends SinkModule[In, NotUsed](shape) {
|
||||
@InternalApi private[akka] final class SubscriberSink[In](subscriber: Subscriber[In], val attributes: Attributes, shape: SinkShape[In]) extends SinkModule[In, NotUsed](shape) {
|
||||
|
||||
override def create(context: MaterializationContext) = (subscriber, NotUsed)
|
||||
|
||||
|
|
@ -140,7 +141,7 @@ final class SubscriberSink[In](subscriber: Subscriber[In], val attributes: Attri
|
|||
* INTERNAL API
|
||||
* A sink that immediately cancels its upstream upon materialization.
|
||||
*/
|
||||
final class CancelSink(val attributes: Attributes, shape: SinkShape[Any]) extends SinkModule[Any, NotUsed](shape) {
|
||||
@InternalApi private[akka] final class CancelSink(val attributes: Attributes, shape: SinkShape[Any]) extends SinkModule[Any, NotUsed](shape) {
|
||||
override def create(context: MaterializationContext): (Subscriber[Any], NotUsed) = (new CancellingSubscriber[Any], NotUsed)
|
||||
override protected def newInstance(shape: SinkShape[Any]): SinkModule[Any, NotUsed] = new CancelSink(attributes, shape)
|
||||
override def withAttributes(attr: Attributes): SinkModule[Any, NotUsed] = new CancelSink(attr, amendShape(attr))
|
||||
|
|
@ -151,7 +152,7 @@ final class CancelSink(val attributes: Attributes, shape: SinkShape[Any]) extend
|
|||
* Creates and wraps an actor into [[org.reactivestreams.Subscriber]] from the given `props`,
|
||||
* which should be [[akka.actor.Props]] for an [[akka.stream.actor.ActorSubscriber]].
|
||||
*/
|
||||
final class ActorSubscriberSink[In](props: Props, val attributes: Attributes, shape: SinkShape[In]) extends SinkModule[In, ActorRef](shape) {
|
||||
@InternalApi 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 = ActorMaterializerHelper.downcast(context.materializer).actorOf(context, props)
|
||||
|
|
@ -165,9 +166,9 @@ final class ActorSubscriberSink[In](props: Props, val attributes: Attributes, sh
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
final class ActorRefSink[In](ref: ActorRef, onCompleteMessage: Any,
|
||||
val attributes: Attributes,
|
||||
shape: SinkShape[In]) extends SinkModule[In, NotUsed](shape) {
|
||||
@InternalApi private[akka] final class ActorRefSink[In](ref: ActorRef, onCompleteMessage: Any,
|
||||
val attributes: Attributes,
|
||||
shape: SinkShape[In]) extends SinkModule[In, NotUsed](shape) {
|
||||
|
||||
override def create(context: MaterializationContext) = {
|
||||
val actorMaterializer = ActorMaterializerHelper.downcast(context.materializer)
|
||||
|
|
@ -184,7 +185,10 @@ final class ActorRefSink[In](ref: ActorRef, onCompleteMessage: Any,
|
|||
new ActorRefSink[In](ref, onCompleteMessage, attr, amendShape(attr))
|
||||
}
|
||||
|
||||
final class LastOptionStage[T] extends GraphStageWithMaterializedValue[SinkShape[T], Future[Option[T]]] {
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi private[akka] final class LastOptionStage[T] extends GraphStageWithMaterializedValue[SinkShape[T], Future[Option[T]]] {
|
||||
|
||||
val in: Inlet[T] = Inlet("lastOption.in")
|
||||
|
||||
|
|
@ -222,7 +226,10 @@ final class LastOptionStage[T] extends GraphStageWithMaterializedValue[SinkShape
|
|||
override def toString: String = "LastOptionStage"
|
||||
}
|
||||
|
||||
final class HeadOptionStage[T] extends GraphStageWithMaterializedValue[SinkShape[T], Future[Option[T]]] {
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi private[akka] final class HeadOptionStage[T] extends GraphStageWithMaterializedValue[SinkShape[T], Future[Option[T]]] {
|
||||
|
||||
val in: Inlet[T] = Inlet("headOption.in")
|
||||
|
||||
|
|
@ -255,7 +262,10 @@ final class HeadOptionStage[T] extends GraphStageWithMaterializedValue[SinkShape
|
|||
override def toString: String = "HeadOptionStage"
|
||||
}
|
||||
|
||||
final class SeqStage[T] extends GraphStageWithMaterializedValue[SinkShape[T], Future[immutable.Seq[T]]] {
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi private[akka] final class SeqStage[T] extends GraphStageWithMaterializedValue[SinkShape[T], Future[immutable.Seq[T]]] {
|
||||
val in = Inlet[T]("seq.in")
|
||||
|
||||
override def toString: String = "SeqStage"
|
||||
|
|
@ -294,7 +304,10 @@ final class SeqStage[T] extends GraphStageWithMaterializedValue[SinkShape[T], Fu
|
|||
}
|
||||
}
|
||||
|
||||
private[stream] object QueueSink {
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi private[akka] object QueueSink {
|
||||
sealed trait Output[+T]
|
||||
final case class Pull[T](promise: Promise[Option[T]]) extends Output[T]
|
||||
case object Cancel extends Output[Nothing]
|
||||
|
|
@ -303,7 +316,7 @@ private[stream] object QueueSink {
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
final class QueueSink[T]() extends GraphStageWithMaterializedValue[SinkShape[T], SinkQueueWithCancel[T]] {
|
||||
@InternalApi private[akka] final class QueueSink[T]() extends GraphStageWithMaterializedValue[SinkShape[T], SinkQueueWithCancel[T]] {
|
||||
type Requested[E] = Promise[Option[E]]
|
||||
|
||||
val in = Inlet[T]("queueSink.in")
|
||||
|
|
@ -395,7 +408,10 @@ final class QueueSink[T]() extends GraphStageWithMaterializedValue[SinkShape[T],
|
|||
}
|
||||
}
|
||||
|
||||
final class SinkQueueAdapter[T](delegate: SinkQueueWithCancel[T]) extends akka.stream.javadsl.SinkQueueWithCancel[T] {
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi private[akka] final class SinkQueueAdapter[T](delegate: SinkQueueWithCancel[T]) extends akka.stream.javadsl.SinkQueueWithCancel[T] {
|
||||
import akka.dispatch.ExecutionContexts.{ sameThreadExecutionContext ⇒ same }
|
||||
def pull(): CompletionStage[Optional[T]] = delegate.pull().map(_.asJava)(same).toJava
|
||||
def cancel(): Unit = delegate.cancel()
|
||||
|
|
@ -407,7 +423,7 @@ final class SinkQueueAdapter[T](delegate: SinkQueueWithCancel[T]) extends akka.s
|
|||
*
|
||||
* Helper class to be able to express collection as a fold using mutable data
|
||||
*/
|
||||
private[akka] final class CollectorState[T, R](val collector: java.util.stream.Collector[T, Any, R]) {
|
||||
@InternalApi private[akka] final class CollectorState[T, R](val collector: java.util.stream.Collector[T, Any, R]) {
|
||||
lazy val accumulated = collector.supplier().get()
|
||||
private lazy val accumulator = collector.accumulator()
|
||||
|
||||
|
|
@ -424,7 +440,7 @@ private[akka] final class CollectorState[T, R](val collector: java.util.stream.C
|
|||
*
|
||||
* Helper class to be able to express reduce as a fold for parallel collector
|
||||
*/
|
||||
private[akka] final class ReducerState[T, R](val collector: java.util.stream.Collector[T, Any, R]) {
|
||||
@InternalApi private[akka] final class ReducerState[T, R](val collector: java.util.stream.Collector[T, Any, R]) {
|
||||
private var reduced: Any = null.asInstanceOf[Any]
|
||||
private lazy val combiner = collector.combiner()
|
||||
|
||||
|
|
@ -440,7 +456,7 @@ private[akka] final class ReducerState[T, R](val collector: java.util.stream.Col
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
final private[stream] class LazySink[T, M](sinkFactory: T ⇒ Future[Sink[T, M]], zeroMat: () ⇒ M) extends GraphStageWithMaterializedValue[SinkShape[T], Future[M]] {
|
||||
@InternalApi final private[stream] class LazySink[T, M](sinkFactory: T ⇒ Future[Sink[T, M]], zeroMat: () ⇒ M) extends GraphStageWithMaterializedValue[SinkShape[T], Future[M]] {
|
||||
val in = Inlet[T]("lazySink.in")
|
||||
override def initialAttributes = DefaultAttributes.lazySink
|
||||
override val shape: SinkShape[T] = SinkShape.of(in)
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue