-str - Improvements and renames in internal streams classes
* Renames Module.grow -> Module.compose * Renames Module.connect -> Module.wire * Renames Module.growConnect -> Module.fuse * Renames Module.wrap -> Module.nest * Adds explicit identity equals and hashCode to InPort and OutPort * Reimplements many of the Source factories to avoid copying * Documents Module.compose, Module.fuse, Module.wire and Module.nest * Removes Attributes.nameLifted * Optimizes Attributes.nameOrDefault
This commit is contained in:
parent
efc659b70a
commit
66a116d3d2
19 changed files with 333 additions and 290 deletions
|
|
@ -3,33 +3,19 @@
|
|||
*/
|
||||
package akka.stream.scaladsl
|
||||
|
||||
import scala.language.higherKinds
|
||||
|
||||
import akka.actor.{ ActorRef, Cancellable, Props }
|
||||
import akka.stream._
|
||||
import akka.stream.impl.Stages.{ DirectProcessor, MaterializingStageFactory, StageModule, DefaultAttributes }
|
||||
import akka.stream.impl.{ EmptyPublisher, ErrorPublisher, _ }
|
||||
import akka.stream.stage.{ Context, PushPullStage, SyncDirective, TerminationDirective }
|
||||
import org.reactivestreams._
|
||||
|
||||
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 }
|
||||
import akka.stream.impl.Stages.{ MaterializingStageFactory, StageModule }
|
||||
import akka.stream.impl.Stages.DefaultAttributes
|
||||
import akka.stream.impl.StreamLayout.Module
|
||||
import akka.stream.impl._
|
||||
import org.reactivestreams.{ Publisher, Subscriber }
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
import scala.concurrent.{ Future, Promise }
|
||||
import scala.language.higherKinds
|
||||
import scala.concurrent.{ ExecutionContext, Future }
|
||||
import akka.stream.{ Materializer, Graph }
|
||||
import akka.stream.impl._
|
||||
import akka.actor.Cancellable
|
||||
import akka.actor.ActorRef
|
||||
import scala.concurrent.Promise
|
||||
import akka.stream.stage.SyncDirective
|
||||
import akka.stream.OverflowStrategy
|
||||
import akka.stream.Attributes
|
||||
|
||||
/**
|
||||
* A `Source` is a set of stream processing steps that has one open output. It can comprise
|
||||
|
|
@ -58,8 +44,8 @@ final class Source[+Out, +Mat](private[stream] override val module: Module)
|
|||
val flowCopy = flow.module.carbonCopy
|
||||
new Source(
|
||||
module
|
||||
.growConnect(flowCopy, shape.outlet, flowCopy.shape.inlets.head, combine)
|
||||
.replaceShape(SourceShape(flowCopy.shape.outlets.head)))
|
||||
.fuse(flowCopy, shape.outlet, flowCopy.shape.inlets.head, combine)
|
||||
.replaceShape(SourceShape(flowCopy.shape.outlets.head))) // FIXME why is not .wrap() needed here?
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -75,7 +61,7 @@ final class Source[+Out, +Mat](private[stream] override val module: Module)
|
|||
*/
|
||||
def toMat[Mat2, Mat3](sink: Graph[SinkShape[Out], Mat2])(combine: (Mat, Mat2) ⇒ Mat3): RunnableGraph[Mat3] = {
|
||||
val sinkCopy = sink.module.carbonCopy
|
||||
RunnableGraph(module.growConnect(sinkCopy, shape.outlet, sinkCopy.shape.inlets.head, combine))
|
||||
RunnableGraph(module.fuse(sinkCopy, shape.outlet, sinkCopy.shape.inlets.head, combine))
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -89,14 +75,14 @@ final class Source[+Out, +Mat](private[stream] override val module: Module)
|
|||
// No need to copy here, op is a fresh instance
|
||||
new Source(
|
||||
module
|
||||
.growConnect(op, shape.outlet, op.inPort)
|
||||
.fuse(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)
|
||||
.fuse(op, shape.outlet, op.inPort, Keep.right)
|
||||
.replaceShape(SourceShape(op.outPort)))
|
||||
}
|
||||
|
||||
|
|
@ -150,14 +136,18 @@ final class Source[+Out, +Mat](private[stream] override val module: Module)
|
|||
*/
|
||||
def ++[Out2 >: Out, M](second: Graph[SourceShape[Out2], M]): Source[Out2, (Mat, M)] = concat(second)
|
||||
|
||||
/**
|
||||
* Nests the current Source and returns a Source with the given Attributes
|
||||
* @param attr the attributes to add
|
||||
* @return a new Source with the added attributes
|
||||
*/
|
||||
override def withAttributes(attr: Attributes): Repr[Out, Mat] =
|
||||
new Source(module.withAttributes(attr).wrap())
|
||||
new Source(module.withAttributes(attr).nest()) // User API
|
||||
|
||||
override def named(name: String): Repr[Out, Mat] = withAttributes(Attributes.name(name))
|
||||
|
||||
/** Converts this Scala DSL element to it's Java DSL counterpart. */
|
||||
def asJava: javadsl.Source[Out, Mat] = new javadsl.Source(this)
|
||||
|
||||
}
|
||||
|
||||
object Source extends SourceApply {
|
||||
|
|
@ -193,7 +183,10 @@ object Source extends SourceApply {
|
|||
* from the downstream transformation steps.
|
||||
*/
|
||||
def apply[T](f: () ⇒ Iterator[T]): Source[T, Unit] =
|
||||
apply(new immutable.Iterable[T] { override def iterator: Iterator[T] = f() })
|
||||
apply(new immutable.Iterable[T] {
|
||||
override def iterator: Iterator[T] = f()
|
||||
override def toString: String = "() => Iterator"
|
||||
})
|
||||
|
||||
/**
|
||||
* A graph with the shape of a source logically is a source, this method makes
|
||||
|
|
@ -223,7 +216,11 @@ object Source extends SourceApply {
|
|||
* The stream terminates with a failure if the `Future` is completed with a failure.
|
||||
*/
|
||||
def apply[T](future: Future[T]): Source[T, Unit] =
|
||||
Source.single(future).mapAsyncUnordered(1)(id).withAttributes(DefaultAttributes.futureSource)
|
||||
new Source(
|
||||
new PublisherSource(
|
||||
SingleElementPublisher(future, "FutureSource"),
|
||||
DefaultAttributes.futureSource,
|
||||
shape("FutureSource"))).mapAsyncUnordered(1)(id)
|
||||
|
||||
/**
|
||||
* Elements are emitted periodically with the specified interval.
|
||||
|
|
@ -240,22 +237,36 @@ object Source extends SourceApply {
|
|||
* Every connected `Sink` of this stream will see an individual stream consisting of one element.
|
||||
*/
|
||||
def single[T](element: T): Source[T, Unit] =
|
||||
apply(SingleElementPublisher(element, "SingleSource")).withAttributes(DefaultAttributes.singleSource)
|
||||
new Source(
|
||||
new PublisherSource(
|
||||
SingleElementPublisher(element, "SingleSource"),
|
||||
DefaultAttributes.singleSource,
|
||||
shape("SingleSource")))
|
||||
|
||||
/**
|
||||
* Create a `Source` that will continually emit the given element.
|
||||
*/
|
||||
def repeat[T](element: T): Source[T, Unit] =
|
||||
apply(new immutable.Iterable[T] {
|
||||
override val iterator: Iterator[T] = Iterator.continually(element)
|
||||
override def toString: String = "Iterable.continually(" + element + ")"
|
||||
}).withAttributes(DefaultAttributes.repeat)
|
||||
new Source(
|
||||
new PublisherSource(
|
||||
SingleElementPublisher(
|
||||
new immutable.Iterable[T] {
|
||||
override val iterator: Iterator[T] = Iterator.continually(element)
|
||||
override def toString: String = "repeat(" + element + ")"
|
||||
}, "RepeatSource"),
|
||||
DefaultAttributes.repeat,
|
||||
shape("RepeatSource"))).mapConcat(id)
|
||||
|
||||
/**
|
||||
* A `Source` with no elements, i.e. an empty stream that is completed immediately for every connected `Sink`.
|
||||
*/
|
||||
def empty[T]: Source[T, Unit] = _empty
|
||||
private[this] val _empty: Source[Nothing, Unit] = apply(EmptyPublisher).withAttributes(DefaultAttributes.emptySource)
|
||||
private[this] val _empty: Source[Nothing, Unit] =
|
||||
new Source(
|
||||
new PublisherSource[Nothing](
|
||||
EmptyPublisher,
|
||||
DefaultAttributes.emptySource,
|
||||
shape("EmptySource")))
|
||||
|
||||
/**
|
||||
* Create a `Source` with no elements, which does not complete its downstream,
|
||||
|
|
@ -273,7 +284,11 @@ object Source extends SourceApply {
|
|||
* 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, "FailedSource")).withAttributes(DefaultAttributes.failedSource)
|
||||
new Source(
|
||||
new PublisherSource(
|
||||
ErrorPublisher(cause, "FailedSource")[T],
|
||||
DefaultAttributes.failedSource,
|
||||
shape("FailedSource")))
|
||||
|
||||
/**
|
||||
* Concatenates two sources so that the first element
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue