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

[core][autoscaler] Autoscaler v2 does not honor minReplicas/replicas count of the worker nodes and constantly terminates after idletimeout #48623

Merged
merged 4 commits into from
Nov 11, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
25 changes: 24 additions & 1 deletion python/ray/autoscaler/v2/scheduler.py
Original file line number Diff line number Diff line change
Expand Up @@ -1568,6 +1568,10 @@ def _enforce_idle_termination(
Args:
ctx: The schedule context.
"""
count_by_node_type = ctx.get_cluster_shape()
node_type_configs = ctx.get_node_type_configs()
terminate_nodes_by_type: Dict[NodeType, int] = defaultdict(int)

nodes = ctx.get_nodes()
s_to_ms = 1000
for node in nodes:
Expand All @@ -1593,13 +1597,32 @@ def _enforce_idle_termination(
# Skip it.
if node.idle_duration_ms > ctx.get_idle_timeout_s() * s_to_ms:
logger.debug(
"Node {}(idle for {} secs) is needed by the cluster resource "
"Node {} (idle for {} secs) is needed by the cluster resource "
"constraints, skip idle termination.".format(
node.ray_node_id, node.idle_duration_ms / s_to_ms
)
)
continue

# Honor the min_worker_nodes setting for the node type.
min_count = 0
node_type = node.node_type
if node_type in node_type_configs:
min_count = node_type_configs[node_type].min_worker_nodes
if (
count_by_node_type.get(node_type, 0)
- terminate_nodes_by_type[node_type]
<= min_count
):
logger.info(
"Node {} (idle for {} secs) belongs to node_type {} and is "
"required by min_worker_nodes, skipping idle termination.".format(
node.ray_node_id, node.idle_duration_ms / s_to_ms, node_type
)
)
continue
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: let's add a debug log?

Copy link
Member Author

Choose a reason for hiding this comment

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

Updated. Btw, I have several questions:

  1. I don't find any place in scheduler to call setLevel. How can I set the logger's level to DEBUG when I launch the autoscaler via CLI?

  2. How to determine it should be INFO or DEBUG?

Thanks!

Copy link
Contributor

Choose a reason for hiding this comment

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

I think for 1 - it's using ray configured logging AFAIK so that should be configured with how ray logging level is configured.

For 2 - it's more arbitrary and style.

Copy link
Member Author

Choose a reason for hiding this comment

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

there seems no way to configure log level when launching the autoscaler via Ray CLI.

Copy link
Member Author

Choose a reason for hiding this comment

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

there seems no way to configure log level when launching the autoscaler via Ray CLI.

I will verify whether this is correct. If so, I will open a PR to make it configurable.


terminate_nodes_by_type[node.node_type] += 1
# The node is idle for too long, terminate it.
node.status = SchedulingNodeStatus.TO_TERMINATE
node.termination_request = TerminationRequest(
Expand Down
85 changes: 85 additions & 0 deletions python/ray/autoscaler/v2/tests/test_scheduler.py
Original file line number Diff line number Diff line change
Expand Up @@ -1349,6 +1349,91 @@ def test_idle_termination(idle_timeout_s, has_resource_constraints):
assert len(to_terminate) == 0


@pytest.mark.parametrize("min_workers", [0, 1])
def test_idle_termination_with_min_worker(min_workers):
"""
Test that idle nodes are terminated.
"""
idle_timeout_s = 1

scheduler = ResourceDemandScheduler(event_logger)

node_type_configs = {
"type_cpu": NodeTypeConfig(
name="type_cpu",
resources={"CPU": 1},
min_worker_nodes=min_workers,
max_worker_nodes=5,
launch_config_hash="hash1",
),
"head_node": NodeTypeConfig(
name="head_node",
resources={"CPU": 0},
launch_config_hash="hash2",
min_worker_nodes=0,
max_worker_nodes=1,
),
}

idle_time_s = 5
constraints = []

request = sched_request(
node_type_configs=node_type_configs,
instances=[
make_autoscaler_instance(
im_instance=Instance(
instance_id="i-1",
instance_type="type_cpu",
status=Instance.RAY_RUNNING,
launch_config_hash="hash1",
node_id="r-1",
),
ray_node=NodeState(
ray_node_type_name="type_cpu",
node_id=b"r-1",
available_resources={"CPU": 1},
total_resources={"CPU": 1},
idle_duration_ms=idle_time_s * 1000,
status=NodeStatus.IDLE,
),
cloud_instance_id="c-1",
),
make_autoscaler_instance(
im_instance=Instance(
instance_id="i-2",
instance_type="head_node",
status=Instance.RAY_RUNNING,
launch_config_hash="hash2",
node_kind=NodeKind.HEAD,
node_id="r-2",
),
ray_node=NodeState(
ray_node_type_name="head_node",
node_id=b"r-2",
available_resources={"CPU": 0},
total_resources={"CPU": 0},
idle_duration_ms=999 * 1000, # idle
status=NodeStatus.IDLE,
),
cloud_instance_id="c-2",
),
],
idle_timeout_s=idle_timeout_s,
cluster_resource_constraints=constraints,
)

reply = scheduler.schedule(request)
_, to_terminate = _launch_and_terminate(reply)
assert idle_timeout_s <= idle_time_s
if min_workers == 0:
assert len(to_terminate) == 1
assert to_terminate == [("i-1", "r-1", TerminationRequest.Cause.IDLE)]
else:
assert min_workers > 0
assert len(to_terminate) == 0


def test_gang_scheduling():
"""
Test that gang scheduling works.
Expand Down