Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[Core] Apparent object store memory leak #25779

Closed
merrysailor opened this issue Jun 14, 2022 · 9 comments
Closed

[Core] Apparent object store memory leak #25779

merrysailor opened this issue Jun 14, 2022 · 9 comments
Labels
bug Something that is supposed to be working; but isn't core Issues that should be addressed in Ray Core core-correctness Leak, crash, hang core-object-store P1 Issue that should be fixed within a few weeks ray 2.8 size:medium
Milestone

Comments

@merrysailor
Copy link

What happened + What you expected to happen

Hello,

I am new ray user and trying to parallelize general python workflows. A very basic example I put together seems to exhaust the object store memory and result in spilling, even though from what I can see the code should remove all object references. Also, the logs highlight some issues:

raylet.out, on every task submission:

[2022-06-14 20:59:38,442 I 191450 191450] (raylet) object_buffer_pool.cc:153: Not enough memory to create requested object 00ffffffffffffffffffffffffffffffffffffff0100000002000000, aborting

python-core-driver-01000000ffffffffffffffffffffffffffffffffffffffffffffffff_191395.log:

[2022-06-14 21:00:49,960 W 191395 191634] reference_count.cc:1225: Spilled object a8485d936ac2e7ccffffffffffffffffffffffff0100000001000000 already out of scope
[2022-06-14 21:00:59,800 W 191395 191634] reference_count.cc:1415: Object locations requested for ec502c4fdc3aeab0ffffffffffffffffffffffff0100000001000000, but ref already removed. This may be a bug in the distributed reference counting protocol.

The fact that spilling takes place is highlight both by the logs and ray memory invocations, e.g.,

--- Aggregate object store stats across all nodes ---
Plasma memory usage 152 MiB, 16 objects, 80.0% full, 5.0% needed
Spilled 314 MiB, 33 objects, avg write throughput 235 MiB/s
Objects consumed by Ray tasks: 343 MiB.

Thank you.

Versions / Dependencies

Ray: 1.12.1, 1.13.0
Python: 3.9
OS: Ubuntu 20.04 LTS

Reproduction script

import gc
import ray

if ray.is_initialized():
    ray.shutdown()
ray.init(address=None, object_store_memory=200_000_000)

@ray.remote
def work(x):
    import random
    import time
    time.sleep(5)
    return random.randbytes(10_000_000)

def launch_ray():
    import random
    x = random.randbytes(10_000_000)
    return work.remote(x)

###############################################################################

# This attempts to pace job submissions

idx = 0
N = 50
pending_tasks = []
while True:
    gc.collect()
    completed_tasks, pending_tasks = ray.wait(pending_tasks)

    while len(pending_tasks) < 8 and idx < N:
        idx += 1

        r = launch_ray()
        pending_tasks.append(r)

    for t in completed_tasks:
        t = ray.get(t)
    del completed_tasks

    if not pending_tasks:
        break
del pending_tasks

Issue Severity

High: It blocks me from completing my task.

@merrysailor merrysailor added bug Something that is supposed to be working; but isn't triage Needs triage (eg: priority, bug/not-bug, and owning component) labels Jun 14, 2022
@merrysailor
Copy link
Author

An even simpler demonstration, a loop, send 10MB to a single task, receive 10MB back

import gc
import ray
import random

ray.init(object_store_memory=100_000_000)

@ray.remote
def work(x):
    import random
    import time
    time.sleep(1)
    return random.randbytes(10_000_000)

for _ in range(100):
    gc.collect()
    ray.get(work.remote(random.randbytes(10_000_000)))

This results in unbounded object spilling:

--- Aggregate object store stats across all nodes ---
Plasma memory usage 76 MiB, 8 objects, 80.0% full, 10.0% needed
Spilled 238 MiB, 25 objects, avg write throughput 221 MiB/s
Objects consumed by Ray tasks: 486 MiB.

And log errors

[2022-06-16 07:51:21,535 I 763823 763823] (raylet) object_buffer_pool.cc:153: Not enough memory to create requested object 00ffffffffffffffffffffffffffffffffffffff0100000017000000, aborting
[2022-06-16 07:51:03,615 W 763769 764016] reference_count.cc:1221: Spilled object 00ffffffffffffffffffffffffffffffffffffff0100000003000000 already out of scope

@scv119 scv119 added the core Issues that should be addressed in Ray Core label Jun 23, 2022
@scv119 scv119 added this to the Core Backlog milestone Jun 23, 2022
@scv119 scv119 added the P1 Issue that should be fixed within a few weeks label Jun 23, 2022
@scv119
Copy link
Contributor

scv119 commented Jun 23, 2022

Thanks for reporting. I think there is some race condition between Ray GC and spill worker, so when the object has been GCed the spill worker already grab the object for spilling.

cc @rkooo567

@scv119 scv119 removed the triage Needs triage (eg: priority, bug/not-bug, and owning component) label Jun 23, 2022
@cadedaniel cadedaniel self-assigned this Sep 7, 2022
@cadedaniel
Copy link
Member

cadedaniel commented Sep 7, 2022

I'm looking into this in my spare time. I've reproduced it and, interestingly, the issue only repros when the task takes an argument and returns data. Furthermore, the amount of memory leaked per invocation+gc.collect() cycle increases proportionally to the size of the argument (not the return value). This indicates that Ray leaks the argument.

I'll continue to root-cause later, just wanted to repro and see whether it was the arg or rval that was leaking.

index name (available_at_start-available_before_alloc) (available_at_start-available_after_gc)
0 arg_only_worker: 0MB -> 0MB
1 arg_only_worker: 0MB -> 0MB
2 arg_only_worker: 0MB -> 0MB
3 arg_only_worker: 0MB -> 0MB
4 arg_only_worker: 0MB -> 0MB
index name (available_at_start-available_before_alloc) (available_at_start-available_after_gc)
0 rval_only_worker: 0MB -> 0MB
1 rval_only_worker: 0MB -> 0MB
2 rval_only_worker: 0MB -> 0MB
3 rval_only_worker: 0MB -> 0MB
4 rval_only_worker: 0MB -> 0MB
index name (available_at_start-available_before_alloc) (available_at_start-available_after_gc)
0 arg_and_rval_worker: 0MB -> 3MB
1 arg_and_rval_worker: 3MB -> 6MB
2 arg_and_rval_worker: 6MB -> 9MB
3 arg_and_rval_worker: 9MB -> 12MB
4 arg_and_rval_worker: 12MB -> 15MB

In this test, the size of the task argument is 3MB, return value is 2MB.

task definitions:

@ray.remote
def rval_only_worker():
    return os.urandom(rval_data_size)

@ray.remote
def arg_only_worker(x):
    assert len(x) == arg_data_size

@ray.remote
def arg_and_rval_worker(x):
    assert len(x) == arg_data_size
    return os.urandom(rval_data_size)

See here for the full test script.

@donaldanixon
Copy link

Is there an ETA for the resolution on this?

@rkooo567
Copy link
Contributor

Right now, we have no bandwidth to prioritize until the next release (ray 2.4. scheduled by end of March). We can try fixing this by ray 2.5

@mshunshin
Copy link

@cadedaniel Thanks for the test script - it reproduces the exact problem I have been experiencing (though mine involves passing 300mb arrays around so I run into trouble often).

As a workarround - if you do a ref = ray.put(large_object), method.remote(ref) the problem dissapears.

Here is the test script modified to demonstrate this.

Hopefully this will help with tracking down the underlying cause.

import ray
import os
import gc
import time

ray.init()
arg_data_size = 3 * 1 << 20  # 3MB
rval_data_size = 2 * 1 << 20  # 2MB


@ray.remote
def rval_only_worker():
    return os.urandom(rval_data_size)


@ray.remote
def arg_only_worker(x):
    pass


@ray.remote
def arg_and_rval_worker(x):
    return os.urandom(rval_data_size)


def test_method(method, needs_arg, pre_load_arg):

    start_mem = ray.available_resources()["object_store_memory"]
    print(f"================================================")
    print(f"{str(method)}")
    print(f"needs_arg {needs_arg} pre_load_arg {pre_load_arg}")
    print("index name (available_at_start-available_before_alloc) (available_at_start-available_after_gc)")

    for i in range(5):
        before = ray.available_resources()["object_store_memory"]

        arg = os.urandom(arg_data_size)

        if needs_arg:
            if pre_load_arg:
                arg_ray = ray.put(arg)
                ref = method.remote(arg_ray)
            else:
                ref = method.remote(arg)
        else:
            ref = method.remote()

        ref = arg_and_rval_worker.remote(ref)
        rval = ray.get(ref)

        if rval:
            assert len(rval) == rval_data_size

        del arg
        del rval
        del ref
        gc.collect()

        time.sleep(
            1
        )  # Allow Ray async gc to catch up, flaky but works in simple case.

        after = ray.available_resources()["object_store_memory"]

        nice_name = method._function_name.split(".")[1]
        print(
            f"{i} {nice_name}: {int(start_mem-before)//(1 << 20)}MB -> {int(start_mem-after)//(1 << 20)}MB"
        )


test_method(arg_only_worker, needs_arg=True, pre_load_arg=False)
test_method(arg_only_worker, needs_arg=True, pre_load_arg=True)
test_method(rval_only_worker, needs_arg=False, pre_load_arg=False)
test_method(arg_and_rval_worker, needs_arg=True, pre_load_arg=False)
test_method(arg_and_rval_worker, needs_arg=True, pre_load_arg=True)

@rynewang
Copy link
Contributor

I think this issue should have been addressed. I tested the repro script from @merrysailor and the one from @cadedaniel and both did NOT repro. I used the off the shelf Ray 2.5, on local desktop and on the workspace.

@rynewang
Copy link
Contributor

side by side test: running the repro script on ray 1.13.0 vs ray 2.5, use ray memory to collect results.

ray 1.13.0 (in progress):

======== Object references status: 2023-06-15 11:06:08.675941 ========
Grouping by node address...        Sorting by object size...        Display allentries per group...


--- Summary for node address: 127.0.0.1 ---
Mem Used by Objects  Local References  Pinned        Used by task   Captured in Objects  Actor Handles
20000003.0 B         1, (0.0 B)        1, (10000000.0 B)  1, (10000003.0 B)  0, (0.0 B)           0, (0.0 B)

--- Object references for node address: 127.0.0.1 ---
IP Address       PID    Type    Call Site               Status          Size    Reference Type      Object Ref
127.0.0.1        42542  Driver  disabled                SCHEDULED       ?       LOCAL_REFERENCE     5f70e045687d2f9affffffffffffffffffffffff0100000001000000

127.0.0.1        42566  Worker  (deserialize task arg)  -               10000000.0 B  PINNED_IN_MEMORY    00ffffffffffffffffffffffffffffffffffffff010000004b000000
                                 i25779.original.work

127.0.0.1        42542  Driver  disabled                FINISHED        10000003.0 B  USED_BY_PENDING_TASK  00ffffffffffffffffffffffffffffffffffffff010000004b000000

To record callsite information for each ObjectRef created, set env variable RAY_record_ref_creation_sites=1

--- Aggregate object store stats across all nodes ---
Plasma memory usage 85 MiB, 9 objects, 90.0% full, 10.0% needed
Spilled 705 MiB, 74 objects, avg write throughput 1129 MiB/s
Objects consumed by Ray tasks: 1420 MiB.

ray 1.13.0 (finished):

======== Object references status: 2023-06-15 11:06:54.465222 ========
Grouping by node address...        Sorting by object size...        Display allentries per group...


To record callsite information for each ObjectRef created, set env variable RAY_record_ref_creation_sites=1

--- Aggregate object store stats across all nodes ---
Plasma memory usage 76 MiB, 8 objects, 80.0% full, 10.0% needed
Spilled 944 MiB, 99 objects, avg write throughput 1370 MiB/s
Objects consumed by Ray tasks: 1907 MiB.

ray 2.5 (finished):

======== Object references status: 2023-06-15 11:03:28.105545 ========
Grouping by node address...        Sorting by object size...        Display allentries per group...


To record callsite information for each ObjectRef created, set env variable RAY_record_ref_creation_sites=1

--- Aggregate object store stats across all nodes ---
Plasma memory usage 0 MiB, 0 objects, 0.0% full, 0.0% needed
Objects consumed by Ray tasks: 1907 MiB.

So we do have a bug in ray 1.13.0 that it spills when it feels memory pressure but did not interact well with GC, while in 2.5 it correctly GCs all used up objects and did not spill.

Repro procedure:

python -m venv 1.13.0
source 1.13.0/bin/activate
pip install -U 'ray[all]==1.13.0'
python i25779.original.py # the script in #issuecomment-1157352693, with a dead loop at the end
# on another shell...
watch ray memory

Same thing for 2.5.

@rkooo567 rkooo567 added Ray 2.7 and removed Ray 2.6 labels Jul 17, 2023
@rkooo567 rkooo567 added ray 2.8 and removed Ray 2.7 labels Aug 24, 2023
@anyscalesam
Copy link
Collaborator

Per the above - we are now closing as we believe Ray 2.5+ has GC operating correctly. @merrysailor please re-open if you still observe this issue.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug Something that is supposed to be working; but isn't core Issues that should be addressed in Ray Core core-correctness Leak, crash, hang core-object-store P1 Issue that should be fixed within a few weeks ray 2.8 size:medium
Projects
None yet
Development

No branches or pull requests

9 participants