!str #19732 Concise and consistent way to mark async boundaries

This commit is contained in:
Johan Andrén 2016-02-10 13:56:38 +01:00
parent d154fad4e9
commit 4e49d75ad8
25 changed files with 125 additions and 194 deletions

View file

@ -287,8 +287,7 @@ public class FlowDocTest {
//#flow-async //#flow-async
Source.range(1, 3) Source.range(1, 3)
.map(x -> x + 1) .map(x -> x + 1).async()
.withAttributes(Attributes.asyncBoundary())
.map(x -> x * 2) .map(x -> x * 2)
.to(Sink.ignore()); .to(Sink.ignore());
//#flow-async //#flow-async

View file

@ -5,6 +5,7 @@ package docs.stream;
import java.util.stream.Stream; import java.util.stream.Stream;
import akka.NotUsed;
import akka.japi.Pair; import akka.japi.Pair;
import akka.stream.javadsl.*; import akka.stream.javadsl.*;
//#asPublisher-import //#asPublisher-import
@ -27,6 +28,11 @@ public class MigrationsJava {
Sink.asPublisher(WITH_FANOUT); // instead of Sink.asPublisher(true) Sink.asPublisher(WITH_FANOUT); // instead of Sink.asPublisher(true)
Sink.asPublisher(WITHOUT_FANOUT); // instead of Sink.asPublisher(false) Sink.asPublisher(WITHOUT_FANOUT); // instead of Sink.asPublisher(false)
//#asPublisher //#asPublisher
//#async
Flow<Integer, Integer, NotUsed> flow = Flow.of(Integer.class).map(n -> n + 1);
Source.range(1, 10).via(flow.async());
//#async
} }
} }

View file

@ -104,6 +104,21 @@ Which is the same as using ``conflateWithSeed`` with an identity function::
Flow.of(Integer.class).conflateWithSeed(x -> x, (a, b) -> a + b) // Add numbers while downstream is not ready Flow.of(Integer.class).conflateWithSeed(x -> x, (a, b) -> a + b) // Add numbers while downstream is not ready
``viaAsync`` and ``viaAsyncMat`` has been replaced with ``async()``
-------------------------------------------------------------------
``async()`` is available from ``Sink``, ``Source``, ``Flow`` and the sub flows. It provides a shortcut for
setting the attribute ``Attributes.asyncBoundary`` on a flow. The existing methods ``Flow.viaAsync`` and
``Flow.viaAsyncMat`` has been removed to make marking out asynchronous boundaries more consistent::
// This no longer works
source.viaAsync(flow)
In Akka 2.4.x this will instead look lile this:
.. includecode:: ../code/docs/stream/MigrationsJava.java#async
Changed Sources / Sinks Changed Sources / Sinks
======================= =======================

View file

@ -243,8 +243,9 @@ The first point can be countered by pre-fusing and then reusing a stream bluepri
.. includecode:: ../code/docs/stream/FlowDocTest.java#explicit-fusing .. includecode:: ../code/docs/stream/FlowDocTest.java#explicit-fusing
In order to balance the effects of the second and third bullet points you will have to insert asynchronous In order to balance the effects of the second and third bullet points you will have to insert asynchronous
boundaries manually into your flows and graphs by way of adding ``Attributes.asyncBoundary`` to pieces that boundaries manually into your flows and graphs by way of adding ``Attributes.asyncBoundary`` using the method
shall communicate with the rest of the graph in an asynchronous fashion. ``async`` on ``Source``, ``Sink`` and ``Flow`` to pieces that shall communicate with the rest of the graph in
an asynchronous fashion.
.. includecode:: ../code/docs/stream/FlowDocTest.java#flow-async .. includecode:: ../code/docs/stream/FlowDocTest.java#flow-async

View file

@ -237,11 +237,8 @@ class FlowDocSpec extends AkkaSpec {
"defining asynchronous boundaries" in { "defining asynchronous boundaries" in {
//#flow-async //#flow-async
import akka.stream.Attributes.asyncBoundary
Source(List(1, 2, 3)) Source(List(1, 2, 3))
.map(_ + 1) .map(_ + 1).async
.withAttributes(asyncBoundary)
.map(_ * 2) .map(_ * 2)
.to(Sink.ignore) .to(Sink.ignore)
//#flow-async //#flow-async

View file

@ -23,6 +23,11 @@ class MigrationsScala extends AkkaSpec {
}) })
}) })
//#expand-state //#expand-state
//#async
val flow = Flow[Int].map(_ + 1)
Source(1 to 10).via(flow.async)
//#async
} }
} }
} }

View file

@ -91,6 +91,21 @@ Which is the same as using ``conflateWithSeed`` with an identity function
Flow[Int].conflateWithSeed(identity)(_ + _) // Add numbers while downstream is not ready Flow[Int].conflateWithSeed(identity)(_ + _) // Add numbers while downstream is not ready
``viaAsync`` and ``viaAsyncMat`` has been replaced with ``async``
-----------------------------------------------------------------
``async`` is available from ``Sink``, ``Source``, ``Flow`` and the sub flows. It provides a shortcut for
setting the attribute ``Attributes.asyncBoundary`` on a flow. The existing methods ``Flow.viaAsync`` and
``Flow.viaAsyncMat`` has been removed to make marking out asynchronous boundaries more consistent::
// This no longer works
source.viaAsync(flow)
In Akka 2.4.x this will instead look lile this:
.. includecode:: ../code/docs/stream/MigrationsScala.scala#async
Changes in Akka HTTP Changes in Akka HTTP
==================== ====================

View file

@ -245,8 +245,9 @@ The first point can be countered by pre-fusing and then reusing a stream bluepri
.. includecode:: ../code/docs/stream/FlowDocSpec.scala#explicit-fusing .. includecode:: ../code/docs/stream/FlowDocSpec.scala#explicit-fusing
In order to balance the effects of the second and third bullet points you will have to insert asynchronous In order to balance the effects of the second and third bullet points you will have to insert asynchronous
boundaries manually into your flows and graphs by way of adding ``Attributes.asyncBoundary`` to pieces that boundaries manually into your flows and graphs by way of adding ``Attributes.asyncBoundary`` using the method
shall communicate with the rest of the graph in an asynchronous fashion. ``async`` on ``Source``, ``Sink`` and ``Flow`` to pieces that shall communicate with the rest of the graph in an
asynchronous fashion.
.. includecode:: ../code/docs/stream/FlowDocSpec.scala#flow-async .. includecode:: ../code/docs/stream/FlowDocSpec.scala#flow-async

View file

@ -95,7 +95,7 @@ class FusingSpec extends AkkaSpec with ScalaFutures with ConversionCheckedTriple
"SubFusingActorMaterializer" must { "SubFusingActorMaterializer" must {
"work with asynchronous boundaries in the subflows" in { "work with asynchronous boundaries in the subflows" in {
val async = Flow[Int].map(_ * 2).withAttributes(Attributes.asyncBoundary) val async = Flow[Int].map(_ * 2).async
Source(0 to 9) Source(0 to 9)
.map(_ * 10) .map(_ * 10)
.flatMapMerge(5, i Source(i to (i + 9)).via(async)) .flatMapMerge(5, i Source(i to (i + 9)).via(async))
@ -110,7 +110,7 @@ class FusingSpec extends AkkaSpec with ScalaFutures with ConversionCheckedTriple
val bus = GraphInterpreter.currentInterpreter.log.asInstanceOf[BusLogging] val bus = GraphInterpreter.currentInterpreter.log.asInstanceOf[BusLogging]
bus.logSource bus.logSource
} }
val async = Flow[Int].map(x { testActor ! ref; x }).withAttributes(Attributes.asyncBoundary) val async = Flow[Int].map(x { testActor ! ref; x }).async
Source(0 to 9) Source(0 to 9)
.map(x { testActor ! ref; x }) .map(x { testActor ! ref; x })
.flatMapMerge(5, i Source.single(i).via(async)) .flatMapMerge(5, i Source.single(i).via(async))
@ -132,7 +132,7 @@ class FusingSpec extends AkkaSpec with ScalaFutures with ConversionCheckedTriple
val flow = Flow[Int].map(x { testActor ! ref; x }) val flow = Flow[Int].map(x { testActor ! ref; x })
Source(0 to 9) Source(0 to 9)
.map(x { testActor ! ref; x }) .map(x { testActor ! ref; x })
.flatMapMerge(5, i Source.single(i).viaAsync(flow)) .flatMapMerge(5, i Source.single(i).via(flow.async))
.grouped(1000) .grouped(1000)
.runWith(Sink.head) .runWith(Sink.head)
.futureValue .futureValue

View file

@ -114,7 +114,7 @@ class BidiFlowSpec extends AkkaSpec with ConversionCheckedTripleEquals {
"suitably override attribute handling methods" in { "suitably override attribute handling methods" in {
import Attributes._ import Attributes._
val b: BidiFlow[Int, Long, ByteString, String, NotUsed] = bidi.withAttributes(name("")).addAttributes(asyncBoundary).named("") val b: BidiFlow[Int, Long, ByteString, String, NotUsed] = bidi.withAttributes(name("")).async.named("")
} }
} }

View file

@ -596,7 +596,7 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
"suitably override attribute handling methods" in { "suitably override attribute handling methods" in {
import Attributes._ import Attributes._
val f: Flow[Int, Int, NotUsed] = Flow[Int].withAttributes(asyncBoundary).addAttributes(none).named("") val f: Flow[Int, Int, NotUsed] = Flow[Int].async.addAttributes(none).named("")
} }
} }

View file

@ -127,7 +127,7 @@ class SinkSpec extends AkkaSpec with ConversionCheckedTripleEquals with ScalaFut
"suitably override attribute handling methods" in { "suitably override attribute handling methods" in {
import Attributes._ import Attributes._
val s: Sink[Int, Future[Int]] = Sink.head[Int].withAttributes(asyncBoundary).addAttributes(none).named("") val s: Sink[Int, Future[Int]] = Sink.head[Int].async.addAttributes(none).named("")
} }
"support contramap" in { "support contramap" in {

View file

@ -272,7 +272,7 @@ class SourceSpec extends AkkaSpec with DefaultTimeout with ScalaFutures {
"A Source" must { "A Source" must {
"suitably override attribute handling methods" in { "suitably override attribute handling methods" in {
import Attributes._ import Attributes._
val s: Source[Int, NotUsed] = Source.single(42).withAttributes(asyncBoundary).addAttributes(none).named("") val s: Source[Int, NotUsed] = Source.single(42).async.addAttributes(none).named("")
} }
} }

View file

@ -26,5 +26,10 @@ trait Graph[+S <: Shape, +M] {
def named(name: String): Graph[S, M] = withAttributes(Attributes.name(name)) def named(name: String): Graph[S, M] = withAttributes(Attributes.name(name))
/**
* Put an asynchronous boundary around this `Graph`
*/
def async: Graph[S, M] = addAttributes(Attributes.asyncBoundary)
def addAttributes(attr: Attributes): Graph[S, M] = withAttributes(module.attributes and attr) def addAttributes(attr: Attributes): Graph[S, M] = withAttributes(module.attributes and attr)
} }

View file

@ -34,6 +34,8 @@ class SubFlowImpl[In, Out, Mat, F[+_], C](val subFlow: Flow[In, Out, NotUsed],
override def named(name: String): SubFlow[Out, Mat, F, C] = override def named(name: String): SubFlow[Out, Mat, F, C] =
new SubFlowImpl[In, Out, Mat, F, C](subFlow.named(name), mergeBackFunction, finishFunction) new SubFlowImpl[In, Out, Mat, F, C](subFlow.named(name), mergeBackFunction, finishFunction)
override def async: Repr[Out] = new SubFlowImpl[In, Out, Mat, F, C](subFlow.async, mergeBackFunction, finishFunction)
override def mergeSubstreamsWithParallelism(breadth: Int): F[Out] = mergeBackFunction(subFlow, breadth) override def mergeSubstreamsWithParallelism(breadth: Int): F[Out] = mergeBackFunction(subFlow, breadth)
def to[M](sink: Graph[SinkShape[Out], M]): C = finishFunction(subFlow.to(sink)) def to[M](sink: Graph[SinkShape[Out], M]): C = finishFunction(subFlow.to(sink))

View file

@ -125,47 +125,6 @@ final class Flow[-In, +Out, +Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends
def viaMat[T, M, M2](flow: Graph[FlowShape[Out, T], M], combine: function.Function2[Mat, M, M2]): javadsl.Flow[In, T, M2] = def viaMat[T, M, M2](flow: Graph[FlowShape[Out, T], M], combine: function.Function2[Mat, M, M2]): javadsl.Flow[In, T, M2] =
new Flow(delegate.viaMat(flow)(combinerToScala(combine))) new Flow(delegate.viaMat(flow)(combinerToScala(combine)))
/**
* Transform this [[Flow]] by appending the given processing steps, ensuring
* that an `asyncBoundary` attribute is set around those steps.
* {{{
* +----------------------------+
* | Resulting Flow |
* | |
* | +------+ +------+ |
* | | | | | |
* In ~~> | this | ~Out~> | flow | ~~> T
* | | | | | |
* | +------+ +------+ |
* +----------------------------+
* }}}
* The materialized value of the combined [[Flow]] will be the materialized
* value of the current flow (ignoring the other Flows value), use
* `viaMat` if a different strategy is needed.
*/
def viaAsync[T, M](flow: Graph[FlowShape[Out, T], M]): javadsl.Flow[In, T, Mat] =
new Flow(delegate.viaAsync(flow))
/**
* Transform this [[Flow]] by appending the given processing steps, ensuring
* that an `asyncBoundary` attribute is set around those steps.
* {{{
* +----------------------------+
* | Resulting Flow |
* | |
* | +------+ +------+ |
* | | | | | |
* In ~~> | this | ~Out~> | flow | ~~> T
* | | | | | |
* | +------+ +------+ |
* +----------------------------+
* }}}
* The `combine` function is used to compose the materialized values of this flow and that
* flow into the materialized value of the resulting Flow.
*/
def viaAsyncMat[T, M, M2](flow: Graph[FlowShape[Out, T], M], combine: function.Function2[Mat, M, M2]): javadsl.Flow[In, T, M2] =
new Flow(delegate.viaAsyncMat(flow)(combinerToScala(combine)))
/** /**
* Connect this [[Flow]] to a [[Sink]], concatenating the processing steps of both. * Connect this [[Flow]] to a [[Sink]], concatenating the processing steps of both.
* {{{ * {{{
@ -1664,6 +1623,12 @@ final class Flow[-In, +Out, +Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends
override def named(name: String): javadsl.Flow[In, Out, Mat] = override def named(name: String): javadsl.Flow[In, Out, Mat] =
new Flow(delegate.named(name)) new Flow(delegate.named(name))
/**
* Put an asynchronous boundary around this `Flow`
*/
override def async: javadsl.Flow[In, Out, Mat] =
new Flow(delegate.async)
/** /**
* Logs elements flowing through the stream as well as completion and erroring. * Logs elements flowing through the stream as well as completion and erroring.
* *

View file

@ -306,4 +306,11 @@ final class Sink[-In, +Mat](delegate: scaladsl.Sink[In, Mat]) extends Graph[Sink
*/ */
override def named(name: String): javadsl.Sink[In, Mat] = override def named(name: String): javadsl.Sink[In, Mat] =
new Sink(delegate.named(name)) new Sink(delegate.named(name))
/**
* Put an asynchronous boundary around this `Sink`
*/
override def async: javadsl.Sink[In, Mat] =
new Sink(delegate.async)
} }

View file

@ -376,47 +376,6 @@ final class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Grap
def viaMat[T, M, M2](flow: Graph[FlowShape[Out, T], M], combine: function.Function2[Mat, M, M2]): javadsl.Source[T, M2] = def viaMat[T, M, M2](flow: Graph[FlowShape[Out, T], M], combine: function.Function2[Mat, M, M2]): javadsl.Source[T, M2] =
new Source(delegate.viaMat(flow)(combinerToScala(combine))) new Source(delegate.viaMat(flow)(combinerToScala(combine)))
/**
* Transform this [[Source]] by appending the given processing stages, ensuring
* that an `asyncBoundary` attribute is set around those steps.
* {{{
* +----------------------------+
* | Resulting Source |
* | |
* | +------+ +------+ |
* | | | | | |
* | | this | ~Out~> | flow | ~~> T
* | | | | | |
* | +------+ +------+ |
* +----------------------------+
* }}}
* The materialized value of the combined [[Flow]] will be the materialized
* value of the current flow (ignoring the other Flows value), use
* `viaMat` if a different strategy is needed.
*/
def viaAsync[T, M](flow: Graph[FlowShape[Out, T], M]): javadsl.Source[T, Mat] =
new Source(delegate.viaAsync(flow))
/**
* Transform this [[Source]] by appending the given processing stages, ensuring
* that an `asyncBoundary` attribute is set around those steps.
* {{{
* +----------------------------+
* | Resulting Source |
* | |
* | +------+ +------+ |
* | | | | | |
* | | this | ~Out~> | flow | ~~> T
* | | | | | |
* | +------+ +------+ |
* +----------------------------+
* }}}
* The `combine` function is used to compose the materialized values of this flow and that
* flow into the materialized value of the resulting Flow.
*/
def viaAsyncMat[T, M, M2](flow: Graph[FlowShape[Out, T], M], combine: function.Function2[Mat, M, M2]): javadsl.Source[T, M2] =
new Source(delegate.viaAsyncMat(flow)(combinerToScala(combine)))
/** /**
* Connect this [[Source]] to a [[Sink]], concatenating the processing steps of both. * Connect this [[Source]] to a [[Sink]], concatenating the processing steps of both.
* {{{ * {{{
@ -1808,6 +1767,12 @@ final class Source[+Out, +Mat](delegate: scaladsl.Source[Out, Mat]) extends Grap
override def named(name: String): javadsl.Source[Out, Mat] = override def named(name: String): javadsl.Source[Out, Mat] =
new Source(delegate.named(name)) new Source(delegate.named(name))
/**
* Put an asynchronous boundary around this `Source`
*/
override def async: javadsl.Source[Out, Mat] =
new Source(delegate.async)
/** /**
* Logs elements flowing through the stream as well as completion and erroring. * Logs elements flowing through the stream as well as completion and erroring.
* *

View file

@ -84,29 +84,6 @@ class SubFlow[-In, +Out, +Mat](delegate: scaladsl.SubFlow[Out, Mat, scaladsl.Flo
def via[T, M](flow: Graph[FlowShape[Out, T], M]): SubFlow[In, T, Mat] = def via[T, M](flow: Graph[FlowShape[Out, T], M]): SubFlow[In, T, Mat] =
new SubFlow(delegate.via(flow)) new SubFlow(delegate.via(flow))
/**
* Transform this [[Flow]] by appending the given processing steps, ensuring
* that an `asyncBoundary` attribute is set around those steps.
*
* {{{
* +----------------------------+
* | Resulting Flow |
* | |
* | +------+ +------+ |
* | | | | | |
* In ~~> | this | ~Out~> | flow | ~~> T
* | | | | | |
* | +------+ +------+ |
* +----------------------------+
* }}}
*
* The materialized value of the combined [[Flow]] will be the materialized
* value of the current flow (ignoring the other Flows value), use
* [[Flow#viaMat viaMat]] if a different strategy is needed.
*/
def viaAsync[T, M](flow: Graph[FlowShape[Out, T], M]): SubFlow[In, T, Mat] =
new SubFlow(delegate.viaAsync(flow))
/** /**
* Connect this [[SubFlow]] to a [[Sink]], concatenating the processing steps of both. * Connect this [[SubFlow]] to a [[Sink]], concatenating the processing steps of both.
* This means that all sub-flows that result from the previous sub-stream operator * This means that all sub-flows that result from the previous sub-stream operator
@ -1191,6 +1168,12 @@ class SubFlow[-In, +Out, +Mat](delegate: scaladsl.SubFlow[Out, Mat, scaladsl.Flo
def named(name: String): SubFlow[In, Out, Mat] = def named(name: String): SubFlow[In, Out, Mat] =
new SubFlow(delegate.named(name)) new SubFlow(delegate.named(name))
/**
* Put an asynchronous boundary around this `SubFlow`
*/
def async: SubFlow[In, Out, Mat] =
new SubFlow(delegate.async)
/** /**
* Logs elements flowing through the stream as well as completion and erroring. * Logs elements flowing through the stream as well as completion and erroring.
* *

View file

@ -82,27 +82,6 @@ class SubSource[+Out, +Mat](delegate: scaladsl.SubFlow[Out, Mat, scaladsl.Source
def via[T, M](flow: Graph[FlowShape[Out, T], M]): SubSource[T, Mat] = def via[T, M](flow: Graph[FlowShape[Out, T], M]): SubSource[T, Mat] =
new SubSource(delegate.via(flow)) new SubSource(delegate.via(flow))
/**
* Transform this [[SubSource]] by appending the given processing steps, ensuring
* that an `asyncBoundary` attribute is set around those steps.
* {{{
* +----------------------------+
* | Resulting Source |
* | |
* | +------+ +------+ |
* | | | | | |
* | | this | ~Out~> | flow | ~~> T
* | | | | | |
* | +------+ +------+ |
* +----------------------------+
* }}}
* The materialized value of the combined [[Flow]] will be the materialized
* value of the current flow (ignoring the other Flows value), use
* [[Flow#viaMat viaMat]] if a different strategy is needed.
*/
def viaAsync[T, M](flow: Graph[FlowShape[Out, T], M]): SubSource[T, Mat] =
new SubSource(delegate.viaAsync(flow))
/** /**
* Connect this [[SubSource]] to a [[Sink]], concatenating the processing steps of both. * Connect this [[SubSource]] to a [[Sink]], concatenating the processing steps of both.
* This means that all sub-flows that result from the previous sub-stream operator * This means that all sub-flows that result from the previous sub-stream operator
@ -1188,6 +1167,12 @@ class SubSource[+Out, +Mat](delegate: scaladsl.SubFlow[Out, Mat, scaladsl.Source
def named(name: String): SubSource[Out, Mat] = def named(name: String): SubSource[Out, Mat] =
new SubSource(delegate.named(name)) new SubSource(delegate.named(name))
/**
* Put an asynchronous boundary around this `SubSource`
*/
def async: SubSource[Out, Mat] =
new SubSource(delegate.async)
/** /**
* Logs elements flowing through the stream as well as completion and erroring. * Logs elements flowing through the stream as well as completion and erroring.
* *

View file

@ -152,7 +152,10 @@ final class BidiFlow[-I1, +O1, -I2, +O2, +Mat](private[stream] override val modu
* Add a ``name`` attribute to this Flow. * Add a ``name`` attribute to this Flow.
*/ */
override def named(name: String): BidiFlow[I1, O1, I2, O2, Mat] = override def named(name: String): BidiFlow[I1, O1, I2, O2, Mat] =
withAttributes(Attributes.name(name)) addAttributes(Attributes.name(name))
override def async: BidiFlow[I1, O1, I2, O2, Mat] =
addAttributes(Attributes.asyncBoundary)
} }
object BidiFlow { object BidiFlow {

View file

@ -214,7 +214,7 @@ final class Flow[-In, +Out, +Mat](private[stream] override val module: Module)
*/ */
override def withAttributes(attr: Attributes): Repr[Out] = override def withAttributes(attr: Attributes): Repr[Out] =
if (isIdentity) this if (isIdentity) this
else new Flow(module.withAttributes(attr).nest()) else new Flow(module.withAttributes(attr))
/** /**
* Add the given attributes to this Flow. Further calls to `withAttributes` * Add the given attributes to this Flow. Further calls to `withAttributes`
@ -227,7 +227,12 @@ final class Flow[-In, +Out, +Mat](private[stream] override val module: Module)
/** /**
* Add a ``name`` attribute to this Flow. * Add a ``name`` attribute to this Flow.
*/ */
override def named(name: String): Repr[Out] = withAttributes(Attributes.name(name)) override def named(name: String): Repr[Out] = addAttributes(Attributes.name(name))
/**
* Put an asynchronous boundary around this `Flow`
*/
override def async: Repr[Out] = addAttributes(Attributes.asyncBoundary)
/** /**
* Connect the `Source` to this `Flow` and then connect it to the `Sink` and run it. The returned tuple contains * Connect the `Source` to this `Flow` and then connect it to the `Sink` and run it. The returned tuple contains
@ -390,26 +395,6 @@ trait FlowOps[+Out, +Mat] {
*/ */
def via[T, Mat2](flow: Graph[FlowShape[Out, T], Mat2]): Repr[T] def via[T, Mat2](flow: Graph[FlowShape[Out, T], Mat2]): Repr[T]
/**
* Transform this [[Flow]] by appending the given processing steps, ensuring
* that an `asyncBoundary` attribute is set around those steps.
* {{{
* +----------------------------+
* | Resulting Flow |
* | |
* | +------+ +------+ |
* | | | | | |
* In ~~> | this | ~Out~> | flow | ~~> T
* | | | | | |
* | +------+ +------+ |
* +----------------------------+
* }}}
* The materialized value of the combined [[Flow]] will be the materialized
* value of the current flow (ignoring the other Flows value), use
* [[Flow#viaMat viaMat]] if a different strategy is needed.
*/
def viaAsync[T, Mat2](flow: Graph[FlowShape[Out, T], Mat2]): Repr[T] = via(flow.addAttributes(Attributes.asyncBoundary))
/** /**
* Recover allows to send last element on failure and gracefully complete the stream * Recover allows to send last element on failure and gracefully complete the stream
* Since the underlying failure signal onError arrives out-of-band, it might jump over existing elements. * Since the underlying failure signal onError arrives out-of-band, it might jump over existing elements.
@ -1743,6 +1728,8 @@ trait FlowOps[+Out, +Mat] {
def named(name: String): Repr[Out] def named(name: String): Repr[Out]
def async: Repr[Out]
/** INTERNAL API */ /** INTERNAL API */
private[scaladsl] def andThen[T](op: SymbolicStage[Out, T]): Repr[T] = private[scaladsl] def andThen[T](op: SymbolicStage[Out, T]): Repr[T] =
via(SymbolicGraphStage(op)) via(SymbolicGraphStage(op))
@ -1787,26 +1774,6 @@ trait FlowOpsMat[+Out, +Mat] extends FlowOps[Out, Mat] {
*/ */
def viaMat[T, Mat2, Mat3](flow: Graph[FlowShape[Out, T], Mat2])(combine: (Mat, Mat2) Mat3): ReprMat[T, Mat3] def viaMat[T, Mat2, Mat3](flow: Graph[FlowShape[Out, T], Mat2])(combine: (Mat, Mat2) Mat3): ReprMat[T, Mat3]
/**
* Transform this [[Flow]] by appending the given processing steps, ensuring
* that an `asyncBoundary` attribute is set around those steps.
* {{{
* +----------------------------+
* | Resulting Flow |
* | |
* | +------+ +------+ |
* | | | | | |
* In ~~> | this | ~Out~> | flow | ~~> T
* | | | | | |
* | +------+ +------+ |
* +----------------------------+
* }}}
* The `combine` function is used to compose the materialized values of this flow and that
* flow into the materialized value of the resulting Flow.
*/
def viaAsyncMat[T, Mat2, Mat3](flow: Graph[FlowShape[Out, T], Mat2])(combine: (Mat, Mat2) Mat3): ReprMat[T, Mat3] =
viaMat(flow.addAttributes(Attributes.asyncBoundary))(combine)
/** /**
* Connect this [[Flow]] to a [[Sink]], concatenating the processing steps of both. * Connect this [[Flow]] to a [[Sink]], concatenating the processing steps of both.
* {{{ * {{{

View file

@ -1007,14 +1007,14 @@ object GraphDSL extends GraphApply {
private class PortOpsImpl[+Out](override val outlet: Outlet[Out @uncheckedVariance], b: Builder[_]) private class PortOpsImpl[+Out](override val outlet: Outlet[Out @uncheckedVariance], b: Builder[_])
extends PortOps[Out] { extends PortOps[Out] {
override def withAttributes(attr: Attributes): Repr[Out] = override def withAttributes(attr: Attributes): Repr[Out] = throw settingAttrNotSupported
throw new UnsupportedOperationException("Cannot set attributes on chained ops from a junction output port") override def addAttributes(attr: Attributes): Repr[Out] = throw settingAttrNotSupported
override def named(name: String): Repr[Out] = throw settingAttrNotSupported
override def async: Repr[Out] = throw settingAttrNotSupported
override def addAttributes(attr: Attributes): Repr[Out] = private def settingAttrNotSupported =
throw new UnsupportedOperationException("Cannot set attributes on chained ops from a junction output port") new UnsupportedOperationException("Cannot set attributes on chained ops from a junction output port")
override def named(name: String): Repr[Out] =
throw new UnsupportedOperationException("Cannot set attributes on chained ops from a junction output port")
override def importAndGetPort(b: Builder[_]): Outlet[Out @uncheckedVariance] = outlet override def importAndGetPort(b: Builder[_]): Outlet[Out @uncheckedVariance] = outlet

View file

@ -72,7 +72,12 @@ final class Sink[-In, +Mat](private[stream] override val module: Module)
/** /**
* Add a ``name`` attribute to this Flow. * Add a ``name`` attribute to this Flow.
*/ */
override def named(name: String): Sink[In, Mat] = withAttributes(Attributes.name(name)) override def named(name: String): Sink[In, Mat] = addAttributes(Attributes.name(name))
/**
* Put an asynchronous boundary around this `Sink`
*/
override def async: Sink[In, Mat] = addAttributes(Attributes.asyncBoundary)
/** Converts this Scala DSL element to it's Java DSL counterpart. */ /** Converts this Scala DSL element to it's Java DSL counterpart. */
def asJava: javadsl.Sink[In, Mat] = new javadsl.Sink(this) def asJava: javadsl.Sink[In, Mat] = new javadsl.Sink(this)

View file

@ -145,7 +145,12 @@ final class Source[+Out, +Mat](private[stream] override val module: Module)
/** /**
* Add a ``name`` attribute to this Flow. * Add a ``name`` attribute to this Flow.
*/ */
override def named(name: String): Repr[Out] = withAttributes(Attributes.name(name)) override def named(name: String): Repr[Out] = addAttributes(Attributes.name(name))
/**
* Put an asynchronous boundary around this `Source`
*/
override def async: Repr[Out] = addAttributes(Attributes.asyncBoundary)
/** Converts this Scala DSL element to it's Java DSL counterpart. */ /** Converts this Scala DSL element to it's Java DSL counterpart. */
def asJava: javadsl.Source[Out, Mat] = new javadsl.Source(this) def asJava: javadsl.Source[Out, Mat] = new javadsl.Source(this)