Skip to content

[docs] Add antipattern for nested ray.get #43184

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 9 commits into from
Jun 2, 2025
Merged
Show file tree
Hide file tree
Changes from 7 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
27 changes: 27 additions & 0 deletions doc/source/ray-core/doc_code/anti_pattern_nested_ray_get.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,27 @@
# __anti_pattern_start__
import ray
import time


@ray.remote
def f():
return 1


@ray.remote
def pass_via_nested_ref(refs):
print(sum(ray.get(refs)))


@ray.remote
def pass_via_direct_arg(*args):
print(sum(args))


# Anti-pattern: Passing nested refs requires `ray.get` in a nested task.
ray.get(pass_via_nested_ref.remote([f.remote() for _ in range(3)]))

# Better approach: Pass refs as direct arguments. Use *args syntax to unpack
# multiple arguments.
ray.get(pass_via_direct_arg.remote(*[f.remote() for _ in range(3)]))
# __anti_pattern_end__
1 change: 1 addition & 0 deletions doc/source/ray-core/patterns/index.rst
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ This section is a collection of common design patterns and anti-patterns for wri
tree-of-actors
pipelining
return-ray-put
nested-ray-get
ray-get-loop
unnecessary-ray-get
ray-get-submission-order
Expand Down
25 changes: 25 additions & 0 deletions doc/source/ray-core/patterns/nested-ray-get.rst
Original file line number Diff line number Diff line change
@@ -0,0 +1,25 @@
.. _nested-ray-get:

Anti-pattern: Calling ray.get on task arguments harms performance
=================================================================


**TLDR:** If possible, pass ``ObjectRefs`` as direct task arguments, instead of passing a list as the task argument and then calling :func:`ray.get() <ray.get>` inside the task.

When a task calls ``ray.get()``, it must block until the value of the ``ObjectRef`` is ready.
If all cores are already occupied, this situation can lead to a deadlock, as the task that produces the ``ObjectRef``'s value may need the caller task's resources in order to run.
To handle this issue, if the caller task would block in ``ray.get()``, Ray temporarily releases the caller's CPU resources to allow the pending task to run.
This behavior can harm performance and stability because the caller continues to use a process and memory to hold its stack while other tasks run.

Therefore, it is always better to pass ``ObjectRefs`` as direct arguments to a task and avoid calling ``ray.get`` inside of the task, if possible.

For example, in the following code, prefer the latter method of invoking the dependent task.
.. literalinclude:: ../doc_code/anti_pattern_nested_ray_get.py
:language: python
:start-after: __anti_pattern_start__
:end-before: __anti_pattern_end__

Avoiding ``ray.get`` in nested tasks may not always be possible. Some valid reasons to call ``ray.get`` include:

- :doc:`nested-tasks`
- If the nested task has multiple ``ObjectRefs`` to ``ray.get``, and it wants to choose the order and number to get.
2 changes: 1 addition & 1 deletion doc/source/ray-core/patterns/nested-tasks.rst
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
.. _task-pattern-nested-tasks:
.. _nested-tasks:

Pattern: Using nested tasks to achieve nested parallelism
=========================================================
Expand Down
2 changes: 2 additions & 0 deletions doc/source/ray-core/patterns/ray-get-loop.rst
Original file line number Diff line number Diff line change
Expand Up @@ -28,5 +28,7 @@ Instead, we should first schedule all remote calls, which are then processed in

Other ``ray.get()`` related anti-patterns are:

- :doc:`nested-ray-get`
- :doc:`unnecessary-ray-get`
- :doc:`ray-get-submission-order`
- :doc:`ray-get-too-many-objects`
2 changes: 2 additions & 0 deletions doc/source/ray-core/patterns/ray-get-too-many-objects.rst
Original file line number Diff line number Diff line change
@@ -1,3 +1,5 @@
.. _ray-get-too-many-objects:

Anti-pattern: Fetching too many objects at once with ray.get causes failure
===========================================================================

Expand Down