doc/source/ray-core/patterns/ray-get-too-many-objects.rst
.. _ray-get-too-many-objects:
TLDR: Avoid calling :func:ray.get() <ray.get> on too many objects since this will lead to heap out-of-memory or object store out-of-space. Instead fetch and process one batch at a time.
If you have a large number of tasks that you want to run in parallel, trying to do ray.get() on all of them at once could lead to failure with heap out-of-memory or object store out-of-space since Ray needs to fetch all the objects to the caller at the same time.
Instead you should get and process the results one batch at a time. Once a batch is processed, Ray will evict objects in that batch to make space for future batches.
.. figure:: ../images/ray-get-too-many-objects.svg
Fetching too many objects at once with ``ray.get()``
Anti-pattern:
.. literalinclude:: ../doc_code/anti_pattern_ray_get_too_many_objects.py :language: python :start-after: anti_pattern_start :end-before: anti_pattern_end
Better approach:
.. literalinclude:: ../doc_code/anti_pattern_ray_get_too_many_objects.py :language: python :start-after: better_approach_start :end-before: better_approach_end
Here besides getting one batch at a time to avoid failure, we are also using ray.wait() to process results in the finish order instead of the submission order to reduce the runtime. See :doc:ray-get-submission-order for more details.