Skip to content

Conversation

orangeQWJ
Copy link
Contributor

Background
Closes #43924.

The _StatsActor suffers from unbounded memory usage in long-running clusters. Its core metadata dictionaries, datasets and dataset_metadatas, lacked a proper garbage collection (GC) mechanism. This could lead to an Out-of-Memory (OOM) error in the _StatsActor when a large number of datasets are created.

Solution
This patch implements an eviction policy for _StatsActor based on the max_stats configuration to effectively limit the size of the datasets and dataset_metadatas dictionaries, preventing their unbounded growth.

The implementation details are as follows:

  1. Optimize Queue Implementation
    The old, unused fifo_queue field has been removed. It is replaced by a new collections.deque, finished_datasets_queue, which serves as a more efficient FIFO queue for storing the tags of completed datasets.

  2. Implement Eviction Logic

    • When a dataset's status is updated to FINISHED or FAILED, its tag is appended to the finished_datasets_queue.
    • A check is then immediately performed to see if the total number of entries in the datasets dictionary exceeds max_stats.
    • If the limit is exceeded, the oldest dataset tag is popped from the front of the finished_datasets_queue, and the corresponding entries are synchronously deleted from the datasets and dataset_metadatas dictionaries.
  3. Clarify Limitation Strategy
    max_stats is not a strict hard limit. Since the eviction logic is only triggered when a dataset completes (FINISHED or FAILED), it is possible for the number of RUNNING datasets to cause the total entry count to temporarily exceed max_stats. This design ensures that metadata for in-progress tasks is never evicted, while still effectively preventing unbounded memory growth and OOM errors by cleaning up the oldest completed data as soon as a task finishes.

Testing
To verify the correctness of this fix, a new unit test, test_stats_actor_datasets_eviction, has been added. This test sets a low max_stats value and asserts that the oldest finished dataset is correctly evicted when the limit is surpassed.

@orangeQWJ orangeQWJ requested a review from a team as a code owner August 26, 2025 03:11
Copy link
Contributor

@gemini-code-assist gemini-code-assist bot left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Code Review

This pull request introduces a crucial fix to prevent unbounded memory growth in the _StatsActor by implementing an eviction policy for finished datasets. The changes are well-structured and include a new collections.deque for efficient FIFO operations and clear eviction logic within update_dataset. The addition of the test_stats_actor_datasets_eviction unit test is excellent, as it thoroughly verifies the new functionality.

My review includes a couple of minor suggestions to improve code conciseness and readability in the eviction logic. Overall, this is a solid improvement to the stability of long-running Ray Data clusters.

Comment on lines 570 to 580
if (
state["state"] == DatasetState.FINISHED.name
or state["state"] == DatasetState.FAILED.name
):
self.finished_datasets_queue.append(dataset_tag)
while len(self.datasets) > self.max_stats and self.finished_datasets_queue:
tag_to_evict = self.finished_datasets_queue.popleft()
if tag_to_evict in self.datasets:
del self.datasets[tag_to_evict]
if tag_to_evict in self.dataset_metadatas:
del self.dataset_metadatas[tag_to_evict]
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ooc did you try considering using 1 OrderedDict instead of a dict + deque to limit the memory usage? Basically on each use of the dataset tag you can move the tag to the end of the ordereddict

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What was the follow-up here?

@goutamvenkat-anyscale goutamvenkat-anyscale changed the title fix(data): Prevent unbounded growth of_StatsActor.datasets [Data] Prevent unbounded growth of_StatsActor.datasets Aug 26, 2025
@goutamvenkat-anyscale goutamvenkat-anyscale added data Ray Data-related issues community-contribution Contributed by the community labels Aug 26, 2025
@goutamvenkat-anyscale
Copy link
Contributor

Thanks for your contribution. Please ensure to sign the commits.

As far as the change goes, I was wondering if you considered using OrderedDict for dataset instead of a separate deque to keep track of the used tags.

@goutamvenkat-anyscale goutamvenkat-anyscale requested a review from a team August 26, 2025 08:00
qiwenju added 2 commits August 26, 2025 16:36
Signed-off-by: qiwenju <qiwenju@xiaomi.com>
Signed-off-by: qiwenju <qiwenju@xiaomi.com>
@jjyao jjyao added the go add ONLY when ready to merge, run all tests label Aug 26, 2025
@alexeykudinkin alexeykudinkin merged commit c44a7ca into ray-project:master Aug 28, 2025
6 checks passed
tohtana pushed a commit to tohtana/ray that referenced this pull request Aug 29, 2025
…5925)

Background
  Closes ray-project#43924.

The _StatsActor suffers from unbounded memory usage in long-running
clusters. Its core metadata dictionaries, `datasets` and
`dataset_metadatas`, lacked a proper garbage collection (GC) mechanism.
This could lead to an Out-of-Memory (OOM) error in the _StatsActor when
a large number of datasets are created.

Solution
This patch implements an eviction policy for _StatsActor based on the
`max_stats` configuration to effectively limit the size of the
`datasets` and `dataset_metadatas` dictionaries, preventing their
unbounded growth.

  The implementation details are as follows:
  1. **Optimize Queue Implementation**
The old, unused `fifo_queue` field has been removed. It is replaced by a
new `collections.deque`, `finished_datasets_queue`, which serves as a
more efficient FIFO queue for storing the tags of completed datasets.

  2. **Implement Eviction Logic**
- When a dataset's status is updated to `FINISHED` or `FAILED`, its tag
is appended to the `finished_datasets_queue`.
- A check is then immediately performed to see if the total number of
entries in the `datasets` dictionary exceeds `max_stats`.
- If the limit is exceeded, the oldest dataset tag is popped from the
front of the `finished_datasets_queue`, and the corresponding entries
are synchronously deleted from the `datasets` and `dataset_metadatas`
dictionaries.

  3. **Clarify Limitation Strategy**
`max_stats` is not a strict hard limit. Since the eviction logic is only
triggered when a dataset completes (`FINISHED` or `FAILED`), it is
possible for the number of `RUNNING` datasets to cause the total entry
count to temporarily exceed `max_stats`. This design ensures that
metadata for in-progress tasks is never evicted, while still effectively
preventing unbounded memory growth and OOM errors by cleaning up the
oldest completed data as soon as a task finishes.

Testing
To verify the correctness of this fix, a new unit test,
`test_stats_actor_datasets_eviction`, has been added. This test sets a
low `max_stats` value and asserts that the oldest finished dataset is
correctly evicted when the limit is surpassed.

---------

Signed-off-by: qiwenju <qiwenju@xiaomi.com>
Co-authored-by: qiwenju <qiwenju@xiaomi.com>
Signed-off-by: Masahiro Tanaka <mtanaka@anyscale.com>
tohtana pushed a commit to tohtana/ray that referenced this pull request Aug 29, 2025
…5925)

Background
  Closes ray-project#43924.

The _StatsActor suffers from unbounded memory usage in long-running
clusters. Its core metadata dictionaries, `datasets` and
`dataset_metadatas`, lacked a proper garbage collection (GC) mechanism.
This could lead to an Out-of-Memory (OOM) error in the _StatsActor when
a large number of datasets are created.

Solution
This patch implements an eviction policy for _StatsActor based on the
`max_stats` configuration to effectively limit the size of the
`datasets` and `dataset_metadatas` dictionaries, preventing their
unbounded growth.

  The implementation details are as follows:
  1. **Optimize Queue Implementation**
The old, unused `fifo_queue` field has been removed. It is replaced by a
new `collections.deque`, `finished_datasets_queue`, which serves as a
more efficient FIFO queue for storing the tags of completed datasets.

  2. **Implement Eviction Logic**
- When a dataset's status is updated to `FINISHED` or `FAILED`, its tag
is appended to the `finished_datasets_queue`.
- A check is then immediately performed to see if the total number of
entries in the `datasets` dictionary exceeds `max_stats`.
- If the limit is exceeded, the oldest dataset tag is popped from the
front of the `finished_datasets_queue`, and the corresponding entries
are synchronously deleted from the `datasets` and `dataset_metadatas`
dictionaries.

  3. **Clarify Limitation Strategy**
`max_stats` is not a strict hard limit. Since the eviction logic is only
triggered when a dataset completes (`FINISHED` or `FAILED`), it is
possible for the number of `RUNNING` datasets to cause the total entry
count to temporarily exceed `max_stats`. This design ensures that
metadata for in-progress tasks is never evicted, while still effectively
preventing unbounded memory growth and OOM errors by cleaning up the
oldest completed data as soon as a task finishes.

Testing
To verify the correctness of this fix, a new unit test,
`test_stats_actor_datasets_eviction`, has been added. This test sets a
low `max_stats` value and asserts that the oldest finished dataset is
correctly evicted when the limit is surpassed.

---------

Signed-off-by: qiwenju <qiwenju@xiaomi.com>
Co-authored-by: qiwenju <qiwenju@xiaomi.com>
Signed-off-by: Masahiro Tanaka <mtanaka@anyscale.com>
gangsf pushed a commit to gangsf/ray that referenced this pull request Sep 2, 2025
…5925)

Background  
  Closes ray-project#43924.  

The _StatsActor suffers from unbounded memory usage in long-running
clusters. Its core metadata dictionaries, `datasets` and
`dataset_metadatas`, lacked a proper garbage collection (GC) mechanism.
This could lead to an Out-of-Memory (OOM) error in the _StatsActor when
a large number of datasets are created.

Solution  
This patch implements an eviction policy for _StatsActor based on the
`max_stats` configuration to effectively limit the size of the
`datasets` and `dataset_metadatas` dictionaries, preventing their
unbounded growth.

  The implementation details are as follows:  
  1. **Optimize Queue Implementation**  
The old, unused `fifo_queue` field has been removed. It is replaced by a
new `collections.deque`, `finished_datasets_queue`, which serves as a
more efficient FIFO queue for storing the tags of completed datasets.

  2. **Implement Eviction Logic**  
- When a dataset's status is updated to `FINISHED` or `FAILED`, its tag
is appended to the `finished_datasets_queue`.
- A check is then immediately performed to see if the total number of
entries in the `datasets` dictionary exceeds `max_stats`.
- If the limit is exceeded, the oldest dataset tag is popped from the
front of the `finished_datasets_queue`, and the corresponding entries
are synchronously deleted from the `datasets` and `dataset_metadatas`
dictionaries.

  3. **Clarify Limitation Strategy**  
`max_stats` is not a strict hard limit. Since the eviction logic is only
triggered when a dataset completes (`FINISHED` or `FAILED`), it is
possible for the number of `RUNNING` datasets to cause the total entry
count to temporarily exceed `max_stats`. This design ensures that
metadata for in-progress tasks is never evicted, while still effectively
preventing unbounded memory growth and OOM errors by cleaning up the
oldest completed data as soon as a task finishes.

Testing  
To verify the correctness of this fix, a new unit test,
`test_stats_actor_datasets_eviction`, has been added. This test sets a
low `max_stats` value and asserts that the oldest finished dataset is
correctly evicted when the limit is surpassed.

---------

Signed-off-by: qiwenju <qiwenju@xiaomi.com>
Co-authored-by: qiwenju <qiwenju@xiaomi.com>
Signed-off-by: Gang Zhao <gang@gang-JQ62HD2C37.local>
sampan-s-nayak pushed a commit to sampan-s-nayak/ray that referenced this pull request Sep 8, 2025
…5925)

Background
  Closes ray-project#43924.

The _StatsActor suffers from unbounded memory usage in long-running
clusters. Its core metadata dictionaries, `datasets` and
`dataset_metadatas`, lacked a proper garbage collection (GC) mechanism.
This could lead to an Out-of-Memory (OOM) error in the _StatsActor when
a large number of datasets are created.

Solution
This patch implements an eviction policy for _StatsActor based on the
`max_stats` configuration to effectively limit the size of the
`datasets` and `dataset_metadatas` dictionaries, preventing their
unbounded growth.

  The implementation details are as follows:
  1. **Optimize Queue Implementation**
The old, unused `fifo_queue` field has been removed. It is replaced by a
new `collections.deque`, `finished_datasets_queue`, which serves as a
more efficient FIFO queue for storing the tags of completed datasets.

  2. **Implement Eviction Logic**
- When a dataset's status is updated to `FINISHED` or `FAILED`, its tag
is appended to the `finished_datasets_queue`.
- A check is then immediately performed to see if the total number of
entries in the `datasets` dictionary exceeds `max_stats`.
- If the limit is exceeded, the oldest dataset tag is popped from the
front of the `finished_datasets_queue`, and the corresponding entries
are synchronously deleted from the `datasets` and `dataset_metadatas`
dictionaries.

  3. **Clarify Limitation Strategy**
`max_stats` is not a strict hard limit. Since the eviction logic is only
triggered when a dataset completes (`FINISHED` or `FAILED`), it is
possible for the number of `RUNNING` datasets to cause the total entry
count to temporarily exceed `max_stats`. This design ensures that
metadata for in-progress tasks is never evicted, while still effectively
preventing unbounded memory growth and OOM errors by cleaning up the
oldest completed data as soon as a task finishes.

Testing
To verify the correctness of this fix, a new unit test,
`test_stats_actor_datasets_eviction`, has been added. This test sets a
low `max_stats` value and asserts that the oldest finished dataset is
correctly evicted when the limit is surpassed.

---------

Signed-off-by: qiwenju <qiwenju@xiaomi.com>
Co-authored-by: qiwenju <qiwenju@xiaomi.com>
Signed-off-by: sampan <sampan@anyscale.com>
jugalshah291 pushed a commit to jugalshah291/ray_fork that referenced this pull request Sep 11, 2025
…5925)

Background
  Closes ray-project#43924.

The _StatsActor suffers from unbounded memory usage in long-running
clusters. Its core metadata dictionaries, `datasets` and
`dataset_metadatas`, lacked a proper garbage collection (GC) mechanism.
This could lead to an Out-of-Memory (OOM) error in the _StatsActor when
a large number of datasets are created.

Solution
This patch implements an eviction policy for _StatsActor based on the
`max_stats` configuration to effectively limit the size of the
`datasets` and `dataset_metadatas` dictionaries, preventing their
unbounded growth.

  The implementation details are as follows:
  1. **Optimize Queue Implementation**
The old, unused `fifo_queue` field has been removed. It is replaced by a
new `collections.deque`, `finished_datasets_queue`, which serves as a
more efficient FIFO queue for storing the tags of completed datasets.

  2. **Implement Eviction Logic**
- When a dataset's status is updated to `FINISHED` or `FAILED`, its tag
is appended to the `finished_datasets_queue`.
- A check is then immediately performed to see if the total number of
entries in the `datasets` dictionary exceeds `max_stats`.
- If the limit is exceeded, the oldest dataset tag is popped from the
front of the `finished_datasets_queue`, and the corresponding entries
are synchronously deleted from the `datasets` and `dataset_metadatas`
dictionaries.

  3. **Clarify Limitation Strategy**
`max_stats` is not a strict hard limit. Since the eviction logic is only
triggered when a dataset completes (`FINISHED` or `FAILED`), it is
possible for the number of `RUNNING` datasets to cause the total entry
count to temporarily exceed `max_stats`. This design ensures that
metadata for in-progress tasks is never evicted, while still effectively
preventing unbounded memory growth and OOM errors by cleaning up the
oldest completed data as soon as a task finishes.

Testing
To verify the correctness of this fix, a new unit test,
`test_stats_actor_datasets_eviction`, has been added. This test sets a
low `max_stats` value and asserts that the oldest finished dataset is
correctly evicted when the limit is surpassed.

---------

Signed-off-by: qiwenju <qiwenju@xiaomi.com>
Co-authored-by: qiwenju <qiwenju@xiaomi.com>
Signed-off-by: jugalshah291 <shah.jugal291@gmail.com>
wyhong3103 pushed a commit to wyhong3103/ray that referenced this pull request Sep 12, 2025
…5925)

Background
  Closes ray-project#43924.

The _StatsActor suffers from unbounded memory usage in long-running
clusters. Its core metadata dictionaries, `datasets` and
`dataset_metadatas`, lacked a proper garbage collection (GC) mechanism.
This could lead to an Out-of-Memory (OOM) error in the _StatsActor when
a large number of datasets are created.

Solution
This patch implements an eviction policy for _StatsActor based on the
`max_stats` configuration to effectively limit the size of the
`datasets` and `dataset_metadatas` dictionaries, preventing their
unbounded growth.

  The implementation details are as follows:
  1. **Optimize Queue Implementation**
The old, unused `fifo_queue` field has been removed. It is replaced by a
new `collections.deque`, `finished_datasets_queue`, which serves as a
more efficient FIFO queue for storing the tags of completed datasets.

  2. **Implement Eviction Logic**
- When a dataset's status is updated to `FINISHED` or `FAILED`, its tag
is appended to the `finished_datasets_queue`.
- A check is then immediately performed to see if the total number of
entries in the `datasets` dictionary exceeds `max_stats`.
- If the limit is exceeded, the oldest dataset tag is popped from the
front of the `finished_datasets_queue`, and the corresponding entries
are synchronously deleted from the `datasets` and `dataset_metadatas`
dictionaries.

  3. **Clarify Limitation Strategy**
`max_stats` is not a strict hard limit. Since the eviction logic is only
triggered when a dataset completes (`FINISHED` or `FAILED`), it is
possible for the number of `RUNNING` datasets to cause the total entry
count to temporarily exceed `max_stats`. This design ensures that
metadata for in-progress tasks is never evicted, while still effectively
preventing unbounded memory growth and OOM errors by cleaning up the
oldest completed data as soon as a task finishes.

Testing
To verify the correctness of this fix, a new unit test,
`test_stats_actor_datasets_eviction`, has been added. This test sets a
low `max_stats` value and asserts that the oldest finished dataset is
correctly evicted when the limit is surpassed.

---------

Signed-off-by: qiwenju <qiwenju@xiaomi.com>
Co-authored-by: qiwenju <qiwenju@xiaomi.com>
Signed-off-by: yenhong.wong <yenhong.wong@grabtaxi.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
community-contribution Contributed by the community data Ray Data-related issues go add ONLY when ready to merge, run all tests
Projects
None yet
Development

Successfully merging this pull request may close these issues.

[Data] _StatsActor has unbounded memory usage
4 participants