!str,htc replace and remove OneBoundedInterpreter
main work by @drewhk with contributions from @2m and @rkuhn This work uncovered many well-hidden bugs in existing Stages, in particular StatefulStage. These were hidden by the behavior of OneBoundedInterpreter that normally behaves more orderly than it guarantees in general, especially with respect to the timeliness of delivery of upstream termination signals; the bugs were then that internal state was not flushed when onComplete arrived “too early”.
This commit is contained in:
parent
20f54435f1
commit
556012b7ee
107 changed files with 2456 additions and 3061 deletions
|
|
@ -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()
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue