mirror of
https://github.com/vale981/ray
synced 2025-03-06 02:21:39 -05:00
[Core] Fix in-container memory limit fetching for cgroups v2 (#23922)
This commit is contained in:
parent
92781c603e
commit
166cd537d5
2 changed files with 64 additions and 6 deletions
|
@ -370,7 +370,12 @@ def open_log(path, unbuffered=False, **kwargs):
|
|||
return stream
|
||||
|
||||
|
||||
def get_system_memory():
|
||||
def get_system_memory(
|
||||
# For cgroups v1:
|
||||
memory_limit_filename="/sys/fs/cgroup/memory/memory.limit_in_bytes",
|
||||
# For cgroups v2:
|
||||
memory_limit_filename_v2="/sys/fs/cgroup/memory.max",
|
||||
):
|
||||
"""Return the total amount of system memory in bytes.
|
||||
|
||||
Returns:
|
||||
|
@ -380,16 +385,17 @@ def get_system_memory():
|
|||
# container. Note that this file is not specific to Docker and its value is
|
||||
# often much larger than the actual amount of memory.
|
||||
docker_limit = None
|
||||
# For cgroups v1:
|
||||
memory_limit_filename = "/sys/fs/cgroup/memory/memory.limit_in_bytes"
|
||||
# For cgroups v2:
|
||||
memory_limit_filename_v2 = "/sys/fs/cgroup/memory.max"
|
||||
if os.path.exists(memory_limit_filename):
|
||||
with open(memory_limit_filename, "r") as f:
|
||||
docker_limit = int(f.read())
|
||||
elif os.path.exists(memory_limit_filename_v2):
|
||||
with open(memory_limit_filename_v2, "r") as f:
|
||||
docker_limit = int(f.read())
|
||||
max_file = f.read()
|
||||
if max_file.isnumeric():
|
||||
docker_limit = int(max_file)
|
||||
else:
|
||||
# max_file is "max", i.e. is unset.
|
||||
docker_limit = None
|
||||
|
||||
# Use psutil if it is available.
|
||||
psutil_memory_in_bytes = psutil.virtual_memory().total
|
||||
|
|
|
@ -9,6 +9,7 @@ from unittest import mock
|
|||
|
||||
import numpy as np
|
||||
import pytest
|
||||
import psutil
|
||||
|
||||
import ray
|
||||
from ray.dashboard import k8s_utils
|
||||
|
@ -273,6 +274,57 @@ def test_accelerator_type_api(shutdown_only):
|
|||
wait_for_condition(lambda: ray.available_resources()[resource_name] < quantity)
|
||||
|
||||
|
||||
def test_get_system_memory():
|
||||
# cgroups v1, set
|
||||
with tempfile.NamedTemporaryFile("w") as memory_limit_file:
|
||||
memory_limit_file.write("100")
|
||||
memory_limit_file.flush()
|
||||
assert (
|
||||
ray._private.utils.get_system_memory(
|
||||
memory_limit_filename=memory_limit_file.name,
|
||||
memory_limit_filename_v2="__does_not_exist__",
|
||||
)
|
||||
== 100
|
||||
)
|
||||
|
||||
# cgroups v1, high
|
||||
with tempfile.NamedTemporaryFile("w") as memory_limit_file:
|
||||
memory_limit_file.write(str(2 ** 64))
|
||||
memory_limit_file.flush()
|
||||
psutil_memory_in_bytes = psutil.virtual_memory().total
|
||||
assert (
|
||||
ray._private.utils.get_system_memory(
|
||||
memory_limit_filename=memory_limit_file.name,
|
||||
memory_limit_filename_v2="__does_not_exist__",
|
||||
)
|
||||
== psutil_memory_in_bytes
|
||||
)
|
||||
# cgroups v2, set
|
||||
with tempfile.NamedTemporaryFile("w") as memory_max_file:
|
||||
memory_max_file.write("100")
|
||||
memory_max_file.flush()
|
||||
assert (
|
||||
ray._private.utils.get_system_memory(
|
||||
memory_limit_filename="__does_not_exist__",
|
||||
memory_limit_filename_v2=memory_max_file.name,
|
||||
)
|
||||
== 100
|
||||
)
|
||||
|
||||
# cgroups v2, not set
|
||||
with tempfile.NamedTemporaryFile("w") as memory_max_file:
|
||||
memory_max_file.write("max")
|
||||
memory_max_file.flush()
|
||||
psutil_memory_in_bytes = psutil.virtual_memory().total
|
||||
assert (
|
||||
ray._private.utils.get_system_memory(
|
||||
memory_limit_filename="__does_not_exist__",
|
||||
memory_limit_filename_v2=memory_max_file.name,
|
||||
)
|
||||
== psutil_memory_in_bytes
|
||||
)
|
||||
|
||||
|
||||
@pytest.mark.skipif(sys.platform == "win32", reason="not relevant for windows")
|
||||
def test_detect_docker_cpus():
|
||||
# No limits set
|
||||
|
|
Loading…
Add table
Reference in a new issue