-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:
Viktor Klang 2015-07-06 22:00:21 +02:00
parent efc659b70a
commit 66a116d3d2
19 changed files with 333 additions and 290 deletions

View file

@ -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