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.18k stars 5.8k forks source link

[Data] _StatsActor has unbounded memory usage #43924

Open jjyao opened 8 months ago

jjyao commented 8 months ago

What happened + What you expected to happen

_StatsActor has a field called datasets that doesn't have bounded length and max_stats doesn't affect it. This means if you have a long running cluster and keeps creating Datasets, _StatsActor memory will keep going up and eventually OOM.

Versions / Dependencies

master

Reproduction script

long_running_many_drivers.aws

Issue Severity

None

bveeramani commented 7 months ago

Re-posting relevant question from Slack:

I've been running into this issue pretty frequently as I am creating and reading many datasets. _StatsActor eventually consumes enough memory that it OOMs whichever node it's running on. Is there a way to restart the _StatsActor periodically or something similar so it can reduce its memory usage? Alternatively, is there a way to force it to run on a node with more memory available?

https://ray-distributed.slack.com/archives/C02PHB3SQHH/p1713984289216789

liuxsh9 commented 3 months ago

We're facing the same issue, which has a significant impact. Any updates? Are there any potential solution? Thanks! @jjyao

liuxsh9 commented 2 months ago

This issue can be reproducibly triggered by submitting a large number of ray data jobs, for example:

import ray

class Actor1:
    def __call__(self, batch):
        return batch

ray.init()
ds = ray.data.from_items(range(50)).repartition(50)
ds = ds.map_batches(
        Actor1,
        concurrency=5,
        batch_size=1,
        num_cpus=0.1,
    )

ds.materialize()

image

After using the Heaptrack tool to investigate, we suspect that the grpcio library may be related to it.

image

We tried to upgrade grpcio from 1.58.0 to 1.66.1, which reduced the memory leak (from 1GB growth per hour to 0.25 GB). However, the leak still persists.

Possible related gRPC community issues: [1] There may be a memory leak after grpcio upgrade #36986.

We currently consider blocking _StatsActor to mitigate this issue and will continue to explore better solutions. @jjyao @anyscalesam