!str #17090 add AsyncStage
This commit is contained in:
parent
ad3829cd74
commit
4c623fade7
66 changed files with 981 additions and 787 deletions
|
|
@ -4,16 +4,21 @@
|
|||
package akka.stream.impl.fusing
|
||||
|
||||
import scala.collection.immutable
|
||||
import akka.stream.OverflowStrategy
|
||||
import akka.stream.impl.FixedSizeBuffer
|
||||
import akka.stream.stage._
|
||||
import akka.stream._
|
||||
import akka.stream.Supervision
|
||||
import scala.concurrent.{ ExecutionContext, Future }
|
||||
import scala.util.{ Try, Success, Failure }
|
||||
import scala.annotation.tailrec
|
||||
import scala.util.control.NonFatal
|
||||
import akka.stream.impl.ReactiveStreamsCompliance
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] final case class Map[In, Out](f: In ⇒ Out, decider: Supervision.Decider) extends PushStage[In, Out] {
|
||||
override def onPush(elem: In, ctx: Context[Out]): Directive = ctx.push(f(elem))
|
||||
override def onPush(elem: In, ctx: Context[Out]): SyncDirective = ctx.push(f(elem))
|
||||
|
||||
override def decide(t: Throwable): Supervision.Directive = decider(t)
|
||||
}
|
||||
|
|
@ -22,7 +27,7 @@ private[akka] final case class Map[In, Out](f: In ⇒ Out, decider: Supervision.
|
|||
* INTERNAL API
|
||||
*/
|
||||
private[akka] final case class Filter[T](p: T ⇒ Boolean, decider: Supervision.Decider) extends PushStage[T, T] {
|
||||
override def onPush(elem: T, ctx: Context[T]): Directive =
|
||||
override def onPush(elem: T, ctx: Context[T]): SyncDirective =
|
||||
if (p(elem)) ctx.push(elem)
|
||||
else ctx.pull()
|
||||
|
||||
|
|
@ -38,7 +43,7 @@ private[akka] final object Collect {
|
|||
|
||||
private[akka] final case class Collect[In, Out](decider: Supervision.Decider)(pf: PartialFunction[In, Out]) extends PushStage[In, Out] {
|
||||
import Collect.NotApplied
|
||||
override def onPush(elem: In, ctx: Context[Out]): Directive =
|
||||
override def onPush(elem: In, ctx: Context[Out]): SyncDirective =
|
||||
pf.applyOrElse(elem, NotApplied) match {
|
||||
case NotApplied ⇒ ctx.pull()
|
||||
case result: Out @unchecked ⇒ ctx.push(result)
|
||||
|
|
@ -53,13 +58,13 @@ private[akka] final case class Collect[In, Out](decider: Supervision.Decider)(pf
|
|||
private[akka] final case class MapConcat[In, Out](f: In ⇒ immutable.Seq[Out], decider: Supervision.Decider) extends PushPullStage[In, Out] {
|
||||
private var currentIterator: Iterator[Out] = Iterator.empty
|
||||
|
||||
override def onPush(elem: In, ctx: Context[Out]): Directive = {
|
||||
override def onPush(elem: In, ctx: Context[Out]): SyncDirective = {
|
||||
currentIterator = f(elem).iterator
|
||||
if (currentIterator.isEmpty) ctx.pull()
|
||||
else ctx.push(currentIterator.next())
|
||||
}
|
||||
|
||||
override def onPull(ctx: Context[Out]): Directive =
|
||||
override def onPull(ctx: Context[Out]): SyncDirective =
|
||||
if (currentIterator.hasNext) ctx.push(currentIterator.next())
|
||||
else if (ctx.isFinishing) ctx.finish()
|
||||
else ctx.pull()
|
||||
|
|
@ -78,7 +83,7 @@ private[akka] final case class MapConcat[In, Out](f: In ⇒ immutable.Seq[Out],
|
|||
private[akka] final case class Take[T](count: Long) extends PushStage[T, T] {
|
||||
private var left: Long = count
|
||||
|
||||
override def onPush(elem: T, ctx: Context[T]): Directive = {
|
||||
override def onPush(elem: T, ctx: Context[T]): SyncDirective = {
|
||||
left -= 1
|
||||
if (left > 0) ctx.push(elem)
|
||||
else if (left == 0) ctx.pushAndFinish(elem)
|
||||
|
|
@ -91,7 +96,7 @@ private[akka] final case class Take[T](count: Long) extends PushStage[T, T] {
|
|||
*/
|
||||
private[akka] final case class Drop[T](count: Long) extends PushStage[T, T] {
|
||||
private var left: Long = count
|
||||
override def onPush(elem: T, ctx: Context[T]): Directive =
|
||||
override def onPush(elem: T, ctx: Context[T]): SyncDirective =
|
||||
if (left > 0) {
|
||||
left -= 1
|
||||
ctx.pull()
|
||||
|
|
@ -104,13 +109,13 @@ private[akka] final case class Drop[T](count: Long) extends PushStage[T, T] {
|
|||
private[akka] final case class Scan[In, Out](zero: Out, f: (Out, In) ⇒ Out, decider: Supervision.Decider) extends PushPullStage[In, Out] {
|
||||
private var aggregator = zero
|
||||
|
||||
override def onPush(elem: In, ctx: Context[Out]): Directive = {
|
||||
override def onPush(elem: In, ctx: Context[Out]): SyncDirective = {
|
||||
val old = aggregator
|
||||
aggregator = f(old, elem)
|
||||
ctx.push(old)
|
||||
}
|
||||
|
||||
override def onPull(ctx: Context[Out]): Directive =
|
||||
override def onPull(ctx: Context[Out]): SyncDirective =
|
||||
if (ctx.isFinishing) ctx.pushAndFinish(aggregator)
|
||||
else ctx.pull()
|
||||
|
||||
|
|
@ -127,12 +132,12 @@ private[akka] final case class Scan[In, Out](zero: Out, f: (Out, In) ⇒ Out, de
|
|||
private[akka] final case class Fold[In, Out](zero: Out, f: (Out, In) ⇒ Out, decider: Supervision.Decider) extends PushPullStage[In, Out] {
|
||||
private var aggregator = zero
|
||||
|
||||
override def onPush(elem: In, ctx: Context[Out]): Directive = {
|
||||
override def onPush(elem: In, ctx: Context[Out]): SyncDirective = {
|
||||
aggregator = f(aggregator, elem)
|
||||
ctx.pull()
|
||||
}
|
||||
|
||||
override def onPull(ctx: Context[Out]): Directive =
|
||||
override def onPull(ctx: Context[Out]): SyncDirective =
|
||||
if (ctx.isFinishing) ctx.pushAndFinish(aggregator)
|
||||
else ctx.pull()
|
||||
|
||||
|
|
@ -154,7 +159,7 @@ private[akka] final case class Grouped[T](n: Int) extends PushPullStage[T, immut
|
|||
}
|
||||
private var left = n
|
||||
|
||||
override def onPush(elem: T, ctx: Context[immutable.Seq[T]]): Directive = {
|
||||
override def onPush(elem: T, ctx: Context[immutable.Seq[T]]): SyncDirective = {
|
||||
buf += elem
|
||||
left -= 1
|
||||
if (left == 0) {
|
||||
|
|
@ -165,7 +170,7 @@ private[akka] final case class Grouped[T](n: Int) extends PushPullStage[T, immut
|
|||
} else ctx.pull()
|
||||
}
|
||||
|
||||
override def onPull(ctx: Context[immutable.Seq[T]]): Directive =
|
||||
override def onPull(ctx: Context[immutable.Seq[T]]): SyncDirective =
|
||||
if (ctx.isFinishing) {
|
||||
val elem = buf.result()
|
||||
buf.clear() //FIXME null out the reference to the `buf`?
|
||||
|
|
@ -184,10 +189,10 @@ private[akka] final case class Grouped[T](n: Int) extends PushPullStage[T, immut
|
|||
private[akka] final case class Buffer[T](size: Int, overflowStrategy: OverflowStrategy) extends DetachedStage[T, T] {
|
||||
import OverflowStrategy._
|
||||
|
||||
private val buffer = FixedSizeBuffer(size)
|
||||
private val buffer = FixedSizeBuffer[T](size)
|
||||
|
||||
override def onPush(elem: T, ctx: DetachedContext[T]): UpstreamDirective =
|
||||
if (ctx.isHolding) ctx.pushAndPull(elem)
|
||||
if (ctx.isHoldingDownstream) ctx.pushAndPull(elem)
|
||||
else enqueueAction(ctx, elem)
|
||||
|
||||
override def onPull(ctx: DetachedContext[T]): DownstreamDirective = {
|
||||
|
|
@ -195,8 +200,8 @@ private[akka] final case class Buffer[T](size: Int, overflowStrategy: OverflowSt
|
|||
val elem = buffer.dequeue().asInstanceOf[T]
|
||||
if (buffer.isEmpty) ctx.pushAndFinish(elem)
|
||||
else ctx.push(elem)
|
||||
} else if (ctx.isHolding) ctx.pushAndPull(buffer.dequeue().asInstanceOf[T])
|
||||
else if (buffer.isEmpty) ctx.hold()
|
||||
} else if (ctx.isHoldingUpstream) ctx.pushAndPull(buffer.dequeue().asInstanceOf[T])
|
||||
else if (buffer.isEmpty) ctx.holdDownstream()
|
||||
else ctx.push(buffer.dequeue().asInstanceOf[T])
|
||||
}
|
||||
|
||||
|
|
@ -223,7 +228,7 @@ private[akka] final case class Buffer[T](size: Int, overflowStrategy: OverflowSt
|
|||
}
|
||||
case Backpressure ⇒ { (ctx, elem) ⇒
|
||||
buffer.enqueue(elem)
|
||||
if (buffer.isFull) ctx.hold()
|
||||
if (buffer.isFull) ctx.holdUpstream()
|
||||
else ctx.pull()
|
||||
}
|
||||
case Fail ⇒ { (ctx, elem) ⇒
|
||||
|
|
@ -241,8 +246,8 @@ private[akka] final case class Buffer[T](size: Int, overflowStrategy: OverflowSt
|
|||
* INTERNAL API
|
||||
*/
|
||||
private[akka] final case class Completed[T]() extends PushPullStage[T, T] {
|
||||
override def onPush(elem: T, ctx: Context[T]): Directive = ctx.finish()
|
||||
override def onPull(ctx: Context[T]): Directive = ctx.finish()
|
||||
override def onPush(elem: T, ctx: Context[T]): SyncDirective = ctx.finish()
|
||||
override def onPull(ctx: Context[T]): SyncDirective = ctx.finish()
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -257,7 +262,7 @@ private[akka] final case class Conflate[In, Out](seed: In ⇒ Out, aggregate: (O
|
|||
if (agg == null) seed(elem)
|
||||
else aggregate(agg.asInstanceOf[Out], elem)
|
||||
|
||||
if (!ctx.isHolding) ctx.pull()
|
||||
if (!ctx.isHoldingDownstream) ctx.pull()
|
||||
else {
|
||||
val result = agg.asInstanceOf[Out]
|
||||
agg = null
|
||||
|
|
@ -273,7 +278,7 @@ private[akka] final case class Conflate[In, Out](seed: In ⇒ Out, aggregate: (O
|
|||
agg = null
|
||||
ctx.pushAndFinish(result)
|
||||
}
|
||||
} else if (agg == null) ctx.hold()
|
||||
} else if (agg == null) ctx.holdDownstream()
|
||||
else {
|
||||
val result = agg.asInstanceOf[Out]
|
||||
if (result == null) throw new NullPointerException
|
||||
|
|
@ -301,24 +306,24 @@ private[akka] final case class Expand[In, Out, Seed](seed: In ⇒ Seed, extrapol
|
|||
s = seed(elem)
|
||||
started = true
|
||||
expanded = false
|
||||
if (ctx.isHolding) {
|
||||
if (ctx.isHoldingDownstream) {
|
||||
val (emit, newS) = extrapolate(s)
|
||||
s = newS
|
||||
expanded = true
|
||||
ctx.pushAndPull(emit)
|
||||
} else ctx.hold()
|
||||
} else ctx.holdUpstream()
|
||||
}
|
||||
|
||||
override def onPull(ctx: DetachedContext[Out]): DownstreamDirective = {
|
||||
if (ctx.isFinishing) {
|
||||
if (!started) ctx.finish()
|
||||
else ctx.pushAndFinish(extrapolate(s)._1)
|
||||
} else if (!started) ctx.hold()
|
||||
} else if (!started) ctx.holdDownstream()
|
||||
else {
|
||||
val (emit, newS) = extrapolate(s)
|
||||
s = newS
|
||||
expanded = true
|
||||
if (ctx.isHolding) ctx.pushAndPull(emit)
|
||||
if (ctx.isHoldingUpstream) ctx.pushAndPull(emit)
|
||||
else ctx.push(emit)
|
||||
}
|
||||
|
||||
|
|
@ -334,3 +339,155 @@ private[akka] final case class Expand[In, Out, Seed](seed: In ⇒ Seed, extrapol
|
|||
final override def restart(): Expand[In, Out, Seed] =
|
||||
throw new UnsupportedOperationException("Expand doesn't support restart")
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] object MapAsync {
|
||||
val NotYetThere = Failure(new Exception)
|
||||
}
|
||||
|
||||
/**
|
||||
* 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])] {
|
||||
import MapAsync._
|
||||
|
||||
type Notification = (Int, Try[Out])
|
||||
|
||||
private var callback: AsyncCallback[Notification] = _
|
||||
private val elemsInFlight = FixedSizeBuffer[Try[Out]](parallelism)
|
||||
|
||||
override def initAsyncInput(ctx: AsyncContext[Out, Notification]): Unit = {
|
||||
callback = ctx.getAsyncCallback()
|
||||
}
|
||||
|
||||
override def decide(ex: Throwable) = decider(ex)
|
||||
|
||||
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()
|
||||
}
|
||||
|
||||
override def onPull(ctx: AsyncContext[Out, (Int, Try[Out])]) = {
|
||||
@tailrec def rec(hasFreedUpSpace: Boolean): DownstreamDirective =
|
||||
if (elemsInFlight.isEmpty && ctx.isFinishing) ctx.finish()
|
||||
else if (elemsInFlight.isEmpty || elemsInFlight.peek == NotYetThere) {
|
||||
if (hasFreedUpSpace && ctx.isHoldingUpstream) ctx.holdDownstreamAndPull()
|
||||
else ctx.holdDownstream()
|
||||
} else elemsInFlight.dequeue() match {
|
||||
case Failure(ex) ⇒ rec(true)
|
||||
case Success(elem) ⇒
|
||||
if (ctx.isHoldingUpstream) ctx.pushAndPull(elem)
|
||||
else ctx.push(elem)
|
||||
}
|
||||
rec(false)
|
||||
}
|
||||
|
||||
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 == NotYetThere) ctx.ignore()
|
||||
else elemsInFlight.dequeue() match {
|
||||
case Failure(ex) ⇒ rec()
|
||||
case Success(elem) ⇒
|
||||
if (ctx.isHoldingUpstream) ctx.pushAndPull(elem)
|
||||
else ctx.push(elem)
|
||||
}
|
||||
|
||||
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 ex = try {
|
||||
ReactiveStreamsCompliance.requireNonNullElement(s.value)
|
||||
elemsInFlight.put(idx, s)
|
||||
null: Exception
|
||||
} catch {
|
||||
case NonFatal(ex) ⇒
|
||||
if (decider(ex) != Supervision.Stop) {
|
||||
elemsInFlight.put(idx, Failure(ex))
|
||||
null: Exception
|
||||
} else ex
|
||||
}
|
||||
if (ex != null) ctx.fail(ex)
|
||||
else if (ctx.isHoldingDownstream) rec()
|
||||
else ctx.ignore()
|
||||
}
|
||||
}
|
||||
|
||||
override def onUpstreamFinish(ctx: AsyncContext[Out, Notification]) =
|
||||
if (ctx.isHoldingUpstream || !elemsInFlight.isEmpty) ctx.absorbTermination()
|
||||
else ctx.finish()
|
||||
}
|
||||
|
||||
/**
|
||||
* 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 var callback: AsyncCallback[Try[Out]] = _
|
||||
private var inFlight = 0
|
||||
private val buffer = FixedSizeBuffer[Out](parallelism)
|
||||
|
||||
private def todo = inFlight + buffer.used
|
||||
|
||||
override def initAsyncInput(ctx: AsyncContext[Out, Try[Out]]): Unit = {
|
||||
callback = ctx.getAsyncCallback()
|
||||
}
|
||||
|
||||
override def decide(ex: Throwable) = decider(ex)
|
||||
|
||||
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()
|
||||
}
|
||||
|
||||
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)
|
||||
}
|
||||
|
||||
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 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()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
override def onUpstreamFinish(ctx: AsyncContext[Out, Try[Out]]) =
|
||||
if (todo > 0) ctx.absorbTermination()
|
||||
else ctx.finish()
|
||||
}
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue