!str #16902: Unify stream internal representation
also =str #16912: Fix StreamTcpSpec flakiness
This commit is contained in:
parent
cac9c9f2fb
commit
8d77fa8b29
230 changed files with 7814 additions and 9596 deletions
|
|
@ -3,39 +3,100 @@
|
|||
*/
|
||||
package akka.stream.scaladsl
|
||||
|
||||
import scala.language.higherKinds
|
||||
import akka.stream.impl.Stages.{ MaterializingStageFactory, StageModule }
|
||||
import akka.stream.{ SourceShape, Inlet, Outlet }
|
||||
import akka.stream.impl.StreamLayout.{ EmptyModule, Module }
|
||||
import akka.stream.stage.{ TerminationDirective, Directive, Context, PushPullStage }
|
||||
|
||||
import scala.annotation.unchecked.uncheckedVariance
|
||||
import scala.language.higherKinds
|
||||
import akka.actor.Props
|
||||
import akka.stream.impl.{ EmptyPublisher, ErrorPublisher, SynchronousIterablePublisher }
|
||||
import org.reactivestreams.Publisher
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
import scala.concurrent.{ ExecutionContext, Future }
|
||||
import akka.stream.FlowMaterializer
|
||||
import akka.stream.{ ActorFlowMaterializer, Graph }
|
||||
import akka.stream.impl._
|
||||
import akka.actor.Cancellable
|
||||
import akka.actor.ActorRef
|
||||
import scala.concurrent.Promise
|
||||
import org.reactivestreams.Subscriber
|
||||
|
||||
/**
|
||||
* A `Source` is a set of stream processing steps that has one open output and an attached input.
|
||||
* Can be used as a `Publisher`
|
||||
* A `Source` is a set of stream processing steps that has one open output. It can comprise
|
||||
* any number of internal sources and transformations that are wired together, or it can be
|
||||
* an “atomic” source, e.g. from a collection or a file. Materialization turns a Source into
|
||||
* a Reactive Streams `Publisher` (at least conceptually).
|
||||
*/
|
||||
trait Source[+Out] extends FlowOps[Out] with Materializable {
|
||||
override type Repr[+O] <: Source[O]
|
||||
final class Source[+Out, +Mat](private[stream] override val module: Module)
|
||||
extends FlowOps[Out, Mat] with Graph[SourceShape[Out], Mat] {
|
||||
|
||||
override type Repr[+O, +M] = Source[O, M]
|
||||
|
||||
override val shape: SourceShape[Out] = module.shape.asInstanceOf[SourceShape[Out]]
|
||||
|
||||
/**
|
||||
* Transform this [[akka.stream.scaladsl.Source]] by appending the given processing stages.
|
||||
*/
|
||||
def via[T](flow: Flow[Out, T]): Source[T]
|
||||
def via[T, Mat2](flow: Flow[Out, T, Mat2]): Source[T, Mat] = viaMat(flow)(Keep.left)
|
||||
|
||||
/**
|
||||
* Transform this [[akka.stream.scaladsl.Source]] by appending the given processing stages.
|
||||
*/
|
||||
def viaMat[T, Mat2, Mat3](flow: Flow[Out, T, Mat2])(combine: (Mat, Mat2) ⇒ Mat3): Source[T, Mat3] = {
|
||||
if (flow.isIdentity) this.asInstanceOf[Source[T, Mat3]]
|
||||
else {
|
||||
val flowCopy = flow.module.carbonCopy
|
||||
new Source(
|
||||
module
|
||||
.growConnect(flowCopy, shape.outlet, flowCopy.shape.inlets.head, combine)
|
||||
.replaceShape(SourceShape(flowCopy.shape.outlets.head)))
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Connect this [[akka.stream.scaladsl.Source]] to a [[akka.stream.scaladsl.Sink]],
|
||||
* concatenating the processing steps of both.
|
||||
*/
|
||||
def to(sink: Sink[Out]): RunnableFlow
|
||||
def to[Mat2](sink: Sink[Out, Mat2]): RunnableFlow[Mat] = toMat(sink)(Keep.left)
|
||||
|
||||
/**
|
||||
* Connect this [[akka.stream.scaladsl.Source]] to a [[akka.stream.scaladsl.Sink]],
|
||||
* concatenating the processing steps of both.
|
||||
*/
|
||||
def toMat[Mat2, Mat3](sink: Sink[Out, Mat2])(combine: (Mat, Mat2) ⇒ Mat3): RunnableFlow[Mat3] = {
|
||||
val sinkCopy = sink.module.carbonCopy
|
||||
RunnableFlow(module.growConnect(sinkCopy, shape.outlet, sinkCopy.shape.inlets.head, combine))
|
||||
}
|
||||
|
||||
/**
|
||||
* Transform only the materialized value of this Source, leaving all other properties as they were.
|
||||
*/
|
||||
def mapMaterialized[Mat2](f: Mat ⇒ Mat2): Repr[Out, Mat2] =
|
||||
new Source(module.transformMaterializedValue(f.asInstanceOf[Any ⇒ Any]))
|
||||
|
||||
/** INTERNAL API */
|
||||
override private[scaladsl] def andThen[U](op: StageModule): Repr[U, Mat] = {
|
||||
// No need to copy here, op is a fresh instance
|
||||
new Source(
|
||||
module
|
||||
.growConnect(op, shape.outlet, op.inPort)
|
||||
.replaceShape(SourceShape(op.outPort)))
|
||||
}
|
||||
|
||||
override private[scaladsl] def andThenMat[U, Mat2](op: MaterializingStageFactory): Repr[U, Mat2] = {
|
||||
new Source(
|
||||
module
|
||||
.growConnect(op, shape.outlet, op.inPort, Keep.right)
|
||||
.replaceShape(SourceShape(op.outPort)))
|
||||
}
|
||||
|
||||
/**
|
||||
* Connect this `Source` to a `Sink` and run it. The returned value is the materialized value
|
||||
* of the `Sink`, e.g. the `Publisher` of a [[akka.stream.scaladsl.Sink#publisher]].
|
||||
*/
|
||||
def runWith(sink: Sink[Out])(implicit materializer: FlowMaterializer): sink.MaterializedType = to(sink).run().get(sink)
|
||||
def runWith[Mat2](sink: Sink[Out, Mat2])(implicit materializer: ActorFlowMaterializer): Mat2 = toMat(sink)(Keep.right).run()
|
||||
|
||||
/**
|
||||
* Shortcut for running this `Source` with a fold function.
|
||||
|
|
@ -45,7 +106,8 @@ trait Source[+Out] extends FlowOps[Out] with Materializable {
|
|||
* function evaluation when the input stream ends, or completed with `Failure`
|
||||
* if there is a failure signaled in the stream.
|
||||
*/
|
||||
def runFold[U](zero: U)(f: (U, Out) ⇒ U)(implicit materializer: FlowMaterializer): Future[U] = runWith(FoldSink(zero)(f)) // FIXME why is fold always an end step?
|
||||
def runFold[U](zero: U)(f: (U, Out) ⇒ U)(implicit materializer: ActorFlowMaterializer): Future[U] =
|
||||
runWith(Sink.fold(zero)(f))
|
||||
|
||||
/**
|
||||
* Shortcut for running this `Source` with a foreach procedure. The given procedure is invoked
|
||||
|
|
@ -54,14 +116,14 @@ trait Source[+Out] extends FlowOps[Out] with Materializable {
|
|||
* normal end of the stream, or completed with `Failure` if there is a failure signaled in
|
||||
* the stream.
|
||||
*/
|
||||
def runForeach(f: Out ⇒ Unit)(implicit materializer: FlowMaterializer): Future[Unit] = runWith(ForeachSink(f))
|
||||
def runForeach(f: Out ⇒ Unit)(implicit materializer: ActorFlowMaterializer): Future[Unit] = runWith(Sink.foreach(f))
|
||||
|
||||
/**
|
||||
* Concatenates a second source so that the first element
|
||||
* emitted by that source is emitted after the last element of this
|
||||
* source.
|
||||
*/
|
||||
def concat[Out2 >: Out](second: Source[Out2]): Source[Out2] = Source.concat(this, second)
|
||||
def concat[Out2 >: Out, M](second: Source[Out2, M]): Source[Out2, (Mat, M)] = Source.concat(this, second)
|
||||
|
||||
/**
|
||||
* Concatenates a second source so that the first element
|
||||
|
|
@ -70,24 +132,37 @@ trait Source[+Out] extends FlowOps[Out] with Materializable {
|
|||
*
|
||||
* This is a shorthand for [[concat]]
|
||||
*/
|
||||
def ++[Out2 >: Out](second: Source[Out2]): Source[Out2] = concat(second)
|
||||
|
||||
/**
|
||||
* Add a key that will have a value available after materialization.
|
||||
* The key can only use other keys if they have been added to the source
|
||||
* before this key. This also includes the keyed source if applicable.
|
||||
*/
|
||||
def withKey(key: Key[_]): Source[Out]
|
||||
def ++[Out2 >: Out, M](second: Source[Out2, M]): Source[Out2, (Mat, M)] = concat(second)
|
||||
|
||||
/**
|
||||
* Applies given [[OperationAttributes]] to a given section.
|
||||
*/
|
||||
def section[T](attributes: OperationAttributes)(section: Source[Out] ⇒ Source[T]): Source[T] =
|
||||
section(this.withAttributes(attributes)).withAttributes(OperationAttributes.none)
|
||||
def section[O, O2 >: Out, Mat2, Mat3](attributes: OperationAttributes, combine: (Mat, Mat2) ⇒ Mat3)(section: Flow[O2, O2, Unit] ⇒ Flow[O2, O, Mat2]): Source[O, Mat3] = {
|
||||
val subFlow = section(Flow[O2]).module.carbonCopy.withAttributes(attributes).wrap()
|
||||
new Source(
|
||||
module
|
||||
.growConnect(subFlow, shape.outlet, subFlow.shape.inlets.head, combine)
|
||||
.replaceShape(SourceShape(subFlow.shape.outlets.head)))
|
||||
}
|
||||
|
||||
def section[O, O2 >: Out, Mat2](attributes: OperationAttributes)(section: Flow[O2, O2, Unit] ⇒ Flow[O2, O, Mat2]): Source[O, Mat2] = {
|
||||
this.section[O, O2, Mat2, Mat2](attributes, (parentm: Mat, subm: Mat2) ⇒ subm)(section)
|
||||
}
|
||||
|
||||
override def withAttributes(attr: OperationAttributes): Repr[Out, Mat] =
|
||||
new Source(module.withAttributes(attr).wrap())
|
||||
|
||||
}
|
||||
|
||||
object Source {
|
||||
object Source extends SourceApply {
|
||||
|
||||
import OperationAttributes.{ none, name ⇒ named }
|
||||
|
||||
private[stream] def apply[Out, Mat](module: SourceModule[Out, Mat]): Source[Out, Mat] =
|
||||
new Source(module)
|
||||
|
||||
private def shape[T](name: String): SourceShape[T] = SourceShape(new Outlet(name + ".out"))
|
||||
|
||||
/**
|
||||
* Helper to create [[Source]] from `Publisher`.
|
||||
*
|
||||
|
|
@ -96,7 +171,19 @@ object Source {
|
|||
* that mediate the flow of elements downstream and the propagation of
|
||||
* back-pressure upstream.
|
||||
*/
|
||||
def apply[T](publisher: Publisher[T]): Source[T] = PublisherSource(publisher)
|
||||
def apply[T](publisher: Publisher[T]): Source[T, Unit] =
|
||||
new Source(new PublisherSource(publisher, none, shape("PublisherSource")))
|
||||
|
||||
/**
|
||||
* Helper to create [[Source]] from `Publisher`.
|
||||
*
|
||||
* Construct a transformation starting with given publisher. The transformation steps
|
||||
* are executed by a series of [[org.reactivestreams.Processor]] instances
|
||||
* that mediate the flow of elements downstream and the propagation of
|
||||
* back-pressure upstream.
|
||||
*/
|
||||
def apply[T](publisher: Publisher[T], name: String): Source[T, Unit] =
|
||||
new Source(new PublisherSource(publisher, named(name), shape(name)))
|
||||
|
||||
/**
|
||||
* Helper to create [[Source]] from `Iterator`.
|
||||
|
|
@ -108,7 +195,33 @@ object Source {
|
|||
* Elements are pulled out of the iterator in accordance with the demand coming
|
||||
* from the downstream transformation steps.
|
||||
*/
|
||||
def apply[T](f: () ⇒ Iterator[T]): Source[T] = apply(new FuncIterable(f))
|
||||
def apply[T](f: () ⇒ Iterator[T]): Source[T, Unit] = {
|
||||
apply(new immutable.Iterable[T] {
|
||||
override def iterator: Iterator[T] = f()
|
||||
})
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper to create [[Source]] from `Iterator`.
|
||||
* Example usage: `Source(() => Iterator.from(0))`
|
||||
*
|
||||
* Start a new `Source` from the given function that produces anIterator.
|
||||
* The produced stream of elements will continue until the iterator runs empty
|
||||
* or fails during evaluation of the `next()` method.
|
||||
* Elements are pulled out of the iterator in accordance with the demand coming
|
||||
* from the downstream transformation steps.
|
||||
*/
|
||||
def apply[T](f: () ⇒ Iterator[T], name: String): Source[T, Unit] = {
|
||||
apply(new immutable.Iterable[T] {
|
||||
override def iterator: Iterator[T] = f()
|
||||
})
|
||||
}
|
||||
|
||||
/**
|
||||
* A graph with the shape of a source logically is a source, this method makes
|
||||
* it so also in type.
|
||||
*/
|
||||
def wrap[T, M](g: Graph[SourceShape[T], M]): Source[T, M] = new Source(g.module)
|
||||
|
||||
/**
|
||||
* Helper to create [[Source]] from `Iterable`.
|
||||
|
|
@ -119,7 +232,48 @@ object Source {
|
|||
* stream will see an individual flow of elements (always starting from the
|
||||
* beginning) regardless of when they subscribed.
|
||||
*/
|
||||
def apply[T](iterable: immutable.Iterable[T]): Source[T] = IterableSource(iterable)
|
||||
def apply[T](iterable: immutable.Iterable[T]): Source[T, Unit] = { // FIXME add naming of outlet
|
||||
|
||||
Source.empty.transform(() ⇒ {
|
||||
new PushPullStage[Nothing, T] {
|
||||
var iterator: Iterator[T] = null
|
||||
|
||||
// Delayed init so we onError instead of failing during construction of the Source
|
||||
def initIterator(): Unit = if (iterator eq null) iterator = iterable.iterator
|
||||
|
||||
// Upstream is guaranteed to be empty
|
||||
override def onPush(elem: Nothing, ctx: Context[T]): Directive =
|
||||
throw new UnsupportedOperationException("The IterableSource stage cannot be pushed")
|
||||
|
||||
override def onUpstreamFinish(ctx: Context[T]): TerminationDirective = {
|
||||
initIterator()
|
||||
if (iterator.hasNext) ctx.absorbTermination()
|
||||
else ctx.finish()
|
||||
}
|
||||
|
||||
override def onPull(ctx: Context[T]): Directive = {
|
||||
if (!ctx.isFinishing) {
|
||||
initIterator()
|
||||
ctx.pull()
|
||||
} else {
|
||||
val elem = iterator.next()
|
||||
if (iterator.hasNext) ctx.push(elem)
|
||||
else ctx.pushAndFinish(elem)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}).withAttributes(OperationAttributes.name("iterable"))
|
||||
}
|
||||
|
||||
/**
|
||||
* Start a new `Source` from the given `Future`. The stream will consist of
|
||||
* one element when the `Future` is completed with a successful value, which
|
||||
* may happen before or after materializing the `Flow`.
|
||||
* The stream terminates with an error if the `Future` is completed with a failure.
|
||||
*/
|
||||
def apply[T](future: Future[T]): Source[T, Unit] =
|
||||
new Source(new FutureSource(future, none, shape("FutureSource")))
|
||||
|
||||
/**
|
||||
* Start a new `Source` from the given `Future`. The stream will consist of
|
||||
|
|
@ -127,7 +281,8 @@ object Source {
|
|||
* may happen before or after materializing the `Flow`.
|
||||
* The stream terminates with a failure if the `Future` is completed with a failure.
|
||||
*/
|
||||
def apply[T](future: Future[T]): Source[T] = FutureSource(future)
|
||||
def apply[T](future: Future[T], name: String): Source[T, Unit] =
|
||||
new Source(new FutureSource(future, named(name), shape(name)))
|
||||
|
||||
/**
|
||||
* Elements are emitted periodically with the specified interval.
|
||||
|
|
@ -136,46 +291,44 @@ object Source {
|
|||
* element is produced it will not receive that tick element later. It will
|
||||
* receive new tick elements as soon as it has requested more elements.
|
||||
*/
|
||||
def apply[T](initialDelay: FiniteDuration, interval: FiniteDuration, tick: T): TickSource[T] =
|
||||
TickSource(initialDelay, interval, tick)
|
||||
def apply[T](initialDelay: FiniteDuration, interval: FiniteDuration, tick: T): Source[T, Cancellable] =
|
||||
new Source(new TickSource(initialDelay, interval, tick, none, shape("TickSource")))
|
||||
|
||||
/**
|
||||
* Creates a `Source` by using an empty [[FlowGraphBuilder]] on a block that expects a [[FlowGraphBuilder]] and
|
||||
* returns the `UndefinedSink`.
|
||||
* Elements are emitted periodically with the specified interval.
|
||||
* The tick element will be delivered to downstream consumers that has requested any elements.
|
||||
* If a consumer has not requested any elements at the point in time when the tick
|
||||
* element is produced it will not receive that tick element later. It will
|
||||
* receive new tick elements as soon as it has requested more elements.
|
||||
*/
|
||||
def apply[T]()(block: FlowGraphBuilder ⇒ UndefinedSink[T]): Source[T] =
|
||||
createSourceFromBuilder(new FlowGraphBuilder(), block)
|
||||
|
||||
/**
|
||||
* Creates a `Source` by using a [[FlowGraphBuilder]] from this [[PartialFlowGraph]] on a block that expects
|
||||
* a [[FlowGraphBuilder]] and returns the `UndefinedSink`.
|
||||
*/
|
||||
def apply[T](graph: PartialFlowGraph)(block: FlowGraphBuilder ⇒ UndefinedSink[T]): Source[T] =
|
||||
createSourceFromBuilder(new FlowGraphBuilder(graph), block)
|
||||
|
||||
private def createSourceFromBuilder[T](builder: FlowGraphBuilder, block: FlowGraphBuilder ⇒ UndefinedSink[T]): Source[T] = {
|
||||
val out = block(builder)
|
||||
builder.partialBuild().toSource(out)
|
||||
}
|
||||
def apply[T](initialDelay: FiniteDuration, interval: FiniteDuration, tick: T, name: String): Source[T, Cancellable] =
|
||||
new Source(new TickSource(initialDelay, interval, tick, named(name), shape(name)))
|
||||
|
||||
/**
|
||||
* Creates a `Source` that is materialized to an [[akka.actor.ActorRef]] which points to an Actor
|
||||
* created according to the passed in [[akka.actor.Props]]. Actor created by the `props` should
|
||||
* be [[akka.stream.actor.ActorPublisher]].
|
||||
*/
|
||||
def apply[T](props: Props): PropsSource[T] = PropsSource(props)
|
||||
def apply[T](props: Props): Source[T, ActorRef] = new Source(new PropsSource(props, none, shape("PropsSource")))
|
||||
|
||||
/**
|
||||
* Creates a `Source` that is materialized to an [[akka.actor.ActorRef]] which points to an Actor
|
||||
* created according to the passed in [[akka.actor.Props]]. Actor created by the `props` should
|
||||
* be [[akka.stream.actor.ActorPublisher]].
|
||||
*/
|
||||
def apply[T](props: Props, name: String): Source[T, ActorRef] = new Source(new PropsSource(props, named(name), shape(name)))
|
||||
|
||||
/**
|
||||
* Create a `Source` with one element.
|
||||
* Every connected `Sink` of this stream will see an individual stream consisting of one element.
|
||||
*/
|
||||
def single[T](element: T): Source[T] = apply(SynchronousIterablePublisher(List(element), "single")) // FIXME optimize
|
||||
def single[T](element: T): Source[T, Unit] = apply(SynchronousIterablePublisher(List(element), "single")) // FIXME optimize
|
||||
|
||||
/**
|
||||
* A `Source` with no elements, i.e. an empty stream that is completed immediately for every connected `Sink`.
|
||||
*/
|
||||
def empty[T](): Source[T] = _empty
|
||||
private[this] val _empty: Source[Nothing] = apply(EmptyPublisher)
|
||||
def empty[T](): Source[T, Unit] = _empty
|
||||
private[this] val _empty: Source[Nothing, Unit] = apply(EmptyPublisher, "EmptySource")
|
||||
|
||||
/**
|
||||
* Create a `Source` with no elements, which does not complete its downstream,
|
||||
|
|
@ -186,38 +339,52 @@ object Source {
|
|||
* be used to externally trigger completion, which the source then signalls
|
||||
* to its downstream.
|
||||
*/
|
||||
def lazyEmpty[T]() = LazyEmptySource[T]()
|
||||
def lazyEmpty[T](): Source[T, Promise[Unit]] = new Source(new LazyEmptySource[T](none, shape("LazyEmptySource")))
|
||||
|
||||
/**
|
||||
* Create a `Source` with no elements, which does not complete its downstream,
|
||||
* until externally triggered to do so.
|
||||
*
|
||||
* It materializes a [[scala.concurrent.Promise]] which will be completed
|
||||
* when the downstream stage of this source cancels. This promise can also
|
||||
* be used to externally trigger completion, which the source then signalls
|
||||
* to its downstream.
|
||||
*/
|
||||
def lazyEmpty[T](name: String): Source[T, Promise[Unit]] = new Source(new LazyEmptySource[T](named(name), shape(name)))
|
||||
|
||||
/**
|
||||
* Create a `Source` that immediately ends the stream with the `cause` error to every connected `Sink`.
|
||||
*/
|
||||
def failed[T](cause: Throwable): Source[T, Unit] = apply(ErrorPublisher(cause, "failed"), "FailedSource")
|
||||
|
||||
/**
|
||||
* Create a `Source` that immediately ends the stream with the `cause` failure to every connected `Sink`.
|
||||
*/
|
||||
def failed[T](cause: Throwable): Source[T] = apply(ErrorPublisher(cause, "failed"))
|
||||
def failed[T](cause: Throwable, name: String): Source[T, Unit] = apply(ErrorPublisher(cause, "failed"), name)
|
||||
|
||||
/**
|
||||
* Concatenates two sources so that the first element
|
||||
* emitted by the second source is emitted after the last element of the first
|
||||
* source.
|
||||
*/
|
||||
def concat[T](source1: Source[T], source2: Source[T]): Source[T] = {
|
||||
val output = UndefinedSink[T]
|
||||
val concat = Concat[T]
|
||||
Source() { b ⇒
|
||||
b.addEdge(source1, Pipe.empty[T], concat.first)
|
||||
.addEdge(source2, Pipe.empty[T], concat.second)
|
||||
.addEdge(concat.out, Pipe.empty[T], output)
|
||||
output
|
||||
}
|
||||
}
|
||||
def concat[T, Mat1, Mat2](source1: Source[T, Mat1], source2: Source[T, Mat2]): Source[T, (Mat1, Mat2)] =
|
||||
wrap(FlowGraph.partial(source1, source2)(Keep.both) { implicit b ⇒
|
||||
(s1, s2) ⇒
|
||||
import FlowGraph.Implicits._
|
||||
val c = b.add(Concat[T]())
|
||||
s1.outlet ~> c.in(0)
|
||||
s2.outlet ~> c.in(1)
|
||||
SourceShape(c.out)
|
||||
})
|
||||
|
||||
/**
|
||||
* Creates a `Source` that is materialized as a [[org.reactivestreams.Subscriber]]
|
||||
*/
|
||||
def subscriber[T]: SubscriberSource[T] = SubscriberSource[T]
|
||||
}
|
||||
def subscriber[T](): Source[T, Subscriber[T]] = new Source(new SubscriberSource[T](none, shape("SubscriberSource")))
|
||||
|
||||
/**
|
||||
* A `Source` that will create an object during materialization that the user will need
|
||||
* to retrieve in order to access aspects of this source (could be a Subscriber, a
|
||||
* Future/Promise, etc.).
|
||||
*/
|
||||
trait KeyedSource[+Out, M] extends Source[Out] with KeyedMaterializable[M]
|
||||
/**
|
||||
* Creates a `Source` that is materialized as a [[org.reactivestreams.Subscriber]]
|
||||
*/
|
||||
def subscriber[T](name: String): Source[T, Subscriber[T]] = new Source(new SubscriberSource[T](named(name), shape(name)))
|
||||
|
||||
}
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue