forked from ray-project/ray
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Revert "Revert "[core] Support generators for tasks with multiple ret…
…urn values (ray-project#25247)" (ray-project#25380)" This reverts commit 80168a0.
- Loading branch information
1 parent
cb151d5
commit 74408f1
Showing
6 changed files
with
232 additions
and
4 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,96 @@ | ||
.. _generators: | ||
|
||
Pattern: Using generators to reduce heap memory usage | ||
===================================================== | ||
|
||
In this pattern, we use **generators** in Python to reduce the total heap memory usage during a task. The key idea is that for tasks that return multiple objects, we can return them one at a time instead of all at once. This allows a worker to free the heap memory used by a previous return value before returning the next one. | ||
|
||
Example use case | ||
---------------- | ||
|
||
You have a task that returns multiple large values. Another possibility is a task that returns a single large value, but you want to stream this value through Ray's object store by breaking it up into smaller chunks. | ||
|
||
Using normal Python functions, we can write such a task like this. Here's an example that returns numpy arrays of size 100MB each: | ||
|
||
.. code-block:: python | ||
import numpy as np | ||
@ray.remote | ||
def large_values(num_returns): | ||
return [ | ||
np.random.randint( | ||
np.iinfo(np.int8).max, size=(100_000_000, 1), dtype=np.int8 | ||
) | ||
for _ in range(num_returns) | ||
] | ||
However, this will require the task to hold all ``num_returns`` arrays in heap memory at the same time at the end of the task. If there are many return values, this can lead to high heap memory usage and potentially an out-of-memory error. | ||
|
||
We can fix the above example by rewriting ``large_values`` as a **generator**. Instead of returning all values at once as a tuple or list, we can ``yield`` one value at a time. | ||
|
||
.. code-block:: python | ||
@ray.remote | ||
def large_values_generator(num_returns): | ||
for _ in range(num_returns): | ||
yield np.random.randint( | ||
np.iinfo(np.int8).max, size=(100_000_000, 1), dtype=np.int8 | ||
) | ||
Code example | ||
------------ | ||
|
||
.. code-block:: python | ||
import numpy as np | ||
import ray | ||
@ray.remote(max_retries=0) | ||
def large_values(num_returns): | ||
return [ | ||
np.random.randint( | ||
np.iinfo(np.int8).max, size=(100_000_000, 1), dtype=np.int8 | ||
) | ||
for _ in range(num_returns) | ||
] | ||
@ray.remote(max_retries=0) | ||
def large_values_generator(num_returns): | ||
for i in range(num_returns): | ||
yield np.random.randint( | ||
np.iinfo(np.int8).max, size=(100_000_000, 1), dtype=np.int8 | ||
) | ||
print(f"yielded return value {i}") | ||
num_returns = 100 | ||
# Worker will likely OOM using normal returns. | ||
print("Using normal functions...") | ||
try: | ||
ray.get(large_values.options(num_returns=num_returns).remote(num_returns)[0]) | ||
except ray.exceptions.WorkerCrashedError: | ||
print("Worker failed with normal function") | ||
# Using a generator will allow the worker to finish. | ||
# Note that this will block until the full task is complete, i.e. the | ||
# last yield finishes. | ||
print("Using generators...") | ||
ray.get( | ||
large_values_generator.options(num_returns=num_returns).remote(num_returns)[0] | ||
) | ||
print("Success!") | ||
.. code-block:: text | ||
$ RAY_IGNORE_UNHANDLED_ERRORS=1 python test.py | ||
Using normal functions... | ||
... -- A worker died or was killed while executing a task by an unexpected system error. To troubleshoot the problem, check the logs for the dead worker... | ||
Worker failed | ||
Using generators... | ||
(large_values_generator pid=373609) yielded return value 0 | ||
(large_values_generator pid=373609) yielded return value 1 | ||
(large_values_generator pid=373609) yielded return value 2 | ||
... | ||
Success! | ||
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,98 @@ | ||
import pytest | ||
import numpy as np | ||
import sys | ||
|
||
import ray | ||
|
||
|
||
def test_generator_oom(ray_start_regular): | ||
@ray.remote(max_retries=0) | ||
def large_values(num_returns): | ||
return [ | ||
np.random.randint( | ||
np.iinfo(np.int8).max, size=(100_000_000, 1), dtype=np.int8 | ||
) | ||
for _ in range(num_returns) | ||
] | ||
|
||
@ray.remote(max_retries=0) | ||
def large_values_generator(num_returns): | ||
for _ in range(num_returns): | ||
yield np.random.randint( | ||
np.iinfo(np.int8).max, size=(100_000_000, 1), dtype=np.int8 | ||
) | ||
|
||
num_returns = 100 | ||
try: | ||
# Worker may OOM using normal returns. | ||
ray.get(large_values.options(num_returns=num_returns).remote(num_returns)[0]) | ||
except ray.exceptions.WorkerCrashedError: | ||
pass | ||
|
||
# Using a generator will allow the worker to finish. | ||
ray.get( | ||
large_values_generator.options(num_returns=num_returns).remote(num_returns)[0] | ||
) | ||
|
||
|
||
@pytest.mark.parametrize("use_actors", [False, True]) | ||
def test_generator_returns(ray_start_regular, use_actors): | ||
remote_generator_fn = None | ||
if use_actors: | ||
|
||
@ray.remote | ||
class Generator: | ||
def __init__(self): | ||
pass | ||
|
||
def generator(self, num_returns): | ||
for i in range(num_returns): | ||
yield i | ||
|
||
g = Generator.remote() | ||
remote_generator_fn = g.generator | ||
else: | ||
|
||
@ray.remote(max_retries=0) | ||
def generator(num_returns): | ||
for i in range(num_returns): | ||
yield i | ||
|
||
remote_generator_fn = generator | ||
|
||
# Check cases when num_returns does not match the number of values returned | ||
# by the generator. | ||
num_returns = 3 | ||
|
||
try: | ||
ray.get( | ||
remote_generator_fn.options(num_returns=num_returns).remote(num_returns - 1) | ||
) | ||
assert False | ||
except ray.exceptions.RayTaskError as e: | ||
assert isinstance(e.as_instanceof_cause(), ValueError) | ||
|
||
try: | ||
ray.get( | ||
remote_generator_fn.options(num_returns=num_returns).remote(num_returns + 1) | ||
) | ||
assert False | ||
except ray.exceptions.RayTaskError as e: | ||
assert isinstance(e.as_instanceof_cause(), ValueError) | ||
|
||
# Check num_returns=1 case, should receive TypeError because generator | ||
# cannot be pickled. | ||
try: | ||
ray.get(remote_generator_fn.remote(num_returns)) | ||
assert False | ||
except ray.exceptions.RayTaskError as e: | ||
assert isinstance(e.as_instanceof_cause(), TypeError) | ||
|
||
# Check return values. | ||
ray.get( | ||
remote_generator_fn.options(num_returns=num_returns).remote(num_returns) | ||
) == list(range(num_returns)) | ||
|
||
|
||
if __name__ == "__main__": | ||
sys.exit(pytest.main(["-v", __file__])) |