From a3b1bed7dc1fd270a25c16fcc588689122589345 Mon Sep 17 00:00:00 2001 From: Daniel Spiewak Date: Sun, 26 Nov 2023 15:18:10 -0600 Subject: [PATCH 01/47] Reimplemented Dispatcher.sequential(false) --- .../scala/cats/effect/std/Dispatcher.scala | 387 ++++++++---------- 1 file changed, 160 insertions(+), 227 deletions(-) diff --git a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala index 48a77506cb..eb835242de 100644 --- a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala +++ b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala @@ -16,16 +16,15 @@ package cats.effect.std -import cats.effect.kernel.{Async, Outcome, Resource} +import cats.Applicative +import cats.effect.kernel.{Async, Concurrent, MonadCancel, Outcome, Resource, Sync} import cats.effect.std.Dispatcher.parasiticEC import cats.syntax.all._ -import scala.annotation.tailrec -import scala.collection.mutable import scala.concurrent.{ExecutionContext, Future, Promise} -import scala.util.{Failure, Success} +import scala.util.Failure -import java.util.concurrent.ThreadLocalRandom +import java.util.concurrent.{LinkedBlockingQueue /*, ThreadLocalRandom*/} import java.util.concurrent.atomic.{AtomicBoolean, AtomicReference} /** @@ -71,10 +70,13 @@ trait Dispatcher[F[_]] extends DispatcherPlatform[F] { */ def unsafeRunAndForget[A](fa: F[A]): Unit = unsafeToFuture(fa).onComplete { - case Failure(ex) => ex.printStackTrace() + case Failure(ex) => reportFailure(ex) case _ => () }(parasiticEC) + protected def reportFailure(t: Throwable): Unit = + t.printStackTrace() + // package-private because it's just an internal utility which supports specific implementations // anyone who needs this type of thing should use unsafeToFuture and then onComplete private[std] def unsafeRunAsync[A](fa: F[A])(cb: Either[Throwable, A] => Unit): Unit = @@ -89,12 +91,7 @@ object Dispatcher { def reportFailure(t: Throwable) = t.printStackTrace() } - private[this] val Cpus: Int = Runtime.getRuntime().availableProcessors() - - private[this] val Noop: () => Unit = () => () - private[this] val Open: () => Unit = () => () - - private[this] val Completed: Either[Throwable, Unit] = Right(()) + // private[this] val Cpus: Int = Runtime.getRuntime().availableProcessors() @deprecated( message = @@ -107,16 +104,14 @@ object Dispatcher { * exits, all active effects will be canceled, and attempts to submit new effects will throw * an exception. */ - def parallel[F[_]: Async]: Resource[F, Dispatcher[F]] = - parallel[F](await = false) + def parallel[F[_]: Async]: Resource[F, Dispatcher[F]] = parallel(false) /** * Create a [[Dispatcher]] that can be used within a resource scope. Once the resource scope * exits, all active effects will be canceled, and attempts to submit new effects will throw * an exception. */ - def sequential[F[_]: Async]: Resource[F, Dispatcher[F]] = - sequential[F](await = false) + def sequential[F[_]: Async]: Resource[F, Dispatcher[F]] = sequential(false) /** * Create a [[Dispatcher]] that can be used within a resource scope. Once the resource scope @@ -153,8 +148,7 @@ object Dispatcher { * - true - wait for the completion of the active fibers * - false - cancel the active fibers */ - def parallel[F[_]: Async](await: Boolean): Resource[F, Dispatcher[F]] = - apply(Mode.Parallel, await) + def parallel[F[_]: Async](await: Boolean): Resource[F, Dispatcher[F]] = sequential[F](await) /** * Create a [[Dispatcher]] that can be used within a resource scope. Once the resource scope @@ -190,240 +184,179 @@ object Dispatcher { * - false - cancel the active fiber */ def sequential[F[_]: Async](await: Boolean): Resource[F, Dispatcher[F]] = - apply(Mode.Sequential, await) - - private[this] def apply[F[_]](mode: Mode, await: Boolean)( - implicit F: Async[F]): Resource[F, Dispatcher[F]] = { - final case class Registration(action: F[Unit], prepareCancel: F[Unit] => Unit) - extends AtomicBoolean(true) - - sealed trait CancelState - case object CancelInit extends CancelState - final case class CanceledNoToken(promise: Promise[Unit]) extends CancelState - final case class CancelToken(cancelToken: () => Future[Unit]) extends CancelState - - val (workers, makeFork) = - mode match { - case Mode.Parallel => - (Cpus, Supervisor[F](await).map(s => s.supervise(_: F[Unit]).map(_.cancel))) - - case Mode.Sequential => - ( - 1, - Resource - .pure[F, F[Unit] => F[F[Unit]]]((_: F[Unit]).as(F.unit).handleError(_ => F.unit))) - } - - for { - fork <- makeFork - - latches <- Resource.eval(F delay { - val latches = new Array[AtomicReference[() => Unit]](workers) - var i = 0 - while (i < workers) { - latches(i) = new AtomicReference(Noop) - i += 1 - } - latches - }) - states <- Resource.eval(F delay { - val states = Array.ofDim[AtomicReference[List[Registration]]](workers, workers) - var i = 0 - while (i < workers) { - var j = 0 - while (j < workers) { - states(i)(j) = new AtomicReference(Nil) - j += 1 - } - i += 1 - } - states - }) - ec <- Resource.eval(F.executionContext) - - // supervisor for the main loop, which needs to always restart unless the Supervisor itself is canceled - // critically, inner actions can be canceled without impacting the loop itself - supervisor <- Supervisor[F](await, Some((_: Outcome[F, Throwable, _]) => true)) - - _ <- { - def step( - state: Array[AtomicReference[List[Registration]]], - await: F[Unit], - doneR: AtomicBoolean): F[Unit] = - for { - done <- F.delay(doneR.get()) - regs <- F delay { - val buffer = mutable.ListBuffer.empty[Registration] - var i = 0 - while (i < workers) { - val st = state(i) - if (st.get() ne null) { - val list = if (done) st.getAndSet(null) else st.getAndSet(Nil) - if ((list ne null) && (list ne Nil)) { - buffer ++= list.reverse // FIFO order here is a form of fairness + Supervisor[F]( + await = await, + checkRestart = Some((_: Outcome[F, Throwable, _]) => true)) flatMap { supervisor => + // we only need this flag to raise the IllegalStateException after closure (Supervisor can't do it for us) + Resource.make(Sync[F].delay(new AtomicBoolean(false)))(doneR => + Sync[F].delay(doneR.set(true))) evalMap { doneR => + // we sequentialize on worker spawning, so we don't need to use Deferred here + Concurrent[F].ref(Applicative[F].unit) flatMap { cancelR => + // in sequential spawning, the only cancelation cancels and restarts the worker itself + def spawn(fu: F[Unit])(setupCancelation: F[Unit] => F[Unit]): F[Unit] = + cancelR.get.flatMap(setupCancelation) *> fu + + Worker[F](supervisor)(spawn) flatMap { worker => + Async[F].executionContext flatMap { ec => + supervisor + .supervise(worker.run) + .flatMap(f => cancelR.set(f.cancel)) + .as(new Dispatcher[F] { + def unsafeToFutureCancelable[A](fa: F[A]): (Future[A], () => Future[Unit]) = { + def inner[E](fe: F[E], forceParallel: Boolean) + : (Future[E], () => Future[Unit]) = { + if (doneR.get()) { + throw new IllegalStateException("Dispatcher already closed") + } + + val p = Promise[E]() + + // forward atomicity guarantees onto promise completion + val fu = MonadCancel[F] uncancelable { poll => + poll(fe).redeemWith( + e => Sync[F].delay(p.failure(e)), + a => Sync[F].delay(p.success(a))) + } + + val reg = + new Registration(fu.void, new AtomicReference[F[Unit]](), forceParallel) + worker.queue.unsafeOffer(reg) + + def cancel(): Future[Unit] = { + reg.action = null.asInstanceOf[F[Unit]] + + // publishes action write + // TODO this provides incorrect semantics for multiple cancel() call sites + val cancelF = reg + .cancelR + .getAndSet(Applicative[F].unit) // anything that isn't null, really + if (cancelF != null) + inner( + cancelF, + true)._1 // this looping is fine, since we drop the cancel + else + Future.successful(()) + } + + (p.future, cancel _) + } + + inner(fa, false) } - } - i += 1 - } - buffer.toList - } - - _ <- - if (regs.isEmpty) { - await - } else { - regs traverse_ { - case r @ Registration(action, prepareCancel) => - val supervise: F[Unit] = - fork(action).flatMap(cancel => F.delay(prepareCancel(cancel))) - - // Check for task cancelation before executing. - F.delay(r.get()).ifM(supervise, F.delay(prepareCancel(F.unit))) - } - } - } yield () - - def dispatcher( - doneR: AtomicBoolean, - latch: AtomicReference[() => Unit], - state: Array[AtomicReference[List[Registration]]]): F[Unit] = { - - val await = - F.async_[Unit] { cb => - if (!latch.compareAndSet(Noop, () => cb(Completed))) { - // state was changed between when we last set the latch and now; complete the callback immediately - cb(Completed) - } - } - F.delay(latch.set(Noop)) *> // reset latch - // if we're marked as done, yield immediately to give other fibers a chance to shut us down - // we might loop on this a few times since we're marked as done before the supervisor is canceled - F.delay(doneR.get()).ifM(F.cede, step(state, await, doneR)) - } - - 0.until(workers).toList traverse_ { n => - Resource.eval(F.delay(new AtomicBoolean(false))) flatMap { doneR => - val latch = latches(n) - val worker = dispatcher(doneR, latch, states(n)) - val release = F.delay(latch.getAndSet(Open)()) - Resource.make(supervisor.supervise(worker)) { _ => - F.delay(doneR.set(true)) *> step(states(n), F.unit, doneR) *> release + override def reportFailure(t: Throwable): Unit = + ec.reportFailure(t) + }) } } } } - } yield { - new Dispatcher[F] { - override def unsafeRunAndForget[A](fa: F[A]): Unit = { - unsafeToFutureCancelable(fa) - ._1 - .onComplete { - case Failure(ex) => ec.reportFailure(ex) - case _ => () - }(parasiticEC) - } + } - def unsafeToFutureCancelable[E](fe: F[E]): (Future[E], () => Future[Unit]) = { - val promise = Promise[E]() + private final class Registration[F[_]]( + var action: F[Unit], + val cancelR: AtomicReference[F[Unit]], + val forceParallel: Boolean) - val action = fe - .flatMap(e => F.delay(promise.success(e))) - .handleErrorWith(t => F.delay(promise.failure(t))) - .void + // the signal is just a skolem for the atomic references; we never actually run it + private final class Worker[F[_]: Sync]( + val queue: UnsafeAsyncQueue[F, Registration[F]], + supervisor: Supervisor[F])( // only needed for cancelation spawning + spawn: F[Unit] => (F[Unit] => F[Unit]) => F[Unit]) { - val cancelState = new AtomicReference[CancelState](CancelInit) + val run: F[Unit] = { + val go = queue.take flatMap { reg => + // println("got registration") - def registerCancel(token: F[Unit]): Unit = { - val cancelToken = () => unsafeToFuture(token) + Sync[F].delay(reg.cancelR.get()) flatMap { sig => + val action = reg.action - @tailrec - def loop(): Unit = { - val state = cancelState.get() - state match { - case CancelInit => - if (!cancelState.compareAndSet(state, CancelToken(cancelToken))) { - loop() - } - case CanceledNoToken(promise) => - if (!cancelState.compareAndSet(state, CancelToken(cancelToken))) { - loop() - } else { - cancelToken().onComplete { - case Success(_) => promise.success(()) - case Failure(ex) => promise.failure(ex) - }(ec) - } - case _ => () - } - } + // double null check catches overly-aggressive memory fencing + if (sig == null && action != null) { + // println("...it wasn't canceled") - loop() - } - - @tailrec - def enqueue(state: AtomicReference[List[Registration]], reg: Registration): Unit = { - val curr = state.get() - if (curr eq null) { - throw new IllegalStateException("dispatcher already shutdown") + if (reg.forceParallel) { + // this branch is only used for cancelation + supervisor.supervise(action).void } else { - val next = reg :: curr - if (!state.compareAndSet(curr, next)) enqueue(state, reg) + // this is the main branch + spawn(action) { cancelF => + // println("...spawned") + + Sync[F].delay( + reg.cancelR.compareAndSet(null.asInstanceOf[F[Unit]], cancelF)) flatMap { + check => + // we need to double-check that we didn't lose the race + if (check) + // don't cancel, already spawned + Applicative[F].unit + else + // cancel spawn (we lost the race) + cancelF + } + } } - } - - val (state, lt) = if (workers > 1) { - val rand = ThreadLocalRandom.current() - val dispatcher = rand.nextInt(workers) - val inner = rand.nextInt(workers) - - (states(dispatcher)(inner), latches(dispatcher)) } else { - (states(0)(0), latches(0)) + // don't spawn, already canceled + Applicative[F].unit } + } + } - val reg = Registration(action, registerCancel _) - enqueue(state, reg) + go.foreverM + } + } - if (lt.get() ne Open) { - val f = lt.getAndSet(Open) - f() - } + private object Worker { + def apply[F[_]: Async](supervisor: Supervisor[F])( + spawn: F[Unit] => (F[Unit] => F[Unit]) => F[Unit]): F[Worker[F]] = + Sync[F].delay( + new Worker[F](new UnsafeAsyncQueue[F, Registration[F]](), supervisor)(spawn)) + } - val cancel = { () => - reg.lazySet(false) - - @tailrec - def loop(): Future[Unit] = { - val state = cancelState.get() - state match { - case CancelInit => - val promise = Promise[Unit]() - if (!cancelState.compareAndSet(state, CanceledNoToken(promise))) { - loop() - } else { - promise.future - } - case CanceledNoToken(promise) => - promise.future - case CancelToken(cancelToken) => - cancelToken() - } - } + private val Signal: Either[Any, Unit] => Unit = _ => () + private val RightUnit: Right[Nothing, Unit] = Right(()) - loop() - } + // MPSC assumption + private final class UnsafeAsyncQueue[F[_]: Async, A] { + private[this] val buffer = new LinkedBlockingQueue[A]() + private[this] val latchR = new AtomicReference[Either[Throwable, Unit] => Unit](null) - (promise.future, cancel) + def unsafeOffer(a: A): Unit = { + buffer.offer(a) + + val f = latchR.get() + if (latchR.compareAndSet(f, Signal)) { + if (f != null) { + f(RightUnit) } + // if f == null, take will loop back around and discover the Signal } } - } - private sealed trait Mode extends Product with Serializable + def take: F[A] = { + val latchF = Async[F].asyncCheckAttempt[Unit] { k => + Sync[F] delay { + if (latchR.compareAndSet(null, k)) { + Left(Some(Applicative[F].unit)) // all cleanup is elsewhere + } else { + val result = latchR.compareAndSet(Signal, null) + require( + result + ) // since this is a single consumer queue, we should never have multiple callbacks + Right(()) + } + } + } - private object Mode { - case object Parallel extends Mode - case object Sequential extends Mode + MonadCancel[F] uncancelable { poll => + Sync[F].delay(buffer.poll()) flatMap { a => + if (a == null) + poll(latchF >> take) + else + Applicative[F].pure(a) + } + } + } } } From f3199a02508ead4435fbff4b47c1e9e763f4a017 Mon Sep 17 00:00:00 2001 From: Daniel Spiewak Date: Sun, 26 Nov 2023 17:07:38 -0600 Subject: [PATCH 02/47] Added support for `Dispatcher.sequential(await = true)` --- .../scala/cats/effect/std/Dispatcher.scala | 120 +++++++++++------- 1 file changed, 71 insertions(+), 49 deletions(-) diff --git a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala index eb835242de..a5b595300d 100644 --- a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala +++ b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala @@ -188,22 +188,24 @@ object Dispatcher { await = await, checkRestart = Some((_: Outcome[F, Throwable, _]) => true)) flatMap { supervisor => // we only need this flag to raise the IllegalStateException after closure (Supervisor can't do it for us) - Resource.make(Sync[F].delay(new AtomicBoolean(false)))(doneR => - Sync[F].delay(doneR.set(true))) evalMap { doneR => + val termination = Resource.make(Sync[F].delay(new AtomicBoolean(false)))(doneR => + Sync[F].delay(doneR.set(true))) + + termination flatMap { doneR => // we sequentialize on worker spawning, so we don't need to use Deferred here - Concurrent[F].ref(Applicative[F].unit) flatMap { cancelR => + Resource.eval(Concurrent[F].ref(Applicative[F].unit)) flatMap { cancelR => // in sequential spawning, the only cancelation cancels and restarts the worker itself def spawn(fu: F[Unit])(setupCancelation: F[Unit] => F[Unit]): F[Unit] = cancelR.get.flatMap(setupCancelation) *> fu - Worker[F](supervisor)(spawn) flatMap { worker => + Worker[F](supervisor)(spawn) evalMap { worker => Async[F].executionContext flatMap { ec => supervisor .supervise(worker.run) .flatMap(f => cancelR.set(f.cancel)) .as(new Dispatcher[F] { def unsafeToFutureCancelable[A](fa: F[A]): (Future[A], () => Future[Unit]) = { - def inner[E](fe: F[E], forceParallel: Boolean) + def inner[E](fe: F[E], isFinalizer: Boolean) : (Future[E], () => Future[Unit]) = { if (doneR.get()) { throw new IllegalStateException("Dispatcher already closed") @@ -218,27 +220,33 @@ object Dispatcher { a => Sync[F].delay(p.success(a))) } - val reg = - new Registration(fu.void, new AtomicReference[F[Unit]](), forceParallel) - worker.queue.unsafeOffer(reg) - - def cancel(): Future[Unit] = { - reg.action = null.asInstanceOf[F[Unit]] - - // publishes action write - // TODO this provides incorrect semantics for multiple cancel() call sites - val cancelF = reg - .cancelR - .getAndSet(Applicative[F].unit) // anything that isn't null, really - if (cancelF != null) - inner( - cancelF, - true)._1 // this looping is fine, since we drop the cancel - else - Future.successful(()) + if (isFinalizer) { + worker.queue.unsafeOffer(Registration.Finalizer(fu.void)) + + // cannot cancel a cancel + (p.future, () => Future.failed(new UnsupportedOperationException)) + } else { + val reg = + new Registration.Primary(fu.void, new AtomicReference[F[Unit]]()) + worker.queue.unsafeOffer(reg) + + def cancel(): Future[Unit] = { + reg.action = null.asInstanceOf[F[Unit]] + + // publishes action write + // TODO this provides incorrect semantics for multiple cancel() call sites + val cancelF = reg + .cancelR + .getAndSet(Applicative[F].unit) // anything that isn't null, really + if (cancelF != null) + // this looping is fine, since we drop the cancel + inner(cancelF, true)._1 + else + Future.successful(()) + } + + (p.future, cancel _) } - - (p.future, cancel _) } inner(fa, false) @@ -253,10 +261,16 @@ object Dispatcher { } } - private final class Registration[F[_]]( - var action: F[Unit], - val cancelR: AtomicReference[F[Unit]], - val forceParallel: Boolean) + private sealed abstract class Registration[F[_]] + + private object Registration { + final class Primary[F[_]](var action: F[Unit], val cancelR: AtomicReference[F[Unit]]) + extends Registration[F] + + final case class Finalizer[F[_]](action: F[Unit]) extends Registration[F] + + final case class PoisonPill[F[_]]() extends Registration[F] + } // the signal is just a skolem for the atomic references; we never actually run it private final class Worker[F[_]: Sync]( @@ -264,22 +278,20 @@ object Dispatcher { supervisor: Supervisor[F])( // only needed for cancelation spawning spawn: F[Unit] => (F[Unit] => F[Unit]) => F[Unit]) { - val run: F[Unit] = { - val go = queue.take flatMap { reg => - // println("got registration") + private[this] val doneR = new AtomicBoolean(false) - Sync[F].delay(reg.cancelR.get()) flatMap { sig => - val action = reg.action + def run: F[Unit] = { + val step = queue.take flatMap { + case reg: Registration.Primary[F] => + // println("got registration") - // double null check catches overly-aggressive memory fencing - if (sig == null && action != null) { - // println("...it wasn't canceled") + Sync[F].delay(reg.cancelR.get()) flatMap { sig => + val action = reg.action + + // double null check catches overly-aggressive memory fencing + if (sig == null && action != null) { + // println("...it wasn't canceled") - if (reg.forceParallel) { - // this branch is only used for cancelation - supervisor.supervise(action).void - } else { - // this is the main branch spawn(action) { cancelF => // println("...spawned") @@ -295,23 +307,33 @@ object Dispatcher { cancelF } } + } else { + // don't spawn, already canceled + Applicative[F].unit } - } else { - // don't spawn, already canceled - Applicative[F].unit } - } + + case Registration.Finalizer(action) => + supervisor.supervise(action).void + + case Registration.PoisonPill() => + Sync[F].delay(doneR.set(true)) } - go.foreverM + Sync[F].delay(doneR.get()).ifM(Applicative[F].unit, step >> run) } } private object Worker { + def apply[F[_]: Async](supervisor: Supervisor[F])( - spawn: F[Unit] => (F[Unit] => F[Unit]) => F[Unit]): F[Worker[F]] = - Sync[F].delay( + spawn: F[Unit] => (F[Unit] => F[Unit]) => F[Unit]): Resource[F, Worker[F]] = { + + val initF = Sync[F].delay( new Worker[F](new UnsafeAsyncQueue[F, Registration[F]](), supervisor)(spawn)) + + Resource.make(initF)(w => Sync[F].delay(w.queue.unsafeOffer(Registration.PoisonPill()))) + } } private val Signal: Either[Any, Unit] => Unit = _ => () From fc873a314f6d296a1e65d3cb3dd8c1add48ab4e8 Mon Sep 17 00:00:00 2001 From: Daniel Spiewak Date: Mon, 27 Nov 2023 09:03:37 -0600 Subject: [PATCH 03/47] Attempting to get `parallel` working --- .../scala/cats/effect/std/Dispatcher.scala | 90 +++++++++++++------ 1 file changed, 63 insertions(+), 27 deletions(-) diff --git a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala index a5b595300d..e8a3f25285 100644 --- a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala +++ b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala @@ -17,14 +17,14 @@ package cats.effect.std import cats.Applicative -import cats.effect.kernel.{Async, Concurrent, MonadCancel, Outcome, Resource, Sync} +import cats.effect.kernel.{Async, Concurrent, MonadCancel, Outcome, Resource, Spawn, Sync} import cats.effect.std.Dispatcher.parasiticEC import cats.syntax.all._ import scala.concurrent.{ExecutionContext, Future, Promise} import scala.util.Failure -import java.util.concurrent.{LinkedBlockingQueue /*, ThreadLocalRandom*/} +import java.util.concurrent.{LinkedBlockingQueue, ThreadLocalRandom} import java.util.concurrent.atomic.{AtomicBoolean, AtomicReference} /** @@ -91,7 +91,7 @@ object Dispatcher { def reportFailure(t: Throwable) = t.printStackTrace() } - // private[this] val Cpus: Int = Runtime.getRuntime().availableProcessors() + private[this] val Cpus: Int = Runtime.getRuntime().availableProcessors() @deprecated( message = @@ -148,7 +148,7 @@ object Dispatcher { * - true - wait for the completion of the active fibers * - false - cancel the active fibers */ - def parallel[F[_]: Async](await: Boolean): Resource[F, Dispatcher[F]] = sequential[F](await) + def parallel[F[_]: Async](await: Boolean): Resource[F, Dispatcher[F]] = impl[F](true, await) /** * Create a [[Dispatcher]] that can be used within a resource scope. Once the resource scope @@ -184,26 +184,53 @@ object Dispatcher { * - false - cancel the active fiber */ def sequential[F[_]: Async](await: Boolean): Resource[F, Dispatcher[F]] = + impl[F](false, await) + + private[this] def impl[F[_]: Async]( + parallel: Boolean, + await: Boolean): Resource[F, Dispatcher[F]] = + // the outer supervisor is for the worker fibers + // the inner supervisor is for tasks (if parallel) and finalizers Supervisor[F]( await = await, - checkRestart = Some((_: Outcome[F, Throwable, _]) => true)) flatMap { supervisor => - // we only need this flag to raise the IllegalStateException after closure (Supervisor can't do it for us) - val termination = Resource.make(Sync[F].delay(new AtomicBoolean(false)))(doneR => - Sync[F].delay(doneR.set(true))) - - termination flatMap { doneR => - // we sequentialize on worker spawning, so we don't need to use Deferred here - Resource.eval(Concurrent[F].ref(Applicative[F].unit)) flatMap { cancelR => - // in sequential spawning, the only cancelation cancels and restarts the worker itself - def spawn(fu: F[Unit])(setupCancelation: F[Unit] => F[Unit]): F[Unit] = - cancelR.get.flatMap(setupCancelation) *> fu - - Worker[F](supervisor)(spawn) evalMap { worker => - Async[F].executionContext flatMap { ec => - supervisor - .supervise(worker.run) - .flatMap(f => cancelR.set(f.cancel)) - .as(new Dispatcher[F] { + checkRestart = Some((_: Outcome[F, Throwable, _]) => true)) flatMap { workervisor => + Supervisor[F](await = await) flatMap { supervisor => + // we only need this flag to raise the IllegalStateException after closure (Supervisor can't do it for us) + val termination = Resource.make(Sync[F].delay(new AtomicBoolean(false)))(doneR => + Sync[F].delay(doneR.set(true))) + + termination flatMap { doneR => + // we sequentialize on worker spawning, so we don't need to use Deferred here + Resource.eval(Concurrent[F].ref(Applicative[F].unit)) flatMap { cancelR => + def spawn(fu: F[Unit])(setupCancelation: F[Unit] => F[Unit]): F[Unit] = { + // TODO move this out + if (parallel) + // in parallel spawning, we have a real fiber which we need to kill off + supervisor.supervise(fu).flatMap(f => setupCancelation(f.cancel)) + else + // in sequential spawning, the only cancelation cancels and restarts the worker itself + cancelR.get.flatMap(setupCancelation) *> fu + } + + val workerF = Worker[F](supervisor)(spawn) + val workersF = + if (parallel) + workerF.replicateA(Cpus).map(_.toArray) + else + workerF.map(w => Array(w)) + + workersF evalMap { workers => + Async[F].executionContext flatMap { ec => + val launchAll = 0.until(workers.length).toList traverse_ { i => + val launch = workervisor.supervise(workers(i).run) + + if (parallel) + launch.void + else + launch.flatMap(f => cancelR.set(f.cancel)) + } + + launchAll.as(new Dispatcher[F] { def unsafeToFutureCancelable[A](fa: F[A]): (Future[A], () => Future[Unit]) = { def inner[E](fe: F[E], isFinalizer: Boolean) : (Future[E], () => Future[Unit]) = { @@ -214,20 +241,28 @@ object Dispatcher { val p = Promise[E]() // forward atomicity guarantees onto promise completion - val fu = MonadCancel[F] uncancelable { poll => + val promisory = MonadCancel[F] uncancelable { poll => poll(fe).redeemWith( e => Sync[F].delay(p.failure(e)), a => Sync[F].delay(p.success(a))) } + val worker = + if (parallel) + workers(ThreadLocalRandom.current().nextInt(Cpus)) + else + workers(0) + if (isFinalizer) { - worker.queue.unsafeOffer(Registration.Finalizer(fu.void)) + worker.queue.unsafeOffer(Registration.Finalizer(promisory.void)) // cannot cancel a cancel (p.future, () => Future.failed(new UnsupportedOperationException)) } else { val reg = - new Registration.Primary(fu.void, new AtomicReference[F[Unit]]()) + new Registration.Primary( + promisory.void, + new AtomicReference[F[Unit]]()) worker.queue.unsafeOffer(reg) def cancel(): Future[Unit] = { @@ -255,6 +290,7 @@ object Dispatcher { override def reportFailure(t: Throwable): Unit = ec.reportFailure(t) }) + } } } } @@ -273,7 +309,7 @@ object Dispatcher { } // the signal is just a skolem for the atomic references; we never actually run it - private final class Worker[F[_]: Sync]( + private final class Worker[F[_]: Async]( val queue: UnsafeAsyncQueue[F, Registration[F]], supervisor: Supervisor[F])( // only needed for cancelation spawning spawn: F[Unit] => (F[Unit] => F[Unit]) => F[Unit]) { @@ -320,7 +356,7 @@ object Dispatcher { Sync[F].delay(doneR.set(true)) } - Sync[F].delay(doneR.get()).ifM(Applicative[F].unit, step >> run) + Sync[F].delay(doneR.get()).ifM(Spawn[F].cede, step >> run) } } From 92eeb7787cdc8cedeb96ef64a1e2ac96fc906879 Mon Sep 17 00:00:00 2001 From: Daniel Spiewak Date: Sat, 9 Dec 2023 13:45:11 +0000 Subject: [PATCH 04/47] Adjusted flaky and/or definitionally-wrong tests --- .../cats/effect/std/DispatcherSpec.scala | 41 +++++++++++-------- 1 file changed, 25 insertions(+), 16 deletions(-) diff --git a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala index 02d3dfb270..2c519badaa 100644 --- a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala +++ b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala @@ -24,6 +24,8 @@ import cats.syntax.all._ import scala.concurrent.{ExecutionContext, Promise} import scala.concurrent.duration._ +import java.util.concurrent.atomic.AtomicInteger + class DispatcherSpec extends BaseSpec with DetectPlatform { override def executionTimeout = 30.seconds @@ -103,6 +105,18 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { TestControl.executeEmbed(rec.use(_ => IO(canceled must beFalse))) } + + "reject new tasks after release action is submitted as a task" in ticked { + implicit ticker => + val test = dispatcher.allocated.flatMap { + case (runner, release) => + IO(runner.unsafeRunAndForget(release)) *> + IO.sleep(100.millis) *> + IO(runner.unsafeRunAndForget(IO(ko)) must throwAn[IllegalStateException]) + } + + test.void must completeAs(()) + } } "parallel dispatcher" should { @@ -229,6 +243,14 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { .as(ok) } } + + /*"fail to terminate when running one's own release in all modes" in ticked { implicit ticker => + val test = dispatcher.allocated flatMap { + case (runner, release) => IO(runner.unsafeRunAndForget(release)) + } + + test must nonTerminate + }*/ } private def common(dispatcher: Resource[IO, Dispatcher[IO]]) = { @@ -250,9 +272,8 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { } "run several IOs back to back" in real { - @volatile - var counter = 0 - val increment = IO(counter += 1) + val counter = new AtomicInteger(0) + val increment = IO(counter.getAndIncrement()).void val num = 10 @@ -260,7 +281,7 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { Resource.eval(IO.fromFuture(IO(runner.unsafeToFuture(increment))).replicateA(num).void) } - rec.use(_ => IO(counter mustEqual num)) + rec.use(_ => IO(counter.get() mustEqual num)) } "raise an error on leaked runner" in real { @@ -369,18 +390,6 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { .replicateA(5) } - "issue 3501: reject new tasks after release action is submitted as a task" in ticked { - implicit ticker => - val test = dispatcher.allocated.flatMap { - case (runner, release) => - IO(runner.unsafeRunAndForget(release)) *> - IO.sleep(100.millis) *> - IO(runner.unsafeRunAndForget(IO(ko)) must throwAn[IllegalStateException]) - } - - test.void must completeAs(()) - } - "cancel inner awaits when canceled" in ticked { implicit ticker => val work = dispatcher.useForever val test = work.background.use(_ => IO.sleep(100.millis)) From 0fb7e3ed86277674fbd2237b9b5c8a33ff97cd58 Mon Sep 17 00:00:00 2001 From: Daniel Spiewak Date: Sat, 9 Dec 2023 15:34:52 +0000 Subject: [PATCH 05/47] Fixed late cancelation with `sequential` --- .../scala/cats/effect/std/Dispatcher.scala | 46 ++++++++++++++----- .../cats/effect/std/DispatcherSpec.scala | 26 ++++++++++- 2 files changed, 58 insertions(+), 14 deletions(-) diff --git a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala index e8a3f25285..69362d8e22 100644 --- a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala +++ b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala @@ -20,6 +20,7 @@ import cats.Applicative import cats.effect.kernel.{Async, Concurrent, MonadCancel, Outcome, Resource, Spawn, Sync} import cats.effect.std.Dispatcher.parasiticEC import cats.syntax.all._ +import cats.effect.kernel.syntax.all._ import scala.concurrent.{ExecutionContext, Future, Promise} import scala.util.Failure @@ -239,10 +240,15 @@ object Dispatcher { } val p = Promise[E]() + val cancelR = new AtomicReference[F[Unit]]() + + val invalidateCancel = + Sync[F].delay(cancelR.set(null.asInstanceOf[F[Unit]])) // forward atomicity guarantees onto promise completion val promisory = MonadCancel[F] uncancelable { poll => - poll(fe).redeemWith( + // invalidate the cancel action when we're done + poll(fe.guarantee(invalidateCancel)).redeemWith( e => Sync[F].delay(p.failure(e)), a => Sync[F].delay(p.success(a))) } @@ -254,15 +260,18 @@ object Dispatcher { workers(0) if (isFinalizer) { - worker.queue.unsafeOffer(Registration.Finalizer(promisory.void)) + // bypass over-eager warning + val abortResult: Any = () + val abort = Sync[F].delay(p.success(abortResult.asInstanceOf[E])).void + + worker + .queue + .unsafeOffer(Registration.Finalizer(promisory.void, cancelR, abort)) // cannot cancel a cancel (p.future, () => Future.failed(new UnsupportedOperationException)) } else { - val reg = - new Registration.Primary( - promisory.void, - new AtomicReference[F[Unit]]()) + val reg = new Registration.Primary(promisory.void, cancelR) worker.queue.unsafeOffer(reg) def cancel(): Future[Unit] = { @@ -270,9 +279,9 @@ object Dispatcher { // publishes action write // TODO this provides incorrect semantics for multiple cancel() call sites - val cancelF = reg - .cancelR - .getAndSet(Applicative[F].unit) // anything that isn't null, really + val cancelF = cancelR.getAndSet( + Applicative[F].unit + ) // anything that isn't null, really if (cancelF != null) // this looping is fine, since we drop the cancel inner(cancelF, true)._1 @@ -303,7 +312,15 @@ object Dispatcher { final class Primary[F[_]](var action: F[Unit], val cancelR: AtomicReference[F[Unit]]) extends Registration[F] - final case class Finalizer[F[_]](action: F[Unit]) extends Registration[F] + // the only reason for cancelR here is to double check the cancelation invalidation when the + // action is *observed* by the worker fiber, which only matters in sequential mode + // this captures the race condition where the cancel action is invoked exactly as the main + // action completes and avoids the pathological case where we accidentally cancel the worker + final case class Finalizer[F[_]]( + action: F[Unit], + cancelR: AtomicReference[F[Unit]], + abort: F[Unit]) + extends Registration[F] final case class PoisonPill[F[_]]() extends Registration[F] } @@ -349,8 +366,13 @@ object Dispatcher { } } - case Registration.Finalizer(action) => - supervisor.supervise(action).void + case Registration.Finalizer(action, cancelR, abort) => + // here's the double-check for late finalization + // if == null then the task is complete and we ignore + if (cancelR.get() != null) + supervisor.supervise(action).void + else + abort case Registration.PoisonPill() => Sync[F].delay(doneR.set(true)) diff --git a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala index 2c519badaa..fdb3d0c6ec 100644 --- a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala +++ b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala @@ -117,6 +117,28 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { test.void must completeAs(()) } + + "invalidate cancelation action of task when complete" in real { + IO.ref(false) flatMap { resultR => + val test = dispatcher use { runner => + for { + latch1 <- IO.deferred[Unit] + latch2 <- IO.deferred[Unit] + + (_, cancel) <- IO(runner.unsafeToFutureCancelable(IO.unit)) + _ <- IO( + runner.unsafeRunAndForget(latch1.complete(()) *> latch2.get *> resultR.set(true))) + + _ <- latch1.get + _ <- IO.fromFuture(IO(cancel())) + _ <- latch2.complete(()) + } yield ok + } + + // if it was canceled, it will be false + (test *> resultR.get).flatMap(b => IO(b must beTrue)) + } + } } "parallel dispatcher" should { @@ -244,12 +266,12 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { } } - /*"fail to terminate when running one's own release in all modes" in ticked { implicit ticker => + /*"fail to terminate when running one's own release in all modes" in real { val test = dispatcher.allocated flatMap { case (runner, release) => IO(runner.unsafeRunAndForget(release)) } - test must nonTerminate + TestControl.executeEmbed(test).attempt.flatMap(e => IO(e must beLeft)) }*/ } From 40156e0c173b9147a106d85e74e1187a27a2c282 Mon Sep 17 00:00:00 2001 From: Daniel Spiewak Date: Wed, 20 Dec 2023 15:20:15 -0600 Subject: [PATCH 06/47] Trying really hard to make it work... --- .../scala/cats/effect/std/Dispatcher.scala | 96 ++++++++++--------- .../cats/effect/std/DispatcherSpec.scala | 88 ++++++++++------- 2 files changed, 106 insertions(+), 78 deletions(-) diff --git a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala index 69362d8e22..b63345697e 100644 --- a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala +++ b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala @@ -233,7 +233,7 @@ object Dispatcher { launchAll.as(new Dispatcher[F] { def unsafeToFutureCancelable[A](fa: F[A]): (Future[A], () => Future[Unit]) = { - def inner[E](fe: F[E], isFinalizer: Boolean) + def inner[E](fe: F[E], checker: Option[F[Boolean]]) : (Future[E], () => Future[Unit]) = { if (doneR.get()) { throw new IllegalStateException("Dispatcher already closed") @@ -241,9 +241,7 @@ object Dispatcher { val p = Promise[E]() val cancelR = new AtomicReference[F[Unit]]() - - val invalidateCancel = - Sync[F].delay(cancelR.set(null.asInstanceOf[F[Unit]])) + val invalidateCancel = Sync[F].delay(cancelR.set(null.asInstanceOf[F[Unit]])) // forward atomicity guarantees onto promise completion val promisory = MonadCancel[F] uncancelable { poll => @@ -259,41 +257,53 @@ object Dispatcher { else workers(0) - if (isFinalizer) { - // bypass over-eager warning - val abortResult: Any = () - val abort = Sync[F].delay(p.success(abortResult.asInstanceOf[E])).void - - worker - .queue - .unsafeOffer(Registration.Finalizer(promisory.void, cancelR, abort)) - - // cannot cancel a cancel - (p.future, () => Future.failed(new UnsupportedOperationException)) - } else { - val reg = new Registration.Primary(promisory.void, cancelR) - worker.queue.unsafeOffer(reg) - - def cancel(): Future[Unit] = { - reg.action = null.asInstanceOf[F[Unit]] - - // publishes action write - // TODO this provides incorrect semantics for multiple cancel() call sites - val cancelF = cancelR.getAndSet( - Applicative[F].unit - ) // anything that isn't null, really - if (cancelF != null) - // this looping is fine, since we drop the cancel - inner(cancelF, true)._1 - else - Future.successful(()) - } - - (p.future, cancel _) + checker match { + // fe is a finalizer + case Some(check) => + println("we found the loop") + + // bypass over-eager warning + // can get rid of this if we GADT encode `inner`'s parameters + val abortResult: Any = () + val abort = Sync[F].delay(p.success(abortResult.asInstanceOf[E])).void + + worker + .queue + .unsafeOffer(Registration.Finalizer(promisory.void, check, abort)) + + // cannot cancel a cancel + (p.future, () => Future.failed(new UnsupportedOperationException)) + + case None => + val reg = new Registration.Primary(promisory.void, cancelR) + worker.queue.unsafeOffer(reg) + + def cancel(): Future[Unit] = { + println("starting to cancel") + reg.action = null.asInstanceOf[F[Unit]] + + val cancelF = cancelR.get() + if (cancelF != null) { + println("cancel token is still valid") + // cannot use null here + if (cancelR.compareAndSet(cancelF, Applicative[F].unit)) { + println("looping on cancel action") + // this looping is fine, since we drop the cancel + // note that action is published here since the CAS passed + inner(cancelF, Some(Sync[F].delay(cancelR.get() != null)))._1 + } else { + Future.successful(()) + } + } else { + Future.successful(()) + } + } + + (p.future, cancel _) } } - inner(fa, false) + inner(fa, None) } override def reportFailure(t: Throwable): Unit = @@ -316,10 +326,7 @@ object Dispatcher { // action is *observed* by the worker fiber, which only matters in sequential mode // this captures the race condition where the cancel action is invoked exactly as the main // action completes and avoids the pathological case where we accidentally cancel the worker - final case class Finalizer[F[_]]( - action: F[Unit], - cancelR: AtomicReference[F[Unit]], - abort: F[Unit]) + final case class Finalizer[F[_]](action: F[Unit], check: F[Boolean], abort: F[Unit]) extends Registration[F] final case class PoisonPill[F[_]]() extends Registration[F] @@ -366,13 +373,10 @@ object Dispatcher { } } - case Registration.Finalizer(action, cancelR, abort) => + case Registration.Finalizer(action, check, abort) => + println("we got the registration") // here's the double-check for late finalization - // if == null then the task is complete and we ignore - if (cancelR.get() != null) - supervisor.supervise(action).void - else - abort + check.ifM({println("check was true"); supervisor.supervise(action).void}, {println("check was false"); abort}) case Registration.PoisonPill() => Sync[F].delay(doneR.set(true)) diff --git a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala index fdb3d0c6ec..e700d1144c 100644 --- a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala +++ b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala @@ -45,7 +45,7 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { } } - "await = false" >> { + /*"await = false" >> { val D = Dispatcher.sequential[IO](await = false) sequential(D) @@ -61,7 +61,7 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { TestControl.executeEmbed(action *> IO(canceled must beTrue)) } - } + }*/ } private def sequential(dispatcher: Resource[IO, Dispatcher[IO]]) = { @@ -119,29 +119,55 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { } "invalidate cancelation action of task when complete" in real { - IO.ref(false) flatMap { resultR => - val test = dispatcher use { runner => + val test = dispatcher use { runner => + for { + latch1 <- IO.deferred[Unit] + latch2 <- IO.deferred[Unit] + latch3 <- IO.deferred[Unit] + + (_, cancel) <- IO(runner.unsafeToFutureCancelable(IO.unit)) + _ <- IO( + runner.unsafeRunAndForget(latch1.complete(()) *> latch2.get *> latch3.complete(()))) + + _ <- latch1.get + _ <- IO.fromFuture(IO(cancel())) + _ <- latch2.complete(()) + + _ <- latch3.get // this will hang if the test is failing + } yield ok + } + + test.parReplicateA_(1000).as(ok) + } + + "invalidate cancelation action when racing with task" in real { + val test = dispatcher use { runner => + IO.ref(false) flatMap { resultR => for { latch1 <- IO.deferred[Unit] latch2 <- IO.deferred[Unit] - (_, cancel) <- IO(runner.unsafeToFutureCancelable(IO.unit)) - _ <- IO( - runner.unsafeRunAndForget(latch1.complete(()) *> latch2.get *> resultR.set(true))) + (_, cancel) <- IO(runner.unsafeToFutureCancelable(latch1.get)) - _ <- latch1.get - _ <- IO.fromFuture(IO(cancel())) - _ <- latch2.complete(()) - } yield ok - } + _ <- latch1.complete(()) + // the particularly scary case is where the cancel action gets in queue before the next action + f <- IO(cancel()) + + // we're testing to make sure this task runs and isn't canceled + _ <- IO(runner.unsafeRunAndForget(resultR.set(true) *> latch2.complete(()))) + _ <- IO.fromFuture(IO.pure(f)) + _ <- latch2.get - // if it was canceled, it will be false - (test *> resultR.get).flatMap(b => IO(b must beTrue)) + b <- resultR.get + } yield b + } } + + test.flatMap(b => IO(b must beTrue)).parReplicateA_(1000).as(ok) } } - "parallel dispatcher" should { + /*"parallel dispatcher" should { "await = true" >> { val D = Dispatcher.parallel[IO](await = true) @@ -189,7 +215,7 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { } yield ok } } - } + }*/ private def parallel(dispatcher: Resource[IO, Dispatcher[IO]]) = { @@ -237,22 +263,6 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { } yield ok } - "forward cancelation onto the inner action" in real { - var canceled = false - - val rec = dispatcher flatMap { runner => - val run = IO { - runner.unsafeToFutureCancelable(IO.never.onCancel(IO { canceled = true }))._2 - } - - Resource eval { - run.flatMap(ct => IO.sleep(500.millis) >> IO.fromFuture(IO(ct()))) - } - } - - TestControl.executeEmbed(rec.use(_ => IO(canceled must beTrue))) - } - // https://github.com/typelevel/cats-effect/issues/3898 "not hang when cancelling" in real { dispatcher.use { dispatcher => @@ -418,6 +428,20 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { test must completeAs(()) } + + "forward cancelation onto the inner action" in real { + val test = dispatcher use { runner => + IO.ref(false) flatMap { resultsR => + val action = IO.never.onCancel(resultsR.set(true)) + IO(runner.unsafeToFutureCancelable(action)) flatMap { + case (_, cancel) => + IO.sleep(500.millis) *> IO.fromFuture(IO(cancel())) *> resultsR.get + } + } + } + + TestControl.executeEmbed(test).flatMap(b => IO(b must beTrue)) + } } private def awaitTermination(dispatcher: Resource[IO, Dispatcher[IO]]) = { From 89010293e945632fbaed8ee86c288e193e882dc3 Mon Sep 17 00:00:00 2001 From: Daniel Spiewak Date: Thu, 28 Dec 2023 15:20:42 -0600 Subject: [PATCH 07/47] Reimplemented part of Dispatcher some more --- .../scala/cats/effect/std/Dispatcher.scala | 218 +++++++++++++----- .../cats/effect/std/DispatcherSpec.scala | 82 +++---- 2 files changed, 197 insertions(+), 103 deletions(-) diff --git a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala index b63345697e..b69c71b880 100644 --- a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala +++ b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala @@ -17,10 +17,10 @@ package cats.effect.std import cats.Applicative -import cats.effect.kernel.{Async, Concurrent, MonadCancel, Outcome, Resource, Spawn, Sync} +import cats.effect.kernel.{Async, Concurrent, Deferred, MonadCancel, Outcome, Ref, Resource, Spawn, Sync} +import cats.effect.kernel.syntax.all._ import cats.effect.std.Dispatcher.parasiticEC import cats.syntax.all._ -import cats.effect.kernel.syntax.all._ import scala.concurrent.{ExecutionContext, Future, Promise} import scala.util.Failure @@ -187,6 +187,46 @@ object Dispatcher { def sequential[F[_]: Async](await: Boolean): Resource[F, Dispatcher[F]] = impl[F](false, await) + /* + * There are two fundamental modes here: sequential and parallel. There is very little overlap + * in semantics between the two apart from the submission side. The whole thing is split up into + * a submission queue with impure enqueue and cancel functions which is drained by the `Worker` and an + * internal execution protocol which also involves a queue. The `Worker` encapsulates all of the + * race conditions and negotiations with impure code, while the `Executor` manages running the + * tasks with appropriate semantics. In parallel mode, we shard the `Worker`s according to the + * number of CPUs and select a random queue (in impure code) as a target. This reduces contention + * at the cost of ordering, which is not guaranteed in parallel mode. With sequential mode, there + * is only a single worker. + * + * On the impure side, the queue bit is the easy part: it's just a `LinkedBlockingQueue` which + * accepts Registration(s). It's easiest to think of this a bit like an actor model, where the + * `Worker` is the actor and the enqueue is the send. Whenever we send a unit of work, that + * message has an `AtomicReference` which allows us to back-propagate a cancelation action. That + * cancelation action can be used in impure code by sending it back to us using the Finalizer + * message. There are certain race conditions involved in canceling work on the queue and work + * which is in the process of being taken off the queue, and those race conditions are negotiated + * between the impure code and the `Worker`. + * + * On the pure side, the two different `Executor`s are very distinct. In parallel mode, it's easy: + * we have a separate `Supervisor` which doesn't respawn actions, and we use that supervisor to + * spawn a new fiber for each task unit. Cancelation in this mode is easy: we just cancel the fiber. + * For sequential mode, we spawn a *single* executor fiber on the main supervisor (which respawns). + * This fiber is paired with a pure unbounded queue and a shutoff latch. New work is placed on the + * queue, which the fiber takes from in order and executes in-place. If the work self-cancels or + * errors, the executor will be restarted. In the case of external cancelation, we shut off the + * latch (to hold new work), drain the entire work queue into a scratch space, then cancel the + * executor fiber in-place so long as we're sure it's actively working on the target task. Once + * that cancelation completes (which will ultimately restart the executor fiber), we re-fill the + * queue and unlock the latch to allow new work (from the `Worker`). + * + * Note that a third mode is possible but not implemented: sequential *without* cancelation. In + * this mode, we execute each task directly on the worker fiber as it comes in, without the + * added indirection of the executor queue. This reduces overhead considerably, but the price is + * we can no longer support external (impure) cancelation. This is because the worker fiber is + * *also* responsible for dequeueing from the impure queue, which is where the cancelation tasks + * come in. The worker can't observe a cancelation action while it's executing another action, so + * cancelation cannot then preempt and is effectively worthless. + */ private[this] def impl[F[_]: Async]( parallel: Boolean, await: Boolean): Resource[F, Dispatcher[F]] = @@ -194,26 +234,21 @@ object Dispatcher { // the inner supervisor is for tasks (if parallel) and finalizers Supervisor[F]( await = await, - checkRestart = Some((_: Outcome[F, Throwable, _]) => true)) flatMap { workervisor => - Supervisor[F](await = await) flatMap { supervisor => + checkRestart = Some((_: Outcome[F, Throwable, _]) => true)) flatMap { supervisor => // we only need this flag to raise the IllegalStateException after closure (Supervisor can't do it for us) val termination = Resource.make(Sync[F].delay(new AtomicBoolean(false)))(doneR => Sync[F].delay(doneR.set(true))) termination flatMap { doneR => - // we sequentialize on worker spawning, so we don't need to use Deferred here - Resource.eval(Concurrent[F].ref(Applicative[F].unit)) flatMap { cancelR => - def spawn(fu: F[Unit])(setupCancelation: F[Unit] => F[Unit]): F[Unit] = { - // TODO move this out - if (parallel) - // in parallel spawning, we have a real fiber which we need to kill off - supervisor.supervise(fu).flatMap(f => setupCancelation(f.cancel)) - else - // in sequential spawning, the only cancelation cancels and restarts the worker itself - cancelR.get.flatMap(setupCancelation) *> fu - } + val executorF: Resource[F, Executor[F]] = if (parallel) + Executor.parallel[F](await) + else + Resource.pure(Executor.inplace[F]) - val workerF = Worker[F](supervisor)(spawn) + // note this scopes the executors *outside* the workers, meaning the workers shut down first + // I think this is what we want, since it avoids enqueue race conditions + executorF flatMap { executor => + val workerF = Worker[F](executor) val workersF = if (parallel) workerF.replicateA(Cpus).map(_.toArray) @@ -223,12 +258,7 @@ object Dispatcher { workersF evalMap { workers => Async[F].executionContext flatMap { ec => val launchAll = 0.until(workers.length).toList traverse_ { i => - val launch = workervisor.supervise(workers(i).run) - - if (parallel) - launch.void - else - launch.flatMap(f => cancelR.set(f.cancel)) + supervisor.supervise(workers(i).run).void } launchAll.as(new Dispatcher[F] { @@ -260,8 +290,6 @@ object Dispatcher { checker match { // fe is a finalizer case Some(check) => - println("we found the loop") - // bypass over-eager warning // can get rid of this if we GADT encode `inner`'s parameters val abortResult: Any = () @@ -279,15 +307,12 @@ object Dispatcher { worker.queue.unsafeOffer(reg) def cancel(): Future[Unit] = { - println("starting to cancel") reg.action = null.asInstanceOf[F[Unit]] val cancelF = cancelR.get() if (cancelF != null) { - println("cancel token is still valid") // cannot use null here if (cancelR.compareAndSet(cancelF, Applicative[F].unit)) { - println("looping on cancel action") // this looping is fine, since we drop the cancel // note that action is published here since the CAS passed inner(cancelF, Some(Sync[F].delay(cancelR.get() != null)))._1 @@ -313,7 +338,6 @@ object Dispatcher { } } } - } } private sealed abstract class Registration[F[_]] @@ -322,10 +346,8 @@ object Dispatcher { final class Primary[F[_]](var action: F[Unit], val cancelR: AtomicReference[F[Unit]]) extends Registration[F] - // the only reason for cancelR here is to double check the cancelation invalidation when the - // action is *observed* by the worker fiber, which only matters in sequential mode - // this captures the race condition where the cancel action is invoked exactly as the main - // action completes and avoids the pathological case where we accidentally cancel the worker + // the check action verifies that we haven't completed in the interim (as determined by the promise) + // the abort action runs in the event that we fail that check and need to complete the cancel promise final case class Finalizer[F[_]](action: F[Unit], check: F[Boolean], abort: F[Unit]) extends Registration[F] @@ -335,36 +357,28 @@ object Dispatcher { // the signal is just a skolem for the atomic references; we never actually run it private final class Worker[F[_]: Async]( val queue: UnsafeAsyncQueue[F, Registration[F]], - supervisor: Supervisor[F])( // only needed for cancelation spawning - spawn: F[Unit] => (F[Unit] => F[Unit]) => F[Unit]) { + executor: Executor[F]) { private[this] val doneR = new AtomicBoolean(false) def run: F[Unit] = { val step = queue.take flatMap { case reg: Registration.Primary[F] => - // println("got registration") - Sync[F].delay(reg.cancelR.get()) flatMap { sig => val action = reg.action // double null check catches overly-aggressive memory fencing if (sig == null && action != null) { - // println("...it wasn't canceled") - - spawn(action) { cancelF => - // println("...spawned") - - Sync[F].delay( - reg.cancelR.compareAndSet(null.asInstanceOf[F[Unit]], cancelF)) flatMap { - check => - // we need to double-check that we didn't lose the race - if (check) - // don't cancel, already spawned - Applicative[F].unit - else - // cancel spawn (we lost the race) - cancelF + executor(action) { cancelF => + // we need to double-check that we weren't canceled while spawning + Sync[F].delay(reg.cancelR.compareAndSet(null.asInstanceOf[F[Unit]], cancelF)) flatMap { + case true => + // we weren't canceled! + Applicative[F].unit + + case false => + // we were canceled while spawning, so forward that on to the cancelation action + cancelF } } } else { @@ -374,9 +388,8 @@ object Dispatcher { } case Registration.Finalizer(action, check, abort) => - println("we got the registration") // here's the double-check for late finalization - check.ifM({println("check was true"); supervisor.supervise(action).void}, {println("check was false"); abort}) + check.ifM(action, abort) case Registration.PoisonPill() => Sync[F].delay(doneR.set(true)) @@ -388,16 +401,107 @@ object Dispatcher { private object Worker { - def apply[F[_]: Async](supervisor: Supervisor[F])( - spawn: F[Unit] => (F[Unit] => F[Unit]) => F[Unit]): Resource[F, Worker[F]] = { - + def apply[F[_]: Async](executor: Executor[F]): Resource[F, Worker[F]] = { val initF = Sync[F].delay( - new Worker[F](new UnsafeAsyncQueue[F, Registration[F]](), supervisor)(spawn)) + new Worker[F](new UnsafeAsyncQueue[F, Registration[F]](), executor)) Resource.make(initF)(w => Sync[F].delay(w.queue.unsafeOffer(Registration.PoisonPill()))) } } + private abstract class Executor[F[_]] { + def apply(task: F[Unit])(registerCancel: F[Unit] => F[Unit]): F[Unit] + } + + private object Executor { + + def inplace[F[_]]: Executor[F] = + new Executor[F] { + def apply(task: F[Unit])(registerCancel: F[Unit] => F[Unit]): F[Unit] = task + } + + // sequential executor which respects cancelation (at the cost of additional overhead); not used + def sequential[F[_]: Concurrent](supervisor: Supervisor[F]): Resource[F, Executor[F]] = { + sealed trait TaskSignal extends Product with Serializable + + object TaskSignal { + final case class Ready(task: F[Unit]) extends TaskSignal + case object Executing extends TaskSignal + case object Void extends TaskSignal + } + + Resource.eval(Queue.unbounded[F, Ref[F, TaskSignal]]) flatMap { tasks => + // knock it out of the task taking + val evict = Concurrent[F].ref[TaskSignal](TaskSignal.Void).flatMap(tasks.offer(_)) + + Resource.make(Concurrent[F].ref(false))(r => r.set(true) >> evict) evalMap { doneR => + Concurrent[F].ref[Option[Deferred[F, Unit]]](None) flatMap { shutoff => + val step = tasks.take flatMap { taskR => + taskR.getAndSet(TaskSignal.Executing) flatMap { + case TaskSignal.Ready(task) => task.guarantee(taskR.set(TaskSignal.Void)) + // Executing should be impossible + case TaskSignal.Executing | TaskSignal.Void => Applicative[F].unit + } + } + + lazy val loop: F[Unit] = doneR.get.ifM(Applicative[F].unit, step >> loop) + val spawnExecutor = supervisor.supervise(loop) + + spawnExecutor flatMap { fiber => + Concurrent[F].ref(fiber) map { fiberR => + new Executor[F] { + def apply(task: F[Unit])(registerCancel: F[Unit] => F[Unit]): F[Unit] = { + Concurrent[F].ref[TaskSignal](TaskSignal.Ready(task)) flatMap { taskR => + val cancelF = taskR.getAndSet(TaskSignal.Void) flatMap { + case TaskSignal.Ready(_) | TaskSignal.Void => + Applicative[F].unit + + case TaskSignal.Executing => + Concurrent[F].deferred[Unit] flatMap { latch => + // TODO if someone else is already canceling, this will create a deadlock + // to fix this deadlock, we need a fourth TaskSignal state and a double-check on that and the shutoff + + // Step 1: Turn off everything + shutoff.set(Some(latch)) >> { + // Step 2: Drain the queue + tasks.tryTakeN(None) flatMap { scratch => + // Step 3: Cancel the executor, put it all back, and restart executor + fiberR.get.flatMap(_.cancel) >> + scratch.traverse_(tasks.offer(_)) >> + spawnExecutor.flatMap(fiberR.set(_)) >> + latch.complete(()) >> + shutoff.set(None) + } + } + } + } + + // in rare cases, this can create mutual ordering issues with quickly enqueued tasks + val optBlock = shutoff.get flatMap { + case Some(latch) => latch.get + case None => Applicative[F].unit + } + + optBlock >> tasks.offer(taskR) >> registerCancel(cancelF) + } + } + } + } + } + } + } + } + } + + def parallel[F[_]: Concurrent](await: Boolean): Resource[F, Executor[F]] = + Supervisor[F](await = await) map { supervisor => + new Executor[F] { + def apply(task: F[Unit])(registerCancel: F[Unit] => F[Unit]): F[Unit] = + supervisor.supervise(task).flatMap(fiber => registerCancel(fiber.cancel)) + } + } + } + private val Signal: Either[Any, Unit] => Unit = _ => () private val RightUnit: Right[Nothing, Unit] = Right(()) diff --git a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala index e700d1144c..21703dcc4c 100644 --- a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala +++ b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala @@ -45,7 +45,7 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { } } - /*"await = false" >> { + "await = false" >> { val D = Dispatcher.sequential[IO](await = false) sequential(D) @@ -61,7 +61,7 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { TestControl.executeEmbed(action *> IO(canceled must beTrue)) } - }*/ + } } private def sequential(dispatcher: Resource[IO, Dispatcher[IO]]) = { @@ -133,7 +133,7 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { _ <- IO.fromFuture(IO(cancel())) _ <- latch2.complete(()) - _ <- latch3.get // this will hang if the test is failing + _ <- latch3.get // this will hang if the test is failing } yield ok } @@ -167,7 +167,7 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { } } - /*"parallel dispatcher" should { + "parallel dispatcher" should { "await = true" >> { val D = Dispatcher.parallel[IO](await = true) @@ -215,7 +215,7 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { } yield ok } } - }*/ + } private def parallel(dispatcher: Resource[IO, Dispatcher[IO]]) = { @@ -283,6 +283,20 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { TestControl.executeEmbed(test).attempt.flatMap(e => IO(e must beLeft)) }*/ + + "forward cancelation onto the inner action" in real { + val test = dispatcher use { runner => + IO.ref(false) flatMap { resultsR => + val action = IO.never.onCancel(resultsR.set(true)) + IO(runner.unsafeToFutureCancelable(action)) flatMap { + case (_, cancel) => + IO.sleep(500.millis) *> IO.fromFuture(IO(cancel())) *> resultsR.get + } + } + } + + TestControl.executeEmbed(test).flatMap(b => IO(b must beTrue)) + } } private def common(dispatcher: Resource[IO, Dispatcher[IO]]) = { @@ -392,34 +406,24 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { } "reject new tasks while shutting down" in real { - (IO.ref(false), IO.ref(false)) - .flatMapN { (resultR, rogueResultR) => - dispatcher - .allocated - .flatMap { - case (runner, release) => - IO(runner.unsafeRunAndForget( - IO.sleep(1.second).uncancelable.guarantee(resultR.set(true)))) *> - IO.sleep(100.millis) *> - release.both( - IO.sleep(500.nanos) *> - IO(runner.unsafeRunAndForget(rogueResultR.set(true))).attempt - ) - } - .flatMap { - case (_, rogueSubmitResult) => - for { - result <- resultR.get - rogueResult <- rogueResultR.get - _ <- IO(result must beTrue) - _ <- IO(if (rogueResult == false) { - // if the rogue task is not completed then we must have failed to submit it - rogueSubmitResult must beLeft - }) - } yield ok - } + val test = (IO.deferred[Unit], IO.deferred[Unit]) flatMapN { (latch1, latch2) => + dispatcher.allocated flatMap { + case (runner, release) => + for { + _ <- IO( + runner.unsafeRunAndForget(IO.unit.guarantee(latch1.complete(()) >> latch2.get))) + _ <- latch1.get + + challenge = IO(runner.unsafeRunAndForget(IO.unit)) + .delayBy(500.millis) // gross sleep to make sure we're actually in the release + .guarantee(latch2.complete(()).void) + + _ <- release &> challenge + } yield ko } - .replicateA(5) + } + + test.attempt.flatMap(r => IO(r must beLeft)).parReplicateA_(50).as(ok) } "cancel inner awaits when canceled" in ticked { implicit ticker => @@ -428,20 +432,6 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { test must completeAs(()) } - - "forward cancelation onto the inner action" in real { - val test = dispatcher use { runner => - IO.ref(false) flatMap { resultsR => - val action = IO.never.onCancel(resultsR.set(true)) - IO(runner.unsafeToFutureCancelable(action)) flatMap { - case (_, cancel) => - IO.sleep(500.millis) *> IO.fromFuture(IO(cancel())) *> resultsR.get - } - } - } - - TestControl.executeEmbed(test).flatMap(b => IO(b must beTrue)) - } } private def awaitTermination(dispatcher: Resource[IO, Dispatcher[IO]]) = { From cda691b9f7b1be26d1f285a36fa3172300bc2a22 Mon Sep 17 00:00:00 2001 From: Daniel Spiewak Date: Thu, 28 Dec 2023 16:58:15 -0600 Subject: [PATCH 08/47] Made cancelable sequential accessible and fixed `cede` loop --- .../scala/cats/effect/std/Dispatcher.scala | 210 +++++++++--------- .../cats/effect/std/DispatcherSpec.scala | 126 +++++++---- 2 files changed, 193 insertions(+), 143 deletions(-) diff --git a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala index b69c71b880..afc922a347 100644 --- a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala +++ b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala @@ -149,7 +149,7 @@ object Dispatcher { * - true - wait for the completion of the active fibers * - false - cancel the active fibers */ - def parallel[F[_]: Async](await: Boolean): Resource[F, Dispatcher[F]] = impl[F](true, await) + def parallel[F[_]: Async](await: Boolean): Resource[F, Dispatcher[F]] = impl[F](true, await, true) /** * Create a [[Dispatcher]] that can be used within a resource scope. Once the resource scope @@ -184,8 +184,8 @@ object Dispatcher { * - true - wait for the completion of the active fiber * - false - cancel the active fiber */ - def sequential[F[_]: Async](await: Boolean): Resource[F, Dispatcher[F]] = - impl[F](false, await) + def sequential[F[_]: Async](await: Boolean, cancelable: Boolean = false): Resource[F, Dispatcher[F]] = + impl[F](false, await, cancelable) /* * There are two fundamental modes here: sequential and parallel. There is very little overlap @@ -229,116 +229,123 @@ object Dispatcher { */ private[this] def impl[F[_]: Async]( parallel: Boolean, - await: Boolean): Resource[F, Dispatcher[F]] = + await: Boolean, + cancelable: Boolean): Resource[F, Dispatcher[F]] = { + val always = Some((_: Outcome[F, Throwable, _]) => true) + // the outer supervisor is for the worker fibers // the inner supervisor is for tasks (if parallel) and finalizers - Supervisor[F]( - await = await, - checkRestart = Some((_: Outcome[F, Throwable, _]) => true)) flatMap { supervisor => - // we only need this flag to raise the IllegalStateException after closure (Supervisor can't do it for us) - val termination = Resource.make(Sync[F].delay(new AtomicBoolean(false)))(doneR => - Sync[F].delay(doneR.set(true))) - - termination flatMap { doneR => - val executorF: Resource[F, Executor[F]] = if (parallel) - Executor.parallel[F](await) - else - Resource.pure(Executor.inplace[F]) - - // note this scopes the executors *outside* the workers, meaning the workers shut down first - // I think this is what we want, since it avoids enqueue race conditions - executorF flatMap { executor => - val workerF = Worker[F](executor) - val workersF = - if (parallel) - workerF.replicateA(Cpus).map(_.toArray) - else - workerF.map(w => Array(w)) - - workersF evalMap { workers => - Async[F].executionContext flatMap { ec => - val launchAll = 0.until(workers.length).toList traverse_ { i => - supervisor.supervise(workers(i).run).void - } + Supervisor[F](await = await, checkRestart = always) flatMap { supervisor => + // we only need this flag to raise the IllegalStateException after closure (Supervisor can't do it for us) + + val termination = Resource.make(Sync[F].delay(new AtomicBoolean(false)))(doneR => + Sync[F].delay(doneR.set(true))) + + val awaitTermination = Resource.make(Concurrent[F].deferred[Unit])(_.complete(()).void) + + (awaitTermination, termination) flatMapN { (terminationLatch, doneR) => + val executorF = if (parallel) + Executor.parallel[F](await) + else if (cancelable) + Executor.sequential(supervisor) + else + Resource.pure[F, Executor[F]](Executor.inplace[F]) + + // note this scopes the executors *outside* the workers, meaning the workers shut down first + // I think this is what we want, since it avoids enqueue race conditions + executorF flatMap { executor => + val workerF = Worker[F](executor, terminationLatch) + val workersF = + if (parallel) + workerF.replicateA(Cpus).map(_.toArray) + else + workerF.map(w => Array(w)) + + workersF evalMap { workers => + Async[F].executionContext flatMap { ec => + val launchAll = 0.until(workers.length).toList traverse_ { i => + supervisor.supervise(workers(i).run).void + } - launchAll.as(new Dispatcher[F] { - def unsafeToFutureCancelable[A](fa: F[A]): (Future[A], () => Future[Unit]) = { - def inner[E](fe: F[E], checker: Option[F[Boolean]]) - : (Future[E], () => Future[Unit]) = { - if (doneR.get()) { - throw new IllegalStateException("Dispatcher already closed") - } + launchAll.as(new Dispatcher[F] { + def unsafeToFutureCancelable[A](fa: F[A]): (Future[A], () => Future[Unit]) = { + def inner[E](fe: F[E], checker: Option[F[Boolean]]) + : (Future[E], () => Future[Unit]) = { + if (doneR.get()) { + throw new IllegalStateException("Dispatcher already closed") + } - val p = Promise[E]() - val cancelR = new AtomicReference[F[Unit]]() - val invalidateCancel = Sync[F].delay(cancelR.set(null.asInstanceOf[F[Unit]])) + val p = Promise[E]() + val cancelR = new AtomicReference[F[Unit]]() + val invalidateCancel = Sync[F].delay(cancelR.set(null.asInstanceOf[F[Unit]])) - // forward atomicity guarantees onto promise completion - val promisory = MonadCancel[F] uncancelable { poll => - // invalidate the cancel action when we're done - poll(fe.guarantee(invalidateCancel)).redeemWith( - e => Sync[F].delay(p.failure(e)), - a => Sync[F].delay(p.success(a))) - } + // forward atomicity guarantees onto promise completion + val promisory = MonadCancel[F] uncancelable { poll => + // invalidate the cancel action when we're done + poll(fe.guarantee(invalidateCancel)).redeemWith( + e => Sync[F].delay(p.failure(e)), + a => Sync[F].delay(p.success(a))) + } - val worker = - if (parallel) - workers(ThreadLocalRandom.current().nextInt(Cpus)) - else - workers(0) - - checker match { - // fe is a finalizer - case Some(check) => - // bypass over-eager warning - // can get rid of this if we GADT encode `inner`'s parameters - val abortResult: Any = () - val abort = Sync[F].delay(p.success(abortResult.asInstanceOf[E])).void - - worker - .queue - .unsafeOffer(Registration.Finalizer(promisory.void, check, abort)) - - // cannot cancel a cancel - (p.future, () => Future.failed(new UnsupportedOperationException)) - - case None => - val reg = new Registration.Primary(promisory.void, cancelR) - worker.queue.unsafeOffer(reg) - - def cancel(): Future[Unit] = { - reg.action = null.asInstanceOf[F[Unit]] - - val cancelF = cancelR.get() - if (cancelF != null) { - // cannot use null here - if (cancelR.compareAndSet(cancelF, Applicative[F].unit)) { - // this looping is fine, since we drop the cancel - // note that action is published here since the CAS passed - inner(cancelF, Some(Sync[F].delay(cancelR.get() != null)))._1 - } else { - Future.successful(()) - } + val worker = + if (parallel) + workers(ThreadLocalRandom.current().nextInt(Cpus)) + else + workers(0) + + checker match { + // fe is a finalizer + case Some(check) => + // bypass over-eager warning + // can get rid of this if we GADT encode `inner`'s parameters + val abortResult: Any = () + val abort = Sync[F].delay(p.success(abortResult.asInstanceOf[E])).void + + worker + .queue + .unsafeOffer(Registration.Finalizer(promisory.void, check, abort)) + + // cannot cancel a cancel + (p.future, () => Future.failed(new UnsupportedOperationException)) + + case None => + val reg = new Registration.Primary(promisory.void, cancelR) + worker.queue.unsafeOffer(reg) + + def cancel(): Future[Unit] = { + reg.action = null.asInstanceOf[F[Unit]] + + val cancelF = cancelR.get() + if (cancelF != null) { + // cannot use null here + if (cancelR.compareAndSet(cancelF, Applicative[F].unit)) { + // this looping is fine, since we drop the cancel + // note that action is published here since the CAS passed + inner(cancelF, Some(Sync[F].delay(cancelR.get() != null)))._1 } else { Future.successful(()) } + } else { + Future.successful(()) } + } - (p.future, cancel _) - } + (p.future, cancel _) } - - inner(fa, None) } - override def reportFailure(t: Throwable): Unit = - ec.reportFailure(t) - }) - } + inner(fa, None) + } + + override def reportFailure(t: Throwable): Unit = + ec.reportFailure(t) + }) } } } + } } + } private sealed abstract class Registration[F[_]] @@ -357,7 +364,8 @@ object Dispatcher { // the signal is just a skolem for the atomic references; we never actually run it private final class Worker[F[_]: Async]( val queue: UnsafeAsyncQueue[F, Registration[F]], - executor: Executor[F]) { + executor: Executor[F], + terminationLatch: Deferred[F, Unit]) { private[this] val doneR = new AtomicBoolean(false) @@ -395,15 +403,19 @@ object Dispatcher { Sync[F].delay(doneR.set(true)) } - Sync[F].delay(doneR.get()).ifM(Spawn[F].cede, step >> run) + // we're poisoned *first* but our supervisor is killed *last* + // when this happens, we just block on the termination latch to + // avoid weirdness. there's still a small gap even then, so we + // toss in a cede to avoid starvation pathologies + Sync[F].delay(doneR.get()).ifM(terminationLatch.get >> Spawn[F].cede, step >> run) } } private object Worker { - def apply[F[_]: Async](executor: Executor[F]): Resource[F, Worker[F]] = { + def apply[F[_]: Async](executor: Executor[F], terminationLatch: Deferred[F, Unit]): Resource[F, Worker[F]] = { val initF = Sync[F].delay( - new Worker[F](new UnsafeAsyncQueue[F, Registration[F]](), executor)) + new Worker[F](new UnsafeAsyncQueue[F, Registration[F]](), executor, terminationLatch)) Resource.make(initF)(w => Sync[F].delay(w.queue.unsafeOffer(Registration.PoisonPill()))) } diff --git a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala index 21703dcc4c..bd4d871e7d 100644 --- a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala +++ b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala @@ -30,11 +30,11 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { override def executionTimeout = 30.seconds - "sequential dispatcher" should { + "sequential dispatcher (cancelable = false)" should { "await = true" >> { val D = Dispatcher.sequential[IO](await = true) - sequential(D) + sequential(D, false) awaitTermination(D) @@ -48,7 +48,7 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { "await = false" >> { val D = Dispatcher.sequential[IO](await = false) - sequential(D) + sequential(D, false) "cancel all inner effects when canceled" in real { var canceled = false @@ -64,9 +64,43 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { } } - private def sequential(dispatcher: Resource[IO, Dispatcher[IO]]) = { + "sequential dispatcher (cancelable = true)" should { + "await = true" >> { + val D = Dispatcher.sequential[IO](await = true, cancelable = true) + + sequential(D, true) + + awaitTermination(D) + + "not hang" in real { + D.use(dispatcher => IO(dispatcher.unsafeRunAndForget(IO.unit))) + .replicateA(if (isJS || isNative) 1 else 10000) + .as(true) + } + } + + "await = false" >> { + val D = Dispatcher.sequential[IO](await = false, cancelable = true) + + sequential(D, true) - common(dispatcher) + "cancel all inner effects when canceled" in real { + var canceled = false + + val body = D use { runner => + IO(runner.unsafeRunAndForget(IO.never.onCancel(IO { canceled = true }))) *> IO.never + } + + val action = body.start.flatMap(f => IO.sleep(500.millis) *> f.cancel) + + TestControl.executeEmbed(action *> IO(canceled must beTrue)) + } + } + } + + private def sequential(dispatcher: Resource[IO, Dispatcher[IO]], cancelable: Boolean) = { + + common(dispatcher, cancelable) "strictly sequentialize multiple IOs" in real { val length = 1000 @@ -88,24 +122,6 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { } yield ok } - "ignore action cancelation" in real { - var canceled = false - - val rec = dispatcher flatMap { runner => - val run = IO { - runner - .unsafeToFutureCancelable(IO.sleep(500.millis).onCancel(IO { canceled = true })) - ._2 - } - - Resource eval { - run.flatMap(ct => IO.sleep(200.millis) >> IO.fromFuture(IO(ct()))) - } - } - - TestControl.executeEmbed(rec.use(_ => IO(canceled must beFalse))) - } - "reject new tasks after release action is submitted as a task" in ticked { implicit ticker => val test = dispatcher.allocated.flatMap { @@ -219,7 +235,7 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { private def parallel(dispatcher: Resource[IO, Dispatcher[IO]]) = { - common(dispatcher) + common(dispatcher, true) "run multiple IOs in parallel" in real { val num = 10 @@ -283,23 +299,9 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { TestControl.executeEmbed(test).attempt.flatMap(e => IO(e must beLeft)) }*/ - - "forward cancelation onto the inner action" in real { - val test = dispatcher use { runner => - IO.ref(false) flatMap { resultsR => - val action = IO.never.onCancel(resultsR.set(true)) - IO(runner.unsafeToFutureCancelable(action)) flatMap { - case (_, cancel) => - IO.sleep(500.millis) *> IO.fromFuture(IO(cancel())) *> resultsR.get - } - } - } - - TestControl.executeEmbed(test).flatMap(b => IO(b must beTrue)) - } } - private def common(dispatcher: Resource[IO, Dispatcher[IO]]) = { + private def common(dispatcher: Resource[IO, Dispatcher[IO]], cancelable: Boolean) = { "run a synchronous IO" in real { val ioa = IO(1).map(_ + 2) @@ -432,6 +434,40 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { test must completeAs(()) } + + if (!cancelable) { + "ignore action cancelation" in real { + var canceled = false + + val rec = dispatcher flatMap { runner => + val run = IO { + runner + .unsafeToFutureCancelable(IO.sleep(500.millis).onCancel(IO { canceled = true })) + ._2 + } + + Resource eval { + run.flatMap(ct => IO.sleep(200.millis) >> IO.fromFuture(IO(ct()))) + } + } + + TestControl.executeEmbed(rec.use(_ => IO(canceled must beFalse))) + } + } else { + "forward cancelation onto the inner action" in real { + val test = dispatcher use { runner => + IO.ref(false) flatMap { resultsR => + val action = IO.never.onCancel(resultsR.set(true)) + IO(runner.unsafeToFutureCancelable(action)) flatMap { + case (_, cancel) => + IO.sleep(500.millis) *> IO.fromFuture(IO(cancel())) *> resultsR.get + } + } + } + + TestControl.executeEmbed(test).flatMap(b => IO(b must beTrue)) + } + } } private def awaitTermination(dispatcher: Resource[IO, Dispatcher[IO]]) = { @@ -466,11 +502,13 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { "issue #3506: await unsafeRunAndForget" in ticked { implicit ticker => val result = for { - resultR <- IO.ref(false) - _ <- dispatcher.use { runner => IO(runner.unsafeRunAndForget(resultR.set(true))) } - result <- resultR.get - } yield result - result must completeAs(true) + latch <- IO.deferred[Unit] + + repro = (latch.complete(()) >> IO.never).uncancelable + _ <- dispatcher.use(runner => IO(runner.unsafeRunAndForget(repro)) >> latch.get) + } yield () + + result must nonTerminate } "cancel active fibers when an error is produced" in real { From 8d85a4cfd84e60942b1aa66eba81d6f95d95aef5 Mon Sep 17 00:00:00 2001 From: Daniel Spiewak Date: Fri, 29 Dec 2023 13:06:01 -0600 Subject: [PATCH 09/47] Rewrote cancelation state machine to be more explicit --- .../scala/cats/effect/std/Dispatcher.scala | 167 ++++++++++-------- 1 file changed, 93 insertions(+), 74 deletions(-) diff --git a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala index afc922a347..54e8b27444 100644 --- a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala +++ b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala @@ -269,22 +269,19 @@ object Dispatcher { launchAll.as(new Dispatcher[F] { def unsafeToFutureCancelable[A](fa: F[A]): (Future[A], () => Future[Unit]) = { - def inner[E](fe: F[E], checker: Option[F[Boolean]]) - : (Future[E], () => Future[Unit]) = { + def inner[E](fe: F[E], result: Promise[E], finalizer: Boolean): () => Future[Unit] = { if (doneR.get()) { throw new IllegalStateException("Dispatcher already closed") } - val p = Promise[E]() - val cancelR = new AtomicReference[F[Unit]]() - val invalidateCancel = Sync[F].delay(cancelR.set(null.asInstanceOf[F[Unit]])) + val stateR = new AtomicReference[TaskState[F]](TaskState.Unstarted) // forward atomicity guarantees onto promise completion val promisory = MonadCancel[F] uncancelable { poll => // invalidate the cancel action when we're done - poll(fe.guarantee(invalidateCancel)).redeemWith( - e => Sync[F].delay(p.failure(e)), - a => Sync[F].delay(p.success(a))) + poll(fe.guarantee(Sync[F].delay(stateR.set(TaskState.Completed)))).redeemWith( + e => Sync[F].delay(result.failure(e)), + a => Sync[F].delay(result.success(a))) } val worker = @@ -293,48 +290,43 @@ object Dispatcher { else workers(0) - checker match { - // fe is a finalizer - case Some(check) => - // bypass over-eager warning - // can get rid of this if we GADT encode `inner`'s parameters - val abortResult: Any = () - val abort = Sync[F].delay(p.success(abortResult.asInstanceOf[E])).void - - worker - .queue - .unsafeOffer(Registration.Finalizer(promisory.void, check, abort)) - - // cannot cancel a cancel - (p.future, () => Future.failed(new UnsupportedOperationException)) - - case None => - val reg = new Registration.Primary(promisory.void, cancelR) - worker.queue.unsafeOffer(reg) - - def cancel(): Future[Unit] = { - reg.action = null.asInstanceOf[F[Unit]] - - val cancelF = cancelR.get() - if (cancelF != null) { - // cannot use null here - if (cancelR.compareAndSet(cancelF, Applicative[F].unit)) { - // this looping is fine, since we drop the cancel - // note that action is published here since the CAS passed - inner(cancelF, Some(Sync[F].delay(cancelR.get() != null)))._1 - } else { - Future.successful(()) - } - } else { - Future.successful(()) - } + if (finalizer) { + worker.queue.unsafeOffer(Registration.Finalizer(promisory.void)) + + // cannot cancel a cancel + () => Future.failed(new UnsupportedOperationException) + } else { + val reg = new Registration.Primary(promisory.void, stateR) + worker.queue.unsafeOffer(reg) + + def cancel(): Future[Unit] = { + reg.action = null.asInstanceOf[F[Unit]] + + stateR.get() match { + case TaskState.Unstarted => + val latch = Promise[Unit]() + + if (stateR.compareAndSet(TaskState.Unstarted, TaskState.CancelRequested(latch))) + latch.future + else + cancel() + + case TaskState.Running(cancel) => + val latch = Promise[Unit]() + val _ = inner(cancel, latch, true) + latch.future + + case TaskState.CancelRequested(latch) => latch.future + case TaskState.Completed => Future.successful(()) } + } - (p.future, cancel _) + cancel _ } } - inner(fa, None) + val result = Promise[A]() + (result.future, inner(fa, result, false)) } override def reportFailure(t: Throwable): Unit = @@ -347,16 +339,22 @@ object Dispatcher { } } + private sealed abstract class TaskState[+F[_]] extends Product with Serializable + + private object TaskState { + case object Unstarted extends TaskState[Nothing] + final case class Running[F[_]](cancel: F[Unit]) extends TaskState[F] + final case class CancelRequested[F[_]](latch: Promise[Unit]) extends TaskState[F] + case object Completed extends TaskState[Nothing] + } + private sealed abstract class Registration[F[_]] private object Registration { - final class Primary[F[_]](var action: F[Unit], val cancelR: AtomicReference[F[Unit]]) + final class Primary[F[_]](var action: F[Unit], val stateR: AtomicReference[TaskState[F]]) extends Registration[F] - // the check action verifies that we haven't completed in the interim (as determined by the promise) - // the abort action runs in the event that we fail that check and need to complete the cancel promise - final case class Finalizer[F[_]](action: F[Unit], check: F[Boolean], abort: F[Unit]) - extends Registration[F] + final case class Finalizer[F[_]](action: F[Unit]) extends Registration[F] final case class PoisonPill[F[_]]() extends Registration[F] } @@ -372,32 +370,48 @@ object Dispatcher { def run: F[Unit] = { val step = queue.take flatMap { case reg: Registration.Primary[F] => - Sync[F].delay(reg.cancelR.get()) flatMap { sig => - val action = reg.action - - // double null check catches overly-aggressive memory fencing - if (sig == null && action != null) { - executor(action) { cancelF => - // we need to double-check that we weren't canceled while spawning - Sync[F].delay(reg.cancelR.compareAndSet(null.asInstanceOf[F[Unit]], cancelF)) flatMap { - case true => - // we weren't canceled! - Applicative[F].unit - - case false => - // we were canceled while spawning, so forward that on to the cancelation action - cancelF + Sync[F] defer { + reg.stateR.get() match { + case TaskState.Unstarted => + val action = reg.action + + if (action == null) { + // this corresponds to a memory race where we see action's write before stateR's + val check = Spawn[F].cede *> Sync[F].delay(reg.stateR.get()) + check.iterateWhile(_ == TaskState.Unstarted) *> Sync[F].delay { + reg.stateR.get() match { + case TaskState.CancelRequested(latch) => + latch.success(()) + () + + case _ => throw new AssertionError + } + } + } else { + executor(action.guarantee(Sync[F].delay(reg.stateR.set(TaskState.Completed)))) { cancelF => + Sync[F] defer { + if (reg.stateR.compareAndSet(TaskState.Unstarted, TaskState.Running(cancelF))) { + Applicative[F].unit + } else { + reg.stateR.get() match { + case TaskState.CancelRequested(latch) => + cancelF.guarantee(Sync[F].delay(latch.success(())).void) + + case _ => + throw new AssertionError // invalid state + } + } + } + } } - } - } else { - // don't spawn, already canceled - Applicative[F].unit + + case TaskState.Running(_) | TaskState.Completed => throw new AssertionError + + case TaskState.CancelRequested(latch) => Sync[F].delay(latch.success(())).void } } - case Registration.Finalizer(action, check, abort) => - // here's the double-check for late finalization - check.ifM(action, abort) + case Registration.Finalizer(action) => action case Registration.PoisonPill() => Sync[F].delay(doneR.set(true)) @@ -427,9 +441,14 @@ object Dispatcher { private object Executor { - def inplace[F[_]]: Executor[F] = + // default sequential executor (ignores cancelation) + def inplace[F[_]: Applicative]: Executor[F] = new Executor[F] { - def apply(task: F[Unit])(registerCancel: F[Unit] => F[Unit]): F[Unit] = task + def apply(task: F[Unit])(registerCancel: F[Unit] => F[Unit]): F[Unit] = { + // we can use unit as a cancel action here since it must always sequence *after* the task + // thus, the task must complete before the cancel action will be picked up + registerCancel(Applicative[F].unit) *> task + } } // sequential executor which respects cancelation (at the cost of additional overhead); not used From 3400bd9506d727d3a801eb0a93a3047e92c0d6a0 Mon Sep 17 00:00:00 2001 From: Daniel Spiewak Date: Fri, 29 Dec 2023 14:39:59 -0600 Subject: [PATCH 10/47] Renamed to avoid insanity --- .../scala/cats/effect/std/Dispatcher.scala | 74 +++++++++---------- .../cats/effect/std/DispatcherSpec.scala | 29 ++++++++ 2 files changed, 66 insertions(+), 37 deletions(-) diff --git a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala index 54e8b27444..bf8408ad6d 100644 --- a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala +++ b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala @@ -274,12 +274,12 @@ object Dispatcher { throw new IllegalStateException("Dispatcher already closed") } - val stateR = new AtomicReference[TaskState[F]](TaskState.Unstarted) + val stateR = new AtomicReference[RegState[F]](RegState.Unstarted) // forward atomicity guarantees onto promise completion val promisory = MonadCancel[F] uncancelable { poll => // invalidate the cancel action when we're done - poll(fe.guarantee(Sync[F].delay(stateR.set(TaskState.Completed)))).redeemWith( + poll(fe.guarantee(Sync[F].delay(stateR.set(RegState.Completed)))).redeemWith( e => Sync[F].delay(result.failure(e)), a => Sync[F].delay(result.success(a))) } @@ -303,21 +303,21 @@ object Dispatcher { reg.action = null.asInstanceOf[F[Unit]] stateR.get() match { - case TaskState.Unstarted => + case RegState.Unstarted => val latch = Promise[Unit]() - if (stateR.compareAndSet(TaskState.Unstarted, TaskState.CancelRequested(latch))) + if (stateR.compareAndSet(RegState.Unstarted, RegState.CancelRequested(latch))) latch.future else cancel() - case TaskState.Running(cancel) => + case RegState.Running(cancel) => val latch = Promise[Unit]() val _ = inner(cancel, latch, true) latch.future - case TaskState.CancelRequested(latch) => latch.future - case TaskState.Completed => Future.successful(()) + case RegState.CancelRequested(latch) => latch.future + case RegState.Completed => Future.successful(()) } } @@ -339,19 +339,19 @@ object Dispatcher { } } - private sealed abstract class TaskState[+F[_]] extends Product with Serializable + private sealed abstract class RegState[+F[_]] extends Product with Serializable - private object TaskState { - case object Unstarted extends TaskState[Nothing] - final case class Running[F[_]](cancel: F[Unit]) extends TaskState[F] - final case class CancelRequested[F[_]](latch: Promise[Unit]) extends TaskState[F] - case object Completed extends TaskState[Nothing] + private object RegState { + case object Unstarted extends RegState[Nothing] + final case class Running[F[_]](cancel: F[Unit]) extends RegState[F] + final case class CancelRequested[F[_]](latch: Promise[Unit]) extends RegState[F] + case object Completed extends RegState[Nothing] } private sealed abstract class Registration[F[_]] private object Registration { - final class Primary[F[_]](var action: F[Unit], val stateR: AtomicReference[TaskState[F]]) + final class Primary[F[_]](var action: F[Unit], val stateR: AtomicReference[RegState[F]]) extends Registration[F] final case class Finalizer[F[_]](action: F[Unit]) extends Registration[F] @@ -372,15 +372,15 @@ object Dispatcher { case reg: Registration.Primary[F] => Sync[F] defer { reg.stateR.get() match { - case TaskState.Unstarted => + case RegState.Unstarted => val action = reg.action if (action == null) { // this corresponds to a memory race where we see action's write before stateR's val check = Spawn[F].cede *> Sync[F].delay(reg.stateR.get()) - check.iterateWhile(_ == TaskState.Unstarted) *> Sync[F].delay { + check.iterateWhile(_ == RegState.Unstarted) *> Sync[F].delay { reg.stateR.get() match { - case TaskState.CancelRequested(latch) => + case RegState.CancelRequested(latch) => latch.success(()) () @@ -388,13 +388,13 @@ object Dispatcher { } } } else { - executor(action.guarantee(Sync[F].delay(reg.stateR.set(TaskState.Completed)))) { cancelF => + executor(action.guarantee(Sync[F].delay(reg.stateR.set(RegState.Completed)))) { cancelF => Sync[F] defer { - if (reg.stateR.compareAndSet(TaskState.Unstarted, TaskState.Running(cancelF))) { + if (reg.stateR.compareAndSet(RegState.Unstarted, RegState.Running(cancelF))) { Applicative[F].unit } else { reg.stateR.get() match { - case TaskState.CancelRequested(latch) => + case RegState.CancelRequested(latch) => cancelF.guarantee(Sync[F].delay(latch.success(())).void) case _ => @@ -405,9 +405,9 @@ object Dispatcher { } } - case TaskState.Running(_) | TaskState.Completed => throw new AssertionError + case RegState.Running(_) | RegState.Completed => throw new AssertionError - case TaskState.CancelRequested(latch) => Sync[F].delay(latch.success(())).void + case RegState.CancelRequested(latch) => Sync[F].delay(latch.success(())).void } } @@ -453,25 +453,25 @@ object Dispatcher { // sequential executor which respects cancelation (at the cost of additional overhead); not used def sequential[F[_]: Concurrent](supervisor: Supervisor[F]): Resource[F, Executor[F]] = { - sealed trait TaskSignal extends Product with Serializable + sealed trait TaskState extends Product with Serializable - object TaskSignal { - final case class Ready(task: F[Unit]) extends TaskSignal - case object Executing extends TaskSignal - case object Void extends TaskSignal + object TaskState { + final case class Ready(task: F[Unit]) extends TaskState + case object Executing extends TaskState + case object Void extends TaskState } - Resource.eval(Queue.unbounded[F, Ref[F, TaskSignal]]) flatMap { tasks => + Resource.eval(Queue.unbounded[F, Ref[F, TaskState]]) flatMap { tasks => // knock it out of the task taking - val evict = Concurrent[F].ref[TaskSignal](TaskSignal.Void).flatMap(tasks.offer(_)) + val evict = Concurrent[F].ref[TaskState](TaskState.Void).flatMap(tasks.offer(_)) Resource.make(Concurrent[F].ref(false))(r => r.set(true) >> evict) evalMap { doneR => Concurrent[F].ref[Option[Deferred[F, Unit]]](None) flatMap { shutoff => val step = tasks.take flatMap { taskR => - taskR.getAndSet(TaskSignal.Executing) flatMap { - case TaskSignal.Ready(task) => task.guarantee(taskR.set(TaskSignal.Void)) + taskR.getAndSet(TaskState.Executing) flatMap { + case TaskState.Ready(task) => task.guarantee(taskR.set(TaskState.Void)) // Executing should be impossible - case TaskSignal.Executing | TaskSignal.Void => Applicative[F].unit + case TaskState.Executing | TaskState.Void => Applicative[F].unit } } @@ -482,15 +482,15 @@ object Dispatcher { Concurrent[F].ref(fiber) map { fiberR => new Executor[F] { def apply(task: F[Unit])(registerCancel: F[Unit] => F[Unit]): F[Unit] = { - Concurrent[F].ref[TaskSignal](TaskSignal.Ready(task)) flatMap { taskR => - val cancelF = taskR.getAndSet(TaskSignal.Void) flatMap { - case TaskSignal.Ready(_) | TaskSignal.Void => + Concurrent[F].ref[TaskState](TaskState.Ready(task)) flatMap { taskR => + val cancelF = taskR.getAndSet(TaskState.Void) flatMap { + case TaskState.Ready(_) | TaskState.Void => Applicative[F].unit - case TaskSignal.Executing => + case TaskState.Executing => Concurrent[F].deferred[Unit] flatMap { latch => // TODO if someone else is already canceling, this will create a deadlock - // to fix this deadlock, we need a fourth TaskSignal state and a double-check on that and the shutoff + // to fix this deadlock, we need a fourth TaskState state and a double-check on that and the shutoff // Step 1: Turn off everything shutoff.set(Some(latch)) >> { diff --git a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala index bd4d871e7d..d39b465af4 100644 --- a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala +++ b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala @@ -467,6 +467,35 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { TestControl.executeEmbed(test).flatMap(b => IO(b must beTrue)) } + + "support multiple concurrent cancelations" in real { + dispatcher use { runner => + for { + latch0 <- IO.deferred[Unit] + latch1 <- IO.deferred[Unit] + latch2 <- IO.deferred[Unit] + + countR <- IO.ref(0) + + action = (latch0.complete(()) *> IO.never).onCancel(latch1.complete(()) *> latch2.get) + + (_, cancel) <- IO(runner.unsafeToFutureCancelable(action)) + + _ <- latch0.get + cancelAction = IO.fromFuture(IO(cancel())) *> countR.update(_ + 1) + _ <- IO(runner.unsafeRunAndForget(cancelAction)) + _ <- IO(runner.unsafeRunAndForget(cancelAction)) + _ <- IO(runner.unsafeRunAndForget(cancelAction)) + + _ <- latch1.get + _ <- IO.sleep(100.millis) + count <- countR.get + _ <- IO(count mustEqual 0) + + _ <- latch2.complete(()) + } yield ok + } + } } } From 6c9217a7b21cc1ef6ca9df05a42dac3b330b3da8 Mon Sep 17 00:00:00 2001 From: Daniel Spiewak Date: Fri, 29 Dec 2023 19:58:20 -0600 Subject: [PATCH 11/47] Fixed sequential multi-cancelation --- .../scala/cats/effect/std/Dispatcher.scala | 98 +++++++++++++------ .../cats/effect/std/DispatcherSpec.scala | 22 +++-- 2 files changed, 80 insertions(+), 40 deletions(-) diff --git a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala index bf8408ad6d..72e2f51c02 100644 --- a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala +++ b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala @@ -384,7 +384,7 @@ object Dispatcher { latch.success(()) () - case _ => throw new AssertionError + case s => throw new AssertionError(s"a => $s") } } } else { @@ -397,21 +397,24 @@ object Dispatcher { case RegState.CancelRequested(latch) => cancelF.guarantee(Sync[F].delay(latch.success(())).void) - case _ => - throw new AssertionError // invalid state + case RegState.Completed => + Applicative[F].unit + + case s => throw new AssertionError(s"b => $s") } } } } } - case RegState.Running(_) | RegState.Completed => throw new AssertionError + case s @ (RegState.Running(_) | RegState.Completed) => throw new AssertionError(s"c => $s") case RegState.CancelRequested(latch) => Sync[F].delay(latch.success(())).void } } - case Registration.Finalizer(action) => action + case Registration.Finalizer(action) => + action case Registration.PoisonPill() => Sync[F].delay(doneR.set(true)) @@ -458,20 +461,29 @@ object Dispatcher { object TaskState { final case class Ready(task: F[Unit]) extends TaskState case object Executing extends TaskState - case object Void extends TaskState + final case class Canceling(latch: Deferred[F, Unit]) extends TaskState + case object Dead extends TaskState } Resource.eval(Queue.unbounded[F, Ref[F, TaskState]]) flatMap { tasks => // knock it out of the task taking - val evict = Concurrent[F].ref[TaskState](TaskState.Void).flatMap(tasks.offer(_)) + val evict = Concurrent[F].ref[TaskState](TaskState.Dead).flatMap(tasks.offer(_)) Resource.make(Concurrent[F].ref(false))(r => r.set(true) >> evict) evalMap { doneR => Concurrent[F].ref[Option[Deferred[F, Unit]]](None) flatMap { shutoff => val step = tasks.take flatMap { taskR => taskR.getAndSet(TaskState.Executing) flatMap { - case TaskState.Ready(task) => task.guarantee(taskR.set(TaskState.Void)) + case TaskState.Ready(task) => + task guarantee { + taskR.getAndSet(TaskState.Dead) flatMap { + // if we finished during cancelation, we need to catch it before it kills us + case TaskState.Canceling(latch) => latch.complete(()).void + case _ => Applicative[F].unit + } + } + // Executing should be impossible - case TaskState.Executing | TaskState.Void => Applicative[F].unit + case TaskState.Executing | TaskState.Canceling(_) | TaskState.Dead => Applicative[F].unit } } @@ -483,29 +495,53 @@ object Dispatcher { new Executor[F] { def apply(task: F[Unit])(registerCancel: F[Unit] => F[Unit]): F[Unit] = { Concurrent[F].ref[TaskState](TaskState.Ready(task)) flatMap { taskR => - val cancelF = taskR.getAndSet(TaskState.Void) flatMap { - case TaskState.Ready(_) | TaskState.Void => - Applicative[F].unit - - case TaskState.Executing => - Concurrent[F].deferred[Unit] flatMap { latch => - // TODO if someone else is already canceling, this will create a deadlock - // to fix this deadlock, we need a fourth TaskState state and a double-check on that and the shutoff - - // Step 1: Turn off everything - shutoff.set(Some(latch)) >> { - // Step 2: Drain the queue - tasks.tryTakeN(None) flatMap { scratch => - // Step 3: Cancel the executor, put it all back, and restart executor - fiberR.get.flatMap(_.cancel) >> - scratch.traverse_(tasks.offer(_)) >> - spawnExecutor.flatMap(fiberR.set(_)) >> - latch.complete(()) >> - shutoff.set(None) - } - } + val cancelF = + Concurrent[F].deferred[Unit] flatMap { cancelLatch => + taskR flatModify { + case TaskState.Ready(_) | TaskState.Dead => + (TaskState.Dead, Applicative[F].unit) + + case TaskState.Canceling(cancelLatch) => + (TaskState.Canceling(cancelLatch), cancelLatch.get) + + case TaskState.Executing => + // we won the race for cancelation and it's already executing + val eff = for { + // lock the door + latch <- Concurrent[F].deferred[Unit] + _ <- shutoff.set(Some(latch)) + + // drain the task queue + scratch <- tasks.tryTakeN(None) + + // double check that execution didn't finish while we drained + _ <- cancelLatch.tryGet flatMap { + case Some(_) => + Applicative[F].unit + + case None => + for { + // kill the current executor + _ <- fiberR.get.flatMap(_.cancel) + + // restore all of the tasks + _ <- scratch.traverse_(tasks.offer(_)) + + // start a new fiber + _ <- spawnExecutor.flatMap(fiberR.set(_)) + + // allow everyone else back in + _ <- latch.complete(()) + _ <- shutoff.set(None) + } yield () + } + + _ <- cancelLatch.complete(()) + } yield () + + (TaskState.Canceling(cancelLatch), eff) } - } + } // in rare cases, this can create mutual ordering issues with quickly enqueued tasks val optBlock = shutoff.get flatMap { diff --git a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala index d39b465af4..4d1465c028 100644 --- a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala +++ b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala @@ -470,29 +470,33 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { "support multiple concurrent cancelations" in real { dispatcher use { runner => + val count = new AtomicInteger(0) + for { latch0 <- IO.deferred[Unit] latch1 <- IO.deferred[Unit] latch2 <- IO.deferred[Unit] - countR <- IO.ref(0) - - action = (latch0.complete(()) *> IO.never).onCancel(latch1.complete(()) *> latch2.get) + action = (latch0.complete(()) *> IO.never) + .onCancel(latch1.complete(()) *> latch2.get) (_, cancel) <- IO(runner.unsafeToFutureCancelable(action)) _ <- latch0.get - cancelAction = IO.fromFuture(IO(cancel())) *> countR.update(_ + 1) - _ <- IO(runner.unsafeRunAndForget(cancelAction)) - _ <- IO(runner.unsafeRunAndForget(cancelAction)) - _ <- IO(runner.unsafeRunAndForget(cancelAction)) + + ec <- IO.executionContext + cancelAction = IO(cancel().onComplete(_ => count.getAndIncrement())(ec)) + _ <- cancelAction + _ <- cancelAction + _ <- cancelAction _ <- latch1.get _ <- IO.sleep(100.millis) - count <- countR.get - _ <- IO(count mustEqual 0) + _ <- IO(count.get() mustEqual 0) _ <- latch2.complete(()) + _ <- IO.sleep(100.millis) + _ <- IO(count.get() mustEqual 3) } yield ok } } From 3413cc31d8d9e7c988282768429ddb507672cc81 Mon Sep 17 00:00:00 2001 From: Daniel Spiewak Date: Fri, 29 Dec 2023 20:19:07 -0600 Subject: [PATCH 12/47] Fixed fun memory race (on ARM no less!) with parallel cancelation --- .../src/main/scala/cats/effect/std/Dispatcher.scala | 4 ++-- .../test/scala/cats/effect/std/DispatcherSpec.scala | 11 +++++++---- 2 files changed, 9 insertions(+), 6 deletions(-) diff --git a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala index 72e2f51c02..27893702e5 100644 --- a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala +++ b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala @@ -300,12 +300,12 @@ object Dispatcher { worker.queue.unsafeOffer(reg) def cancel(): Future[Unit] = { - reg.action = null.asInstanceOf[F[Unit]] - stateR.get() match { case RegState.Unstarted => val latch = Promise[Unit]() + reg.action = null.asInstanceOf[F[Unit]] + if (stateR.compareAndSet(RegState.Unstarted, RegState.CancelRequested(latch))) latch.future else diff --git a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala index 4d1465c028..bf29b09792 100644 --- a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala +++ b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala @@ -281,15 +281,18 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { // https://github.com/typelevel/cats-effect/issues/3898 "not hang when cancelling" in real { - dispatcher.use { dispatcher => - IO.fromFuture { + val test = dispatcher.use { dispatcher => + val action = IO.fromFuture { IO { val (_, cancel) = dispatcher.unsafeToFutureCancelable(IO.never) cancel() } - }.replicateA_(1000) - .as(ok) + } + + action.replicateA_(1000) } + + test.parReplicateA_(100).as(ok) } /*"fail to terminate when running one's own release in all modes" in real { From b28706dde7945d8eb564f3aff1e104abbf9b488c Mon Sep 17 00:00:00 2001 From: Daniel Spiewak Date: Fri, 29 Dec 2023 20:28:55 -0600 Subject: [PATCH 13/47] Removed bad idea --- .../src/test/scala/cats/effect/std/DispatcherSpec.scala | 8 -------- 1 file changed, 8 deletions(-) diff --git a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala index bf29b09792..f6a2aa4e8d 100644 --- a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala +++ b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala @@ -294,14 +294,6 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { test.parReplicateA_(100).as(ok) } - - /*"fail to terminate when running one's own release in all modes" in real { - val test = dispatcher.allocated flatMap { - case (runner, release) => IO(runner.unsafeRunAndForget(release)) - } - - TestControl.executeEmbed(test).attempt.flatMap(e => IO(e must beLeft)) - }*/ } private def common(dispatcher: Resource[IO, Dispatcher[IO]], cancelable: Boolean) = { From ba091cc9209b567a0281ccd41108563ec5fcfb5d Mon Sep 17 00:00:00 2001 From: Daniel Spiewak Date: Fri, 29 Dec 2023 20:34:47 -0600 Subject: [PATCH 14/47] Made new `Dispatcher` mode package-private until we can decide what to do with it --- .../scala/cats/effect/std/Dispatcher.scala | 31 ++++++++++--------- .../cats/effect/std/DispatcherSpec.scala | 4 +-- 2 files changed, 18 insertions(+), 17 deletions(-) diff --git a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala index 27893702e5..f43f9a48f3 100644 --- a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala +++ b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala @@ -184,18 +184,22 @@ object Dispatcher { * - true - wait for the completion of the active fiber * - false - cancel the active fiber */ - def sequential[F[_]: Async](await: Boolean, cancelable: Boolean = false): Resource[F, Dispatcher[F]] = - impl[F](false, await, cancelable) + def sequential[F[_]: Async](await: Boolean): Resource[F, Dispatcher[F]] = + impl[F](false, await, false) + + // TODO decide if we want other people to use this + private[std] def sequentialCancelable[F[_]: Async](await: Boolean): Resource[F, Dispatcher[F]] = + impl[F](false, await, true) /* - * There are two fundamental modes here: sequential and parallel. There is very little overlap - * in semantics between the two apart from the submission side. The whole thing is split up into + * There are three fundamental modes here: sequential, parallel, and sequential-cancelable. There + * is very little overlap in semantics between the three apart from the submission side. The whole thing is split up into * a submission queue with impure enqueue and cancel functions which is drained by the `Worker` and an * internal execution protocol which also involves a queue. The `Worker` encapsulates all of the * race conditions and negotiations with impure code, while the `Executor` manages running the * tasks with appropriate semantics. In parallel mode, we shard the `Worker`s according to the * number of CPUs and select a random queue (in impure code) as a target. This reduces contention - * at the cost of ordering, which is not guaranteed in parallel mode. With sequential mode, there + * at the cost of ordering, which is not guaranteed in parallel mode. With the sequential modes, there * is only a single worker. * * On the impure side, the queue bit is the easy part: it's just a `LinkedBlockingQueue` which @@ -207,10 +211,15 @@ object Dispatcher { * which is in the process of being taken off the queue, and those race conditions are negotiated * between the impure code and the `Worker`. * - * On the pure side, the two different `Executor`s are very distinct. In parallel mode, it's easy: + * On the pure side, the three different `Executor`s are very distinct. In parallel mode, it's easy: * we have a separate `Supervisor` which doesn't respawn actions, and we use that supervisor to * spawn a new fiber for each task unit. Cancelation in this mode is easy: we just cancel the fiber. - * For sequential mode, we spawn a *single* executor fiber on the main supervisor (which respawns). + * + * Sequential mode is the simplest of all: all work is executed in-place and cannot be canceled. + * The cancelation action in all cases is simply `unit` because the impure submission will not be + * seen until after the work is completed *anyway*, so there's no point in being fancy. + * + * For sequential-cancelable mode, we spawn a *single* executor fiber on the main supervisor (which respawns). * This fiber is paired with a pure unbounded queue and a shutoff latch. New work is placed on the * queue, which the fiber takes from in order and executes in-place. If the work self-cancels or * errors, the executor will be restarted. In the case of external cancelation, we shut off the @@ -218,14 +227,6 @@ object Dispatcher { * executor fiber in-place so long as we're sure it's actively working on the target task. Once * that cancelation completes (which will ultimately restart the executor fiber), we re-fill the * queue and unlock the latch to allow new work (from the `Worker`). - * - * Note that a third mode is possible but not implemented: sequential *without* cancelation. In - * this mode, we execute each task directly on the worker fiber as it comes in, without the - * added indirection of the executor queue. This reduces overhead considerably, but the price is - * we can no longer support external (impure) cancelation. This is because the worker fiber is - * *also* responsible for dequeueing from the impure queue, which is where the cancelation tasks - * come in. The worker can't observe a cancelation action while it's executing another action, so - * cancelation cannot then preempt and is effectively worthless. */ private[this] def impl[F[_]: Async]( parallel: Boolean, diff --git a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala index f6a2aa4e8d..57d3f16fbf 100644 --- a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala +++ b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala @@ -66,7 +66,7 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { "sequential dispatcher (cancelable = true)" should { "await = true" >> { - val D = Dispatcher.sequential[IO](await = true, cancelable = true) + val D = Dispatcher.sequentialCancelable[IO](await = true) sequential(D, true) @@ -80,7 +80,7 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { } "await = false" >> { - val D = Dispatcher.sequential[IO](await = false, cancelable = true) + val D = Dispatcher.sequentialCancelable[IO](await = false) sequential(D, true) From 17620262ce8c304217cfe3cf2592941f757b6bae Mon Sep 17 00:00:00 2001 From: Daniel Spiewak Date: Sat, 30 Dec 2023 11:50:41 -0600 Subject: [PATCH 15/47] Replaced `LinkedBlockingQueue` with `UnsafeUnbounded` --- .../scala/cats/effect/std/Dispatcher.scala | 20 +++++++------------ 1 file changed, 7 insertions(+), 13 deletions(-) diff --git a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala index f43f9a48f3..25aca5d6e7 100644 --- a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala +++ b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala @@ -25,7 +25,7 @@ import cats.syntax.all._ import scala.concurrent.{ExecutionContext, Future, Promise} import scala.util.Failure -import java.util.concurrent.{LinkedBlockingQueue, ThreadLocalRandom} +import java.util.concurrent.ThreadLocalRandom import java.util.concurrent.atomic.{AtomicBoolean, AtomicReference} /** @@ -575,11 +575,11 @@ object Dispatcher { // MPSC assumption private final class UnsafeAsyncQueue[F[_]: Async, A] { - private[this] val buffer = new LinkedBlockingQueue[A]() + private[this] val buffer = new UnsafeUnbounded[A]() private[this] val latchR = new AtomicReference[Either[Throwable, Unit] => Unit](null) def unsafeOffer(a: A): Unit = { - buffer.offer(a) + val _ = buffer.put(a) val f = latchR.get() if (latchR.compareAndSet(f, Signal)) { @@ -596,22 +596,16 @@ object Dispatcher { if (latchR.compareAndSet(null, k)) { Left(Some(Applicative[F].unit)) // all cleanup is elsewhere } else { - val result = latchR.compareAndSet(Signal, null) - require( - result - ) // since this is a single consumer queue, we should never have multiple callbacks + // since this is a single consumer queue, we should never have multiple callbacks + latchR.set(null) Right(()) } } } MonadCancel[F] uncancelable { poll => - Sync[F].delay(buffer.poll()) flatMap { a => - if (a == null) - poll(latchF >> take) - else - Applicative[F].pure(a) - } + // emptiness is reported as a FailureSignal error + Sync[F].delay(buffer.take()).handleErrorWith(_ => poll(latchF *> take)) } } } From aebf92daf2790ec7b7d6bff79d6c8e8a65b62881 Mon Sep 17 00:00:00 2001 From: Daniel Spiewak Date: Sat, 30 Dec 2023 12:09:18 -0600 Subject: [PATCH 16/47] Scalafmt --- .../scala/cats/effect/std/Dispatcher.scala | 87 ++++++++++++------- 1 file changed, 56 insertions(+), 31 deletions(-) diff --git a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala index 25aca5d6e7..134fc66f3b 100644 --- a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala +++ b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala @@ -17,7 +17,17 @@ package cats.effect.std import cats.Applicative -import cats.effect.kernel.{Async, Concurrent, Deferred, MonadCancel, Outcome, Ref, Resource, Spawn, Sync} +import cats.effect.kernel.{ + Async, + Concurrent, + Deferred, + MonadCancel, + Outcome, + Ref, + Resource, + Spawn, + Sync +} import cats.effect.kernel.syntax.all._ import cats.effect.std.Dispatcher.parasiticEC import cats.syntax.all._ @@ -149,7 +159,8 @@ object Dispatcher { * - true - wait for the completion of the active fibers * - false - cancel the active fibers */ - def parallel[F[_]: Async](await: Boolean): Resource[F, Dispatcher[F]] = impl[F](true, await, true) + def parallel[F[_]: Async](await: Boolean): Resource[F, Dispatcher[F]] = + impl[F](true, await, true) /** * Create a [[Dispatcher]] that can be used within a resource scope. Once the resource scope @@ -188,7 +199,8 @@ object Dispatcher { impl[F](false, await, false) // TODO decide if we want other people to use this - private[std] def sequentialCancelable[F[_]: Async](await: Boolean): Resource[F, Dispatcher[F]] = + private[std] def sequentialCancelable[F[_]: Async]( + await: Boolean): Resource[F, Dispatcher[F]] = impl[F](false, await, true) /* @@ -245,12 +257,13 @@ object Dispatcher { val awaitTermination = Resource.make(Concurrent[F].deferred[Unit])(_.complete(()).void) (awaitTermination, termination) flatMapN { (terminationLatch, doneR) => - val executorF = if (parallel) - Executor.parallel[F](await) - else if (cancelable) - Executor.sequential(supervisor) - else - Resource.pure[F, Executor[F]](Executor.inplace[F]) + val executorF = + if (parallel) + Executor.parallel[F](await) + else if (cancelable) + Executor.sequential(supervisor) + else + Resource.pure[F, Executor[F]](Executor.inplace[F]) // note this scopes the executors *outside* the workers, meaning the workers shut down first // I think this is what we want, since it avoids enqueue race conditions @@ -270,7 +283,8 @@ object Dispatcher { launchAll.as(new Dispatcher[F] { def unsafeToFutureCancelable[A](fa: F[A]): (Future[A], () => Future[Unit]) = { - def inner[E](fe: F[E], result: Promise[E], finalizer: Boolean): () => Future[Unit] = { + def inner[E](fe: F[E], result: Promise[E], finalizer: Boolean) + : () => Future[Unit] = { if (doneR.get()) { throw new IllegalStateException("Dispatcher already closed") } @@ -280,9 +294,10 @@ object Dispatcher { // forward atomicity guarantees onto promise completion val promisory = MonadCancel[F] uncancelable { poll => // invalidate the cancel action when we're done - poll(fe.guarantee(Sync[F].delay(stateR.set(RegState.Completed)))).redeemWith( - e => Sync[F].delay(result.failure(e)), - a => Sync[F].delay(result.success(a))) + poll(fe.guarantee(Sync[F].delay(stateR.set(RegState.Completed)))) + .redeemWith( + e => Sync[F].delay(result.failure(e)), + a => Sync[F].delay(result.success(a))) } val worker = @@ -307,7 +322,9 @@ object Dispatcher { reg.action = null.asInstanceOf[F[Unit]] - if (stateR.compareAndSet(RegState.Unstarted, RegState.CancelRequested(latch))) + if (stateR.compareAndSet( + RegState.Unstarted, + RegState.CancelRequested(latch))) latch.future else cancel() @@ -389,26 +406,31 @@ object Dispatcher { } } } else { - executor(action.guarantee(Sync[F].delay(reg.stateR.set(RegState.Completed)))) { cancelF => - Sync[F] defer { - if (reg.stateR.compareAndSet(RegState.Unstarted, RegState.Running(cancelF))) { - Applicative[F].unit - } else { - reg.stateR.get() match { - case RegState.CancelRequested(latch) => - cancelF.guarantee(Sync[F].delay(latch.success(())).void) - - case RegState.Completed => - Applicative[F].unit - - case s => throw new AssertionError(s"b => $s") + executor( + action.guarantee(Sync[F].delay(reg.stateR.set(RegState.Completed)))) { + cancelF => + Sync[F] defer { + if (reg + .stateR + .compareAndSet(RegState.Unstarted, RegState.Running(cancelF))) { + Applicative[F].unit + } else { + reg.stateR.get() match { + case RegState.CancelRequested(latch) => + cancelF.guarantee(Sync[F].delay(latch.success(())).void) + + case RegState.Completed => + Applicative[F].unit + + case s => throw new AssertionError(s"b => $s") + } } } - } } } - case s @ (RegState.Running(_) | RegState.Completed) => throw new AssertionError(s"c => $s") + case s @ (RegState.Running(_) | RegState.Completed) => + throw new AssertionError(s"c => $s") case RegState.CancelRequested(latch) => Sync[F].delay(latch.success(())).void } @@ -431,7 +453,9 @@ object Dispatcher { private object Worker { - def apply[F[_]: Async](executor: Executor[F], terminationLatch: Deferred[F, Unit]): Resource[F, Worker[F]] = { + def apply[F[_]: Async]( + executor: Executor[F], + terminationLatch: Deferred[F, Unit]): Resource[F, Worker[F]] = { val initF = Sync[F].delay( new Worker[F](new UnsafeAsyncQueue[F, Registration[F]](), executor, terminationLatch)) @@ -484,7 +508,8 @@ object Dispatcher { } // Executing should be impossible - case TaskState.Executing | TaskState.Canceling(_) | TaskState.Dead => Applicative[F].unit + case TaskState.Executing | TaskState.Canceling(_) | TaskState.Dead => + Applicative[F].unit } } From 3c40393c721f185a872d24310622642db3c2f186 Mon Sep 17 00:00:00 2001 From: Daniel Spiewak Date: Sat, 30 Dec 2023 12:22:51 -0600 Subject: [PATCH 17/47] Fixed compilation on Scala 2.12 and 3.x --- .../src/main/scala/cats/effect/std/Dispatcher.scala | 6 ++++-- .../test/scala/cats/effect/std/DispatcherSpec.scala | 10 +++++++--- 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala index 134fc66f3b..866bd46198 100644 --- a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala +++ b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala @@ -329,12 +329,14 @@ object Dispatcher { else cancel() - case RegState.Running(cancel) => + case r: RegState.Running[_] => + val cancel = r.cancel // indirection needed for Scala 2.12 + val latch = Promise[Unit]() val _ = inner(cancel, latch, true) latch.future - case RegState.CancelRequested(latch) => latch.future + case r: RegState.CancelRequested[_] => r.latch.future case RegState.Completed => Future.successful(()) } } diff --git a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala index 57d3f16fbf..15c08a0280 100644 --- a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala +++ b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala @@ -141,7 +141,9 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { latch2 <- IO.deferred[Unit] latch3 <- IO.deferred[Unit] - (_, cancel) <- IO(runner.unsafeToFutureCancelable(IO.unit)) + pair <- IO(runner.unsafeToFutureCancelable(IO.unit)) + (_, cancel) = pair + _ <- IO( runner.unsafeRunAndForget(latch1.complete(()) *> latch2.get *> latch3.complete(()))) @@ -163,7 +165,8 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { latch1 <- IO.deferred[Unit] latch2 <- IO.deferred[Unit] - (_, cancel) <- IO(runner.unsafeToFutureCancelable(latch1.get)) + pair <- IO(runner.unsafeToFutureCancelable(latch1.get)) + (_, cancel) = pair _ <- latch1.complete(()) // the particularly scary case is where the cancel action gets in queue before the next action @@ -475,7 +478,8 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { action = (latch0.complete(()) *> IO.never) .onCancel(latch1.complete(()) *> latch2.get) - (_, cancel) <- IO(runner.unsafeToFutureCancelable(action)) + pair <- IO(runner.unsafeToFutureCancelable(action)) + (_, cancel) = pair _ <- latch0.get From 64e674cf8e44878f03bb218f6574023e7e993d2c Mon Sep 17 00:00:00 2001 From: Daniel Spiewak Date: Sat, 30 Dec 2023 12:37:46 -0600 Subject: [PATCH 18/47] Filtered out spurious mima warnings --- build.sbt | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index 3b6d213387..73dac32616 100644 --- a/build.sbt +++ b/build.sbt @@ -975,7 +975,13 @@ lazy val std = crossProject(JSPlatform, JVMPlatform, NativePlatform) "cats.effect.std.Queue$UnsafeUnbounded$Cell"), // introduced by #3480 // adds method to sealed Hotswap - ProblemFilters.exclude[ReversedMissingMethodProblem]("cats.effect.std.Hotswap.get") + ProblemFilters.exclude[ReversedMissingMethodProblem]("cats.effect.std.Hotswap.get"), + // introduced by #3923 + // Rewrote Dispatcher + ProblemFilters.exclude[MissingClassProblem]("cats.effect.std.Dispatcher$Mode"), + ProblemFilters.exclude[MissingClassProblem]("cats.effect.std.Dispatcher$Mode$"), + ProblemFilters.exclude[MissingClassProblem]("cats.effect.std.Dispatcher$Mode$Parallel$"), + ProblemFilters.exclude[MissingClassProblem]("cats.effect.std.Dispatcher$Mode$Sequential$") ) ) .jsSettings( From 30b9444297cda8a190aa500f12ec220f15d6146e Mon Sep 17 00:00:00 2001 From: Daniel Spiewak Date: Sat, 30 Dec 2023 13:00:29 -0600 Subject: [PATCH 19/47] Skip hammering on race condition on non-JVM platforms --- .../src/test/scala/cats/effect/std/DispatcherSpec.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala index 15c08a0280..3d29daea4f 100644 --- a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala +++ b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala @@ -295,7 +295,10 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { action.replicateA_(1000) } - test.parReplicateA_(100).as(ok) + if (isJVM) + test.parReplicateA_(100).as(ok) + else + test.as(ok) } } From 79f2d98ee88d5dd249578c7cb51978d22aecd16e Mon Sep 17 00:00:00 2001 From: Daniel Spiewak Date: Sat, 30 Dec 2023 14:27:37 -0600 Subject: [PATCH 20/47] =?UTF-8?q?V8=20*really*=20doesn't=20like=20this=20t?= =?UTF-8?q?est=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../shared/src/test/scala/cats/effect/std/DispatcherSpec.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala index 3d29daea4f..7b32b7d5c1 100644 --- a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala +++ b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala @@ -292,7 +292,7 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { } } - action.replicateA_(1000) + action.replicateA_(if (isJVM) 1000 else 1) } if (isJVM) From 8f0d60c2af074930256eaf9dd1cae39fb6a5eb70 Mon Sep 17 00:00:00 2001 From: Daniel Spiewak Date: Fri, 12 Jan 2024 09:43:05 -0600 Subject: [PATCH 21/47] Scalafmt --- build.sbt | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/build.sbt b/build.sbt index 73dac32616..74f88ad7aa 100644 --- a/build.sbt +++ b/build.sbt @@ -980,8 +980,10 @@ lazy val std = crossProject(JSPlatform, JVMPlatform, NativePlatform) // Rewrote Dispatcher ProblemFilters.exclude[MissingClassProblem]("cats.effect.std.Dispatcher$Mode"), ProblemFilters.exclude[MissingClassProblem]("cats.effect.std.Dispatcher$Mode$"), - ProblemFilters.exclude[MissingClassProblem]("cats.effect.std.Dispatcher$Mode$Parallel$"), - ProblemFilters.exclude[MissingClassProblem]("cats.effect.std.Dispatcher$Mode$Sequential$") + ProblemFilters.exclude[MissingClassProblem]( + "cats.effect.std.Dispatcher$Mode$Parallel$"), + ProblemFilters.exclude[MissingClassProblem]( + "cats.effect.std.Dispatcher$Mode$Sequential$") ) ) .jsSettings( From 1b90b603a04a76313b10569b65d8452ab2d1ce6f Mon Sep 17 00:00:00 2001 From: Daniel Spiewak Date: Fri, 12 Jan 2024 09:44:45 -0600 Subject: [PATCH 22/47] Fixed outdated comment --- std/shared/src/main/scala/cats/effect/std/Dispatcher.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala index 866bd46198..079cb69cfd 100644 --- a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala +++ b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala @@ -214,7 +214,7 @@ object Dispatcher { * at the cost of ordering, which is not guaranteed in parallel mode. With the sequential modes, there * is only a single worker. * - * On the impure side, the queue bit is the easy part: it's just a `LinkedBlockingQueue` which + * On the impure side, the queue bit is the easy part: it's just a `UnsafeUnbounded` (queue) which * accepts Registration(s). It's easiest to think of this a bit like an actor model, where the * `Worker` is the actor and the enqueue is the send. Whenever we send a unit of work, that * message has an `AtomicReference` which allows us to back-propagate a cancelation action. That From 2e48a1f34eaf54b6f567227d4a44586d3c088cb5 Mon Sep 17 00:00:00 2001 From: Daniel Spiewak Date: Sat, 13 Jan 2024 16:50:13 -0600 Subject: [PATCH 23/47] Fixed missing case in `unsafeOffer` cas loop --- .../scala/cats/effect/std/Dispatcher.scala | 89 +++++++++++++------ 1 file changed, 60 insertions(+), 29 deletions(-) diff --git a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala index 079cb69cfd..3cd8a094a5 100644 --- a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala +++ b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala @@ -32,6 +32,7 @@ import cats.effect.kernel.syntax.all._ import cats.effect.std.Dispatcher.parasiticEC import cats.syntax.all._ +import scala.annotation.tailrec import scala.concurrent.{ExecutionContext, Future, Promise} import scala.util.Failure @@ -324,10 +325,11 @@ object Dispatcher { if (stateR.compareAndSet( RegState.Unstarted, - RegState.CancelRequested(latch))) + RegState.CancelRequested(latch))) { latch.future - else + } else { cancel() + } case r: RegState.Running[_] => val cancel = r.cancel // indirection needed for Scala 2.12 @@ -336,8 +338,11 @@ object Dispatcher { val _ = inner(cancel, latch, true) latch.future - case r: RegState.CancelRequested[_] => r.latch.future - case RegState.Completed => Future.successful(()) + case r: RegState.CancelRequested[_] => + r.latch.future + + case RegState.Completed => + Future.successful(()) } } @@ -404,37 +409,40 @@ object Dispatcher { latch.success(()) () - case s => throw new AssertionError(s"a => $s") + case s => + throw new AssertionError(s"a => $s") } } } else { - executor( - action.guarantee(Sync[F].delay(reg.stateR.set(RegState.Completed)))) { - cancelF => - Sync[F] defer { - if (reg - .stateR - .compareAndSet(RegState.Unstarted, RegState.Running(cancelF))) { - Applicative[F].unit - } else { - reg.stateR.get() match { - case RegState.CancelRequested(latch) => - cancelF.guarantee(Sync[F].delay(latch.success(())).void) - - case RegState.Completed => - Applicative[F].unit - - case s => throw new AssertionError(s"b => $s") - } + val withCompletion = + action.guarantee(Sync[F].delay(reg.stateR.set(RegState.Completed))) + + executor(withCompletion) { cancelF => + Sync[F] defer { + if (reg + .stateR + .compareAndSet(RegState.Unstarted, RegState.Running(cancelF))) { + Applicative[F].unit + } else { + reg.stateR.get() match { + case RegState.CancelRequested(latch) => + cancelF.guarantee(Sync[F].delay(latch.success(())).void) + + case RegState.Completed => + Applicative[F].unit + + case s => throw new AssertionError(s"b => $s") } } + } } } case s @ (RegState.Running(_) | RegState.Completed) => throw new AssertionError(s"c => $s") - case RegState.CancelRequested(latch) => Sync[F].delay(latch.success(())).void + case RegState.CancelRequested(latch) => + Sync[F].delay(latch.success(())).void } } @@ -603,18 +611,40 @@ object Dispatcher { // MPSC assumption private final class UnsafeAsyncQueue[F[_]: Async, A] { private[this] val buffer = new UnsafeUnbounded[A]() + + /* + * State machine: + * + * - null => no taker, buffer state undefined + * - Signal => no (unnotified) taker, buffer state likely non-empty + * - other => taker, buffer state empty + */ private[this] val latchR = new AtomicReference[Either[Throwable, Unit] => Unit](null) def unsafeOffer(a: A): Unit = { val _ = buffer.put(a) - val f = latchR.get() - if (latchR.compareAndSet(f, Signal)) { - if (f != null) { - f(RightUnit) + @tailrec + def notify(): Unit = { + latchR.get() match { + case null => + if (!latchR.compareAndSet(null, Signal)) { + // someone suspended while we were looking, retry notification + notify() + } + + // in this case, someone else already awakened the taker, so we're good + case Signal => () + + case f => + // failed cas is fine, since it means some other producer woke up the consumer + if (latchR.compareAndSet(f, Signal)) { + f(RightUnit) + } } - // if f == null, take will loop back around and discover the Signal } + + notify() } def take: F[A] = { @@ -624,6 +654,7 @@ object Dispatcher { Left(Some(Applicative[F].unit)) // all cleanup is elsewhere } else { // since this is a single consumer queue, we should never have multiple callbacks + // assert(latchR.get() == Signal) latchR.set(null) Right(()) } From ecd9008d0f037bf1a7bcfcb14cd7c0587cbb3428 Mon Sep 17 00:00:00 2001 From: Sam Pillsworth Date: Sat, 13 Jan 2024 19:01:19 -0500 Subject: [PATCH 24/47] test ordering maintained during finalization --- .../shared/src/test/scala/cats/effect/std/DispatcherSpec.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala index 7b32b7d5c1..c5f47e7920 100644 --- a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala +++ b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala @@ -114,8 +114,9 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { 0.until(length) foreach { i => runner.unsafeRunAndForget(results.update(_ :+ i).guarantee(gate.release)) } - } *> gate.await + } } + _ <- gate.await vec <- results.get _ <- IO(vec mustEqual 0.until(length).toVector) From b967cd6f514c62704d02988df70b90fd8876ed74 Mon Sep 17 00:00:00 2001 From: Daniel Spiewak Date: Sat, 13 Jan 2024 20:14:15 -0600 Subject: [PATCH 25/47] Revert "test ordering maintained during finalization" This reverts commit ecd9008d0f037bf1a7bcfcb14cd7c0587cbb3428. --- .../shared/src/test/scala/cats/effect/std/DispatcherSpec.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala index c5f47e7920..7b32b7d5c1 100644 --- a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala +++ b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala @@ -114,9 +114,8 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { 0.until(length) foreach { i => runner.unsafeRunAndForget(results.update(_ :+ i).guarantee(gate.release)) } - } + } *> gate.await } - _ <- gate.await vec <- results.get _ <- IO(vec mustEqual 0.until(length).toVector) From 3041bc90964fbbae3ca6d5019d3f0fedf6a3d51d Mon Sep 17 00:00:00 2001 From: Daniel Spiewak Date: Mon, 15 Jan 2024 13:23:40 -0600 Subject: [PATCH 26/47] Added tests to assert queue draining behavior during shutdown --- .../cats/effect/std/DispatcherSpec.scala | 28 +++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala index 7b32b7d5c1..fb6becb8f7 100644 --- a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala +++ b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala @@ -43,6 +43,34 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { .replicateA(if (isJS || isNative) 1 else 10000) .as(true) } + + "await work queue drain on shutdown" in real { + val count = 1000 + + IO.ref(0) flatMap { resultsR => + val increments = D use { runner => + IO { + 0.until(count).foreach(_ => runner.unsafeRunAndForget(resultsR.update(_ + 1))) + } + } + + increments *> resultsR.get.flatMap(r => IO(r mustEqual count)) + } + } + + "terminating worker preserves task order" in real { + val count = 10 + + IO.ref(Vector[Int]()) flatMap { resultsR => + val appends = D use { runner => + IO { + 0.until(count).foreach(i => runner.unsafeRunAndForget(resultsR.update(_ :+ i))) + } + } + + appends *> resultsR.get.flatMap(r => IO(r mustEqual 0.until(count).toVector)) + } + } } "await = false" >> { From b62fadc3e0bce8846d7fa09a0935915c32e45e00 Mon Sep 17 00:00:00 2001 From: Sam Pillsworth Date: Fri, 19 Jan 2024 09:58:35 -0500 Subject: [PATCH 27/47] test blocking cancellation --- .../test/scala/cats/effect/std/DispatcherSpec.scala | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala index fb6becb8f7..804be91a87 100644 --- a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala +++ b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala @@ -530,6 +530,18 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { } yield ok } } + + "does long cancellation block a worker?" in real { + dispatcher + .use { runner => + val run = IO { runner.unsafeToFutureCancelable(IO.never[Unit].uncancelable)._2 } + run.flatMap(cancel => IO(cancel())).parReplicateA_(1000) *> IO.fromFuture( + IO(runner.unsafeToFuture(IO.unit))) + } + .replicateA_(1000) + .as(ok) + } + } } From 8d31c63e07b1d5877e90aaa23dfe1a2fa3e3387c Mon Sep 17 00:00:00 2001 From: Arman Bilge Date: Fri, 19 Jan 2024 15:40:40 +0000 Subject: [PATCH 28/47] Move blocked-cancelation test to parallel only --- .../cats/effect/std/DispatcherSpec.scala | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala index 804be91a87..0535a55120 100644 --- a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala +++ b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala @@ -328,6 +328,17 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { else test.as(ok) } + + "does long cancellation block a worker?" in real { + dispatcher + .use { runner => + val run = IO { runner.unsafeToFutureCancelable(IO.never[Unit].uncancelable)._2 } + run.flatMap(cancel => IO(cancel())).parReplicateA_(1000) *> IO.fromFuture( + IO(runner.unsafeToFuture(IO.unit))) + } + .replicateA_(1000) + .as(ok) + } } private def common(dispatcher: Resource[IO, Dispatcher[IO]], cancelable: Boolean) = { @@ -531,17 +542,6 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { } } - "does long cancellation block a worker?" in real { - dispatcher - .use { runner => - val run = IO { runner.unsafeToFutureCancelable(IO.never[Unit].uncancelable)._2 } - run.flatMap(cancel => IO(cancel())).parReplicateA_(1000) *> IO.fromFuture( - IO(runner.unsafeToFuture(IO.unit))) - } - .replicateA_(1000) - .as(ok) - } - } } From abf680d64527489b41067b51224ca2629eedc21d Mon Sep 17 00:00:00 2001 From: Arman Bilge Date: Fri, 19 Jan 2024 15:59:43 +0000 Subject: [PATCH 29/47] Separate blocked-cancelation into two tests --- .../cats/effect/std/DispatcherSpec.scala | 39 ++++++++++++++++--- 1 file changed, 34 insertions(+), 5 deletions(-) diff --git a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala index 0535a55120..418c74ea48 100644 --- a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala +++ b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala @@ -329,14 +329,43 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { test.as(ok) } - "does long cancellation block a worker?" in real { + "cancelation does not block a worker" in real { dispatcher .use { runner => - val run = IO { runner.unsafeToFutureCancelable(IO.never[Unit].uncancelable)._2 } - run.flatMap(cancel => IO(cancel())).parReplicateA_(1000) *> IO.fromFuture( - IO(runner.unsafeToFuture(IO.unit))) + val clogUp = IO.deferred[Unit].flatMap { gate => + IO { + val task = gate.complete(()) *> IO.never[Unit].uncancelable + runner.unsafeToFutureCancelable(task)._2 + }.flatMap { cancel => + // wait until task starts, then cancel + gate.get *> IO(cancel()) + } + } + clogUp.parReplicateA_(1000) *> + // now try to run a new task + IO.fromFuture(IO(runner.unsafeToFuture(IO.unit))) + } + .replicateA_(if (isJVM) 1000 else 1) + .as(ok) + } + + "cancelation race does not block a worker" in real { + dispatcher + .use { runner => + val clogUp = IO { + val task = IO.never[Unit].uncancelable + runner.unsafeToFutureCancelable(task)._2 + }.flatMap { cancel => + // cancel concurrently + // We want to trigger race condition where task starts but then discovers it was canceled + IO(cancel()) + } + + clogUp.parReplicateA_(1000) *> + // now try to run a new task + IO.fromFuture(IO(runner.unsafeToFuture(IO.unit))) } - .replicateA_(1000) + .replicateA_(if (isJVM) 1000 else 1) .as(ok) } } From a474e134e4f3df74d16ad8799117522548f3ddec Mon Sep 17 00:00:00 2001 From: Arman Bilge Date: Fri, 19 Jan 2024 20:10:20 +0000 Subject: [PATCH 30/47] Implement `UnsafeAsyncQueue#take` with `Async#cont` --- .../scala/cats/effect/std/Dispatcher.scala | 37 +++++++++++-------- 1 file changed, 22 insertions(+), 15 deletions(-) diff --git a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala index 3cd8a094a5..21fbe47238 100644 --- a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala +++ b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala @@ -20,8 +20,10 @@ import cats.Applicative import cats.effect.kernel.{ Async, Concurrent, + Cont, Deferred, MonadCancel, + MonadCancelThrow, Outcome, Ref, Resource, @@ -647,24 +649,29 @@ object Dispatcher { notify() } - def take: F[A] = { - val latchF = Async[F].asyncCheckAttempt[Unit] { k => - Sync[F] delay { - if (latchR.compareAndSet(null, k)) { - Left(Some(Applicative[F].unit)) // all cleanup is elsewhere - } else { - // since this is a single consumer queue, we should never have multiple callbacks - // assert(latchR.get() == Signal) - latchR.set(null) - Right(()) + def take: F[A] = Async[F].cont[Unit, A] { + new Cont[F, Unit, A] { + def apply[G[_]: MonadCancelThrow] = { (k, get, lift) => + MonadCancel[G].uncancelable { poll => + val takeF = lift(Sync[F].delay(buffer.take())) + + val latchF = lift(Sync[F].delay(latchR.compareAndSet(null, k))).flatTap { + case true => poll(get) // no `onCancel`, all cleanup is elsewhere + case false => + // since this is a single consumer queue, we should never have multiple callbacks + // assert(latchR.get() == Signal) + lift(Sync[F].delay(latchR.set(null))) + } + + takeF.handleErrorWith { _ => // emptiness is reported as a FailureSignal error + latchF *> takeF.handleErrorWith { _ => + latchF *> takeF // guaranteed to succeed + } + } } } } - - MonadCancel[F] uncancelable { poll => - // emptiness is reported as a FailureSignal error - Sync[F].delay(buffer.take()).handleErrorWith(_ => poll(latchF *> take)) - } } + } } From 2942847855f06ec6d0d6066eb63eaeb7f874cf6a Mon Sep 17 00:00:00 2001 From: Arman Bilge Date: Fri, 19 Jan 2024 20:33:24 +0000 Subject: [PATCH 31/47] Simplify `UnsafeAsyncQueue` --- .../scala/cats/effect/std/Dispatcher.scala | 47 +++---------------- 1 file changed, 6 insertions(+), 41 deletions(-) diff --git a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala index 21fbe47238..bf90515c22 100644 --- a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala +++ b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala @@ -34,7 +34,6 @@ import cats.effect.kernel.syntax.all._ import cats.effect.std.Dispatcher.parasiticEC import cats.syntax.all._ -import scala.annotation.tailrec import scala.concurrent.{ExecutionContext, Future, Promise} import scala.util.Failure @@ -607,46 +606,18 @@ object Dispatcher { } } - private val Signal: Either[Any, Unit] => Unit = _ => () private val RightUnit: Right[Nothing, Unit] = Right(()) // MPSC assumption private final class UnsafeAsyncQueue[F[_]: Async, A] { private[this] val buffer = new UnsafeUnbounded[A]() - /* - * State machine: - * - * - null => no taker, buffer state undefined - * - Signal => no (unnotified) taker, buffer state likely non-empty - * - other => taker, buffer state empty - */ private[this] val latchR = new AtomicReference[Either[Throwable, Unit] => Unit](null) def unsafeOffer(a: A): Unit = { val _ = buffer.put(a) - - @tailrec - def notify(): Unit = { - latchR.get() match { - case null => - if (!latchR.compareAndSet(null, Signal)) { - // someone suspended while we were looking, retry notification - notify() - } - - // in this case, someone else already awakened the taker, so we're good - case Signal => () - - case f => - // failed cas is fine, since it means some other producer woke up the consumer - if (latchR.compareAndSet(f, Signal)) { - f(RightUnit) - } - } - } - - notify() + val back = latchR.getAndSet(null) + if (back ne null) back(RightUnit) } def take: F[A] = Async[F].cont[Unit, A] { @@ -654,18 +625,12 @@ object Dispatcher { def apply[G[_]: MonadCancelThrow] = { (k, get, lift) => MonadCancel[G].uncancelable { poll => val takeF = lift(Sync[F].delay(buffer.take())) - - val latchF = lift(Sync[F].delay(latchR.compareAndSet(null, k))).flatTap { - case true => poll(get) // no `onCancel`, all cleanup is elsewhere - case false => - // since this is a single consumer queue, we should never have multiple callbacks - // assert(latchR.get() == Signal) - lift(Sync[F].delay(latchR.set(null))) - } + val setLatchF = lift(Sync[F].delay(latchR.set(k))) + val unsetLatchF = lift(Sync[F].delay(latchR.set(null))) takeF.handleErrorWith { _ => // emptiness is reported as a FailureSignal error - latchF *> takeF.handleErrorWith { _ => - latchF *> takeF // guaranteed to succeed + setLatchF *> (takeF <* unsetLatchF).handleErrorWith { _ => // double-check + poll(get) *> takeF // guaranteed to succeed } } } From 70f16eb928cde14a2856e51a33bf07fd9b1e5ffa Mon Sep 17 00:00:00 2001 From: Arman Bilge Date: Fri, 19 Jan 2024 20:50:38 +0000 Subject: [PATCH 32/47] Optimize `UnsafeAsyncQueue` --- std/shared/src/main/scala/cats/effect/std/Dispatcher.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala index bf90515c22..384cb5ecd2 100644 --- a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala +++ b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala @@ -609,10 +609,10 @@ object Dispatcher { private val RightUnit: Right[Nothing, Unit] = Right(()) // MPSC assumption - private final class UnsafeAsyncQueue[F[_]: Async, A] { - private[this] val buffer = new UnsafeUnbounded[A]() + private final class UnsafeAsyncQueue[F[_]: Async, A] + extends AtomicReference[Either[Throwable, Unit] => Unit](null) { latchR => - private[this] val latchR = new AtomicReference[Either[Throwable, Unit] => Unit](null) + private[this] val buffer = new UnsafeUnbounded[A]() def unsafeOffer(a: A): Unit = { val _ = buffer.put(a) From 0e01519aa9c46b35e7f42495f4c357787d04f651 Mon Sep 17 00:00:00 2001 From: Daniel Spiewak Date: Fri, 26 Jan 2024 20:11:50 -0600 Subject: [PATCH 33/47] Fixed new cancelation tests and fixed functionality --- .../scala/cats/effect/std/Dispatcher.scala | 23 ++++++--- .../cats/effect/std/DispatcherSpec.scala | 49 +++++++++---------- 2 files changed, 41 insertions(+), 31 deletions(-) diff --git a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala index 384cb5ecd2..240f75b52f 100644 --- a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala +++ b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala @@ -388,6 +388,7 @@ object Dispatcher { // the signal is just a skolem for the atomic references; we never actually run it private final class Worker[F[_]: Async]( val queue: UnsafeAsyncQueue[F, Registration[F]], + supervisor: Supervisor[F], executor: Executor[F], terminationLatch: Deferred[F, Unit]) { @@ -427,7 +428,10 @@ object Dispatcher { } else { reg.stateR.get() match { case RegState.CancelRequested(latch) => - cancelF.guarantee(Sync[F].delay(latch.success(())).void) + supervisor + .supervise( + cancelF.guarantee(Sync[F].delay(latch.success(())).void)) + .void case RegState.Completed => Applicative[F].unit @@ -448,7 +452,7 @@ object Dispatcher { } case Registration.Finalizer(action) => - action + supervisor.supervise(action).void case Registration.PoisonPill() => Sync[F].delay(doneR.set(true)) @@ -467,10 +471,17 @@ object Dispatcher { def apply[F[_]: Async]( executor: Executor[F], terminationLatch: Deferred[F, Unit]): Resource[F, Worker[F]] = { - val initF = Sync[F].delay( - new Worker[F](new UnsafeAsyncQueue[F, Registration[F]](), executor, terminationLatch)) - - Resource.make(initF)(w => Sync[F].delay(w.queue.unsafeOffer(Registration.PoisonPill()))) + // we make a new supervisor just for cancelation actions + Supervisor[F](false) flatMap { supervisor => + val initF = Sync[F].delay( + new Worker[F]( + new UnsafeAsyncQueue[F, Registration[F]](), + supervisor, + executor, + terminationLatch)) + + Resource.make(initF)(w => Sync[F].delay(w.queue.unsafeOffer(Registration.PoisonPill()))) + } } } diff --git a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala index 418c74ea48..c8627e5eb2 100644 --- a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala +++ b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala @@ -330,40 +330,39 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { } "cancelation does not block a worker" in real { - dispatcher - .use { runner => - val clogUp = IO.deferred[Unit].flatMap { gate => - IO { - val task = gate.complete(()) *> IO.never[Unit].uncancelable - runner.unsafeToFutureCancelable(task)._2 - }.flatMap { cancel => - // wait until task starts, then cancel - gate.get *> IO(cancel()) + TestControl executeEmbed { + dispatcher use { runner => + (IO.deferred[Unit], IO.deferred[Unit]) flatMapN { (latch1, latch2) => + val task = (latch1.complete(()) *> latch2.get).uncancelable + + IO(runner.unsafeToFutureCancelable(task)._2) flatMap { cancel => + latch1.get *> + IO(cancel()) *> + IO(runner.unsafeRunAndForget(latch2.complete(()))) *> + latch2.get.as(ok) } } - clogUp.parReplicateA_(1000) *> - // now try to run a new task - IO.fromFuture(IO(runner.unsafeToFuture(IO.unit))) } - .replicateA_(if (isJVM) 1000 else 1) - .as(ok) + } } "cancelation race does not block a worker" in real { dispatcher .use { runner => - val clogUp = IO { - val task = IO.never[Unit].uncancelable - runner.unsafeToFutureCancelable(task)._2 - }.flatMap { cancel => - // cancel concurrently - // We want to trigger race condition where task starts but then discovers it was canceled - IO(cancel()) - } + IO.deferred[Unit] flatMap { latch => + val clogUp = IO { + val task = latch.get.uncancelable + runner.unsafeToFutureCancelable(task)._2 + }.flatMap { cancel => + // cancel concurrently + // We want to trigger race condition where task starts but then discovers it was canceled + IO(cancel()) + } - clogUp.parReplicateA_(1000) *> - // now try to run a new task - IO.fromFuture(IO(runner.unsafeToFuture(IO.unit))) + clogUp.parReplicateA_(1000) *> + // now try to run a new task + IO.fromFuture(IO(runner.unsafeToFuture(latch.complete(())))) + } } .replicateA_(if (isJVM) 1000 else 1) .as(ok) From e105187cffeb3556367e174118c4c984a04423b7 Mon Sep 17 00:00:00 2001 From: Daniel Urban Date: Mon, 29 Jan 2024 21:09:25 +0100 Subject: [PATCH 34/47] Add (sometimes) failing test --- .../test/scala/cats/effect/std/DispatcherSpec.scala | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala index c8627e5eb2..6d78881368 100644 --- a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala +++ b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala @@ -570,6 +570,17 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { } } + "complete / cancel race" in real { + val tsk = dispatcher.use { dispatcher => + IO.fromFuture(IO { + val (_, cancel) = dispatcher.unsafeToFutureCancelable(IO.unit) + val cancelFut = cancel() + cancelFut + }) + } + + tsk.replicateA_(if (isJVM) 10000 else 1).as(ok) + } } } From c959d93c18a0dfb4aba4f6b2138e4d873f7e3c88 Mon Sep 17 00:00:00 2001 From: Daniel Urban Date: Mon, 29 Jan 2024 21:32:45 +0100 Subject: [PATCH 35/47] Try to fix the bug --- .../scala/cats/effect/std/Dispatcher.scala | 29 +++++++++++++------ 1 file changed, 20 insertions(+), 9 deletions(-) diff --git a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala index 240f75b52f..652e92e1e2 100644 --- a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala +++ b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala @@ -296,10 +296,21 @@ object Dispatcher { // forward atomicity guarantees onto promise completion val promisory = MonadCancel[F] uncancelable { poll => // invalidate the cancel action when we're done - poll(fe.guarantee(Sync[F].delay(stateR.set(RegState.Completed)))) - .redeemWith( - e => Sync[F].delay(result.failure(e)), - a => Sync[F].delay(result.success(a))) + val completeState = Sync[F].delay { + stateR.getAndSet(RegState.Completed) match { + case RegState.CancelRequested(latch) => + // we already have a cancel, must complete it: + latch.trySuccess(()) // `try` because we're racing with `Worker` + () + case RegState.Unstarted | RegState.Running(_) => + () + case RegState.Completed => + throw new AssertionError("unexpected Completed state") + } + } + poll(fe.guarantee(completeState)).redeemWith( + e => Sync[F].delay(result.failure(e)), + a => Sync[F].delay(result.success(a))) } val worker = @@ -416,10 +427,8 @@ object Dispatcher { } } } else { - val withCompletion = - action.guarantee(Sync[F].delay(reg.stateR.set(RegState.Completed))) - executor(withCompletion) { cancelF => + executor(action) { cancelF => Sync[F] defer { if (reg .stateR @@ -429,8 +438,10 @@ object Dispatcher { reg.stateR.get() match { case RegState.CancelRequested(latch) => supervisor - .supervise( - cancelF.guarantee(Sync[F].delay(latch.success(())).void)) + .supervise(cancelF.guarantee(Sync[F].delay { + latch.trySuccess(()) + () + })) .void case RegState.Completed => From d96fdca4c0b20cf89558541e30f921f61f271c87 Mon Sep 17 00:00:00 2001 From: Daniel Spiewak Date: Mon, 29 Jan 2024 15:38:06 -0600 Subject: [PATCH 36/47] Update std/shared/src/main/scala/cats/effect/std/Dispatcher.scala Co-authored-by: Daniel Urban --- std/shared/src/main/scala/cats/effect/std/Dispatcher.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala index 240f75b52f..8878ef110b 100644 --- a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala +++ b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala @@ -280,7 +280,7 @@ object Dispatcher { workersF evalMap { workers => Async[F].executionContext flatMap { ec => val launchAll = 0.until(workers.length).toList traverse_ { i => - supervisor.supervise(workers(i).run).void + supervisor.supervise(workers(i).run) } launchAll.as(new Dispatcher[F] { From f145a94cce516875344eb2e7f5933c71c9999d20 Mon Sep 17 00:00:00 2001 From: Daniel Spiewak Date: Mon, 29 Jan 2024 15:38:18 -0600 Subject: [PATCH 37/47] Update std/shared/src/main/scala/cats/effect/std/Dispatcher.scala Co-authored-by: Daniel Urban --- std/shared/src/main/scala/cats/effect/std/Dispatcher.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala index 8878ef110b..7640ea6bbd 100644 --- a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala +++ b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala @@ -317,6 +317,7 @@ object Dispatcher { val reg = new Registration.Primary(promisory.void, stateR) worker.queue.unsafeOffer(reg) + @tailrec def cancel(): Future[Unit] = { stateR.get() match { case RegState.Unstarted => From 0675dae7dd752ba605e7b9d90a3e6fef9d7c2647 Mon Sep 17 00:00:00 2001 From: Daniel Spiewak Date: Mon, 29 Jan 2024 16:17:47 -0600 Subject: [PATCH 38/47] Added missing import --- std/shared/src/main/scala/cats/effect/std/Dispatcher.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala index 23d53719dc..3cb1b899c6 100644 --- a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala +++ b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala @@ -34,6 +34,7 @@ import cats.effect.kernel.syntax.all._ import cats.effect.std.Dispatcher.parasiticEC import cats.syntax.all._ +import scala.annotation.tailrec import scala.concurrent.{ExecutionContext, Future, Promise} import scala.util.Failure From a5aaa962ecf845aaa725fdbe30476b8bda7a03d6 Mon Sep 17 00:00:00 2001 From: Arman Bilge Date: Sat, 3 Feb 2024 17:24:18 +0000 Subject: [PATCH 39/47] Fix `UnsafeAsyncQueue` --- .../scala/cats/effect/std/Dispatcher.scala | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala index 3cb1b899c6..b6e15cb9e5 100644 --- a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala +++ b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala @@ -629,25 +629,24 @@ object Dispatcher { def unsafeOffer(a: A): Unit = { val _ = buffer.put(a) - val back = latchR.getAndSet(null) + val back = latchR.get() if (back ne null) back(RightUnit) } def take: F[A] = Async[F].cont[Unit, A] { new Cont[F, Unit, A] { def apply[G[_]: MonadCancelThrow] = { (k, get, lift) => - MonadCancel[G].uncancelable { poll => - val takeF = lift(Sync[F].delay(buffer.take())) - val setLatchF = lift(Sync[F].delay(latchR.set(k))) - val unsetLatchF = lift(Sync[F].delay(latchR.set(null))) - - takeF.handleErrorWith { _ => // emptiness is reported as a FailureSignal error - setLatchF *> (takeF <* unsetLatchF).handleErrorWith { _ => // double-check - poll(get) *> takeF // guaranteed to succeed - } + val takeG = lift(Sync[F].delay(buffer.take())) + val setLatchG = lift(Sync[F].delay(latchR.set(k))) + val unsetLatchG = lift(Sync[F].delay(latchR.lazySet(null))) + + takeG.handleErrorWith { _ => // emptiness is reported as a FailureSignal error + setLatchG *> (takeG <* unsetLatchG).handleErrorWith { _ => // double-check + get *> unsetLatchG *> lift(take) // recurse } } } + } } From 2e9ee31586c13a688f789b80d8dde00cdaaf3842 Mon Sep 17 00:00:00 2001 From: Daniel Spiewak Date: Sat, 3 Feb 2024 12:29:47 -0600 Subject: [PATCH 40/47] Fixed gadt issues on 2.12 --- .../src/main/scala/cats/effect/std/Dispatcher.scala | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala index 4bbb3fe2e2..dd36860998 100644 --- a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala +++ b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala @@ -299,14 +299,16 @@ object Dispatcher { // invalidate the cancel action when we're done val completeState = Sync[F].delay { stateR.getAndSet(RegState.Completed) match { - case RegState.CancelRequested(latch) => + case st: RegState.CancelRequested[_] => // we already have a cancel, must complete it: - latch.trySuccess(()) // `try` because we're racing with `Worker` - () - case RegState.Unstarted | RegState.Running(_) => + st.latch.trySuccess(()) // `try` because we're racing with `Worker` () + case RegState.Completed => throw new AssertionError("unexpected Completed state") + + case _ => + () } } poll(fe.guarantee(completeState)).redeemWith( From b3d86fcacfdf9de374b52d96debbaba037d0b7f0 Mon Sep 17 00:00:00 2001 From: Daniel Urban Date: Mon, 5 Feb 2024 03:56:02 +0100 Subject: [PATCH 41/47] Handle stateR more precisely --- .../scala/cats/effect/std/Dispatcher.scala | 52 ++++++++++++------- 1 file changed, 33 insertions(+), 19 deletions(-) diff --git a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala index dd36860998..197bd79bbb 100644 --- a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala +++ b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala @@ -16,7 +16,7 @@ package cats.effect.std -import cats.Applicative +import cats.{Applicative, MonadThrow} import cats.effect.kernel.{ Async, Concurrent, @@ -301,7 +301,7 @@ object Dispatcher { stateR.getAndSet(RegState.Completed) match { case st: RegState.CancelRequested[_] => // we already have a cancel, must complete it: - st.latch.trySuccess(()) // `try` because we're racing with `Worker` + st.latch.success(()) () case RegState.Completed => @@ -420,15 +420,19 @@ object Dispatcher { if (action == null) { // this corresponds to a memory race where we see action's write before stateR's val check = Spawn[F].cede *> Sync[F].delay(reg.stateR.get()) - check.iterateWhile(_ == RegState.Unstarted) *> Sync[F].delay { - reg.stateR.get() match { - case RegState.CancelRequested(latch) => - latch.success(()) - () - - case s => - throw new AssertionError(s"a => $s") - } + check.iterateWhile(_ == RegState.Unstarted).flatMap { + case cr @ RegState.CancelRequested(latch) => + Sync[F].delay { + if (reg.stateR.compareAndSet(cr, RegState.Completed)) { + latch.success(()) + () + } else { + val s = reg.stateR.get() + throw new AssertionError(s"d => $s") + } + } + case s => + MonadThrow[F].raiseError[Unit](new AssertionError(s"a => $s")) } } else { @@ -440,18 +444,28 @@ object Dispatcher { Applicative[F].unit } else { reg.stateR.get() match { - case RegState.CancelRequested(latch) => - supervisor - .supervise(cancelF.guarantee(Sync[F].delay { - latch.trySuccess(()) - () - })) - .void + case cr @ RegState.CancelRequested(latch) => + if (reg.stateR.compareAndSet(cr, RegState.Running(cancelF))) { + supervisor + .supervise(cancelF.guarantee(Sync[F].delay { + latch.success(()) + () + })) + .void + } else { + reg.stateR.get() match { + case RegState.Completed => + Applicative[F].unit + case s => + throw new AssertionError(s"e => $s") + } + } case RegState.Completed => Applicative[F].unit - case s => throw new AssertionError(s"b => $s") + case s => + throw new AssertionError(s"b => $s") } } } From d01ddfb9fd7d3a700090ae36464301d9c2308090 Mon Sep 17 00:00:00 2001 From: Daniel Spiewak Date: Sat, 10 Feb 2024 11:56:21 -0600 Subject: [PATCH 42/47] Avoid weirdly depending on cancelation semantics in parallel deadlock test --- .../shared/src/test/scala/cats/effect/std/DispatcherSpec.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala index ca10194e11..b4631c2b7f 100644 --- a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala +++ b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala @@ -285,7 +285,7 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { } } - rec.use(_ => IO.unit) + rec.use(_ => awaitAll) } } yield ok } From 63164ef06ca6211146afdf4759f0cff85cc9e77b Mon Sep 17 00:00:00 2001 From: Daniel Spiewak Date: Sat, 10 Feb 2024 13:13:34 -0600 Subject: [PATCH 43/47] Unconditionally eat supervisor errors on finalizers --- std/shared/src/main/scala/cats/effect/std/Dispatcher.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala index 197bd79bbb..fdb4f7162c 100644 --- a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala +++ b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala @@ -481,7 +481,7 @@ object Dispatcher { } case Registration.Finalizer(action) => - supervisor.supervise(action).void + supervisor.supervise(action).void.voidError case Registration.PoisonPill() => Sync[F].delay(doneR.set(true)) From a3c285dcbe9134394efe71d70957949d23de18ac Mon Sep 17 00:00:00 2001 From: Daniel Urban Date: Wed, 21 Feb 2024 01:29:48 +0100 Subject: [PATCH 44/47] Add failing test --- .../cats/effect/std/DispatcherSpec.scala | 24 +++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala index b4631c2b7f..d66d34a94d 100644 --- a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala +++ b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala @@ -71,6 +71,30 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { appends *> resultsR.get.flatMap(r => IO(r mustEqual 0.until(count).toVector)) } } + + "correctly backpressure cancellation" in real { + D.use { dispatcher => + IO.ref(0).flatMap { ctr1 => + IO.ref(0).flatMap { ctr2 => + IO.fromFuture(IO { + val (_, cancel) = dispatcher.unsafeToFutureCancelable(IO.uncancelable { _ => + ctr1.update(_ + 1) *> IO.sleep(0.1.second) *> ctr2.update(_ + 1) + }) + val cancelFut = cancel() + cancelFut + }).flatMap { _ => + // if we're here, `cancel()` finished, so + // either the task didn't run at all (i.e., + // it was cancelled before starting), or + // it ran and already finished completely: + (ctr1.get, ctr2.get).flatMapN { (v1, v2) => + IO(v1 mustEqual v2) + } + } + } + } + }.replicateA_(10000).as(ok) + } } "await = false" >> { From 6a9cfa617676887dd34ae450aff05dac0ddb5072 Mon Sep 17 00:00:00 2001 From: Daniel Urban Date: Wed, 21 Feb 2024 02:15:29 +0100 Subject: [PATCH 45/47] Fix it --- .../src/main/scala/cats/effect/std/Dispatcher.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala index fdb4f7162c..b96b20fd34 100644 --- a/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala +++ b/std/shared/src/main/scala/cats/effect/std/Dispatcher.scala @@ -521,12 +521,12 @@ object Dispatcher { private object Executor { // default sequential executor (ignores cancelation) - def inplace[F[_]: Applicative]: Executor[F] = + def inplace[F[_]: Concurrent]: Executor[F] = new Executor[F] { def apply(task: F[Unit])(registerCancel: F[Unit] => F[Unit]): F[Unit] = { - // we can use unit as a cancel action here since it must always sequence *after* the task - // thus, the task must complete before the cancel action will be picked up - registerCancel(Applicative[F].unit) *> task + Concurrent[F].deferred[Unit].flatMap { d => + (registerCancel(d.get) *> task).guarantee(d.complete(()).void) + } } } From a5d85dd41c1447042f6b13c8db9b1166efe6b01d Mon Sep 17 00:00:00 2001 From: Daniel Urban Date: Wed, 21 Feb 2024 02:17:21 +0100 Subject: [PATCH 46/47] scalafmt --- .../src/test/scala/cats/effect/std/DispatcherSpec.scala | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala index d66d34a94d..69a0c17bd3 100644 --- a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala +++ b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala @@ -87,13 +87,12 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { // either the task didn't run at all (i.e., // it was cancelled before starting), or // it ran and already finished completely: - (ctr1.get, ctr2.get).flatMapN { (v1, v2) => - IO(v1 mustEqual v2) - } + (ctr1.get, ctr2.get).flatMapN { (v1, v2) => IO(v1 mustEqual v2) } } } } - }.replicateA_(10000).as(ok) + }.replicateA_(10000) + .as(ok) } } From 31b24f4477b6581442a5147f54022f099fc68d7f Mon Sep 17 00:00:00 2001 From: Daniel Urban Date: Wed, 21 Feb 2024 02:41:44 +0100 Subject: [PATCH 47/47] Only replicateA_ on JVM --- .../shared/src/test/scala/cats/effect/std/DispatcherSpec.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala index 69a0c17bd3..22bd124e18 100644 --- a/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala +++ b/tests/shared/src/test/scala/cats/effect/std/DispatcherSpec.scala @@ -91,7 +91,7 @@ class DispatcherSpec extends BaseSpec with DetectPlatform { } } } - }.replicateA_(10000) + }.replicateA_(if (isJVM) 10000 else 1) .as(ok) } }