ray
ray copied to clipboard
[core] Object store memory leak in simple while loop
What happened + What you expected to happen
Running the following script leads to an infinite object store memory leak and disk spilling. Interestingly, ray memory
doesn't show any leak of references at the ref counting layer.
From discuss: https://discuss.ray.io/t/object-store-spilling-terabytes-of-data/8699/5
Versions / Dependencies
master
Reproduction script
import ray
ray.init()
@ray.remote
def process(data):
return b"\0" * 100_000_000
while True:
data = ray.put(b"\0" * 100_000_000)
ref = process.remote(data)
ray.get(ref)
Issue Severity
High: It blocks me from completing my task.
@iycheng for initial triaging.
import ray
ray.init()
@ray.remote
def process(data):
return b"\0" * 100_000_000
while True:
data = ray.put(b"\0" * 100_000_000)
ref = process.remote(data)
ray.get(ref)
del ref
Won't have the leak. Maybe related to py gc. Still checking.
[2023-01-04 21:17:14,005 D 3290269 3290314] (raylet) store.cc:401: Received create request for object 6d83b3e72b6da8ce4f1fb3b11f2a02d5aeeedd2e6459ec7100000000 assigned request ID 1, 8 bytes
[2023-01-04 21:17:14,005 D 3290269 3290314] (raylet) object_lifecycle_manager.cc:38: attempting to create object 6d83b3e72b6da8ce4f1fb3b11f2a02d5aeeedd2e6459ec7100000000 size 8
[2023-01-04 21:17:14,005 D 3290269 3290314] (raylet) object_store.cc:28: attempting to create object 6d83b3e72b6da8ce4f1fb3b11f2a02d5aeeedd2e6459ec7100000000 size 8
[2023-01-04 21:17:14,005 D 3290269 3290314] (raylet) plasma_allocator.cc:85: allocating 8
[2023-01-04 21:17:14,005 D 3290269 3290314] (raylet) plasma_allocator.cc:87: allocated 8 at 0x7f7a2b4bf040
[2023-01-04 21:17:14,005 I 3290269 3290314] (raylet) object_store.cc:35: Object store current usage 8e-09 / 14.1717 GB.
[2023-01-04 21:17:14,005 D 3290269 3290314] (raylet) object_store.cc:50: create object 6d83b3e72b6da8ce4f1fb3b11f2a02d5aeeedd2e6459ec7100000000 succeeded
[2023-01-04 21:17:14,005 D 3290269 3290314] (raylet) object_lifecycle_manager.cc:136: Object 6d83b3e72b6da8ce4f1fb3b11f2a02d5aeeedd2e6459ec7100000000 reference has incremented, num bytes in use is now 8
[2023-01-04 21:17:14,005 D 3290269 3290314] (raylet) store.cc:530: Finishing create object 6d83b3e72b6da8ce4f1fb3b11f2a02d5aeeedd2e6459ec7100000000 request ID 1
[2023-01-04 21:17:14,005 D 3290269 3290314] (raylet) store.cc:288: sealing object 6d83b3e72b6da8ce4f1fb3b11f2a02d5aeeedd2e6459ec7100000000
[2023-01-04 21:17:14,005 D 3290269 3290269] (raylet) object_manager.cc:195: Object added 6d83b3e72b6da8ce4f1fb3b11f2a02d5aeeedd2e6459ec7100000000
[2023-01-04 21:17:14,005 D 3290269 3290269] (raylet) ownership_based_object_directory.cc:126: Object 6d83b3e72b6da8ce4f1fb3b11f2a02d5aeeedd2e6459ec7100000000 does not have owner. ReportObjectAdded becomes a no-op.This should only happen for Plasma store warmup objects.
[2023-01-04 21:17:14,005 D 3290269 3290314] (raylet) store.cc:267: Object 6d83b3e72b6da8ce4f1fb3b11f2a02d5aeeedd2e6459ec7100000000 no longer in use by client
[2023-01-04 21:17:14,005 D 3290269 3290269] (raylet) node_manager.cc:2309: Object local 6d83b3e72b6da8ce4f1fb3b11f2a02d5aeeedd2e6459ec7100000000, on 6abc583d112707082aaa32c114328467b72e03aa9ad86649dfa49ca0, 0 tasks ready
[2023-01-04 21:17:14,005 D 3290269 3290314] (raylet) object_lifecycle_manager.cc:157: Releasing object no longer in use 6d83b3e72b6da8ce4f1fb3b11f2a02d5aeeedd2e6459ec7100000000, num bytes in use is now 0
[2023-01-04 21:17:14,006 D 3290269 3290269] (raylet) node_manager.cc:1249: [Worker] Message FreeObjectsInObjectStoreRequest(17) from worker with PID 3290398
[2023-01-04 21:17:14,006 D 3290269 3290314] (raylet) plasma_allocator.cc:121: deallocating 8 at 0x7f7a2b4bf040
[2023-01-04 21:17:14,006 D 3290269 3290269] (raylet) ownership_based_object_directory.cc:149: Object 6d83b3e72b6da8ce4f1fb3b11f2a02d5aeeedd2e6459ec7100000000 does not have owner. ReportObjectRemoved becomes a no-op. This should only happen for Plasma store warmup objects.
Kind of weird..
[2023-01-04 22:03:49,667 D 3319491 3319536] (raylet) store.cc:401: Received create request for object c8ef45ccd0112571ffffffffffffffffffffffff0100000001000000 assigned request ID 19, 100000003 bytes
[2023-01-04 22:03:49,667 D 3319491 3319536] (raylet) object_lifecycle_manager.cc:38: attempting to create object c8ef45ccd0112571ffffffffffffffffffffffff0100000001000000 size 100000000
[2023-01-04 22:03:49,667 D 3319491 3319536] (raylet) object_store.cc:28: attempting to create object c8ef45ccd0112571ffffffffffffffffffffffff0100000001000000 size 100000000
[2023-01-04 22:03:49,667 D 3319491 3319536] (raylet) object_store.cc:50: create object c8ef45ccd0112571ffffffffffffffffffffffff0100000001000000 succeeded
[2023-01-04 22:03:49,667 D 3319491 3319536] (raylet) object_lifecycle_manager.cc:136: Object c8ef45ccd0112571ffffffffffffffffffffffff0100000001000000 reference has incremented, num bytes in use is now 200000006
[2023-01-04 22:03:49,667 D 3319491 3319536] (raylet) store.cc:530: Finishing create object c8ef45ccd0112571ffffffffffffffffffffffff0100000001000000 request ID 19
[2023-01-04 22:03:49,681 D 3319491 3319536] (raylet) store.cc:288: sealing object c8ef45ccd0112571ffffffffffffffffffffffff0100000001000000
[2023-01-04 22:03:49,681 D 3319491 3319491] (raylet) object_manager.cc:195: Object added c8ef45ccd0112571ffffffffffffffffffffffff0100000001000000
[2023-01-04 22:03:49,681 D 3319491 3319491] (raylet) node_manager.cc:2309: Object local c8ef45ccd0112571ffffffffffffffffffffffff0100000001000000, on 62daa5a16e20784c818d8c21cb1ae9235b79616b2116e3a2faacb2be, 0 tasks ready
[2023-01-04 22:03:49,681 D 3319491 3319536] (raylet) object_lifecycle_manager.cc:136: Object c8ef45ccd0112571ffffffffffffffffffffffff0100000001000000 reference has incremented, num bytes in use is now 200000006
[2023-01-04 22:03:49,681 D 3319491 3319536] (raylet) protocol.cc:607: Sending object info, id: c8ef45ccd0112571ffffffffffffffffffffffff0100000001000000 data_size: 100000000 metadata_size: 3
[2023-01-04 22:03:49,681 D 3319491 3319491] (raylet) local_object_manager.cc:43: Pinning object c8ef45ccd0112571ffffffffffffffffffffffff0100000001000000
[2023-01-04 22:03:49,681 D 3319491 3319536] (raylet) store.cc:267: Object c8ef45ccd0112571ffffffffffffffffffffffff0100000001000000 no longer in use by client
[2023-01-04 22:03:49,691 D 3319491 3319491] (raylet) dependency_manager.cc:119: Worker 01000000ffffffffffffffffffffffffffffffffffffffffffffffff called ray.get on object c8ef45ccd0112571ffffffffffffffffffffffff0100000001000000
[2023-01-04 22:03:49,691 D 3319491 3319491] (raylet) pull_manager.cc:73: Pull of object c8ef45ccd0112571ffffffffffffffffffffffff0100000001000000
[2023-01-04 22:03:49,691 D 3319491 3319536] (raylet) object_lifecycle_manager.cc:136: Object c8ef45ccd0112571ffffffffffffffffffffffff0100000001000000 reference has incremented, num bytes in use is now 200000006
[2023-01-04 22:03:49,691 D 3319491 3319536] (raylet) protocol.cc:607: Sending object info, id: c8ef45ccd0112571ffffffffffffffffffffffff0100000001000000 data_size: 100000000 metadata_size: 3
[2023-01-04 22:03:49,691 D 3319491 3319491] (raylet) pull_manager.cc:337: Removing an object pull request of id: c8ef45ccd0112571ffffffffffffffffffffffff0100000001000000
[2023-01-04 22:03:49,691 D 3319491 3319491] (raylet) dependency_manager.cc:41: Object c8ef45ccd0112571ffffffffffffffffffffffff0100000001000000 no longer needed
[2023-01-04 22:03:49,743 D 3319491 3319536] (raylet) store.cc:267: Object c8ef45ccd0112571ffffffffffffffffffffffff0100000001000000 no longer in use by client
[2023-01-04 22:03:49,802 D 3319491 3319491] (raylet) local_object_manager.cc:117: Unpinning object c8ef45ccd0112571ffffffffffffffffffffffff0100000001000000
[2023-01-04 22:03:49,802 D 3319491 3319536] (raylet) store.cc:267: Object c8ef45ccd0112571ffffffffffffffffffffffff0100000001000000 no longer in use by client
[2023-01-04 22:03:49,802 D 3319491 3319536] (raylet) object_lifecycle_manager.cc:157: Releasing object no longer in use c8ef45ccd0112571ffffffffffffffffffffffff0100000001000000, num bytes in use is now 200000006
[2023-01-04 22:03:50,193 D 3319491 3319491] (raylet) node_manager.cc:2362: Object missing c8ef45ccd0112571ffffffffffffffffffffffff0100000001000000, on 62daa5a16e20784c818d8c21cb1ae9235b79616b2116e3a2faacb2be, 0 tasks waiting
I feel the root cause is related to the lineage reconstruction. Basically, A->B and when B is deleted, A is still there because the lineage count 1 (fail to decrease).
still checking.
verified, turn off lineage reconstruction spilling disappeared:
RAY_lineage_pinning_enabled=0 python ./leak.py
should we make this more clear from the output of ray memory?
To be clear, this is a bug. The lineage refs should be freed because the loop throws away all refs on each iteration.
But yeah we should also show this status of possible.
@iycheng it's probably some bug in the order of deleting the refs? Like if ref
gets deleted first it's fine, but not if data
is deleted first? That would explain why the explicit del fixes the issue.
@ericl verified that your theory is correct.
Summary:
- del ref ==> no leak
- del data or del nothing ==> leak
- disable lineage ==> no leak
I think when we delete data and del ref somehow data is not deleted in the store.
@stephanie-wang could you take a look at this https://github.com/ray-project/ray/pull/31488
if (arg_it->second.ShouldDelete(lineage_pinning_enabled_)) {
// We only decremented the lineage ref count, so the argument value
// should already be released.
RAY_CHECK(arg_it->second.on_ref_removed == nullptr);
lineage_bytes_evicted += ReleaseLineageReferences(arg_it);
EraseReference(arg_it);
ReleasePlasmaObject(arg_it);
}
I'm confused about
// We only decremented the lineage ref count, so the argument value
// should already be released.
Adding ReleasePlasmaObject does fix this one.
Oh yeah, this is indeed a bug with lineage reconstruction. We pin ray.put
objects even if they're only referenced by lineage, but we don't release it later when the lineage gets deleted too.
// We only decremented the lineage ref count, so the argument value
// should already be released.
Ah yeah this was trying to say that we only modified the lineage ref count, not the physical ObjectRef count. So the assumption was that the object's value should have already been freed, which is wrong for ray.put
objects :(