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
|
|
@ -4,15 +4,18 @@
|
|||
package akka.stream.impl.fusing
|
||||
|
||||
import java.util.concurrent.TimeUnit.NANOSECONDS
|
||||
|
||||
import akka.annotation.{ DoNotInherit, InternalApi }
|
||||
import akka.event.Logging.LogLevel
|
||||
import akka.event.{ LogSource, Logging, LoggingAdapter }
|
||||
import akka.stream.Attributes.{ InputBuffer, LogLevels }
|
||||
import akka.stream.OverflowStrategies._
|
||||
import akka.stream.impl.fusing.GraphStages.SimpleLinearGraphStage
|
||||
import akka.stream.impl.{ Buffer ⇒ BufferImpl, Stages, ReactiveStreamsCompliance }
|
||||
import akka.stream.scaladsl.{ SourceQueue, Source }
|
||||
import akka.stream.impl.{ ReactiveStreamsCompliance, Stages, Buffer ⇒ BufferImpl }
|
||||
import akka.stream.scaladsl.{ Source, SourceQueue }
|
||||
import akka.stream.stage._
|
||||
import akka.stream.{ Supervision, _ }
|
||||
|
||||
import scala.annotation.tailrec
|
||||
import scala.collection.immutable
|
||||
import scala.collection.immutable.VectorBuilder
|
||||
|
|
@ -20,13 +23,14 @@ import scala.concurrent.Future
|
|||
import scala.util.control.NonFatal
|
||||
import scala.util.{ Failure, Success, Try }
|
||||
import akka.stream.ActorAttributes.SupervisionStrategy
|
||||
|
||||
import scala.concurrent.duration.{ FiniteDuration, _ }
|
||||
import akka.stream.impl.Stages.DefaultAttributes
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
final case class Map[In, Out](f: In ⇒ Out) extends GraphStage[FlowShape[In, Out]] {
|
||||
@InternalApi private[akka] final case class Map[In, Out](f: In ⇒ Out) extends GraphStage[FlowShape[In, Out]] {
|
||||
val in = Inlet[In]("Map.in")
|
||||
val out = Outlet[Out]("Map.out")
|
||||
override val shape = FlowShape(in, out)
|
||||
|
|
@ -58,7 +62,7 @@ final case class Map[In, Out](f: In ⇒ Out) extends GraphStage[FlowShape[In, Ou
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
final case class Filter[T](p: T ⇒ Boolean) extends SimpleLinearGraphStage[T] {
|
||||
@InternalApi private[akka] final case class Filter[T](p: T ⇒ Boolean) extends SimpleLinearGraphStage[T] {
|
||||
override def initialAttributes: Attributes = DefaultAttributes.filter
|
||||
|
||||
override def toString: String = "Filter"
|
||||
|
|
@ -92,7 +96,7 @@ final case class Filter[T](p: T ⇒ Boolean) extends SimpleLinearGraphStage[T] {
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
final case class TakeWhile[T](p: T ⇒ Boolean, inclusive: Boolean = false) extends SimpleLinearGraphStage[T] {
|
||||
@InternalApi private[akka] final case class TakeWhile[T](p: T ⇒ Boolean, inclusive: Boolean = false) extends SimpleLinearGraphStage[T] {
|
||||
override def initialAttributes: Attributes = DefaultAttributes.takeWhile
|
||||
|
||||
override def toString: String = "TakeWhile"
|
||||
|
|
@ -129,7 +133,7 @@ final case class TakeWhile[T](p: T ⇒ Boolean, inclusive: Boolean = false) exte
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
final case class DropWhile[T](p: T ⇒ Boolean) extends SimpleLinearGraphStage[T] {
|
||||
@InternalApi private[akka] final case class DropWhile[T](p: T ⇒ Boolean) extends SimpleLinearGraphStage[T] {
|
||||
override def initialAttributes: Attributes = DefaultAttributes.dropWhile
|
||||
|
||||
def createLogic(inheritedAttributes: Attributes) = new SupervisedGraphStageLogic(inheritedAttributes, shape) with InHandler with OutHandler {
|
||||
|
|
@ -161,7 +165,7 @@ final case class DropWhile[T](p: T ⇒ Boolean) extends SimpleLinearGraphStage[T
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
abstract class SupervisedGraphStageLogic(inheritedAttributes: Attributes, shape: Shape) extends GraphStageLogic(shape) {
|
||||
@DoNotInherit private[akka] abstract class SupervisedGraphStageLogic(inheritedAttributes: Attributes, shape: Shape) extends GraphStageLogic(shape) {
|
||||
private lazy val decider = inheritedAttributes.get[SupervisionStrategy].map(_.decider).getOrElse(Supervision.stoppingDecider)
|
||||
|
||||
def withSupervision[T](f: () ⇒ T): Option[T] =
|
||||
|
|
@ -194,7 +198,7 @@ private[stream] object Collect {
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
final case class Collect[In, Out](pf: PartialFunction[In, Out]) extends GraphStage[FlowShape[In, Out]] {
|
||||
@InternalApi private[akka] final case class Collect[In, Out](pf: PartialFunction[In, Out]) extends GraphStage[FlowShape[In, Out]] {
|
||||
val in = Inlet[In]("Collect.in")
|
||||
val out = Outlet[Out]("Collect.out")
|
||||
override val shape = FlowShape(in, out)
|
||||
|
|
@ -228,7 +232,7 @@ final case class Collect[In, Out](pf: PartialFunction[In, Out]) extends GraphSta
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
final case class Recover[T](pf: PartialFunction[Throwable, T]) extends SimpleLinearGraphStage[T] {
|
||||
@InternalApi private[akka] final case class Recover[T](pf: PartialFunction[Throwable, T]) extends SimpleLinearGraphStage[T] {
|
||||
override protected def initialAttributes: Attributes = DefaultAttributes.recover
|
||||
|
||||
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new GraphStageLogic(shape) with InHandler with OutHandler {
|
||||
|
|
@ -276,7 +280,7 @@ final case class Recover[T](pf: PartialFunction[Throwable, T]) extends SimpleLin
|
|||
* it as an error in the process. So in that sense it is NOT exactly equivalent to `recover(t => throw t2)` since recover
|
||||
* would log the `t2` error.
|
||||
*/
|
||||
final case class MapError[T](f: PartialFunction[Throwable, Throwable]) extends SimpleLinearGraphStage[T] {
|
||||
@InternalApi private[akka] final case class MapError[T](f: PartialFunction[Throwable, Throwable]) extends SimpleLinearGraphStage[T] {
|
||||
override def createLogic(attr: Attributes) =
|
||||
new GraphStageLogic(shape) with InHandler with OutHandler {
|
||||
override def onPush(): Unit = push(out, grab(in))
|
||||
|
|
@ -294,7 +298,7 @@ final case class MapError[T](f: PartialFunction[Throwable, Throwable]) extends S
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
final case class Take[T](count: Long) extends SimpleLinearGraphStage[T] {
|
||||
@InternalApi private[akka] final case class Take[T](count: Long) extends SimpleLinearGraphStage[T] {
|
||||
override def initialAttributes: Attributes = DefaultAttributes.take
|
||||
|
||||
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new GraphStageLogic(shape) with InHandler with OutHandler {
|
||||
|
|
@ -322,7 +326,7 @@ final case class Take[T](count: Long) extends SimpleLinearGraphStage[T] {
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
final case class Drop[T](count: Long) extends SimpleLinearGraphStage[T] {
|
||||
@InternalApi private[akka] final case class Drop[T](count: Long) extends SimpleLinearGraphStage[T] {
|
||||
override def initialAttributes: Attributes = DefaultAttributes.drop
|
||||
|
||||
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new GraphStageLogic(shape) with InHandler with OutHandler {
|
||||
|
|
@ -346,7 +350,7 @@ final case class Drop[T](count: Long) extends SimpleLinearGraphStage[T] {
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
final case class Scan[In, Out](zero: Out, f: (Out, In) ⇒ Out) extends GraphStage[FlowShape[In, Out]] {
|
||||
@InternalApi private[akka] final case class Scan[In, Out](zero: Out, f: (Out, In) ⇒ Out) extends GraphStage[FlowShape[In, Out]] {
|
||||
override val shape = FlowShape[In, Out](Inlet("Scan.in"), Outlet("Scan.out"))
|
||||
|
||||
override def initialAttributes: Attributes = DefaultAttributes.scan
|
||||
|
|
@ -404,7 +408,7 @@ final case class Scan[In, Out](zero: Out, f: (Out, In) ⇒ Out) extends GraphSta
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
final case class ScanAsync[In, Out](zero: Out, f: (Out, In) ⇒ Future[Out]) extends GraphStage[FlowShape[In, Out]] {
|
||||
@InternalApi private[akka] final case class ScanAsync[In, Out](zero: Out, f: (Out, In) ⇒ Future[Out]) extends GraphStage[FlowShape[In, Out]] {
|
||||
|
||||
import akka.dispatch.ExecutionContexts
|
||||
|
||||
|
|
@ -512,7 +516,7 @@ final case class ScanAsync[In, Out](zero: Out, f: (Out, In) ⇒ Future[Out]) ext
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
final case class Fold[In, Out](zero: Out, f: (Out, In) ⇒ Out) extends GraphStage[FlowShape[In, Out]] {
|
||||
@InternalApi private[akka] final case class Fold[In, Out](zero: Out, f: (Out, In) ⇒ Out) extends GraphStage[FlowShape[In, Out]] {
|
||||
|
||||
val in = Inlet[In]("Fold.in")
|
||||
val out = Outlet[Out]("Fold.out")
|
||||
|
|
@ -567,7 +571,7 @@ final case class Fold[In, Out](zero: Out, f: (Out, In) ⇒ Out) extends GraphSta
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
final class FoldAsync[In, Out](zero: Out, f: (Out, In) ⇒ Future[Out]) extends GraphStage[FlowShape[In, Out]] {
|
||||
@InternalApi private[akka] final class FoldAsync[In, Out](zero: Out, f: (Out, In) ⇒ Future[Out]) extends GraphStage[FlowShape[In, Out]] {
|
||||
|
||||
import akka.dispatch.ExecutionContexts
|
||||
|
||||
|
|
@ -662,7 +666,7 @@ final class FoldAsync[In, Out](zero: Out, f: (Out, In) ⇒ Future[Out]) extends
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
final case class Intersperse[T](start: Option[T], inject: T, end: Option[T]) extends SimpleLinearGraphStage[T] {
|
||||
@InternalApi private[akka] final case class Intersperse[T](start: Option[T], inject: T, end: Option[T]) extends SimpleLinearGraphStage[T] {
|
||||
ReactiveStreamsCompliance.requireNonNullElement(inject)
|
||||
if (start.isDefined) ReactiveStreamsCompliance.requireNonNullElement(start.get)
|
||||
if (end.isDefined) ReactiveStreamsCompliance.requireNonNullElement(end.get)
|
||||
|
|
@ -701,7 +705,7 @@ final case class Intersperse[T](start: Option[T], inject: T, end: Option[T]) ext
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
final case class Grouped[T](n: Int) extends GraphStage[FlowShape[T, immutable.Seq[T]]] {
|
||||
@InternalApi private[akka] final case class Grouped[T](n: Int) extends GraphStage[FlowShape[T, immutable.Seq[T]]] {
|
||||
require(n > 0, "n must be greater than 0")
|
||||
|
||||
val in = Inlet[T]("Grouped.in")
|
||||
|
|
@ -755,7 +759,7 @@ final case class Grouped[T](n: Int) extends GraphStage[FlowShape[T, immutable.Se
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
final case class LimitWeighted[T](val n: Long, val costFn: T ⇒ Long) extends SimpleLinearGraphStage[T] {
|
||||
@InternalApi private[akka] final case class LimitWeighted[T](val n: Long, val costFn: T ⇒ Long) extends SimpleLinearGraphStage[T] {
|
||||
override def initialAttributes: Attributes = DefaultAttributes.limitWeighted
|
||||
|
||||
def createLogic(inheritedAttributes: Attributes) = new SupervisedGraphStageLogic(inheritedAttributes, shape) with InHandler with OutHandler {
|
||||
|
|
@ -789,7 +793,7 @@ final case class LimitWeighted[T](val n: Long, val costFn: T ⇒ Long) extends S
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
final case class Sliding[T](val n: Int, val step: Int) extends GraphStage[FlowShape[T, immutable.Seq[T]]] {
|
||||
@InternalApi private[akka] final case class Sliding[T](val n: Int, val step: Int) extends GraphStage[FlowShape[T, immutable.Seq[T]]] {
|
||||
require(n > 0, "n must be greater than 0")
|
||||
require(step > 0, "step must be greater than 0")
|
||||
|
||||
|
|
@ -847,7 +851,7 @@ final case class Sliding[T](val n: Int, val step: Int) extends GraphStage[FlowSh
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
final case class Buffer[T](size: Int, overflowStrategy: OverflowStrategy) extends SimpleLinearGraphStage[T] {
|
||||
@InternalApi private[akka] final case class Buffer[T](size: Int, overflowStrategy: OverflowStrategy) extends SimpleLinearGraphStage[T] {
|
||||
|
||||
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new GraphStageLogic(shape) with InHandler with OutHandler {
|
||||
|
||||
|
|
@ -920,7 +924,7 @@ final case class Buffer[T](size: Int, overflowStrategy: OverflowStrategy) extend
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
final case class Batch[In, Out](val max: Long, val costFn: In ⇒ Long, val seed: In ⇒ Out, val aggregate: (Out, In) ⇒ Out)
|
||||
@InternalApi private[akka] final case class Batch[In, Out](val max: Long, val costFn: In ⇒ Long, val seed: In ⇒ Out, val aggregate: (Out, In) ⇒ Out)
|
||||
extends GraphStage[FlowShape[In, Out]] {
|
||||
|
||||
val in = Inlet[In]("Batch.in")
|
||||
|
|
@ -1042,7 +1046,7 @@ final case class Batch[In, Out](val max: Long, val costFn: In ⇒ Long, val seed
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
final class Expand[In, Out](val extrapolate: In ⇒ Iterator[Out]) extends GraphStage[FlowShape[In, Out]] {
|
||||
@InternalApi private[akka] final class Expand[In, Out](val extrapolate: In ⇒ Iterator[Out]) extends GraphStage[FlowShape[In, Out]] {
|
||||
private val in = Inlet[In]("expand.in")
|
||||
private val out = Outlet[Out]("expand.out")
|
||||
|
||||
|
|
@ -1096,7 +1100,7 @@ final class Expand[In, Out](val extrapolate: In ⇒ Iterator[Out]) extends Graph
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] object MapAsync {
|
||||
@InternalApi private[akka] object MapAsync {
|
||||
|
||||
final class Holder[T](var elem: Try[T], val cb: AsyncCallback[Holder[T]]) extends (Try[T] ⇒ Unit) {
|
||||
def setElem(t: Try[T]): Unit =
|
||||
|
|
@ -1117,7 +1121,7 @@ private[akka] object MapAsync {
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
final case class MapAsync[In, Out](parallelism: Int, f: In ⇒ Future[Out])
|
||||
@InternalApi private[akka] final case class MapAsync[In, Out](parallelism: Int, f: In ⇒ Future[Out])
|
||||
extends GraphStage[FlowShape[In, Out]] {
|
||||
|
||||
import MapAsync._
|
||||
|
|
@ -1195,7 +1199,7 @@ final case class MapAsync[In, Out](parallelism: Int, f: In ⇒ Future[Out])
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
final case class MapAsyncUnordered[In, Out](parallelism: Int, f: In ⇒ Future[Out])
|
||||
@InternalApi private[akka] final case class MapAsyncUnordered[In, Out](parallelism: Int, f: In ⇒ Future[Out])
|
||||
extends GraphStage[FlowShape[In, Out]] {
|
||||
|
||||
private val in = Inlet[In]("MapAsyncUnordered.in")
|
||||
|
|
@ -1273,7 +1277,7 @@ final case class MapAsyncUnordered[In, Out](parallelism: Int, f: In ⇒ Future[O
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
final case class Log[T](
|
||||
@InternalApi private[akka] final case class Log[T](
|
||||
name: String,
|
||||
extract: T ⇒ Any,
|
||||
logAdapter: Option[LoggingAdapter]) extends SimpleLinearGraphStage[T] {
|
||||
|
|
@ -1357,7 +1361,7 @@ final case class Log[T](
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] object Log {
|
||||
@InternalApi private[akka] object Log {
|
||||
|
||||
/**
|
||||
* Must be located here to be visible for implicit resolution, when [[Materializer]] is passed to [[Logging]]
|
||||
|
|
@ -1385,7 +1389,7 @@ private[akka] object Log {
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[stream] object TimerKeys {
|
||||
@InternalApi private[stream] object TimerKeys {
|
||||
|
||||
case object TakeWithinTimerKey
|
||||
|
||||
|
|
@ -1395,7 +1399,10 @@ private[stream] object TimerKeys {
|
|||
|
||||
}
|
||||
|
||||
final class GroupedWithin[T](val n: Int, val d: FiniteDuration) extends GraphStage[FlowShape[T, immutable.Seq[T]]] {
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi private[akka] final class GroupedWithin[T](val n: Int, val d: FiniteDuration) extends GraphStage[FlowShape[T, immutable.Seq[T]]] {
|
||||
require(n > 0, "n must be greater than 0")
|
||||
require(d > Duration.Zero)
|
||||
|
||||
|
|
@ -1473,7 +1480,10 @@ final class GroupedWithin[T](val n: Int, val d: FiniteDuration) extends GraphSta
|
|||
}
|
||||
}
|
||||
|
||||
final class Delay[T](val d: FiniteDuration, val strategy: DelayOverflowStrategy) extends SimpleLinearGraphStage[T] {
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi private[akka] final class Delay[T](val d: FiniteDuration, val strategy: DelayOverflowStrategy) extends SimpleLinearGraphStage[T] {
|
||||
private[this] def timerName = "DelayedTimer"
|
||||
|
||||
override def initialAttributes: Attributes = DefaultAttributes.delay
|
||||
|
|
@ -1587,7 +1597,10 @@ final class Delay[T](val d: FiniteDuration, val strategy: DelayOverflowStrategy)
|
|||
override def toString = "Delay"
|
||||
}
|
||||
|
||||
final class TakeWithin[T](val timeout: FiniteDuration) extends SimpleLinearGraphStage[T] {
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi private[akka] final class TakeWithin[T](val timeout: FiniteDuration) extends SimpleLinearGraphStage[T] {
|
||||
|
||||
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new TimerGraphStageLogic(shape) with InHandler with OutHandler {
|
||||
def onPush(): Unit = push(out, grab(in))
|
||||
|
|
@ -1604,7 +1617,10 @@ final class TakeWithin[T](val timeout: FiniteDuration) extends SimpleLinearGraph
|
|||
override def toString = "TakeWithin"
|
||||
}
|
||||
|
||||
final class DropWithin[T](val timeout: FiniteDuration) extends SimpleLinearGraphStage[T] {
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi private[akka] final class DropWithin[T](val timeout: FiniteDuration) extends SimpleLinearGraphStage[T] {
|
||||
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new GraphStageLogic(shape) with InHandler with OutHandler {
|
||||
|
||||
private val startNanoTime = System.nanoTime()
|
||||
|
|
@ -1634,7 +1650,7 @@ final class DropWithin[T](val timeout: FiniteDuration) extends SimpleLinearGraph
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
final class Reduce[T](val f: (T, T) ⇒ T) extends SimpleLinearGraphStage[T] {
|
||||
@InternalApi private[akka] final class Reduce[T](val f: (T, T) ⇒ T) extends SimpleLinearGraphStage[T] {
|
||||
override def initialAttributes: Attributes = DefaultAttributes.reduce
|
||||
|
||||
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new GraphStageLogic(shape) with InHandler with OutHandler {
|
||||
|
|
@ -1695,11 +1711,11 @@ final class Reduce[T](val f: (T, T) ⇒ T) extends SimpleLinearGraphStage[T] {
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[stream] object RecoverWith {
|
||||
@InternalApi private[stream] object RecoverWith {
|
||||
val InfiniteRetries = -1
|
||||
}
|
||||
|
||||
final class RecoverWith[T, M](val maximumRetries: Int, val pf: PartialFunction[Throwable, Graph[SourceShape[T], M]]) extends SimpleLinearGraphStage[T] {
|
||||
@InternalApi private[akka] final class RecoverWith[T, M](val maximumRetries: Int, val pf: PartialFunction[Throwable, Graph[SourceShape[T], M]]) extends SimpleLinearGraphStage[T] {
|
||||
require(maximumRetries >= -1, "number of retries must be non-negative or equal to -1")
|
||||
|
||||
override def initialAttributes = DefaultAttributes.recoverWith
|
||||
|
|
@ -1753,7 +1769,7 @@ final class RecoverWith[T, M](val maximumRetries: Int, val pf: PartialFunction[T
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
final class StatefulMapConcat[In, Out](val f: () ⇒ In ⇒ immutable.Iterable[Out]) extends GraphStage[FlowShape[In, Out]] {
|
||||
@InternalApi private[akka] final class StatefulMapConcat[In, Out](val f: () ⇒ In ⇒ immutable.Iterable[Out]) extends GraphStage[FlowShape[In, Out]] {
|
||||
val in = Inlet[In]("StatefulMapConcat.in")
|
||||
val out = Outlet[Out]("StatefulMapConcat.out")
|
||||
override val shape = FlowShape(in, out)
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue