Skip to content

[Data] AutoscalingCoordinator prevent double-allocates resources if there are multiple datasets#59740

Merged
bveeramani merged 10 commits intoray-project:masterfrom
machichima:59685-double-allocate-resource
Dec 31, 2025
Merged

[Data] AutoscalingCoordinator prevent double-allocates resources if there are multiple datasets#59740
bveeramani merged 10 commits intoray-project:masterfrom
machichima:59685-double-allocate-resource

Conversation

@machichima
Copy link
Copy Markdown
Contributor

@machichima machichima commented Dec 29, 2025

Description

Prevent double-allocates remaining resources when there are multiple datasets (multiple requests).

This PR divides the remaining resources with the number of remaining resource requests to form the fair resource allocation

Related issues

Closes #59685

Additional information

Optional: Add implementation details, API changes, usage examples, screenshots, etc.

Signed-off-by: machichima <nary12321@gmail.com>
Signed-off-by: machichima <nary12321@gmail.com>
@machichima machichima requested a review from a team as a code owner December 29, 2025 13:36
Copy link
Copy Markdown
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 effectively addresses the issue of over-allocating remaining resources when multiple datasets have pending requests. The new logic correctly divides the available resources among the requesters, ensuring a fair distribution. The implementation is sound and is well-supported by a new unit test that verifies the corrected behavior. I have one minor suggestion to improve the readability and efficiency of the resource allocation logic.

@machichima
Copy link
Copy Markdown
Contributor Author

@bveeramani I draft a fix to perform the fair remaining resource allocation. PTAL
Thank you!

Signed-off-by: machichima <nary12321@gmail.com>
Signed-off-by: machichima <nary12321@gmail.com>
@ray-gardener ray-gardener bot added data Ray Data-related issues community-contribution Contributed by the community labels Dec 29, 2025
Copy link
Copy Markdown
Member

@bveeramani bveeramani left a comment

Choose a reason for hiding this comment

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

Overall looks good. Just a few questions

Comment on lines +404 to +417
# NOTE: to handle the case where multiple datasets are running concurrently,
# we divide remaining resources equally to all requesters with `request_remaining=True`.
num_remaining_requesters = len(remaining_resource_requesters)
if num_remaining_requesters > 0:
for node_resource in cluster_node_resources:
# Divide remaining resources equally among requesters.
# NOTE: Integer division may leave some resources unallocated.
divided_resource = {
k: v // num_remaining_requesters
for k, v in node_resource.items()
}
for ongoing_req in remaining_resource_requesters:
if any(v > 0 for v in divided_resource.values()):
ongoing_req.allocated_resources.append(divided_resource)
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

What happens if we perform true division rather than integer devision? Should we?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

I think we can only use integer devision here as the Autoscaler SDK only support integer value?

# Round up the resource values to integers,
# because the Autoscaler SDK only accepts integer values.
for r in resources:
for k in r:
r[k] = math.ceil(r[k])

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Ah, got it. Underallocating seems reasonable for now

Comment on lines +428 to +432
with (
patch("ray.nodes", return_value=cluster_nodes),
patch("time.time", mock_time),
patch("ray.autoscaler.sdk.request_resources"),
):
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Out-of-scope for this PR, but I think this DefaultAutoscalingCoordinator would be more maintainable long-term if we used explicit seams rather than implicit depedencies:

class DefaultAutoscalingCoordinator:
    def __init__(
        self,
        ...,
        get_node_resources: Callable[[], List[ResourceDict]],
        get_time: Callable[[], float] = time.time,
        request_resources: Callable[[List[ResourceDict], None] = ray.autoscaler.sdk.request_resources
)

The problem with patching is that this test will break if we change some of these implementation details (e.g., use time.perf_counter instead of time.time), and it's also less clear what the dependencies of the component actually are.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Make sense! Do you think it's ok for me to do this in the follow-up PR? It seems like we need to do for both DefaultAutoscalingCoordinator and _AutoscalingCoordinatorActor?

Copy link
Copy Markdown
Member

@bveeramani bveeramani Dec 31, 2025

Choose a reason for hiding this comment

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

Yeah, I think that's totally fine as follow-up.

It seems like we need to do for both DefaultAutoscalingCoordinator and _AutoscalingCoordinatorActor?

I looked through the tests, and I think it's okay if we just add it to _AutoscalingCoordinatorActor for now. Seems like most of the core unit tests (test_basic and the newly-added test_double_allocation_with_multiple_request_remaining) are against the _AutoscalingCoordinatorActor layer of abstraction, rather than DefaultAutoscalingCoordinator

machichima and others added 4 commits December 30, 2025 20:55
Signed-off-by: machichima <nary12321@gmail.com>
…s used

Signed-off-by: machichima <nary12321@gmail.com>
Signed-off-by: machichima <nary12321@gmail.com>
}
for ongoing_req in remaining_resource_requesters:
if any(v > 0 for v in divided_resource.values()):
ongoing_req.allocated_resources.append(divided_resource)
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Shared dictionary reference across multiple requesters

The divided_resource dictionary is created once per node resource (outside the inner loop) and then the same dictionary object is appended to all requesters' allocated_resources lists. This causes all requesters with request_remaining=True to share references to identical dictionary objects. While this works for read-only access and value comparisons, any modification to a shared dictionary would unintentionally affect all requesters. The dictionary creation should be moved inside the inner loop or a .copy() should be used when appending.

Fix in Cursor Fix in Web

Copy link
Copy Markdown
Member

@bveeramani bveeramani left a comment

Choose a reason for hiding this comment

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

LGTM!

@bveeramani bveeramani added the go add ONLY when ready to merge, run all tests label Dec 31, 2025
@bveeramani bveeramani merged commit 394cd2d into ray-project:master Dec 31, 2025
6 checks passed
AYou0207 pushed a commit to AYou0207/ray that referenced this pull request Jan 13, 2026
… there are multiple datasets (ray-project#59740)

## Description

Prevent double-allocates remaining resources when there are multiple
datasets (multiple requests).

This PR divides the remaining resources with the number of remaining
resource requests to form the fail resource allocation

## Related issues

Closes ray-project#59685

## Additional information
> Optional: Add implementation details, API changes, usage examples,
screenshots, etc.

---------

Signed-off-by: machichima <nary12321@gmail.com>
Co-authored-by: Balaji Veeramani <balaji@anyscale.com>
Signed-off-by: jasonwrwang <jasonwrwang@tencent.com>
bveeramani added a commit that referenced this pull request Jan 13, 2026
…60037)

## Description

As mentioned in
#59740 (comment),
add explicit args in `_AutoscalingCoordinatorActor` constructor to
improve maintainability.

## Related issues

Follow-up: #59740

## Additional information
- Pass in mock function in testing as args rather than using `patch`

---------

Signed-off-by: machichima <nary12321@gmail.com>
Co-authored-by: Balaji Veeramani <balaji@anyscale.com>
rushikeshadhav pushed a commit to rushikeshadhav/ray that referenced this pull request Jan 14, 2026
…ay-project#60037)

## Description

As mentioned in
ray-project#59740 (comment),
add explicit args in `_AutoscalingCoordinatorActor` constructor to
improve maintainability.

## Related issues

Follow-up: ray-project#59740

## Additional information
- Pass in mock function in testing as args rather than using `patch`

---------

Signed-off-by: machichima <nary12321@gmail.com>
Co-authored-by: Balaji Veeramani <balaji@anyscale.com>
jeffery4011 pushed a commit to jeffery4011/ray that referenced this pull request Jan 20, 2026
…ay-project#60037)

## Description

As mentioned in
ray-project#59740 (comment),
add explicit args in `_AutoscalingCoordinatorActor` constructor to
improve maintainability.

## Related issues

Follow-up: ray-project#59740

## Additional information
- Pass in mock function in testing as args rather than using `patch`

---------

Signed-off-by: machichima <nary12321@gmail.com>
Co-authored-by: Balaji Veeramani <balaji@anyscale.com>
Signed-off-by: jeffery4011 <jefferyshen1015@gmail.com>
lee1258561 pushed a commit to pinterest/ray that referenced this pull request Feb 3, 2026
… there are multiple datasets (ray-project#59740)

## Description

Prevent double-allocates remaining resources when there are multiple
datasets (multiple requests).

This PR divides the remaining resources with the number of remaining
resource requests to form the fail resource allocation

## Related issues

Closes ray-project#59685

## Additional information
> Optional: Add implementation details, API changes, usage examples,
screenshots, etc.

---------

Signed-off-by: machichima <nary12321@gmail.com>
Co-authored-by: Balaji Veeramani <balaji@anyscale.com>
ryanaoleary pushed a commit to ryanaoleary/ray that referenced this pull request Feb 3, 2026
… there are multiple datasets (ray-project#59740)

## Description

Prevent double-allocates remaining resources when there are multiple
datasets (multiple requests).

This PR divides the remaining resources with the number of remaining
resource requests to form the fail resource allocation

## Related issues

Closes ray-project#59685

## Additional information
> Optional: Add implementation details, API changes, usage examples,
screenshots, etc.

---------

Signed-off-by: machichima <nary12321@gmail.com>
Co-authored-by: Balaji Veeramani <balaji@anyscale.com>
ryanaoleary pushed a commit to ryanaoleary/ray that referenced this pull request Feb 3, 2026
…ay-project#60037)

## Description

As mentioned in
ray-project#59740 (comment),
add explicit args in `_AutoscalingCoordinatorActor` constructor to
improve maintainability.

## Related issues

Follow-up: ray-project#59740

## Additional information
- Pass in mock function in testing as args rather than using `patch`

---------

Signed-off-by: machichima <nary12321@gmail.com>
Co-authored-by: Balaji Veeramani <balaji@anyscale.com>
peterxcli pushed a commit to peterxcli/ray that referenced this pull request Feb 25, 2026
… there are multiple datasets (ray-project#59740)

## Description

Prevent double-allocates remaining resources when there are multiple
datasets (multiple requests).

This PR divides the remaining resources with the number of remaining
resource requests to form the fail resource allocation

## Related issues

Closes ray-project#59685

## Additional information
> Optional: Add implementation details, API changes, usage examples,
screenshots, etc.

---------

Signed-off-by: machichima <nary12321@gmail.com>
Co-authored-by: Balaji Veeramani <balaji@anyscale.com>
Signed-off-by: peterxcli <peterxcli@gmail.com>
peterxcli pushed a commit to peterxcli/ray that referenced this pull request Feb 25, 2026
…ay-project#60037)

## Description

As mentioned in
ray-project#59740 (comment),
add explicit args in `_AutoscalingCoordinatorActor` constructor to
improve maintainability.

## Related issues

Follow-up: ray-project#59740

## Additional information
- Pass in mock function in testing as args rather than using `patch`

---------

Signed-off-by: machichima <nary12321@gmail.com>
Co-authored-by: Balaji Veeramani <balaji@anyscale.com>
Signed-off-by: peterxcli <peterxcli@gmail.com>
peterxcli pushed a commit to peterxcli/ray that referenced this pull request Feb 25, 2026
… there are multiple datasets (ray-project#59740)

## Description

Prevent double-allocates remaining resources when there are multiple
datasets (multiple requests).

This PR divides the remaining resources with the number of remaining
resource requests to form the fail resource allocation

## Related issues

Closes ray-project#59685

## Additional information
> Optional: Add implementation details, API changes, usage examples,
screenshots, etc.

---------

Signed-off-by: machichima <nary12321@gmail.com>
Co-authored-by: Balaji Veeramani <balaji@anyscale.com>
Signed-off-by: peterxcli <peterxcli@gmail.com>
peterxcli pushed a commit to peterxcli/ray that referenced this pull request Feb 25, 2026
…ay-project#60037)

## Description

As mentioned in
ray-project#59740 (comment),
add explicit args in `_AutoscalingCoordinatorActor` constructor to
improve maintainability.

## Related issues

Follow-up: ray-project#59740

## Additional information
- Pass in mock function in testing as args rather than using `patch`

---------

Signed-off-by: machichima <nary12321@gmail.com>
Co-authored-by: Balaji Veeramani <balaji@anyscale.com>
Signed-off-by: peterxcli <peterxcli@gmail.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] AutoscalingCoordinator double-allocates resources if there are multiple datasets

2 participants