diff --git a/python/ray/tests/test_reference_counting_2.py b/python/ray/tests/test_reference_counting_2.py index 0177f21f99fe..4fe5c6f36c95 100644 --- a/python/ray/tests/test_reference_counting_2.py +++ b/python/ray/tests/test_reference_counting_2.py @@ -777,6 +777,27 @@ def remote_generator(): _fill_object_store_and_get(r_oid, succeed=False) +def test_lineage_leak(shutdown_only): + ray.init() + + @ray.remote + def process(data): + return b"\0" * 100_000_000 + + data = ray.put(b"\0" * 100_000_000) + ref = process.remote(data) + ray.get(ref) + del data + del ref + + def check_usage(): + from ray._private.internal_api import memory_summary + + return "Plasma memory usage 0 MiB" in memory_summary(stats_only=True) + + wait_for_condition(check_usage) + + if __name__ == "__main__": import sys diff --git a/src/ray/core_worker/reference_count.cc b/src/ray/core_worker/reference_count.cc index 8f54e07feca6..3c285cd97b4c 100644 --- a/src/ray/core_worker/reference_count.cc +++ b/src/ray/core_worker/reference_count.cc @@ -478,10 +478,9 @@ int64_t ReferenceCounter::ReleaseLineageReferences(ReferenceTable::iterator ref) RAY_LOG(DEBUG) << "Releasing lineage internal for argument " << argument_id; arg_it->second.lineage_ref_count--; 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); + ReleasePlasmaObject(arg_it); EraseReference(arg_it); } }