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
When you have a task returning a Ray Object, and the task is in "waiting dependencies" status; and you do ray.cancel(ref) on the ref. You expect the next ray.get(ref) raises TaskCancelledError but it hangs now.
This issue also applies for streaming generators, and when the task is resubmitted via lineage reconstruction.
// This case is reached for tasks that have unresolved dependencies.
// No executing tasks, so cancelling is a noop.
where if a task being cancelled have unresolved deps, it does noop.
This may be true in 4 yrs ago, but now we need to fail the task and mark all objects as failed with the exception.
Versions / Dependencies
master
Reproduction script
Run this script multiple times, it's 70% hanging and 30% passing.
importrayimportnumpyasnpimportpytestfromray._private.test_utilsimportSignalActorimporttimedeftest_cancel_pending_arg_running():
withray.init():
@ray.remote(max_retries=-1)defwait_forever():
print("wait_and_reset starting")
whileTrue:
time.sleep(10000)
@ray.remote(max_retries=-1)defhas_deps(may_block):
big=may_blockreturnbig*2may_block=wait_forever.remote()
ref=has_deps.remote(may_block)
print(f"{ref=}")
ready, not_ready=ray.wait([may_block, ref], timeout=1)
assertnotreadyassertlen(not_ready) ==2print(f"both are not ready: {may_block=}, {ref=}")
# Now, the generator is pending arg. Cancel it.print("Cancelling")
ray.cancel(ref)
print("Getting")
withpytest.raises(ray.exceptions.TaskCancelledError):
ray.get(ref) # This should raise TaskCancelledError, but hangsif__name__=="__main__":
importos,sysifos.environ.get("PARALLEL_CI"):
sys.exit(pytest.main(["-n", "auto", "--boxed", "-vs", __file__]))
else:
sys.exit(pytest.main(["-sv", __file__]))
Issue Severity
High: It blocks me from completing my task.
The text was updated successfully, but these errors were encountered:
rynewang
added
bug
Something that is supposed to be working; but isn't
triage
Needs triage (eg: priority, bug/not-bug, and owning component)
labels
Jun 28, 2024
jjyao
added
core
Issues that should be addressed in Ray Core
P0
Issues that should be fixed in short order
and removed
triage
Needs triage (eg: priority, bug/not-bug, and owning component)
labels
Jun 28, 2024
What happened + What you expected to happen
When you have a task returning a Ray Object, and the task is in "waiting dependencies" status; and you do
ray.cancel(ref)
on the ref. You expect the nextray.get(ref)
raisesTaskCancelledError
but it hangs now.This issue also applies for streaming generators, and when the task is resubmitted via lineage reconstruction.
This comes from
ray/src/ray/core_worker/transport/direct_task_transport.cc
Lines 798 to 799 in 755a49b
where if a task being cancelled have unresolved deps, it does noop.
This may be true in 4 yrs ago, but now we need to fail the task and mark all objects as failed with the exception.
Versions / Dependencies
master
Reproduction script
Run this script multiple times, it's 70% hanging and 30% passing.
Issue Severity
High: It blocks me from completing my task.
The text was updated successfully, but these errors were encountered: