Merge branch 'wip-RIP-OneBoundedInterpreter2' into release-2.3-dev
This commit is contained in:
commit
06ce968b16
107 changed files with 2456 additions and 3061 deletions
|
|
@ -1,7 +1,7 @@
|
|||
package akka.stream
|
||||
|
||||
import akka.event._
|
||||
import akka.stream.impl.fusing.{ GraphInterpreterSpecKit, GraphStages, Map => MapStage, OneBoundedInterpreter }
|
||||
import akka.stream.impl.fusing.{ GraphInterpreterSpecKit, GraphStages, Map => MapStage }
|
||||
import akka.stream.impl.fusing.GraphStages.Identity
|
||||
import akka.stream.impl.fusing.GraphInterpreter.{ DownstreamBoundaryStageLogic, UpstreamBoundaryStageLogic }
|
||||
import akka.stream.stage._
|
||||
|
|
@ -18,10 +18,10 @@ class InterpreterBenchmark {
|
|||
import InterpreterBenchmark._
|
||||
|
||||
// manual, and not via @Param, because we want @OperationsPerInvocation on our tests
|
||||
final val data100k = (1 to 100000).toVector
|
||||
final val data100k: Vector[Int] = (1 to 100000).toVector
|
||||
|
||||
@Param(Array("1", "5", "10"))
|
||||
val numberOfIds = 0
|
||||
val numberOfIds: Int = 0
|
||||
|
||||
@Benchmark
|
||||
@OperationsPerInvocation(100000)
|
||||
|
|
@ -47,33 +47,13 @@ class InterpreterBenchmark {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Benchmark
|
||||
@OperationsPerInvocation(100000)
|
||||
def onebounded_interpreter_100k_elements() {
|
||||
val lock = new Lock()
|
||||
lock.acquire()
|
||||
val sink = OneBoundedDataSink(data100k.size)
|
||||
val ops = Vector.fill(numberOfIds)(new PushPullStage[Int, Int] {
|
||||
override def onPull(ctx: _root_.akka.stream.stage.Context[Int]) = ctx.pull()
|
||||
override def onPush(elem: Int, ctx: _root_.akka.stream.stage.Context[Int]) = ctx.push(elem)
|
||||
})
|
||||
val interpreter = new OneBoundedInterpreter(OneBoundedDataSource(data100k) +: ops :+ sink,
|
||||
(op, ctx, event) ⇒ (),
|
||||
Logging(NoopBus, classOf[InterpreterBenchmark]),
|
||||
null,
|
||||
Attributes.none,
|
||||
forkLimit = 100, overflowToHeap = false)
|
||||
interpreter.init()
|
||||
sink.requestOne()
|
||||
}
|
||||
}
|
||||
|
||||
object InterpreterBenchmark {
|
||||
|
||||
case class GraphDataSource[T](override val toString: String, data: Vector[T]) extends UpstreamBoundaryStageLogic[T] {
|
||||
var idx = 0
|
||||
val out = Outlet[T]("out")
|
||||
var idx: Int = 0
|
||||
override val out: akka.stream.Outlet[T] = Outlet[T]("out")
|
||||
out.id = 0
|
||||
|
||||
setHandler(out, new OutHandler {
|
||||
|
|
@ -81,8 +61,7 @@ object InterpreterBenchmark {
|
|||
if (idx < data.size) {
|
||||
push(out, data(idx))
|
||||
idx += 1
|
||||
}
|
||||
else {
|
||||
} else {
|
||||
completeStage()
|
||||
}
|
||||
}
|
||||
|
|
@ -91,7 +70,7 @@ object InterpreterBenchmark {
|
|||
}
|
||||
|
||||
case class GraphDataSink[T](override val toString: String, var expected: Int) extends DownstreamBoundaryStageLogic[T] {
|
||||
val in = Inlet[T]("in")
|
||||
override val in: akka.stream.Inlet[T] = Inlet[T]("in")
|
||||
in.id = 0
|
||||
|
||||
setHandler(in, new InHandler {
|
||||
|
|
@ -104,49 +83,7 @@ object InterpreterBenchmark {
|
|||
override def onUpstreamFailure(ex: Throwable): Unit = failStage(ex)
|
||||
})
|
||||
|
||||
def requestOne() = pull(in)
|
||||
}
|
||||
|
||||
case class OneBoundedDataSource[T](data: Vector[T]) extends BoundaryStage {
|
||||
var idx = 0
|
||||
|
||||
override def onDownstreamFinish(ctx: BoundaryContext): TerminationDirective = {
|
||||
ctx.finish()
|
||||
}
|
||||
|
||||
override def onPull(ctx: BoundaryContext): Directive = {
|
||||
if (idx < data.size) {
|
||||
idx += 1
|
||||
ctx.push(data(idx - 1))
|
||||
}
|
||||
else {
|
||||
ctx.finish()
|
||||
}
|
||||
}
|
||||
|
||||
override def onPush(elem: Any, ctx: BoundaryContext): Directive =
|
||||
throw new UnsupportedOperationException("Cannot push the boundary")
|
||||
}
|
||||
|
||||
case class OneBoundedDataSink(var expected: Int) extends BoundaryStage {
|
||||
override def onPush(elem: Any, ctx: BoundaryContext): Directive = {
|
||||
expected -= 1
|
||||
if (expected == 0) ctx.exit()
|
||||
else ctx.pull()
|
||||
}
|
||||
|
||||
override def onUpstreamFinish(ctx: BoundaryContext): TerminationDirective = {
|
||||
ctx.finish()
|
||||
}
|
||||
|
||||
override def onUpstreamFailure(cause: Throwable, ctx: BoundaryContext): TerminationDirective = {
|
||||
ctx.finish()
|
||||
}
|
||||
|
||||
override def onPull(ctx: BoundaryContext): Directive =
|
||||
throw new UnsupportedOperationException("Cannot pull the boundary")
|
||||
|
||||
def requestOne(): Unit = enterAndPull()
|
||||
def requestOne(): Unit = pull(in)
|
||||
}
|
||||
|
||||
val NoopBus = new LoggingBus {
|
||||
|
|
|
|||
|
|
@ -159,8 +159,8 @@ public class HttpServerExampleDocTest {
|
|||
Flow.of(HttpRequest.class)
|
||||
.via(failureDetection)
|
||||
.map(request -> {
|
||||
Source<ByteString, ?> bytes = request.entity().getDataBytes();
|
||||
HttpEntity.Chunked entity = HttpEntities.create(ContentTypes.TEXT_PLAIN, (Source<ByteString, Object>) bytes);
|
||||
Source<ByteString, Object> bytes = request.entity().getDataBytes();
|
||||
HttpEntity.Chunked entity = HttpEntities.create(ContentTypes.TEXT_PLAIN, bytes);
|
||||
|
||||
return HttpResponse.create()
|
||||
.withEntity(entity);
|
||||
|
|
|
|||
263
akka-docs-dev/rst/migration-guide-1.0-2.x.rst
Normal file
263
akka-docs-dev/rst/migration-guide-1.0-2.x.rst
Normal file
|
|
@ -0,0 +1,263 @@
|
|||
.. _migration-2.0:
|
||||
|
||||
############################
|
||||
Migration Guide 1.0 to 2.x
|
||||
############################
|
||||
|
||||
The 2.0 release contains some structural changes that require some
|
||||
simple, mechanical source-level changes in client code.
|
||||
|
||||
|
||||
Introduced proper named constructor methods insted of ``wrap()``
|
||||
================================================================
|
||||
|
||||
There were several, unrelated uses of ``wrap()`` which made it hard to find and hard to understand the intention of
|
||||
the call. Therefore these use-cases now have methods with different names, helping Java 8 type inference (by reducing
|
||||
the number of overloads) and finding relevant methods in the documentation.
|
||||
|
||||
Creating a Flow from other stages
|
||||
---------------------------------
|
||||
|
||||
It was possible to create a ``Flow`` from a graph with the correct shape (``FlowShape``) using ``wrap()``. Now this
|
||||
must be done with the more descriptive method ``Flow.fromGraph()``.
|
||||
|
||||
It was possible to create a ``Flow`` from a ``Source`` and a ``Sink`` using ``wrap()``. Now this functionality can
|
||||
be accessed trough the more descriptive methods ``Flow.fromSinkAndSource`` and ``Flow.fromSinkAndSourceMat``.
|
||||
|
||||
Creating a BidiFlow from other stages
|
||||
-------------------------------------
|
||||
|
||||
It was possible to create a ``BidiFlow`` from a graph with the correct shape (``BidiShape``) using ``wrap()``. Now this
|
||||
must be done with the more descriptive method ``BidiFlow.fromGraph()``.
|
||||
|
||||
It was possible to create a ``BidiFlow`` from two ``Flow`` s using ``wrap()``. Now this functionality can
|
||||
be accessed trough the more descriptive methods ``BidiFlow.fromFlows`` and ``BidiFlow.fromFlowsMat``.
|
||||
|
||||
It was possible to create a ``BidiFlow`` from two functions using ``apply()`` (Scala DSL) or ``create()`` (Java DSL).
|
||||
Now this functionality can be accessed trough the more descriptive method ``BidiFlow.fromFunctions``.
|
||||
|
||||
Update procedure
|
||||
----------------
|
||||
|
||||
1. Replace all uses of ``Flow.wrap`` when it converts a ``Graph`` to a ``Flow`` with ``Flow.fromGraph``
|
||||
2. Replace all uses of ``Flow.wrap`` when it converts a ``Source`` and ``Sink`` to a ``Flow`` with
|
||||
``Flow.fromSinkAndSource`` or ``Flow.fromSinkAndSourceMat``
|
||||
3. Replace all uses of ``BidiFlow.wrap`` when it converts a ``Graph`` to a ``BidiFlow`` with ``BidiFlow.fromGraph``
|
||||
4. Replace all uses of ``BidiFlow.wrap`` when it converts two ``Flow``s to a ``BidiFlow`` with
|
||||
``BidiFlow.fromFlows`` or ``BidiFlow.fromFlowsMat``
|
||||
5. Repplace all uses of ``BidiFlow.apply()`` (Scala DSL) or ``BidiFlow.create()`` (Java DSL) when it converts two
|
||||
functions to a ``BidiFlow`` with ``BidiFlow.fromFunctions``
|
||||
|
||||
TODO: Code example
|
||||
|
||||
FlowGraph builder methods have been renamed
|
||||
===========================================
|
||||
|
||||
There is now only one graph creation method called ``create`` which is analogous to the old ``partial`` method. For
|
||||
closed graphs now it is explicitly required to return ``ClosedShape`` at the end of the builder block.
|
||||
|
||||
Update procedure
|
||||
----------------
|
||||
|
||||
1. Replace all occurrences of ``FlowGraph.create()`` with ``FlowGraph.partial()``
|
||||
2. Add ``ClosedShape`` as a return value of the builder block
|
||||
|
||||
TODO: Code sample
|
||||
|
||||
Methods that create Source, Sink, Flow from Graphs have been removed
|
||||
====================================================================
|
||||
|
||||
Previously there were convenience methods available on ``Sink``, ``Source``, ``Flow`` an ``BidiFlow`` to create
|
||||
these DSL elements from a graph builder directly. Now this requires two explicit steps to reduce the number of overloaded
|
||||
methods (helps Java 8 type inference) and also reduces the ways how these elements can be created. There is only one
|
||||
graph creation method to learn (``FlowGraph.create``) and then there is only one conversion method to use ``fromGraph()``.
|
||||
|
||||
This means that the following methods have been removed:
|
||||
- ``adapt()`` method on ``Source``, ``Sink``, ``Flow`` and ``BidiFlow`` (both DSLs)
|
||||
- ``apply()`` overloads providing a graph ``Builder`` on ``Source``, ``Sink``, ``Flow`` and ``BidiFlow`` (Scala DSL)
|
||||
- ``create()`` overloads providing a graph ``Builder`` on ``Source``, ``Sink``, ``Flow`` and ``BidiFlow`` (Java DSL)
|
||||
|
||||
Update procedure
|
||||
----------------
|
||||
|
||||
Everywhere where ``Source``, ``Sink``, ``Flow`` and ``BidiFlow`` is created from a graph using a builder have to
|
||||
be replaced with two steps
|
||||
|
||||
1. Create a ``Graph`` with the correct ``Shape`` using ``FlowGraph.create`` (e.g.. for ``Source`` it means first
|
||||
creating a ``Graph`` with ``SourceShape``)
|
||||
2. Create the required DSL element by calling ``fromGraph()`` on the required DSL element (e.g. ``Source.fromGraph``)
|
||||
passing the graph created in the previous step
|
||||
|
||||
TODO code example
|
||||
|
||||
Some graph Builder methods in the Java DSL have been renamed
|
||||
============================================================
|
||||
|
||||
Due to the high number of overloads Java 8 type inference suffered, and it was also hard to figure out which time
|
||||
to use which method. Therefore various redundant methods have been removed.
|
||||
|
||||
Update procedure
|
||||
----------------
|
||||
|
||||
1. All uses of builder.addEdge(Outlet, Inlet) should be replaced by the alternative builder.from(…).to(…)
|
||||
2. All uses of builder.addEdge(Outlet, FlowShape, Inlet) should be replaced by builder.from(…).via(…).to(…)
|
||||
|
||||
Builder.source => use builder.from(…).via(…).to(…)
|
||||
Builder.flow => use builder.from(…).via(…).to(…)
|
||||
Builder.sink => use builder.from(…).via(…).to(…)
|
||||
|
||||
TODO: code example
|
||||
|
||||
Builder overloads from the Scala DSL have been removed
|
||||
======================================================
|
||||
|
||||
scaladsl.Builder.addEdge(Outlet, Inlet) => use the DSL (~> and <~)
|
||||
scaladsl.Builder.addEdge(Outlet, FlowShape, Inlet) => use the DSL (~> and <~)
|
||||
|
||||
Source constructor name changes
|
||||
===============================
|
||||
|
||||
``Source.lazyEmpty`` have been replaced by ``Source.maybe`` which returns a ``Promise`` that can be completed by one or
|
||||
zero elements by providing an ``Option``. This is different from ``lazyEmpty`` which only allowed completion to be
|
||||
sent, but no elements.
|
||||
|
||||
The ``apply()`` and ``from()`` overloads on ``Source`` that provide a tick source (``Source(delay,interval,tick)``)
|
||||
are replaced by the named method ``Source.tick()`` to reduce the number of overloads and to make the function more
|
||||
discoverable.
|
||||
|
||||
Update procedure
|
||||
----------------
|
||||
|
||||
1. Replace all uses of ``Source(delay,interval,tick)`` and ``Source.from(delay,interval,tick)`` with the method
|
||||
``Source.tick()``
|
||||
2. All uses of ``Source.lazyEmpty`` should be replaced by ``Source.maybe`` and the returned ``Promise`` completed with
|
||||
a ``None`` (an empty ``Option``)
|
||||
|
||||
TODO: code example
|
||||
|
||||
``Flow.empty()`` has been removed from the Java DSL
|
||||
===================================================
|
||||
|
||||
The ``empty()`` method has been removed since it behaves exactly the same as ``create()``, creating a ``Flow`` with no
|
||||
transformations added yet.
|
||||
|
||||
Update procedure
|
||||
----------------
|
||||
|
||||
1. Replace all uses of ``Flow.empty()`` with ``Flow.create``.
|
||||
|
||||
TODO: code example
|
||||
|
||||
``flatten(FlattenStrategy)`` has been replaced by named counterparts
|
||||
====================================================================
|
||||
|
||||
To simplify type inference in Java 8 and to make the method more discoverable, ``flatten(FlattenStrategy.concat)``
|
||||
has been removed and replaced with the alternative method ``flatten(FlattenStrategy.concat)``.
|
||||
|
||||
Update procedure
|
||||
----------------
|
||||
|
||||
1. Replace all occurences of ``flatten(FlattenStrategy.concat)`` with ``flattenConcat()``
|
||||
|
||||
TODO: code example
|
||||
|
||||
FlexiMerge an FlexiRoute has been replaced by GraphStage
|
||||
========================================================
|
||||
|
||||
The ``FlexiMerge`` and ``FlexiRoute`` DSLs have been removed since they provided an abstraction that was too limiting
|
||||
and a better abstraction have been created which is called ``GraphStage``. ``GraphStage`` can express fan-in and
|
||||
fan-out stages, but many other constructs as well with possibly multiple input and output ports (e.g. a ``BidiStage``).
|
||||
|
||||
This new abstraction provides a more uniform way to crate custom stream processing stages of arbitrary ``Shape``. In
|
||||
fact, all of the built-in fan-in and fan-out stages are now implemented in terms of ``GraphStage``.
|
||||
|
||||
Update procedure
|
||||
----------------
|
||||
|
||||
*There is no simple update procedure. The affected stages must be ported to the new ``GraphStage`` DSL manually. Please
|
||||
read the* ``GraphStage`` *documentation (TODO) for details.*
|
||||
|
||||
Variance of Inlet and Outlet (Scala DSL)
|
||||
========================================
|
||||
|
||||
Scala uses *declaration site variance* which was cumbersome in the cases of ``Inlet`` and ``Outlet`` as they are
|
||||
purely symbolic object containing no fields or methods and which are used both in input and output locations (wiring
|
||||
an ``Outlet`` into an ``Inlet``; reading in a stage from an ``Inlet``). Because of this reasons all users of these
|
||||
port abstractions now use *use-site variance* (just like Java variance works). This in general does not affect user
|
||||
code expect the case of custom shapes, which now require ``@uncheckedVariance`` annotations on their ``Inlet`` and
|
||||
``Outlet`` members (since these are now invariant, but the Scala compiler does not know that they have no fields or
|
||||
methods that would violate variance constraints)
|
||||
|
||||
This change does not affect Java DSL users.
|
||||
|
||||
TODO: code example
|
||||
|
||||
Update procedure
|
||||
----------------
|
||||
|
||||
1. All custom shapes must use ``@uncheckedVariance`` on their ``Inlet`` and ``Outlet`` members.
|
||||
|
||||
Semantic change in ``isHoldingUpstream`` in the DetachedStage DSL
|
||||
=================================================================
|
||||
|
||||
The ``isHoldingUpstream`` method used to return true if the upstream port was in holding state and a completion arrived
|
||||
(inside the ``onUpstreamFinished`` callback). Now it returns ``false`` when the upstream is completed.
|
||||
|
||||
Update procedure
|
||||
----------------
|
||||
|
||||
1. Those stages that relied on the previous behavior need to introduce an extra ``Boolean`` field with initial value
|
||||
``false``
|
||||
2. This field must be set on every call to ``holdUpstream()`` (and variants).
|
||||
3. In completion, instead of calling ``isHoldingUpstream`` read this variable instead.
|
||||
|
||||
TODO: code example
|
||||
|
||||
|
||||
AsyncStage has been replaced by GraphStage
|
||||
==========================================
|
||||
|
||||
Due to its complexity and relative inflexibility ``AsyncStage`` have been removed.
|
||||
|
||||
TODO explanation
|
||||
|
||||
Update procedure
|
||||
----------------
|
||||
|
||||
1. The subclass of ``AsyncStage`` should be replaced by ``GraphStage``
|
||||
2. The new subclass must define an ``in`` and ``out`` port (``Inlet`` and ``Outlet`` instance) and override the ``shape``
|
||||
method returning a ``FlowShape``
|
||||
3. An instance of ``GraphStageLogic`` must be returned by overriding ``createLogic()``. The original processing logic and
|
||||
state will be encapsulated in this ``GraphStageLogic``
|
||||
4. Using ``setHandler(port, handler)`` and ``InHandler`` instance should be set on ``in`` and an ``OutHandler`` should
|
||||
be set on ``out``
|
||||
5. ``onPush``, ``onUpstreamFinished`` and ``onUpstreamFailed`` are now available in the ``InHandler`` subclass created
|
||||
by the user
|
||||
6. ``onPull`` and ``onDownstreamFinished`` are now available in the ``OutHandler`` subclass created by the user
|
||||
7. the callbacks above no longer take an extra `ctxt` context parameter.
|
||||
8. ``onPull`` only signals the stage, the actual element can be obtained by calling ``grab(in)``
|
||||
9. ``ctx.push(elem)`` is now ``push(out, elem)``
|
||||
10. ``ctx.pull()`` is now ``pull(in)``
|
||||
11. ``ctx.finish()`` is now ``completeStage()``
|
||||
12. ``ctx.pushAndFinish(elem)`` is now simply two calls: ``push(out, elem); completeStage()``
|
||||
13. ``ctx.fail(cause)`` is now ``failStage(cause)``
|
||||
14. ``ctx.isFinishing()`` is now ``isClosed(in)``
|
||||
15. ``ctx.absorbTermination()`` can be replaced with ``if (isAvailable(shape.outlet)) <call the onPull() handler>``
|
||||
16. ``ctx.pushAndPull(elem)`` can be replaced with ``push(out, elem); pull(in)``
|
||||
17. ``ctx.holdUpstreamAndPush`` and ``context.holdDownstreamAndPull`` can be replaced by simply ``push(elem)`` and
|
||||
``pull()`` respectively
|
||||
18. The following calls should be removed: ``ctx.ignore()``, ``ctx.holdUpstream()`` and ``ctx.holdDownstream()``.
|
||||
19. ``ctx.isHoldingUpstream()`` can be replaced with ``isAvailable(out)``
|
||||
20. ``ctx.isHoldingDowntream()`` can be replaced with ``!(isClosed(in) || hasBeenPulled(in))``
|
||||
21. ``ctx.getAsyncCallback()`` is now ``getAsyncCallback(callback)`` which now takes a callback as a parameter. This
|
||||
would correspond to the ``onAsyncInput()`` callback in the original ``AsyncStage``
|
||||
|
||||
We show the necessary steps in terms of an example ``AsyncStage``
|
||||
|
||||
TODO: code sample
|
||||
|
||||
|
||||
|
||||
TODO: Code example
|
||||
|
||||
|
||||
|
|
@ -352,7 +352,7 @@ class HttpServerExampleSpec extends WordSpec with Matchers {
|
|||
pathEnd {
|
||||
(put | parameter('method ! "put")) {
|
||||
// form extraction from multipart or www-url-encoded forms
|
||||
formFields('email, 'total.as[Money]).as(Order) { order =>
|
||||
formFields(('email, 'total.as[Money])).as(Order) { order =>
|
||||
complete {
|
||||
// complete with serialized Future result
|
||||
(myDbActor ? Update(order)).mapTo[TransactionResult]
|
||||
|
|
@ -373,7 +373,7 @@ class HttpServerExampleSpec extends WordSpec with Matchers {
|
|||
path("items") {
|
||||
get {
|
||||
// parameters to case class extraction
|
||||
parameters('size.as[Int], 'color ?, 'dangerous ? "no")
|
||||
parameters(('size.as[Int], 'color ?, 'dangerous ? "no"))
|
||||
.as(OrderItem) { orderItem =>
|
||||
// ... route using case class instance created from
|
||||
// required and optional query parameters
|
||||
|
|
|
|||
|
|
@ -26,9 +26,6 @@ class FutureDirectivesExamplesSpec extends RoutingSpec {
|
|||
ctx.complete((InternalServerError, "Unsuccessful future!"))
|
||||
}
|
||||
|
||||
val resourceActor = system.actorOf(Props(new Actor {
|
||||
def receive = { case _ => sender ! "resource" }
|
||||
}))
|
||||
implicit val responseTimeout = Timeout(2, TimeUnit.SECONDS)
|
||||
|
||||
"onComplete" in {
|
||||
|
|
|
|||
|
|
@ -7,6 +7,7 @@ package directives
|
|||
|
||||
import akka.http.scaladsl.model._
|
||||
import akka.http.scaladsl.server.{ Route, ValidationRejection }
|
||||
import akka.testkit.EventFilter
|
||||
|
||||
class RouteDirectivesExamplesSpec extends RoutingSpec {
|
||||
|
||||
|
|
@ -83,7 +84,7 @@ class RouteDirectivesExamplesSpec extends RoutingSpec {
|
|||
}
|
||||
}
|
||||
|
||||
"failwith-examples" in {
|
||||
"failwith-examples" in EventFilter[RuntimeException](start = "Error during processing of request", occurrences = 1).intercept {
|
||||
val route =
|
||||
path("foo") {
|
||||
failWith(new RuntimeException("Oops."))
|
||||
|
|
|
|||
|
|
@ -12,6 +12,7 @@ import scala.math._
|
|||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
import scala.collection.immutable
|
||||
import akka.testkit.TestLatch
|
||||
|
||||
class RateTransformationDocSpec extends AkkaSpec {
|
||||
|
||||
|
|
@ -84,9 +85,14 @@ class RateTransformationDocSpec extends AkkaSpec {
|
|||
case (lastElement, drift) => ((lastElement, drift), (lastElement, drift + 1))
|
||||
}
|
||||
//#expand-drift
|
||||
val latch = TestLatch(2)
|
||||
val realDriftFlow = Flow[Double]
|
||||
.expand(d => { latch.countDown(); (d, 0) }) {
|
||||
case (lastElement, drift) => ((lastElement, drift), (lastElement, drift + 1))
|
||||
}
|
||||
|
||||
val (pub, sub) = TestSource.probe[Double]
|
||||
.via(driftFlow)
|
||||
.via(realDriftFlow)
|
||||
.toMat(TestSink.probe[(Double, Int)])(Keep.both)
|
||||
.run()
|
||||
|
||||
|
|
@ -98,6 +104,7 @@ class RateTransformationDocSpec extends AkkaSpec {
|
|||
sub.requestNext((1.0, 2))
|
||||
|
||||
pub.sendNext(2.0)
|
||||
Await.ready(latch, 1.second)
|
||||
sub.requestNext((2.0, 0))
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -139,7 +139,7 @@ class ReactiveStreamsDocSpec extends AkkaSpec {
|
|||
// An example Processor factory
|
||||
def createProcessor: Processor[Int, Int] = Flow[Int].toProcessor.run()
|
||||
|
||||
val flow: Flow[Int, Int, Unit] = Flow(() => createProcessor)
|
||||
val flow: Flow[Int, Int, Unit] = Flow.fromProcessor(() => createProcessor)
|
||||
//#use-processor
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -27,9 +27,15 @@ class RecipeByteStrings extends RecipeSpec {
|
|||
|
||||
override def onPull(ctx: Context[ByteString]): SyncDirective = emitChunkOrPull(ctx)
|
||||
|
||||
override def onUpstreamFinish(ctx: Context[ByteString]): TerminationDirective =
|
||||
if (buffer.nonEmpty) ctx.absorbTermination()
|
||||
else ctx.finish()
|
||||
|
||||
private def emitChunkOrPull(ctx: Context[ByteString]): SyncDirective = {
|
||||
if (buffer.isEmpty) ctx.pull()
|
||||
else {
|
||||
if (buffer.isEmpty) {
|
||||
if (ctx.isFinishing) ctx.finish()
|
||||
else ctx.pull()
|
||||
} else {
|
||||
val (emit, nextBuffer) = buffer.splitAt(chunkSize)
|
||||
buffer = nextBuffer
|
||||
ctx.push(emit)
|
||||
|
|
|
|||
|
|
@ -41,6 +41,8 @@ class RecipeKeepAlive extends RecipeSpec {
|
|||
|
||||
val subscription = sub.expectSubscription()
|
||||
|
||||
// FIXME RK: remove (because I think this cannot deterministically be tested and it might also not do what it should anymore)
|
||||
|
||||
tickPub.sendNext(())
|
||||
|
||||
// pending data will overcome the keepalive
|
||||
|
|
|
|||
|
|
@ -2,8 +2,9 @@ package docs.stream.cookbook
|
|||
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.testkit._
|
||||
|
||||
import scala.concurrent.duration._
|
||||
import akka.testkit.TestLatch
|
||||
import scala.concurrent.Await
|
||||
|
||||
class RecipeMissedTicks extends RecipeSpec {
|
||||
|
||||
|
|
@ -22,8 +23,12 @@ class RecipeMissedTicks extends RecipeSpec {
|
|||
Flow[Tick].conflate(seed = (_) => 0)(
|
||||
(missedTicks, tick) => missedTicks + 1)
|
||||
//#missed-ticks
|
||||
val latch = TestLatch(3)
|
||||
val realMissedTicks: Flow[Tick, Int, Unit] =
|
||||
Flow[Tick].conflate(seed = (_) => 0)(
|
||||
(missedTicks, tick) => { latch.countDown(); missedTicks + 1 })
|
||||
|
||||
tickStream.via(missedTicks).to(sink).run()
|
||||
tickStream.via(realMissedTicks).to(sink).run()
|
||||
|
||||
pub.sendNext(())
|
||||
pub.sendNext(())
|
||||
|
|
@ -31,6 +36,8 @@ class RecipeMissedTicks extends RecipeSpec {
|
|||
pub.sendNext(())
|
||||
|
||||
val subscription = sub.expectSubscription()
|
||||
Await.ready(latch, 1.second)
|
||||
|
||||
subscription.request(1)
|
||||
sub.expectNext(3)
|
||||
|
||||
|
|
|
|||
|
|
@ -2,8 +2,9 @@ package docs.stream.cookbook
|
|||
|
||||
import akka.stream.scaladsl.{ Flow, Sink, Source }
|
||||
import akka.stream.testkit._
|
||||
|
||||
import scala.concurrent.duration._
|
||||
import akka.testkit.TestLatch
|
||||
import scala.concurrent.Await
|
||||
|
||||
class RecipeSimpleDrop extends RecipeSpec {
|
||||
|
||||
|
|
@ -15,13 +16,16 @@ class RecipeSimpleDrop extends RecipeSpec {
|
|||
val droppyStream: Flow[Message, Message, Unit] =
|
||||
Flow[Message].conflate(seed = identity)((lastMessage, newMessage) => newMessage)
|
||||
//#simple-drop
|
||||
val latch = TestLatch(2)
|
||||
val realDroppyStream =
|
||||
Flow[Message].conflate(seed = identity)((lastMessage, newMessage) => { latch.countDown(); newMessage })
|
||||
|
||||
val pub = TestPublisher.probe[Message]()
|
||||
val sub = TestSubscriber.manualProbe[Message]()
|
||||
val messageSource = Source(pub)
|
||||
val sink = Sink(sub)
|
||||
|
||||
messageSource.via(droppyStream).to(sink).run()
|
||||
messageSource.via(realDroppyStream).to(sink).run()
|
||||
|
||||
val subscription = sub.expectSubscription()
|
||||
sub.expectNoMsg(100.millis)
|
||||
|
|
@ -30,6 +34,8 @@ class RecipeSimpleDrop extends RecipeSpec {
|
|||
pub.sendNext("2")
|
||||
pub.sendNext("3")
|
||||
|
||||
Await.ready(latch, 1.second)
|
||||
|
||||
subscription.request(1)
|
||||
sub.expectNext("3")
|
||||
|
||||
|
|
|
|||
1
akka-docs-dev/src/test/resources/application.conf
Normal file
1
akka-docs-dev/src/test/resources/application.conf
Normal file
|
|
@ -0,0 +1 @@
|
|||
akka.loggers = ["akka.testkit.TestEventListener"]
|
||||
|
|
@ -81,7 +81,7 @@ public interface HttpEntity {
|
|||
/**
|
||||
* Returns a stream of data bytes this entity consists of.
|
||||
*/
|
||||
public abstract Source<ByteString, ?> getDataBytes();
|
||||
public abstract Source<ByteString, Object> getDataBytes();
|
||||
|
||||
/**
|
||||
* Returns a future of a strict entity that contains the same data as this entity
|
||||
|
|
|
|||
|
|
@ -117,7 +117,7 @@ private[http] object OutgoingConnectionBlueprint {
|
|||
|
||||
val shape = new FanInShape2(requests, responses, out)
|
||||
|
||||
override def createLogic = new GraphStageLogic(shape) {
|
||||
override def createLogic(effectiveAttributes: Attributes) = new GraphStageLogic(shape) {
|
||||
passAlong(requests, out, doFinish = false, doFail = true)
|
||||
setHandler(out, eagerTerminateOutput)
|
||||
|
||||
|
|
@ -147,7 +147,7 @@ private[http] object OutgoingConnectionBlueprint {
|
|||
|
||||
val shape = new FanInShape2(dataInput, methodBypassInput, out)
|
||||
|
||||
override def createLogic = new GraphStageLogic(shape) {
|
||||
override def createLogic(effectiveAttributes: Attributes) = new GraphStageLogic(shape) {
|
||||
// each connection uses a single (private) response parser instance for all its responses
|
||||
// which builds a cache of all header instances seen on that connection
|
||||
val parser = rootParser.createShallowCopy()
|
||||
|
|
|
|||
|
|
@ -115,7 +115,7 @@ private object PoolConductor {
|
|||
|
||||
override val shape = new FanInShape2(ctxIn, slotIn, out)
|
||||
|
||||
override def createLogic = new GraphStageLogic(shape) {
|
||||
override def createLogic(effectiveAttributes: Attributes) = new GraphStageLogic(shape) {
|
||||
val slotStates = Array.fill[SlotState](slotCount)(Unconnected)
|
||||
var nextSlot = 0
|
||||
|
||||
|
|
@ -207,7 +207,7 @@ private object PoolConductor {
|
|||
|
||||
override val shape = new UniformFanOutShape[SwitchCommand, RequestContext](slotCount)
|
||||
|
||||
override def createLogic = new GraphStageLogic(shape) {
|
||||
override def createLogic(effectiveAttributes: Attributes) = new GraphStageLogic(shape) {
|
||||
shape.outArray foreach { setHandler(_, ignoreTerminateOutput) }
|
||||
|
||||
val in = shape.in
|
||||
|
|
|
|||
|
|
@ -57,10 +57,10 @@ private object PoolSlot {
|
|||
import FlowGraph.Implicits._
|
||||
|
||||
val slotProcessor = b.add {
|
||||
Flow[RequestContext].andThenMat { () ⇒
|
||||
Flow.fromProcessor { () ⇒
|
||||
val actor = system.actorOf(Props(new SlotProcessor(slotIx, connectionFlow, settings)).withDeploy(Deploy.local),
|
||||
slotProcessorActorName.next())
|
||||
(ActorProcessor[RequestContext, List[ProcessorOut]](actor), ())
|
||||
ActorProcessor[RequestContext, List[ProcessorOut]](actor)
|
||||
}.mapConcat(identity)
|
||||
}
|
||||
val split = b.add(Broadcast[ProcessorOut](2))
|
||||
|
|
|
|||
|
|
@ -179,7 +179,7 @@ private[http] object HttpServerBluePrint {
|
|||
|
||||
override val shape = new FanInShape3(bypassInput, oneHundredContinue, applicationInput, out)
|
||||
|
||||
override def createLogic = new GraphStageLogic(shape) {
|
||||
override def createLogic(effectiveAttributes: Attributes) = new GraphStageLogic(shape) {
|
||||
var requestStart: RequestStart = _
|
||||
|
||||
setHandler(bypassInput, new InHandler {
|
||||
|
|
@ -334,7 +334,7 @@ private[http] object HttpServerBluePrint {
|
|||
|
||||
override val shape = new FanOutShape2(in, httpOut, wsOut)
|
||||
|
||||
override def createLogic = new GraphStageLogic(shape) {
|
||||
override def createLogic(effectiveAttributes: Attributes) = new GraphStageLogic(shape) {
|
||||
var target = httpOut
|
||||
|
||||
setHandler(in, new InHandler {
|
||||
|
|
@ -362,7 +362,7 @@ private[http] object HttpServerBluePrint {
|
|||
|
||||
override val shape = new FanInShape2(httpIn, wsIn, out)
|
||||
|
||||
override def createLogic = new GraphStageLogic(shape) {
|
||||
override def createLogic(effectiveAttributes: Attributes) = new GraphStageLogic(shape) {
|
||||
var websocketHandlerWasInstalled = false
|
||||
|
||||
setHandler(httpIn, conditionalTerminateInput(() ⇒ !websocketHandlerWasInstalled))
|
||||
|
|
@ -407,7 +407,7 @@ private[http] object HttpServerBluePrint {
|
|||
|
||||
override val shape = new FanInShape2(bytes, token, out)
|
||||
|
||||
override def createLogic = new GraphStageLogic(shape) {
|
||||
override def createLogic(effectiveAttributes: Attributes) = new GraphStageLogic(shape) {
|
||||
passAlong(bytes, out, doFinish = true, doFail = true)
|
||||
passAlong(token, out, doFinish = false, doFail = true)
|
||||
setHandler(out, eagerTerminateOutput)
|
||||
|
|
|
|||
|
|
@ -7,12 +7,16 @@ package akka.http.impl.engine.ws
|
|||
import akka.http.impl.engine.ws.Protocol.Opcode
|
||||
import akka.util.ByteString
|
||||
|
||||
private[http] sealed trait FrameEventOrError
|
||||
|
||||
private[http] final case class FrameError(p: ProtocolException) extends FrameEventOrError
|
||||
|
||||
/**
|
||||
* The low-level Websocket framing model.
|
||||
*
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[http] sealed trait FrameEvent {
|
||||
private[http] sealed trait FrameEvent extends FrameEventOrError {
|
||||
def data: ByteString
|
||||
def lastPart: Boolean
|
||||
def withData(data: ByteString): FrameEvent
|
||||
|
|
|
|||
|
|
@ -4,11 +4,10 @@
|
|||
|
||||
package akka.http.impl.engine.ws
|
||||
|
||||
import akka.http.impl.util.{ ByteReader, ByteStringParserStage }
|
||||
import akka.stream.stage.{ StageState, SyncDirective, Context }
|
||||
import akka.util.ByteString
|
||||
|
||||
import scala.annotation.tailrec
|
||||
import akka.stream.io.ByteStringParser
|
||||
import akka.stream.Attributes
|
||||
|
||||
/**
|
||||
* Streaming parser for the Websocket framing protocol as defined in RFC6455
|
||||
|
|
@ -36,108 +35,81 @@ import scala.annotation.tailrec
|
|||
*
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[http] class FrameEventParser extends ByteStringParserStage[FrameEvent] {
|
||||
protected def onTruncation(ctx: Context[FrameEvent]): SyncDirective =
|
||||
ctx.fail(new ProtocolException("Data truncated"))
|
||||
private[http] object FrameEventParser extends ByteStringParser[FrameEvent] {
|
||||
import ByteStringParser._
|
||||
|
||||
def initial: StageState[ByteString, FrameEvent] = ReadFrameHeader
|
||||
override def createLogic(attr: Attributes) = new ParsingLogic {
|
||||
startWith(ReadFrameHeader)
|
||||
|
||||
object ReadFrameHeader extends ByteReadingState {
|
||||
def read(reader: ByteReader, ctx: Context[FrameEvent]): SyncDirective = {
|
||||
import Protocol._
|
||||
|
||||
val flagsAndOp = reader.readByte()
|
||||
val maskAndLength = reader.readByte()
|
||||
|
||||
val flags = flagsAndOp & FLAGS_MASK
|
||||
val op = flagsAndOp & OP_MASK
|
||||
|
||||
val maskBit = (maskAndLength & MASK_MASK) != 0
|
||||
val length7 = maskAndLength & LENGTH_MASK
|
||||
|
||||
val length =
|
||||
length7 match {
|
||||
case 126 ⇒ reader.readShortBE().toLong
|
||||
case 127 ⇒ reader.readLongBE()
|
||||
case x ⇒ x.toLong
|
||||
}
|
||||
|
||||
if (length < 0) ctx.fail(new ProtocolException("Highest bit of 64bit length was set"))
|
||||
|
||||
val mask =
|
||||
if (maskBit) Some(reader.readIntBE())
|
||||
else None
|
||||
|
||||
def isFlagSet(mask: Int): Boolean = (flags & mask) != 0
|
||||
val header =
|
||||
FrameHeader(Opcode.forCode(op.toByte),
|
||||
mask,
|
||||
length,
|
||||
fin = isFlagSet(FIN_MASK),
|
||||
rsv1 = isFlagSet(RSV1_MASK),
|
||||
rsv2 = isFlagSet(RSV2_MASK),
|
||||
rsv3 = isFlagSet(RSV3_MASK))
|
||||
|
||||
val data = reader.remainingData
|
||||
val takeNow = (header.length min Int.MaxValue).toInt
|
||||
val thisFrameData = data.take(takeNow)
|
||||
val remaining = data.drop(takeNow)
|
||||
|
||||
val nextState =
|
||||
if (thisFrameData.length == length) ReadFrameHeader
|
||||
else readData(length - thisFrameData.length)
|
||||
|
||||
pushAndBecomeWithRemaining(FrameStart(header, thisFrameData.compact), nextState, remaining, ctx)
|
||||
}
|
||||
}
|
||||
|
||||
def readData(_remaining: Long): State =
|
||||
new State {
|
||||
var remaining = _remaining
|
||||
def onPush(elem: ByteString, ctx: Context[FrameEvent]): SyncDirective =
|
||||
if (elem.size < remaining) {
|
||||
remaining -= elem.size
|
||||
ctx.push(FrameData(elem, lastPart = false))
|
||||
} else {
|
||||
require(remaining <= Int.MaxValue) // safe because, remaining <= elem.size <= Int.MaxValue
|
||||
val frameData = elem.take(remaining.toInt)
|
||||
val remainingData = elem.drop(remaining.toInt)
|
||||
|
||||
pushAndBecomeWithRemaining(FrameData(frameData.compact, lastPart = true), ReadFrameHeader, remainingData, ctx)
|
||||
}
|
||||
trait Step extends ParseStep[FrameEvent] {
|
||||
override def onTruncation(): Unit = failStage(new ProtocolException("Data truncated"))
|
||||
}
|
||||
|
||||
def becomeWithRemaining(nextState: State, remainingData: ByteString, ctx: Context[FrameEvent]): SyncDirective = {
|
||||
become(nextState)
|
||||
nextState.onPush(remainingData, ctx)
|
||||
}
|
||||
def pushAndBecomeWithRemaining(elem: FrameEvent, nextState: State, remainingData: ByteString, ctx: Context[FrameEvent]): SyncDirective =
|
||||
if (remainingData.isEmpty) {
|
||||
become(nextState)
|
||||
ctx.push(elem)
|
||||
} else {
|
||||
become(waitForPull(nextState, remainingData))
|
||||
ctx.push(elem)
|
||||
}
|
||||
object ReadFrameHeader extends Step {
|
||||
override def parse(reader: ByteReader): (FrameEvent, Step) = {
|
||||
import Protocol._
|
||||
|
||||
def waitForPull(nextState: State, remainingData: ByteString): State =
|
||||
new State {
|
||||
def onPush(elem: ByteString, ctx: Context[FrameEvent]): SyncDirective =
|
||||
throw new IllegalStateException("Mustn't push in this state")
|
||||
val flagsAndOp = reader.readByte()
|
||||
val maskAndLength = reader.readByte()
|
||||
|
||||
override def onPull(ctx: Context[FrameEvent]): SyncDirective = {
|
||||
become(nextState)
|
||||
nextState.onPush(remainingData, ctx)
|
||||
val flags = flagsAndOp & FLAGS_MASK
|
||||
val op = flagsAndOp & OP_MASK
|
||||
|
||||
val maskBit = (maskAndLength & MASK_MASK) != 0
|
||||
val length7 = maskAndLength & LENGTH_MASK
|
||||
|
||||
val length =
|
||||
length7 match {
|
||||
case 126 ⇒ reader.readShortBE().toLong
|
||||
case 127 ⇒ reader.readLongBE()
|
||||
case x ⇒ x.toLong
|
||||
}
|
||||
|
||||
if (length < 0) throw new ProtocolException("Highest bit of 64bit length was set")
|
||||
|
||||
val mask =
|
||||
if (maskBit) Some(reader.readIntBE())
|
||||
else None
|
||||
|
||||
def isFlagSet(mask: Int): Boolean = (flags & mask) != 0
|
||||
val header =
|
||||
FrameHeader(Opcode.forCode(op.toByte),
|
||||
mask,
|
||||
length,
|
||||
fin = isFlagSet(FIN_MASK),
|
||||
rsv1 = isFlagSet(RSV1_MASK),
|
||||
rsv2 = isFlagSet(RSV2_MASK),
|
||||
rsv3 = isFlagSet(RSV3_MASK))
|
||||
|
||||
val takeNow = (header.length min reader.remainingSize).toInt
|
||||
val thisFrameData = reader.take(takeNow)
|
||||
|
||||
val nextState =
|
||||
if (thisFrameData.length == length) ReadFrameHeader
|
||||
else new ReadData(length - thisFrameData.length)
|
||||
|
||||
(FrameStart(header, thisFrameData.compact), nextState)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
object FrameEventParser {
|
||||
class ReadData(_remaining: Long) extends Step {
|
||||
var remaining = _remaining
|
||||
override def parse(reader: ByteReader): (FrameEvent, Step) =
|
||||
if (reader.remainingSize < remaining) {
|
||||
remaining -= reader.remainingSize
|
||||
(FrameData(reader.takeAll(), lastPart = false), this)
|
||||
} else {
|
||||
(FrameData(reader.take(remaining.toInt), lastPart = true), ReadFrameHeader)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
def mask(bytes: ByteString, _mask: Option[Int]): ByteString =
|
||||
_mask match {
|
||||
case Some(m) ⇒ mask(bytes, m)._1
|
||||
case None ⇒ bytes
|
||||
}
|
||||
|
||||
def mask(bytes: ByteString, mask: Int): (ByteString, Int) = {
|
||||
@tailrec def rec(bytes: Array[Byte], offset: Int, mask: Int): Int =
|
||||
if (offset >= bytes.length) mask
|
||||
|
|
|
|||
|
|
@ -19,10 +19,10 @@ import scala.util.control.NonFatal
|
|||
*/
|
||||
private[http] object FrameHandler {
|
||||
|
||||
def create(server: Boolean): Flow[FrameEvent, Output, Unit] =
|
||||
Flow[FrameEvent].transform(() ⇒ new HandlerStage(server))
|
||||
def create(server: Boolean): Flow[FrameEventOrError, Output, Unit] =
|
||||
Flow[FrameEventOrError].transform(() ⇒ new HandlerStage(server))
|
||||
|
||||
private class HandlerStage(server: Boolean) extends StatefulStage[FrameEvent, Output] {
|
||||
private class HandlerStage(server: Boolean) extends StatefulStage[FrameEventOrError, Output] {
|
||||
type Ctx = Context[Output]
|
||||
def initial: State = Idle
|
||||
|
||||
|
|
@ -79,11 +79,6 @@ private[http] object FrameHandler {
|
|||
}
|
||||
}
|
||||
|
||||
private object Closed extends State {
|
||||
def onPush(elem: FrameEvent, ctx: Ctx): SyncDirective =
|
||||
ctx.pull() // ignore
|
||||
}
|
||||
|
||||
private def becomeAndHandleWith(newState: State, part: FrameEvent)(implicit ctx: Ctx): SyncDirective = {
|
||||
become(newState)
|
||||
current.onPush(part, ctx)
|
||||
|
|
@ -132,7 +127,7 @@ private[http] object FrameHandler {
|
|||
}
|
||||
|
||||
private object CloseAfterPeerClosed extends State {
|
||||
def onPush(elem: FrameEvent, ctx: Context[Output]): SyncDirective =
|
||||
def onPush(elem: FrameEventOrError, ctx: Context[Output]): SyncDirective =
|
||||
elem match {
|
||||
case FrameStart(FrameHeader(Opcode.Close, _, length, _, _, _, _), data) ⇒
|
||||
become(WaitForPeerTcpClose)
|
||||
|
|
@ -141,7 +136,7 @@ private[http] object FrameHandler {
|
|||
}
|
||||
}
|
||||
private object WaitForPeerTcpClose extends State {
|
||||
def onPush(elem: FrameEvent, ctx: Context[Output]): SyncDirective =
|
||||
def onPush(elem: FrameEventOrError, ctx: Context[Output]): SyncDirective =
|
||||
ctx.pull() // ignore
|
||||
}
|
||||
|
||||
|
|
@ -168,10 +163,11 @@ private[http] object FrameHandler {
|
|||
def handleFrameData(data: FrameData)(implicit ctx: Ctx): SyncDirective
|
||||
def handleFrameStart(start: FrameStart)(implicit ctx: Ctx): SyncDirective
|
||||
|
||||
def onPush(part: FrameEvent, ctx: Ctx): SyncDirective =
|
||||
def onPush(part: FrameEventOrError, ctx: Ctx): SyncDirective =
|
||||
part match {
|
||||
case data: FrameData ⇒ handleFrameData(data)(ctx)
|
||||
case start: FrameStart ⇒ handleFrameStart(start)(ctx)
|
||||
case FrameError(ex) ⇒ ctx.fail(ex)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -15,13 +15,19 @@ import akka.stream.stage.{ SyncDirective, Context, StatefulStage }
|
|||
* INTERNAL API
|
||||
*/
|
||||
private[http] object Masking {
|
||||
def apply(serverSide: Boolean, maskRandom: () ⇒ Random): BidiFlow[ /* net in */ FrameEvent, /* app out */ FrameEvent, /* app in */ FrameEvent, /* net out */ FrameEvent, Unit] =
|
||||
def apply(serverSide: Boolean, maskRandom: () ⇒ Random): BidiFlow[ /* net in */ FrameEvent, /* app out */ FrameEventOrError, /* app in */ FrameEvent, /* net out */ FrameEvent, Unit] =
|
||||
BidiFlow.fromFlowsMat(unmaskIf(serverSide), maskIf(!serverSide, maskRandom))(Keep.none)
|
||||
|
||||
def maskIf(condition: Boolean, maskRandom: () ⇒ Random): Flow[FrameEvent, FrameEvent, Unit] =
|
||||
if (condition) Flow[FrameEvent].transform(() ⇒ new Masking(maskRandom())) // new random per materialization
|
||||
if (condition)
|
||||
Flow[FrameEvent]
|
||||
.transform(() ⇒ new Masking(maskRandom())) // new random per materialization
|
||||
.map {
|
||||
case f: FrameEvent ⇒ f
|
||||
case FrameError(ex) ⇒ throw ex
|
||||
}
|
||||
else Flow[FrameEvent]
|
||||
def unmaskIf(condition: Boolean): Flow[FrameEvent, FrameEvent, Unit] =
|
||||
def unmaskIf(condition: Boolean): Flow[FrameEvent, FrameEventOrError, Unit] =
|
||||
if (condition) Flow[FrameEvent].transform(() ⇒ new Unmasking())
|
||||
else Flow[FrameEvent]
|
||||
|
||||
|
|
@ -41,19 +47,25 @@ private[http] object Masking {
|
|||
}
|
||||
|
||||
/** Implements both masking and unmasking which is mostly symmetric (because of XOR) */
|
||||
private abstract class Masker extends StatefulStage[FrameEvent, FrameEvent] {
|
||||
private abstract class Masker extends StatefulStage[FrameEvent, FrameEventOrError] {
|
||||
def extractMask(header: FrameHeader): Int
|
||||
def setNewMask(header: FrameHeader, mask: Int): FrameHeader
|
||||
|
||||
def initial: State = Idle
|
||||
|
||||
object Idle extends State {
|
||||
def onPush(part: FrameEvent, ctx: Context[FrameEvent]): SyncDirective =
|
||||
private object Idle extends State {
|
||||
def onPush(part: FrameEvent, ctx: Context[FrameEventOrError]): SyncDirective =
|
||||
part match {
|
||||
case start @ FrameStart(header, data) ⇒
|
||||
val mask = extractMask(header)
|
||||
become(new Running(mask))
|
||||
current.onPush(start.copy(header = setNewMask(header, mask)), ctx)
|
||||
try {
|
||||
val mask = extractMask(header)
|
||||
become(new Running(mask))
|
||||
current.onPush(start.copy(header = setNewMask(header, mask)), ctx)
|
||||
} catch {
|
||||
case p: ProtocolException ⇒
|
||||
become(Done)
|
||||
ctx.push(FrameError(p))
|
||||
}
|
||||
case _: FrameData ⇒
|
||||
ctx.fail(new IllegalStateException("unexpected FrameData (need FrameStart first)"))
|
||||
}
|
||||
|
|
@ -61,7 +73,7 @@ private[http] object Masking {
|
|||
private class Running(initialMask: Int) extends State {
|
||||
var mask = initialMask
|
||||
|
||||
def onPush(part: FrameEvent, ctx: Context[FrameEvent]): SyncDirective = {
|
||||
def onPush(part: FrameEvent, ctx: Context[FrameEventOrError]): SyncDirective = {
|
||||
if (part.lastPart) become(Idle)
|
||||
|
||||
val (masked, newMask) = FrameEventParser.mask(part.data, mask)
|
||||
|
|
@ -69,5 +81,8 @@ private[http] object Masking {
|
|||
ctx.push(part.withData(data = masked))
|
||||
}
|
||||
}
|
||||
private object Done extends State {
|
||||
def onPush(part: FrameEvent, ctx: Context[FrameEventOrError]): SyncDirective = ctx.pull()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -40,12 +40,12 @@ private[http] object Websocket {
|
|||
/** The lowest layer that implements the binary protocol */
|
||||
def framing: BidiFlow[ByteString, FrameEvent, FrameEvent, ByteString, Unit] =
|
||||
BidiFlow.fromFlowsMat(
|
||||
Flow[ByteString].transform(() ⇒ new FrameEventParser),
|
||||
Flow[ByteString].via(FrameEventParser),
|
||||
Flow[FrameEvent].transform(() ⇒ new FrameEventRenderer))(Keep.none)
|
||||
.named("ws-framing")
|
||||
|
||||
/** The layer that handles masking using the rules defined in the specification */
|
||||
def masking(serverSide: Boolean, maskingRandomFactory: () ⇒ Random): BidiFlow[FrameEvent, FrameEvent, FrameEvent, FrameEvent, Unit] =
|
||||
def masking(serverSide: Boolean, maskingRandomFactory: () ⇒ Random): BidiFlow[FrameEvent, FrameEventOrError, FrameEvent, FrameEvent, Unit] =
|
||||
Masking(serverSide, maskingRandomFactory)
|
||||
.named("ws-masking")
|
||||
|
||||
|
|
@ -55,7 +55,7 @@ private[http] object Websocket {
|
|||
*/
|
||||
def frameHandling(serverSide: Boolean = true,
|
||||
closeTimeout: FiniteDuration,
|
||||
log: LoggingAdapter): BidiFlow[FrameEvent, FrameHandler.Output, FrameOutHandler.Input, FrameStart, Unit] =
|
||||
log: LoggingAdapter): BidiFlow[FrameEventOrError, FrameHandler.Output, FrameOutHandler.Input, FrameStart, Unit] =
|
||||
BidiFlow.fromFlowsMat(
|
||||
FrameHandler.create(server = serverSide),
|
||||
FrameOutHandler.create(serverSide, closeTimeout, log))(Keep.none)
|
||||
|
|
@ -156,7 +156,7 @@ private[http] object Websocket {
|
|||
|
||||
val shape = new FanOutShape2(in, bypass, user)
|
||||
|
||||
def createLogic = new GraphStageLogic(shape) {
|
||||
def createLogic(effectiveAttributes: Attributes) = new GraphStageLogic(shape) {
|
||||
|
||||
setHandler(in, new InHandler {
|
||||
override def onPush(): Unit = {
|
||||
|
|
@ -187,7 +187,7 @@ private[http] object Websocket {
|
|||
|
||||
val shape = new FanInShape3(bypass, user, tick, out)
|
||||
|
||||
def createLogic = new GraphStageLogic(shape) {
|
||||
def createLogic(effectiveAttributes: Attributes) = new GraphStageLogic(shape) {
|
||||
|
||||
passAlong(bypass, out, doFinish = true, doFail = true)
|
||||
passAlong(user, out, doFinish = false, doFail = false)
|
||||
|
|
@ -210,7 +210,7 @@ private[http] object Websocket {
|
|||
|
||||
val shape = new FlowShape(in, out)
|
||||
|
||||
def createLogic = new GraphStageLogic(shape) {
|
||||
def createLogic(effectiveAttributes: Attributes) = new GraphStageLogic(shape) {
|
||||
setHandler(out, new OutHandler {
|
||||
override def onPull(): Unit = pull(in)
|
||||
})
|
||||
|
|
|
|||
|
|
@ -21,17 +21,17 @@ private[akka] class ByteReader(input: ByteString) {
|
|||
|
||||
def currentOffset: Int = off
|
||||
def remainingData: ByteString = input.drop(off)
|
||||
def fromStartToHere: ByteString = input.take(currentOffset)
|
||||
def fromStartToHere: ByteString = input.take(off)
|
||||
|
||||
def readByte(): Int =
|
||||
if (off < input.length) {
|
||||
val x = input(off)
|
||||
off += 1
|
||||
x.toInt & 0xFF
|
||||
x & 0xFF
|
||||
} else throw NeedMoreData
|
||||
def readShortLE(): Int = readByte() | (readByte() << 8)
|
||||
def readIntLE(): Int = readShortLE() | (readShortLE() << 16)
|
||||
def readLongLE(): Long = (readIntBE() & 0xffffffffL) | ((readIntLE() & 0xffffffffL) << 32)
|
||||
def readLongLE(): Long = (readIntLE() & 0xffffffffL) | ((readIntLE() & 0xffffffffL) << 32)
|
||||
|
||||
def readShortBE(): Int = (readByte() << 8) | readByte()
|
||||
def readIntBE(): Int = (readShortBE() << 16) | readShortBE()
|
||||
|
|
|
|||
|
|
@ -128,7 +128,7 @@ package object util {
|
|||
package util {
|
||||
|
||||
import akka.http.scaladsl.model.{ ContentType, HttpEntity }
|
||||
import akka.stream.{ Outlet, Inlet, FlowShape }
|
||||
import akka.stream.{ Attributes, Outlet, Inlet, FlowShape }
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
|
||||
private[http] class ToStrict(timeout: FiniteDuration, contentType: ContentType)
|
||||
|
|
@ -138,7 +138,7 @@ package util {
|
|||
val out = Outlet[HttpEntity.Strict]("out")
|
||||
override val shape = FlowShape(in, out)
|
||||
|
||||
override def createLogic: GraphStageLogic = new GraphStageLogic(shape) {
|
||||
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new TimerGraphStageLogic(shape) {
|
||||
var bytes = ByteString.newBuilder
|
||||
private var emptyStream = false
|
||||
|
||||
|
|
|
|||
|
|
@ -75,7 +75,7 @@ sealed trait HttpEntity extends jm.HttpEntity {
|
|||
def withContentType(contentType: ContentType): HttpEntity
|
||||
|
||||
/** Java API */
|
||||
def getDataBytes: stream.javadsl.Source[ByteString, _] = stream.javadsl.Source.fromGraph(dataBytes)
|
||||
def getDataBytes: stream.javadsl.Source[ByteString, AnyRef] = stream.javadsl.Source.fromGraph(dataBytes.asInstanceOf[Source[ByteString, AnyRef]])
|
||||
|
||||
/** Java API */
|
||||
def getContentLengthOption: japi.Option[JLong] =
|
||||
|
|
@ -147,9 +147,11 @@ object HttpEntity {
|
|||
def apply(contentType: ContentType, data: Source[ByteString, Any]): Chunked =
|
||||
Chunked.fromData(contentType, data)
|
||||
|
||||
def apply(contentType: ContentType, file: File, chunkSize: Int = SynchronousFileSource.DefaultChunkSize): UniversalEntity = {
|
||||
def apply(contentType: ContentType, file: File, chunkSize: Int = -1): UniversalEntity = {
|
||||
val fileLength = file.length
|
||||
if (fileLength > 0) Default(contentType, fileLength, SynchronousFileSource(file, chunkSize))
|
||||
if (fileLength > 0)
|
||||
Default(contentType, fileLength,
|
||||
if (chunkSize > 0) SynchronousFileSource(file, chunkSize) else SynchronousFileSource(file))
|
||||
else empty(contentType)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -194,7 +194,7 @@ object Multipart {
|
|||
* To create an instance with several parts or for multiple files, use
|
||||
* ``FormData(BodyPart.fromFile("field1", ...), BodyPart.fromFile("field2", ...)``
|
||||
*/
|
||||
def fromFile(name: String, contentType: ContentType, file: File, chunkSize: Int = SynchronousFileSource.DefaultChunkSize): FormData =
|
||||
def fromFile(name: String, contentType: ContentType, file: File, chunkSize: Int = -1): FormData =
|
||||
FormData(Source.single(BodyPart.fromFile(name, contentType, file, chunkSize)))
|
||||
|
||||
/**
|
||||
|
|
@ -237,7 +237,7 @@ object Multipart {
|
|||
/**
|
||||
* Creates a BodyPart backed by a File that will be streamed using a SynchronousFileSource.
|
||||
*/
|
||||
def fromFile(name: String, contentType: ContentType, file: File, chunkSize: Int = SynchronousFileSource.DefaultChunkSize): BodyPart =
|
||||
def fromFile(name: String, contentType: ContentType, file: File, chunkSize: Int = -1): BodyPart =
|
||||
BodyPart(name, HttpEntity(contentType, file, chunkSize), Map("filename" -> file.getName))
|
||||
|
||||
def unapply(value: BodyPart): Option[(String, BodyPartEntity, Map[String, String], immutable.Seq[HttpHeader])] =
|
||||
|
|
|
|||
|
|
@ -1,4 +1,5 @@
|
|||
akka {
|
||||
loggers = ["akka.testkit.TestEventListener"]
|
||||
actor {
|
||||
serialize-creators = on
|
||||
serialize-messages = on
|
||||
|
|
|
|||
|
|
@ -57,8 +57,8 @@ class ConnectionPoolSpec extends AkkaSpec("""
|
|||
|
||||
requestIn.sendNext(HttpRequest(uri = "/") -> 42)
|
||||
|
||||
acceptIncomingConnection()
|
||||
responseOutSub.request(1)
|
||||
acceptIncomingConnection()
|
||||
val (Success(response), 42) = responseOut.expectNext()
|
||||
response.headers should contain(RawHeader("Req-Host", s"$serverHostName:$serverPort"))
|
||||
}
|
||||
|
|
@ -116,8 +116,8 @@ class ConnectionPoolSpec extends AkkaSpec("""
|
|||
val (requestIn, responseOut, responseOutSub, hcp) = cachedHostConnectionPool[Int]()
|
||||
|
||||
requestIn.sendNext(HttpRequest(uri = "/a") -> 42)
|
||||
acceptIncomingConnection()
|
||||
responseOutSub.request(1)
|
||||
acceptIncomingConnection()
|
||||
val (Success(response1), 42) = responseOut.expectNext()
|
||||
connNr(response1) shouldEqual 1
|
||||
|
||||
|
|
@ -222,8 +222,8 @@ class ConnectionPoolSpec extends AkkaSpec("""
|
|||
|
||||
requestIn.sendNext(HttpRequest(uri = "/") -> 42)
|
||||
|
||||
acceptIncomingConnection()
|
||||
responseOutSub.request(1)
|
||||
acceptIncomingConnection()
|
||||
val (Success(_), 42) = responseOut.expectNext()
|
||||
}
|
||||
}
|
||||
|
|
@ -346,7 +346,7 @@ class ConnectionPoolSpec extends AkkaSpec("""
|
|||
def flowTestBench[T, Mat](poolFlow: Flow[(HttpRequest, T), (Try[HttpResponse], T), Mat]) = {
|
||||
val requestIn = TestPublisher.probe[(HttpRequest, T)]()
|
||||
val responseOut = TestSubscriber.manualProbe[(Try[HttpResponse], T)]
|
||||
val hcp = Source(requestIn).viaMat(poolFlow)(Keep.right).toMat(Sink(responseOut))(Keep.left).run()
|
||||
val hcp = Source(requestIn).viaMat(poolFlow)(Keep.right).to(Sink(responseOut)).run()
|
||||
val responseOutSub = responseOut.expectSubscription()
|
||||
(requestIn, responseOut, responseOutSub, hcp)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -62,7 +62,7 @@ class LowLevelOutgoingConnectionSpec extends AkkaSpec("akka.loggers = []\n akka.
|
|||
|
|
||||
|""")
|
||||
val sub = probe.expectSubscription()
|
||||
sub.expectRequest(4)
|
||||
sub.expectRequest()
|
||||
sub.sendNext(ByteString("ABC"))
|
||||
expectWireData("ABC")
|
||||
sub.sendNext(ByteString("DEF"))
|
||||
|
|
@ -228,7 +228,7 @@ class LowLevelOutgoingConnectionSpec extends AkkaSpec("akka.loggers = []\n akka.
|
|||
|
|
||||
|""")
|
||||
val sub = probe.expectSubscription()
|
||||
sub.expectRequest(4)
|
||||
sub.expectRequest()
|
||||
sub.sendNext(ByteString("ABC"))
|
||||
expectWireData("ABC")
|
||||
sub.sendNext(ByteString("DEF"))
|
||||
|
|
@ -254,7 +254,7 @@ class LowLevelOutgoingConnectionSpec extends AkkaSpec("akka.loggers = []\n akka.
|
|||
|
|
||||
|""")
|
||||
val sub = probe.expectSubscription()
|
||||
sub.expectRequest(4)
|
||||
sub.expectRequest()
|
||||
sub.sendNext(ByteString("ABC"))
|
||||
expectWireData("ABC")
|
||||
sub.sendNext(ByteString("DEF"))
|
||||
|
|
|
|||
|
|
@ -6,20 +6,18 @@ package akka.http.impl.engine.client
|
|||
|
||||
import org.scalatest.concurrent.PatienceConfiguration.Timeout
|
||||
import org.scalatest.concurrent.ScalaFutures
|
||||
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.stream.io._
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.testkit.AkkaSpec
|
||||
|
||||
import akka.http.impl.util._
|
||||
|
||||
import akka.http.scaladsl.{ HttpsContext, Http }
|
||||
import akka.http.scaladsl.model.{ StatusCodes, HttpResponse, HttpRequest }
|
||||
import akka.http.scaladsl.model.headers.Host
|
||||
import org.scalatest.time.{ Span, Seconds }
|
||||
|
||||
import scala.concurrent.Future
|
||||
import akka.testkit.EventFilter
|
||||
import javax.net.ssl.SSLException
|
||||
|
||||
class TlsEndpointVerificationSpec extends AkkaSpec("""
|
||||
#akka.loggers = []
|
||||
|
|
@ -30,7 +28,7 @@ class TlsEndpointVerificationSpec extends AkkaSpec("""
|
|||
val timeout = Timeout(Span(3, Seconds))
|
||||
|
||||
"The client implementation" should {
|
||||
"not accept certificates signed by unknown CA" in {
|
||||
"not accept certificates signed by unknown CA" in EventFilter[SSLException](occurrences = 1).intercept {
|
||||
val pipe = pipeline(Http().defaultClientHttpsContext, hostname = "akka.example.org") // default context doesn't include custom CA
|
||||
|
||||
whenReady(pipe(HttpRequest(uri = "https://akka.example.org/")).failed, timeout) { e ⇒
|
||||
|
|
|
|||
|
|
@ -307,13 +307,12 @@ class FramingSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
|||
}
|
||||
|
||||
private def parseToEvents(bytes: Seq[ByteString]): immutable.Seq[FrameEvent] =
|
||||
Source(bytes.toVector).transform(newParser).runFold(Vector.empty[FrameEvent])(_ :+ _)
|
||||
Source(bytes.toVector).via(FrameEventParser).runFold(Vector.empty[FrameEvent])(_ :+ _)
|
||||
.awaitResult(1.second)
|
||||
private def renderToByteString(events: immutable.Seq[FrameEvent]): ByteString =
|
||||
Source(events).transform(newRenderer).runFold(ByteString.empty)(_ ++ _)
|
||||
.awaitResult(1.second)
|
||||
|
||||
protected def newParser(): Stage[ByteString, FrameEvent] = new FrameEventParser
|
||||
protected def newRenderer(): Stage[FrameEvent, ByteString] = new FrameEventRenderer
|
||||
|
||||
import scala.language.implicitConversions
|
||||
|
|
|
|||
|
|
@ -13,6 +13,7 @@ import akka.stream.testkit._
|
|||
import akka.util.ByteString
|
||||
import akka.http.scaladsl.model.ws._
|
||||
import Protocol.Opcode
|
||||
import akka.testkit.EventFilter
|
||||
|
||||
class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
||||
import WSTestUtils._
|
||||
|
|
@ -595,15 +596,18 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
|||
netIn.expectCancellation()
|
||||
}
|
||||
"if user handler fails" in new ServerTestSetup {
|
||||
messageOut.sendError(new RuntimeException("Oops, user handler failed!"))
|
||||
expectCloseCodeOnNetwork(Protocol.CloseCodes.UnexpectedCondition)
|
||||
EventFilter[RuntimeException](message = "Oops, user handler failed!", occurrences = 1)
|
||||
.intercept {
|
||||
messageOut.sendError(new RuntimeException("Oops, user handler failed!"))
|
||||
expectCloseCodeOnNetwork(Protocol.CloseCodes.UnexpectedCondition)
|
||||
|
||||
expectNoNetworkData() // wait for peer to close regularly
|
||||
pushInput(closeFrame(Protocol.CloseCodes.Regular, mask = true))
|
||||
expectNoNetworkData() // wait for peer to close regularly
|
||||
pushInput(closeFrame(Protocol.CloseCodes.Regular, mask = true))
|
||||
|
||||
expectComplete(messageIn)
|
||||
netOut.expectComplete()
|
||||
netIn.expectCancellation()
|
||||
expectComplete(messageIn)
|
||||
netOut.expectComplete()
|
||||
netIn.expectCancellation()
|
||||
}
|
||||
}
|
||||
"if peer closes with invalid close frame" - {
|
||||
"close code outside of the valid range" in new ServerTestSetup {
|
||||
|
|
@ -828,7 +832,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
|||
|
||||
Source(netIn)
|
||||
.via(printEvent("netIn"))
|
||||
.transform(() ⇒ new FrameEventParser)
|
||||
.via(FrameEventParser)
|
||||
.via(Websocket.stack(serverSide, maskingRandomFactory = Randoms.SecureRandomInstances, closeTimeout = closeTimeout, log = system.log).join(messageHandler))
|
||||
.via(printEvent("frameRendererIn"))
|
||||
.transform(() ⇒ new FrameEventRenderer)
|
||||
|
|
|
|||
|
|
@ -24,8 +24,8 @@ import akka.http.scaladsl.model.HttpMethods._
|
|||
import akka.http.scaladsl.model._
|
||||
import akka.http.scaladsl.model.headers._
|
||||
import akka.http.impl.util._
|
||||
|
||||
import scala.util.{ Failure, Try, Success }
|
||||
import java.net.BindException
|
||||
|
||||
class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
|
||||
val testConf: Config = ConfigFactory.parseString("""
|
||||
|
|
@ -56,7 +56,7 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
|
|||
sub.cancel()
|
||||
}
|
||||
|
||||
"report failure if bind fails" in {
|
||||
"report failure if bind fails" in EventFilter[BindException](occurrences = 2).intercept {
|
||||
val (_, hostname, port) = TestUtils.temporaryServerHostnameAndPort()
|
||||
val binding = Http().bind(hostname, port)
|
||||
val probe1 = TestSubscriber.manualProbe[Http.IncomingConnection]()
|
||||
|
|
@ -170,7 +170,8 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
|
|||
// waiting for the timeout to happen on the client
|
||||
intercept[StreamTcpException] { Await.result(clientsResponseFuture, 2.second) }
|
||||
|
||||
(System.nanoTime() - serverReceivedRequestAtNanos).millis should be >= theIdleTimeout
|
||||
val fudge = 100.millis
|
||||
((System.nanoTime() - serverReceivedRequestAtNanos).nanos + fudge) should be >= theIdleTimeout
|
||||
}
|
||||
|
||||
"log materialization errors in `bindAndHandle`" which {
|
||||
|
|
|
|||
|
|
@ -1,4 +1,5 @@
|
|||
akka {
|
||||
loggers = ["akka.testkit.TestEventListener"]
|
||||
actor {
|
||||
serialize-creators = on
|
||||
serialize-messages = on
|
||||
|
|
|
|||
|
|
@ -7,6 +7,7 @@ package akka.http.scaladsl.server
|
|||
import akka.http.scaladsl.model
|
||||
import model.HttpMethods._
|
||||
import model.StatusCodes
|
||||
import akka.testkit.EventFilter
|
||||
|
||||
class BasicRouteSpecs extends RoutingSpec {
|
||||
|
||||
|
|
@ -134,7 +135,7 @@ class BasicRouteSpecs extends RoutingSpec {
|
|||
|
||||
case object MyException extends RuntimeException
|
||||
"Route sealing" should {
|
||||
"catch route execution exceptions" in {
|
||||
"catch route execution exceptions" in EventFilter[MyException.type](occurrences = 1).intercept {
|
||||
Get("/abc") ~> Route.seal {
|
||||
get { ctx ⇒
|
||||
throw MyException
|
||||
|
|
@ -143,7 +144,7 @@ class BasicRouteSpecs extends RoutingSpec {
|
|||
status shouldEqual StatusCodes.InternalServerError
|
||||
}
|
||||
}
|
||||
"catch route building exceptions" in {
|
||||
"catch route building exceptions" in EventFilter[MyException.type](occurrences = 1).intercept {
|
||||
Get("/abc") ~> Route.seal {
|
||||
get {
|
||||
throw MyException
|
||||
|
|
@ -152,7 +153,7 @@ class BasicRouteSpecs extends RoutingSpec {
|
|||
status shouldEqual StatusCodes.InternalServerError
|
||||
}
|
||||
}
|
||||
"convert all rejections to responses" in {
|
||||
"convert all rejections to responses" in EventFilter[RuntimeException](occurrences = 1).intercept {
|
||||
object MyRejection extends Rejection
|
||||
Get("/abc") ~> Route.seal {
|
||||
get {
|
||||
|
|
|
|||
|
|
@ -7,8 +7,8 @@ package directives
|
|||
|
||||
import akka.http.scaladsl.model.{ MediaTypes, MediaRanges, StatusCodes }
|
||||
import akka.http.scaladsl.model.headers._
|
||||
|
||||
import scala.concurrent.Future
|
||||
import akka.testkit.EventFilter
|
||||
|
||||
class ExecutionDirectivesSpec extends RoutingSpec {
|
||||
object MyException extends RuntimeException
|
||||
|
|
@ -51,7 +51,7 @@ class ExecutionDirectivesSpec extends RoutingSpec {
|
|||
}
|
||||
}
|
||||
}
|
||||
"not interfere with alternative routes" in {
|
||||
"not interfere with alternative routes" in EventFilter[MyException.type](occurrences = 1).intercept {
|
||||
Get("/abc") ~>
|
||||
get {
|
||||
handleExceptions(handler)(reject) ~ { ctx ⇒
|
||||
|
|
@ -62,22 +62,22 @@ class ExecutionDirectivesSpec extends RoutingSpec {
|
|||
responseAs[String] shouldEqual "There was an internal server error."
|
||||
}
|
||||
}
|
||||
"not handle other exceptions" in {
|
||||
"not handle other exceptions" in EventFilter[RuntimeException](occurrences = 1, message = "buh").intercept {
|
||||
Get("/abc") ~>
|
||||
get {
|
||||
handleExceptions(handler) {
|
||||
throw new RuntimeException
|
||||
throw new RuntimeException("buh")
|
||||
}
|
||||
} ~> check {
|
||||
status shouldEqual StatusCodes.InternalServerError
|
||||
responseAs[String] shouldEqual "There was an internal server error."
|
||||
}
|
||||
}
|
||||
"always fall back to a default content type" in {
|
||||
"always fall back to a default content type" in EventFilter[RuntimeException](occurrences = 2, message = "buh2").intercept {
|
||||
Get("/abc") ~> Accept(MediaTypes.`application/json`) ~>
|
||||
get {
|
||||
handleExceptions(handler) {
|
||||
throw new RuntimeException
|
||||
throw new RuntimeException("buh2")
|
||||
}
|
||||
} ~> check {
|
||||
status shouldEqual StatusCodes.InternalServerError
|
||||
|
|
@ -87,7 +87,7 @@ class ExecutionDirectivesSpec extends RoutingSpec {
|
|||
Get("/abc") ~> Accept(MediaTypes.`text/xml`, MediaRanges.`*/*`.withQValue(0f)) ~>
|
||||
get {
|
||||
handleExceptions(handler) {
|
||||
throw new RuntimeException
|
||||
throw new RuntimeException("buh2")
|
||||
}
|
||||
} ~> check {
|
||||
status shouldEqual StatusCodes.InternalServerError
|
||||
|
|
|
|||
|
|
@ -6,8 +6,8 @@ package akka.http.scaladsl.server
|
|||
package directives
|
||||
|
||||
import akka.http.scaladsl.model.StatusCodes
|
||||
|
||||
import scala.concurrent.Future
|
||||
import akka.testkit.EventFilter
|
||||
|
||||
class FutureDirectivesSpec extends RoutingSpec {
|
||||
|
||||
|
|
@ -56,7 +56,7 @@ class FutureDirectivesSpec extends RoutingSpec {
|
|||
responseAs[String] shouldEqual "yes"
|
||||
}
|
||||
}
|
||||
"propagate the exception in the failure case" in {
|
||||
"propagate the exception in the failure case" in EventFilter[Exception](occurrences = 1, message = "XXX").intercept {
|
||||
Get() ~> onSuccess(Future.failed(TestException)) { echoComplete } ~> check {
|
||||
status shouldEqual StatusCodes.InternalServerError
|
||||
}
|
||||
|
|
@ -67,7 +67,7 @@ class FutureDirectivesSpec extends RoutingSpec {
|
|||
responseAs[String] shouldEqual s"Oops. akka.http.scaladsl.server.directives.FutureDirectivesSpec$$TestException: EX when ok"
|
||||
}
|
||||
}
|
||||
"catch an exception in the failure case" in {
|
||||
"catch an exception in the failure case" in EventFilter[Exception](occurrences = 1, message = "XXX").intercept {
|
||||
Get() ~> onSuccess(Future.failed(TestException)) { throwTestException("EX when ") } ~> check {
|
||||
status shouldEqual StatusCodes.InternalServerError
|
||||
responseAs[String] shouldEqual "There was an internal server error."
|
||||
|
|
|
|||
|
|
@ -8,7 +8,6 @@ import akka.http.scaladsl.marshallers.sprayjson.SprayJsonSupport
|
|||
import akka.http.scaladsl.marshallers.xml.ScalaXmlSupport
|
||||
import akka.stream.scaladsl.Sink
|
||||
import org.scalatest.FreeSpec
|
||||
|
||||
import scala.concurrent.{ Future, Promise }
|
||||
import akka.http.scaladsl.marshallers.xml.ScalaXmlSupport._
|
||||
import akka.http.scaladsl.marshalling._
|
||||
|
|
@ -18,8 +17,8 @@ import akka.http.impl.util._
|
|||
import headers._
|
||||
import StatusCodes._
|
||||
import MediaTypes._
|
||||
|
||||
import scala.xml.NodeSeq
|
||||
import akka.testkit.EventFilter
|
||||
|
||||
class RouteDirectivesSpec extends FreeSpec with GenericRoutingSpec {
|
||||
|
||||
|
|
@ -47,7 +46,7 @@ class RouteDirectivesSpec extends FreeSpec with GenericRoutingSpec {
|
|||
"for successful futures and marshalling" in {
|
||||
Get() ~> complete(Promise.successful("yes").future) ~> check { responseAs[String] shouldEqual "yes" }
|
||||
}
|
||||
"for failed futures and marshalling" in {
|
||||
"for failed futures and marshalling" in EventFilter[RuntimeException](occurrences = 1).intercept {
|
||||
object TestException extends RuntimeException
|
||||
Get() ~> complete(Promise.failed[String](TestException).future) ~>
|
||||
check {
|
||||
|
|
|
|||
|
|
@ -9,6 +9,7 @@ import scala.concurrent.Future
|
|||
import akka.http.scaladsl.model._
|
||||
import akka.http.scaladsl.model.headers._
|
||||
import akka.http.scaladsl.server.AuthenticationFailedRejection.{ CredentialsRejected, CredentialsMissing }
|
||||
import akka.testkit.EventFilter
|
||||
|
||||
class SecurityDirectivesSpec extends RoutingSpec {
|
||||
val dontBasicAuth = authenticateBasicAsync[String]("MyRealm", _ ⇒ Future.successful(None))
|
||||
|
|
@ -60,11 +61,13 @@ class SecurityDirectivesSpec extends RoutingSpec {
|
|||
}
|
||||
"properly handle exceptions thrown in its inner route" in {
|
||||
object TestException extends RuntimeException
|
||||
Get() ~> Authorization(BasicHttpCredentials("Alice", "")) ~> {
|
||||
Route.seal {
|
||||
doBasicAuth { _ ⇒ throw TestException }
|
||||
}
|
||||
} ~> check { status shouldEqual StatusCodes.InternalServerError }
|
||||
EventFilter[TestException.type](occurrences = 1).intercept {
|
||||
Get() ~> Authorization(BasicHttpCredentials("Alice", "")) ~> {
|
||||
Route.seal {
|
||||
doBasicAuth { _ ⇒ throw TestException }
|
||||
}
|
||||
} ~> check { status shouldEqual StatusCodes.InternalServerError }
|
||||
}
|
||||
}
|
||||
}
|
||||
"bearer token authentication" should {
|
||||
|
|
@ -108,11 +111,13 @@ class SecurityDirectivesSpec extends RoutingSpec {
|
|||
}
|
||||
"properly handle exceptions thrown in its inner route" in {
|
||||
object TestException extends RuntimeException
|
||||
Get() ~> Authorization(OAuth2BearerToken("myToken")) ~> {
|
||||
Route.seal {
|
||||
doOAuth2Auth { _ ⇒ throw TestException }
|
||||
}
|
||||
} ~> check { status shouldEqual StatusCodes.InternalServerError }
|
||||
EventFilter[TestException.type](occurrences = 1).intercept {
|
||||
Get() ~> Authorization(OAuth2BearerToken("myToken")) ~> {
|
||||
Route.seal {
|
||||
doOAuth2Auth { _ ⇒ throw TestException }
|
||||
}
|
||||
} ~> check { status shouldEqual StatusCodes.InternalServerError }
|
||||
}
|
||||
}
|
||||
}
|
||||
"authentication directives" should {
|
||||
|
|
|
|||
|
|
@ -3,11 +3,11 @@
|
|||
*/
|
||||
package akka.stream.tck
|
||||
|
||||
import akka.stream.{ ActorMaterializer, ActorMaterializerSettings }
|
||||
import akka.stream.impl.Stages
|
||||
import akka.stream._
|
||||
import akka.stream.impl.Stages.Identity
|
||||
import akka.stream.scaladsl.Flow
|
||||
import org.reactivestreams.Processor
|
||||
import akka.stream.Attributes
|
||||
|
||||
class FusableProcessorTest extends AkkaIdentityProcessorVerification[Int] {
|
||||
|
||||
|
|
@ -18,7 +18,7 @@ class FusableProcessorTest extends AkkaIdentityProcessorVerification[Int] {
|
|||
implicit val materializer = ActorMaterializer(settings)(system)
|
||||
|
||||
// withAttributes "wraps" the underlying identity and protects it from automatic removal
|
||||
Flow[Int].andThen[Int](Identity()).named("identity").toProcessor.run()
|
||||
Flow[Int].via(Stages.identityGraph.asInstanceOf[Graph[FlowShape[Int, Int], Unit]]).named("identity").toProcessor.run()
|
||||
}
|
||||
|
||||
override def createElement(element: Int): Int = element
|
||||
|
|
|
|||
|
|
@ -9,11 +9,12 @@ import akka.stream.impl.StreamLayout.Module
|
|||
import akka.stream.impl._
|
||||
import akka.testkit.TestProbe
|
||||
import org.reactivestreams.{ Publisher, Subscriber, Subscription }
|
||||
|
||||
import scala.annotation.tailrec
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.duration._
|
||||
import scala.language.existentials
|
||||
import java.io.StringWriter
|
||||
import java.io.PrintWriter
|
||||
|
||||
/**
|
||||
* Provides factory methods for various Publishers.
|
||||
|
|
@ -183,7 +184,16 @@ object TestSubscriber {
|
|||
final case class OnSubscribe(subscription: Subscription) extends SubscriberEvent
|
||||
final case class OnNext[I](element: I) extends SubscriberEvent
|
||||
final case object OnComplete extends SubscriberEvent
|
||||
final case class OnError(cause: Throwable) extends SubscriberEvent
|
||||
final case class OnError(cause: Throwable) extends SubscriberEvent {
|
||||
override def toString: String = {
|
||||
val str = new StringWriter
|
||||
val out = new PrintWriter(str)
|
||||
out.print("OnError(")
|
||||
cause.printStackTrace(out)
|
||||
out.print(")")
|
||||
str.toString
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Probe that implements [[org.reactivestreams.Subscriber]] interface.
|
||||
|
|
|
|||
|
|
@ -12,7 +12,7 @@ import org.junit.Test;
|
|||
import akka.stream.Attributes;
|
||||
|
||||
public class AttributesTest {
|
||||
|
||||
|
||||
final Attributes attributes =
|
||||
Attributes.name("a")
|
||||
.and(Attributes.name("b"))
|
||||
|
|
@ -27,12 +27,12 @@ public class AttributesTest {
|
|||
Collections.singletonList(new Attributes.InputBuffer(1, 2)),
|
||||
attributes.getAttributeList(Attributes.InputBuffer.class));
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void mustGetAttributeByClass() {
|
||||
assertEquals(
|
||||
new Attributes.Name("a"),
|
||||
new Attributes.Name("b"),
|
||||
attributes.getAttribute(Attributes.Name.class, new Attributes.Name("default")));
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -16,6 +16,7 @@ import akka.stream.testkit.AkkaSpec;
|
|||
import akka.stream.testkit.TestPublisher;
|
||||
import akka.testkit.JavaTestKit;
|
||||
import org.junit.ClassRule;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
import org.reactivestreams.Publisher;
|
||||
import scala.concurrent.Await;
|
||||
|
|
@ -179,7 +180,7 @@ public class FlowTest extends StreamTest {
|
|||
}
|
||||
|
||||
|
||||
@Test
|
||||
@Ignore("StatefulStage to be converted to GraphStage when Java Api is available (#18817)") @Test
|
||||
public void mustBeAbleToUseTransform() {
|
||||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
final Iterable<Integer> input = Arrays.asList(0, 1, 2, 3, 4, 5, 6, 7);
|
||||
|
|
@ -204,15 +205,15 @@ public class FlowTest extends StreamTest {
|
|||
return emit(Arrays.asList(element, element).iterator(), ctx);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public TerminationDirective onUpstreamFinish(Context<Integer> ctx) {
|
||||
return terminationEmit(Collections.singletonList(sum).iterator(), ctx);
|
||||
}
|
||||
|
||||
|
||||
};
|
||||
}
|
||||
});
|
||||
|
|
@ -355,12 +356,12 @@ public class FlowTest extends StreamTest {
|
|||
return new akka.japi.function.Creator<Stage<T, T>>() {
|
||||
@Override
|
||||
public PushPullStage<T, T> create() throws Exception {
|
||||
return new PushPullStage<T, T>() {
|
||||
return new PushPullStage<T, T>() {
|
||||
@Override
|
||||
public SyncDirective onPush(T element, Context<T> ctx) {
|
||||
return ctx.push(element);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public SyncDirective onPull(Context<T> ctx) {
|
||||
return ctx.pull();
|
||||
|
|
@ -374,17 +375,17 @@ public class FlowTest extends StreamTest {
|
|||
public void mustBeAbleToUseMerge() throws Exception {
|
||||
final Flow<String, String, BoxedUnit> f1 =
|
||||
Flow.of(String.class).transform(FlowTest.this.<String> op()).named("f1");
|
||||
final Flow<String, String, BoxedUnit> f2 =
|
||||
final Flow<String, String, BoxedUnit> f2 =
|
||||
Flow.of(String.class).transform(FlowTest.this.<String> op()).named("f2");
|
||||
@SuppressWarnings("unused")
|
||||
final Flow<String, String, BoxedUnit> f3 =
|
||||
final Flow<String, String, BoxedUnit> f3 =
|
||||
Flow.of(String.class).transform(FlowTest.this.<String> op()).named("f3");
|
||||
|
||||
final Source<String, BoxedUnit> in1 = Source.from(Arrays.asList("a", "b", "c"));
|
||||
final Source<String, BoxedUnit> in2 = Source.from(Arrays.asList("d", "e", "f"));
|
||||
|
||||
final Sink<String, Publisher<String>> publisher = Sink.publisher();
|
||||
|
||||
|
||||
final Source<String, BoxedUnit> source = Source.fromGraph(
|
||||
FlowGraph.create(new Function<FlowGraph.Builder<BoxedUnit>, SourceShape<String>>() {
|
||||
@Override
|
||||
|
|
|
|||
|
|
@ -20,6 +20,7 @@ import akka.stream.testkit.AkkaSpec;
|
|||
import akka.stream.testkit.TestPublisher;
|
||||
import akka.testkit.JavaTestKit;
|
||||
import org.junit.ClassRule;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
import scala.concurrent.Await;
|
||||
import scala.concurrent.Future;
|
||||
|
|
@ -107,7 +108,7 @@ public class SourceTest extends StreamTest {
|
|||
probe.expectMsgEquals("()");
|
||||
}
|
||||
|
||||
@Test
|
||||
@Ignore("StatefulStage to be converted to GraphStage when Java Api is available (#18817)") @Test
|
||||
public void mustBeAbleToUseTransform() {
|
||||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
final Iterable<Integer> input = Arrays.asList(0, 1, 2, 3, 4, 5, 6, 7);
|
||||
|
|
@ -415,7 +416,7 @@ public class SourceTest extends StreamTest {
|
|||
@Test
|
||||
public void mustProduceTicks() throws Exception {
|
||||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
Source<String, Cancellable> tickSource = Source.from(FiniteDuration.create(1, TimeUnit.SECONDS),
|
||||
Source<String, Cancellable> tickSource = Source.from(FiniteDuration.create(1, TimeUnit.SECONDS),
|
||||
FiniteDuration.create(500, TimeUnit.MILLISECONDS), "tick");
|
||||
Cancellable cancellable = tickSource.to(Sink.foreach(new Procedure<String>() {
|
||||
public void apply(String elem) {
|
||||
|
|
@ -457,7 +458,7 @@ public class SourceTest extends StreamTest {
|
|||
String result = Await.result(future2, probe.dilated(FiniteDuration.create(3, TimeUnit.SECONDS)));
|
||||
assertEquals("A", result);
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void mustRepeat() throws Exception {
|
||||
final Future<List<Integer>> f = Source.repeat(42).grouped(10000).runWith(Sink.<List<Integer>> head(), materializer);
|
||||
|
|
@ -465,7 +466,7 @@ public class SourceTest extends StreamTest {
|
|||
assertEquals(result.size(), 10000);
|
||||
for (Integer i: result) assertEquals(i, (Integer) 42);
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void mustBeAbleToUseActorRefSource() throws Exception {
|
||||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
|
|
|
|||
|
|
@ -1,4 +1,5 @@
|
|||
akka {
|
||||
loggers = ["akka.testkit.TestEventListener"]
|
||||
actor {
|
||||
serialize-creators = on
|
||||
serialize-messages = on
|
||||
|
|
@ -26,20 +26,16 @@ class DslConsistencySpec extends WordSpec with Matchers {
|
|||
Set("equals", "hashCode", "notify", "notifyAll", "wait", "toString", "getClass") ++
|
||||
Set("productArity", "canEqual", "productPrefix", "copy", "productIterator", "productElement") ++
|
||||
Set("create", "apply", "ops", "appendJava", "andThen", "andThenMat", "isIdentity", "withAttributes", "transformMaterializing") ++
|
||||
Set("asScala", "asJava")
|
||||
Set("asScala", "asJava", "deprecatedAndThen", "deprecatedAndThenMat")
|
||||
|
||||
val allowMissing: Map[Class[_], Set[String]] = Map(
|
||||
sFlowClass -> Set("of"),
|
||||
sSourceClass -> Set("adapt", "from"),
|
||||
sSinkClass -> Set("adapt"),
|
||||
|
||||
// TODO timerTransform is to be removed or replaced. See https://github.com/akka/akka/issues/16393
|
||||
jFlowClass -> Set("timerTransform"),
|
||||
jSourceClass -> Set("timerTransform"),
|
||||
jSinkClass -> Set(),
|
||||
|
||||
sRunnableGraphClass -> Set("builder"),
|
||||
jRunnableGraphClass → Set("graph", "cyclesAllowed"))
|
||||
sRunnableGraphClass -> Set("builder"))
|
||||
|
||||
def materializing(m: Method): Boolean = m.getParameterTypes.contains(classOf[ActorMaterializer])
|
||||
|
||||
|
|
|
|||
|
|
@ -1,120 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.stream.impl
|
||||
|
||||
import akka.stream.Supervision._
|
||||
import akka.stream._
|
||||
import akka.stream.impl.fusing.{ ActorInterpreter, InterpreterLifecycleSpecKit }
|
||||
import akka.stream.stage.Stage
|
||||
import akka.stream.testkit.Utils.TE
|
||||
import akka.stream.testkit.{ AkkaSpec, _ }
|
||||
|
||||
import scala.concurrent.duration._
|
||||
|
||||
class ActorInterpreterLifecycleSpec extends AkkaSpec with InterpreterLifecycleSpecKit {
|
||||
|
||||
implicit val mat = ActorMaterializer()
|
||||
|
||||
class Setup(ops: List[Stage[_, _]] = List(fusing.Map({ x: Any ⇒ x }, stoppingDecider))) {
|
||||
val up = TestPublisher.manualProbe[Int]()
|
||||
val down = TestSubscriber.manualProbe[Int]
|
||||
private val props = ActorInterpreter.props(mat.settings, ops, mat).withDispatcher("akka.test.stream-dispatcher")
|
||||
val actor = system.actorOf(props)
|
||||
val processor = ActorProcessorFactory[Int, Int](actor)
|
||||
}
|
||||
|
||||
"An ActorInterpreter" must {
|
||||
|
||||
"call preStart in order on stages" in new Setup(List(
|
||||
PreStartAndPostStopIdentity(onStart = _ ⇒ testActor ! "start-a"),
|
||||
PreStartAndPostStopIdentity(onStart = _ ⇒ testActor ! "start-b"),
|
||||
PreStartAndPostStopIdentity(onStart = _ ⇒ testActor ! "start-c"))) {
|
||||
processor.subscribe(down)
|
||||
val sub = down.expectSubscription()
|
||||
sub.cancel()
|
||||
up.subscribe(processor)
|
||||
val upsub = up.expectSubscription()
|
||||
upsub.expectCancellation()
|
||||
|
||||
expectMsg("start-a")
|
||||
expectMsg("start-b")
|
||||
expectMsg("start-c")
|
||||
}
|
||||
|
||||
"call postStart in order on stages - when upstream completes" in new Setup(List(
|
||||
PreStartAndPostStopIdentity(onStop = () ⇒ testActor ! "stop-a"),
|
||||
PreStartAndPostStopIdentity(onStop = () ⇒ testActor ! "stop-b"),
|
||||
PreStartAndPostStopIdentity(onStop = () ⇒ testActor ! "stop-c"))) {
|
||||
processor.subscribe(down)
|
||||
val sub = down.expectSubscription()
|
||||
up.subscribe(processor)
|
||||
val upsub = up.expectSubscription()
|
||||
upsub.sendComplete()
|
||||
down.expectComplete()
|
||||
|
||||
expectMsg("stop-a")
|
||||
expectMsg("stop-b")
|
||||
expectMsg("stop-c")
|
||||
}
|
||||
|
||||
"call postStart in order on stages - when downstream cancels" in new Setup(List(
|
||||
PreStartAndPostStopIdentity(onStop = () ⇒ testActor ! "stop-a"),
|
||||
PreStartAndPostStopIdentity(onStop = () ⇒ testActor ! "stop-b"),
|
||||
PreStartAndPostStopIdentity(onStop = () ⇒ testActor ! "stop-c"))) {
|
||||
processor.subscribe(down)
|
||||
val sub = down.expectSubscription()
|
||||
sub.cancel()
|
||||
up.subscribe(processor)
|
||||
val upsub = up.expectSubscription()
|
||||
upsub.expectCancellation()
|
||||
|
||||
expectMsg("stop-c")
|
||||
expectMsg("stop-b")
|
||||
expectMsg("stop-a")
|
||||
}
|
||||
|
||||
"onError downstream when preStart fails" in new Setup(List(
|
||||
PreStartFailer(() ⇒ throw TE("Boom!")))) {
|
||||
processor.subscribe(down)
|
||||
val sub = down.expectSubscription()
|
||||
up.subscribe(processor)
|
||||
val upsub = up.expectSubscription()
|
||||
down.expectError(TE("Boom!"))
|
||||
}
|
||||
|
||||
"onError only once even with Supervision.restart" in new Setup(List(
|
||||
PreStartFailer(() ⇒ throw TE("Boom!")))) {
|
||||
processor.subscribe(down)
|
||||
val sub = down.expectSubscription()
|
||||
up.subscribe(processor)
|
||||
val upsub = up.expectSubscription()
|
||||
down.expectError(TE("Boom!"))
|
||||
down.expectNoMsg(1.second)
|
||||
}
|
||||
|
||||
"onError downstream when preStart fails with 'most downstream' failure, when multiple stages fail" in new Setup(List(
|
||||
PreStartFailer(() ⇒ throw TE("Boom 1!")),
|
||||
PreStartFailer(() ⇒ throw TE("Boom 2!")),
|
||||
PreStartFailer(() ⇒ throw TE("Boom 3!")))) {
|
||||
processor.subscribe(down)
|
||||
val sub = down.expectSubscription()
|
||||
up.subscribe(processor)
|
||||
val upsub = up.expectSubscription()
|
||||
down.expectError(TE("Boom 3!"))
|
||||
down.expectNoMsg(300.millis)
|
||||
}
|
||||
|
||||
"continue with stream shutdown when postStop fails" in new Setup(List(
|
||||
PostStopFailer(() ⇒ throw TE("Boom!")))) {
|
||||
processor.subscribe(down)
|
||||
val sub = down.expectSubscription()
|
||||
up.subscribe(processor)
|
||||
val upsub = up.expectSubscription()
|
||||
upsub.sendComplete()
|
||||
down.expectComplete()
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -1,190 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.stream.impl
|
||||
|
||||
import akka.stream.Supervision._
|
||||
import akka.stream.impl.ReactiveStreamsCompliance.SpecViolation
|
||||
import akka.stream.testkit.AkkaSpec
|
||||
import akka.stream._
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.testkit._
|
||||
import akka.stream.impl.fusing.ActorInterpreter
|
||||
import akka.stream.stage.Stage
|
||||
import akka.stream.stage.PushPullStage
|
||||
import akka.stream.stage.Context
|
||||
import akka.testkit.TestLatch
|
||||
import org.reactivestreams.{ Subscription, Subscriber, Publisher }
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
|
||||
class ActorInterpreterSpec extends AkkaSpec {
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
implicit val mat = ActorMaterializer()
|
||||
|
||||
class Setup(ops: List[Stage[_, _]] = List(fusing.Map({ x: Any ⇒ x }, stoppingDecider))) {
|
||||
val up = TestPublisher.manualProbe[Int]()
|
||||
val down = TestSubscriber.manualProbe[Int]
|
||||
private val props = ActorInterpreter.props(mat.settings, ops, mat).withDispatcher("akka.test.stream-dispatcher")
|
||||
val actor = system.actorOf(props)
|
||||
val processor = ActorProcessorFactory[Int, Int](actor)
|
||||
}
|
||||
|
||||
"An ActorInterpreter" must {
|
||||
|
||||
"pass along early cancellation" in new Setup {
|
||||
processor.subscribe(down)
|
||||
val sub = down.expectSubscription()
|
||||
sub.cancel()
|
||||
up.subscribe(processor)
|
||||
val upsub = up.expectSubscription()
|
||||
upsub.expectCancellation()
|
||||
}
|
||||
|
||||
"heed cancellation signal while large demand is outstanding" in {
|
||||
val latch = TestLatch()
|
||||
val infinite = new PushPullStage[Int, Int] {
|
||||
override def onPush(elem: Int, ctx: Context[Int]) = ???
|
||||
override def onPull(ctx: Context[Int]) = {
|
||||
Await.ready(latch, 5.seconds)
|
||||
ctx.push(42)
|
||||
}
|
||||
}
|
||||
val N = system.settings.config.getInt("akka.stream.materializer.output-burst-limit")
|
||||
|
||||
new Setup(infinite :: Nil) {
|
||||
processor.subscribe(down)
|
||||
val sub = down.expectSubscription()
|
||||
up.subscribe(processor)
|
||||
val upsub = up.expectSubscription()
|
||||
sub.request(100000000)
|
||||
sub.cancel()
|
||||
watch(actor)
|
||||
latch.countDown()
|
||||
for (i ← 1 to N) withClue(s"iteration $i: ") {
|
||||
try down.expectNext(42) catch { case e: Throwable ⇒ fail(e) }
|
||||
}
|
||||
// now cancellation request is processed
|
||||
down.expectNoMsg(500.millis)
|
||||
upsub.expectCancellation()
|
||||
expectTerminated(actor)
|
||||
}
|
||||
}
|
||||
|
||||
"heed upstream failure while large demand is outstanding" in {
|
||||
val latch = TestLatch()
|
||||
val infinite = new PushPullStage[Int, Int] {
|
||||
override def onPush(elem: Int, ctx: Context[Int]) = ???
|
||||
override def onPull(ctx: Context[Int]) = {
|
||||
Await.ready(latch, 5.seconds)
|
||||
ctx.push(42)
|
||||
}
|
||||
}
|
||||
val N = system.settings.config.getInt("akka.stream.materializer.output-burst-limit")
|
||||
|
||||
new Setup(infinite :: Nil) {
|
||||
processor.subscribe(down)
|
||||
val sub = down.expectSubscription()
|
||||
up.subscribe(processor)
|
||||
val upsub = up.expectSubscription()
|
||||
sub.request(100000000)
|
||||
val ex = new Exception("FAIL!")
|
||||
upsub.sendError(ex)
|
||||
latch.countDown()
|
||||
for (i ← 1 to N) withClue(s"iteration $i: ") {
|
||||
try down.expectNext(42) catch { case e: Throwable ⇒ fail(e) }
|
||||
}
|
||||
down.expectError(ex)
|
||||
}
|
||||
}
|
||||
|
||||
"hold back upstream completion while large demand is outstanding" in {
|
||||
val latch = TestLatch()
|
||||
val N = 3 * system.settings.config.getInt("akka.stream.materializer.output-burst-limit")
|
||||
val infinite = new PushPullStage[Int, Int] {
|
||||
private var remaining = N
|
||||
override def onPush(elem: Int, ctx: Context[Int]) = ???
|
||||
override def onPull(ctx: Context[Int]) = {
|
||||
Await.ready(latch, 5.seconds)
|
||||
remaining -= 1
|
||||
if (remaining >= 0) ctx.push(42)
|
||||
else ctx.finish()
|
||||
}
|
||||
override def onUpstreamFinish(ctx: Context[Int]) = {
|
||||
if (remaining > 0) ctx.absorbTermination()
|
||||
else ctx.finish()
|
||||
}
|
||||
}
|
||||
|
||||
new Setup(infinite :: Nil) {
|
||||
processor.subscribe(down)
|
||||
val sub = down.expectSubscription()
|
||||
up.subscribe(processor)
|
||||
val upsub = up.expectSubscription()
|
||||
sub.request(100000000)
|
||||
upsub.sendComplete()
|
||||
latch.countDown()
|
||||
for (i ← 1 to N) withClue(s"iteration $i: ") {
|
||||
try down.expectNext(42) catch { case e: Throwable ⇒ fail(e) }
|
||||
}
|
||||
down.expectComplete()
|
||||
}
|
||||
}
|
||||
|
||||
"satisfy large demand" in largeDemand(0)
|
||||
"satisfy larger demand" in largeDemand(1)
|
||||
|
||||
"handle spec violations" in {
|
||||
a[AbruptTerminationException] should be thrownBy {
|
||||
Await.result(
|
||||
Source(new Publisher[String] {
|
||||
def subscribe(s: Subscriber[_ >: String]) = {
|
||||
s.onSubscribe(new Subscription {
|
||||
def cancel() = ()
|
||||
def request(n: Long) = sys.error("test error")
|
||||
})
|
||||
}
|
||||
}).runFold("")(_ + _),
|
||||
3.seconds)
|
||||
}
|
||||
}
|
||||
|
||||
"handle failed stage factories" in {
|
||||
a[RuntimeException] should be thrownBy
|
||||
Await.result(
|
||||
Source.empty[Int].transform(() ⇒ sys.error("test error")).runWith(Sink.head),
|
||||
3.seconds)
|
||||
}
|
||||
|
||||
def largeDemand(extra: Int): Unit = {
|
||||
val N = 3 * system.settings.config.getInt("akka.stream.materializer.output-burst-limit")
|
||||
val large = new PushPullStage[Int, Int] {
|
||||
private var remaining = N
|
||||
override def onPush(elem: Int, ctx: Context[Int]) = ???
|
||||
override def onPull(ctx: Context[Int]) = {
|
||||
remaining -= 1
|
||||
if (remaining >= 0) ctx.push(42)
|
||||
else ctx.finish()
|
||||
}
|
||||
}
|
||||
|
||||
new Setup(large :: Nil) {
|
||||
processor.subscribe(down)
|
||||
val sub = down.expectSubscription()
|
||||
up.subscribe(processor)
|
||||
val upsub = up.expectSubscription()
|
||||
sub.request(100000000)
|
||||
watch(actor)
|
||||
for (i ← 1 to N) withClue(s"iteration $i: ") {
|
||||
try down.expectNext(42) catch { case e: Throwable ⇒ fail(e) }
|
||||
}
|
||||
down.expectComplete()
|
||||
upsub.expectCancellation()
|
||||
expectTerminated(actor)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,177 @@
|
|||
/**
|
||||
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.stream.impl
|
||||
|
||||
import akka.stream.testkit.AkkaSpec
|
||||
import akka.stream._
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.stage._
|
||||
import akka.stream.testkit.Utils.assertAllStagesStopped
|
||||
import akka.stream.testkit.scaladsl.TestSink
|
||||
import akka.stream.impl.fusing._
|
||||
import akka.stream.impl.fusing.GraphInterpreter._
|
||||
import org.scalactic.ConversionCheckedTripleEquals
|
||||
import scala.concurrent.duration.Duration
|
||||
|
||||
class GraphStageLogicSpec extends GraphInterpreterSpecKit with ConversionCheckedTripleEquals {
|
||||
|
||||
implicit val mat = ActorMaterializer()
|
||||
|
||||
object emit1234 extends GraphStage[FlowShape[Int, Int]] {
|
||||
val in = Inlet[Int]("in")
|
||||
val out = Outlet[Int]("out")
|
||||
override val shape = FlowShape(in, out)
|
||||
override def createLogic(attr: Attributes) = new GraphStageLogic(shape) {
|
||||
setHandler(in, eagerTerminateInput)
|
||||
setHandler(out, eagerTerminateOutput)
|
||||
override def preStart(): Unit = {
|
||||
emit(out, 1, () ⇒ emit(out, 2))
|
||||
emit(out, 3, () ⇒ emit(out, 4))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
object emit5678 extends GraphStage[FlowShape[Int, Int]] {
|
||||
val in = Inlet[Int]("in")
|
||||
val out = Outlet[Int]("out")
|
||||
override val shape = FlowShape(in, out)
|
||||
override def createLogic(attr: Attributes) = new GraphStageLogic(shape) {
|
||||
setHandler(in, new InHandler {
|
||||
override def onPush(): Unit = push(out, grab(in))
|
||||
override def onUpstreamFinish(): Unit = {
|
||||
emit(out, 5, () ⇒ emit(out, 6))
|
||||
emit(out, 7, () ⇒ emit(out, 8))
|
||||
completeStage()
|
||||
}
|
||||
})
|
||||
setHandler(out, new OutHandler {
|
||||
override def onPull(): Unit = pull(in)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
object passThrough extends GraphStage[FlowShape[Int, Int]] {
|
||||
val in = Inlet[Int]("in")
|
||||
val out = Outlet[Int]("out")
|
||||
override val shape = FlowShape(in, out)
|
||||
override def createLogic(attr: Attributes) = new GraphStageLogic(shape) {
|
||||
setHandler(in, new InHandler {
|
||||
override def onPush(): Unit = push(out, grab(in))
|
||||
override def onUpstreamFinish(): Unit = complete(out)
|
||||
})
|
||||
setHandler(out, new OutHandler {
|
||||
override def onPull(): Unit = pull(in)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
class FusedGraph[S <: Shape](ga: GraphAssembly, s: S, a: Attributes = Attributes.none) extends Graph[S, Unit] {
|
||||
override def shape = s
|
||||
override val module = GraphModule(ga, s, a)
|
||||
override def withAttributes(attr: Attributes) = new FusedGraph(ga, s, attr)
|
||||
}
|
||||
|
||||
"A GraphStageLogic" must {
|
||||
|
||||
"emit all things before completing" in assertAllStagesStopped {
|
||||
|
||||
Source.empty.via(emit1234.named("testStage")).runWith(TestSink.probe)
|
||||
.request(5)
|
||||
.expectNext(1, 2, 3, 4)
|
||||
.expectComplete()
|
||||
|
||||
}
|
||||
|
||||
"emit all things before completing with two fused stages" in assertAllStagesStopped {
|
||||
new Builder {
|
||||
val g = new FusedGraph(
|
||||
builder(emit1234, emit5678)
|
||||
.connect(Upstream, emit1234.in)
|
||||
.connect(emit1234.out, emit5678.in)
|
||||
.connect(emit5678.out, Downstream)
|
||||
.buildAssembly(),
|
||||
FlowShape(emit1234.in, emit5678.out))
|
||||
|
||||
Source.empty.via(g).runWith(TestSink.probe)
|
||||
.request(9)
|
||||
.expectNextN(1 to 8)
|
||||
.expectComplete()
|
||||
}
|
||||
}
|
||||
|
||||
"emit all things before completing with three fused stages" in assertAllStagesStopped {
|
||||
new Builder {
|
||||
val g = new FusedGraph(
|
||||
builder(emit1234, passThrough, emit5678)
|
||||
.connect(Upstream, emit1234.in)
|
||||
.connect(emit1234.out, passThrough.in)
|
||||
.connect(passThrough.out, emit5678.in)
|
||||
.connect(emit5678.out, Downstream)
|
||||
.buildAssembly(),
|
||||
FlowShape(emit1234.in, emit5678.out))
|
||||
|
||||
Source.empty.via(g).runWith(TestSink.probe)
|
||||
.request(9)
|
||||
.expectNextN(1 to 8)
|
||||
.expectComplete()
|
||||
}
|
||||
}
|
||||
|
||||
"invoke lifecycle hooks in the right order" in assertAllStagesStopped {
|
||||
val g = new GraphStage[FlowShape[Int, Int]] {
|
||||
val in = Inlet[Int]("in")
|
||||
val out = Outlet[Int]("out")
|
||||
override val shape = FlowShape(in, out)
|
||||
override def createLogic(attr: Attributes) = new GraphStageLogic(shape) {
|
||||
setHandler(in, eagerTerminateInput)
|
||||
setHandler(out, new OutHandler {
|
||||
override def onPull(): Unit = {
|
||||
completeStage()
|
||||
testActor ! "pulled"
|
||||
}
|
||||
})
|
||||
override def preStart(): Unit = testActor ! "preStart"
|
||||
override def postStop(): Unit = testActor ! "postStop"
|
||||
}
|
||||
}
|
||||
Source.single(1).via(g).runWith(Sink.ignore)
|
||||
expectMsg("preStart")
|
||||
expectMsg("pulled")
|
||||
expectMsg("postStop")
|
||||
}
|
||||
|
||||
"not double-terminate a single stage" in new Builder {
|
||||
object g extends GraphStage[FlowShape[Int, Int]] {
|
||||
val in = Inlet[Int]("in")
|
||||
val out = Outlet[Int]("out")
|
||||
override val shape = FlowShape(in, out)
|
||||
override def createLogic(attr: Attributes) = new GraphStageLogic(shape) {
|
||||
setHandler(in, eagerTerminateInput)
|
||||
setHandler(out, eagerTerminateOutput)
|
||||
override def postStop(): Unit = testActor ! "postStop2"
|
||||
}
|
||||
}
|
||||
|
||||
builder(g, passThrough)
|
||||
.connect(Upstream, g.in)
|
||||
.connect(g.out, passThrough.in)
|
||||
.connect(passThrough.out, Downstream)
|
||||
.init()
|
||||
|
||||
interpreter.complete(0)
|
||||
interpreter.cancel(1)
|
||||
interpreter.execute(2)
|
||||
|
||||
expectMsg("postStop2")
|
||||
expectNoMsg(Duration.Zero)
|
||||
|
||||
interpreter.isCompleted should ===(false)
|
||||
interpreter.isSuspended should ===(false)
|
||||
interpreter.isStageCompleted(interpreter.logics(0)) should ===(true)
|
||||
interpreter.isStageCompleted(interpreter.logics(1)) should ===(false)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -171,7 +171,7 @@ class StreamLayoutSpec extends AkkaSpec {
|
|||
override def onNext(t: Any): Unit = ()
|
||||
}
|
||||
|
||||
class FlatTestMaterializer(_module: Module) extends MaterializerSession(_module) {
|
||||
class FlatTestMaterializer(_module: Module) extends MaterializerSession(_module, Attributes()) {
|
||||
var publishers = Vector.empty[TestPublisher]
|
||||
var subscribers = Vector.empty[TestSubscriber]
|
||||
|
||||
|
|
|
|||
|
|
@ -47,7 +47,7 @@ class ActorGraphInterpreterSpec extends AkkaSpec {
|
|||
val out2 = Outlet[Int]("out2")
|
||||
val shape = BidiShape(in1, out1, in2, out2)
|
||||
|
||||
override def createLogic: GraphStageLogic = new GraphStageLogic(shape) {
|
||||
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new GraphStageLogic(shape) {
|
||||
setHandler(in1, new InHandler {
|
||||
override def onPush(): Unit = push(out1, grab(in1))
|
||||
override def onUpstreamFinish(): Unit = complete(out1)
|
||||
|
|
@ -88,7 +88,7 @@ class ActorGraphInterpreterSpec extends AkkaSpec {
|
|||
val out2 = Outlet[Int]("out2")
|
||||
val shape = BidiShape(in1, out1, in2, out2)
|
||||
|
||||
override def createLogic: GraphStageLogic = new GraphStageLogic(shape) {
|
||||
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new GraphStageLogic(shape) {
|
||||
setHandler(in1, new InHandler {
|
||||
override def onPush(): Unit = push(out1, grab(in1))
|
||||
|
||||
|
|
@ -134,7 +134,7 @@ class ActorGraphInterpreterSpec extends AkkaSpec {
|
|||
val out2 = Outlet[Int]("out2")
|
||||
val shape = BidiShape(in1, out1, in2, out2)
|
||||
|
||||
override def createLogic: GraphStageLogic = new GraphStageLogic(shape) {
|
||||
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new GraphStageLogic(shape) {
|
||||
setHandler(in1, new InHandler {
|
||||
override def onPush(): Unit = push(out1, grab(in1))
|
||||
|
||||
|
|
@ -183,7 +183,7 @@ class ActorGraphInterpreterSpec extends AkkaSpec {
|
|||
val out2 = Outlet[Int]("out2")
|
||||
val shape = BidiShape(in1, out1, in2, out2)
|
||||
|
||||
override def createLogic: GraphStageLogic = new GraphStageLogic(shape) {
|
||||
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new GraphStageLogic(shape) {
|
||||
setHandler(in1, new InHandler {
|
||||
override def onPush(): Unit = push(out2, grab(in1))
|
||||
|
||||
|
|
|
|||
|
|
@ -3,6 +3,7 @@
|
|||
*/
|
||||
package akka.stream.impl.fusing
|
||||
|
||||
import akka.stream.Attributes
|
||||
import akka.stream.testkit.AkkaSpec
|
||||
import akka.stream.scaladsl.{ Merge, Broadcast, Balance, Zip }
|
||||
import GraphInterpreter._
|
||||
|
|
@ -45,6 +46,7 @@ class GraphInterpreterSpec extends GraphInterpreterSpecKit {
|
|||
// Constructing an assembly by hand and resolving ambiguities
|
||||
val assembly = new GraphAssembly(
|
||||
stages = Array(identity, identity),
|
||||
originalAttributes = Array(Attributes.none, Attributes.none),
|
||||
ins = Array(identity.in, identity.in, null),
|
||||
inOwners = Array(0, 1, -1),
|
||||
outs = Array(null, identity.out, identity.out),
|
||||
|
|
|
|||
|
|
@ -5,35 +5,32 @@ package akka.stream.impl.fusing
|
|||
|
||||
import akka.event.Logging
|
||||
import akka.stream._
|
||||
import akka.stream.impl.fusing.GraphInterpreter.{ Failed, GraphAssembly, DownstreamBoundaryStageLogic, UpstreamBoundaryStageLogic }
|
||||
import akka.stream.stage.{ InHandler, OutHandler, GraphStage, GraphStageLogic }
|
||||
import akka.stream.impl.fusing.GraphInterpreter.{ DownstreamBoundaryStageLogic, Failed, GraphAssembly, UpstreamBoundaryStageLogic }
|
||||
import akka.stream.stage.AbstractStage.PushPullGraphStage
|
||||
import akka.stream.stage.{ GraphStage, GraphStageLogic, InHandler, OutHandler, _ }
|
||||
import akka.stream.testkit.AkkaSpec
|
||||
import akka.stream.testkit.Utils.TE
|
||||
import akka.stream.impl.fusing.GraphInterpreter.GraphAssembly
|
||||
|
||||
trait GraphInterpreterSpecKit extends AkkaSpec {
|
||||
|
||||
sealed trait TestEvent {
|
||||
def source: GraphStageLogic
|
||||
}
|
||||
|
||||
case class OnComplete(source: GraphStageLogic) extends TestEvent
|
||||
case class Cancel(source: GraphStageLogic) extends TestEvent
|
||||
case class OnError(source: GraphStageLogic, cause: Throwable) extends TestEvent
|
||||
case class OnNext(source: GraphStageLogic, elem: Any) extends TestEvent
|
||||
case class RequestOne(source: GraphStageLogic) extends TestEvent
|
||||
case class RequestAnother(source: GraphStageLogic) extends TestEvent
|
||||
|
||||
case class PreStart(source: GraphStageLogic) extends TestEvent
|
||||
case class PostStop(source: GraphStageLogic) extends TestEvent
|
||||
|
||||
abstract class TestSetup {
|
||||
protected var lastEvent: Set[TestEvent] = Set.empty
|
||||
abstract class Builder {
|
||||
private var _interpreter: GraphInterpreter = _
|
||||
protected def interpreter: GraphInterpreter = _interpreter
|
||||
|
||||
def stepAll(): Unit = interpreter.execute(eventLimit = Int.MaxValue)
|
||||
def step(): Unit = interpreter.execute(eventLimit = 1)
|
||||
|
||||
object Upstream extends UpstreamBoundaryStageLogic[Int] {
|
||||
override val out = Outlet[Int]("up")
|
||||
out.id = 0
|
||||
}
|
||||
|
||||
object Downstream extends DownstreamBoundaryStageLogic[Int] {
|
||||
override val in = Inlet[Int]("down")
|
||||
in.id = 0
|
||||
}
|
||||
|
||||
class AssemblyBuilder(stages: Seq[GraphStage[_ <: Shape]]) {
|
||||
var upstreams = Vector.empty[(UpstreamBoundaryStageLogic[_], Inlet[_])]
|
||||
var downstreams = Vector.empty[(Outlet[_], DownstreamBoundaryStageLogic[_])]
|
||||
|
|
@ -54,20 +51,25 @@ trait GraphInterpreterSpecKit extends AkkaSpec {
|
|||
this
|
||||
}
|
||||
|
||||
def init(): Unit = {
|
||||
def buildAssembly(): GraphAssembly = {
|
||||
val ins = upstreams.map(_._2) ++ connections.map(_._2)
|
||||
val outs = connections.map(_._1) ++ downstreams.map(_._1)
|
||||
val inOwners = ins.map { in ⇒ stages.indexWhere(_.shape.inlets.contains(in)) }
|
||||
val outOwners = outs.map { out ⇒ stages.indexWhere(_.shape.outlets.contains(out)) }
|
||||
|
||||
val assembly = new GraphAssembly(
|
||||
new GraphAssembly(
|
||||
stages.toArray,
|
||||
Array.fill(stages.size)(Attributes.none),
|
||||
(ins ++ Vector.fill(downstreams.size)(null)).toArray,
|
||||
(inOwners ++ Vector.fill(downstreams.size)(-1)).toArray,
|
||||
(Vector.fill(upstreams.size)(null) ++ outs).toArray,
|
||||
(Vector.fill(upstreams.size)(-1) ++ outOwners).toArray)
|
||||
}
|
||||
|
||||
val (inHandlers, outHandlers, logics, _) = assembly.materialize()
|
||||
def init(): Unit = {
|
||||
val assembly = buildAssembly()
|
||||
|
||||
val (inHandlers, outHandlers, logics, _) = assembly.materialize(Attributes.none)
|
||||
_interpreter = new GraphInterpreter(assembly, NoMaterializer, Logging(system, classOf[TestSetup]), inHandlers, outHandlers, logics, (_, _, _) ⇒ ())
|
||||
|
||||
for ((upstream, i) ← upstreams.zipWithIndex) {
|
||||
|
|
@ -83,11 +85,29 @@ trait GraphInterpreterSpecKit extends AkkaSpec {
|
|||
}
|
||||
|
||||
def manualInit(assembly: GraphAssembly): Unit = {
|
||||
val (inHandlers, outHandlers, logics, _) = assembly.materialize()
|
||||
val (inHandlers, outHandlers, logics, _) = assembly.materialize(Attributes.none)
|
||||
_interpreter = new GraphInterpreter(assembly, NoMaterializer, Logging(system, classOf[TestSetup]), inHandlers, outHandlers, logics, (_, _, _) ⇒ ())
|
||||
}
|
||||
|
||||
def builder(stages: GraphStage[_ <: Shape]*): AssemblyBuilder = new AssemblyBuilder(stages.toSeq)
|
||||
def builder(stages: GraphStage[_ <: Shape]*): AssemblyBuilder = new AssemblyBuilder(stages)
|
||||
}
|
||||
|
||||
abstract class TestSetup extends Builder {
|
||||
|
||||
sealed trait TestEvent {
|
||||
def source: GraphStageLogic
|
||||
}
|
||||
|
||||
case class OnComplete(source: GraphStageLogic) extends TestEvent
|
||||
case class Cancel(source: GraphStageLogic) extends TestEvent
|
||||
case class OnError(source: GraphStageLogic, cause: Throwable) extends TestEvent
|
||||
case class OnNext(source: GraphStageLogic, elem: Any) extends TestEvent
|
||||
case class RequestOne(source: GraphStageLogic) extends TestEvent
|
||||
case class RequestAnother(source: GraphStageLogic) extends TestEvent
|
||||
case class PreStart(source: GraphStageLogic) extends TestEvent
|
||||
case class PostStop(source: GraphStageLogic) extends TestEvent
|
||||
|
||||
protected var lastEvent: Set[TestEvent] = Set.empty
|
||||
|
||||
def lastEvents(): Set[TestEvent] = {
|
||||
val result = lastEvent
|
||||
|
|
@ -158,7 +178,7 @@ trait GraphInterpreterSpecKit extends AkkaSpec {
|
|||
|
||||
// Modified onPush that does not grab() automatically the element. This accesses some internals.
|
||||
override def onPush(): Unit = {
|
||||
val internalEvent = interpreter.connectionSlots(inToConn(in.id))
|
||||
val internalEvent = interpreter.connectionSlots(portToConn(in.id))
|
||||
|
||||
internalEvent match {
|
||||
case Failed(_, elem) ⇒ lastEvent += OnNext(DownstreamPortProbe.this, elem)
|
||||
|
|
@ -173,6 +193,7 @@ trait GraphInterpreterSpecKit extends AkkaSpec {
|
|||
|
||||
private val assembly = new GraphAssembly(
|
||||
stages = Array.empty,
|
||||
originalAttributes = Array.empty,
|
||||
ins = Array(null),
|
||||
inOwners = Array(-1),
|
||||
outs = Array(null),
|
||||
|
|
@ -233,7 +254,7 @@ trait GraphInterpreterSpecKit extends AkkaSpec {
|
|||
|
||||
private val sandwitchStage = new GraphStage[FlowShape[Int, Int]] {
|
||||
override def shape = stageshape
|
||||
override def createLogic: GraphStageLogic = stage
|
||||
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = stage
|
||||
}
|
||||
|
||||
class UpstreamPortProbe[T] extends UpstreamProbe[T]("upstreamPort") {
|
||||
|
|
@ -253,4 +274,133 @@ trait GraphInterpreterSpecKit extends AkkaSpec {
|
|||
.init()
|
||||
}
|
||||
|
||||
abstract class OneBoundedSetup[T](ops: Array[GraphStageWithMaterializedValue[Shape, Any]]) extends Builder {
|
||||
|
||||
def this(ops: Iterable[Stage[_, _]]) = {
|
||||
this(ops.map { op ⇒
|
||||
new PushPullGraphStage[Any, Any, Any](
|
||||
(_) ⇒ op.asInstanceOf[Stage[Any, Any]],
|
||||
Attributes.none)
|
||||
}.toArray.asInstanceOf[Array[GraphStageWithMaterializedValue[Shape, Any]]])
|
||||
}
|
||||
|
||||
val upstream = new UpstreamOneBoundedProbe[T]
|
||||
val downstream = new DownstreamOneBoundedPortProbe[T]
|
||||
var lastEvent = Set.empty[TestEvent]
|
||||
|
||||
sealed trait TestEvent
|
||||
|
||||
case object OnComplete extends TestEvent
|
||||
case object Cancel extends TestEvent
|
||||
case class OnError(cause: Throwable) extends TestEvent
|
||||
case class OnNext(elem: Any) extends TestEvent
|
||||
case object RequestOne extends TestEvent
|
||||
case object RequestAnother extends TestEvent
|
||||
|
||||
private def run() = interpreter.execute(Int.MaxValue)
|
||||
|
||||
private def initialize(): Unit = {
|
||||
import GraphInterpreter.Boundary
|
||||
|
||||
var i = 0
|
||||
val attributes = Array.fill[Attributes](ops.length)(Attributes.none)
|
||||
val ins = Array.ofDim[Inlet[_]](ops.length + 1)
|
||||
val inOwners = Array.ofDim[Int](ops.length + 1)
|
||||
val outs = Array.ofDim[Outlet[_]](ops.length + 1)
|
||||
val outOwners = Array.ofDim[Int](ops.length + 1)
|
||||
|
||||
ins(ops.length) = null
|
||||
inOwners(ops.length) = Boundary
|
||||
outs(0) = null
|
||||
outOwners(0) = Boundary
|
||||
|
||||
while (i < ops.length) {
|
||||
val stage = ops(i).asInstanceOf[PushPullGraphStage[_, _, _]]
|
||||
ins(i) = stage.shape.inlet
|
||||
inOwners(i) = i
|
||||
outs(i + 1) = stage.shape.outlet
|
||||
outOwners(i + 1) = i
|
||||
i += 1
|
||||
}
|
||||
|
||||
manualInit(new GraphAssembly(ops, attributes, ins, inOwners, outs, outOwners))
|
||||
interpreter.attachUpstreamBoundary(0, upstream)
|
||||
interpreter.attachDownstreamBoundary(ops.length, downstream)
|
||||
|
||||
interpreter.init()
|
||||
|
||||
}
|
||||
|
||||
initialize()
|
||||
run() // Detached stages need the prefetch
|
||||
|
||||
def lastEvents(): Set[TestEvent] = {
|
||||
val events = lastEvent
|
||||
lastEvent = Set.empty
|
||||
events
|
||||
}
|
||||
|
||||
class UpstreamOneBoundedProbe[T] extends UpstreamBoundaryStageLogic[T] {
|
||||
val out = Outlet[T]("out")
|
||||
out.id = 0
|
||||
|
||||
setHandler(out, new OutHandler {
|
||||
override def onPull(): Unit = {
|
||||
if (lastEvent.contains(RequestOne)) lastEvent += RequestAnother
|
||||
else lastEvent += RequestOne
|
||||
}
|
||||
|
||||
override def onDownstreamFinish(): Unit = lastEvent += Cancel
|
||||
})
|
||||
|
||||
def onNext(elem: T): Unit = {
|
||||
push(out, elem)
|
||||
run()
|
||||
}
|
||||
def onComplete(): Unit = {
|
||||
complete(out)
|
||||
run()
|
||||
}
|
||||
|
||||
def onNextAndComplete(elem: T): Unit = {
|
||||
push(out, elem)
|
||||
complete(out)
|
||||
run()
|
||||
}
|
||||
|
||||
def onError(ex: Throwable): Unit = {
|
||||
fail(out, ex)
|
||||
run()
|
||||
}
|
||||
}
|
||||
|
||||
class DownstreamOneBoundedPortProbe[T] extends DownstreamBoundaryStageLogic[T] {
|
||||
val in = Inlet[T]("in")
|
||||
in.id = 0
|
||||
|
||||
setHandler(in, new InHandler {
|
||||
|
||||
// Modified onPush that does not grab() automatically the element. This accesses some internals.
|
||||
override def onPush(): Unit = {
|
||||
lastEvent += OnNext(grab(in))
|
||||
}
|
||||
|
||||
override def onUpstreamFinish() = lastEvent += OnComplete
|
||||
override def onUpstreamFailure(ex: Throwable) = lastEvent += OnError(ex)
|
||||
})
|
||||
|
||||
def requestOne(): Unit = {
|
||||
pull(in)
|
||||
run()
|
||||
}
|
||||
|
||||
def cancel(): Unit = {
|
||||
cancel(in)
|
||||
run()
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -4,17 +4,24 @@
|
|||
package akka.stream.impl.fusing
|
||||
|
||||
import akka.stream.stage._
|
||||
import akka.stream.testkit.AkkaSpec
|
||||
import akka.testkit.EventFilter
|
||||
|
||||
import scala.util.control.NoStackTrace
|
||||
import akka.stream.Supervision
|
||||
|
||||
class InterpreterSpec extends InterpreterSpecKit {
|
||||
class InterpreterSpec extends AkkaSpec with GraphInterpreterSpecKit {
|
||||
import Supervision.stoppingDecider
|
||||
|
||||
/*
|
||||
* These tests were writtern for the previous veryion of the interpreter, the so called OneBoundedInterpreter.
|
||||
* These stages are now properly emulated by the GraphInterpreter and many of the edge cases were relevant to
|
||||
* the execution model of the old one. Still, these tests are very valuable, so please do not remove.
|
||||
*/
|
||||
|
||||
"Interpreter" must {
|
||||
|
||||
"implement map correctly" in new TestSetup(Seq(Map((x: Int) ⇒ x + 1, stoppingDecider))) {
|
||||
"implement map correctly" in new OneBoundedSetup[Int](Seq(Map((x: Int) ⇒ x + 1, stoppingDecider))) {
|
||||
lastEvents() should be(Set.empty)
|
||||
|
||||
downstream.requestOne()
|
||||
|
|
@ -33,7 +40,7 @@ class InterpreterSpec extends InterpreterSpecKit {
|
|||
lastEvents() should be(Set(OnComplete))
|
||||
}
|
||||
|
||||
"implement chain of maps correctly" in new TestSetup(Seq(
|
||||
"implement chain of maps correctly" in new OneBoundedSetup[Int](Seq(
|
||||
Map((x: Int) ⇒ x + 1, stoppingDecider),
|
||||
Map((x: Int) ⇒ x * 2, stoppingDecider),
|
||||
Map((x: Int) ⇒ x + 1, stoppingDecider))) {
|
||||
|
|
@ -56,7 +63,7 @@ class InterpreterSpec extends InterpreterSpecKit {
|
|||
lastEvents() should be(Set(Cancel))
|
||||
}
|
||||
|
||||
"work with only boundary ops" in new TestSetup(Seq.empty) {
|
||||
"work with only boundary ops" in new OneBoundedSetup[Int](Seq.empty) {
|
||||
lastEvents() should be(Set.empty)
|
||||
|
||||
downstream.requestOne()
|
||||
|
|
@ -69,7 +76,7 @@ class InterpreterSpec extends InterpreterSpecKit {
|
|||
lastEvents() should be(Set(OnComplete))
|
||||
}
|
||||
|
||||
"implement one-to-many many-to-one chain correctly" in new TestSetup(Seq(
|
||||
"implement one-to-many many-to-one chain correctly" in new OneBoundedSetup[Int](Seq(
|
||||
Doubler(),
|
||||
Filter((x: Int) ⇒ x != 0, stoppingDecider))) {
|
||||
|
||||
|
|
@ -94,7 +101,7 @@ class InterpreterSpec extends InterpreterSpecKit {
|
|||
lastEvents() should be(Set(OnComplete))
|
||||
}
|
||||
|
||||
"implement many-to-one one-to-many chain correctly" in new TestSetup(Seq(
|
||||
"implement many-to-one one-to-many chain correctly" in new OneBoundedSetup[Int](Seq(
|
||||
Filter((x: Int) ⇒ x != 0, stoppingDecider),
|
||||
Doubler())) {
|
||||
|
||||
|
|
@ -119,7 +126,7 @@ class InterpreterSpec extends InterpreterSpecKit {
|
|||
lastEvents() should be(Set(Cancel))
|
||||
}
|
||||
|
||||
"implement take" in new TestSetup(Seq(Take(2))) {
|
||||
"implement take" in new OneBoundedSetup[Int](Seq(Take(2))) {
|
||||
|
||||
lastEvents() should be(Set.empty)
|
||||
|
||||
|
|
@ -136,7 +143,7 @@ class InterpreterSpec extends InterpreterSpecKit {
|
|||
lastEvents() should be(Set(OnNext(1), Cancel, OnComplete))
|
||||
}
|
||||
|
||||
"implement take inside a chain" in new TestSetup(Seq(
|
||||
"implement take inside a chain" in new OneBoundedSetup[Int](Seq(
|
||||
Filter((x: Int) ⇒ x != 0, stoppingDecider),
|
||||
Take(2),
|
||||
Map((x: Int) ⇒ x + 1, stoppingDecider))) {
|
||||
|
|
@ -159,7 +166,7 @@ class InterpreterSpec extends InterpreterSpecKit {
|
|||
lastEvents() should be(Set(Cancel, OnComplete, OnNext(3)))
|
||||
}
|
||||
|
||||
"implement fold" in new TestSetup(Seq(Fold(0, (agg: Int, x: Int) ⇒ agg + x, stoppingDecider))) {
|
||||
"implement fold" in new OneBoundedSetup[Int](Seq(Fold(0, (agg: Int, x: Int) ⇒ agg + x, stoppingDecider))) {
|
||||
lastEvents() should be(Set.empty)
|
||||
|
||||
downstream.requestOne()
|
||||
|
|
@ -178,7 +185,7 @@ class InterpreterSpec extends InterpreterSpecKit {
|
|||
lastEvents() should be(Set(OnNext(3), OnComplete))
|
||||
}
|
||||
|
||||
"implement fold with proper cancel" in new TestSetup(Seq(Fold(0, (agg: Int, x: Int) ⇒ agg + x, stoppingDecider))) {
|
||||
"implement fold with proper cancel" in new OneBoundedSetup[Int](Seq(Fold(0, (agg: Int, x: Int) ⇒ agg + x, stoppingDecider))) {
|
||||
|
||||
lastEvents() should be(Set.empty)
|
||||
|
||||
|
|
@ -198,7 +205,7 @@ class InterpreterSpec extends InterpreterSpecKit {
|
|||
lastEvents() should be(Set(Cancel))
|
||||
}
|
||||
|
||||
"work if fold completes while not in a push position" in new TestSetup(Seq(Fold(0, (agg: Int, x: Int) ⇒ agg + x, stoppingDecider))) {
|
||||
"work if fold completes while not in a push position" in new OneBoundedSetup[Int](Seq(Fold(0, (agg: Int, x: Int) ⇒ agg + x, stoppingDecider))) {
|
||||
|
||||
lastEvents() should be(Set.empty)
|
||||
|
||||
|
|
@ -209,7 +216,7 @@ class InterpreterSpec extends InterpreterSpecKit {
|
|||
lastEvents() should be(Set(OnComplete, OnNext(0)))
|
||||
}
|
||||
|
||||
"implement grouped" in new TestSetup(Seq(Grouped(3))) {
|
||||
"implement grouped" in new OneBoundedSetup[Int](Seq(Grouped(3))) {
|
||||
lastEvents() should be(Set.empty)
|
||||
|
||||
downstream.requestOne()
|
||||
|
|
@ -234,7 +241,7 @@ class InterpreterSpec extends InterpreterSpecKit {
|
|||
lastEvents() should be(Set(OnNext(Vector(3)), OnComplete))
|
||||
}
|
||||
|
||||
"implement conflate" in new TestSetup(Seq(Conflate(
|
||||
"implement conflate" in new OneBoundedSetup[Int](Seq(Conflate(
|
||||
(in: Int) ⇒ in,
|
||||
(agg: Int, x: Int) ⇒ agg + x,
|
||||
stoppingDecider))) {
|
||||
|
|
@ -266,7 +273,7 @@ class InterpreterSpec extends InterpreterSpecKit {
|
|||
lastEvents() should be(Set(Cancel))
|
||||
}
|
||||
|
||||
"implement expand" in new TestSetup(Seq(Expand(
|
||||
"implement expand" in new OneBoundedSetup[Int](Seq(Expand(
|
||||
(in: Int) ⇒ in,
|
||||
(agg: Int) ⇒ (agg, agg)))) {
|
||||
|
||||
|
|
@ -294,7 +301,7 @@ class InterpreterSpec extends InterpreterSpecKit {
|
|||
lastEvents() should be(Set(OnComplete))
|
||||
}
|
||||
|
||||
"work with conflate-conflate" in new TestSetup(Seq(
|
||||
"work with conflate-conflate" in new OneBoundedSetup[Int](Seq(
|
||||
Conflate(
|
||||
(in: Int) ⇒ in,
|
||||
(agg: Int, x: Int) ⇒ agg + x,
|
||||
|
|
@ -332,7 +339,7 @@ class InterpreterSpec extends InterpreterSpecKit {
|
|||
|
||||
}
|
||||
|
||||
"work with expand-expand" in new TestSetup(Seq(
|
||||
"work with expand-expand" in new OneBoundedSetup[Int](Seq(
|
||||
Expand(
|
||||
(in: Int) ⇒ in,
|
||||
(agg: Int) ⇒ (agg, agg + 1)),
|
||||
|
|
@ -369,7 +376,7 @@ class InterpreterSpec extends InterpreterSpecKit {
|
|||
lastEvents() should be(Set(OnComplete, OnNext(12)))
|
||||
}
|
||||
|
||||
"implement conflate-expand" in new TestSetup(Seq(
|
||||
"implement conflate-expand" in new OneBoundedSetup[Int](Seq(
|
||||
Conflate(
|
||||
(in: Int) ⇒ in,
|
||||
(agg: Int, x: Int) ⇒ agg + x,
|
||||
|
|
@ -405,12 +412,7 @@ class InterpreterSpec extends InterpreterSpecKit {
|
|||
lastEvents() should be(Set(Cancel))
|
||||
}
|
||||
|
||||
"implement expand-conflate" in {
|
||||
pending
|
||||
// Needs to detect divergent loops
|
||||
}
|
||||
|
||||
"implement doubler-conflate" in new TestSetup(Seq(
|
||||
"implement doubler-conflate" in new OneBoundedSetup[Int](Seq(
|
||||
Doubler(),
|
||||
Conflate(
|
||||
(in: Int) ⇒ in,
|
||||
|
|
@ -429,7 +431,8 @@ class InterpreterSpec extends InterpreterSpecKit {
|
|||
|
||||
}
|
||||
|
||||
"work with jumpback table and completed elements" in new TestSetup(Seq(
|
||||
// Note, the new interpreter has no jumpback table, still did not want to remove the test
|
||||
"work with jumpback table and completed elements" in new OneBoundedSetup[Int](Seq(
|
||||
Map((x: Int) ⇒ x, stoppingDecider),
|
||||
Map((x: Int) ⇒ x, stoppingDecider),
|
||||
KeepGoing(),
|
||||
|
|
@ -461,7 +464,7 @@ class InterpreterSpec extends InterpreterSpecKit {
|
|||
|
||||
}
|
||||
|
||||
"work with pushAndFinish if upstream completes with pushAndFinish" in new TestSetup(Seq(
|
||||
"work with pushAndFinish if upstream completes with pushAndFinish" in new OneBoundedSetup[Int](Seq(
|
||||
new PushFinishStage)) {
|
||||
|
||||
lastEvents() should be(Set.empty)
|
||||
|
|
@ -469,11 +472,11 @@ class InterpreterSpec extends InterpreterSpecKit {
|
|||
downstream.requestOne()
|
||||
lastEvents() should be(Set(RequestOne))
|
||||
|
||||
upstream.onNextAndComplete("foo")
|
||||
lastEvents() should be(Set(OnNext("foo"), OnComplete))
|
||||
upstream.onNextAndComplete(0)
|
||||
lastEvents() should be(Set(OnNext(0), OnComplete))
|
||||
}
|
||||
|
||||
"work with pushAndFinish if indirect upstream completes with pushAndFinish" in new TestSetup(Seq(
|
||||
"work with pushAndFinish if indirect upstream completes with pushAndFinish" in new OneBoundedSetup[Int](Seq(
|
||||
Map((x: Any) ⇒ x, stoppingDecider),
|
||||
new PushFinishStage,
|
||||
Map((x: Any) ⇒ x, stoppingDecider))) {
|
||||
|
|
@ -483,24 +486,24 @@ class InterpreterSpec extends InterpreterSpecKit {
|
|||
downstream.requestOne()
|
||||
lastEvents() should be(Set(RequestOne))
|
||||
|
||||
upstream.onNextAndComplete("foo")
|
||||
lastEvents() should be(Set(OnNext("foo"), OnComplete))
|
||||
upstream.onNextAndComplete(1)
|
||||
lastEvents() should be(Set(OnNext(1), OnComplete))
|
||||
}
|
||||
|
||||
"work with pushAndFinish if upstream completes with pushAndFinish and downstream immediately pulls" in new TestSetup(Seq(
|
||||
"work with pushAndFinish if upstream completes with pushAndFinish and downstream immediately pulls" in new OneBoundedSetup[Int](Seq(
|
||||
new PushFinishStage,
|
||||
Fold("", (x: String, y: String) ⇒ x + y, stoppingDecider))) {
|
||||
Fold(0, (x: Int, y: Int) ⇒ x + y, stoppingDecider))) {
|
||||
|
||||
lastEvents() should be(Set.empty)
|
||||
|
||||
downstream.requestOne()
|
||||
lastEvents() should be(Set(RequestOne))
|
||||
|
||||
upstream.onNextAndComplete("foo")
|
||||
lastEvents() should be(Set(OnNext("foo"), OnComplete))
|
||||
upstream.onNextAndComplete(1)
|
||||
lastEvents() should be(Set(OnNext(1), OnComplete))
|
||||
}
|
||||
|
||||
"report error if pull is called while op is terminating" in new TestSetup(Seq(new PushPullStage[Any, Any] {
|
||||
"report error if pull is called while op is terminating" in new OneBoundedSetup[Int](Seq(new PushPullStage[Any, Any] {
|
||||
override def onPull(ctx: Context[Any]): SyncDirective = ctx.pull()
|
||||
override def onPush(elem: Any, ctx: Context[Any]): SyncDirective = ctx.pull()
|
||||
override def onUpstreamFinish(ctx: Context[Any]): TerminationDirective = ctx.absorbTermination()
|
||||
|
|
@ -514,12 +517,12 @@ class InterpreterSpec extends InterpreterSpecKit {
|
|||
val ev = lastEvents()
|
||||
ev.nonEmpty should be(true)
|
||||
ev.forall {
|
||||
case OnError(_: IllegalStateException) ⇒ true
|
||||
case _ ⇒ false
|
||||
case OnError(_: IllegalArgumentException) ⇒ true
|
||||
case _ ⇒ false
|
||||
} should be(true)
|
||||
}
|
||||
|
||||
"implement take-take" in new TestSetup(Seq(
|
||||
"implement take-take" in new OneBoundedSetup[Int](Seq(
|
||||
Take(1),
|
||||
Take(1))) {
|
||||
lastEvents() should be(Set.empty)
|
||||
|
|
@ -527,12 +530,12 @@ class InterpreterSpec extends InterpreterSpecKit {
|
|||
downstream.requestOne()
|
||||
lastEvents() should be(Set(RequestOne))
|
||||
|
||||
upstream.onNext("foo")
|
||||
lastEvents() should be(Set(OnNext("foo"), OnComplete, Cancel))
|
||||
upstream.onNext(1)
|
||||
lastEvents() should be(Set(OnNext(1), OnComplete, Cancel))
|
||||
|
||||
}
|
||||
|
||||
"implement take-take with pushAndFinish from upstream" in new TestSetup(Seq(
|
||||
"implement take-take with pushAndFinish from upstream" in new OneBoundedSetup[Int](Seq(
|
||||
Take(1),
|
||||
Take(1))) {
|
||||
lastEvents() should be(Set.empty)
|
||||
|
|
@ -540,8 +543,8 @@ class InterpreterSpec extends InterpreterSpecKit {
|
|||
downstream.requestOne()
|
||||
lastEvents() should be(Set(RequestOne))
|
||||
|
||||
upstream.onNextAndComplete("foo")
|
||||
lastEvents() should be(Set(OnNext("foo"), OnComplete))
|
||||
upstream.onNextAndComplete(1)
|
||||
lastEvents() should be(Set(OnNext(1), OnComplete))
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -551,7 +554,7 @@ class InterpreterSpec extends InterpreterSpecKit {
|
|||
override def onDownstreamFinish(ctx: Context[Int]): TerminationDirective = ctx.absorbTermination()
|
||||
}
|
||||
|
||||
"not allow absorbTermination from onDownstreamFinish()" in new TestSetup(Seq(
|
||||
"not allow absorbTermination from onDownstreamFinish()" in new OneBoundedSetup[Int](Seq(
|
||||
new InvalidAbsorbTermination)) {
|
||||
lastEvents() should be(Set.empty)
|
||||
|
||||
|
|
@ -564,4 +567,51 @@ class InterpreterSpec extends InterpreterSpecKit {
|
|||
|
||||
}
|
||||
|
||||
private[akka] case class Doubler[T]() extends PushPullStage[T, T] {
|
||||
var oneMore: Boolean = false
|
||||
var lastElem: T = _
|
||||
|
||||
override def onPush(elem: T, ctx: Context[T]): SyncDirective = {
|
||||
lastElem = elem
|
||||
oneMore = true
|
||||
ctx.push(elem)
|
||||
}
|
||||
|
||||
override def onPull(ctx: Context[T]): SyncDirective = {
|
||||
if (oneMore) {
|
||||
oneMore = false
|
||||
ctx.push(lastElem)
|
||||
} else ctx.pull()
|
||||
}
|
||||
}
|
||||
|
||||
private[akka] case class KeepGoing[T]() extends PushPullStage[T, T] {
|
||||
var lastElem: T = _
|
||||
|
||||
override def onPush(elem: T, ctx: Context[T]): SyncDirective = {
|
||||
lastElem = elem
|
||||
ctx.push(elem)
|
||||
}
|
||||
|
||||
override def onPull(ctx: Context[T]): SyncDirective = {
|
||||
if (ctx.isFinishing) {
|
||||
ctx.push(lastElem)
|
||||
} else ctx.pull()
|
||||
}
|
||||
|
||||
override def onUpstreamFinish(ctx: Context[T]): TerminationDirective = ctx.absorbTermination()
|
||||
}
|
||||
|
||||
// This test is related to issue #17351
|
||||
private[akka] class PushFinishStage(onPostStop: () ⇒ Unit = () ⇒ ()) extends PushStage[Any, Any] {
|
||||
override def onPush(elem: Any, ctx: Context[Any]): SyncDirective =
|
||||
ctx.pushAndFinish(elem)
|
||||
|
||||
override def onUpstreamFinish(ctx: Context[Any]): TerminationDirective =
|
||||
ctx.fail(akka.stream.testkit.Utils.TE("Cannot happen"))
|
||||
|
||||
override def postStop(): Unit =
|
||||
onPostStop()
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,183 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.stream.impl.fusing
|
||||
|
||||
import akka.event.Logging
|
||||
import akka.stream.stage._
|
||||
import akka.stream.testkit.AkkaSpec
|
||||
import akka.stream.{ ActorMaterializer, Attributes }
|
||||
import akka.testkit.TestProbe
|
||||
|
||||
trait InterpreterLifecycleSpecKit {
|
||||
private[akka] case class PreStartAndPostStopIdentity[T](
|
||||
onStart: LifecycleContext ⇒ Unit = _ ⇒ (),
|
||||
onStop: () ⇒ Unit = () ⇒ (),
|
||||
onUpstreamCompleted: () ⇒ Unit = () ⇒ (),
|
||||
onUpstreamFailed: Throwable ⇒ Unit = ex ⇒ ())
|
||||
extends PushStage[T, T] {
|
||||
override def preStart(ctx: LifecycleContext) = onStart(ctx)
|
||||
|
||||
override def onPush(elem: T, ctx: Context[T]) = ctx.push(elem)
|
||||
|
||||
override def onUpstreamFinish(ctx: Context[T]): TerminationDirective = {
|
||||
onUpstreamCompleted()
|
||||
super.onUpstreamFinish(ctx)
|
||||
}
|
||||
|
||||
override def onUpstreamFailure(cause: Throwable, ctx: Context[T]): TerminationDirective = {
|
||||
onUpstreamFailed(cause)
|
||||
super.onUpstreamFailure(cause, ctx)
|
||||
}
|
||||
|
||||
override def postStop() = onStop()
|
||||
}
|
||||
|
||||
private[akka] case class PreStartFailer[T](pleaseThrow: () ⇒ Unit) extends PushStage[T, T] {
|
||||
|
||||
override def preStart(ctx: LifecycleContext) =
|
||||
pleaseThrow()
|
||||
|
||||
override def onPush(elem: T, ctx: Context[T]) = ctx.push(elem)
|
||||
}
|
||||
|
||||
private[akka] case class PostStopFailer[T](ex: () ⇒ Throwable) extends PushStage[T, T] {
|
||||
override def onUpstreamFinish(ctx: Context[T]) = ctx.finish()
|
||||
override def onPush(elem: T, ctx: Context[T]) = ctx.push(elem)
|
||||
|
||||
override def postStop(): Unit = throw ex()
|
||||
}
|
||||
|
||||
// This test is related to issue #17351
|
||||
private[akka] class PushFinishStage(onPostStop: () ⇒ Unit = () ⇒ ()) extends PushStage[Any, Any] {
|
||||
override def onPush(elem: Any, ctx: Context[Any]): SyncDirective =
|
||||
ctx.pushAndFinish(elem)
|
||||
|
||||
override def onUpstreamFinish(ctx: Context[Any]): TerminationDirective =
|
||||
ctx.fail(akka.stream.testkit.Utils.TE("Cannot happen"))
|
||||
|
||||
override def postStop(): Unit =
|
||||
onPostStop()
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
trait InterpreterSpecKit extends AkkaSpec with InterpreterLifecycleSpecKit {
|
||||
|
||||
case object OnComplete
|
||||
case object Cancel
|
||||
case class OnError(cause: Throwable)
|
||||
case class OnNext(elem: Any)
|
||||
case object RequestOne
|
||||
case object RequestAnother
|
||||
|
||||
private[akka] case class Doubler[T]() extends PushPullStage[T, T] {
|
||||
var oneMore: Boolean = false
|
||||
var lastElem: T = _
|
||||
|
||||
override def onPush(elem: T, ctx: Context[T]): SyncDirective = {
|
||||
lastElem = elem
|
||||
oneMore = true
|
||||
ctx.push(elem)
|
||||
}
|
||||
|
||||
override def onPull(ctx: Context[T]): SyncDirective = {
|
||||
if (oneMore) {
|
||||
oneMore = false
|
||||
ctx.push(lastElem)
|
||||
} else ctx.pull()
|
||||
}
|
||||
}
|
||||
|
||||
private[akka] case class KeepGoing[T]() extends PushPullStage[T, T] {
|
||||
var lastElem: T = _
|
||||
|
||||
override def onPush(elem: T, ctx: Context[T]): SyncDirective = {
|
||||
lastElem = elem
|
||||
ctx.push(elem)
|
||||
}
|
||||
|
||||
override def onPull(ctx: Context[T]): SyncDirective = {
|
||||
if (ctx.isFinishing) {
|
||||
ctx.push(lastElem)
|
||||
} else ctx.pull()
|
||||
}
|
||||
|
||||
override def onUpstreamFinish(ctx: Context[T]): TerminationDirective = ctx.absorbTermination()
|
||||
}
|
||||
|
||||
abstract class TestSetup(ops: Seq[Stage[_, _]], forkLimit: Int = 100, overflowToHeap: Boolean = false) {
|
||||
private var lastEvent: Set[Any] = Set.empty
|
||||
|
||||
val upstream = new UpstreamProbe
|
||||
val downstream = new DownstreamProbe
|
||||
val sidechannel = TestProbe()
|
||||
val interpreter = new OneBoundedInterpreter(upstream +: ops :+ downstream,
|
||||
(op, ctx, event) ⇒ sidechannel.ref ! ActorInterpreter.AsyncInput(op, ctx, event),
|
||||
Logging(system, classOf[TestSetup]),
|
||||
ActorMaterializer(),
|
||||
Attributes.none,
|
||||
forkLimit, overflowToHeap)
|
||||
interpreter.init()
|
||||
|
||||
def lastEvents(): Set[Any] = {
|
||||
val result = lastEvent
|
||||
lastEvent = Set.empty
|
||||
result
|
||||
}
|
||||
|
||||
private[akka] class UpstreamProbe extends BoundaryStage {
|
||||
|
||||
override def onDownstreamFinish(ctx: BoundaryContext): TerminationDirective = {
|
||||
lastEvent += Cancel
|
||||
ctx.finish()
|
||||
}
|
||||
|
||||
override def onPull(ctx: BoundaryContext): Directive = {
|
||||
if (lastEvent(RequestOne))
|
||||
lastEvent += RequestAnother
|
||||
else
|
||||
lastEvent += RequestOne
|
||||
ctx.exit()
|
||||
}
|
||||
|
||||
override def onPush(elem: Any, ctx: BoundaryContext): Directive =
|
||||
throw new UnsupportedOperationException("Cannot push the boundary")
|
||||
|
||||
def onNext(elem: Any): Unit = enterAndPush(elem)
|
||||
def onComplete(): Unit = enterAndFinish()
|
||||
def onNextAndComplete(elem: Any): Unit = {
|
||||
context.enter()
|
||||
context.pushAndFinish(elem)
|
||||
context.execute()
|
||||
}
|
||||
def onError(cause: Throwable): Unit = enterAndFail(cause)
|
||||
|
||||
}
|
||||
|
||||
private[akka] class DownstreamProbe extends BoundaryStage {
|
||||
override def onPush(elem: Any, ctx: BoundaryContext): Directive = {
|
||||
lastEvent += OnNext(elem)
|
||||
ctx.exit()
|
||||
}
|
||||
|
||||
override def onUpstreamFinish(ctx: BoundaryContext): TerminationDirective = {
|
||||
lastEvent += OnComplete
|
||||
ctx.finish()
|
||||
}
|
||||
|
||||
override def onUpstreamFailure(cause: Throwable, ctx: BoundaryContext): TerminationDirective = {
|
||||
lastEvent += OnError(cause)
|
||||
ctx.finish()
|
||||
}
|
||||
|
||||
override def onPull(ctx: BoundaryContext): Directive =
|
||||
throw new UnsupportedOperationException("Cannot pull the boundary")
|
||||
|
||||
def requestOne(): Unit = enterAndPull()
|
||||
|
||||
def cancel(): Unit = enterAndFinish()
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
@ -3,18 +3,27 @@
|
|||
*/
|
||||
package akka.stream.impl.fusing
|
||||
|
||||
import akka.stream.Supervision
|
||||
import akka.stream.{ Attributes, Shape, Supervision }
|
||||
import akka.stream.stage.AbstractStage.PushPullGraphStage
|
||||
import akka.stream.stage.GraphStageWithMaterializedValue
|
||||
import akka.stream.testkit.AkkaSpec
|
||||
|
||||
class InterpreterStressSpec extends InterpreterSpecKit {
|
||||
class InterpreterStressSpec extends AkkaSpec with GraphInterpreterSpecKit {
|
||||
import Supervision.stoppingDecider
|
||||
|
||||
val chainLength = 1000 * 1000
|
||||
val halfLength = chainLength / 2
|
||||
val repetition = 100
|
||||
|
||||
val f = (x: Int) ⇒ x + 1
|
||||
|
||||
val map: GraphStageWithMaterializedValue[Shape, Any] =
|
||||
new PushPullGraphStage[Int, Int, Unit]((_) ⇒ Map(f, stoppingDecider), Attributes.none)
|
||||
.asInstanceOf[GraphStageWithMaterializedValue[Shape, Any]]
|
||||
|
||||
"Interpreter" must {
|
||||
|
||||
"work with a massive chain of maps" in new TestSetup(Seq.fill(chainLength)(Map((x: Int) ⇒ x + 1, stoppingDecider))) {
|
||||
"work with a massive chain of maps" in new OneBoundedSetup[Int](Array.fill(chainLength)(map).asInstanceOf[Array[GraphStageWithMaterializedValue[Shape, Any]]]) {
|
||||
lastEvents() should be(Set.empty)
|
||||
val tstamp = System.nanoTime()
|
||||
|
||||
|
|
@ -32,11 +41,11 @@ class InterpreterStressSpec extends InterpreterSpecKit {
|
|||
lastEvents() should be(Set(OnComplete))
|
||||
|
||||
val time = (System.nanoTime() - tstamp) / (1000.0 * 1000.0 * 1000.0)
|
||||
// FIXME: Not a real benchmark, should be replaced by a proper JMH bench
|
||||
// Not a real benchmark, just for sanity check
|
||||
info(s"Chain finished in $time seconds ${(chainLength * repetition) / (time * 1000 * 1000)} million maps/s")
|
||||
}
|
||||
|
||||
"work with a massive chain of maps with early complete" in new TestSetup(Seq.fill(halfLength)(Map((x: Int) ⇒ x + 1, stoppingDecider)) ++
|
||||
"work with a massive chain of maps with early complete" in new OneBoundedSetup[Int](Iterable.fill(halfLength)(Map((x: Int) ⇒ x + 1, stoppingDecider)) ++
|
||||
Seq(Take(repetition / 2)) ++
|
||||
Seq.fill(halfLength)(Map((x: Int) ⇒ x + 1, stoppingDecider))) {
|
||||
|
||||
|
|
@ -60,11 +69,11 @@ class InterpreterStressSpec extends InterpreterSpecKit {
|
|||
lastEvents() should be(Set(Cancel, OnComplete, OnNext(0 + chainLength)))
|
||||
|
||||
val time = (System.nanoTime() - tstamp) / (1000.0 * 1000.0 * 1000.0)
|
||||
// FIXME: Not a real benchmark, should be replaced by a proper JMH bench
|
||||
// Not a real benchmark, just for sanity check
|
||||
info(s"Chain finished in $time seconds ${(chainLength * repetition) / (time * 1000 * 1000)} million maps/s")
|
||||
}
|
||||
|
||||
"work with a massive chain of takes" in new TestSetup(Seq.fill(chainLength)(Take(1))) {
|
||||
"work with a massive chain of takes" in new OneBoundedSetup[Int](Iterable.fill(chainLength)(Take(1))) {
|
||||
lastEvents() should be(Set.empty)
|
||||
|
||||
downstream.requestOne()
|
||||
|
|
@ -75,7 +84,7 @@ class InterpreterStressSpec extends InterpreterSpecKit {
|
|||
|
||||
}
|
||||
|
||||
"work with a massive chain of drops" in new TestSetup(Seq.fill(chainLength / 1000)(Drop(1))) {
|
||||
"work with a massive chain of drops" in new OneBoundedSetup[Int](Iterable.fill(chainLength / 1000)(Drop(1))) {
|
||||
lastEvents() should be(Set.empty)
|
||||
|
||||
downstream.requestOne()
|
||||
|
|
@ -93,12 +102,10 @@ class InterpreterStressSpec extends InterpreterSpecKit {
|
|||
|
||||
}
|
||||
|
||||
"work with a massive chain of conflates by overflowing to the heap" in new TestSetup(Seq.fill(100000)(Conflate(
|
||||
"work with a massive chain of conflates by overflowing to the heap" in new OneBoundedSetup[Int](Iterable.fill(100000)(Conflate(
|
||||
(in: Int) ⇒ in,
|
||||
(agg: Int, in: Int) ⇒ agg + in,
|
||||
Supervision.stoppingDecider)),
|
||||
forkLimit = 100,
|
||||
overflowToHeap = true) {
|
||||
Supervision.stoppingDecider))) {
|
||||
|
||||
lastEvents() should be(Set(RequestOne))
|
||||
|
||||
|
|
|
|||
|
|
@ -70,7 +70,7 @@ object InterpreterSupervisionSpec {
|
|||
|
||||
}
|
||||
|
||||
class InterpreterSupervisionSpec extends InterpreterSpecKit {
|
||||
class InterpreterSupervisionSpec extends GraphInterpreterSpecKit {
|
||||
import InterpreterSupervisionSpec._
|
||||
import Supervision.stoppingDecider
|
||||
import Supervision.resumingDecider
|
||||
|
|
@ -78,7 +78,7 @@ class InterpreterSupervisionSpec extends InterpreterSpecKit {
|
|||
|
||||
"Interpreter error handling" must {
|
||||
|
||||
"handle external failure" in new TestSetup(Seq(Map((x: Int) ⇒ x + 1, stoppingDecider))) {
|
||||
"handle external failure" in new OneBoundedSetup[Int](Seq(Map((x: Int) ⇒ x + 1, stoppingDecider))) {
|
||||
lastEvents() should be(Set.empty)
|
||||
|
||||
upstream.onError(TE)
|
||||
|
|
@ -86,7 +86,7 @@ class InterpreterSupervisionSpec extends InterpreterSpecKit {
|
|||
|
||||
}
|
||||
|
||||
"emit failure when op throws" in new TestSetup(Seq(Map((x: Int) ⇒ if (x == 0) throw TE else x, stoppingDecider))) {
|
||||
"emit failure when op throws" in new OneBoundedSetup[Int](Seq(Map((x: Int) ⇒ if (x == 0) throw TE else x, stoppingDecider))) {
|
||||
downstream.requestOne()
|
||||
lastEvents() should be(Set(RequestOne))
|
||||
upstream.onNext(2)
|
||||
|
|
@ -98,7 +98,7 @@ class InterpreterSupervisionSpec extends InterpreterSpecKit {
|
|||
lastEvents() should be(Set(Cancel, OnError(TE)))
|
||||
}
|
||||
|
||||
"emit failure when op throws in middle of the chain" in new TestSetup(Seq(
|
||||
"emit failure when op throws in middle of the chain" in new OneBoundedSetup[Int](Seq(
|
||||
Map((x: Int) ⇒ x + 1, stoppingDecider),
|
||||
Map((x: Int) ⇒ if (x == 0) throw TE else x + 10, stoppingDecider),
|
||||
Map((x: Int) ⇒ x + 100, stoppingDecider))) {
|
||||
|
|
@ -114,7 +114,7 @@ class InterpreterSupervisionSpec extends InterpreterSpecKit {
|
|||
lastEvents() should be(Set(Cancel, OnError(TE)))
|
||||
}
|
||||
|
||||
"resume when Map throws" in new TestSetup(Seq(Map((x: Int) ⇒ if (x == 0) throw TE else x, resumingDecider))) {
|
||||
"resume when Map throws" in new OneBoundedSetup[Int](Seq(Map((x: Int) ⇒ if (x == 0) throw TE else x, resumingDecider))) {
|
||||
downstream.requestOne()
|
||||
lastEvents() should be(Set(RequestOne))
|
||||
upstream.onNext(2)
|
||||
|
|
@ -138,7 +138,7 @@ class InterpreterSupervisionSpec extends InterpreterSpecKit {
|
|||
lastEvents() should be(Set(OnNext(4)))
|
||||
}
|
||||
|
||||
"resume when Map throws in middle of the chain" in new TestSetup(Seq(
|
||||
"resume when Map throws in middle of the chain" in new OneBoundedSetup[Int](Seq(
|
||||
Map((x: Int) ⇒ x + 1, resumingDecider),
|
||||
Map((x: Int) ⇒ if (x == 0) throw TE else x + 10, resumingDecider),
|
||||
Map((x: Int) ⇒ x + 100, resumingDecider))) {
|
||||
|
|
@ -157,7 +157,7 @@ class InterpreterSupervisionSpec extends InterpreterSpecKit {
|
|||
lastEvents() should be(Set(OnNext(114)))
|
||||
}
|
||||
|
||||
"resume when Map throws before Grouped" in new TestSetup(Seq(
|
||||
"resume when Map throws before Grouped" in new OneBoundedSetup[Int](Seq(
|
||||
Map((x: Int) ⇒ x + 1, resumingDecider),
|
||||
Map((x: Int) ⇒ if (x <= 0) throw TE else x + 10, resumingDecider),
|
||||
Grouped(3))) {
|
||||
|
|
@ -177,7 +177,7 @@ class InterpreterSupervisionSpec extends InterpreterSpecKit {
|
|||
lastEvents() should be(Set(OnNext(Vector(13, 14, 15))))
|
||||
}
|
||||
|
||||
"complete after resume when Map throws before Grouped" in new TestSetup(Seq(
|
||||
"complete after resume when Map throws before Grouped" in new OneBoundedSetup[Int](Seq(
|
||||
Map((x: Int) ⇒ x + 1, resumingDecider),
|
||||
Map((x: Int) ⇒ if (x <= 0) throw TE else x + 10, resumingDecider),
|
||||
Grouped(1000))) {
|
||||
|
|
@ -205,7 +205,7 @@ class InterpreterSupervisionSpec extends InterpreterSpecKit {
|
|||
}
|
||||
}
|
||||
|
||||
new TestSetup(Seq(
|
||||
new OneBoundedSetup[Int](Seq(
|
||||
Map((x: Int) ⇒ x + 1, restartingDecider),
|
||||
stage,
|
||||
Map((x: Int) ⇒ x + 100, restartingDecider))) {
|
||||
|
|
@ -228,14 +228,13 @@ class InterpreterSupervisionSpec extends InterpreterSpecKit {
|
|||
"restart when onPush throws after ctx.push" in {
|
||||
val stage = new RestartTestStage {
|
||||
override def onPush(elem: Int, ctx: Context[Int]): SyncDirective = {
|
||||
val ret = ctx.push(sum)
|
||||
super.onPush(elem, ctx)
|
||||
val ret = ctx.push(elem)
|
||||
if (elem <= 0) throw TE
|
||||
ret
|
||||
}
|
||||
}
|
||||
|
||||
new TestSetup(Seq(
|
||||
new OneBoundedSetup[Int](Seq(
|
||||
Map((x: Int) ⇒ x + 1, restartingDecider),
|
||||
stage,
|
||||
Map((x: Int) ⇒ x + 100, restartingDecider))) {
|
||||
|
|
@ -248,6 +247,10 @@ class InterpreterSupervisionSpec extends InterpreterSpecKit {
|
|||
downstream.requestOne()
|
||||
lastEvents() should be(Set(RequestOne))
|
||||
upstream.onNext(-1) // boom
|
||||
// The element has been pushed before the exception, there is no way back
|
||||
lastEvents() should be(Set(OnNext(100)))
|
||||
|
||||
downstream.requestOne()
|
||||
lastEvents() should be(Set(RequestOne))
|
||||
|
||||
upstream.onNext(3)
|
||||
|
|
@ -263,7 +266,7 @@ class InterpreterSupervisionSpec extends InterpreterSpecKit {
|
|||
}
|
||||
}
|
||||
|
||||
new TestSetup(Seq(
|
||||
new OneBoundedSetup[Int](Seq(
|
||||
Map((x: Int) ⇒ x + 1, restartingDecider),
|
||||
stage,
|
||||
Map((x: Int) ⇒ x + 100, restartingDecider))) {
|
||||
|
|
@ -283,7 +286,7 @@ class InterpreterSupervisionSpec extends InterpreterSpecKit {
|
|||
}
|
||||
}
|
||||
|
||||
"resume when Filter throws" in new TestSetup(Seq(
|
||||
"resume when Filter throws" in new OneBoundedSetup[Int](Seq(
|
||||
Filter((x: Int) ⇒ if (x == 0) throw TE else true, resumingDecider))) {
|
||||
downstream.requestOne()
|
||||
lastEvents() should be(Set(RequestOne))
|
||||
|
|
@ -299,7 +302,7 @@ class InterpreterSupervisionSpec extends InterpreterSpecKit {
|
|||
lastEvents() should be(Set(OnNext(3)))
|
||||
}
|
||||
|
||||
"resume when MapConcat throws" in new TestSetup(Seq(
|
||||
"resume when MapConcat throws" in new OneBoundedSetup[Int](Seq(
|
||||
MapConcat((x: Int) ⇒ if (x == 0) throw TE else List(x, -x), resumingDecider))) {
|
||||
downstream.requestOne()
|
||||
lastEvents() should be(Set(RequestOne))
|
||||
|
|
@ -323,7 +326,7 @@ class InterpreterSupervisionSpec extends InterpreterSpecKit {
|
|||
// TODO can't get type inference to work with `pf` inlined
|
||||
val pf: PartialFunction[Int, Int] =
|
||||
{ case x: Int ⇒ if (x == 0) throw TE else x }
|
||||
new TestSetup(Seq(
|
||||
new OneBoundedSetup[Int](Seq(
|
||||
Collect(pf, restartingDecider))) {
|
||||
downstream.requestOne()
|
||||
lastEvents() should be(Set(RequestOne))
|
||||
|
|
@ -340,7 +343,7 @@ class InterpreterSupervisionSpec extends InterpreterSpecKit {
|
|||
}
|
||||
}
|
||||
|
||||
"resume when Scan throws" in new TestSetup(Seq(
|
||||
"resume when Scan throws" in new OneBoundedSetup[Int](Seq(
|
||||
Scan(1, (acc: Int, x: Int) ⇒ if (x == 10) throw TE else acc + x, resumingDecider))) {
|
||||
downstream.requestOne()
|
||||
lastEvents() should be(Set(OnNext(1)))
|
||||
|
|
@ -358,7 +361,7 @@ class InterpreterSupervisionSpec extends InterpreterSpecKit {
|
|||
lastEvents() should be(Set(OnNext(7))) // 1 + 2 + 4
|
||||
}
|
||||
|
||||
"restart when Scan throws" in new TestSetup(Seq(
|
||||
"restart when Scan throws" in new OneBoundedSetup[Int](Seq(
|
||||
Scan(1, (acc: Int, x: Int) ⇒ if (x == 10) throw TE else acc + x, restartingDecider))) {
|
||||
downstream.requestOne()
|
||||
lastEvents() should be(Set(OnNext(1)))
|
||||
|
|
@ -383,7 +386,7 @@ class InterpreterSupervisionSpec extends InterpreterSpecKit {
|
|||
lastEvents() should be(Set(OnNext(25))) // 1 + 4 + 20
|
||||
}
|
||||
|
||||
"restart when Conflate `seed` throws" in new TestSetup(Seq(Conflate(
|
||||
"restart when Conflate `seed` throws" in new OneBoundedSetup[Int](Seq(Conflate(
|
||||
(in: Int) ⇒ if (in == 1) throw TE else in,
|
||||
(agg: Int, x: Int) ⇒ agg + x,
|
||||
restartingDecider))) {
|
||||
|
|
@ -412,7 +415,7 @@ class InterpreterSupervisionSpec extends InterpreterSpecKit {
|
|||
lastEvents() should be(Set.empty)
|
||||
}
|
||||
|
||||
"restart when Conflate `aggregate` throws" in new TestSetup(Seq(Conflate(
|
||||
"restart when Conflate `aggregate` throws" in new OneBoundedSetup[Int](Seq(Conflate(
|
||||
(in: Int) ⇒ in,
|
||||
(agg: Int, x: Int) ⇒ if (x == 2) throw TE else agg + x,
|
||||
restartingDecider))) {
|
||||
|
|
@ -447,7 +450,7 @@ class InterpreterSupervisionSpec extends InterpreterSpecKit {
|
|||
lastEvents() should be(Set(Cancel))
|
||||
}
|
||||
|
||||
"fail when Expand `seed` throws" in new TestSetup(Seq(Expand(
|
||||
"fail when Expand `seed` throws" in new OneBoundedSetup[Int](Seq(Expand(
|
||||
(in: Int) ⇒ if (in == 2) throw TE else in,
|
||||
(agg: Int) ⇒ (agg, -math.abs(agg))))) {
|
||||
|
||||
|
|
@ -469,7 +472,7 @@ class InterpreterSupervisionSpec extends InterpreterSpecKit {
|
|||
lastEvents() should be(Set(OnError(TE), Cancel))
|
||||
}
|
||||
|
||||
"fail when Expand `extrapolate` throws" in new TestSetup(Seq(Expand(
|
||||
"fail when Expand `extrapolate` throws" in new OneBoundedSetup[Int](Seq(Expand(
|
||||
(in: Int) ⇒ in,
|
||||
(agg: Int) ⇒ if (agg == 2) throw TE else (agg, -math.abs(agg))))) {
|
||||
|
||||
|
|
@ -493,7 +496,7 @@ class InterpreterSupervisionSpec extends InterpreterSpecKit {
|
|||
|
||||
"fail when onPull throws before pushing all generated elements" in {
|
||||
def test(decider: Supervision.Decider, absorbTermination: Boolean): Unit = {
|
||||
new TestSetup(Seq(
|
||||
new OneBoundedSetup[Int](Seq(
|
||||
OneToManyTestStage(decider, absorbTermination))) {
|
||||
|
||||
downstream.requestOne()
|
||||
|
|
|
|||
|
|
@ -3,16 +3,18 @@
|
|||
*/
|
||||
package akka.stream.impl.fusing
|
||||
|
||||
import akka.stream.stage._
|
||||
import akka.stream.testkit.AkkaSpec
|
||||
import akka.stream.testkit.Utils.TE
|
||||
|
||||
import scala.concurrent.duration._
|
||||
|
||||
class LifecycleInterpreterSpec extends InterpreterSpecKit {
|
||||
class LifecycleInterpreterSpec extends GraphInterpreterSpecKit {
|
||||
import akka.stream.Supervision._
|
||||
|
||||
"Interpreter" must {
|
||||
|
||||
"call preStart in order on stages" in new TestSetup(Seq(
|
||||
"call preStart in order on stages" in new OneBoundedSetup[String](Seq(
|
||||
PreStartAndPostStopIdentity(onStart = _ ⇒ testActor ! "start-a"),
|
||||
PreStartAndPostStopIdentity(onStart = _ ⇒ testActor ! "start-b"),
|
||||
PreStartAndPostStopIdentity(onStart = _ ⇒ testActor ! "start-c"))) {
|
||||
|
|
@ -23,7 +25,7 @@ class LifecycleInterpreterSpec extends InterpreterSpecKit {
|
|||
upstream.onComplete()
|
||||
}
|
||||
|
||||
"call postStop in order on stages - when upstream completes" in new TestSetup(Seq(
|
||||
"call postStop in order on stages - when upstream completes" in new OneBoundedSetup[String](Seq(
|
||||
PreStartAndPostStopIdentity(onUpstreamCompleted = () ⇒ testActor ! "complete-a", onStop = () ⇒ testActor ! "stop-a"),
|
||||
PreStartAndPostStopIdentity(onUpstreamCompleted = () ⇒ testActor ! "complete-b", onStop = () ⇒ testActor ! "stop-b"),
|
||||
PreStartAndPostStopIdentity(onUpstreamCompleted = () ⇒ testActor ! "complete-c", onStop = () ⇒ testActor ! "stop-c"))) {
|
||||
|
|
@ -37,7 +39,7 @@ class LifecycleInterpreterSpec extends InterpreterSpecKit {
|
|||
expectNoMsg(300.millis)
|
||||
}
|
||||
|
||||
"call postStop in order on stages - when upstream onErrors" in new TestSetup(Seq(
|
||||
"call postStop in order on stages - when upstream onErrors" in new OneBoundedSetup[String](Seq(
|
||||
PreStartAndPostStopIdentity(
|
||||
onUpstreamFailed = ex ⇒ testActor ! ex.getMessage,
|
||||
onStop = () ⇒ testActor ! "stop-c"))) {
|
||||
|
|
@ -48,7 +50,7 @@ class LifecycleInterpreterSpec extends InterpreterSpecKit {
|
|||
expectNoMsg(300.millis)
|
||||
}
|
||||
|
||||
"call postStop in order on stages - when downstream cancels" in new TestSetup(Seq(
|
||||
"call postStop in order on stages - when downstream cancels" in new OneBoundedSetup[String](Seq(
|
||||
PreStartAndPostStopIdentity(onStop = () ⇒ testActor ! "stop-a"),
|
||||
PreStartAndPostStopIdentity(onStop = () ⇒ testActor ! "stop-b"),
|
||||
PreStartAndPostStopIdentity(onStop = () ⇒ testActor ! "stop-c"))) {
|
||||
|
|
@ -59,7 +61,7 @@ class LifecycleInterpreterSpec extends InterpreterSpecKit {
|
|||
expectNoMsg(300.millis)
|
||||
}
|
||||
|
||||
"call preStart before postStop" in new TestSetup(Seq(
|
||||
"call preStart before postStop" in new OneBoundedSetup[String](Seq(
|
||||
PreStartAndPostStopIdentity(onStart = _ ⇒ testActor ! "start-a", onStop = () ⇒ testActor ! "stop-a"))) {
|
||||
expectMsg("start-a")
|
||||
expectNoMsg(300.millis)
|
||||
|
|
@ -68,25 +70,25 @@ class LifecycleInterpreterSpec extends InterpreterSpecKit {
|
|||
expectNoMsg(300.millis)
|
||||
}
|
||||
|
||||
"onError when preStart fails" in new TestSetup(Seq(
|
||||
"onError when preStart fails" in new OneBoundedSetup[String](Seq(
|
||||
PreStartFailer(() ⇒ throw TE("Boom!")))) {
|
||||
lastEvents() should ===(Set(Cancel, OnError(TE("Boom!"))))
|
||||
}
|
||||
|
||||
"not blow up when postStop fails" in new TestSetup(Seq(
|
||||
"not blow up when postStop fails" in new OneBoundedSetup[String](Seq(
|
||||
PostStopFailer(() ⇒ throw TE("Boom!")))) {
|
||||
upstream.onComplete()
|
||||
lastEvents() should ===(Set(OnComplete))
|
||||
}
|
||||
|
||||
"onError when preStart fails with stages after" in new TestSetup(Seq(
|
||||
"onError when preStart fails with stages after" in new OneBoundedSetup[String](Seq(
|
||||
Map((x: Int) ⇒ x, stoppingDecider),
|
||||
PreStartFailer(() ⇒ throw TE("Boom!")),
|
||||
Map((x: Int) ⇒ x, stoppingDecider))) {
|
||||
lastEvents() should ===(Set(Cancel, OnError(TE("Boom!"))))
|
||||
}
|
||||
|
||||
"continue with stream shutdown when postStop fails" in new TestSetup(Seq(
|
||||
"continue with stream shutdown when postStop fails" in new OneBoundedSetup[String](Seq(
|
||||
PostStopFailer(() ⇒ throw TE("Boom!")))) {
|
||||
lastEvents() should ===(Set())
|
||||
|
||||
|
|
@ -94,7 +96,7 @@ class LifecycleInterpreterSpec extends InterpreterSpecKit {
|
|||
lastEvents should ===(Set(OnComplete))
|
||||
}
|
||||
|
||||
"postStop when pushAndFinish called if upstream completes with pushAndFinish" in new TestSetup(Seq(
|
||||
"postStop when pushAndFinish called if upstream completes with pushAndFinish" in new OneBoundedSetup[String](Seq(
|
||||
new PushFinishStage(onPostStop = () ⇒ testActor ! "stop"))) {
|
||||
|
||||
lastEvents() should be(Set.empty)
|
||||
|
|
@ -107,7 +109,7 @@ class LifecycleInterpreterSpec extends InterpreterSpecKit {
|
|||
expectMsg("stop")
|
||||
}
|
||||
|
||||
"postStop when pushAndFinish called with pushAndFinish if indirect upstream completes with pushAndFinish" in new TestSetup(Seq(
|
||||
"postStop when pushAndFinish called with pushAndFinish if indirect upstream completes with pushAndFinish" in new OneBoundedSetup[String](Seq(
|
||||
Map((x: Any) ⇒ x, stoppingDecider),
|
||||
new PushFinishStage(onPostStop = () ⇒ testActor ! "stop"),
|
||||
Map((x: Any) ⇒ x, stoppingDecider))) {
|
||||
|
|
@ -122,7 +124,7 @@ class LifecycleInterpreterSpec extends InterpreterSpecKit {
|
|||
expectMsg("stop")
|
||||
}
|
||||
|
||||
"postStop when pushAndFinish called with pushAndFinish if upstream completes with pushAndFinish and downstream immediately pulls" in new TestSetup(Seq(
|
||||
"postStop when pushAndFinish called with pushAndFinish if upstream completes with pushAndFinish and downstream immediately pulls" in new OneBoundedSetup[String](Seq(
|
||||
new PushFinishStage(onPostStop = () ⇒ testActor ! "stop"),
|
||||
Fold("", (x: String, y: String) ⇒ x + y, stoppingDecider))) {
|
||||
|
||||
|
|
@ -138,4 +140,54 @@ class LifecycleInterpreterSpec extends InterpreterSpecKit {
|
|||
|
||||
}
|
||||
|
||||
private[akka] case class PreStartAndPostStopIdentity[T](
|
||||
onStart: LifecycleContext ⇒ Unit = _ ⇒ (),
|
||||
onStop: () ⇒ Unit = () ⇒ (),
|
||||
onUpstreamCompleted: () ⇒ Unit = () ⇒ (),
|
||||
onUpstreamFailed: Throwable ⇒ Unit = ex ⇒ ())
|
||||
extends PushStage[T, T] {
|
||||
override def preStart(ctx: LifecycleContext) = onStart(ctx)
|
||||
|
||||
override def onPush(elem: T, ctx: Context[T]) = ctx.push(elem)
|
||||
|
||||
override def onUpstreamFinish(ctx: Context[T]): TerminationDirective = {
|
||||
onUpstreamCompleted()
|
||||
super.onUpstreamFinish(ctx)
|
||||
}
|
||||
|
||||
override def onUpstreamFailure(cause: Throwable, ctx: Context[T]): TerminationDirective = {
|
||||
onUpstreamFailed(cause)
|
||||
super.onUpstreamFailure(cause, ctx)
|
||||
}
|
||||
|
||||
override def postStop() = onStop()
|
||||
}
|
||||
|
||||
private[akka] case class PreStartFailer[T](pleaseThrow: () ⇒ Unit) extends PushStage[T, T] {
|
||||
|
||||
override def preStart(ctx: LifecycleContext) =
|
||||
pleaseThrow()
|
||||
|
||||
override def onPush(elem: T, ctx: Context[T]) = ctx.push(elem)
|
||||
}
|
||||
|
||||
private[akka] case class PostStopFailer[T](ex: () ⇒ Throwable) extends PushStage[T, T] {
|
||||
override def onUpstreamFinish(ctx: Context[T]) = ctx.finish()
|
||||
override def onPush(elem: T, ctx: Context[T]) = ctx.push(elem)
|
||||
|
||||
override def postStop(): Unit = throw ex()
|
||||
}
|
||||
|
||||
// This test is related to issue #17351
|
||||
private[akka] class PushFinishStage(onPostStop: () ⇒ Unit = () ⇒ ()) extends PushStage[Any, Any] {
|
||||
override def onPush(elem: Any, ctx: Context[Any]): SyncDirective =
|
||||
ctx.pushAndFinish(elem)
|
||||
|
||||
override def onUpstreamFinish(ctx: Context[Any]): TerminationDirective =
|
||||
ctx.fail(akka.stream.testkit.Utils.TE("Cannot happen"))
|
||||
|
||||
override def postStop(): Unit =
|
||||
onPostStop()
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -55,10 +55,10 @@ class InputStreamSinkSpec extends AkkaSpec(UnboundedMailboxConfig) {
|
|||
class InputStreamSinkTestStage(val timeout: FiniteDuration)
|
||||
extends InputStreamSinkStage(timeout) {
|
||||
|
||||
override def createLogicAndMaterializedValue = {
|
||||
val (logic, inputStream) = super.createLogicAndMaterializedValue
|
||||
val inHandler = logic.inHandlers(in.id)
|
||||
logic.inHandlers(in.id) = new InHandler {
|
||||
override def createLogicAndMaterializedValue(inheritedAttributes: Attributes) = {
|
||||
val (logic, inputStream) = super.createLogicAndMaterializedValue(inheritedAttributes)
|
||||
val inHandler = logic.handlers(in.id).asInstanceOf[InHandler]
|
||||
logic.handlers(in.id) = new InHandler {
|
||||
override def onPush(): Unit = {
|
||||
probe.ref ! InputStreamSinkTestMessages.Push
|
||||
inHandler.onPush()
|
||||
|
|
|
|||
|
|
@ -49,10 +49,10 @@ class OutputStreamSourceSpec extends AkkaSpec(UnboundedMailboxConfig) {
|
|||
class OutputStreamSourceTestStage(val timeout: FiniteDuration)
|
||||
extends OutputStreamSourceStage(timeout) {
|
||||
|
||||
override def createLogicAndMaterializedValue = {
|
||||
val (logic, inputStream) = super.createLogicAndMaterializedValue
|
||||
val outHandler = logic.outHandlers(out.id)
|
||||
logic.outHandlers(out.id) = new OutHandler {
|
||||
override def createLogicAndMaterializedValue(inheritedAttributes: Attributes) = {
|
||||
val (logic, inputStream) = super.createLogicAndMaterializedValue(inheritedAttributes)
|
||||
val outHandler = logic.handlers(out.id).asInstanceOf[OutHandler]
|
||||
logic.handlers(out.id) = new OutHandler {
|
||||
override def onDownstreamFinish(): Unit = {
|
||||
probe.ref ! OutputStreamSourceTestMessages.Finish
|
||||
outHandler.onDownstreamFinish()
|
||||
|
|
|
|||
|
|
@ -107,7 +107,9 @@ class SynchronousFileSinkSpec extends AkkaSpec(UnboundedMailboxConfig) {
|
|||
}
|
||||
}
|
||||
|
||||
// FIXME: overriding dispatcher should be made available with dispatcher alias support in materializer (#17929)
|
||||
"allow overriding the dispatcher using Attributes" in assertAllStagesStopped {
|
||||
pending
|
||||
targetFile { f ⇒
|
||||
val sys = ActorSystem("dispatcher-testing", UnboundedMailboxConfig)
|
||||
val mat = ActorMaterializer()(sys)
|
||||
|
|
|
|||
|
|
@ -180,7 +180,9 @@ class SynchronousFileSourceSpec extends AkkaSpec(UnboundedMailboxConfig) {
|
|||
} finally shutdown(sys)
|
||||
}
|
||||
|
||||
//FIXME: overriding dispatcher should be made available with dispatcher alias support in materializer (#17929)
|
||||
"allow overriding the dispatcher using Attributes" in {
|
||||
pending
|
||||
val sys = ActorSystem("dispatcher-testing", UnboundedMailboxConfig)
|
||||
val mat = ActorMaterializer()(sys)
|
||||
implicit val timeout = Timeout(500.millis)
|
||||
|
|
|
|||
|
|
@ -13,10 +13,11 @@ import akka.stream.testkit.Utils._
|
|||
import akka.stream.testkit._
|
||||
import akka.stream.{ ActorMaterializer, BindFailedException, StreamTcpException }
|
||||
import akka.util.{ ByteString, Helpers }
|
||||
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.{ Promise, Await }
|
||||
import scala.concurrent.duration._
|
||||
import java.net.BindException
|
||||
import akka.testkit.EventFilter
|
||||
|
||||
class TcpSpec extends AkkaSpec("akka.io.tcp.windows-connection-abort-workaround-enabled=auto\nakka.stream.materializer.subscription-timeout.timeout = 3s") with TcpHelper {
|
||||
var demand = 0L
|
||||
|
|
@ -350,12 +351,14 @@ class TcpSpec extends AkkaSpec("akka.io.tcp.windows-connection-abort-workaround-
|
|||
conn.flow.join(writeButIgnoreRead).run()
|
||||
})(Keep.left).run(), 3.seconds)
|
||||
|
||||
val result = Source.maybe[ByteString]
|
||||
val (promise, result) = Source.maybe[ByteString]
|
||||
.via(Tcp().outgoingConnection(serverAddress.getHostName, serverAddress.getPort))
|
||||
.runFold(ByteString.empty)(_ ++ _)
|
||||
.toMat(Sink.fold(ByteString.empty)(_ ++ _))(Keep.both)
|
||||
.run()
|
||||
|
||||
Await.result(result, 3.seconds) should ===(ByteString("Early response"))
|
||||
|
||||
promise.success(None) // close client upstream, no more data
|
||||
binding.unbind()
|
||||
}
|
||||
|
||||
|
|
@ -453,7 +456,7 @@ class TcpSpec extends AkkaSpec("akka.io.tcp.windows-connection-abort-workaround-
|
|||
Await.result(echoServerFinish, 1.second)
|
||||
}
|
||||
|
||||
"bind and unbind correctly" in {
|
||||
"bind and unbind correctly" in EventFilter[BindException](occurrences = 2).intercept {
|
||||
if (Helpers.isWindows) {
|
||||
info("On Windows unbinding is not immediate")
|
||||
pending
|
||||
|
|
|
|||
|
|
@ -12,7 +12,7 @@ import scala.util.Random
|
|||
|
||||
import akka.actor.ActorSystem
|
||||
import akka.pattern.{ after ⇒ later }
|
||||
import akka.stream.{ ClosedShape, ActorMaterializer }
|
||||
import akka.stream._
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.stage._
|
||||
import akka.stream.testkit._
|
||||
|
|
@ -52,35 +52,28 @@ object TlsSpec {
|
|||
* independent of the traffic going through. The purpose is to include the last seen
|
||||
* element in the exception message to help in figuring out what went wrong.
|
||||
*/
|
||||
class Timeout(duration: FiniteDuration)(implicit system: ActorSystem) extends AsyncStage[ByteString, ByteString, Unit] {
|
||||
private var last: ByteString = _
|
||||
class Timeout(duration: FiniteDuration)(implicit system: ActorSystem) extends GraphStage[FlowShape[ByteString, ByteString]] {
|
||||
|
||||
override def preStart(ctx: AsyncContext[ByteString, Unit]) = {
|
||||
val cb = ctx.getAsyncCallback
|
||||
system.scheduler.scheduleOnce(duration)(cb.invoke(()))(system.dispatcher)
|
||||
}
|
||||
private val in = Inlet[ByteString]("in")
|
||||
private val out = Outlet[ByteString]("out")
|
||||
override val shape = FlowShape(in, out)
|
||||
|
||||
override def onAsyncInput(u: Unit, ctx: AsyncContext[ByteString, Unit]) =
|
||||
ctx.fail(new TimeoutException(s"timeout expired, last element was $last"))
|
||||
override def createLogic(attr: Attributes) = new TimerGraphStageLogic(shape) {
|
||||
override def preStart(): Unit = scheduleOnce((), duration)
|
||||
|
||||
override def onPush(elem: ByteString, ctx: AsyncContext[ByteString, Unit]) = {
|
||||
last = elem
|
||||
if (ctx.isHoldingDownstream) ctx.pushAndPull(elem)
|
||||
else ctx.holdUpstream()
|
||||
}
|
||||
|
||||
override def onPull(ctx: AsyncContext[ByteString, Unit]) =
|
||||
if (ctx.isFinishing) ctx.pushAndFinish(last)
|
||||
else if (ctx.isHoldingUpstream) ctx.pushAndPull(last)
|
||||
else ctx.holdDownstream()
|
||||
|
||||
override def onUpstreamFinish(ctx: AsyncContext[ByteString, Unit]) =
|
||||
if (ctx.isHoldingUpstream) ctx.absorbTermination()
|
||||
else ctx.finish()
|
||||
|
||||
override def onDownstreamFinish(ctx: AsyncContext[ByteString, Unit]) = {
|
||||
system.log.debug("cancelled")
|
||||
ctx.finish()
|
||||
var last: ByteString = _
|
||||
setHandler(in, new InHandler {
|
||||
override def onPush(): Unit = {
|
||||
last = grab(in)
|
||||
push(out, last)
|
||||
}
|
||||
})
|
||||
setHandler(out, new OutHandler {
|
||||
override def onPull(): Unit = pull(in)
|
||||
})
|
||||
override def onTimer(x: Any): Unit = {
|
||||
failStage(new TimeoutException(s"timeout expired, last element was $last"))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -363,7 +356,7 @@ class TlsSpec extends AkkaSpec("akka.loglevel=INFO\nakka.actor.debug.receive=off
|
|||
.via(debug)
|
||||
.collect { case SessionBytes(_, b) ⇒ b }
|
||||
.scan(ByteString.empty)(_ ++ _)
|
||||
.transform(() ⇒ new Timeout(6.seconds))
|
||||
.via(new Timeout(6.seconds))
|
||||
.dropWhile(_.size < scenario.output.size)
|
||||
.runWith(Sink.head)
|
||||
|
||||
|
|
|
|||
|
|
@ -0,0 +1,65 @@
|
|||
/**
|
||||
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.stream.scaladsl
|
||||
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.stream.ActorMaterializerSettings
|
||||
import akka.stream.Attributes
|
||||
import akka.stream.Attributes._
|
||||
import akka.stream.MaterializationContext
|
||||
import akka.stream.SinkShape
|
||||
import akka.stream.testkit._
|
||||
import scala.concurrent.Future
|
||||
import scala.concurrent.Promise
|
||||
import akka.stream.impl.SinkModule
|
||||
import akka.stream.impl.StreamLayout.Module
|
||||
import org.scalatest.concurrent.ScalaFutures
|
||||
import akka.stream.impl.BlackholeSubscriber
|
||||
|
||||
object AttributesSpec {
|
||||
|
||||
object AttributesSink {
|
||||
def apply(): Sink[Nothing, Future[Attributes]] =
|
||||
new Sink(new AttributesSink(Attributes.name("attributesSink"), Sink.shape("attributesSink")))
|
||||
}
|
||||
|
||||
final class AttributesSink(val attributes: Attributes, shape: SinkShape[Nothing]) extends SinkModule[Nothing, Future[Attributes]](shape) {
|
||||
override def create(context: MaterializationContext) =
|
||||
(new BlackholeSubscriber(0, Promise()), Future.successful(context.effectiveAttributes))
|
||||
|
||||
override protected def newInstance(shape: SinkShape[Nothing]): SinkModule[Nothing, Future[Attributes]] =
|
||||
new AttributesSink(attributes, shape)
|
||||
|
||||
override def withAttributes(attr: Attributes): Module =
|
||||
new AttributesSink(attr, amendShape(attr))
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
class AttributesSpec extends AkkaSpec with ScalaFutures {
|
||||
import AttributesSpec._
|
||||
|
||||
val settings = ActorMaterializerSettings(system)
|
||||
.withInputBuffer(initialSize = 2, maxSize = 16)
|
||||
|
||||
implicit val materializer = ActorMaterializer(settings)
|
||||
|
||||
"attributes" must {
|
||||
|
||||
"be overridable on a module basis" in {
|
||||
val runnable = Source.empty.toMat(AttributesSink().withAttributes(Attributes.name("new-name")))(Keep.right)
|
||||
whenReady(runnable.run()) { attributes ⇒
|
||||
attributes.get[Name] should contain(Name("new-name"))
|
||||
}
|
||||
}
|
||||
|
||||
"keep the outermost attribute as the least specific" in {
|
||||
val runnable = Source.empty.toMat(AttributesSink())(Keep.right).withAttributes(Attributes.name("new-name"))
|
||||
whenReady(runnable.run()) { attributes ⇒
|
||||
attributes.get[Name] should contain(Name("attributesSink"))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -21,46 +21,13 @@ import scala.util.Try
|
|||
import scala.concurrent.ExecutionContext
|
||||
import scala.util.Failure
|
||||
import scala.util.Success
|
||||
import scala.annotation.tailrec
|
||||
import scala.concurrent.Promise
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import java.util.concurrent.LinkedBlockingQueue
|
||||
import org.scalatest.concurrent.ScalaFutures
|
||||
|
||||
object FlowMapAsyncSpec {
|
||||
class MapAsyncOne[In, Out](f: In ⇒ Future[Out])(implicit ec: ExecutionContext) extends AsyncStage[In, Out, Try[Out]] {
|
||||
private var elemInFlight: Out = _
|
||||
|
||||
override def onPush(elem: In, ctx: AsyncContext[Out, Try[Out]]) = {
|
||||
val future = f(elem)
|
||||
val cb = ctx.getAsyncCallback
|
||||
future.onComplete(cb.invoke)
|
||||
ctx.holdUpstream()
|
||||
}
|
||||
|
||||
override def onPull(ctx: AsyncContext[Out, Try[Out]]) =
|
||||
if (elemInFlight != null) {
|
||||
val e = elemInFlight
|
||||
elemInFlight = null.asInstanceOf[Out]
|
||||
pushIt(e, ctx)
|
||||
} else ctx.holdDownstream()
|
||||
|
||||
override def onAsyncInput(input: Try[Out], ctx: AsyncContext[Out, Try[Out]]) =
|
||||
input match {
|
||||
case Failure(ex) ⇒ ctx.fail(ex)
|
||||
case Success(e) if ctx.isHoldingDownstream ⇒ pushIt(e, ctx)
|
||||
case Success(e) ⇒
|
||||
elemInFlight = e
|
||||
ctx.ignore()
|
||||
}
|
||||
|
||||
override def onUpstreamFinish(ctx: AsyncContext[Out, Try[Out]]) =
|
||||
if (ctx.isHoldingUpstream) ctx.absorbTermination()
|
||||
else ctx.finish()
|
||||
|
||||
private def pushIt(elem: Out, ctx: AsyncContext[Out, Try[Out]]) =
|
||||
if (ctx.isFinishing) ctx.pushAndFinish(elem)
|
||||
else ctx.pushAndPull(elem)
|
||||
}
|
||||
}
|
||||
|
||||
class FlowMapAsyncSpec extends AkkaSpec {
|
||||
import FlowMapAsyncSpec._
|
||||
class FlowMapAsyncSpec extends AkkaSpec with ScalaFutures {
|
||||
|
||||
implicit val materializer = ActorMaterializer()
|
||||
|
||||
|
|
@ -102,11 +69,9 @@ class FlowMapAsyncSpec extends AkkaSpec {
|
|||
n
|
||||
}).to(Sink(c)).run()
|
||||
val sub = c.expectSubscription()
|
||||
// running 8 in parallel
|
||||
probe.receiveN(8).toSet should be((1 to 8).toSet)
|
||||
probe.expectNoMsg(500.millis)
|
||||
sub.request(1)
|
||||
probe.expectMsg(9)
|
||||
probe.receiveN(9).toSet should be((1 to 9).toSet)
|
||||
probe.expectNoMsg(500.millis)
|
||||
sub.request(2)
|
||||
probe.receiveN(2).toSet should be(Set(10, 11))
|
||||
|
|
@ -246,44 +211,50 @@ class FlowMapAsyncSpec extends AkkaSpec {
|
|||
|
||||
}
|
||||
|
||||
}
|
||||
"not run more futures than configured" in assertAllStagesStopped {
|
||||
val parallelism = 8
|
||||
|
||||
"A MapAsyncOne" must {
|
||||
import system.dispatcher
|
||||
val counter = new AtomicInteger
|
||||
val queue = new LinkedBlockingQueue[(Promise[Int], Long)]
|
||||
|
||||
"work in the happy case" in {
|
||||
val probe = TestProbe()
|
||||
val N = 100
|
||||
val f = Source(1 to N).transform(() ⇒ new MapAsyncOne(i ⇒ {
|
||||
probe.ref ! i
|
||||
Future { Thread.sleep(10); probe.ref ! (i + 10); i * 2 }
|
||||
})).grouped(N + 10).runWith(Sink.head)
|
||||
Await.result(f, 2.seconds) should ===((1 to N).map(_ * 2))
|
||||
probe.receiveN(2 * N) should ===((1 to N).flatMap(x ⇒ List(x, x + 10)))
|
||||
probe.expectNoMsg(100.millis)
|
||||
}
|
||||
val timer = new Thread {
|
||||
val delay = 50000 // nanoseconds
|
||||
var count = 0
|
||||
@tailrec final override def run(): Unit = {
|
||||
val cont = try {
|
||||
val (promise, enqueued) = queue.take()
|
||||
val wakeup = enqueued + delay
|
||||
while (System.nanoTime() < wakeup) {}
|
||||
counter.decrementAndGet()
|
||||
promise.success(count)
|
||||
count += 1
|
||||
true
|
||||
} catch {
|
||||
case _: InterruptedException ⇒ false
|
||||
}
|
||||
if (cont) run()
|
||||
}
|
||||
}
|
||||
timer.start
|
||||
|
||||
"work when futures fail" in {
|
||||
val probe = TestSubscriber.manualProbe[Int]
|
||||
val ex = new Exception("KABOOM")
|
||||
Source.single(1)
|
||||
.transform(() ⇒ new MapAsyncOne(_ ⇒ Future.failed(ex)))
|
||||
.runWith(Sink(probe))
|
||||
val sub = probe.expectSubscription()
|
||||
sub.request(1)
|
||||
probe.expectError(ex)
|
||||
}
|
||||
def deferred(): Future[Int] = {
|
||||
if (counter.incrementAndGet() > parallelism) Future.failed(new Exception("parallelism exceeded"))
|
||||
else {
|
||||
val p = Promise[Int]
|
||||
queue.offer(p -> System.nanoTime())
|
||||
p.future
|
||||
}
|
||||
}
|
||||
|
||||
"work when futures fail later" in {
|
||||
val probe = TestSubscriber.manualProbe[Int]
|
||||
val ex = new Exception("KABOOM")
|
||||
Source(List(1, 2))
|
||||
.transform(() ⇒ new MapAsyncOne(x ⇒ if (x == 1) Future.successful(1) else Future.failed(ex)))
|
||||
.runWith(Sink(probe))
|
||||
val sub = probe.expectSubscription()
|
||||
sub.request(1)
|
||||
probe.expectNext(1)
|
||||
probe.expectError(ex)
|
||||
try {
|
||||
val N = 100000
|
||||
Source(1 to N)
|
||||
.mapAsync(parallelism)(i ⇒ deferred())
|
||||
.runFold(0)((c, _) ⇒ c + 1)
|
||||
.futureValue(PatienceConfig(3.seconds)) should ===(N)
|
||||
} finally {
|
||||
timer.interrupt()
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -7,7 +7,6 @@ import scala.concurrent.Await
|
|||
import scala.concurrent.Future
|
||||
import scala.concurrent.duration._
|
||||
import scala.util.control.NoStackTrace
|
||||
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.stream.testkit._
|
||||
import akka.stream.testkit.scaladsl._
|
||||
|
|
@ -17,8 +16,15 @@ import akka.testkit.TestProbe
|
|||
import akka.stream.ActorAttributes.supervisionStrategy
|
||||
import akka.stream.Supervision.resumingDecider
|
||||
import akka.stream.impl.ReactiveStreamsCompliance
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import java.util.concurrent.ConcurrentLinkedQueue
|
||||
import scala.concurrent.Promise
|
||||
import java.util.concurrent.LinkedBlockingQueue
|
||||
import scala.annotation.tailrec
|
||||
import org.scalatest.concurrent.ScalaFutures
|
||||
import org.scalactic.ConversionCheckedTripleEquals
|
||||
|
||||
class FlowMapAsyncUnorderedSpec extends AkkaSpec {
|
||||
class FlowMapAsyncUnorderedSpec extends AkkaSpec with ScalaFutures with ConversionCheckedTripleEquals {
|
||||
|
||||
implicit val materializer = ActorMaterializer()
|
||||
|
||||
|
|
@ -54,13 +60,11 @@ class FlowMapAsyncUnorderedSpec extends AkkaSpec {
|
|||
n
|
||||
}).to(Sink(c)).run()
|
||||
val sub = c.expectSubscription()
|
||||
// first four run immediately
|
||||
probe.expectMsgAllOf(1, 2, 3, 4)
|
||||
c.expectNoMsg(200.millis)
|
||||
probe.expectNoMsg(Duration.Zero)
|
||||
sub.request(1)
|
||||
var got = Set(c.expectNext())
|
||||
probe.expectMsg(5)
|
||||
probe.expectMsgAllOf(1, 2, 3, 4, 5)
|
||||
probe.expectNoMsg(500.millis)
|
||||
sub.request(25)
|
||||
probe.expectMsgAllOf(6 to 20: _*)
|
||||
|
|
@ -176,11 +180,11 @@ class FlowMapAsyncUnorderedSpec extends AkkaSpec {
|
|||
.to(Sink(c)).run()
|
||||
val sub = c.expectSubscription()
|
||||
sub.request(10)
|
||||
for (elem ← List("a", "c")) c.expectNext(elem)
|
||||
c.expectNextUnordered("a", "c")
|
||||
c.expectComplete()
|
||||
}
|
||||
|
||||
"should handle cancel properly" in assertAllStagesStopped {
|
||||
"handle cancel properly" in assertAllStagesStopped {
|
||||
val pub = TestPublisher.manualProbe[Int]()
|
||||
val sub = TestSubscriber.manualProbe[Int]()
|
||||
|
||||
|
|
@ -195,5 +199,51 @@ class FlowMapAsyncUnorderedSpec extends AkkaSpec {
|
|||
|
||||
}
|
||||
|
||||
"not run more futures than configured" in assertAllStagesStopped {
|
||||
val parallelism = 8
|
||||
|
||||
val counter = new AtomicInteger
|
||||
val queue = new LinkedBlockingQueue[(Promise[Int], Long)]
|
||||
|
||||
val timer = new Thread {
|
||||
val delay = 50000 // nanoseconds
|
||||
var count = 0
|
||||
@tailrec final override def run(): Unit = {
|
||||
val cont = try {
|
||||
val (promise, enqueued) = queue.take()
|
||||
val wakeup = enqueued + delay
|
||||
while (System.nanoTime() < wakeup) {}
|
||||
counter.decrementAndGet()
|
||||
promise.success(count)
|
||||
count += 1
|
||||
true
|
||||
} catch {
|
||||
case _: InterruptedException ⇒ false
|
||||
}
|
||||
if (cont) run()
|
||||
}
|
||||
}
|
||||
timer.start
|
||||
|
||||
def deferred(): Future[Int] = {
|
||||
if (counter.incrementAndGet() > parallelism) Future.failed(new Exception("parallelism exceeded"))
|
||||
else {
|
||||
val p = Promise[Int]
|
||||
queue.offer(p -> System.nanoTime())
|
||||
p.future
|
||||
}
|
||||
}
|
||||
|
||||
try {
|
||||
val N = 100000
|
||||
Source(1 to N)
|
||||
.mapAsyncUnordered(parallelism)(i ⇒ deferred())
|
||||
.runFold(0)((c, _) ⇒ c + 1)
|
||||
.futureValue(PatienceConfig(3.seconds)) should ===(N)
|
||||
} finally {
|
||||
timer.interrupt()
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -6,12 +6,14 @@ package akka.stream.scaladsl
|
|||
import akka.actor._
|
||||
import akka.stream.Supervision._
|
||||
import akka.stream.impl._
|
||||
import akka.stream.impl.fusing.ActorInterpreter
|
||||
import akka.stream.stage.Stage
|
||||
import akka.stream.impl.fusing.{ ActorGraphInterpreter }
|
||||
import akka.stream.impl.fusing.GraphInterpreter.GraphAssembly
|
||||
import akka.stream.stage.AbstractStage.PushPullGraphStage
|
||||
import akka.stream.stage.{ GraphStageLogic, OutHandler, InHandler, Stage }
|
||||
import akka.stream.testkit.Utils._
|
||||
import akka.stream.testkit._
|
||||
import akka.stream.testkit.scaladsl.TestSink
|
||||
import akka.stream.{ AbruptTerminationException, ActorMaterializer, ActorMaterializerSettings, Attributes }
|
||||
import akka.stream._
|
||||
import akka.testkit.TestEvent.{ Mute, UnMute }
|
||||
import akka.testkit.{ EventFilter, TestDuration }
|
||||
import com.typesafe.config.ConfigFactory
|
||||
|
|
@ -32,7 +34,7 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
|
|||
import FlowSpec._
|
||||
|
||||
val settings = ActorMaterializerSettings(system)
|
||||
.withInputBuffer(initialSize = 2, maxSize = 16)
|
||||
.withInputBuffer(initialSize = 2, maxSize = 2)
|
||||
|
||||
implicit val mat = ActorMaterializer(settings)
|
||||
|
||||
|
|
@ -40,30 +42,51 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
|
|||
val identity2: Flow[Any, Any, _] ⇒ Flow[Any, Any, _] = in ⇒ identity(in)
|
||||
|
||||
class BrokenActorInterpreter(
|
||||
_assembly: GraphAssembly,
|
||||
_inHandlers: Array[InHandler],
|
||||
_outHandlers: Array[OutHandler],
|
||||
_logics: Array[GraphStageLogic],
|
||||
_shape: Shape,
|
||||
_settings: ActorMaterializerSettings,
|
||||
_ops: Seq[Stage[_, _]],
|
||||
_mat: Materializer,
|
||||
brokenMessage: Any)
|
||||
extends ActorInterpreter(_settings, _ops, mat, Attributes.none) {
|
||||
extends ActorGraphInterpreter(_assembly, _inHandlers, _outHandlers, _logics, _shape, _settings, _mat) {
|
||||
|
||||
import akka.stream.actor.ActorSubscriberMessage._
|
||||
|
||||
override protected[akka] def aroundReceive(receive: Receive, msg: Any) = {
|
||||
msg match {
|
||||
case OnNext(m) if m == brokenMessage ⇒
|
||||
case ActorGraphInterpreter.OnNext(0, m) if m == brokenMessage ⇒
|
||||
throw new NullPointerException(s"I'm so broken [$m]")
|
||||
case _ ⇒ super.aroundReceive(receive, msg)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
val faultyFlow: Flow[Any, Any, _] ⇒ Flow[Any, Any, _] = in ⇒ in.andThenMat { () ⇒
|
||||
val props = Props(new BrokenActorInterpreter(settings, List(fusing.Map({ x: Any ⇒ x }, stoppingDecider)), "a3"))
|
||||
val faultyFlow: Flow[Any, Any, _] ⇒ Flow[Any, Any, _] = in ⇒ in.via({
|
||||
val stage = new PushPullGraphStage((_) ⇒ fusing.Map({ x: Any ⇒ x }, stoppingDecider), Attributes.none)
|
||||
|
||||
val assembly = new GraphAssembly(
|
||||
Array(stage),
|
||||
Array(Attributes.none),
|
||||
Array(stage.shape.inlet, null),
|
||||
Array(0, -1),
|
||||
Array(null, stage.shape.outlet),
|
||||
Array(-1, 0))
|
||||
|
||||
val (inHandlers, outHandlers, logics, _) = assembly.materialize(Attributes.none)
|
||||
|
||||
val props = Props(new BrokenActorInterpreter(assembly, inHandlers, outHandlers, logics, stage.shape, settings, mat, "a3"))
|
||||
.withDispatcher("akka.test.stream-dispatcher").withDeploy(Deploy.local)
|
||||
val processor = ActorProcessorFactory[Any, Any](system.actorOf(
|
||||
props,
|
||||
"borken-stage-actor"))
|
||||
(processor, ())
|
||||
}
|
||||
val impl = system.actorOf(props, "borken-stage-actor")
|
||||
|
||||
val subscriber = new ActorGraphInterpreter.BoundarySubscriber(impl, 0)
|
||||
val publisher = new ActorPublisher[Any](impl) { override val wakeUpMsg = ActorGraphInterpreter.SubscribePending(0) }
|
||||
|
||||
impl ! ActorGraphInterpreter.ExposedPublisher(0, publisher)
|
||||
|
||||
Flow.fromSinkAndSource(Sink(subscriber), Source(publisher))
|
||||
})
|
||||
|
||||
val toPublisher: (Source[Any, _], ActorMaterializer) ⇒ Publisher[Any] =
|
||||
(f, m) ⇒ f.runWith(Sink.publisher)(m)
|
||||
|
|
@ -81,15 +104,15 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
|
|||
|
||||
for ((name, op) ← List("identity" -> identity, "identity2" -> identity2); n ← List(1, 2, 4)) {
|
||||
s"request initial elements from upstream ($name, $n)" in {
|
||||
new ChainSetup(op, settings.withInputBuffer(initialSize = n, maxSize = settings.maxInputBufferSize), toPublisher) {
|
||||
upstream.expectRequest(upstreamSubscription, settings.initialInputBufferSize)
|
||||
new ChainSetup(op, settings.withInputBuffer(initialSize = n, maxSize = n), toPublisher) {
|
||||
upstream.expectRequest(upstreamSubscription, settings.maxInputBufferSize)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
"request more elements from upstream when downstream requests more elements" in {
|
||||
new ChainSetup(identity, settings, toPublisher) {
|
||||
upstream.expectRequest(upstreamSubscription, settings.initialInputBufferSize)
|
||||
upstream.expectRequest(upstreamSubscription, settings.maxInputBufferSize)
|
||||
downstreamSubscription.request(1)
|
||||
upstream.expectNoMsg(100.millis)
|
||||
downstreamSubscription.request(2)
|
||||
|
|
@ -132,7 +155,7 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
|
|||
}
|
||||
|
||||
"cancel upstream when single subscriber cancels subscription while receiving data" in {
|
||||
new ChainSetup(identity, settings.withInputBuffer(initialSize = 1, maxSize = settings.maxInputBufferSize), toPublisher) {
|
||||
new ChainSetup(identity, settings.withInputBuffer(initialSize = 1, maxSize = 1), toPublisher) {
|
||||
downstreamSubscription.request(5)
|
||||
upstreamSubscription.expectRequest(1)
|
||||
upstreamSubscription.sendNext("test")
|
||||
|
|
@ -291,7 +314,7 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
|
|||
|
||||
"be possible to convert to a processor, and should be able to take a Processor" in {
|
||||
val identity1 = Flow[Int].toProcessor
|
||||
val identity2 = Flow(() ⇒ identity1.run())
|
||||
val identity2 = Flow.fromProcessor(() ⇒ identity1.run())
|
||||
Await.result(
|
||||
Source(1 to 10).via(identity2).grouped(100).runWith(Sink.head),
|
||||
3.seconds) should ===(1 to 10)
|
||||
|
|
|
|||
|
|
@ -9,7 +9,7 @@ import akka.stream._
|
|||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
|
||||
class GraphPreferredMergeSpec extends TwoStreamsSetup {
|
||||
class GraphMergePreferredSpec extends TwoStreamsSetup {
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
override type Outputs = Int
|
||||
|
|
@ -1,9 +1,9 @@
|
|||
package akka.stream.scaladsl
|
||||
|
||||
import akka.actor.ActorRef
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.stream.{ Attributes, ActorMaterializer }
|
||||
import akka.stream.impl.fusing.GraphStages.SimpleLinearGraphStage
|
||||
import akka.stream.stage.{ OutHandler, AsyncCallback, InHandler }
|
||||
import akka.stream.stage.{ TimerGraphStageLogic, OutHandler, AsyncCallback, InHandler }
|
||||
import akka.stream.testkit.{ AkkaSpec, TestPublisher }
|
||||
import akka.testkit.TestDuration
|
||||
|
||||
|
|
@ -39,13 +39,17 @@ class GraphStageTimersSpec extends AkkaSpec {
|
|||
implicit val mat = ActorMaterializer()
|
||||
|
||||
class TestStage(probe: ActorRef, sideChannel: SideChannel) extends SimpleLinearGraphStage[Int] {
|
||||
override def createLogic = new SimpleLinearStageLogic {
|
||||
override def createLogic(inheritedAttributes: Attributes) = new TimerGraphStageLogic(shape) {
|
||||
val tickCount = Iterator from 1
|
||||
|
||||
setHandler(in, new InHandler {
|
||||
override def onPush() = push(out, grab(in))
|
||||
})
|
||||
|
||||
setHandler(out, new OutHandler {
|
||||
override def onPull(): Unit = pull(in)
|
||||
})
|
||||
|
||||
override def preStart() = {
|
||||
sideChannel.asyncCallback = getAsyncCallback(onTestEvent)
|
||||
}
|
||||
|
|
@ -147,7 +151,7 @@ class GraphStageTimersSpec extends AkkaSpec {
|
|||
}
|
||||
|
||||
class TestStage2 extends SimpleLinearGraphStage[Int] {
|
||||
override def createLogic = new SimpleLinearStageLogic {
|
||||
override def createLogic(inheritedAttributes: Attributes) = new TimerGraphStageLogic(shape) {
|
||||
var tickCount = 0
|
||||
|
||||
override def preStart(): Unit = schedulePeriodically("tick", 100.millis)
|
||||
|
|
@ -195,13 +199,17 @@ class GraphStageTimersSpec extends AkkaSpec {
|
|||
val downstream = TestSubscriber.probe[Int]()
|
||||
|
||||
Source(upstream).via(new SimpleLinearGraphStage[Int] {
|
||||
override def createLogic = new SimpleLinearStageLogic {
|
||||
override def createLogic(inheritedAttributes: Attributes) = new TimerGraphStageLogic(shape) {
|
||||
override def preStart(): Unit = scheduleOnce("tick", 100.millis)
|
||||
|
||||
setHandler(in, new InHandler {
|
||||
override def onPush() = () // Ingore
|
||||
})
|
||||
|
||||
setHandler(out, new OutHandler {
|
||||
override def onPull(): Unit = pull(in)
|
||||
})
|
||||
|
||||
override def onTimer(timerKey: Any) = throw exception
|
||||
}
|
||||
}).runWith(Sink(downstream))
|
||||
|
|
|
|||
|
|
@ -73,7 +73,7 @@ class SourceSpec extends AkkaSpec {
|
|||
}
|
||||
|
||||
"Maybe Source" must {
|
||||
"complete materialized future with None when stream cancels" in {
|
||||
"complete materialized future with None when stream cancels" in Utils.assertAllStagesStopped {
|
||||
val neverSource = Source.maybe[Int]
|
||||
val pubSink = Sink.publisher[Int]
|
||||
|
||||
|
|
@ -90,7 +90,7 @@ class SourceSpec extends AkkaSpec {
|
|||
Await.result(f.future, 500.millis) shouldEqual None
|
||||
}
|
||||
|
||||
"allow external triggering of empty completion" in {
|
||||
"allow external triggering of empty completion" in Utils.assertAllStagesStopped {
|
||||
val neverSource = Source.maybe[Int].filter(_ ⇒ false)
|
||||
val counterSink = Sink.fold[Int, Int](0) { (acc, _) ⇒ acc + 1 }
|
||||
|
||||
|
|
@ -102,7 +102,7 @@ class SourceSpec extends AkkaSpec {
|
|||
Await.result(counterFuture, 500.millis) shouldEqual 0
|
||||
}
|
||||
|
||||
"allow external triggering of non-empty completion" in {
|
||||
"allow external triggering of non-empty completion" in Utils.assertAllStagesStopped {
|
||||
val neverSource = Source.maybe[Int]
|
||||
val counterSink = Sink.head[Int]
|
||||
|
||||
|
|
@ -114,7 +114,7 @@ class SourceSpec extends AkkaSpec {
|
|||
Await.result(counterFuture, 500.millis) shouldEqual 6
|
||||
}
|
||||
|
||||
"allow external triggering of onError" in {
|
||||
"allow external triggering of onError" in Utils.assertAllStagesStopped {
|
||||
val neverSource = Source.maybe[Int]
|
||||
val counterSink = Sink.fold[Int, Int](0) { (acc, _) ⇒ acc + 1 }
|
||||
|
||||
|
|
|
|||
|
|
@ -11,7 +11,7 @@ import scala.concurrent.duration._
|
|||
|
||||
import scala.concurrent.Await
|
||||
|
||||
class SubscriberSourceSpec extends AkkaSpec("akka.loglevel=DEBUG\nakka.actor.debug.lifecycle=on") {
|
||||
class SubscriberSourceSpec extends AkkaSpec {
|
||||
|
||||
implicit val materializer = ActorMaterializer()
|
||||
|
||||
|
|
|
|||
|
|
@ -150,7 +150,7 @@ class SubstreamSubscriptionTimeoutSpec(conf: String) extends AkkaSpec(conf) {
|
|||
private def watchGroupByActor(flowNr: Int): ActorRef = {
|
||||
implicit val t = Timeout(300.millis)
|
||||
import akka.pattern.ask
|
||||
val path = s"/user/$$a/flow-${flowNr}-1-publisherSource-groupBy"
|
||||
val path = s"/user/$$a/flow-${flowNr}-1-groupBy"
|
||||
val gropByPath = system.actorSelection(path)
|
||||
val groupByActor = try {
|
||||
Await.result((gropByPath ? Identify("")).mapTo[ActorIdentity], 300.millis).ref.get
|
||||
|
|
|
|||
|
|
@ -49,7 +49,7 @@ class UnzipWith1[In, [#A1#]](unzipper: In ⇒ ([#A1#])) extends GraphStage[FanOu
|
|||
[#def out0: Outlet[A1] = shape.out0#
|
||||
]
|
||||
|
||||
override def createLogic: GraphStageLogic = new GraphStageLogic(shape) {
|
||||
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new GraphStageLogic(shape) {
|
||||
var pendingCount = 1
|
||||
var downstreamRunning = 1
|
||||
|
||||
|
|
|
|||
|
|
@ -30,7 +30,7 @@ class ZipWith1[[#A1#], O] (zipper: ([#A1#]) ⇒ O) extends GraphStage[FanInShape
|
|||
[#val in0: Inlet[A1] = shape.in0#
|
||||
]
|
||||
|
||||
override def createLogic: GraphStageLogic = new GraphStageLogic(shape) {
|
||||
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new GraphStageLogic(shape) {
|
||||
var pending = 1
|
||||
|
||||
private def pushAll(): Unit = push(out, zipper([#grab(in0)#]))
|
||||
|
|
|
|||
|
|
@ -31,11 +31,11 @@ object ActorMaterializer {
|
|||
* the processing steps. The default `namePrefix` is `"flow"`. The actor names are built up of
|
||||
* `namePrefix-flowNumber-flowStepNumber-stepName`.
|
||||
*/
|
||||
def apply(materializerSettings: Option[ActorMaterializerSettings] = None, namePrefix: Option[String] = None, optimizations: Optimizations = Optimizations.none)(implicit context: ActorRefFactory): ActorMaterializer = {
|
||||
def apply(materializerSettings: Option[ActorMaterializerSettings] = None, namePrefix: Option[String] = None)(implicit context: ActorRefFactory): ActorMaterializer = {
|
||||
val system = actorSystemOf(context)
|
||||
|
||||
val settings = materializerSettings getOrElse ActorMaterializerSettings(system)
|
||||
apply(settings, namePrefix.getOrElse("flow"), optimizations)(context)
|
||||
apply(settings, namePrefix.getOrElse("flow"))(context)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -49,7 +49,7 @@ object ActorMaterializer {
|
|||
* the processing steps. The default `namePrefix` is `"flow"`. The actor names are built up of
|
||||
* `namePrefix-flowNumber-flowStepNumber-stepName`.
|
||||
*/
|
||||
def apply(materializerSettings: ActorMaterializerSettings, namePrefix: String, optimizations: Optimizations)(implicit context: ActorRefFactory): ActorMaterializer = {
|
||||
def apply(materializerSettings: ActorMaterializerSettings, namePrefix: String)(implicit context: ActorRefFactory): ActorMaterializer = {
|
||||
val haveShutDown = new AtomicBoolean(false)
|
||||
val system = actorSystemOf(context)
|
||||
|
||||
|
|
@ -60,8 +60,7 @@ object ActorMaterializer {
|
|||
context.actorOf(StreamSupervisor.props(materializerSettings, haveShutDown).withDispatcher(materializerSettings.dispatcher)),
|
||||
haveShutDown,
|
||||
FlowNameCounter(system).counter,
|
||||
namePrefix,
|
||||
optimizations)
|
||||
namePrefix)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -199,11 +198,10 @@ object ActorMaterializerSettings {
|
|||
supervisionDecider: Supervision.Decider,
|
||||
subscriptionTimeoutSettings: StreamSubscriptionTimeoutSettings,
|
||||
debugLogging: Boolean,
|
||||
outputBurstLimit: Int,
|
||||
optimizations: Optimizations) =
|
||||
outputBurstLimit: Int) =
|
||||
new ActorMaterializerSettings(
|
||||
initialInputBufferSize, maxInputBufferSize, dispatcher, supervisionDecider, subscriptionTimeoutSettings, debugLogging,
|
||||
outputBurstLimit, optimizations)
|
||||
outputBurstLimit)
|
||||
|
||||
/**
|
||||
* Create [[ActorMaterializerSettings]].
|
||||
|
|
@ -228,8 +226,7 @@ object ActorMaterializerSettings {
|
|||
supervisionDecider = Supervision.stoppingDecider,
|
||||
subscriptionTimeoutSettings = StreamSubscriptionTimeoutSettings(config),
|
||||
debugLogging = config.getBoolean("debug-logging"),
|
||||
outputBurstLimit = config.getInt("output-burst-limit"),
|
||||
optimizations = Optimizations.none)
|
||||
outputBurstLimit = config.getInt("output-burst-limit"))
|
||||
|
||||
/**
|
||||
* Java API
|
||||
|
|
@ -263,8 +260,7 @@ final class ActorMaterializerSettings(
|
|||
val supervisionDecider: Supervision.Decider,
|
||||
val subscriptionTimeoutSettings: StreamSubscriptionTimeoutSettings,
|
||||
val debugLogging: Boolean,
|
||||
val outputBurstLimit: Int,
|
||||
val optimizations: Optimizations) {
|
||||
val outputBurstLimit: Int) {
|
||||
|
||||
require(initialInputBufferSize > 0, "initialInputBufferSize must be > 0")
|
||||
|
||||
|
|
@ -278,11 +274,10 @@ final class ActorMaterializerSettings(
|
|||
supervisionDecider: Supervision.Decider = this.supervisionDecider,
|
||||
subscriptionTimeoutSettings: StreamSubscriptionTimeoutSettings = this.subscriptionTimeoutSettings,
|
||||
debugLogging: Boolean = this.debugLogging,
|
||||
outputBurstLimit: Int = this.outputBurstLimit,
|
||||
optimizations: Optimizations = this.optimizations) =
|
||||
outputBurstLimit: Int = this.outputBurstLimit) =
|
||||
new ActorMaterializerSettings(
|
||||
initialInputBufferSize, maxInputBufferSize, dispatcher, supervisionDecider, subscriptionTimeoutSettings, debugLogging,
|
||||
outputBurstLimit, optimizations)
|
||||
outputBurstLimit)
|
||||
|
||||
def withInputBuffer(initialSize: Int, maxSize: Int): ActorMaterializerSettings =
|
||||
copy(initialInputBufferSize = initialSize, maxInputBufferSize = maxSize)
|
||||
|
|
@ -316,9 +311,6 @@ final class ActorMaterializerSettings(
|
|||
def withDebugLogging(enable: Boolean): ActorMaterializerSettings =
|
||||
copy(debugLogging = enable)
|
||||
|
||||
def withOptimizations(optimizations: Optimizations): ActorMaterializerSettings =
|
||||
copy(optimizations = optimizations)
|
||||
|
||||
private def requirePowerOfTwo(n: Integer, name: String): Unit = {
|
||||
require(n > 0, s"$name must be > 0")
|
||||
require((n & (n - 1)) == 0, s"$name must be a power of two")
|
||||
|
|
@ -365,11 +357,3 @@ object StreamSubscriptionTimeoutTerminationMode {
|
|||
|
||||
}
|
||||
|
||||
final object Optimizations {
|
||||
val none: Optimizations = Optimizations(collapsing = false, elision = false, simplification = false, fusion = false)
|
||||
val all: Optimizations = Optimizations(collapsing = true, elision = true, simplification = true, fusion = true)
|
||||
}
|
||||
|
||||
final case class Optimizations(collapsing: Boolean, elision: Boolean, simplification: Boolean, fusion: Boolean) {
|
||||
def isEnabled: Boolean = collapsing || elision || simplification || fusion
|
||||
}
|
||||
|
|
|
|||
|
|
@ -4,10 +4,10 @@
|
|||
package akka.stream
|
||||
|
||||
import akka.event.Logging
|
||||
|
||||
import scala.annotation.tailrec
|
||||
import scala.collection.immutable
|
||||
import akka.stream.impl.Stages.StageModule
|
||||
import scala.reflect.{ classTag, ClassTag }
|
||||
import akka.stream.impl.Stages.SymbolicStage
|
||||
import akka.japi.function
|
||||
|
||||
/**
|
||||
|
|
@ -16,7 +16,7 @@ import akka.japi.function
|
|||
*
|
||||
* Note that more attributes for the [[ActorMaterializer]] are defined in [[ActorAttributes]].
|
||||
*/
|
||||
final case class Attributes private (attributeList: immutable.Seq[Attributes.Attribute] = Nil) {
|
||||
final case class Attributes(attributeList: List[Attributes.Attribute] = Nil) {
|
||||
|
||||
import Attributes._
|
||||
|
||||
|
|
@ -44,22 +44,47 @@ final case class Attributes private (attributeList: immutable.Seq[Attributes.Att
|
|||
}
|
||||
|
||||
/**
|
||||
* Get first attribute of a given `Class` or subclass thereof.
|
||||
* Java API: Get the last (most specific) attribute of a given `Class` or subclass thereof.
|
||||
* If no such attribute exists the `default` value is returned.
|
||||
*/
|
||||
def getAttribute[T <: Attribute](c: Class[T], default: T): T =
|
||||
attributeList.find(c.isInstance) match {
|
||||
case Some(a) ⇒ c.cast(a)
|
||||
getAttribute(c) match {
|
||||
case Some(a) ⇒ a
|
||||
case None ⇒ default
|
||||
}
|
||||
|
||||
/**
|
||||
* Java API: Get the last (most specific) attribute of a given `Class` or subclass thereof.
|
||||
*/
|
||||
def getAttribute[T <: Attribute](c: Class[T]): Option[T] =
|
||||
Option(attributeList.foldLeft(null.asInstanceOf[T])((acc, attr) ⇒ if (c.isInstance(attr)) c.cast(attr) else acc))
|
||||
|
||||
/**
|
||||
* Get the last (most specific) attribute of a given type parameter T `Class` or subclass thereof.
|
||||
* If no such attribute exists the `default` value is returned.
|
||||
*/
|
||||
def get[T <: Attribute: ClassTag](default: T) =
|
||||
getAttribute(classTag[T].runtimeClass.asInstanceOf[Class[T]], default)
|
||||
|
||||
/**
|
||||
* Get the last (most specific) attribute of a given type parameter T `Class` or subclass thereof.
|
||||
*/
|
||||
def get[T <: Attribute: ClassTag] =
|
||||
getAttribute(classTag[T].runtimeClass.asInstanceOf[Class[T]])
|
||||
|
||||
/**
|
||||
* Adds given attributes to the end of these attributes.
|
||||
*/
|
||||
def and(other: Attributes): Attributes =
|
||||
if (attributeList.isEmpty) other
|
||||
else if (other.attributeList.isEmpty) this
|
||||
else Attributes(attributeList ++ other.attributeList)
|
||||
else Attributes(attributeList ::: other.attributeList)
|
||||
|
||||
/**
|
||||
* Adds given attribute to the end of these attributes.
|
||||
*/
|
||||
def and(other: Attribute): Attributes =
|
||||
Attributes(attributeList :+ other)
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
@ -76,12 +101,7 @@ final case class Attributes private (attributeList: immutable.Seq[Attributes.Att
|
|||
case Name(n) ⇒
|
||||
if (buf ne null) concatNames(i, null, buf.append('-').append(n))
|
||||
else if (first ne null) {
|
||||
val b = new StringBuilder(
|
||||
(first.length() + n.length()) match {
|
||||
case x if x < 0 ⇒ throw new IllegalStateException("Names too long to concatenate")
|
||||
case y if y > Int.MaxValue / 2 ⇒ Int.MaxValue
|
||||
case z ⇒ Math.max(Integer.highestOneBit(z) * 2, 32)
|
||||
})
|
||||
val b = new StringBuilder((first.length() + n.length()) * 2)
|
||||
concatNames(i, null, b.append(first).append('-').append(n))
|
||||
} else concatNames(i, n, null)
|
||||
case _ ⇒ concatNames(i, first, buf)
|
||||
|
|
@ -95,16 +115,6 @@ final case class Attributes private (attributeList: immutable.Seq[Attributes.Att
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] def logLevels: Option[LogLevels] =
|
||||
attributeList.collectFirst { case l: LogLevels ⇒ l }
|
||||
|
||||
private[akka] def transform(node: StageModule): StageModule =
|
||||
if ((this eq Attributes.none) || (this eq node.attributes)) node
|
||||
else node.withAttributes(attributes = this and node.attributes)
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -10,7 +10,7 @@ import akka.dispatch.Dispatchers
|
|||
import akka.pattern.ask
|
||||
import akka.stream.actor.ActorSubscriber
|
||||
import akka.stream.impl.StreamLayout.Module
|
||||
import akka.stream.impl.fusing.{ ActorGraphInterpreter, GraphModule, ActorInterpreter }
|
||||
import akka.stream.impl.fusing.{ ActorGraphInterpreter, GraphModule }
|
||||
import akka.stream.impl.io.SslTlsCipherActor
|
||||
import akka.stream._
|
||||
import akka.stream.io.SslTls.TlsModule
|
||||
|
|
@ -24,14 +24,13 @@ import scala.concurrent.{ Await, ExecutionContextExecutor }
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] case class ActorMaterializerImpl(val system: ActorSystem,
|
||||
private[akka] case class ActorMaterializerImpl(system: ActorSystem,
|
||||
override val settings: ActorMaterializerSettings,
|
||||
dispatchers: Dispatchers,
|
||||
val supervisor: ActorRef,
|
||||
val haveShutDown: AtomicBoolean,
|
||||
supervisor: ActorRef,
|
||||
haveShutDown: AtomicBoolean,
|
||||
flowNameCounter: AtomicLong,
|
||||
namePrefix: String,
|
||||
optimizations: Optimizations) extends ActorMaterializer {
|
||||
namePrefix: String) extends ActorMaterializer {
|
||||
import akka.stream.impl.Stages._
|
||||
|
||||
override def shutdown(): Unit =
|
||||
|
|
@ -45,6 +44,12 @@ private[akka] case class ActorMaterializerImpl(val system: ActorSystem,
|
|||
|
||||
private[this] def createFlowName(): String = s"$namePrefix-${nextFlowNameCount()}"
|
||||
|
||||
private val initialAttributes = Attributes(
|
||||
Attributes.InputBuffer(settings.initialInputBufferSize, settings.maxInputBufferSize) ::
|
||||
ActorAttributes.Dispatcher(settings.dispatcher) ::
|
||||
ActorAttributes.SupervisionStrategy(settings.supervisionDecider) ::
|
||||
Nil)
|
||||
|
||||
override def effectiveSettings(opAttr: Attributes): ActorMaterializerSettings = {
|
||||
import Attributes._
|
||||
import ActorAttributes._
|
||||
|
|
@ -70,7 +75,7 @@ private[akka] case class ActorMaterializerImpl(val system: ActorSystem,
|
|||
throw new IllegalStateException("Attempted to call materialize() after the ActorMaterializer has been shut down.")
|
||||
if (StreamLayout.Debug) StreamLayout.validate(runnableGraph.module)
|
||||
|
||||
val session = new MaterializerSession(runnableGraph.module) {
|
||||
val session = new MaterializerSession(runnableGraph.module, initialAttributes) {
|
||||
private val flowName = createFlowName()
|
||||
private var nextId = 0
|
||||
private def stageName(attr: Attributes): String = {
|
||||
|
|
@ -93,6 +98,7 @@ private[akka] case class ActorMaterializerImpl(val system: ActorSystem,
|
|||
assignPort(source.shape.outlet, pub.asInstanceOf[Publisher[Any]])
|
||||
mat
|
||||
|
||||
// FIXME: Remove this, only stream-of-stream ops need it
|
||||
case stage: StageModule ⇒
|
||||
val (processor, mat) = processorFor(stage, effectiveAttributes, effectiveSettings(effectiveAttributes))
|
||||
assignPort(stage.inPort, processor)
|
||||
|
|
@ -118,7 +124,7 @@ private[akka] case class ActorMaterializerImpl(val system: ActorSystem,
|
|||
|
||||
case graph: GraphModule ⇒
|
||||
val calculatedSettings = effectiveSettings(effectiveAttributes)
|
||||
val (inHandlers, outHandlers, logics, mat) = graph.assembly.materialize()
|
||||
val (inHandlers, outHandlers, logics, mat) = graph.assembly.materialize(effectiveAttributes)
|
||||
|
||||
val props = ActorGraphInterpreter.props(
|
||||
graph.assembly, inHandlers, outHandlers, logics, graph.shape, calculatedSettings, ActorMaterializerImpl.this)
|
||||
|
|
@ -137,11 +143,11 @@ private[akka] case class ActorMaterializerImpl(val system: ActorSystem,
|
|||
}
|
||||
}
|
||||
|
||||
// FIXME: Remove this, only stream-of-stream ops need it
|
||||
private def processorFor(op: StageModule,
|
||||
effectiveAttributes: Attributes,
|
||||
effectiveSettings: ActorMaterializerSettings): (Processor[Any, Any], Any) = op match {
|
||||
case DirectProcessor(processorFactory, _) ⇒ processorFactory()
|
||||
case Identity(attr) ⇒ (new VirtualProcessor, ())
|
||||
case _ ⇒
|
||||
val (opprops, mat) = ActorProcessorFactory.props(ActorMaterializerImpl.this, op, effectiveAttributes)
|
||||
ActorProcessorFactory[Any, Any](
|
||||
|
|
@ -248,47 +254,12 @@ private[akka] object ActorProcessorFactory {
|
|||
// USE THIS TO AVOID CLOSING OVER THE MATERIALIZER BELOW
|
||||
// Also, otherwise the attributes will not affect the settings properly!
|
||||
val settings = materializer.effectiveSettings(att)
|
||||
def interp(s: Stage[_, _]): (Props, Unit) = (ActorInterpreter.props(settings, List(s), materializer, att), ())
|
||||
def interpAttr(s: Stage[_, _], newAttributes: Attributes): (Props, Unit) = (ActorInterpreter.props(settings, List(s), materializer, newAttributes), ())
|
||||
def inputSizeAttr(n: Long) = {
|
||||
if (n <= 0)
|
||||
inputBuffer(initial = 1, max = 1) and att
|
||||
else if (n <= materializer.settings.maxInputBufferSize)
|
||||
inputBuffer(initial = n.toInt, max = n.toInt) and att
|
||||
else
|
||||
att
|
||||
}
|
||||
op match {
|
||||
case Map(f, _) ⇒ interp(fusing.Map(f, settings.supervisionDecider))
|
||||
case Filter(p, _) ⇒ interp(fusing.Filter(p, settings.supervisionDecider))
|
||||
case Drop(n, _) ⇒ interp(fusing.Drop(n))
|
||||
case Take(n, _) ⇒ interpAttr(fusing.Take(n), inputSizeAttr(n))
|
||||
case TakeWhile(p, _) ⇒ interp(fusing.TakeWhile(p, settings.supervisionDecider))
|
||||
case DropWhile(p, _) ⇒ interp(fusing.DropWhile(p, settings.supervisionDecider))
|
||||
case Collect(pf, _) ⇒ interp(fusing.Collect(pf, settings.supervisionDecider))
|
||||
case Scan(z, f, _) ⇒ interp(fusing.Scan(z, f, settings.supervisionDecider))
|
||||
case Fold(z, f, _) ⇒ interp(fusing.Fold(z, f, settings.supervisionDecider))
|
||||
case Intersperse(s, i, e, _) ⇒ interp(fusing.Intersperse(s, i, e))
|
||||
case Recover(pf, _) ⇒ interp(fusing.Recover(pf))
|
||||
case Expand(s, f, _) ⇒ interp(fusing.Expand(s, f))
|
||||
case Conflate(s, f, _) ⇒ interp(fusing.Conflate(s, f, settings.supervisionDecider))
|
||||
case Buffer(n, s, _) ⇒ interp(fusing.Buffer(n, s))
|
||||
case MapConcat(f, _) ⇒ interp(fusing.MapConcat(f, settings.supervisionDecider))
|
||||
case MapAsync(p, f, _) ⇒ interp(fusing.MapAsync(p, f, settings.supervisionDecider))
|
||||
case MapAsyncUnordered(p, f, _) ⇒ interp(fusing.MapAsyncUnordered(p, f, settings.supervisionDecider))
|
||||
case Grouped(n, _) ⇒ interp(fusing.Grouped(n))
|
||||
case Sliding(n, step, _) ⇒ interp(fusing.Sliding(n, step))
|
||||
case Log(n, e, l, _) ⇒ interp(fusing.Log(n, e, l))
|
||||
case GroupBy(f, _) ⇒ (GroupByProcessorImpl.props(settings, f), ())
|
||||
case PrefixAndTail(n, _) ⇒ (PrefixAndTailImpl.props(settings, n), ())
|
||||
case Split(d, _) ⇒ (SplitWhereProcessorImpl.props(settings, d), ())
|
||||
case ConcatAll(_) ⇒ (ConcatAllImpl.props(materializer), ())
|
||||
case StageFactory(mkStage, _) ⇒ interp(mkStage())
|
||||
case MaterializingStageFactory(mkStageAndMat, _) ⇒
|
||||
val s_m = mkStageAndMat()
|
||||
(ActorInterpreter.props(settings, List(s_m._1), materializer, att), s_m._2)
|
||||
case GroupBy(f, _) ⇒ (GroupByProcessorImpl.props(settings, f), ())
|
||||
case PrefixAndTail(n, _) ⇒ (PrefixAndTailImpl.props(settings, n), ())
|
||||
case Split(d, _) ⇒ (SplitWhereProcessorImpl.props(settings, d), ())
|
||||
case ConcatAll(_) ⇒ (ConcatAllImpl.props(materializer), ())
|
||||
case DirectProcessor(p, m) ⇒ throw new AssertionError("DirectProcessor cannot end up in ActorProcessorFactory")
|
||||
case Identity(_) ⇒ throw new AssertionError("Identity cannot end up in ActorProcessorFactory")
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -26,6 +26,7 @@ private[akka] object FixedSizeBuffer {
|
|||
else new ModuloFixedSizeBuffer(size)
|
||||
|
||||
sealed abstract class FixedSizeBuffer[T](val size: Int) {
|
||||
override def toString = s"Buffer($size, $readIdx, $writeIdx)(${(readIdx until writeIdx).map(get).mkString(", ")})"
|
||||
private val buffer = new Array[AnyRef](size)
|
||||
|
||||
protected var readIdx = 0
|
||||
|
|
|
|||
|
|
@ -4,11 +4,14 @@
|
|||
package akka.stream.impl
|
||||
|
||||
import akka.event.LoggingAdapter
|
||||
import akka.stream.ActorAttributes.SupervisionStrategy
|
||||
import akka.stream.Supervision.Decider
|
||||
import akka.stream.impl.SplitDecision.SplitDecision
|
||||
import akka.stream.impl.StreamLayout._
|
||||
import akka.stream.{ OverflowStrategy, Attributes }
|
||||
import akka.stream._
|
||||
import akka.stream.Attributes._
|
||||
import akka.stream.stage.Stage
|
||||
import akka.stream.stage.AbstractStage.PushPullGraphStage
|
||||
import akka.stream.stage.{ GraphStageLogic, GraphStage, Stage }
|
||||
import org.reactivestreams.Processor
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.Future
|
||||
|
|
@ -96,112 +99,121 @@ private[stream] object Stages {
|
|||
|
||||
import DefaultAttributes._
|
||||
|
||||
// FIXME: To be deprecated as soon as stream-of-stream operations are stages
|
||||
sealed trait StageModule extends FlowModule[Any, Any, Any] {
|
||||
def attributes: Attributes
|
||||
def withAttributes(attributes: Attributes): StageModule
|
||||
override def carbonCopy: Module = withAttributes(attributes)
|
||||
}
|
||||
|
||||
final case class StageFactory(mkStage: () ⇒ Stage[_, _], attributes: Attributes = stageFactory) extends StageModule {
|
||||
override def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
/*
|
||||
* Stage that is backed by a GraphStage but can be symbolically introspected
|
||||
*/
|
||||
case class SymbolicGraphStage[-In, +Out, Ext](symbolicStage: SymbolicStage[In, Out])
|
||||
extends PushPullGraphStage[In, Out, Ext](
|
||||
symbolicStage.create,
|
||||
symbolicStage.attributes) {
|
||||
}
|
||||
|
||||
final case class MaterializingStageFactory(
|
||||
mkStageAndMaterialized: () ⇒ (Stage[_, _], Any),
|
||||
attributes: Attributes = stageFactory) extends StageModule {
|
||||
override def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
val identityGraph = SymbolicGraphStage[Any, Any, Any](Identity)
|
||||
|
||||
sealed trait SymbolicStage[-In, +Out] {
|
||||
def attributes: Attributes
|
||||
def create(effectiveAttributes: Attributes): Stage[In, Out]
|
||||
|
||||
// FIXME: No supervision hooked in yet.
|
||||
|
||||
protected def supervision(attributes: Attributes): Decider =
|
||||
attributes.get[SupervisionStrategy](SupervisionStrategy(Supervision.stoppingDecider)).decider
|
||||
|
||||
}
|
||||
|
||||
final case class Identity(attributes: Attributes = identityOp) extends StageModule {
|
||||
override def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
object Identity extends SymbolicStage[Any, Any] {
|
||||
override val attributes: Attributes = identityOp
|
||||
|
||||
def apply[T]: SymbolicStage[T, T] = this.asInstanceOf[SymbolicStage[T, T]]
|
||||
|
||||
override def create(attr: Attributes): Stage[Any, Any] = fusing.Map(identity, supervision(attr))
|
||||
}
|
||||
|
||||
final case class Map(f: Any ⇒ Any, attributes: Attributes = map) extends StageModule {
|
||||
override def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
final case class Map[In, Out](f: In ⇒ Out, attributes: Attributes = map) extends SymbolicStage[In, Out] {
|
||||
override def create(attr: Attributes): Stage[In, Out] = fusing.Map(f, supervision(attr))
|
||||
}
|
||||
|
||||
final case class Log(name: String, extract: Any ⇒ Any, loggingAdapter: Option[LoggingAdapter], attributes: Attributes = log) extends StageModule {
|
||||
override def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
final case class Log[T](name: String, extract: T ⇒ Any, loggingAdapter: Option[LoggingAdapter], attributes: Attributes = log) extends SymbolicStage[T, T] {
|
||||
override def create(attr: Attributes): Stage[T, T] = fusing.Log(name, extract, loggingAdapter)
|
||||
}
|
||||
|
||||
final case class Filter(p: Any ⇒ Boolean, attributes: Attributes = filter) extends StageModule {
|
||||
override def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
final case class Filter[T](p: T ⇒ Boolean, attributes: Attributes = filter) extends SymbolicStage[T, T] {
|
||||
override def create(attr: Attributes): Stage[T, T] = fusing.Filter(p, supervision(attr))
|
||||
}
|
||||
|
||||
final case class Collect(pf: PartialFunction[Any, Any], attributes: Attributes = collect) extends StageModule {
|
||||
override def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
final case class Collect[In, Out](pf: PartialFunction[In, Out], attributes: Attributes = collect) extends SymbolicStage[In, Out] {
|
||||
override def create(attr: Attributes): Stage[In, Out] = fusing.Collect(pf, supervision(attr))
|
||||
}
|
||||
|
||||
final case class Recover(pf: PartialFunction[Any, Any], attributes: Attributes = recover) extends StageModule {
|
||||
override def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
final case class Recover[In, Out >: In](pf: PartialFunction[Throwable, Out], attributes: Attributes = recover) extends SymbolicStage[In, Out] {
|
||||
override def create(attr: Attributes): Stage[In, Out] = fusing.Recover(pf)
|
||||
}
|
||||
|
||||
final case class MapAsync(parallelism: Int, f: Any ⇒ Future[Any], attributes: Attributes = mapAsync) extends StageModule {
|
||||
override def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
}
|
||||
|
||||
final case class MapAsyncUnordered(parallelism: Int, f: Any ⇒ Future[Any], attributes: Attributes = mapAsyncUnordered) extends StageModule {
|
||||
override def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
}
|
||||
|
||||
final case class Grouped(n: Int, attributes: Attributes = grouped) extends StageModule {
|
||||
final case class Grouped[T](n: Int, attributes: Attributes = grouped) extends SymbolicStage[T, immutable.Seq[T]] {
|
||||
require(n > 0, "n must be greater than 0")
|
||||
override def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
override def create(attr: Attributes): Stage[T, immutable.Seq[T]] = fusing.Grouped(n)
|
||||
}
|
||||
|
||||
final case class Sliding(n: Int, step: Int, attributes: Attributes = sliding) extends StageModule {
|
||||
final case class Sliding[T](n: Int, step: Int, attributes: Attributes = sliding) extends SymbolicStage[T, immutable.Seq[T]] {
|
||||
require(n > 0, "n must be greater than 0")
|
||||
require(step > 0, "step must be greater than 0")
|
||||
override def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
|
||||
override def create(attr: Attributes): Stage[T, immutable.Seq[T]] = fusing.Sliding(n, step)
|
||||
}
|
||||
|
||||
final case class Take(n: Long, attributes: Attributes = take) extends StageModule {
|
||||
override def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
final case class Take[T](n: Long, attributes: Attributes = take) extends SymbolicStage[T, T] {
|
||||
override def create(attr: Attributes): Stage[T, T] = fusing.Take(n)
|
||||
}
|
||||
|
||||
final case class Drop(n: Long, attributes: Attributes = drop) extends StageModule {
|
||||
override def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
final case class Drop[T](n: Long, attributes: Attributes = drop) extends SymbolicStage[T, T] {
|
||||
override def create(attr: Attributes): Stage[T, T] = fusing.Drop(n)
|
||||
}
|
||||
|
||||
final case class TakeWhile(p: Any ⇒ Boolean, attributes: Attributes = takeWhile) extends StageModule {
|
||||
override def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
|
||||
final case class TakeWhile[T](p: T ⇒ Boolean, attributes: Attributes = takeWhile) extends SymbolicStage[T, T] {
|
||||
override def create(attr: Attributes): Stage[T, T] = fusing.TakeWhile(p, supervision(attr))
|
||||
}
|
||||
|
||||
final case class DropWhile(p: Any ⇒ Boolean, attributes: Attributes = dropWhile) extends StageModule {
|
||||
override def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
final case class DropWhile[T](p: T ⇒ Boolean, attributes: Attributes = dropWhile) extends SymbolicStage[T, T] {
|
||||
override def create(attr: Attributes): Stage[T, T] = fusing.DropWhile(p, supervision(attr))
|
||||
}
|
||||
|
||||
final case class Scan(zero: Any, f: (Any, Any) ⇒ Any, attributes: Attributes = scan) extends StageModule {
|
||||
override def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
final case class Scan[In, Out](zero: Out, f: (Out, In) ⇒ Out, attributes: Attributes = scan) extends SymbolicStage[In, Out] {
|
||||
override def create(attr: Attributes): Stage[In, Out] = fusing.Scan(zero, f, supervision(attr))
|
||||
}
|
||||
|
||||
final case class Fold(zero: Any, f: (Any, Any) ⇒ Any, attributes: Attributes = fold) extends StageModule {
|
||||
override def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
final case class Intersperse[T](start: Option[T], inject: T, end: Option[T], attributes: Attributes = intersperse) extends SymbolicStage[T, T] {
|
||||
override def create(attr: Attributes) = fusing.Intersperse(start, inject, end)
|
||||
}
|
||||
|
||||
final case class Intersperse(start: Option[Any], inject: Any, end: Option[Any], attributes: Attributes = intersperse) extends StageModule {
|
||||
override def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
final case class Fold[In, Out](zero: Out, f: (Out, In) ⇒ Out, attributes: Attributes = fold) extends SymbolicStage[In, Out] {
|
||||
override def create(attr: Attributes): Stage[In, Out] = fusing.Fold(zero, f, supervision(attr))
|
||||
}
|
||||
|
||||
final case class Buffer(size: Int, overflowStrategy: OverflowStrategy, attributes: Attributes = buffer) extends StageModule {
|
||||
final case class Buffer[T](size: Int, overflowStrategy: OverflowStrategy, attributes: Attributes = buffer) extends SymbolicStage[T, T] {
|
||||
require(size > 0, s"Buffer size must be larger than zero but was [$size]")
|
||||
|
||||
override def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
override def create(attr: Attributes): Stage[T, T] = fusing.Buffer(size, overflowStrategy)
|
||||
}
|
||||
|
||||
final case class Conflate(seed: Any ⇒ Any, aggregate: (Any, Any) ⇒ Any, attributes: Attributes = conflate) extends StageModule {
|
||||
override def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
final case class Conflate[In, Out](seed: In ⇒ Out, aggregate: (Out, In) ⇒ Out, attributes: Attributes = conflate) extends SymbolicStage[In, Out] {
|
||||
override def create(attr: Attributes): Stage[In, Out] = fusing.Conflate(seed, aggregate, supervision(attr))
|
||||
}
|
||||
|
||||
final case class Expand(seed: Any ⇒ Any, extrapolate: Any ⇒ (Any, Any), attributes: Attributes = expand) extends StageModule {
|
||||
override def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
final case class Expand[In, Out, Seed](seed: In ⇒ Seed, extrapolate: Seed ⇒ (Out, Seed), attributes: Attributes = expand) extends SymbolicStage[In, Out] {
|
||||
override def create(attr: Attributes): Stage[In, Out] = fusing.Expand(seed, extrapolate)
|
||||
}
|
||||
|
||||
final case class MapConcat(f: Any ⇒ immutable.Iterable[Any], attributes: Attributes = mapConcat) extends StageModule {
|
||||
override def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
final case class MapConcat[In, Out](f: In ⇒ immutable.Iterable[Out], attributes: Attributes = mapConcat) extends SymbolicStage[In, Out] {
|
||||
override def create(attr: Attributes): Stage[In, Out] = fusing.MapConcat(f, supervision(attr))
|
||||
}
|
||||
|
||||
// FIXME: These are not yet proper stages, therefore they use the deprecated StageModule infrastructure
|
||||
|
||||
final case class GroupBy(f: Any ⇒ Any, attributes: Attributes = groupBy) extends StageModule {
|
||||
override def withAttributes(attributes: Attributes) = copy(attributes = attributes)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -184,7 +184,7 @@ private[akka] object StreamLayout {
|
|||
downstreams,
|
||||
upstreams,
|
||||
Transform(f, if (this.isSealed) Atomic(this) else this.materializedValueComputation),
|
||||
attributes)
|
||||
if (this.isSealed) Attributes.none else attributes)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -225,7 +225,7 @@ private[akka] object StreamLayout {
|
|||
upstreams ++ that.upstreams,
|
||||
// would like to optimize away this allocation for Keep.{left,right} but that breaks side-effecting transformations
|
||||
Combine(f.asInstanceOf[(Any, Any) ⇒ Any], matComputation1, matComputation2),
|
||||
attributes)
|
||||
Attributes.none)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -595,7 +595,7 @@ private[stream] object MaterializerSession {
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[stream] abstract class MaterializerSession(val topLevel: StreamLayout.Module) {
|
||||
private[stream] abstract class MaterializerSession(val topLevel: StreamLayout.Module, val initialAttributes: Attributes) {
|
||||
import StreamLayout._
|
||||
|
||||
private var subscribersStack: List[mutable.Map[InPort, Subscriber[Any]]] =
|
||||
|
|
@ -653,7 +653,7 @@ private[stream] abstract class MaterializerSession(val topLevel: StreamLayout.Mo
|
|||
require(
|
||||
topLevel.isRunnable,
|
||||
s"The top level module cannot be materialized because it has unconnected ports: ${(topLevel.inPorts ++ topLevel.outPorts).mkString(", ")}")
|
||||
try materializeModule(topLevel, topLevel.attributes)
|
||||
try materializeModule(topLevel, initialAttributes and topLevel.attributes)
|
||||
catch {
|
||||
case NonFatal(cause) ⇒
|
||||
// PANIC!!! THE END OF THE MATERIALIZATION IS NEAR!
|
||||
|
|
|
|||
|
|
@ -339,40 +339,41 @@ private[stream] class ActorGraphInterpreter(
|
|||
override def receive: Receive = {
|
||||
// Cases that are most likely on the hot path, in decreasing order of frequency
|
||||
case OnNext(id: Int, e: Any) ⇒
|
||||
if (GraphInterpreter.Debug) println(s" onNext $e id=$id")
|
||||
if (GraphInterpreter.Debug) println(s"${interpreter.Name} onNext $e id=$id")
|
||||
inputs(id).onNext(e)
|
||||
runBatch()
|
||||
case RequestMore(id: Int, demand: Long) ⇒
|
||||
if (GraphInterpreter.Debug) println(s" request $demand id=$id")
|
||||
if (GraphInterpreter.Debug) println(s"${interpreter.Name} request $demand id=$id")
|
||||
outputs(id).requestMore(demand)
|
||||
runBatch()
|
||||
case Resume ⇒
|
||||
resumeScheduled = false
|
||||
if (interpreter.isSuspended) runBatch()
|
||||
case AsyncInput(logic, event, handler) ⇒
|
||||
if (GraphInterpreter.Debug) println(s"ASYNC $event")
|
||||
if (!interpreter.isStageCompleted(logic.stageId)) {
|
||||
if (GraphInterpreter.Debug) println(s"${interpreter.Name} ASYNC $event ($handler) [$logic]")
|
||||
if (!interpreter.isStageCompleted(logic)) {
|
||||
try handler(event)
|
||||
catch {
|
||||
case NonFatal(e) ⇒ logic.failStage(e)
|
||||
}
|
||||
interpreter.afterStageHasRun(logic)
|
||||
}
|
||||
runBatch()
|
||||
|
||||
// Initialization and completion messages
|
||||
case OnError(id: Int, cause: Throwable) ⇒
|
||||
if (GraphInterpreter.Debug) println(s" onError id=$id")
|
||||
if (GraphInterpreter.Debug) println(s"${interpreter.Name} onError id=$id")
|
||||
inputs(id).onError(cause)
|
||||
runBatch()
|
||||
case OnComplete(id: Int) ⇒
|
||||
if (GraphInterpreter.Debug) println(s" onComplete id=$id")
|
||||
if (GraphInterpreter.Debug) println(s"${interpreter.Name} onComplete id=$id")
|
||||
inputs(id).onComplete()
|
||||
runBatch()
|
||||
case OnSubscribe(id: Int, subscription: Subscription) ⇒
|
||||
subscribesPending -= 1
|
||||
inputs(id).onSubscribe(subscription)
|
||||
case Cancel(id: Int) ⇒
|
||||
if (GraphInterpreter.Debug) println(s" cancel id=$id")
|
||||
if (GraphInterpreter.Debug) println(s"${interpreter.Name} cancel id=$id")
|
||||
outputs(id).cancel()
|
||||
runBatch()
|
||||
case SubscribePending(id: Int) ⇒
|
||||
|
|
|
|||
|
|
@ -1,389 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.stream.impl.fusing
|
||||
|
||||
import akka.actor._
|
||||
import akka.stream.impl.ReactiveStreamsCompliance._
|
||||
import akka.stream.{ AbruptTerminationException, ActorMaterializerSettings, Attributes, ActorMaterializer }
|
||||
import akka.stream.actor.ActorSubscriber.OnSubscribe
|
||||
import akka.stream.actor.ActorSubscriberMessage.{ OnNext, OnError, OnComplete }
|
||||
import akka.stream.impl._
|
||||
import akka.stream.impl.fusing.OneBoundedInterpreter.{ InitializationFailed, InitializationSuccessful }
|
||||
import akka.stream.stage._
|
||||
import org.reactivestreams.{ Subscriber, Subscription }
|
||||
import akka.event.{ Logging, LoggingAdapter }
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] class BatchingActorInputBoundary(val size: Int, val name: String)
|
||||
extends BoundaryStage {
|
||||
|
||||
require(size > 0, "buffer size cannot be zero")
|
||||
require((size & (size - 1)) == 0, "buffer size must be a power of two")
|
||||
|
||||
// TODO: buffer and batch sizing heuristics
|
||||
private var upstream: Subscription = _
|
||||
private val inputBuffer = Array.ofDim[AnyRef](size)
|
||||
private var inputBufferElements = 0
|
||||
private var nextInputElementCursor = 0
|
||||
private var upstreamCompleted = false
|
||||
private var downstreamWaiting = false
|
||||
private var downstreamCanceled = false
|
||||
private val IndexMask = size - 1
|
||||
|
||||
private def requestBatchSize = math.max(1, inputBuffer.length / 2)
|
||||
private var batchRemaining = requestBatchSize
|
||||
|
||||
val subreceive: SubReceive = new SubReceive(waitingForUpstream)
|
||||
|
||||
def isFinished = upstreamCompleted && ((upstream ne null) || downstreamCanceled)
|
||||
|
||||
def setDownstreamCanceled(): Unit = downstreamCanceled = true
|
||||
|
||||
private def dequeue(): Any = {
|
||||
val elem = inputBuffer(nextInputElementCursor)
|
||||
require(elem ne null)
|
||||
inputBuffer(nextInputElementCursor) = null
|
||||
|
||||
batchRemaining -= 1
|
||||
if (batchRemaining == 0 && !upstreamCompleted) {
|
||||
tryRequest(upstream, requestBatchSize)
|
||||
batchRemaining = requestBatchSize
|
||||
}
|
||||
|
||||
inputBufferElements -= 1
|
||||
nextInputElementCursor = (nextInputElementCursor + 1) & IndexMask
|
||||
elem
|
||||
}
|
||||
|
||||
private def enqueue(elem: Any): Unit = {
|
||||
if (OneBoundedInterpreter.Debug) println(f" enq $elem%-19s $name")
|
||||
if (!upstreamCompleted) {
|
||||
if (inputBufferElements == size) throw new IllegalStateException("Input buffer overrun")
|
||||
inputBuffer((nextInputElementCursor + inputBufferElements) & IndexMask) = elem.asInstanceOf[AnyRef]
|
||||
inputBufferElements += 1
|
||||
}
|
||||
}
|
||||
|
||||
override def onPush(elem: Any, ctx: BoundaryContext): Directive =
|
||||
throw new UnsupportedOperationException("BUG: Cannot push the upstream boundary")
|
||||
|
||||
override def onPull(ctx: BoundaryContext): Directive = {
|
||||
if (inputBufferElements > 1) ctx.push(dequeue())
|
||||
else if (inputBufferElements == 1) {
|
||||
if (upstreamCompleted) ctx.pushAndFinish(dequeue())
|
||||
else ctx.push(dequeue())
|
||||
} else if (upstreamCompleted) {
|
||||
ctx.finish()
|
||||
} else {
|
||||
downstreamWaiting = true
|
||||
ctx.exit()
|
||||
}
|
||||
}
|
||||
|
||||
override def onDownstreamFinish(ctx: BoundaryContext): TerminationDirective = {
|
||||
cancel()
|
||||
ctx.finish()
|
||||
}
|
||||
|
||||
def cancel(): Unit = {
|
||||
if (!upstreamCompleted) {
|
||||
upstreamCompleted = true
|
||||
if (upstream ne null) tryCancel(upstream)
|
||||
downstreamWaiting = false
|
||||
clear()
|
||||
}
|
||||
}
|
||||
|
||||
private def clear(): Unit = {
|
||||
java.util.Arrays.fill(inputBuffer, 0, inputBuffer.length, null)
|
||||
inputBufferElements = 0
|
||||
}
|
||||
|
||||
private def onComplete(): Unit =
|
||||
if (!upstreamCompleted) {
|
||||
upstreamCompleted = true
|
||||
// onUpstreamFinish is not back-pressured, stages need to deal with this
|
||||
if (inputBufferElements == 0) enterAndFinish()
|
||||
}
|
||||
|
||||
private def onSubscribe(subscription: Subscription): Unit = {
|
||||
require(subscription != null)
|
||||
if (upstreamCompleted)
|
||||
tryCancel(subscription)
|
||||
else if (downstreamCanceled) {
|
||||
upstreamCompleted = true
|
||||
tryCancel(subscription)
|
||||
} else {
|
||||
upstream = subscription
|
||||
// Prefetch
|
||||
tryRequest(upstream, inputBuffer.length)
|
||||
subreceive.become(upstreamRunning)
|
||||
}
|
||||
}
|
||||
|
||||
// Call this when an error happens that does not come from the usual onError channel
|
||||
// (exceptions while calling RS interfaces, abrupt termination etc)
|
||||
def onInternalError(e: Throwable): Unit = {
|
||||
if (!(upstreamCompleted || downstreamCanceled) && (upstream ne null)) {
|
||||
upstream.cancel()
|
||||
}
|
||||
onError(e)
|
||||
}
|
||||
|
||||
def onError(e: Throwable): Unit = {
|
||||
if (!upstreamCompleted) {
|
||||
upstreamCompleted = true
|
||||
enterAndFail(e)
|
||||
}
|
||||
}
|
||||
|
||||
private def waitingForUpstream: Actor.Receive = {
|
||||
case OnComplete ⇒ onComplete()
|
||||
case OnSubscribe(subscription) ⇒ onSubscribe(subscription)
|
||||
case OnError(cause) ⇒ onError(cause)
|
||||
}
|
||||
|
||||
private def upstreamRunning: Actor.Receive = {
|
||||
case OnNext(element) ⇒
|
||||
enqueue(element)
|
||||
if (downstreamWaiting) {
|
||||
downstreamWaiting = false
|
||||
enterAndPush(dequeue())
|
||||
}
|
||||
|
||||
case OnComplete ⇒ onComplete()
|
||||
case OnError(cause) ⇒ onError(cause)
|
||||
case OnSubscribe(subscription) ⇒ tryCancel(subscription) // spec rule 2.5
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private[akka] object ActorOutputBoundary {
|
||||
/**
|
||||
* INTERNAL API.
|
||||
*/
|
||||
private case object ContinuePulling extends DeadLetterSuppression with NoSerializationVerificationNeeded
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] class ActorOutputBoundary(val actor: ActorRef,
|
||||
val debugLogging: Boolean,
|
||||
val log: LoggingAdapter,
|
||||
val outputBurstLimit: Int)
|
||||
extends BoundaryStage {
|
||||
import ReactiveStreamsCompliance._
|
||||
import ActorOutputBoundary._
|
||||
|
||||
private var exposedPublisher: ActorPublisher[Any] = _
|
||||
|
||||
private var subscriber: Subscriber[Any] = _
|
||||
private var downstreamDemand: Long = 0L
|
||||
// This flag is only used if complete/fail is called externally since this op turns into a Finished one inside the
|
||||
// interpreter (i.e. inside this op this flag has no effects since if it is completed the op will not be invoked)
|
||||
private var downstreamCompleted = false
|
||||
// this is true while we “hold the ball”; while “false” incoming demand will just be queued up
|
||||
private var upstreamWaiting = true
|
||||
// when upstream failed before we got the exposed publisher
|
||||
private var upstreamFailed: Option[Throwable] = None
|
||||
// the number of elements emitted during a single execution is bounded
|
||||
private var burstRemaining = outputBurstLimit
|
||||
|
||||
private def tryBounceBall(ctx: BoundaryContext) = {
|
||||
burstRemaining -= 1
|
||||
if (burstRemaining > 0) ctx.pull()
|
||||
else {
|
||||
actor ! ContinuePulling
|
||||
takeBallOut(ctx)
|
||||
}
|
||||
}
|
||||
|
||||
private def takeBallOut(ctx: BoundaryContext) = {
|
||||
upstreamWaiting = true
|
||||
ctx.exit()
|
||||
}
|
||||
|
||||
private def tryPutBallIn() =
|
||||
if (upstreamWaiting) {
|
||||
burstRemaining = outputBurstLimit
|
||||
upstreamWaiting = false
|
||||
enterAndPull()
|
||||
}
|
||||
|
||||
val subreceive = new SubReceive(waitingExposedPublisher)
|
||||
|
||||
private def onNext(elem: Any): Unit = {
|
||||
downstreamDemand -= 1
|
||||
tryOnNext(subscriber, elem)
|
||||
}
|
||||
|
||||
private def complete(): Unit = {
|
||||
if (!downstreamCompleted) {
|
||||
downstreamCompleted = true
|
||||
if (exposedPublisher ne null) exposedPublisher.shutdown(None)
|
||||
if (subscriber ne null) tryOnComplete(subscriber)
|
||||
}
|
||||
}
|
||||
|
||||
def fail(e: Throwable): Unit = {
|
||||
if (!downstreamCompleted) {
|
||||
downstreamCompleted = true
|
||||
if (debugLogging)
|
||||
log.debug("fail due to: {}", e.getMessage)
|
||||
if (exposedPublisher ne null) exposedPublisher.shutdown(Some(e))
|
||||
if ((subscriber ne null) && !e.isInstanceOf[SpecViolation]) tryOnError(subscriber, e)
|
||||
} else if (exposedPublisher == null && upstreamFailed.isEmpty) {
|
||||
// fail called before the exposed publisher arrived, we must store it and fail when we're first able to
|
||||
upstreamFailed = Some(e)
|
||||
}
|
||||
}
|
||||
|
||||
override def onPush(elem: Any, ctx: BoundaryContext): Directive = {
|
||||
onNext(elem)
|
||||
if (downstreamCompleted) ctx.finish()
|
||||
else if (downstreamDemand > 0) tryBounceBall(ctx)
|
||||
else takeBallOut(ctx)
|
||||
}
|
||||
|
||||
override def onPull(ctx: BoundaryContext): Directive =
|
||||
throw new UnsupportedOperationException("BUG: Cannot pull the downstream boundary")
|
||||
|
||||
override def onUpstreamFinish(ctx: BoundaryContext): TerminationDirective = {
|
||||
complete()
|
||||
ctx.finish()
|
||||
}
|
||||
|
||||
override def onUpstreamFailure(cause: Throwable, ctx: BoundaryContext): TerminationDirective = {
|
||||
fail(cause)
|
||||
ctx.fail(cause)
|
||||
}
|
||||
|
||||
private def subscribePending(subscribers: Seq[Subscriber[Any]]): Unit =
|
||||
subscribers foreach { sub ⇒
|
||||
if (subscriber eq null) {
|
||||
subscriber = sub
|
||||
tryOnSubscribe(subscriber, new ActorSubscription(actor, subscriber))
|
||||
} else
|
||||
rejectAdditionalSubscriber(subscriber, s"${Logging.simpleName(this)}")
|
||||
}
|
||||
|
||||
protected def waitingExposedPublisher: Actor.Receive = {
|
||||
case ExposedPublisher(publisher) ⇒
|
||||
upstreamFailed match {
|
||||
case _: Some[_] ⇒
|
||||
publisher.shutdown(upstreamFailed)
|
||||
case _ ⇒
|
||||
exposedPublisher = publisher
|
||||
subreceive.become(downstreamRunning)
|
||||
}
|
||||
case other ⇒
|
||||
throw new IllegalStateException(s"The first message must be ExposedPublisher but was [$other]")
|
||||
}
|
||||
|
||||
protected def downstreamRunning: Actor.Receive = {
|
||||
case SubscribePending ⇒
|
||||
subscribePending(exposedPublisher.takePendingSubscribers())
|
||||
case RequestMore(subscription, elements) ⇒
|
||||
if (elements < 1) {
|
||||
enterAndFinish()
|
||||
fail(ReactiveStreamsCompliance.numberOfElementsInRequestMustBePositiveException)
|
||||
} else {
|
||||
downstreamDemand += elements
|
||||
if (downstreamDemand < 0)
|
||||
downstreamDemand = Long.MaxValue // Long overflow, Reactive Streams Spec 3:17: effectively unbounded
|
||||
if (OneBoundedInterpreter.Debug) {
|
||||
val s = s"$downstreamDemand (+$elements)"
|
||||
println(f" dem $s%-19s ${actor.path}")
|
||||
}
|
||||
tryPutBallIn()
|
||||
}
|
||||
|
||||
case ContinuePulling ⇒
|
||||
if (!downstreamCompleted && downstreamDemand > 0) tryPutBallIn()
|
||||
|
||||
case Cancel(subscription) ⇒
|
||||
downstreamCompleted = true
|
||||
subscriber = null
|
||||
exposedPublisher.shutdown(Some(new ActorPublisher.NormalShutdownException))
|
||||
enterAndFinish()
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] object ActorInterpreter {
|
||||
def props(settings: ActorMaterializerSettings, ops: Seq[Stage[_, _]], materializer: ActorMaterializer, attributes: Attributes = Attributes.none): Props =
|
||||
Props(new ActorInterpreter(settings, ops, materializer, attributes)).withDeploy(Deploy.local)
|
||||
|
||||
case class AsyncInput(op: AsyncStage[Any, Any, Any], ctx: AsyncContext[Any, Any], event: Any) extends DeadLetterSuppression with NoSerializationVerificationNeeded
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] class ActorInterpreter(val settings: ActorMaterializerSettings, val ops: Seq[Stage[_, _]], val materializer: ActorMaterializer, val attributes: Attributes)
|
||||
extends Actor with ActorLogging {
|
||||
import ActorInterpreter._
|
||||
|
||||
private val upstream = new BatchingActorInputBoundary(settings.initialInputBufferSize, context.self.path.toString)
|
||||
private val downstream = new ActorOutputBoundary(self, settings.debugLogging, log, settings.outputBurstLimit)
|
||||
private val interpreter =
|
||||
new OneBoundedInterpreter(upstream +: ops :+ downstream,
|
||||
(op, ctx, event) ⇒ self ! AsyncInput(op, ctx, event),
|
||||
Logging(this),
|
||||
materializer,
|
||||
attributes,
|
||||
name = context.self.path.toString)
|
||||
|
||||
interpreter.init() match {
|
||||
case failed: InitializationFailed ⇒
|
||||
// the Actor will be stopped thanks to aroundReceive checking interpreter.isFinished
|
||||
upstream.setDownstreamCanceled()
|
||||
downstream.fail(failed.mostDownstream.ex)
|
||||
case InitializationSuccessful ⇒ // ok
|
||||
}
|
||||
|
||||
def receive: Receive =
|
||||
upstream.subreceive
|
||||
.orElse[Any, Unit](downstream.subreceive)
|
||||
.orElse[Any, Unit] {
|
||||
case AsyncInput(op, ctx, event) ⇒
|
||||
ctx.enter()
|
||||
op.onAsyncInput(event, ctx)
|
||||
ctx.execute()
|
||||
}
|
||||
|
||||
override protected[akka] def aroundReceive(receive: Actor.Receive, msg: Any): Unit = {
|
||||
super.aroundReceive(receive, msg)
|
||||
|
||||
if (interpreter.isFinished) {
|
||||
if (upstream.isFinished) context.stop(self)
|
||||
else upstream.setDownstreamCanceled()
|
||||
}
|
||||
}
|
||||
|
||||
override def postStop(): Unit = {
|
||||
// This should handle termination while interpreter is running. If the upstream have been closed already this
|
||||
// call has no effect and therefore do the right thing: nothing.
|
||||
val ex = AbruptTerminationException(self)
|
||||
try upstream.onInternalError(ex)
|
||||
// Will only have an effect if the above call to the interpreter failed to emit a proper failure to the downstream
|
||||
// otherwise this will have no effect
|
||||
finally {
|
||||
downstream.fail(ex)
|
||||
upstream.cancel()
|
||||
}
|
||||
}
|
||||
|
||||
override def postRestart(reason: Throwable): Unit = {
|
||||
super.postRestart(reason)
|
||||
throw new IllegalStateException("This actor cannot be restarted", reason)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -7,9 +7,9 @@ import java.util.Arrays
|
|||
|
||||
import akka.event.LoggingAdapter
|
||||
import akka.stream.stage._
|
||||
import akka.stream.{ Materializer, Shape, Inlet, Outlet }
|
||||
import scala.annotation.tailrec
|
||||
import scala.collection.immutable
|
||||
import akka.stream._
|
||||
import scala.util.control.NonFatal
|
||||
|
||||
/**
|
||||
|
|
@ -55,12 +55,17 @@ private[stream] object GraphInterpreter {
|
|||
def in: Inlet[T]
|
||||
}
|
||||
|
||||
val singleNoAttribute: Array[Attributes] = Array(Attributes.none)
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*
|
||||
* A GraphAssembly represents a small stream processing graph to be executed by the interpreter. Instances of this
|
||||
* class **must not** be mutated after construction.
|
||||
*
|
||||
* The array ``originalAttributes`` may contain the attribute information of the original atomic module, otherwise
|
||||
* it must contain a none (otherwise the enclosing module could not overwrite attributes defined in this array).
|
||||
*
|
||||
* The arrays [[ins]] and [[outs]] correspond to the notion of a *connection* in the [[GraphInterpreter]]. Each slot
|
||||
* *i* contains the input and output port corresponding to connection *i*. Slots where the graph is not closed (i.e.
|
||||
* ports are exposed to the external world) are marked with *null* values. For example if an input port *p* is
|
||||
|
|
@ -88,6 +93,7 @@ private[stream] object GraphInterpreter {
|
|||
*
|
||||
*/
|
||||
final class GraphAssembly(val stages: Array[GraphStageWithMaterializedValue[Shape, Any]],
|
||||
val originalAttributes: Array[Attributes],
|
||||
val ins: Array[Inlet[_]],
|
||||
val inOwners: Array[Int],
|
||||
val outs: Array[Outlet[_]],
|
||||
|
|
@ -106,7 +112,7 @@ private[stream] object GraphInterpreter {
|
|||
* - array of the logics
|
||||
* - materialized value
|
||||
*/
|
||||
def materialize(): (Array[InHandler], Array[OutHandler], Array[GraphStageLogic], Any) = {
|
||||
def materialize(inheritedAttributes: Attributes): (Array[InHandler], Array[OutHandler], Array[GraphStageLogic], Any) = {
|
||||
val logics = Array.ofDim[GraphStageLogic](stages.length)
|
||||
var finalMat: Any = ()
|
||||
|
||||
|
|
@ -134,7 +140,7 @@ private[stream] object GraphInterpreter {
|
|||
}
|
||||
|
||||
// FIXME: Support for materialized values in fused islands is not yet figured out!
|
||||
val logicAndMat = stages(i).createLogicAndMaterializedValue
|
||||
val logicAndMat = stages(i).createLogicAndMaterializedValue(inheritedAttributes and originalAttributes(i))
|
||||
// FIXME: Current temporary hack to support non-fused stages. If there is one stage that will be under index 0.
|
||||
if (i == 0) finalMat = logicAndMat._2
|
||||
|
||||
|
|
@ -148,20 +154,21 @@ private[stream] object GraphInterpreter {
|
|||
i = 0
|
||||
while (i < connectionCount) {
|
||||
if (ins(i) ne null) {
|
||||
val l = logics(inOwners(i))
|
||||
l.inHandlers(ins(i).id) match {
|
||||
case null ⇒ throw new IllegalStateException(s"no handler defined in stage $l for port ${ins(i)}")
|
||||
case h ⇒ inHandlers(i) = h
|
||||
val logic = logics(inOwners(i))
|
||||
logic.handlers(ins(i).id) match {
|
||||
case null ⇒ throw new IllegalStateException(s"no handler defined in stage $logic for port ${ins(i)}")
|
||||
case h: InHandler ⇒ inHandlers(i) = h
|
||||
}
|
||||
l.inToConn(ins(i).id) = i
|
||||
logics(inOwners(i)).portToConn(ins(i).id) = i
|
||||
}
|
||||
if (outs(i) ne null) {
|
||||
val l = logics(outOwners(i))
|
||||
l.outHandlers(outs(i).id) match {
|
||||
case null ⇒ throw new IllegalStateException(s"no handler defined in stage $l for port ${outs(i)}")
|
||||
case h ⇒ outHandlers(i) = h
|
||||
val logic = logics(outOwners(i))
|
||||
val inCount = logic.inCount
|
||||
logic.handlers(outs(i).id + inCount) match {
|
||||
case null ⇒ throw new IllegalStateException(s"no handler defined in stage $logic for port ${outs(i)}")
|
||||
case h: OutHandler ⇒ outHandlers(i) = h
|
||||
}
|
||||
l.outToConn(outs(i).id) = i
|
||||
logic.portToConn(outs(i).id + inCount) = i
|
||||
}
|
||||
i += 1
|
||||
}
|
||||
|
|
@ -206,6 +213,7 @@ private[stream] object GraphInterpreter {
|
|||
|
||||
val assembly = new GraphAssembly(
|
||||
stages.toArray,
|
||||
GraphInterpreter.singleNoAttribute,
|
||||
add(inlets.iterator, Array.ofDim(connectionCount), 0),
|
||||
markBoundary(Array.ofDim(connectionCount), inletsSize, connectionCount),
|
||||
add(outlets.iterator, Array.ofDim(connectionCount), inletsSize),
|
||||
|
|
@ -288,7 +296,7 @@ private[stream] object GraphInterpreter {
|
|||
*
|
||||
* Because of the FIFO construction of the queue the interpreter is fair, i.e. a pending event is always executed
|
||||
* after a bounded number of other events. This property, together with suspendability means that even infinite cycles can
|
||||
* be modeled, or even dissolved (if preempted and a "stealing" external even is injected; for example the non-cycle
|
||||
* be modeled, or even dissolved (if preempted and a "stealing" external event is injected; for example the non-cycle
|
||||
* edge of a balance is pulled, dissolving the original cycle).
|
||||
*/
|
||||
private[stream] final class GraphInterpreter(
|
||||
|
|
@ -309,7 +317,7 @@ private[stream] final class GraphInterpreter(
|
|||
// of the class for a full description.
|
||||
val portStates = Array.fill[Int](assembly.connectionCount)(InReady)
|
||||
|
||||
private[this] var activeStageId = Boundary
|
||||
private[this] var activeStage: GraphStageLogic = _
|
||||
|
||||
// The number of currently running stages. Once this counter reaches zero, the interpreter is considered to be
|
||||
// completed
|
||||
|
|
@ -323,19 +331,33 @@ private[stream] final class GraphInterpreter(
|
|||
|
||||
// An event queue implemented as a circular buffer
|
||||
// FIXME: This calculates the maximum size ever needed, but most assemblies can run on a smaller queue
|
||||
private[this] val eventQueue = Array.ofDim[Int](1 << Integer.highestOneBit(assembly.connectionCount))
|
||||
private[this] val eventQueue = Array.ofDim[Int](1 << (32 - Integer.numberOfLeadingZeros(assembly.connectionCount - 1)))
|
||||
private[this] val mask = eventQueue.length - 1
|
||||
private[this] var queueHead: Int = 0
|
||||
private[this] var queueTail: Int = 0
|
||||
|
||||
private def queueStatus: String = {
|
||||
val contents = (queueHead until queueTail).map(idx ⇒ {
|
||||
val conn = eventQueue(idx & mask)
|
||||
(conn, portStates(conn), connectionSlots(conn))
|
||||
})
|
||||
s"(${eventQueue.length}, $queueHead, $queueTail)(${contents.mkString(", ")})"
|
||||
}
|
||||
private[this] var _Name: String = _
|
||||
def Name: String =
|
||||
if (_Name eq null) {
|
||||
_Name = f"${System.identityHashCode(this)}%08X"
|
||||
_Name
|
||||
} else _Name
|
||||
|
||||
/**
|
||||
* Assign the boundary logic to a given connection. This will serve as the interface to the external world
|
||||
* (outside the interpreter) to process and inject events.
|
||||
*/
|
||||
def attachUpstreamBoundary(connection: Int, logic: UpstreamBoundaryStageLogic[_]): Unit = {
|
||||
logic.outToConn(logic.out.id) = connection
|
||||
logic.portToConn(logic.out.id + logic.inCount) = connection
|
||||
logic.interpreter = this
|
||||
outHandlers(connection) = logic.outHandlers(0)
|
||||
outHandlers(connection) = logic.handlers(0).asInstanceOf[OutHandler]
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -343,16 +365,16 @@ private[stream] final class GraphInterpreter(
|
|||
* (outside the interpreter) to process and inject events.
|
||||
*/
|
||||
def attachDownstreamBoundary(connection: Int, logic: DownstreamBoundaryStageLogic[_]): Unit = {
|
||||
logic.inToConn(logic.in.id) = connection
|
||||
logic.portToConn(logic.in.id) = connection
|
||||
logic.interpreter = this
|
||||
inHandlers(connection) = logic.inHandlers(0)
|
||||
inHandlers(connection) = logic.handlers(0).asInstanceOf[InHandler]
|
||||
}
|
||||
|
||||
/**
|
||||
* Dynamic handler changes are communicated from a GraphStageLogic by this method.
|
||||
*/
|
||||
def setHandler(connection: Int, handler: InHandler): Unit = {
|
||||
if (GraphInterpreter.Debug) println(s"SETHANDLER ${inOwnerName(connection)} (in) $handler")
|
||||
if (Debug) println(s"$Name SETHANDLER ${inOwnerName(connection)} (in) $handler")
|
||||
inHandlers(connection) = handler
|
||||
}
|
||||
|
||||
|
|
@ -360,7 +382,7 @@ private[stream] final class GraphInterpreter(
|
|||
* Dynamic handler changes are communicated from a GraphStageLogic by this method.
|
||||
*/
|
||||
def setHandler(connection: Int, handler: OutHandler): Unit = {
|
||||
if (GraphInterpreter.Debug) println(s"SETHANDLER ${outOwnerName(connection)} (out) $handler")
|
||||
if (Debug) println(s"$Name SETHANDLER ${outOwnerName(connection)} (out) $handler")
|
||||
outHandlers(connection) = handler
|
||||
}
|
||||
|
||||
|
|
@ -389,6 +411,7 @@ private[stream] final class GraphInterpreter(
|
|||
} catch {
|
||||
case NonFatal(e) ⇒ logic.failStage(e)
|
||||
}
|
||||
afterStageHasRun(logic)
|
||||
i += 1
|
||||
}
|
||||
}
|
||||
|
|
@ -399,7 +422,8 @@ private[stream] final class GraphInterpreter(
|
|||
def finish(): Unit = {
|
||||
var i = 0
|
||||
while (i < logics.length) {
|
||||
if (!isStageCompleted(i)) finalizeStage(logics(i))
|
||||
val logic = logics(i)
|
||||
if (!isStageCompleted(logic)) finalizeStage(logic)
|
||||
i += 1
|
||||
}
|
||||
}
|
||||
|
|
@ -418,84 +442,97 @@ private[stream] final class GraphInterpreter(
|
|||
case owner ⇒ assembly.stages(owner).toString
|
||||
}
|
||||
|
||||
// Debug name for a connections input part
|
||||
private def inLogicName(connection: Int): String =
|
||||
assembly.inOwners(connection) match {
|
||||
case Boundary ⇒ "DownstreamBoundary"
|
||||
case owner ⇒ logics(owner).toString
|
||||
}
|
||||
|
||||
// Debug name for a connections ouput part
|
||||
private def outLogicName(connection: Int): String =
|
||||
assembly.outOwners(connection) match {
|
||||
case Boundary ⇒ "UpstreamBoundary"
|
||||
case owner ⇒ logics(owner).toString
|
||||
}
|
||||
|
||||
/**
|
||||
* Executes pending events until the given limit is met. If there were remaining events, isSuspended will return
|
||||
* true.
|
||||
*/
|
||||
def execute(eventLimit: Int): Unit = {
|
||||
if (GraphInterpreter.Debug) println("---------------- EXECUTE")
|
||||
if (Debug) println(s"$Name ---------------- EXECUTE (running=$runningStages, shutdown=${shutdownCounter.mkString(",")})")
|
||||
var eventsRemaining = eventLimit
|
||||
var connection = dequeue()
|
||||
while (eventsRemaining > 0 && connection != NoEvent) {
|
||||
try processEvent(connection)
|
||||
catch {
|
||||
case NonFatal(e) ⇒
|
||||
if (activeStageId == Boundary) throw e
|
||||
else logics(activeStageId).failStage(e)
|
||||
if (activeStage == null) throw e
|
||||
else activeStage.failStage(e)
|
||||
}
|
||||
afterStageHasRun(activeStage)
|
||||
eventsRemaining -= 1
|
||||
if (eventsRemaining > 0) connection = dequeue()
|
||||
}
|
||||
if (Debug) println(s"$Name ---------------- $queueStatus (running=$runningStages, shutdown=${shutdownCounter.mkString(",")})")
|
||||
// TODO: deadlock detection
|
||||
}
|
||||
|
||||
// Decodes and processes a single event for the given connection
|
||||
private def processEvent(connection: Int): Unit = {
|
||||
def safeLogics(id: Int) =
|
||||
if (id == Boundary) null
|
||||
else logics(id)
|
||||
|
||||
def processElement(elem: Any): Unit = {
|
||||
if (GraphInterpreter.Debug) println(s"PUSH ${outOwnerName(connection)} -> ${inOwnerName(connection)}, $elem (${inHandlers(connection)})")
|
||||
activeStageId = assembly.inOwners(connection)
|
||||
def processElement(): Unit = {
|
||||
if (Debug) println(s"$Name PUSH ${outOwnerName(connection)} -> ${inOwnerName(connection)}, ${connectionSlots(connection)} (${inHandlers(connection)}) [${inLogicName(connection)}]")
|
||||
activeStage = safeLogics(assembly.inOwners(connection))
|
||||
portStates(connection) ^= PushEndFlip
|
||||
inHandlers(connection).onPush()
|
||||
}
|
||||
|
||||
// this must be the state after returning without delivering any signals, to avoid double-finalization of some unlucky stage
|
||||
// (this can happen if a stage completes voluntarily while connection close events are still queued)
|
||||
activeStage = null
|
||||
val code = portStates(connection)
|
||||
|
||||
// Manual fast decoding, fast paths are PUSH and PULL
|
||||
// PUSH
|
||||
if ((code & (Pushing | InClosed | OutClosed)) == Pushing) {
|
||||
processElement(connectionSlots(connection))
|
||||
processElement()
|
||||
|
||||
// PULL
|
||||
} else if ((code & (Pulling | OutClosed | InClosed)) == Pulling) {
|
||||
if (GraphInterpreter.Debug) println(s"PULL ${inOwnerName(connection)} -> ${outOwnerName(connection)} (${outHandlers(connection)})")
|
||||
if (Debug) println(s"$Name PULL ${inOwnerName(connection)} -> ${outOwnerName(connection)} (${outHandlers(connection)}) [${outLogicName(connection)}]")
|
||||
portStates(connection) ^= PullEndFlip
|
||||
activeStageId = assembly.outOwners(connection)
|
||||
activeStage = safeLogics(assembly.outOwners(connection))
|
||||
outHandlers(connection).onPull()
|
||||
|
||||
// CANCEL
|
||||
} else if ((code & (OutClosed | InClosed)) == InClosed) {
|
||||
val stageId = assembly.outOwners(connection)
|
||||
if (GraphInterpreter.Debug) println(s"CANCEL ${inOwnerName(connection)} -> ${outOwnerName(connection)} (${outHandlers(connection)})")
|
||||
activeStage = safeLogics(stageId)
|
||||
if (Debug) println(s"$Name CANCEL ${inOwnerName(connection)} -> ${outOwnerName(connection)} (${outHandlers(connection)}) [${outLogicName(connection)}]")
|
||||
portStates(connection) |= OutClosed
|
||||
activeStageId = assembly.outOwners(connection)
|
||||
outHandlers(connection).onDownstreamFinish()
|
||||
completeConnection(stageId)
|
||||
outHandlers(connection).onDownstreamFinish()
|
||||
} else if ((code & (OutClosed | InClosed)) == OutClosed) {
|
||||
// COMPLETIONS
|
||||
|
||||
val stageId = assembly.inOwners(connection)
|
||||
|
||||
if ((code & Pushing) == 0) {
|
||||
// Normal completion (no push pending)
|
||||
if (GraphInterpreter.Debug) println(s"COMPLETE ${outOwnerName(connection)} -> ${inOwnerName(connection)} (${inHandlers(connection)})")
|
||||
if (Debug) println(s"$Name COMPLETE ${outOwnerName(connection)} -> ${inOwnerName(connection)} (${inHandlers(connection)}) [${inLogicName(connection)}]")
|
||||
portStates(connection) |= InClosed
|
||||
activeStageId = assembly.inOwners(connection)
|
||||
val stageId = assembly.inOwners(connection)
|
||||
activeStage = safeLogics(stageId)
|
||||
completeConnection(stageId)
|
||||
if ((portStates(connection) & InFailed) == 0) inHandlers(connection).onUpstreamFinish()
|
||||
else inHandlers(connection).onUpstreamFailure(connectionSlots(connection).asInstanceOf[Failed].ex)
|
||||
completeConnection(stageId)
|
||||
} else {
|
||||
// Push is pending, first process push, then re-enqueue closing event
|
||||
// Non-failure case
|
||||
val code = portStates(connection) & (InClosed | InFailed)
|
||||
if (code == 0) {
|
||||
processElement(connectionSlots(connection))
|
||||
enqueue(connection)
|
||||
} else if (code == InFailed) {
|
||||
// Failure case
|
||||
processElement(connectionSlots(connection).asInstanceOf[Failed].previousElem)
|
||||
enqueue(connection)
|
||||
}
|
||||
processElement()
|
||||
enqueue(connection)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -513,26 +550,26 @@ private[stream] final class GraphInterpreter(
|
|||
}
|
||||
|
||||
private def enqueue(connection: Int): Unit = {
|
||||
if (Debug) if (queueTail - queueHead > mask) new Exception(s"$Name internal queue full ($queueStatus) + $connection").printStackTrace()
|
||||
eventQueue(queueTail & mask) = connection
|
||||
queueTail += 1
|
||||
}
|
||||
|
||||
def afterStageHasRun(logic: GraphStageLogic): Unit =
|
||||
if (isStageCompleted(logic)) {
|
||||
runningStages -= 1
|
||||
finalizeStage(logic)
|
||||
}
|
||||
|
||||
// Returns true if the given stage is alredy completed
|
||||
def isStageCompleted(stageId: Int): Boolean = stageId != Boundary && shutdownCounter(stageId) == 0
|
||||
def isStageCompleted(stage: GraphStageLogic): Boolean = stage != null && shutdownCounter(stage.stageId) == 0
|
||||
|
||||
// Register that a connection in which the given stage participated has been completed and therefore the stage
|
||||
// itself might stop, too.
|
||||
private def completeConnection(stageId: Int): Unit = {
|
||||
if (stageId != Boundary) {
|
||||
val activeConnections = shutdownCounter(stageId)
|
||||
if (activeConnections > 0) {
|
||||
shutdownCounter(stageId) = activeConnections - 1
|
||||
// This was the last active connection keeping this stage alive
|
||||
if (activeConnections == 1) {
|
||||
runningStages -= 1
|
||||
finalizeStage(logics(stageId))
|
||||
}
|
||||
}
|
||||
if (activeConnections > 0) shutdownCounter(stageId) = activeConnections - 1
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -564,35 +601,32 @@ private[stream] final class GraphInterpreter(
|
|||
|
||||
private[stream] def complete(connection: Int): Unit = {
|
||||
val currentState = portStates(connection)
|
||||
if (Debug) println(s"$Name complete($connection) [$currentState]")
|
||||
portStates(connection) = currentState | OutClosed
|
||||
|
||||
if ((currentState & (InClosed | Pushing)) == 0) {
|
||||
enqueue(connection)
|
||||
}
|
||||
|
||||
completeConnection(assembly.outOwners(connection))
|
||||
if ((currentState & (InClosed | Pushing | Pulling)) == 0) enqueue(connection)
|
||||
if ((currentState & OutClosed) == 0) completeConnection(assembly.outOwners(connection))
|
||||
}
|
||||
|
||||
private[stream] def fail(connection: Int, ex: Throwable): Unit = {
|
||||
val currentState = portStates(connection)
|
||||
if (Debug) println(s"$Name fail($connection, $ex) [$currentState]")
|
||||
portStates(connection) = currentState | (OutClosed | InFailed)
|
||||
if ((currentState & InClosed) == 0) {
|
||||
connectionSlots(connection) = Failed(ex, connectionSlots(connection))
|
||||
enqueue(connection)
|
||||
if ((currentState & (Pulling | Pushing)) == 0) enqueue(connection)
|
||||
}
|
||||
|
||||
completeConnection(assembly.outOwners(connection))
|
||||
if ((currentState & OutClosed) == 0) completeConnection(assembly.outOwners(connection))
|
||||
}
|
||||
|
||||
private[stream] def cancel(connection: Int): Unit = {
|
||||
val currentState = portStates(connection)
|
||||
if (Debug) println(s"$Name cancel($connection) [$currentState]")
|
||||
portStates(connection) = currentState | InClosed
|
||||
if ((currentState & OutClosed) == 0) {
|
||||
connectionSlots(connection) = Empty
|
||||
enqueue(connection)
|
||||
if ((currentState & (Pulling | Pushing)) == 0) enqueue(connection)
|
||||
}
|
||||
|
||||
completeConnection(assembly.inOwners(connection))
|
||||
if ((currentState & InClosed) == 0) completeConnection(assembly.inOwners(connection))
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -24,21 +24,18 @@ object GraphStages {
|
|||
val in = Inlet[T]("in")
|
||||
val out = Outlet[T]("out")
|
||||
override val shape = FlowShape(in, out)
|
||||
|
||||
protected abstract class SimpleLinearStageLogic extends GraphStageLogic(shape) {
|
||||
setHandler(out, new OutHandler {
|
||||
override def onPull(): Unit = pull(in)
|
||||
})
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
class Identity[T] extends SimpleLinearGraphStage[T] {
|
||||
|
||||
override def createLogic: GraphStageLogic = new SimpleLinearStageLogic() {
|
||||
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new GraphStageLogic(shape) {
|
||||
setHandler(in, new InHandler {
|
||||
override def onPush(): Unit = push(out, grab(in))
|
||||
})
|
||||
|
||||
setHandler(out, new OutHandler {
|
||||
override def onPull(): Unit = pull(in)
|
||||
})
|
||||
}
|
||||
|
||||
override def toString = "Identity"
|
||||
|
|
@ -49,7 +46,7 @@ object GraphStages {
|
|||
val out = Outlet[T]("out")
|
||||
override val shape = FlowShape(in, out)
|
||||
|
||||
override def createLogic: GraphStageLogic = new GraphStageLogic(shape) {
|
||||
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new GraphStageLogic(shape) {
|
||||
var initialized = false
|
||||
|
||||
setHandler(in, new InHandler {
|
||||
|
|
@ -99,13 +96,13 @@ object GraphStages {
|
|||
val out = Outlet[T]("TimerSource.out")
|
||||
override val shape = SourceShape(out)
|
||||
|
||||
override def createLogicAndMaterializedValue: (GraphStageLogic, Cancellable) = {
|
||||
override def createLogicAndMaterializedValue(inheritedAttributes: Attributes): (GraphStageLogic, Cancellable) = {
|
||||
import TickSource._
|
||||
|
||||
val cancelled = new AtomicBoolean(false)
|
||||
val cancellable = new TickSourceCancellable(cancelled)
|
||||
|
||||
val logic = new GraphStageLogic(shape) {
|
||||
val logic = new TimerGraphStageLogic(shape) {
|
||||
override def preStart() = {
|
||||
schedulePeriodicallyWithInitialDelay("TickTimer", initialDelay, interval)
|
||||
val callback = getAsyncCallback[Unit]((_) ⇒ {
|
||||
|
|
|
|||
|
|
@ -3,64 +3,71 @@
|
|||
*/
|
||||
package akka.stream.impl.fusing
|
||||
|
||||
import akka.event.NoLogging
|
||||
import akka.event.{ Logging, NoLogging }
|
||||
import akka.stream._
|
||||
import akka.stream.impl.fusing.GraphInterpreter.{ GraphAssembly, DownstreamBoundaryStageLogic, UpstreamBoundaryStageLogic }
|
||||
import akka.stream.stage.AbstractStage.PushPullGraphStage
|
||||
import akka.stream.stage._
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] object IteratorInterpreter {
|
||||
final case class IteratorUpstream[T](input: Iterator[T]) extends PushPullStage[T, T] {
|
||||
|
||||
final case class IteratorUpstream[T](input: Iterator[T]) extends UpstreamBoundaryStageLogic[T] {
|
||||
val out: Outlet[T] = Outlet[T]("IteratorUpstream.out")
|
||||
out.id = 0
|
||||
|
||||
private var hasNext = input.hasNext
|
||||
|
||||
override def onPush(elem: T, ctx: Context[T]): SyncDirective =
|
||||
throw new UnsupportedOperationException("IteratorUpstream operates as a source, it cannot be pushed")
|
||||
|
||||
override def onPull(ctx: Context[T]): SyncDirective = {
|
||||
if (!hasNext) ctx.finish()
|
||||
else {
|
||||
val elem = input.next()
|
||||
hasNext = input.hasNext
|
||||
if (!hasNext) ctx.pushAndFinish(elem)
|
||||
else ctx.push(elem)
|
||||
setHandler(out, new OutHandler {
|
||||
override def onPull(): Unit = {
|
||||
if (!hasNext) completeStage()
|
||||
else {
|
||||
val elem = input.next()
|
||||
hasNext = input.hasNext
|
||||
if (!hasNext) {
|
||||
push(out, elem)
|
||||
complete(out)
|
||||
} else push(out, elem)
|
||||
}
|
||||
}
|
||||
})
|
||||
|
||||
}
|
||||
|
||||
// don't let toString consume the iterator
|
||||
override def toString: String = "IteratorUpstream"
|
||||
override def toString = "IteratorUpstream"
|
||||
}
|
||||
|
||||
final case class IteratorDownstream[T]() extends BoundaryStage with Iterator[T] {
|
||||
final case class IteratorDownstream[T]() extends DownstreamBoundaryStageLogic[T] with Iterator[T] {
|
||||
val in: Inlet[T] = Inlet[T]("IteratorDownstream.in")
|
||||
in.id = 0
|
||||
|
||||
private var done = false
|
||||
private var nextElem: T = _
|
||||
private var needsPull = true
|
||||
private var lastFailure: Throwable = null
|
||||
|
||||
override def onPush(elem: Any, ctx: BoundaryContext): Directive = {
|
||||
nextElem = elem.asInstanceOf[T]
|
||||
needsPull = false
|
||||
ctx.exit()
|
||||
}
|
||||
setHandler(in, new InHandler {
|
||||
override def onPush(): Unit = {
|
||||
nextElem = grab(in)
|
||||
needsPull = false
|
||||
}
|
||||
|
||||
override def onPull(ctx: BoundaryContext): Directive =
|
||||
throw new UnsupportedOperationException("IteratorDownstream operates as a sink, it cannot be pulled")
|
||||
override def onUpstreamFinish(): Unit = {
|
||||
done = true
|
||||
completeStage()
|
||||
}
|
||||
|
||||
override def onUpstreamFinish(ctx: BoundaryContext): TerminationDirective = {
|
||||
done = true
|
||||
ctx.finish()
|
||||
}
|
||||
|
||||
override def onUpstreamFailure(cause: Throwable, ctx: BoundaryContext): TerminationDirective = {
|
||||
done = true
|
||||
lastFailure = cause
|
||||
ctx.finish()
|
||||
}
|
||||
override def onUpstreamFailure(cause: Throwable): Unit = {
|
||||
done = true
|
||||
lastFailure = cause
|
||||
completeStage()
|
||||
}
|
||||
})
|
||||
|
||||
private def pullIfNeeded(): Unit = {
|
||||
if (needsPull) {
|
||||
enterAndPull() // will eventually result in a finish, or an onPush which exits
|
||||
pull(in)
|
||||
interpreter.execute(Int.MaxValue)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -84,8 +91,8 @@ private[akka] object IteratorInterpreter {
|
|||
|
||||
// don't let toString consume the iterator
|
||||
override def toString: String = "IteratorDownstream"
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -96,11 +103,52 @@ private[akka] class IteratorInterpreter[I, O](val input: Iterator[I], val ops: S
|
|||
|
||||
private val upstream = IteratorUpstream(input)
|
||||
private val downstream = IteratorDownstream[O]()
|
||||
private val interpreter = new OneBoundedInterpreter(upstream +: ops.asInstanceOf[Seq[Stage[_, _]]] :+ downstream,
|
||||
(op, ctx, evt) ⇒ throw new UnsupportedOperationException("IteratorInterpreter is fully synchronous"),
|
||||
NoLogging,
|
||||
NoMaterializer)
|
||||
interpreter.init()
|
||||
|
||||
private def init(): Unit = {
|
||||
import GraphInterpreter.Boundary
|
||||
|
||||
var i = 0
|
||||
val length = ops.length
|
||||
val attributes = Array.fill[Attributes](ops.length)(Attributes.none)
|
||||
val ins = Array.ofDim[Inlet[_]](length + 1)
|
||||
val inOwners = Array.ofDim[Int](length + 1)
|
||||
val outs = Array.ofDim[Outlet[_]](length + 1)
|
||||
val outOwners = Array.ofDim[Int](length + 1)
|
||||
val stages = Array.ofDim[GraphStageWithMaterializedValue[Shape, Any]](length)
|
||||
|
||||
ins(ops.length) = null
|
||||
inOwners(ops.length) = Boundary
|
||||
outs(0) = null
|
||||
outOwners(0) = Boundary
|
||||
|
||||
val opsIterator = ops.iterator
|
||||
while (opsIterator.hasNext) {
|
||||
val op = opsIterator.next().asInstanceOf[Stage[Any, Any]]
|
||||
val stage = new PushPullGraphStage((_) ⇒ op, Attributes.none)
|
||||
stages(i) = stage
|
||||
ins(i) = stage.shape.inlet
|
||||
inOwners(i) = i
|
||||
outs(i + 1) = stage.shape.outlet
|
||||
outOwners(i + 1) = i
|
||||
i += 1
|
||||
}
|
||||
val assembly = new GraphAssembly(stages, attributes, ins, inOwners, outs, outOwners)
|
||||
|
||||
val (inHandlers, outHandlers, logics, _) = assembly.materialize(Attributes.none)
|
||||
val interpreter = new GraphInterpreter(
|
||||
assembly,
|
||||
NoMaterializer,
|
||||
NoLogging,
|
||||
inHandlers,
|
||||
outHandlers,
|
||||
logics,
|
||||
(_, _, _) ⇒ throw new UnsupportedOperationException("IteratorInterpreter does not support asynchronous events."))
|
||||
interpreter.attachUpstreamBoundary(0, upstream)
|
||||
interpreter.attachDownstreamBoundary(ops.length, downstream)
|
||||
interpreter.init()
|
||||
}
|
||||
|
||||
init()
|
||||
|
||||
def iterator: Iterator[O] = downstream
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,747 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.stream.impl.fusing
|
||||
|
||||
import akka.event.LoggingAdapter
|
||||
import akka.stream.impl.ReactiveStreamsCompliance
|
||||
import akka.stream.stage._
|
||||
import akka.stream.{ Materializer, Attributes, Supervision }
|
||||
|
||||
import scala.annotation.{ switch, tailrec }
|
||||
import scala.collection.{ breakOut, immutable }
|
||||
import scala.util.control.NonFatal
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] object OneBoundedInterpreter {
|
||||
final val Debug = false
|
||||
|
||||
/** INTERNAL API */
|
||||
private[akka] sealed trait InitializationStatus
|
||||
/** INTERNAL API */
|
||||
private[akka] case object InitializationSuccessful extends InitializationStatus
|
||||
/** INTERNAL API */
|
||||
private[akka] final case class InitializationFailed(failures: immutable.Seq[InitializationFailure]) extends InitializationStatus {
|
||||
// exceptions are reverse ordered here, below methods help to avoid confusion when used from the outside
|
||||
def mostUpstream = failures.last
|
||||
def mostDownstream = failures.head
|
||||
}
|
||||
|
||||
/** INTERNAL API */
|
||||
private[akka] case class InitializationFailure(op: Int, ex: Throwable)
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*
|
||||
* This artificial op is used as a boundary to prevent two forked paths of execution (complete, cancel) to cross
|
||||
* paths again. When finishing an op this op is injected in its place to isolate upstream and downstream execution
|
||||
* domains.
|
||||
*/
|
||||
private[akka] object Finished extends BoundaryStage {
|
||||
override def onPush(elem: Any, ctx: BoundaryContext): UpstreamDirective = ctx.finish()
|
||||
override def onPull(ctx: BoundaryContext): DownstreamDirective = ctx.finish()
|
||||
override def onUpstreamFinish(ctx: BoundaryContext): TerminationDirective = ctx.exit()
|
||||
override def onDownstreamFinish(ctx: BoundaryContext): TerminationDirective = ctx.exit()
|
||||
override def onUpstreamFailure(cause: Throwable, ctx: BoundaryContext): TerminationDirective = ctx.exit()
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*
|
||||
* This artificial op is used as a boundary to prevent the first forked onPush of execution of a pushFinish to enter
|
||||
* the originating stage again. This stage allows the forked upstream onUpstreamFinish to pass through if there was
|
||||
* no onPull called on the stage. Calling onPull on this op makes it a Finished op, which absorbs the
|
||||
* onUpstreamTermination, but otherwise onUpstreamTermination results in calling finish()
|
||||
*/
|
||||
private[akka] object PushFinished extends BoundaryStage {
|
||||
override def onPush(elem: Any, ctx: BoundaryContext): UpstreamDirective = ctx.finish()
|
||||
override def onPull(ctx: BoundaryContext): DownstreamDirective = ctx.finish()
|
||||
// This allows propagation of an onUpstreamFinish call. Note that if onPull has been called on this stage
|
||||
// before, then the call ctx.finish() in onPull already turned this op to a normal Finished, i.e. it will no longer
|
||||
// propagate onUpstreamFinish.
|
||||
override def onUpstreamFinish(ctx: BoundaryContext): TerminationDirective = ctx.finish()
|
||||
override def onDownstreamFinish(ctx: BoundaryContext): TerminationDirective = ctx.exit()
|
||||
override def onUpstreamFailure(cause: Throwable, ctx: BoundaryContext): TerminationDirective = ctx.exit()
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*
|
||||
* One-bounded interpreter for a linear chain of stream operations (graph support is possible and will be implemented
|
||||
* later)
|
||||
*
|
||||
* The ideas in this interpreter are an amalgamation of earlier ideas, notably:
|
||||
* - The original effect-tracking implementation by Johannes Rudolph -- the difference here that effects are not chained
|
||||
* together as classes but the callstack is used instead and only certain combinations are allowed.
|
||||
* - The on-stack reentrant implementation by Mathias Doenitz -- the difference here that reentrancy is handled by the
|
||||
* interpreter itself, not user code, and the interpreter is able to use the heap when needed instead of the
|
||||
* callstack.
|
||||
* - The pinball interpreter by Endre Sándor Varga -- the difference here that the restriction for "one ball" is
|
||||
* lifted by using isolated execution regions, completion handling is introduced and communication with the external
|
||||
* world is done via boundary ops.
|
||||
*
|
||||
* The design goals/features of this interpreter are:
|
||||
* - bounded callstack and heapless execution whenever possible
|
||||
* - callstack usage should be constant for the most common ops independently of the size of the op-chain
|
||||
* - allocation-free execution on the hot paths
|
||||
* - enforced backpressure-safety (boundedness) on user defined ops at compile-time (and runtime in a few cases)
|
||||
*
|
||||
* The main driving idea of this interpreter is the concept of 1-bounded execution of well-formed free choice Petri
|
||||
* nets (J. Desel and J. Esparza: Free Choice Petri Nets - https://www7.in.tum.de/~esparza/bookfc.html). Technically
|
||||
* different kinds of operations partition the chain of ops into regions where *exactly one* event is active all the
|
||||
* time. This "exactly one" property is enforced by proper types and runtime checks where needed. Currently there are
|
||||
* three kinds of ops:
|
||||
*
|
||||
* - PushPullStage implementations participate in 1-bounded regions. For every external non-completion signal these
|
||||
* ops produce *exactly one* signal (completion is different, explained later) therefore keeping the number of events
|
||||
* the same: exactly one.
|
||||
*
|
||||
* - DetachedStage implementations are boundaries between 1-bounded regions. This means that they need to enforce the
|
||||
* "exactly one" property both on their upstream and downstream regions. As a consequence a DetachedStage can never
|
||||
* answer an onPull with a ctx.pull() or answer an onPush() with a ctx.push() since such an action would "steal"
|
||||
* the event from one region (resulting in zero signals) and would inject it to the other region (resulting in two
|
||||
* signals). However DetachedStages have the ability to call ctx.hold() as a response to onPush/onPull which temporarily
|
||||
* takes the signal off and stops execution, at the same time putting the op in a "holding" state. If the op is in a
|
||||
* holding state it contains one absorbed signal, therefore in this state the only possible command to call is
|
||||
* ctx.pushAndPull() which results in two events making the balance right again:
|
||||
* 1 hold + 1 external event = 2 external event
|
||||
* This mechanism allows synchronization between the upstream and downstream regions which otherwise can progress
|
||||
* independently.
|
||||
*
|
||||
* - BoundaryStage implementations are meant to communicate with the external world. These ops do not have most of the
|
||||
* safety properties enforced and should be used carefully. One important ability of BoundaryStages that they can take
|
||||
* off an execution signal by calling ctx.exit(). This is typically used immediately after an external signal has
|
||||
* been produced (for example an actor message). BoundaryStages can also kickstart execution by calling enter() which
|
||||
* returns a context they can use to inject signals into the interpreter. There is no checks in place to enforce that
|
||||
* the number of signals taken out by exit() and the number of signals returned via enter() are the same -- using this
|
||||
* op type needs extra care from the implementer.
|
||||
* BoundaryStages are the elements that make the interpreter *tick*, there is no other way to start the interpreter
|
||||
* than using a BoundaryStage.
|
||||
*
|
||||
* Operations are allowed to do early completion and cancel/complete their upstreams and downstreams. It is *not*
|
||||
* allowed however to do these independently to avoid isolated execution islands. The only call possible is ctx.finish()
|
||||
* which is a combination of cancel/complete.
|
||||
* Since onComplete is not a backpressured signal it is sometimes preferable to push a final element and then immediately
|
||||
* finish. This combination is exposed as pushAndFinish() which enables op writers to propagate completion events without
|
||||
* waiting for an extra round of pull.
|
||||
* Another peculiarity is how to convert termination events (complete/failure) into elements. The problem
|
||||
* here is that the termination events are not backpressured while elements are. This means that simply calling ctx.push()
|
||||
* as a response to onUpstreamFinished() will very likely break boundedness and result in a buffer overflow somewhere.
|
||||
* Therefore the only allowed command in this case is ctx.absorbTermination() which stops the propagation of the
|
||||
* termination signal, and puts the op in a finishing state. Depending on whether the op has a pending pull signal it has
|
||||
* not yet "consumed" by a push its onPull() handler might be called immediately.
|
||||
*
|
||||
* In order to execute different individual execution regions the interpreter uses the callstack to schedule these. The
|
||||
* current execution forking operations are
|
||||
* - ctx.finish() which starts a wave of completion and cancellation in two directions. When an op calls finish()
|
||||
* it is immediately replaced by an artificial Finished op which makes sure that the two execution paths are isolated
|
||||
* forever.
|
||||
* - ctx.fail() which is similar to finish()
|
||||
* - ctx.pushAndPull() which (as a response to a previous ctx.hold()) starts a wave of downstream push and upstream
|
||||
* pull. The two execution paths are isolated by the op itself since onPull() from downstream can only be answered by hold or
|
||||
* push, while onPush() from upstream can only answered by hold or pull -- it is impossible to "cross" the op.
|
||||
* - ctx.pushAndFinish() which is different from the forking ops above because the execution of push and finish happens on
|
||||
* the same execution region and they are order dependent, too.
|
||||
* The interpreter tracks the depth of recursive forking and allows various strategies of dealing with the situation
|
||||
* when this depth reaches a certain limit. In the simplest case a failure is reported (this is very useful for stress
|
||||
* testing and finding callstack wasting bugs), in the other case the forked call is scheduled via a list -- i.e. instead
|
||||
* of the stack the heap is used.
|
||||
*/
|
||||
private[akka] class OneBoundedInterpreter(ops: Seq[Stage[_, _]],
|
||||
onAsyncInput: (AsyncStage[Any, Any, Any], AsyncContext[Any, Any], Any) ⇒ Unit,
|
||||
log: LoggingAdapter,
|
||||
materializer: Materializer,
|
||||
attributes: Attributes = Attributes.none,
|
||||
val forkLimit: Int = 100,
|
||||
val overflowToHeap: Boolean = true,
|
||||
val name: String = "") {
|
||||
import AbstractStage._
|
||||
import OneBoundedInterpreter._
|
||||
|
||||
type UntypedOp = AbstractStage[Any, Any, Directive, Directive, Context[Any], LifecycleContext]
|
||||
require(ops.nonEmpty, "OneBoundedInterpreter cannot be created without at least one Op")
|
||||
|
||||
private final val pipeline: Array[UntypedOp] = ops.map(_.asInstanceOf[UntypedOp])(breakOut)
|
||||
|
||||
/**
|
||||
* This table is used to accelerate demand propagation upstream. All ops that implement PushStage are guaranteed
|
||||
* to only do upstream propagation of demand signals, therefore it is not necessary to execute them but enough to
|
||||
* "jump over" them. This means that when a chain of one million maps gets a downstream demand it is propagated
|
||||
* to the upstream *in one step* instead of one million onPull() calls.
|
||||
* This table maintains the positions where execution should jump from a current position when a pull event is to
|
||||
* be executed.
|
||||
*/
|
||||
private final val jumpBacks: Array[Int] = calculateJumpBacks
|
||||
|
||||
private final val Upstream = 0
|
||||
private final val Downstream = pipeline.length - 1
|
||||
|
||||
// Var to hold the current element if pushing. The only reason why this var is needed is to avoid allocations and
|
||||
// make it possible for the Pushing state to be an object
|
||||
private var elementInFlight: Any = _
|
||||
// Points to the current point of execution inside the pipeline
|
||||
private var activeOpIndex = -1
|
||||
// Points to the last point of exit
|
||||
private var lastExitedIndex = Downstream
|
||||
// The current interpreter state that decides what happens at the next round
|
||||
private var state: State = _
|
||||
|
||||
// Counter that keeps track of the depth of recursive forked executions
|
||||
private var forkCount = 0
|
||||
// List that is used as an auxiliary stack if fork recursion depth reaches forkLimit
|
||||
private var overflowStack = List.empty[(Int, State, Any)]
|
||||
|
||||
private var lastOpFailing: Int = -1
|
||||
|
||||
private def pipeName(op: UntypedOp): String = {
|
||||
val o = op: AbstractStage[_, _, _, _, _, _]
|
||||
(o match {
|
||||
case Finished ⇒ "finished"
|
||||
case _: BoundaryStage ⇒ "boundary"
|
||||
case _: StatefulStage[_, _] ⇒ "stateful"
|
||||
case _: PushStage[_, _] ⇒ "push"
|
||||
case _: PushPullStage[_, _] ⇒ "pushpull"
|
||||
case _: DetachedStage[_, _] ⇒ "detached"
|
||||
case _ ⇒ "other"
|
||||
}) + f"(${o.bits}%04X)"
|
||||
}
|
||||
override def toString =
|
||||
s"""|OneBoundedInterpreter($name)
|
||||
| pipeline = ${pipeline map pipeName mkString ":"}
|
||||
| lastExit=$lastExitedIndex activeOp=$activeOpIndex state=$state elem=$elementInFlight forks=$forkCount""".stripMargin
|
||||
|
||||
@inline private def currentOp: UntypedOp = pipeline(activeOpIndex)
|
||||
|
||||
// see the jumpBacks variable for explanation
|
||||
private def calculateJumpBacks: Array[Int] = {
|
||||
val table = Array.ofDim[Int](pipeline.length)
|
||||
var nextJumpBack = -1
|
||||
for (pos ← pipeline.indices) {
|
||||
table(pos) = nextJumpBack
|
||||
if (!pipeline(pos).isInstanceOf[PushStage[_, _]]) nextJumpBack = pos
|
||||
}
|
||||
table
|
||||
}
|
||||
|
||||
private def updateJumpBacks(lastNonCompletedIndex: Int): Unit = {
|
||||
var pos = lastNonCompletedIndex
|
||||
// For every jump that would jump over us we change them to jump into us
|
||||
while (pos < pipeline.length && jumpBacks(pos) < lastNonCompletedIndex) {
|
||||
jumpBacks(pos) = lastNonCompletedIndex
|
||||
pos += 1
|
||||
}
|
||||
}
|
||||
|
||||
private sealed trait State extends DetachedContext[Any] with BoundaryContext with AsyncContext[Any, Any] {
|
||||
def enter(): Unit = throw new IllegalStateException("cannot enter an ordinary Context")
|
||||
|
||||
final def execute(): Unit = OneBoundedInterpreter.this.execute()
|
||||
|
||||
final def progress(): Unit = {
|
||||
advance()
|
||||
if (inside) run()
|
||||
else exit()
|
||||
}
|
||||
|
||||
/**
|
||||
* Override this method to do execution steps necessary after executing an op, and advance the activeOpIndex
|
||||
* to another value (next or previous steps). Do NOT put code that invokes the next op, override run instead.
|
||||
*/
|
||||
def advance(): Unit
|
||||
|
||||
/**
|
||||
* Override this method to enter the current op and execute it. Do NOT put code that should be executed after the
|
||||
* op has been invoked, that should be in the advance() method of the next state resulting from the invocation of
|
||||
* the op.
|
||||
*/
|
||||
def run(): Unit
|
||||
|
||||
/**
|
||||
* This method shall return the bit set representing the incoming ball (if any).
|
||||
*/
|
||||
def incomingBall: Int
|
||||
|
||||
protected def hasBits(b: Int): Boolean = ((currentOp.bits | incomingBall) & b) == b
|
||||
protected def addBits(b: Int): Unit = currentOp.bits |= b
|
||||
protected def removeBits(b: Int): Unit = currentOp.bits &= ~b
|
||||
|
||||
protected def mustHave(b: Int): Unit =
|
||||
if (!hasBits(b)) {
|
||||
def format(b: Int) = {
|
||||
val ballStatus = (b & BothBalls: @switch) match {
|
||||
case 0 ⇒ "no balls"
|
||||
case UpstreamBall ⇒ "upstream ball"
|
||||
case DownstreamBall ⇒ "downstream ball"
|
||||
case BothBalls ⇒ "upstream & downstream balls"
|
||||
}
|
||||
if ((b & NoTerminationPending) > 0) ballStatus + " and not isFinishing"
|
||||
else ballStatus + " and isFinishing"
|
||||
}
|
||||
throw new IllegalStateException(s"operation requires [${format(b)}] while holding [${format(currentOp.bits)}] and receiving [${format(incomingBall)}]")
|
||||
}
|
||||
|
||||
override def push(elem: Any): DownstreamDirective = {
|
||||
ReactiveStreamsCompliance.requireNonNullElement(elem)
|
||||
if (currentOp.isDetached) {
|
||||
if (incomingBall == UpstreamBall)
|
||||
throw new IllegalStateException("Cannot push during onPush, only pull, pushAndPull or holdUpstreamAndPush")
|
||||
mustHave(DownstreamBall)
|
||||
}
|
||||
removeBits(PrecedingWasPull | DownstreamBall)
|
||||
elementInFlight = elem
|
||||
state = Pushing
|
||||
null
|
||||
}
|
||||
|
||||
override def pull(): UpstreamDirective = {
|
||||
var requirements = NoTerminationPending
|
||||
if (currentOp.isDetached) {
|
||||
if (incomingBall == DownstreamBall)
|
||||
throw new IllegalStateException("Cannot pull during onPull, only push, pushAndPull or holdDownstreamAndPull")
|
||||
requirements |= UpstreamBall
|
||||
}
|
||||
mustHave(requirements)
|
||||
removeBits(UpstreamBall)
|
||||
addBits(PrecedingWasPull)
|
||||
state = Pulling
|
||||
null
|
||||
}
|
||||
|
||||
override def getAsyncCallback: AsyncCallback[Any] = {
|
||||
val current = currentOp.asInstanceOf[AsyncStage[Any, Any, Any]]
|
||||
val context = current.context // avoid concurrent access (to avoid @volatile)
|
||||
new AsyncCallback[Any] {
|
||||
override def invoke(evt: Any): Unit = onAsyncInput(current, context, evt)
|
||||
}
|
||||
}
|
||||
|
||||
override def ignore(): AsyncDirective = {
|
||||
if (incomingBall != 0) throw new IllegalStateException("Can only ignore from onAsyncInput")
|
||||
exit()
|
||||
}
|
||||
|
||||
override def finish(): FreeDirective = {
|
||||
finishCurrentOp()
|
||||
fork(Completing)
|
||||
state = Cancelling
|
||||
null
|
||||
}
|
||||
|
||||
def isFinishing: Boolean = !hasBits(NoTerminationPending)
|
||||
|
||||
final protected def pushAndFinishCommon(elem: Any, finishState: UntypedOp): Unit = {
|
||||
finishCurrentOp(finishState)
|
||||
ReactiveStreamsCompliance.requireNonNullElement(elem)
|
||||
if (currentOp.isDetached) {
|
||||
mustHave(DownstreamBall)
|
||||
}
|
||||
removeBits(DownstreamBall | PrecedingWasPull)
|
||||
}
|
||||
|
||||
override def pushAndFinish(elem: Any): DownstreamDirective = {
|
||||
// Spit the execution domain in two and invoke postStop
|
||||
pushAndFinishCommon(elem, Finished.asInstanceOf[UntypedOp])
|
||||
|
||||
// This MUST be an unsafeFork because the execution of PushFinish MUST strictly come before the finish execution
|
||||
// path. Other forks are not order dependent because they execute on isolated execution domains which cannot
|
||||
// "cross paths". This unsafeFork is relatively safe here because PushAndFinish simply absorbs all later downstream
|
||||
// calls of pushAndFinish since the finish event has been scheduled already.
|
||||
// It might be that there are some degenerate cases where this can blow up the stack with a very long chain but I
|
||||
// am not aware of such scenario yet. If you know one, put it in InterpreterStressSpec :)
|
||||
unsafeFork(PushFinish, elem)
|
||||
|
||||
// Same as finish, without calling finishCurrentOp
|
||||
elementInFlight = null
|
||||
fork(Completing)
|
||||
state = Cancelling
|
||||
null
|
||||
}
|
||||
|
||||
override def fail(cause: Throwable): FreeDirective = {
|
||||
fork(Failing(cause))
|
||||
state = Cancelling
|
||||
null
|
||||
}
|
||||
|
||||
override def holdUpstream(): UpstreamDirective = {
|
||||
mustHave(NoTerminationPending)
|
||||
removeBits(PrecedingWasPull)
|
||||
addBits(UpstreamBall)
|
||||
exit()
|
||||
}
|
||||
|
||||
override def holdUpstreamAndPush(elem: Any): UpstreamDirective = {
|
||||
ReactiveStreamsCompliance.requireNonNullElement(elem)
|
||||
if (incomingBall != UpstreamBall)
|
||||
throw new IllegalStateException("can only holdUpstreamAndPush from onPush")
|
||||
mustHave(BothBallsAndNoTerminationPending)
|
||||
removeBits(PrecedingWasPull | DownstreamBall)
|
||||
addBits(UpstreamBall)
|
||||
elementInFlight = elem
|
||||
state = Pushing
|
||||
null
|
||||
}
|
||||
|
||||
override def isHoldingUpstream: Boolean = (currentOp.bits & UpstreamBall) != 0
|
||||
|
||||
override def holdDownstream(): DownstreamDirective = {
|
||||
addBits(DownstreamBall)
|
||||
exit()
|
||||
}
|
||||
|
||||
override def holdDownstreamAndPull(): DownstreamDirective = {
|
||||
if (incomingBall != DownstreamBall)
|
||||
throw new IllegalStateException("can only holdDownstreamAndPull from onPull")
|
||||
mustHave(BothBallsAndNoTerminationPending)
|
||||
addBits(PrecedingWasPull | DownstreamBall)
|
||||
removeBits(UpstreamBall)
|
||||
state = Pulling
|
||||
null
|
||||
}
|
||||
|
||||
override def isHoldingDownstream: Boolean = (currentOp.bits & DownstreamBall) != 0
|
||||
|
||||
override def pushAndPull(elem: Any): FreeDirective = {
|
||||
ReactiveStreamsCompliance.requireNonNullElement(elem)
|
||||
mustHave(BothBallsAndNoTerminationPending)
|
||||
addBits(PrecedingWasPull)
|
||||
removeBits(BothBalls)
|
||||
fork(Pushing, elem)
|
||||
state = Pulling
|
||||
null
|
||||
}
|
||||
|
||||
override def absorbTermination(): TerminationDirective = {
|
||||
updateJumpBacks(activeOpIndex)
|
||||
removeBits(BothBallsAndNoTerminationPending)
|
||||
finish()
|
||||
}
|
||||
|
||||
override def exit(): FreeDirective = {
|
||||
elementInFlight = null
|
||||
lastExitedIndex = activeOpIndex
|
||||
activeOpIndex = -1
|
||||
null
|
||||
}
|
||||
|
||||
override def materializer: Materializer = OneBoundedInterpreter.this.materializer
|
||||
override def attributes: Attributes = OneBoundedInterpreter.this.attributes
|
||||
}
|
||||
|
||||
private final val Pushing: State = new State {
|
||||
override def advance(): Unit = activeOpIndex += 1
|
||||
override def run(): Unit = currentOp.onPush(elementInFlight, ctx = this)
|
||||
override def incomingBall = UpstreamBall
|
||||
override def toString = "Pushing"
|
||||
}
|
||||
|
||||
private final val PushFinish: State = new State {
|
||||
override def advance(): Unit = activeOpIndex += 1
|
||||
override def run(): Unit = currentOp.onPush(elementInFlight, ctx = this)
|
||||
|
||||
override def pushAndFinish(elem: Any): DownstreamDirective = {
|
||||
// PushFinished
|
||||
// Put an isolation barrier that will prevent the onPull of this op to be called again. This barrier
|
||||
// is different from simple Finished that it allows onUpstreamTerminated to pass through, unless onPull
|
||||
// has been called on the stage
|
||||
pushAndFinishCommon(elem, PushFinished.asInstanceOf[UntypedOp])
|
||||
|
||||
elementInFlight = elem
|
||||
state = PushFinish
|
||||
null
|
||||
}
|
||||
|
||||
override def finish(): FreeDirective = {
|
||||
state = Completing
|
||||
null
|
||||
}
|
||||
|
||||
override def incomingBall = UpstreamBall
|
||||
|
||||
override def toString = "PushFinish"
|
||||
}
|
||||
|
||||
private final val Pulling: State = new State {
|
||||
override def advance(): Unit = {
|
||||
elementInFlight = null
|
||||
activeOpIndex = jumpBacks(activeOpIndex)
|
||||
}
|
||||
|
||||
override def run(): Unit = currentOp.onPull(ctx = this)
|
||||
|
||||
override def incomingBall = DownstreamBall
|
||||
|
||||
override def toString = "Pulling"
|
||||
}
|
||||
|
||||
private final val Completing: State = new State {
|
||||
override def advance(): Unit = {
|
||||
elementInFlight = null
|
||||
finishCurrentOp()
|
||||
activeOpIndex += 1
|
||||
}
|
||||
|
||||
override def run(): Unit = {
|
||||
if (!hasBits(NoTerminationPending)) exit()
|
||||
else currentOp.onUpstreamFinish(ctx = this)
|
||||
}
|
||||
|
||||
override def finish(): FreeDirective = {
|
||||
state = Completing
|
||||
null
|
||||
}
|
||||
|
||||
override def absorbTermination(): TerminationDirective = {
|
||||
removeBits(NoTerminationPending)
|
||||
removeBits(UpstreamBall)
|
||||
updateJumpBacks(activeOpIndex)
|
||||
if (hasBits(DownstreamBall) || (!currentOp.isDetached && hasBits(PrecedingWasPull))) {
|
||||
removeBits(DownstreamBall)
|
||||
currentOp.onPull(ctx = Pulling)
|
||||
} else exit()
|
||||
null
|
||||
}
|
||||
|
||||
override def incomingBall = UpstreamBall
|
||||
|
||||
override def toString = "Completing"
|
||||
}
|
||||
|
||||
private final val Cancelling: State = new State {
|
||||
override def advance(): Unit = {
|
||||
elementInFlight = null
|
||||
finishCurrentOp()
|
||||
activeOpIndex -= 1
|
||||
}
|
||||
|
||||
def run(): Unit = {
|
||||
if (!hasBits(NoTerminationPending)) exit()
|
||||
else currentOp.onDownstreamFinish(ctx = this)
|
||||
}
|
||||
|
||||
override def finish(): FreeDirective = {
|
||||
state = Cancelling
|
||||
null
|
||||
}
|
||||
|
||||
override def incomingBall = DownstreamBall
|
||||
|
||||
override def toString = "Cancelling"
|
||||
|
||||
override def absorbTermination(): TerminationDirective = {
|
||||
val ex = new UnsupportedOperationException("It is not allowed to call absorbTermination() from onDownstreamFinish.")
|
||||
// This MUST be logged here, since the downstream has cancelled, i.e. there is noone to send onError to, the
|
||||
// stage is just about to finish so noone will catch it anyway just the interpreter
|
||||
log.error(ex.getMessage)
|
||||
throw ex // We still throw for correctness (although a finish() would also work here)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private final case class Failing(cause: Throwable) extends State {
|
||||
override def advance(): Unit = {
|
||||
elementInFlight = null
|
||||
finishCurrentOp()
|
||||
activeOpIndex += 1
|
||||
}
|
||||
|
||||
def run(): Unit = currentOp.onUpstreamFailure(cause, ctx = this)
|
||||
|
||||
override def absorbTermination(): TerminationDirective = {
|
||||
removeBits(NoTerminationPending)
|
||||
removeBits(UpstreamBall)
|
||||
updateJumpBacks(activeOpIndex)
|
||||
if (hasBits(DownstreamBall) || (!currentOp.isDetached && hasBits(PrecedingWasPull))) {
|
||||
removeBits(DownstreamBall)
|
||||
currentOp.onPull(ctx = Pulling)
|
||||
} else exit()
|
||||
null
|
||||
}
|
||||
|
||||
override def incomingBall = UpstreamBall
|
||||
}
|
||||
|
||||
private def inside: Boolean = activeOpIndex > -1 && activeOpIndex < pipeline.length
|
||||
|
||||
private def printDebug(): Unit = {
|
||||
val padding = " " * activeOpIndex
|
||||
val icon: String = state match {
|
||||
case Pushing | PushFinish ⇒ padding + s"---> $elementInFlight"
|
||||
case Pulling ⇒
|
||||
(" " * jumpBacks(activeOpIndex)) +
|
||||
"<---" +
|
||||
("----" * (activeOpIndex - jumpBacks(activeOpIndex) - 1))
|
||||
case Completing ⇒ padding + "---|"
|
||||
case Cancelling ⇒ padding + "|---"
|
||||
case Failing(e) ⇒ padding + s"---X ${e.getMessage} => ${decide(e)}"
|
||||
case other ⇒ padding + s"---? $state"
|
||||
}
|
||||
println(f"$icon%-24s $name")
|
||||
}
|
||||
|
||||
@tailrec private def execute(): Unit = {
|
||||
while (inside) {
|
||||
try {
|
||||
if (Debug) printDebug()
|
||||
state.progress()
|
||||
} catch {
|
||||
case NonFatal(e) if lastOpFailing != activeOpIndex ⇒
|
||||
lastOpFailing = activeOpIndex
|
||||
decide(e) match {
|
||||
case Supervision.Stop ⇒ state.fail(e)
|
||||
case Supervision.Resume ⇒
|
||||
// reset, purpose of lastOpFailing is to avoid infinite loops when fail fails -- double fault
|
||||
lastOpFailing = -1
|
||||
afterRecovery()
|
||||
case Supervision.Restart ⇒
|
||||
// reset, purpose of lastOpFailing is to avoid infinite loops when fail fails -- double fault
|
||||
lastOpFailing = -1
|
||||
pipeline(activeOpIndex) = pipeline(activeOpIndex).restart().asInstanceOf[UntypedOp]
|
||||
afterRecovery()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// FIXME push this into AbstractStage so it can be customized
|
||||
def afterRecovery(): Unit = state match {
|
||||
case _: EntryState ⇒ // no ball to be juggled with
|
||||
case _ ⇒ state.pull()
|
||||
}
|
||||
|
||||
// Execute all delayed forks that were put on the heap if the fork limit has been reached
|
||||
if (overflowStack.nonEmpty) {
|
||||
val memo = overflowStack.head
|
||||
activeOpIndex = memo._1
|
||||
state = memo._2
|
||||
elementInFlight = memo._3
|
||||
overflowStack = overflowStack.tail
|
||||
execute()
|
||||
}
|
||||
}
|
||||
|
||||
def decide(e: Throwable): Supervision.Directive =
|
||||
if (state == Pulling || state == Cancelling) Supervision.Stop
|
||||
else currentOp.decide(e)
|
||||
|
||||
/**
|
||||
* Forks off execution of the pipeline by saving current position, fully executing the effects of the given
|
||||
* forkState then setting back the position to the saved value.
|
||||
* By default forking is executed by using the callstack. If the depth of forking ever reaches the configured forkLimit
|
||||
* this method either fails (useful for testing) or starts using the heap instead of the callstack to avoid a
|
||||
* stack overflow.
|
||||
*/
|
||||
private def fork(forkState: State, elem: Any = null): Unit = {
|
||||
forkCount += 1
|
||||
if (forkCount == forkLimit) {
|
||||
if (!overflowToHeap) throw new IllegalStateException("Fork limit reached")
|
||||
else overflowStack ::= ((activeOpIndex, forkState, elem))
|
||||
} else unsafeFork(forkState, elem)
|
||||
forkCount -= 1
|
||||
}
|
||||
|
||||
/**
|
||||
* Unsafe fork always uses the stack for execution. This call is needed by pushAndComplete where the forked execution
|
||||
* is order dependent since the push and complete events travel in the same direction and not isolated by a boundary
|
||||
*/
|
||||
private def unsafeFork(forkState: State, elem: Any = null): Unit = {
|
||||
val savePos = activeOpIndex
|
||||
elementInFlight = elem
|
||||
state = forkState
|
||||
execute()
|
||||
activeOpIndex = savePos
|
||||
}
|
||||
|
||||
/**
|
||||
* Initializes all stages setting their initial context and calling [[AbstractStage.preStart]] on each.
|
||||
*/
|
||||
def init(): InitializationStatus = {
|
||||
val failures = initBoundaries()
|
||||
runDetached()
|
||||
|
||||
if (failures.isEmpty) InitializationSuccessful
|
||||
else {
|
||||
val failure = failures.head
|
||||
activeOpIndex = failure.op
|
||||
currentOp.enterAndFail(failure.ex)
|
||||
InitializationFailed(failures)
|
||||
}
|
||||
}
|
||||
|
||||
def isFinished: Boolean = pipeline(Upstream) == Finished && pipeline(Downstream) == Finished
|
||||
|
||||
private class EntryState(name: String, position: Int) extends State {
|
||||
val entryPoint = position
|
||||
|
||||
final override def enter(): Unit = {
|
||||
activeOpIndex = entryPoint
|
||||
if (Debug) {
|
||||
val s = " " * entryPoint + "ENTR"
|
||||
println(f"$s%-24s ${OneBoundedInterpreter.this.name}")
|
||||
}
|
||||
}
|
||||
|
||||
override def run(): Unit = ()
|
||||
override def advance(): Unit = ()
|
||||
|
||||
override def incomingBall = 0
|
||||
|
||||
override def toString = s"$name($entryPoint)"
|
||||
}
|
||||
|
||||
/**
|
||||
* This method injects a Context to each of the BoundaryStages and AsyncStages. This will be the context returned by enter().
|
||||
*/
|
||||
private def initBoundaries(): List[InitializationFailure] = {
|
||||
var failures: List[InitializationFailure] = Nil
|
||||
var op = 0
|
||||
while (op < pipeline.length) {
|
||||
(pipeline(op): Any) match {
|
||||
case b: BoundaryStage ⇒
|
||||
b.context = new EntryState("boundary", op)
|
||||
|
||||
case a: AsyncStage[Any, Any, Any] @unchecked ⇒
|
||||
a.context = new EntryState("async", op)
|
||||
activeOpIndex = op
|
||||
a.preStart(a.context)
|
||||
|
||||
case a: AbstractStage[Any, Any, Any, Any, Any, Any] @unchecked ⇒
|
||||
val state = new EntryState("stage", op)
|
||||
a.context = state
|
||||
try a.preStart(state) catch {
|
||||
case NonFatal(ex) ⇒
|
||||
failures ::= InitializationFailure(op, ex) // not logging here as 'most downstream' exception will be signaled via onError
|
||||
}
|
||||
}
|
||||
op += 1
|
||||
}
|
||||
failures
|
||||
}
|
||||
|
||||
private def finishCurrentOp(finishState: UntypedOp = Finished.asInstanceOf[UntypedOp]): Unit = {
|
||||
try pipeline(activeOpIndex).postStop()
|
||||
catch { case NonFatal(ex) ⇒ log.error(s"Stage [{}] postStop failed", ex) }
|
||||
finally pipeline(activeOpIndex) = finishState
|
||||
}
|
||||
|
||||
/**
|
||||
* Starts execution of detached regions.
|
||||
*
|
||||
* Since detached ops partition the pipeline into different 1-bounded domains is is necessary to inject a starting
|
||||
* signal into these regions (since there is no external signal that would kick off their execution otherwise).
|
||||
*/
|
||||
private def runDetached(): Unit = {
|
||||
var op = pipeline.length - 1
|
||||
while (op >= 0) {
|
||||
if (pipeline(op).isDetached) {
|
||||
activeOpIndex = op
|
||||
state = Pulling
|
||||
execute()
|
||||
}
|
||||
op -= 1
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -10,7 +10,6 @@ import akka.stream.impl.fusing.GraphStages.SimpleLinearGraphStage
|
|||
import akka.stream.impl.{ FixedSizeBuffer, ReactiveStreamsCompliance }
|
||||
import akka.stream.stage._
|
||||
import akka.stream.{ Supervision, _ }
|
||||
|
||||
import scala.annotation.tailrec
|
||||
import scala.collection.immutable
|
||||
import scala.collection.immutable.VectorBuilder
|
||||
|
|
@ -18,6 +17,7 @@ import scala.concurrent.Future
|
|||
import scala.concurrent.duration.FiniteDuration
|
||||
import scala.util.control.NonFatal
|
||||
import scala.util.{ Failure, Success, Try }
|
||||
import akka.stream.ActorAttributes.SupervisionStrategy
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
@ -537,149 +537,150 @@ private[akka] object MapAsync {
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] final case class MapAsync[In, Out](parallelism: Int, f: In ⇒ Future[Out], decider: Supervision.Decider)
|
||||
extends AsyncStage[In, Out, (Int, Try[Out])] {
|
||||
private[akka] final case class MapAsync[In, Out](parallelism: Int, f: In ⇒ Future[Out])
|
||||
extends GraphStage[FlowShape[In, Out]] {
|
||||
|
||||
import MapAsync._
|
||||
|
||||
type Notification = (Int, Try[Out])
|
||||
private val in = Inlet[In]("in")
|
||||
private val out = Outlet[Out]("out")
|
||||
|
||||
private var callback: AsyncCallback[Notification] = _
|
||||
private val elemsInFlight = FixedSizeBuffer[Try[Out]](parallelism)
|
||||
override val shape = FlowShape(in, out)
|
||||
|
||||
override def preStart(ctx: AsyncContext[Out, Notification]): Unit = {
|
||||
callback = ctx.getAsyncCallback
|
||||
}
|
||||
override def createLogic(inheritedAttributes: Attributes) = new GraphStageLogic(shape) {
|
||||
override def toString = s"MapAsync.Logic(buffer=$buffer)"
|
||||
|
||||
override def decide(ex: Throwable) = decider(ex)
|
||||
val decider =
|
||||
inheritedAttributes.getAttribute(classOf[SupervisionStrategy])
|
||||
.map(_.decider).getOrElse(Supervision.stoppingDecider)
|
||||
|
||||
override def onPush(elem: In, ctx: AsyncContext[Out, Notification]) = {
|
||||
val future = f(elem)
|
||||
val idx = elemsInFlight.enqueue(NotYetThere)
|
||||
future.onComplete(t ⇒ callback.invoke((idx, t)))(akka.dispatch.ExecutionContexts.sameThreadExecutionContext)
|
||||
if (elemsInFlight.isFull) ctx.holdUpstream()
|
||||
else ctx.pull()
|
||||
}
|
||||
val buffer = FixedSizeBuffer[Try[Out]](parallelism)
|
||||
def todo = buffer.used
|
||||
|
||||
override def onPull(ctx: AsyncContext[Out, (Int, Try[Out])]) = {
|
||||
@tailrec def rec(): DownstreamDirective =
|
||||
if (elemsInFlight.isEmpty && ctx.isFinishing) ctx.finish()
|
||||
else if (elemsInFlight.isEmpty || elemsInFlight.peek == NotYetThere) {
|
||||
if (!elemsInFlight.isFull && ctx.isHoldingUpstream) ctx.holdDownstreamAndPull()
|
||||
else ctx.holdDownstream()
|
||||
} else elemsInFlight.dequeue() match {
|
||||
case Failure(ex) ⇒ rec()
|
||||
@tailrec private def pushOne(): Unit =
|
||||
if (buffer.isEmpty) {
|
||||
if (isClosed(in)) completeStage()
|
||||
else if (!hasBeenPulled(in)) pull(in)
|
||||
} else if (buffer.peek == NotYetThere) {
|
||||
if (todo < parallelism && !hasBeenPulled(in)) tryPull(in)
|
||||
} else buffer.dequeue() match {
|
||||
case Failure(ex) ⇒ pushOne()
|
||||
case Success(elem) ⇒
|
||||
if (ctx.isHoldingUpstream) ctx.pushAndPull(elem)
|
||||
else ctx.push(elem)
|
||||
}
|
||||
rec()
|
||||
}
|
||||
|
||||
override def onAsyncInput(input: (Int, Try[Out]), ctx: AsyncContext[Out, Notification]) = {
|
||||
@tailrec def rec(): Directive =
|
||||
if (elemsInFlight.isEmpty && ctx.isFinishing) ctx.finish()
|
||||
else if (elemsInFlight.isEmpty || (elemsInFlight.peek eq NotYetThere)) {
|
||||
if (!elemsInFlight.isFull && ctx.isHoldingUpstream) ctx.pull()
|
||||
else ctx.ignore()
|
||||
} else elemsInFlight.dequeue() match {
|
||||
case Failure(ex) ⇒ rec()
|
||||
case Success(elem) ⇒
|
||||
if (ctx.isHoldingUpstream) ctx.pushAndPull(elem)
|
||||
else ctx.push(elem)
|
||||
push(out, elem)
|
||||
if (todo < parallelism && !hasBeenPulled(in)) tryPull(in)
|
||||
}
|
||||
|
||||
input match {
|
||||
case (idx, f @ Failure(ex)) ⇒
|
||||
if (decider(ex) != Supervision.Stop) {
|
||||
elemsInFlight.put(idx, f)
|
||||
if (ctx.isHoldingDownstream) rec()
|
||||
else ctx.ignore()
|
||||
} else ctx.fail(ex)
|
||||
case (idx, s: Success[_]) ⇒
|
||||
val exception = try {
|
||||
ReactiveStreamsCompliance.requireNonNullElement(s.value)
|
||||
elemsInFlight.put(idx, s)
|
||||
null: Exception
|
||||
def failOrPull(idx: Int, f: Failure[Out]) =
|
||||
if (decider(f.exception) == Supervision.Stop) failStage(f.exception)
|
||||
else {
|
||||
buffer.put(idx, f)
|
||||
if (isAvailable(out)) pushOne()
|
||||
}
|
||||
|
||||
val futureCB =
|
||||
getAsyncCallback[(Int, Try[Out])]({
|
||||
case (idx, f: Failure[_]) ⇒ failOrPull(idx, f)
|
||||
case (idx, s @ Success(elem)) ⇒
|
||||
if (elem == null) {
|
||||
val ex = ReactiveStreamsCompliance.elementMustNotBeNullException
|
||||
failOrPull(idx, Failure(ex))
|
||||
} else {
|
||||
buffer.put(idx, s)
|
||||
if (isAvailable(out)) pushOne()
|
||||
}
|
||||
})
|
||||
|
||||
setHandler(in, new InHandler {
|
||||
override def onPush(): Unit = {
|
||||
try {
|
||||
val future = f(grab(in))
|
||||
val idx = buffer.enqueue(NotYetThere)
|
||||
future.onComplete(result ⇒ futureCB.invoke(idx -> result))(akka.dispatch.ExecutionContexts.sameThreadExecutionContext)
|
||||
} catch {
|
||||
case NonFatal(ex) ⇒
|
||||
if (decider(ex) != Supervision.Stop) {
|
||||
elemsInFlight.put(idx, Failure(ex))
|
||||
null: Exception
|
||||
} else ex
|
||||
if (decider(ex) == Supervision.Stop) failStage(ex)
|
||||
}
|
||||
if (exception != null) ctx.fail(exception)
|
||||
else if (ctx.isHoldingDownstream) rec()
|
||||
else ctx.ignore()
|
||||
}
|
||||
}
|
||||
if (todo < parallelism) tryPull(in)
|
||||
}
|
||||
override def onUpstreamFinish(): Unit = {
|
||||
if (todo == 0) completeStage()
|
||||
}
|
||||
})
|
||||
|
||||
override def onUpstreamFinish(ctx: AsyncContext[Out, Notification]) =
|
||||
if (ctx.isHoldingUpstream || !elemsInFlight.isEmpty) ctx.absorbTermination()
|
||||
else ctx.finish()
|
||||
setHandler(out, new OutHandler {
|
||||
override def onPull(): Unit = pushOne()
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] final case class MapAsyncUnordered[In, Out](parallelism: Int, f: In ⇒ Future[Out], decider: Supervision.Decider)
|
||||
extends AsyncStage[In, Out, Try[Out]] {
|
||||
private[akka] final case class MapAsyncUnordered[In, Out](parallelism: Int, f: In ⇒ Future[Out])
|
||||
extends GraphStage[FlowShape[In, Out]] {
|
||||
|
||||
private var callback: AsyncCallback[Try[Out]] = _
|
||||
private var inFlight = 0
|
||||
private val buffer = FixedSizeBuffer[Out](parallelism)
|
||||
private val in = Inlet[In]("in")
|
||||
private val out = Outlet[Out]("out")
|
||||
|
||||
private def todo = inFlight + buffer.used
|
||||
override val shape = FlowShape(in, out)
|
||||
|
||||
override def preStart(ctx: AsyncContext[Out, Try[Out]]): Unit =
|
||||
callback = ctx.getAsyncCallback
|
||||
override def createLogic(inheritedAttributes: Attributes) = new GraphStageLogic(shape) {
|
||||
override def toString = s"MapAsyncUnordered.Logic(inFlight=$inFlight, buffer=$buffer)"
|
||||
|
||||
override def decide(ex: Throwable) = decider(ex)
|
||||
val decider =
|
||||
inheritedAttributes.getAttribute(classOf[SupervisionStrategy])
|
||||
.map(_.decider).getOrElse(Supervision.stoppingDecider)
|
||||
|
||||
override def onPush(elem: In, ctx: AsyncContext[Out, Try[Out]]) = {
|
||||
val future = f(elem)
|
||||
inFlight += 1
|
||||
future.onComplete(callback.invoke)(akka.dispatch.ExecutionContexts.sameThreadExecutionContext)
|
||||
if (todo == parallelism) ctx.holdUpstream()
|
||||
else ctx.pull()
|
||||
}
|
||||
var inFlight = 0
|
||||
val buffer = FixedSizeBuffer[Out](parallelism)
|
||||
def todo = inFlight + buffer.used
|
||||
|
||||
override def onPull(ctx: AsyncContext[Out, Try[Out]]) =
|
||||
if (buffer.isEmpty) {
|
||||
if (ctx.isFinishing && inFlight == 0) ctx.finish() else ctx.holdDownstream()
|
||||
} else {
|
||||
val elem = buffer.dequeue()
|
||||
if (ctx.isHoldingUpstream) ctx.pushAndPull(elem)
|
||||
else ctx.push(elem)
|
||||
}
|
||||
def failOrPull(ex: Throwable) =
|
||||
if (decider(ex) == Supervision.Stop) failStage(ex)
|
||||
else if (isClosed(in) && todo == 0) completeStage()
|
||||
else if (!hasBeenPulled(in)) tryPull(in)
|
||||
|
||||
override def onAsyncInput(input: Try[Out], ctx: AsyncContext[Out, Try[Out]]) = {
|
||||
def ignoreOrFail(ex: Throwable) =
|
||||
if (decider(ex) == Supervision.Stop) ctx.fail(ex)
|
||||
else if (ctx.isHoldingUpstream) ctx.pull()
|
||||
else if (ctx.isFinishing && todo == 0) ctx.finish()
|
||||
else ctx.ignore()
|
||||
|
||||
inFlight -= 1
|
||||
input match {
|
||||
case Failure(ex) ⇒ ignoreOrFail(ex)
|
||||
case Success(elem) ⇒
|
||||
if (elem == null) {
|
||||
val ex = ReactiveStreamsCompliance.elementMustNotBeNullException
|
||||
ignoreOrFail(ex)
|
||||
} else if (ctx.isHoldingDownstream) {
|
||||
if (ctx.isHoldingUpstream) ctx.pushAndPull(elem)
|
||||
else ctx.push(elem)
|
||||
} else {
|
||||
buffer.enqueue(elem)
|
||||
ctx.ignore()
|
||||
val futureCB =
|
||||
getAsyncCallback((result: Try[Out]) ⇒ {
|
||||
inFlight -= 1
|
||||
result match {
|
||||
case Failure(ex) ⇒ failOrPull(ex)
|
||||
case Success(elem) ⇒
|
||||
if (elem == null) {
|
||||
val ex = ReactiveStreamsCompliance.elementMustNotBeNullException
|
||||
failOrPull(ex)
|
||||
} else if (isAvailable(out)) {
|
||||
if (!hasBeenPulled(in)) tryPull(in)
|
||||
push(out, elem)
|
||||
} else buffer.enqueue(elem)
|
||||
}
|
||||
}
|
||||
}
|
||||
}).invoke _
|
||||
|
||||
override def onUpstreamFinish(ctx: AsyncContext[Out, Try[Out]]) =
|
||||
if (todo > 0) ctx.absorbTermination()
|
||||
else ctx.finish()
|
||||
setHandler(in, new InHandler {
|
||||
override def onPush(): Unit = {
|
||||
try {
|
||||
val future = f(grab(in))
|
||||
inFlight += 1
|
||||
future.onComplete(futureCB)(akka.dispatch.ExecutionContexts.sameThreadExecutionContext)
|
||||
} catch {
|
||||
case NonFatal(ex) ⇒
|
||||
if (decider(ex) == Supervision.Stop) failStage(ex)
|
||||
}
|
||||
if (todo < parallelism) tryPull(in)
|
||||
}
|
||||
override def onUpstreamFinish(): Unit = {
|
||||
if (todo == 0) completeStage()
|
||||
}
|
||||
})
|
||||
|
||||
setHandler(out, new OutHandler {
|
||||
override def onPull(): Unit = {
|
||||
if (!buffer.isEmpty) push(out, buffer.dequeue())
|
||||
else if (isClosed(in) && todo == 0) completeStage()
|
||||
if (todo < parallelism && !hasBeenPulled(in)) tryPull(in)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -695,7 +696,7 @@ private[akka] final case class Log[T](name: String, extract: T ⇒ Any, logAdapt
|
|||
// TODO more optimisations can be done here - prepare logOnPush function etc
|
||||
|
||||
override def preStart(ctx: LifecycleContext): Unit = {
|
||||
logLevels = ctx.attributes.logLevels.getOrElse(DefaultLogLevels)
|
||||
logLevels = ctx.attributes.get[LogLevels](DefaultLogLevels)
|
||||
log = logAdapter match {
|
||||
case Some(l) ⇒ l
|
||||
case _ ⇒
|
||||
|
|
@ -787,7 +788,7 @@ private[stream] class GroupedWithin[T](n: Int, d: FiniteDuration) extends GraphS
|
|||
val out = Outlet[immutable.Seq[T]]("out")
|
||||
val shape = FlowShape(in, out)
|
||||
|
||||
override def createLogic: GraphStageLogic = new GraphStageLogic(shape) {
|
||||
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new TimerGraphStageLogic(shape) {
|
||||
private val buf: VectorBuilder[T] = new VectorBuilder
|
||||
// True if:
|
||||
// - buf is nonEmpty
|
||||
|
|
@ -855,13 +856,17 @@ private[stream] class GroupedWithin[T](n: Int, d: FiniteDuration) extends GraphS
|
|||
|
||||
private[stream] class TakeWithin[T](timeout: FiniteDuration) extends SimpleLinearGraphStage[T] {
|
||||
|
||||
override def createLogic: GraphStageLogic = new SimpleLinearStageLogic {
|
||||
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new TimerGraphStageLogic(shape) {
|
||||
setHandler(in, new InHandler {
|
||||
override def onPush(): Unit = push(out, grab(in))
|
||||
override def onUpstreamFinish(): Unit = completeStage()
|
||||
override def onUpstreamFailure(ex: Throwable): Unit = failStage(ex)
|
||||
})
|
||||
|
||||
setHandler(out, new OutHandler {
|
||||
override def onPull(): Unit = pull(in)
|
||||
})
|
||||
|
||||
final override protected def onTimer(key: Any): Unit =
|
||||
completeStage()
|
||||
|
||||
|
|
@ -874,7 +879,7 @@ private[stream] class TakeWithin[T](timeout: FiniteDuration) extends SimpleLinea
|
|||
private[stream] class DropWithin[T](timeout: FiniteDuration) extends SimpleLinearGraphStage[T] {
|
||||
private var allow = false
|
||||
|
||||
override def createLogic: GraphStageLogic = new SimpleLinearStageLogic {
|
||||
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new TimerGraphStageLogic(shape) {
|
||||
setHandler(in, new InHandler {
|
||||
override def onPush(): Unit =
|
||||
if (allow) push(out, grab(in))
|
||||
|
|
@ -883,6 +888,10 @@ private[stream] class DropWithin[T](timeout: FiniteDuration) extends SimpleLinea
|
|||
override def onUpstreamFailure(ex: Throwable): Unit = failStage(ex)
|
||||
})
|
||||
|
||||
setHandler(out, new OutHandler {
|
||||
override def onPull(): Unit = pull(in)
|
||||
})
|
||||
|
||||
final override protected def onTimer(key: Any): Unit = allow = true
|
||||
|
||||
override def preStart(): Unit = scheduleOnce("DropWithinTimer", timeout)
|
||||
|
|
|
|||
|
|
@ -1,15 +1,12 @@
|
|||
package akka.stream.impl.io
|
||||
|
||||
import akka.stream.ActorAttributes.Dispatcher
|
||||
import akka.stream.{ ActorMaterializer, MaterializationContext }
|
||||
import akka.stream.ActorAttributes
|
||||
import akka.stream.Attributes
|
||||
|
||||
private[stream] object IOSettings {
|
||||
|
||||
/** Picks default akka.stream.blocking-io-dispatcher or the Attributes configured one */
|
||||
def blockingIoDispatcher(context: MaterializationContext): String = {
|
||||
val mat = ActorMaterializer.downcast(context.materializer)
|
||||
context.effectiveAttributes.attributeList.collectFirst { case d: Dispatcher ⇒ d.dispatcher } getOrElse {
|
||||
mat.system.settings.config.getString("akka.stream.blocking-io-dispatcher")
|
||||
}
|
||||
}
|
||||
}
|
||||
final val SyncFileSourceDefaultChunkSize = 8192
|
||||
final val SyncFileSourceName = Attributes.name("synchronousFileSource")
|
||||
final val SyncFileSinkName = Attributes.name("synchronousFileSink")
|
||||
final val IODispatcher = ActorAttributes.Dispatcher("akka.stream.default-blocking-io-dispatcher")
|
||||
}
|
||||
|
|
|
|||
|
|
@ -4,13 +4,11 @@
|
|||
package akka.stream.impl.io
|
||||
|
||||
import java.io.{ File, OutputStream }
|
||||
import java.lang.{ Long ⇒ JLong }
|
||||
|
||||
import akka.stream._
|
||||
import akka.stream.impl.SinkModule
|
||||
import akka.stream.impl.StreamLayout.Module
|
||||
import akka.stream.{ ActorMaterializer, MaterializationContext, Attributes, SinkShape }
|
||||
import akka.stream.ActorAttributes.Dispatcher
|
||||
import akka.util.ByteString
|
||||
|
||||
import scala.concurrent.{ Future, Promise }
|
||||
|
||||
/**
|
||||
|
|
@ -27,7 +25,7 @@ private[akka] final class SynchronousFileSink(f: File, append: Boolean, val attr
|
|||
|
||||
val bytesWrittenPromise = Promise[Long]()
|
||||
val props = SynchronousFileSubscriber.props(f, bytesWrittenPromise, settings.maxInputBufferSize, append)
|
||||
val dispatcher = IOSettings.blockingIoDispatcher(context)
|
||||
val dispatcher = context.effectiveAttributes.get[Dispatcher](IOSettings.IODispatcher).dispatcher
|
||||
|
||||
val ref = mat.actorOf(context, props.withDispatcher(dispatcher))
|
||||
(akka.stream.actor.ActorSubscriber[ByteString](ref), bytesWrittenPromise.future)
|
||||
|
|
|
|||
|
|
@ -10,6 +10,7 @@ import java.util.concurrent.{ LinkedBlockingQueue, BlockingQueue }
|
|||
import akka.actor.{ ActorRef, Deploy }
|
||||
import akka.japi
|
||||
import akka.stream._
|
||||
import akka.stream.ActorAttributes.Dispatcher
|
||||
import akka.stream.impl.StreamLayout.Module
|
||||
import akka.stream.impl.{ ErrorPublisher, SourceModule }
|
||||
import akka.stream.scaladsl.{ Source, FlowGraph }
|
||||
|
|
@ -27,13 +28,13 @@ import scala.util.control.NonFatal
|
|||
private[akka] final class SynchronousFileSource(f: File, chunkSize: Int, val attributes: Attributes, shape: SourceShape[ByteString])
|
||||
extends SourceModule[ByteString, Future[Long]](shape) {
|
||||
override def create(context: MaterializationContext) = {
|
||||
// FIXME rewrite to be based on AsyncStage rather than dangerous downcasts
|
||||
// FIXME rewrite to be based on GraphStage rather than dangerous downcasts
|
||||
val mat = ActorMaterializer.downcast(context.materializer)
|
||||
val settings = mat.effectiveSettings(context.effectiveAttributes)
|
||||
|
||||
val bytesReadPromise = Promise[Long]()
|
||||
val props = SynchronousFilePublisher.props(f, bytesReadPromise, chunkSize, settings.initialInputBufferSize, settings.maxInputBufferSize)
|
||||
val dispatcher = IOSettings.blockingIoDispatcher(context)
|
||||
val dispatcher = context.effectiveAttributes.get[Dispatcher](IOSettings.IODispatcher).dispatcher
|
||||
|
||||
val ref = mat.actorOf(context, props.withDispatcher(dispatcher))
|
||||
|
||||
|
|
|
|||
|
|
@ -5,15 +5,14 @@ package akka.stream.impl.io
|
|||
|
||||
import java.io.{ IOException, InputStream }
|
||||
import java.util.concurrent.{ BlockingQueue, LinkedBlockingDeque, TimeUnit }
|
||||
|
||||
import akka.stream.Attributes.InputBuffer
|
||||
import akka.stream.impl.io.InputStreamSinkStage._
|
||||
import akka.stream.stage._
|
||||
import akka.util.ByteString
|
||||
|
||||
import scala.annotation.tailrec
|
||||
import scala.concurrent.Future
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
import akka.stream.Attributes
|
||||
|
||||
private[akka] object InputStreamSinkStage {
|
||||
|
||||
|
|
@ -38,7 +37,7 @@ private[akka] class InputStreamSinkStage(timeout: FiniteDuration) extends SinkSt
|
|||
val maxBuffer = module.attributes.getAttribute(classOf[InputBuffer], InputBuffer(16, 16)).max
|
||||
require(maxBuffer > 0, "Buffer size must be greater than 0")
|
||||
|
||||
override def createLogicAndMaterializedValue: (GraphStageLogic, InputStream) = {
|
||||
override def createLogicAndMaterializedValue(inheritedAttributes: Attributes): (GraphStageLogic, InputStream) = {
|
||||
|
||||
val dataQueue = new LinkedBlockingDeque[StreamToAdapterMessage](maxBuffer + 1)
|
||||
var pullRequestIsSent = true
|
||||
|
|
|
|||
|
|
@ -7,6 +7,7 @@ import java.io.{ IOException, OutputStream }
|
|||
import java.util.concurrent.atomic.AtomicReference
|
||||
import java.util.concurrent.{ BlockingQueue, LinkedBlockingQueue }
|
||||
|
||||
import akka.stream.Attributes
|
||||
import akka.stream.Attributes.InputBuffer
|
||||
import akka.stream.impl.io.OutputStreamSourceStage._
|
||||
import akka.stream.stage._
|
||||
|
|
@ -35,7 +36,7 @@ private[akka] class OutputStreamSourceStage(timeout: FiniteDuration) extends Sou
|
|||
val maxBuffer = module.attributes.getAttribute(classOf[InputBuffer], InputBuffer(16, 16)).max
|
||||
require(maxBuffer > 0, "Buffer size must be greater than 0")
|
||||
|
||||
override def createLogicAndMaterializedValue: (GraphStageLogic, OutputStream) = {
|
||||
override def createLogicAndMaterializedValue(inheritedAttributes: Attributes): (GraphStageLogic, OutputStream) = {
|
||||
val dataQueue = new LinkedBlockingQueue[ByteString](maxBuffer)
|
||||
|
||||
var flush: Option[Promise[Unit]] = None
|
||||
|
|
|
|||
|
|
@ -156,11 +156,11 @@ private[akka] class TcpListenStreamActor(localAddressPromise: Promise[InetSocket
|
|||
val tcpStreamActor = context.watch(context.actorOf(TcpStreamActor.inboundProps(connection, halfClose, settings)))
|
||||
val processor = ActorProcessor[ByteString, ByteString](tcpStreamActor)
|
||||
|
||||
import scala.concurrent.duration._
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
val handler = (idleTimeout match {
|
||||
case d: FiniteDuration ⇒ Flow[ByteString].join(Timeouts.idleTimeoutBidi[ByteString, ByteString](d))
|
||||
case _ ⇒ Flow[ByteString]
|
||||
}).andThenMat(() ⇒ (processor, ()))
|
||||
}).via(Flow.fromProcessor(() ⇒ processor))
|
||||
|
||||
val conn = StreamTcp.IncomingConnection(
|
||||
connected.localAddress,
|
||||
|
|
|
|||
119
akka-stream/src/main/scala/akka/stream/io/ByteStringParser.scala
Normal file
119
akka-stream/src/main/scala/akka/stream/io/ByteStringParser.scala
Normal file
|
|
@ -0,0 +1,119 @@
|
|||
/**
|
||||
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.stream.io
|
||||
|
||||
import scala.util.control.NoStackTrace
|
||||
import akka.stream._
|
||||
import akka.stream.stage._
|
||||
import akka.util.ByteString
|
||||
import scala.annotation.tailrec
|
||||
|
||||
abstract class ByteStringParser[T] extends GraphStage[FlowShape[ByteString, T]] {
|
||||
import ByteStringParser._
|
||||
|
||||
private val bytesIn = Inlet[ByteString]("bytesIn")
|
||||
private val objOut = Outlet[T]("objOut")
|
||||
|
||||
final override val shape = FlowShape(bytesIn, objOut)
|
||||
|
||||
class ParsingLogic extends GraphStageLogic(shape) {
|
||||
override def preStart(): Unit = pull(bytesIn)
|
||||
setHandler(objOut, eagerTerminateOutput)
|
||||
|
||||
private var buffer = ByteString.empty
|
||||
private var current: ParseStep[T] = FinishedParser
|
||||
|
||||
final protected def startWith(step: ParseStep[T]): Unit = current = step
|
||||
|
||||
@tailrec private def doParse(): Unit =
|
||||
if (buffer.nonEmpty) {
|
||||
val cont = try {
|
||||
val reader = new ByteReader(buffer)
|
||||
val (elem, next) = current.parse(reader)
|
||||
emit(objOut, elem)
|
||||
if (next == FinishedParser) {
|
||||
completeStage()
|
||||
false
|
||||
} else {
|
||||
buffer = reader.remainingData
|
||||
current = next
|
||||
true
|
||||
}
|
||||
} catch {
|
||||
case NeedMoreData ⇒
|
||||
pull(bytesIn)
|
||||
false
|
||||
}
|
||||
if (cont) doParse()
|
||||
} else pull(bytesIn)
|
||||
|
||||
setHandler(bytesIn, new InHandler {
|
||||
override def onPush(): Unit = {
|
||||
buffer ++= grab(bytesIn)
|
||||
doParse()
|
||||
}
|
||||
override def onUpstreamFinish(): Unit =
|
||||
if (buffer.isEmpty) completeStage()
|
||||
else current.onTruncation()
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
object ByteStringParser {
|
||||
|
||||
trait ParseStep[+T] {
|
||||
def parse(reader: ByteReader): (T, ParseStep[T])
|
||||
def onTruncation(): Unit = throw new IllegalStateException("truncated data in ByteStringParser")
|
||||
}
|
||||
|
||||
object FinishedParser extends ParseStep[Nothing] {
|
||||
def parse(reader: ByteReader) =
|
||||
throw new IllegalStateException("no initial parser installed: you must use startWith(...)")
|
||||
}
|
||||
|
||||
val NeedMoreData = new Exception with NoStackTrace
|
||||
|
||||
class ByteReader(input: ByteString) {
|
||||
|
||||
private[this] var off = 0
|
||||
|
||||
def hasRemaining: Boolean = off < input.size
|
||||
def remainingSize: Int = input.size - off
|
||||
|
||||
def currentOffset: Int = off
|
||||
def remainingData: ByteString = input.drop(off)
|
||||
def fromStartToHere: ByteString = input.take(off)
|
||||
|
||||
def take(n: Int): ByteString =
|
||||
if (off + n <= input.length) {
|
||||
val o = off
|
||||
off = o + n
|
||||
input.slice(o, off)
|
||||
} else throw NeedMoreData
|
||||
def takeAll(): ByteString = {
|
||||
val ret = remainingData
|
||||
off = input.size
|
||||
ret
|
||||
}
|
||||
|
||||
def readByte(): Int =
|
||||
if (off < input.length) {
|
||||
val x = input(off)
|
||||
off += 1
|
||||
x & 0xFF
|
||||
} else throw NeedMoreData
|
||||
def readShortLE(): Int = readByte() | (readByte() << 8)
|
||||
def readIntLE(): Int = readShortLE() | (readShortLE() << 16)
|
||||
def readLongLE(): Long = (readIntLE() & 0xffffffffL) | ((readIntLE() & 0xffffffffL) << 32)
|
||||
|
||||
def readShortBE(): Int = (readByte() << 8) | readByte()
|
||||
def readIntBE(): Int = (readShortBE() << 16) | readShortBE()
|
||||
def readLongBE(): Long = ((readIntBE() & 0xffffffffL) << 32) | (readIntBE() & 0xffffffffL)
|
||||
|
||||
def skip(numBytes: Int): Unit =
|
||||
if (off + numBytes <= input.length) off += numBytes
|
||||
else throw NeedMoreData
|
||||
def skipZeroTerminatedString(): Unit = while (readByte() != 0) {}
|
||||
}
|
||||
}
|
||||
|
|
@ -5,7 +5,6 @@ package akka.stream.io
|
|||
|
||||
import java.io.File
|
||||
|
||||
import akka.stream.impl.io.SynchronousFileSink
|
||||
import akka.stream.{ Attributes, javadsl, ActorAttributes }
|
||||
import akka.stream.scaladsl.Sink
|
||||
import akka.util.ByteString
|
||||
|
|
@ -16,8 +15,10 @@ import scala.concurrent.Future
|
|||
* Sink which writes incoming [[ByteString]]s to the given file
|
||||
*/
|
||||
object SynchronousFileSink {
|
||||
import akka.stream.impl.io.IOSettings._
|
||||
import akka.stream.impl.io.SynchronousFileSink
|
||||
|
||||
final val DefaultAttributes = Attributes.name("synchronousFileSink")
|
||||
final val DefaultAttributes = SyncFileSinkName and IODispatcher
|
||||
|
||||
/**
|
||||
* Synchronous (Java 6 compatible) Sink that writes incoming [[ByteString]] elements to the given file.
|
||||
|
|
|
|||
|
|
@ -10,9 +10,10 @@ import akka.util.ByteString
|
|||
import scala.concurrent.Future
|
||||
|
||||
object SynchronousFileSource {
|
||||
import akka.stream.impl.io.IOSettings._
|
||||
import akka.stream.impl.io.SynchronousFileSource
|
||||
final val DefaultChunkSize = 8192
|
||||
final val DefaultAttributes = Attributes.name("synchronousFileSource")
|
||||
|
||||
final val DefaultAttributes = SyncFileSourceName and IODispatcher
|
||||
|
||||
/**
|
||||
* Creates a synchronous (Java 6 compatible) Source from a Files contents.
|
||||
|
|
@ -24,7 +25,7 @@ object SynchronousFileSource {
|
|||
*
|
||||
* It materializes a [[Future]] containing the number of bytes read from the source file upon completion.
|
||||
*/
|
||||
def apply(f: File, chunkSize: Int = DefaultChunkSize): Source[ByteString, Future[Long]] =
|
||||
def apply(f: File, chunkSize: Int = SyncFileSourceDefaultChunkSize): Source[ByteString, Future[Long]] =
|
||||
new Source(new SynchronousFileSource(f, chunkSize, DefaultAttributes, Source.shape("SynchronousFileSource")).nest()) // TO DISCUSS: I had to add wrap() here to make the name available
|
||||
|
||||
/**
|
||||
|
|
@ -37,7 +38,7 @@ object SynchronousFileSource {
|
|||
*
|
||||
* It materializes a [[Future]] containing the number of bytes read from the source file upon completion.
|
||||
*/
|
||||
def create(f: File): javadsl.Source[ByteString, Future[java.lang.Long]] = create(f, DefaultChunkSize)
|
||||
def create(f: File): javadsl.Source[ByteString, Future[java.lang.Long]] = create(f, SyncFileSourceDefaultChunkSize)
|
||||
|
||||
/**
|
||||
* Creates a synchronous (Java 6 compatible) Source from a Files contents.
|
||||
|
|
|
|||
|
|
@ -5,7 +5,7 @@ import java.util.concurrent.{ TimeUnit, TimeoutException }
|
|||
import akka.stream.impl.fusing.GraphStages.SimpleLinearGraphStage
|
||||
import akka.stream.scaladsl.{ BidiFlow, Flow }
|
||||
import akka.stream.stage._
|
||||
import akka.stream.{ BidiShape, Inlet, Outlet }
|
||||
import akka.stream.{ BidiShape, Inlet, Outlet, Attributes }
|
||||
|
||||
import scala.concurrent.duration.{ Deadline, FiniteDuration }
|
||||
|
||||
|
|
@ -62,7 +62,7 @@ object Timeouts {
|
|||
|
||||
private class InitialTimeout[T](timeout: FiniteDuration) extends SimpleLinearGraphStage[T] {
|
||||
|
||||
override def createLogic: GraphStageLogic = new SimpleLinearStageLogic {
|
||||
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new TimerGraphStageLogic(shape) {
|
||||
private var initialHasPassed = false
|
||||
|
||||
setHandler(in, new InHandler {
|
||||
|
|
@ -72,6 +72,10 @@ object Timeouts {
|
|||
}
|
||||
})
|
||||
|
||||
setHandler(out, new OutHandler {
|
||||
override def onPull(): Unit = pull(in)
|
||||
})
|
||||
|
||||
final override protected def onTimer(key: Any): Unit =
|
||||
if (!initialHasPassed)
|
||||
failStage(new TimeoutException(s"The first element has not yet passed through in $timeout."))
|
||||
|
|
@ -84,11 +88,15 @@ object Timeouts {
|
|||
|
||||
private class CompletionTimeout[T](timeout: FiniteDuration) extends SimpleLinearGraphStage[T] {
|
||||
|
||||
override def createLogic: GraphStageLogic = new SimpleLinearStageLogic {
|
||||
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new TimerGraphStageLogic(shape) {
|
||||
setHandler(in, new InHandler {
|
||||
override def onPush(): Unit = push(out, grab(in))
|
||||
})
|
||||
|
||||
setHandler(out, new OutHandler {
|
||||
override def onPull(): Unit = pull(in)
|
||||
})
|
||||
|
||||
final override protected def onTimer(key: Any): Unit =
|
||||
failStage(new TimeoutException(s"The stream has not been completed in $timeout."))
|
||||
|
||||
|
|
@ -101,7 +109,7 @@ object Timeouts {
|
|||
private class IdleTimeout[T](timeout: FiniteDuration) extends SimpleLinearGraphStage[T] {
|
||||
private var nextDeadline: Deadline = Deadline.now + timeout
|
||||
|
||||
override def createLogic: GraphStageLogic = new SimpleLinearStageLogic {
|
||||
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new TimerGraphStageLogic(shape) {
|
||||
setHandler(in, new InHandler {
|
||||
override def onPush(): Unit = {
|
||||
nextDeadline = Deadline.now + timeout
|
||||
|
|
@ -109,6 +117,10 @@ object Timeouts {
|
|||
}
|
||||
})
|
||||
|
||||
setHandler(out, new OutHandler {
|
||||
override def onPull(): Unit = pull(in)
|
||||
})
|
||||
|
||||
final override protected def onTimer(key: Any): Unit =
|
||||
if (nextDeadline.isOverdue())
|
||||
failStage(new TimeoutException(s"No elements passed in the last $timeout."))
|
||||
|
|
@ -128,7 +140,7 @@ object Timeouts {
|
|||
|
||||
override def toString = "IdleTimeoutBidi"
|
||||
|
||||
override def createLogic: GraphStageLogic = new GraphStageLogic(shape) {
|
||||
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new TimerGraphStageLogic(shape) {
|
||||
private var nextDeadline: Deadline = Deadline.now + timeout
|
||||
|
||||
setHandler(in1, new InHandler {
|
||||
|
|
|
|||
|
|
@ -22,8 +22,14 @@ object Flow {
|
|||
/** Create a `Flow` which can process elements of type `T`. */
|
||||
def create[T](): javadsl.Flow[T, T, Unit] = fromGraph(scaladsl.Flow[T])
|
||||
|
||||
def create[I, O](processorFactory: function.Creator[Processor[I, O]]): javadsl.Flow[I, O, Unit] =
|
||||
new Flow(scaladsl.Flow(() ⇒ processorFactory.create()))
|
||||
def fromProcessor[I, O](processorFactory: function.Creator[Processor[I, O]]): javadsl.Flow[I, O, Unit] =
|
||||
new Flow(scaladsl.Flow.fromProcessor(() ⇒ processorFactory.create()))
|
||||
|
||||
def fromProcessorMat[I, O, Mat](processorFactory: function.Creator[Pair[Processor[I, O], Mat]]): javadsl.Flow[I, O, Mat] =
|
||||
new Flow(scaladsl.Flow.fromProcessorMat { () ⇒
|
||||
val javaPair = processorFactory.create()
|
||||
(javaPair.first, javaPair.second)
|
||||
})
|
||||
|
||||
/** Create a `Flow` which can process elements of type `T`. */
|
||||
def of[T](clazz: Class[T]): javadsl.Flow[T, T, Unit] = create[T]()
|
||||
|
|
|
|||
Some files were not shown because too many files have changed in this diff Show more
Loading…
Add table
Add a link
Reference in a new issue