Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[Autoscaler][Placement Group] Skip placed bundle when requesting resource #48924

Open
wants to merge 4 commits into
base: master
Choose a base branch
from

Conversation

mimiliaogo
Copy link
Contributor

@mimiliaogo mimiliaogo commented Nov 25, 2024

Why are these changes needed?

Before the PR, when a node in a placement group (PG) goes down, the autoscaler attempts to reschedule the entire PG (all bundles). However, this will lead to overprovisioning. Details: #40212

This PR solved this by skipping already placed bundles (i.e., bundles with an associated node_id) when demanding resources in autoscaler.

Before: Every bundles get rescheduled

image

After: Only one node will be scaled up

Screenshot 2024-11-25 at 12 37 39 PM

Related issue number

Closes #40212

Checks

  • I've signed off every commit(by using the -s flag, i.e., git commit -s) in this PR.
  • I've run scripts/format.sh to lint the changes in this PR.
  • I've included any doc changes needed for https://docs.ray.io/en/master/.
    • I've added any new APIs to the API Reference. For example, if I added a
      method in Tune, I've added it in doc/source/tune/api/ under the
      corresponding .rst file.
  • I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/
  • Testing Strategy
    • Unit tests
    • Release tests
    • This PR is not tested :(

@mimiliaogo mimiliaogo requested review from hongchaodeng and a team as code owners November 25, 2024 18:57
@jcotant1 jcotant1 added the core Issues that should be addressed in Ray Core label Nov 25, 2024
@kevin85421 kevin85421 self-assigned this Nov 26, 2024
Copy link
Member

@kevin85421 kevin85421 left a comment

Choose a reason for hiding this comment

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

add a test

shapes = [dict(bundle.unit_resources) for bundle in placement_group.bundles]
# Skip **placed** bundle (which has node id associated with it).
for bundle in placement_group.bundles:
if bundle.node_id:
Copy link
Member

Choose a reason for hiding this comment

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

Is it an empty string or None? If it is None, use is instead.

Suggested change
if bundle.node_id:
if bundle.node_id is not None:

Copy link
Contributor Author

Choose a reason for hiding this comment

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

fix in 7a44207, it should be an empty byte string.

Signed-off-by: Mimi Liao <mimiliao2000@gmail.com>
Signed-off-by: Mimi Liao <mimiliao2000@gmail.com>
break

# TODO(mimi): kill_raylet won't trigger reschedule in autoscaler v1
# kill_raylet(node["NodeManagerAddress"], node["NodeManagerPort"])
Copy link
Contributor Author

Choose a reason for hiding this comment

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

I found that when using kill_raylet, reschedule won't be triggered in autoscaler v1, even when the cluster status shows the node is killed. In this case, v1 will fail and v2 pass.
Both v1 and v2 pass when using kill_node.

Signed-off-by: Mimi Liao <mimiliao2000@gmail.com>

from ray.autoscaler.v2.sdk import get_cluster_status

def verify_nodes(active=3, idle=1):
Copy link
Member

Choose a reason for hiding this comment

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

Suggested change
def verify_nodes(active=3, idle=1):
def verify_nodes(active, idle):


def kill_node(node_id):
# kill -9
import subprocess
Copy link
Member

Choose a reason for hiding this comment

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

Move import to top-level. Typically, Ray uses deferred import only to avoid circular dependencies.

wait_for_condition(lambda: verify_nodes(3, 1))

# Kill a node
def kill_raylet(ip, port, graceful=True):
Copy link
Member

Choose a reason for hiding this comment

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

Remove this function because it is not used for now.

# Wait for the node to be removed
wait_for_condition(lambda: verify_nodes(2, 1), 20)

# Check that the placement group is rescheduled
Copy link
Member

Choose a reason for hiding this comment

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

where is the logic to check the placement group is rescheduled?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

wait_for_condition(lambda: verify_nodes(3, 1)) is to check the autoscaler rescheduling. However, this comment is redundant, I've already removed it.


ray.get(pg.ready())

from ray.autoscaler.v2.sdk import get_cluster_status
Copy link
Member

Choose a reason for hiding this comment

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

Do we need to import this? It seems to have already been imported at the top level.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Above suggestions fixed in 415dcf8

Signed-off-by: Mimi Liao <mimiliao2000@gmail.com>
@kevin85421 kevin85421 added the go add ONLY when ready to merge, run all tests label Dec 17, 2024
@kevin85421
Copy link
Member

CI fails. Can you fix the CI errors?

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
core Issues that should be addressed in Ray Core go add ONLY when ready to merge, run all tests
Projects
None yet
Development

Successfully merging this pull request may close these issues.

[core][autoscaler][v1] Autoscaler overprovisions nodes when strict placement group is rescheduling
3 participants