diff --git a/doc/source/ray-core/doc_code/ray_oom_prevention.py b/doc/source/ray-core/doc_code/ray_oom_prevention.py
index 9eead11891ae3..03aa021c56eb5 100644
--- a/doc/source/ray-core/doc_code/ray_oom_prevention.py
+++ b/doc/source/ray-core/doc_code/ray_oom_prevention.py
@@ -7,22 +7,22 @@
},
)
# fmt: off
-# __oom_start__
+# __last_task_start__
import ray
-@ray.remote(max_retries=0)
-def allocate_memory():
+@ray.remote(max_retries=-1)
+def leaks_memory():
chunks = []
- bits_to_allocate = 8 * 100 * 1024 * 1024 # ~0.1 GiB
+ bits_to_allocate = 8 * 100 * 1024 * 1024 # ~100 MiB
while True:
chunks.append([0] * bits_to_allocate)
try:
- ray.get(allocate_memory.remote())
+ ray.get(leaks_memory.remote())
except ray.exceptions.OutOfMemoryError as ex:
print("task failed with OutOfMemoryError, which is expected")
-# __oom_end__
+# __last_task_end__
# fmt: on
@@ -78,10 +78,10 @@ def allocate(self, bytes_to_allocate: float) -> None:
ray.get(first_actor_task)
except ray.exceptions.OutOfMemoryError as ex:
error_thrown = True
- print("first actor was killed by memory monitor")
+ print("First started actor, which is retriable, was killed by the memory monitor.")
assert error_thrown
ray.get(second_actor_task)
-print("finished second actor")
+print("Second started actor, which is not-retriable, finished.")
# __two_actors_end__
# fmt: on
diff --git a/doc/source/ray-core/images/oom_killer_example.svg b/doc/source/ray-core/images/oom_killer_example.svg
new file mode 100644
index 0000000000000..b1977b023522b
--- /dev/null
+++ b/doc/source/ray-core/images/oom_killer_example.svg
@@ -0,0 +1 @@
+
\ No newline at end of file
diff --git a/doc/source/ray-core/images/oom_killer_example_killed_one.svg b/doc/source/ray-core/images/oom_killer_example_killed_one.svg
new file mode 100644
index 0000000000000..31e5f448674e3
--- /dev/null
+++ b/doc/source/ray-core/images/oom_killer_example_killed_one.svg
@@ -0,0 +1 @@
+
\ No newline at end of file
diff --git a/doc/source/ray-core/images/oom_killer_example_killed_two.svg b/doc/source/ray-core/images/oom_killer_example_killed_two.svg
new file mode 100644
index 0000000000000..1ae1d01e3cdd6
--- /dev/null
+++ b/doc/source/ray-core/images/oom_killer_example_killed_two.svg
@@ -0,0 +1 @@
+
\ No newline at end of file
diff --git a/doc/source/ray-core/scheduling/ray-oom-prevention.rst b/doc/source/ray-core/scheduling/ray-oom-prevention.rst
index ca60d826313c0..c3c79828c6367 100644
--- a/doc/source/ray-core/scheduling/ray-oom-prevention.rst
+++ b/doc/source/ray-core/scheduling/ray-oom-prevention.rst
@@ -16,42 +16,22 @@ What is the memory monitor?
The memory monitor is a component that runs within the :ref:`raylet ` process on each node. It periodically checks the memory usage, which includes the worker heap, the object store, and the raylet as described in :ref:`memory management `. If the combined usage exceeds a configurable threshold the raylet will kill a task or actor process to free up memory and prevent Ray from failing.
-.. note::
+It is available on Linux and is tested with Ray running inside a container that is using cgroup v1. If you encounter issues when running the memory monitor outside of a container or the container is using cgroup v2, please :ref:`file an issue or post a question `.
- The memory monitor is in :ref:`beta `. It is enabled by default and can be disabled by setting the environment variable ``RAY_memory_monitor_refresh_ms`` to zero when Ray starts. It is available on Linux and is tested with Ray running inside a container that is using cgroup v1. If you encounter issues when running the memory monitor outside of a container or the container is using cgroup v2, please :ref:`file an issue or post a question `.
+How do I disable the memory monitor?
+--------------------------------------
+
+The memory monitor is enabled by default and can be disabled by setting the environment variable ``RAY_memory_monitor_refresh_ms`` to zero when Ray starts (e.g., RAY_memory_monitor_refresh_ms=0 ray start ...).
How do I configure the memory monitor?
--------------------------------------
The memory monitor is controlled by the following environment variables:
-- ``RAY_memory_monitor_refresh_ms (int, defaults to 250)`` is the interval to check memory usage and kill tasks or actors if needed. It is disabled when this value is 0.
+- ``RAY_memory_monitor_refresh_ms (int, defaults to 250)`` is the interval to check memory usage and kill tasks or actors if needed. Task killing is disabled when this value is 0. The memory monitor selects and kills one task at a time and waits for it to be killed before choosing another one, regardless of how frequent the memory monitor runs.
- ``RAY_memory_usage_threshold (float, defaults to 0.95)`` is the threshold when the node is beyond the memory
- capacity. If the memory usage is above this value and the free space is
- below min_memory_free_bytes then it will start killing processes to free up space. Ranges from [0, 1].
-
-- ``RAY_task_oom_retries (int, defaults to 15):`` The number of retries for the task or actor when
- it fails due to the process being killed by the memory monitor.
- If the task or actor is not retriable then this value is not used. This value is used
- only when the process is killed by the memory monitor, and the retry counter of the
- task or actor (:ref:`max_retries ` or :ref:`max_restarts `) is used when it fails
- in other ways. If the process is killed by the operating system OOM killer it will use the task retry and not ``task_oom_retries``.
- It will retry indefinitely if this value is set to -1.
-
- When the task is retried due to OOM it applies a delay before re-executing the task. The delay is calculated as
-
- .. code-block:: bash
-
- delay_seconds = 2 ^ attempt
-
- Where the first retry will be delayed by 1 second as ``attempt`` starts from 0.
-
- .. note::
-
- Keep the value of ``RAY_task_oom_retries`` low, below 25, to avoid extremely long delays as it is using exponential backoff.
- The delay is capped at 1 minute.
-
+ capacity. If the memory usage is above this fraction it will start killing processes to free up memory. Ranges from [0, 1].
Using the Memory Monitor
------------------------
@@ -59,74 +39,63 @@ Using the Memory Monitor
Retry policy
~~~~~~~~~~~~
-When a task or actor is killed by the memory monitor, it will retry using a separate retry counter based off of ``RAY_task_oom_retries`` instead of the typical number of retries specified by :ref:`max_retries ` for tasks and :ref:`max_restarts ` for actors. The number of memory monitor retries is the same for tasks and actors and defaults to ``15``. To override this value, the environment variable ``RAY_task_oom_retries`` should be set when starting Ray as well as the application.
+When a task or actor is killed by the memory monitor it will be retried with exponential backoff. There is a cap on the retry delay, which is 60 seconds. If tasks are killed by the memory monitor, it retries infinitely (not respecting :ref:`max_retries `). If actors are killed by the memory monitor, it doesn't recreate the actor infinitely (It respects :ref:`max_restarts `, which is 0 by default).
-Let's create an application oom.py that will trigger the out-of-memory condition.
+Worker killing policy
+~~~~~~~~~~~~~~~~~~~~~
-.. literalinclude:: ../doc_code/ray_oom_prevention.py
- :language: python
- :start-after: __oom_start__
- :end-before: __oom_end__
+The memory monitor avoids infinite loops of task retries by ensuring at least one task is able to run for each caller on each node. If it is unable to ensure this, the workload will fail with an OOM error. Note that this is only an issue for tasks, since the memory monitor will not indefinitely retry actors. If the workload fails, refer to :ref:`how to address memory issues ` on how to adjust the workload to make it pass. For code example, see the :ref:`last task ` example below.
+When a worker needs to be killed, the policy first prioritizes tasks that are retriable, i.e. when :ref:`max_retries ` or :ref:`max_restarts ` is > 0. This is done to minimize workload failure. Actors by default are not retriable since :ref:`max_restarts ` defaults to 0. Therefore, by default, tasks are preferred to actors when it comes to what gets killed first.
-To speed up the example, set ``RAY_task_oom_retries=1`` on the application so the task will only retry once if it is killed by the memory monitor. Also set ``RAY_event_stats_print_interval_ms=1000`` so it prints the worker kill summary, which by default is every minute.
+When there are multiple callers that has created tasks, the policy will pick a task from the caller with the most number of running tasks. If two callers have the same number of tasks it picks the caller whose earliest task has a later start time. This is done to ensure fairness and allow each caller to make progress.
-.. code-block:: bash
+Amongst the tasks that share the same caller, the latest started task will be killed first.
- RAY_event_stats_print_interval_ms=1000 RAY_task_oom_retries=1 python oom.py
+Below is an example to demonstrate the policy. In the example we have a script that creates two tasks, which in turn creates four more tasks each. The tasks are colored such that each color forms a "group" of tasks where they belong to the same caller.
- 2022-11-17 09:16:40,792 INFO worker.py:1534 -- Started a local Ray instance. View the dashboard at http://127.0.0.1:8265
+.. image:: ../images/oom_killer_example.svg
+ :width: 1024
+ :alt: Initial state of the task graph
- (raylet) [2022-11-17 09:16:52,264 E 90996 90996] (raylet) node_manager.cc:3096: 1 Workers (tasks / actors) killed due to memory pressure (OOM), 0 Workers crashed due to other reasons at node (ID: 90efe99b630d4b1f6ac1504df64764732d555b526049638f9d86552f, IP: 172.17.0.2) over the last time period. To see more information about the Workers killed on this node, use `ray logs raylet.out -ip 172.17.0.2`
- (raylet)
- (raylet) Refer to the documentation on how to address the out of memory issue: https://docs.ray.io/en/latest/ray-core/scheduling/ray-oom-prevention.html. Consider provisioning more memory on this node or reducing task parallelism by requesting more CPUs per task. To adjust the kill threshold, set the environment variable `RAY_memory_usage_threshold` when starting Ray. To disable worker killing, set the environment variable `RAY_memory_monitor_refresh_ms` to zero.
+If, at this point, the node runs out of memory, it will pick a task from the caller with the most number of tasks, and kill its task whose started the last:
- (raylet) [2022-11-17 09:17:03,461 E 90996 90996] (raylet) node_manager.cc:3096: 1 Workers (tasks / actors) killed due to memory pressure (OOM), 0 Workers crashed due to other reasons at node (ID: 90efe99b630d4b1f6ac1504df64764732d555b526049638f9d86552f, IP: 172.17.0.2) over the last time period. To see more information about the Workers killed on this node, use `ray logs raylet.out -ip 172.17.0.2`
- (raylet)
- (raylet) Refer to the documentation on how to address the out of memory issue: https://docs.ray.io/en/latest/ray-core/scheduling/ray-oom-prevention.html. Consider provisioning more memory on this node or reducing task parallelism by requesting more CPUs per task. To adjust the kill threshold, set the environment variable `RAY_memory_usage_threshold` when starting Ray. To disable worker killing, set the environment variable `RAY_memory_monitor_refresh_ms` to zero.
+.. image:: ../images/oom_killer_example_killed_one.svg
+ :width: 1024
+ :alt: Initial state of the task graph
- Traceback (most recent call last):
- File "oom.py", line 11, in
- ray.get(allocate_memory.remote())
- File "/home/ray/github/rayclarng/ray/python/ray/_private/client_mode_hook.py", line 105, in wrapper
- return func(*args, **kwargs)
- File "/home/ray/github/rayclarng/ray/python/ray/_private/worker.py", line 2310, in get
- raise value
- ray.exceptions.OutOfMemoryError: Task was killed due to the node running low on memory.
- Memory on the node (IP: 172.17.0.2, ID: 90efe99b630d4b1f6ac1504df64764732d555b526049638f9d86552f) where the task (task ID: a6755d1708846b10007fda8a687eb57eb8a083c001000000, name=allocate_memory, pid=91085, memory used=24.99GB) was running was 32.62GB / 33.28GB (0.980175), which exceeds the memory usage threshold of 0.96. Ray killed this worker (ID: a8101629b7605f88776a08193f108adcc637248d976add819bbecbba) because it was the most recently scheduled task; to see more information about memory usage on this node, use `ray logs raylet.out -ip 172.17.0.2`. To see the logs of the worker, use `ray logs worker-a8101629b7605f88776a08193f108adcc637248d976add819bbecbba*out -ip 172.17.0.2.Top 10 memory users:
- PID MEM(GB) COMMAND
- 91085 24.99 ray::allocate_memory
- 57330 2.63 /home/ray/.vscode-server/extensions/ms-vscode.cpptools-1.12.4-linux-x64/bin/cpptools
- 48949 1.70 /home/ray/.vscode-server/bin/d045a5eda657f4d7b676dedbfa7aab8207f8a075/node /home/ray/.vscode-server/...
- 54387 0.80 bazel(ray) -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=/home/ray/.cache/bazel/_bazel_ray/8c472b...
- 35099 0.66 /home/ray/.vscode-server/extensions/ms-vscode.cpptools-1.12.4-linux-x64/bin/cpptools-srv 57330 {1729...
- 16821 0.23 /home/ray/.vscode-server/bin/d045a5eda657f4d7b676dedbfa7aab8207f8a075/node /home/ray/.vscode-server/...
- 61800 0.17 /home/ray/.vscode-server/extensions/ms-vscode.cpptools-1.12.4-linux-x64/bin/cpptools-srv 57330 {54EF...
- 91043 0.07 /home/ray/anaconda3/bin/python -u /home/ray/github/rayclarng/ray/python/ray/dashboard/agent.py --nod...
- 90935 0.07 /home/ray/anaconda3/bin/python /home/ray/github/rayclarng/ray/python/ray/dashboard/dashboard.py --ho...
- 90870 0.07 python oom.py
- Refer to the documentation on how to address the out of memory issue: https://docs.ray.io/en/latest/ray-core/scheduling/ray-oom-prevention.html. Consider provisioning more memory on this node or reducing task parallelism by requesting more CPUs per task. To adjust the kill threshold, set the environment variable `RAY_memory_usage_threshold` when starting Ray. To disable worker killing, set the environment variable `RAY_memory_monitor_refresh_ms` to zero.
+If, at this point, the node still runs out of memory, the process will repeat:
-Verify the task was indeed executed twice via ``task_oom_retry``:
+.. image:: ../images/oom_killer_example_killed_two.svg
+ :width: 1024
+ :alt: Initial state of the task graph
-.. code-block:: bash
+.. _last-task-example:
- $ grep -r "retries left" /tmp/ray/session_latest/logs/
+.. dropdown:: Example: Workloads fails if the last task of the caller is killed
- /tmp/ray/session_latest/logs/python-core-driver-01000000ffffffffffffffffffffffffffffffffffffffffffffffff_87487.log:[2022-11-15 13:50:27,653 I 87487 87703] task_manager.cc:458: task c8ef45ccd0112571ffffffffffffffffffffffff01000000 retries left: 3, oom retries left: 1, task failed due to oom: 1
+ Let's create an application oom.py that runs a single task that requires more memory than what is available. It is set to infinite retry by setting ``max_retries`` to -1.
- /tmp/ray/session_latest/logs/python-core-driver-01000000ffffffffffffffffffffffffffffffffffffffffffffffff_87487.log:[2022-11-15 13:50:36,671 I 87487 87703] task_manager.cc:458: task c8ef45ccd0112571ffffffffffffffffffffffff01000000 retries left: 3, oom retries left: 0, task failed due to oom: 1
+ The worker killer policy sees that it is the last task of the caller, and will fail the workload when it kills the task as it is the last one for the caller, even when the task is set to retry forver.
- /tmp/ray/session_latest/logs/python-core-driver-01000000ffffffffffffffffffffffffffffffffffffffffffffffff_87487.log:[2022-11-15 13:50:36,671 I 87487 87703] task_manager.cc:466: No retries left for task c8ef45ccd0112571ffffffffffffffffffffffff01000000, not going to resubmit.
+ .. literalinclude:: ../doc_code/ray_oom_prevention.py
+ :language: python
+ :start-after: __last_task_start__
+ :end-before: __last_task_end__
-.. note::
- Actors by default are not retriable since :ref:`max_restarts ` defaults to 0, therefore tasks are preferred to actors when it comes to what gets killed first. Actors currently don't use ``RAY_task_oom_retries`` and instead use :ref:`max_restarts ` when killed by the memory monitor.
+ Set ``RAY_event_stats_print_interval_ms=1000`` so it prints the worker kill summary every second, since by default it prints every minute.
-Worker killing policy
-~~~~~~~~~~~~~~~~~~~~~
+ .. code-block:: bash
+
+ RAY_event_stats_print_interval_ms=1000 python oom.py
+
+ (raylet) node_manager.cc:3040: 1 Workers (tasks / actors) killed due to memory pressure (OOM), 0 Workers crashed due to other reasons at node (ID: 2c82620270df6b9dd7ae2791ef51ee4b5a9d5df9f795986c10dd219c, IP: 172.31.183.172) over the last time period. To see more information about the Workers killed on this node, use `ray logs raylet.out -ip 172.31.183.172`
+ (raylet)
+ (raylet) Refer to the documentation on how to address the out of memory issue: https://docs.ray.io/en/latest/ray-core/scheduling/ray-oom-prevention.html. Consider provisioning more memory on this node or reducing task parallelism by requesting more CPUs per task. To adjust the kill threshold, set the environment variable `RAY_memory_usage_threshold` when starting Ray. To disable worker killing, set the environment variable `RAY_memory_monitor_refresh_ms` to zero.
+ task failed with OutOfMemoryError, which is expected
+ Verify the task was indeed executed twice via ``task_oom_retry``:
-The raylet prioritizes killing tasks that are retriable, i.e. when ``max_retries`` or ``max_restarts`` is > 0. This is done to minimize workload failure. It then looks for the last one to start executing and kills that worker process. It selects and kills one process at a time and waits for it to be killed before choosing another one, regardless of how frequent the monitor runs.
.. dropdown:: Example: memory monitor prefers to kill a retriable task
@@ -151,9 +120,10 @@ The raylet prioritizes killing tasks that are retriable, i.e. when ``max_retries
$ python two_actors.py
- first actor was killed by memory monitor
- finished second actor
+ First started actor, which is retriable, was killed by the memory monitor.
+ Second started actor, which is not-retriable, finished.
+.. _addressing-memory-issues:
Addressing memory issues
------------------------