[Data] AutoscalingCoordinator prevent double-allocates resources if there are multiple datasets#59740
Conversation
Signed-off-by: machichima <nary12321@gmail.com>
Signed-off-by: machichima <nary12321@gmail.com>
There was a problem hiding this comment.
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.
python/ray/data/_internal/cluster_autoscaler/default_autoscaling_coordinator.py
Outdated
Show resolved
Hide resolved
|
@bveeramani I draft a fix to perform the fair remaining resource allocation. PTAL |
Signed-off-by: machichima <nary12321@gmail.com>
Signed-off-by: machichima <nary12321@gmail.com>
bveeramani
left a comment
There was a problem hiding this comment.
Overall looks good. Just a few questions
| # 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) |
There was a problem hiding this comment.
What happens if we perform true division rather than integer devision? Should we?
There was a problem hiding this comment.
I think we can only use integer devision here as the Autoscaler SDK only support integer value?
There was a problem hiding this comment.
Ah, got it. Underallocating seems reasonable for now
| with ( | ||
| patch("ray.nodes", return_value=cluster_nodes), | ||
| patch("time.time", mock_time), | ||
| patch("ray.autoscaler.sdk.request_resources"), | ||
| ): |
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
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?
There was a problem hiding this comment.
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
python/ray/data/_internal/cluster_autoscaler/default_autoscaling_coordinator.py
Outdated
Show resolved
Hide resolved
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) |
There was a problem hiding this comment.
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.
… 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>
…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>
…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>
…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>
… 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>
… 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>
…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>
… 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>
…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>
… 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>
…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>
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