ray-project / ray

Ray is an AI compute engine. Ray consists of a core distributed runtime and a set of AI Libraries for accelerating ML workloads.
https://ray.io
Apache License 2.0
34.24k stars 5.81k forks source link

[Ray Core] Memory resource monitor on cluster takes into account cache memory #35989

Closed achordia20 closed 10 months ago

achordia20 commented 1 year ago

What happened + What you expected to happen

I started seeing an issue on our Ray cluster where I'd see nodes with no activity still displaying high memory usage. When I looked into the k8s pod, I didn't see any tasks running or internal ray processes using significant memory. As I dug more, I found that most of the memory was being assigned from disk cache due to our jobs doing significant disk I/O.

I've attached screenshots of what I saw.

Screen Shot 2023-06-01 at 10 55 33 AM Screen Shot 2023-06-01 at 10 56 04 AM

I was also able to check this through the ray apis.

>>> def _get_mem_usage():
...     import ray
...     total = ray._private.utils.get_system_memory()
...     used = ray._private.utils.get_used_memory()
...     available = total - used
...     percent = round(used / total, 3) * 100
...     return total, available, percent, used
...
>>> _get_mem_usage()
(137438953472, 63870889984, 53.5, 73568063488)

Node memory stats did show a high amount of cache memory but since I'm running in k8s, it's harder to prove that it belongs to the ray pod. I was able to prove that the node memory cleared up once I cleared the cache memory.

cat /sys/fs/cgroup/memory/memory.stat
cache 75654529024
rss 770076672
rss_huge 0
shmem 118947840
mapped_file 132734976
dirty 0
writeback 7569408
pgpgin 50643603417
pgpgout 50624962189
pgfault 50566393449
pgmajfault 6501
inactive_anon 51986432
active_anon 840511488
inactive_file 2895663104
active_file 72646320128
unevictable 0
hierarchical_memory_limit 137438953472
total_cache 75654529024
total_rss 770076672
total_rss_huge 0
total_shmem 118947840
total_mapped_file 132734976
total_dirty 0
total_writeback 7569408
total_pgpgin 50643603417
total_pgpgout 50624962189
total_pgfault 50566393449
total_pgmajfault 6501
total_inactive_anon 51986432
total_active_anon 840511488
total_inactive_file 2895663104
total_active_file 72646320128
total_unevictable 0

What this ended up resulting in was ray killing our tasks because the cache memory was being accounted for as process memory.

Error Type: OUT_OF_MEMORY

Task was killed due to the node running low on memory.
Memory on the node (IP: 172.26.48.188, ID: 81a2fdccdb0b6661460f0979a8fa4768e37cb76f70f817fd554b2d87) where the task (task ID: f50a1e531867a01b91ce54eb6ef2d1646def8fcde7070000, name=abhinav-sharded-rms-gt-task, pid=4199, memory used=4.58GB) was running was 122.14GB / 128.00GB (0.954213), which exceeds the memory usage threshold of 0.95. Ray killed this worker (ID: 1670d4098031279e8e28aff41a566b141b705b733d1edc11994d9043) 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.26.48.188`. To see the logs of the worker, use `ray logs worker-1670d4098031279e8e28aff41a566b141b705b733d1edc11994d9043*out -ip 172.26.48.188. Top 10 memory users:
PID MEM(GB) COMMAND
118484  15.70   ray::abhinav-sharded-rms-gt-task
125845  15.67   ray::abhinav-sharded-rms-gt-task
118151  8.33    ray::abhinav-sharded-rms-gt-task
4199    4.58    ray::abhinav-sharded-rms-gt-task
113 0.49    /home/ray/anaconda3/lib/python3.10/site-packages/ray/core/src/ray/raylet/raylet --raylet_socket_name...
310 0.22    /home/ray/anaconda3/bin/python -u /home/ray/anaconda3/lib/python3.10/site-packages/ray/dashboard/age...
7   0.09    /home/ray/anaconda3/bin/python /home/ray/anaconda3/bin/ray start --address=ray-cluster-head-svc.ray-...
186 0.04    /home/ray/anaconda3/bin/python -u /home/ray/anaconda3/lib/python3.10/site-packages/ray/_private/log_...
118354  0.00    /tmp/ray/session_2023-05-25_18-12-40_361764_7/runtime_resources/pip/f0c7b6d444d14248d95c410d2da45452...
118743  0.00    /tmp/ray/session_2023-05-25_18-12-40_361764_7/runtime_resources/pip/f0c7b6d444d14248d95c410d2da45452...
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.

Versions / Dependencies

Ray 2.4.0

Reproduction script

Not sure how exactly to prove this yet with a repro script.

Issue Severity

Medium: It is a significant difficulty but I can work around it.

achordia20 commented 1 year ago

I'm seeing this on VM's as well instead of just a Kubernetes cluster.

scv119 commented 1 year ago

@achordia20 we are using /proc/pid/smaps_rollup to determine the memory usage of a process during OOM killing, specifically we only look into Private_Clean/Dirty/Hugetlb section, which seems not account for the page cache usage (managed by kernel for speeding up disk data access).

Do you have a script for us to reproduce this behavior? Also if you can show the context of /proc/pid/smaps_rollup if you got a repro that would be nice.

another option is that you can turn off the OOM killing feature, which can be controlled by RAY_memory_usage_threshold https://docs.ray.io/en/latest/ray-core/scheduling/ray-oom-prevention.html#how-do-i-configure-the-memory-monitor

yvmilir commented 1 year ago

Unfortunately, I also encountered this issue with version 2.4.0, but my cache does not seem to be particularly large. It seems that the memory has been increasing since we started submitting jobs to the ray-cluster(kuberay), until it reaches around 50%. Even if no jobs are running, the memory usage is still high.

sys/fs/cgroup/memory# cat memory.stat
cache 7702360064
rss 4729700352
rss_huge 2789212160
shmem 6167658496
mapped_file 6168662016
dirty 8192
writeback 0
swap 0
pgpgin 178125506
pgpgout 177528593
pgfault 195961384
pgmajfault 954
inactive_anon 5990920192
active_anon 4934561792
inactive_file 1026723840
active_file 507977728
unevictable 0
hierarchical_memory_limit 24000000000
hierarchical_memsw_limit 24000000000
total_cache 7702360064
total_rss 4729700352
total_rss_huge 2789212160
total_shmem 6167658496
total_mapped_file 6168662016
total_dirty 8192
total_writeback 0
total_swap 0
total_pgpgin 178125506
total_pgpgout 177528593
total_pgfault 195961384
total_pgmajfault 954
total_inactive_anon 5990920192
total_active_anon 4934561792
total_inactive_file 1026723840
total_active_file 507977728
total_unevictable 0
WeichenXu123 commented 10 months ago

I have filed a RP for this https://github.com/ray-project/ray/pull/42508 (But only for CGroup V1)

WeichenXu123 commented 10 months ago

@achordia20 @yvmilir Would you help test against my fix PR https://github.com/ray-project/ray/pull/42508 ?

You can install my custom built package:

pip install "ray[default] @ https://github.com/WeichenXu123/packages/raw/c5d6cedacec0ec2446a8c0803b14f35937b5fe0e/ray/spark-df-loader/ray-3.0.0.dev0-cp310-cp310-linux_x86_64.whl"