You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
[core] Fix race condition when canceling task that hasn't started yet (#52703)
See linked issue for the original symptom and reproduction.
When a `CancelTask` RPC message is received, we need to handle 4
possible cases:
1. The `PushTask` RPC hasn't been received yet.
2. The `PushTask` RPC has been received but the task isn't executing
yet.
3. The `PushTask` RPC has been received and the task is now executing.
4. The task finished executing and the `PushTask` RPC reply has been
sent.
The code currently handles (1) and (4) by relying on client-side
retries: we return `success=False` and expect the client to retry the
cancellation (unless the task has already finished in case (4), which it
knows).
However, there is a race condition between cases (2) and (3) where the
task is no longer considered queued in the
`OutOfOrderActorSchedulingQueue`, but it hasn't actually started
executing yet and therefore there is no future to cancel. This can
happen because:
- We [erase the task
ID](https://github.com/ray-project/ray/blob/master/src/ray/core_worker/transport/out_of_order_actor_scheduling_queue.cc#L240)
from the pending map before actually executing the task. After this,
`CancelTaskIfFound` will return false.
- We then post the work to start running the request [to the
io_service_](https://github.com/ray-project/ray/blob/master/src/ray/core_worker/transport/out_of_order_actor_scheduling_queue.cc#L245).
- We post the `RunRequest` callback that eventually runs the task [to
the fiber
thread](https://github.com/ray-project/ray/blob/master/src/ray/core_worker/transport/out_of_order_actor_scheduling_queue.cc#L156).
- The logic to cancel the task runs on the
[task_execution_service_](https://github.com/ray-project/ray/blob/master/src/ray/core_worker/core_worker.cc#L4485).
This means there is no guarantee that the task has actually started to
execute when we call
[cancel_async_task_](https://github.com/ray-project/ray/blob/master/src/ray/core_worker/core_worker.cc#L4462).
This PR fixes the problem by extending the reliance on client retries:
we return a boolean from `cancel_async_task_` that indicates if the task
was cancelled. If not, it's up to the client to retry.
The proper long-term fix would be to serialize the executions and
cancellations inside of the scheduling queue / task executor, but that
will require a lot of refactoring work. We need to simplify the
concurrency model in these classes.
Closes#52628
---------
Signed-off-by: Edward Oakes <ed.nmi.oakes@gmail.com>
0 commit comments