From 50129ebf3ea813e297cb701508bffe292d671641 Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Tue, 27 Feb 2024 20:59:22 +0000 Subject: [PATCH 01/45] Multi-host replica deletion logic initial commit Signed-off-by: ryanaoleary --- .../_private/kuberay/node_provider.py | 19 ++++++++++++++++++- 1 file changed, 18 insertions(+), 1 deletion(-) diff --git a/python/ray/autoscaler/_private/kuberay/node_provider.py b/python/ray/autoscaler/_private/kuberay/node_provider.py index 1e312d76d3601..9e92f17d04362 100644 --- a/python/ray/autoscaler/_private/kuberay/node_provider.py +++ b/python/ray/autoscaler/_private/kuberay/node_provider.py @@ -38,6 +38,9 @@ RAY_HEAD_POD_NAME = os.getenv("RAY_HEAD_POD_NAME") +# Key for label that identifies which multi-host relica a pod belongs to +MULTIHOST_REPLICA_KEY = "multihost-replica" + # Design: # Each modification the autoscaler wants to make is posted to the API server goal state @@ -72,7 +75,8 @@ def node_data_from_pod(pod: Dict[str, Any]) -> NodeData: kind, type = kind_and_type(pod) status = status_tag(pod) ip = pod_ip(pod) - return NodeData(kind=kind, type=type, status=status, ip=ip) + multihost_replica = multihost_replica_id(pod) + return NodeData(kind=kind, type=type, multihost_replica=multihost_replica, status=status, ip=ip) def kind_and_type(pod: Dict[str, Any]) -> Tuple[NodeKind, NodeType]: @@ -88,6 +92,12 @@ def kind_and_type(pod: Dict[str, Any]) -> Tuple[NodeKind, NodeType]: type = labels[KUBERAY_LABEL_KEY_TYPE] return kind, type +def multihost_replica_id(pod: Dict[str, Any]) -> string: + """ Returns the replica identifier for a multi-host worker group. + If the pod belongs to a single-host group, returns an empty string. + """ + labels = pod["metadata"]["labels"] + return labels[MULTIHOST_REPLICA_KEY] def pod_ip(pod: Dict[str, Any]) -> NodeIP: return pod["status"].get("podIP", "IP not yet assigned") @@ -201,6 +211,9 @@ def _worker_group_replicas(raycluster: Dict[str, Any], group_index: int): # 1 is the default replicas value used by the KubeRay operator return raycluster["spec"]["workerGroupSpecs"][group_index].get("replicas", 1) +def _worker_group_num_of_hosts(raycluster: Dict[str, Any], group_index: int): + # Extract NumOfHosts of a worker group. 1 is the default NumOfHosts value used by the Kuberay operator. + return raycluster["spec"]["workerGroupSpecs"][group_index].get("numOfHosts", 1) class KubeRayNodeProvider(BatchingNodeProvider): # type: ignore def __init__( @@ -362,6 +375,10 @@ def _scale_request_to_patch_payload( for node_type, target_replicas in scale_request.desired_num_workers.items(): group_index = _worker_group_index(raycluster, node_type) group_max_replicas = _worker_group_max_replicas(raycluster, group_index) + group_num_of_hosts = _worker_group_num_of_hosts(raycluster, group_index) + # Account for multi-host worker groups + if group_num_of_hosts != 0: + target_replicas /= group_num_of_hosts # Cap the replica count to maxReplicas. if group_max_replicas is not None and group_max_replicas < target_replicas: logger.warning( From f532cf1d2f610142aba87659b2b179579dbecc53 Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Tue, 27 Feb 2024 21:03:54 +0000 Subject: [PATCH 02/45] Multi-host replica deletion logic Signed-off-by: ryanaoleary --- python/ray/autoscaler/_private/autoscaler.py | 57 +++++++++++++++++++ .../ray/autoscaler/batching_node_provider.py | 27 ++++++++- python/ray/autoscaler/tags.py | 2 + 3 files changed, 83 insertions(+), 3 deletions(-) diff --git a/python/ray/autoscaler/_private/autoscaler.py b/python/ray/autoscaler/_private/autoscaler.py index d621602f39267..14c831c918b46 100644 --- a/python/ray/autoscaler/_private/autoscaler.py +++ b/python/ray/autoscaler/_private/autoscaler.py @@ -126,6 +126,8 @@ def __init__(self, provider: NodeProvider): self.worker_ids: List[NodeID] = [] # The head node (node kind "head") self.head_id: Optional[NodeID] = None + # Map of multi-host replica IDs to nodes in each replica + self.multi_host_replicas_to_workers = defaultdict(list) for node in self.all_node_ids: node_kind = provider.node_tags(node)[TAG_RAY_NODE_KIND] @@ -133,6 +135,8 @@ def __init__(self, provider: NodeProvider): self.worker_ids.append(node) elif node_kind == NODE_KIND_HEAD: self.head_id = node + node_multihost_replica = provider.node_tags(node)[TAG_RAY_MULTIHOST_REPLICA] + self.multi_host_replicas_to_workers.append(node) # Note: For typical use-cases, self.all_node_ids == self.worker_ids + # [self.head_id]. The difference being in the case of unmanaged nodes. @@ -409,6 +413,9 @@ def _update(self): # This will accumulate the nodes we need to terminate. self.nodes_to_terminate = [] + # This will accumulate the multi-host replicas we need to terminate. + self.multi_host_replicas_to_delete = [] + # Update running nodes gauge num_workers = len(self.non_terminated_nodes.worker_ids) self.prom_metrics.running_workers.set(num_workers) @@ -481,6 +488,7 @@ def terminate_nodes_to_enforce_config_constraints(self, now: float): (3) Terminates outdated nodes, namely nodes whose configs don't match `node_config` for the relevant node type. + (4) Terminate nodes part of a multi-host replica being deleted. Avoids terminating non-outdated nodes required by autoscaler.sdk.request_resources(). @@ -527,6 +535,11 @@ def keep_node(node_id: NodeID) -> None: should_keep_or_terminate, reason = self._keep_worker_of_node_type( node_id, node_type_counts ) + if should_keep_or_terminate == KeepOrTerminate.terminate: + self.schedule_node_termination(node_id, reason, logger.info) + continue + # Check if node is part of a multi-host replica that is being deleted + should_keep_or_terminate, reason = self._keep_worker_of_multihost_replica(node_id) if should_keep_or_terminate == KeepOrTerminate.terminate: self.schedule_node_termination(node_id, reason, logger.info) continue @@ -604,6 +617,12 @@ def schedule_node_termination( aggregate=operator.add, ) self.nodes_to_terminate.append(node_id) + # Scale down entire multi-host replica of node being deleted + tags = self.provider.node_tags(node_id) + if TAG_RAY_MULTIHOST_REPLICA in tags: + multi_host_replica_id = tags[TAG_RAY_MULTIHOST_REPLICA] + if multi_host_replica_id not in self.multi_host_replicas_to_delete: + self.multi_host_replicas_to_delete.append(multi_host_replica_id) def terminate_scheduled_nodes(self): """Terminate scheduled nodes and clean associated autoscaler state.""" @@ -623,6 +642,13 @@ def terminate_scheduled_nodes(self): for node in self.nodes_to_terminate: self.node_tracker.untrack(node) self.prom_metrics.stopped_nodes.inc() + # Clean up multi-host replicas to delete + tags = self.provider.node_tags(node) + if TAG_RAY_MULTIHOST_REPLICA in tags: + multi_host_replica_id = tags[TAG_RAY_MULTIHOST_REPLICA] + self.multi_host_replicas_to_workers[multi_host_replica_id].remove(node) + if len(self.multi_host_replicas_to_workers[multi_host_replica_id]) == 0: + self.multi_host_replicas_to_delete.remove(multi_host_replica_id) # Update internal node lists self.non_terminated_nodes.remove_terminating_nodes(self.nodes_to_terminate) @@ -1008,6 +1034,37 @@ def _keep_worker_of_node_type( return KeepOrTerminate.decide_later, None + def _keep_worker_of_multihost_replica(self, node_id: NodeID) -> Tuple[KeepOrTerminate, Optional[str]]: + """Determines if a worker should be kept based on whether any + other nodes in the same multi-host replica have been deleted. + + Returns KeepOrTerminate.terminate when: + (a) The worker belongs to a multi-host replica scheduled to delete. + + Return KeepOrTerminate.keep otherwise. + + Returns: + KeepOrTerminate: keep if the node should be kept, terminate if the + node should be terminated + Optional[str]: reason for termination. Not None on + KeepOrTerminate.terminate, None otherwise. + """ + # For type checking, assert that this object has been instantitiated. + assert self.provider + + tags = self.provider.node_tags(node_id) + if TAG_RAY_MULTIHOST_REPLICA in tags: + multi_host_replica_id = tags[TAG_RAY_MULTIHOST_REPLICA] + + if multi_host_replica_id in self.multi_host_replicas_to_delete: + # All pods in this multi-host replica should be deleted + return ( + KeepOrTerminate.terminate, + f"Pod belongs to a multi-host replica being deleted: {multi_host_replica_id}", + ) + + return KeepOrTerminate.keep, None + def _node_resources(self, node_id): node_type = self.provider.node_tags(node_id).get(TAG_RAY_USER_NODE_TYPE) if self.available_node_types: diff --git a/python/ray/autoscaler/batching_node_provider.py b/python/ray/autoscaler/batching_node_provider.py index 7c7061c5cf507..1101b3de8e6a7 100644 --- a/python/ray/autoscaler/batching_node_provider.py +++ b/python/ray/autoscaler/batching_node_provider.py @@ -15,6 +15,7 @@ TAG_RAY_NODE_KIND, TAG_RAY_NODE_STATUS, TAG_RAY_USER_NODE_TYPE, + TAG_RAY_MULTIHOST_REPLICA, ) provider_exists = False @@ -43,6 +44,9 @@ class NodeData: Attributes: kind: Whether the node is the head or a worker. type: The user-defined type of the node. + multihost_replica: An identifier for nodes belonging to a replica of a multi-host worker group. + * This value is set as a label by a webhook when NumOfHosts > 1 in a worker group + * This var is empty for pods belonging to single-host worker groups ip: Cluster-internal ip of the node. ip can be None if the ip has not yet been assigned. status: The status of the node. You must adhere to the following semantics @@ -56,6 +60,7 @@ class NodeData: kind: NodeKind type: NodeType + multihost_replica: string ip: Optional[NodeIP] status: NodeStatus @@ -160,6 +165,11 @@ def non_terminated_nodes(self, tag_filters: Dict[str, str]) -> List[str]: workers_to_delete=set(), # No workers to delete yet ) all_nodes = list(self.node_data_dict.keys()) + # Initialize multi-host replica to workers map + self.multi_host_replicas_to_workers = defaultdict(list) + for node_id in all_nodes: + multi_host_replica = self.node_data_dict[node_id].multihost_replica + self.multi_host_replicas_to_workers[multi_host_replica].append(node_id) # Support filtering by TAG_RAY_NODE_KIND, TAG_RAY_NODE_STATUS, and # TAG_RAY_USER_NODE_TYPE. # The autoscaler only uses tag_filters={}, @@ -191,6 +201,7 @@ def node_tags(self, node_id: str) -> Dict[str, str]: TAG_RAY_NODE_KIND: node_data.kind, TAG_RAY_NODE_STATUS: node_data.status, TAG_RAY_USER_NODE_TYPE: node_data.type, + TAG_RAY_MULTIHOST_REPLICA: node_data.multihost_replica, } def internal_ip(self, node_id: str) -> str: @@ -229,7 +240,17 @@ def terminate_node(self, node_id: str) -> Optional[Dict[str, Any]]: "NodeProvider attempted to request less than 0 workers of type " f"{node_type}. Skipping termination request." ) - - self.scale_request.desired_num_workers[node_type] -= 1 - self.scale_request.workers_to_delete.add(node_id) + + # Scale down entire replica if part of a multi-host group + node_multihost_replica = self.node_data_dict[node_id].multihost_replica + if node_multihost_replica != "": + for worker in self.multi_host_replicas_to_workers[node_multihost_replica]: + # Check if worker has already been scheduled to delete + if node_id not in self.scale_request.workers_to_delete: + # Assume all workers in a group are of the same type + self.scale_request.desired_num_workers[node_type] -= 1 + self.scale_request.workers_to_delete.add(node_id) + else: + self.scale_request.desired_num_workers[node_type] -= 1 + self.scale_request.workers_to_delete.add(node_id) self.scale_change_needed = True diff --git a/python/ray/autoscaler/tags.py b/python/ray/autoscaler/tags.py index 380b4450d6eca..f2d7514415f40 100644 --- a/python/ray/autoscaler/tags.py +++ b/python/ray/autoscaler/tags.py @@ -13,6 +13,8 @@ # Tag for user defined node types (e.g., m4xl_spot). This is used for multi # node type clusters. TAG_RAY_USER_NODE_TYPE = "ray-user-node-type" +# Tag for multi-host replica node belongs to. Used for multi-host worker groups. +TAG_RAY_MULTIHOST_REPLICA = "ray-multihost-replica" # Tag for autofilled node types for legacy cluster yamls without multi # node type defined in the cluster configs. NODE_TYPE_LEGACY_HEAD = "ray-legacy-head-node-type" From 8a1d4b919ae4de95c2e8e45f6a114cc2df9f88a1 Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Wed, 8 May 2024 10:22:20 +0000 Subject: [PATCH 03/45] Fix broken tests Signed-off-by: ryanaoleary --- .../_private/kuberay/node_provider.py | 14 ++++ .../tests/kuberay/test_autoscaling_config.py | 35 +++++++-- .../kuberay/test_kuberay_node_provider.py | 71 +++++++++++++++---- 3 files changed, 101 insertions(+), 19 deletions(-) diff --git a/python/ray/autoscaler/_private/kuberay/node_provider.py b/python/ray/autoscaler/_private/kuberay/node_provider.py index 9e92f17d04362..33b6eb1e45822 100644 --- a/python/ray/autoscaler/_private/kuberay/node_provider.py +++ b/python/ray/autoscaler/_private/kuberay/node_provider.py @@ -92,6 +92,7 @@ def kind_and_type(pod: Dict[str, Any]) -> Tuple[NodeKind, NodeType]: type = labels[KUBERAY_LABEL_KEY_TYPE] return kind, type + def multihost_replica_id(pod: Dict[str, Any]) -> string: """ Returns the replica identifier for a multi-host worker group. If the pod belongs to a single-host group, returns an empty string. @@ -99,6 +100,19 @@ def multihost_replica_id(pod: Dict[str, Any]) -> string: labels = pod["metadata"]["labels"] return labels[MULTIHOST_REPLICA_KEY] + +def replica_index_label(pod: Dict[str, Any]) -> Optional[str]: + """Returns the replicaIndex label for a Pod in a TPU worker group. + The replicaIndex label is set by the GKE TPU Ray webhook and is of + the form {$WORKER_GROUP_NAME-$REPLICA_INDEX} where $REPLICA_INDEX + is an integer from 0 to Replicas-1. + """ + labels = pod["metadata"]["labels"] + if REPLICA_INDEX_KEY in labels: + labels[REPLICA_INDEX_KEY] + return None + + def pod_ip(pod: Dict[str, Any]) -> NodeIP: return pod["status"].get("podIP", "IP not yet assigned") diff --git a/python/ray/tests/kuberay/test_autoscaling_config.py b/python/ray/tests/kuberay/test_autoscaling_config.py index c6f59ab5de492..3e5de36ded042 100644 --- a/python/ray/tests/kuberay/test_autoscaling_config.py +++ b/python/ray/tests/kuberay/test_autoscaling_config.py @@ -19,7 +19,7 @@ def get_basic_ray_cr() -> dict: """Returns the example Ray CR included in the Ray documentation, - modified to include a GPU worker group. + modified to include a GPU worker group and a TPU worker group.. """ cr_path = str( Path(__file__).resolve().parents[2] @@ -35,6 +35,20 @@ def get_basic_ray_cr() -> dict: ) gpu_group["maxReplicas"] = 200 config["spec"]["workerGroupSpecs"].append(gpu_group) + tpu_group = copy.deepcopy(config["spec"]["workerGroupSpecs"][0]) + tpu_group["groupName"] = "tpu-group" + tpu_group["template"]["spec"]["containers"][0]["resources"]["limits"].setdefault( + "google.com/tpu", 8 + ) + tpu_group["template"]["spec"]["nodeSelector"] = {} + tpu_group["template"]["spec"]["nodeSelector"][ + "cloud.google.com/gke-tpu-topology" + ] = "2x2x2" + tpu_group["template"]["spec"]["nodeSelector"][ + "cloud.google.com/gke-tpu-accelerator" + ] = "tpu-v4-podslice" + tpu_group["maxReplicas"] = 4 + config["spec"]["workerGroupSpecs"].append(tpu_group) return config @@ -88,6 +102,19 @@ def _get_basic_autoscaling_config() -> dict: "GPU": 3, }, }, + # Same as "small-group" with a TPU resource entry added + # and modified max_workers and node_config. + "tpu-group": { + "max_workers": 4, + "min_workers": 1, + "node_config": {}, + "resources": { + "CPU": 1, + "memory": 536870912, + "Custom2": 5, + "Custom3": 1, + }, + }, }, "auth": {}, "cluster_synced_files": [], @@ -98,7 +125,7 @@ def _get_basic_autoscaling_config() -> dict: "head_start_ray_commands": [], "idle_timeout_minutes": 1.0, "initialization_commands": [], - "max_workers": 500, + "max_workers": 504, "setup_commands": [], "upscaling_speed": 1000, "worker_setup_commands": [], @@ -280,8 +307,8 @@ def test_cr_image_consistency(): cr = get_basic_ray_cr() group_specs = [cr["spec"]["headGroupSpec"]] + cr["spec"]["workerGroupSpecs"] - # Head, CPU group, GPU group. - assert len(group_specs) == 3 + # Head, CPU group, GPU group, TPU group. + assert len(group_specs) == 4 ray_containers = [ group_spec["template"]["spec"]["containers"][0] for group_spec in group_specs diff --git a/python/ray/tests/kuberay/test_kuberay_node_provider.py b/python/ray/tests/kuberay/test_kuberay_node_provider.py index 6c212d6152d4c..40c5e27a9a32f 100644 --- a/python/ray/tests/kuberay/test_kuberay_node_provider.py +++ b/python/ray/tests/kuberay/test_kuberay_node_provider.py @@ -22,7 +22,9 @@ def _get_basic_ray_cr_workers_to_delete( - cpu_workers_to_delete: List[NodeID], gpu_workers_to_delete: List[NodeID] + cpu_workers_to_delete: List[NodeID], + gpu_workers_to_delete: List[NodeID], + tpu_workers_to_delete: List[NodeID], ): """Generate a Ray cluster with non-empty workersToDelete field.""" raycluster = get_basic_ray_cr() @@ -32,6 +34,9 @@ def _get_basic_ray_cr_workers_to_delete( raycluster["spec"]["workerGroupSpecs"][1]["scaleStrategy"] = { "workersToDelete": gpu_workers_to_delete } + raycluster["spec"]["workerGroupSpecs"][2]["scaleStrategy"] = { + "workersToDelete": tpu_workers_to_delete + } return raycluster @@ -57,7 +62,7 @@ def test_worker_group_index(group_name, expected_index): @pytest.mark.skipif(sys.platform.startswith("win"), reason="Not relevant on Windows.") @pytest.mark.parametrize( "group_index,expected_max_replicas,expected_replicas", - [(0, 300, 1), (1, 200, 1), (2, None, 0)], + [(0, 300, 1), (1, 200, 1), (2, 4, 1), (3, None, 0)], ) def test_worker_group_replicas(group_index, expected_max_replicas, expected_replicas): """Basic unit test for _worker_group_max_replicas and _worker_group_replicas @@ -119,10 +124,18 @@ def test_create_node_cap_at_max( "podlist1.yaml", { "raycluster-autoscaler-head-8zsc8": NodeData( - kind="head", type="head-group", ip="10.4.2.6", status="up-to-date" + kind="head", + type="head-group", + replica_index=None, + ip="10.4.2.6", + status="up-to-date", ), # up-to-date status because the Ray container is in running status "raycluster-autoscaler-worker-small-group-dkz2r": NodeData( - kind="worker", type="small-group", ip="10.4.1.8", status="waiting" + kind="worker", + type="small-group", + replica_index=None, + ip="10.4.1.8", + status="waiting", ), # waiting status, because Ray container's state is "waiting". # The pod list includes a worker with non-null deletion timestamp. # It is excluded from the node data because it is considered @@ -134,23 +147,30 @@ def test_create_node_cap_at_max( "podlist2.yaml", { "raycluster-autoscaler-head-8zsc8": NodeData( - kind="head", type="head-group", ip="10.4.2.6", status="up-to-date" + kind="head", + type="head-group", + replica_index=None, + ip="10.4.2.6", + status="up-to-date", ), "raycluster-autoscaler-worker-fake-gpu-group-2qnhv": NodeData( kind="worker", type="fake-gpu-group", + replica_index=None, ip="10.4.0.6", status="up-to-date", ), "raycluster-autoscaler-worker-small-group-dkz2r": NodeData( kind="worker", type="small-group", + replica_index=None, ip="10.4.1.8", status="up-to-date", ), "raycluster-autoscaler-worker-small-group-lbfm4": NodeData( kind="worker", type="small-group", + replica_index=None, ip="10.4.0.5", status="up-to-date", ), @@ -187,23 +207,30 @@ def mock_get(node_provider, path): ( { "raycluster-autoscaler-head-8zsc8": NodeData( - kind="head", type="head-group", ip="10.4.2.6", status="up-to-date" + kind="head", + type="head-group", + replica_index=None, + ip="10.4.2.6", + status="up-to-date", ), "raycluster-autoscaler-worker-fake-gpu-group-2qnhv": NodeData( kind="worker", type="fake-gpu-group", + replica_index=None, ip="10.4.0.6", status="up-to-date", ), "raycluster-autoscaler-worker-small-group-dkz2r": NodeData( kind="worker", type="small-group", + replica_index=None, ip="10.4.1.8", status="up-to-date", ), "raycluster-autoscaler-worker-small-group-lbfm4": NodeData( kind="worker", type="small-group", + replica_index=None, ip="10.4.0.5", status="up-to-date", ), @@ -221,7 +248,7 @@ def mock_get(node_provider, path): [ { "op": "replace", - "path": "/spec/workerGroupSpecs/2/replicas", + "path": "/spec/workerGroupSpecs/3/replicas", "value": 5, }, { @@ -258,18 +285,20 @@ def test_submit_scale_request(node_data_dict, scale_request, expected_patch_payl @pytest.mark.parametrize("node_set", [{"A", "B", "C", "D", "E"}]) @pytest.mark.parametrize("cpu_workers_to_delete", ["A", "Z"]) @pytest.mark.parametrize("gpu_workers_to_delete", ["B", "Y"]) +@pytest.mark.parametrize("tpu_workers_to_delete", ["C", "X"]) @pytest.mark.skipif(sys.platform.startswith("win"), reason="Not relevant on Windows.") def test_safe_to_scale( node_set: Set[NodeID], cpu_workers_to_delete: List[NodeID], gpu_workers_to_delete: List[NodeID], + tpu_workers_to_delete: List[NodeID], ): # NodeData values unimportant for this test. - mock_node_data = NodeData("-", "-", "-", "-") + mock_node_data = NodeData("-", "-", "-", "-", "-") node_data_dict = {node_id: mock_node_data for node_id in node_set} raycluster = _get_basic_ray_cr_workers_to_delete( - cpu_workers_to_delete, gpu_workers_to_delete + cpu_workers_to_delete, gpu_workers_to_delete, tpu_workers_to_delete ) def mock_patch(kuberay_provider, path, patch_payload): @@ -286,12 +315,19 @@ def mock_patch(kuberay_provider, path, patch_payload): kr_node_provider.node_data_dict = node_data_dict actual_safe = kr_node_provider.safe_to_scale() - expected_safe = not any( - cpu_worker_to_delete in node_set - for cpu_worker_to_delete in cpu_workers_to_delete - ) and not any( - gpu_worker_to_delete in node_set - for gpu_worker_to_delete in gpu_workers_to_delete + expected_safe = ( + not any( + cpu_worker_to_delete in node_set + for cpu_worker_to_delete in cpu_workers_to_delete + ) + and not any( + gpu_worker_to_delete in node_set + for gpu_worker_to_delete in gpu_workers_to_delete + ) + and not any( + tpu_worker_to_delete in node_set + for tpu_worker_to_delete in tpu_workers_to_delete + ) ) assert expected_safe is actual_safe patched_cpu_workers_to_delete = kr_node_provider._patched_raycluster["spec"][ @@ -300,15 +336,20 @@ def mock_patch(kuberay_provider, path, patch_payload): patched_gpu_workers_to_delete = kr_node_provider._patched_raycluster["spec"][ "workerGroupSpecs" ][1]["scaleStrategy"]["workersToDelete"] + patched_tpu_workers_to_delete = kr_node_provider._patched_raycluster["spec"][ + "workerGroupSpecs" + ][2]["scaleStrategy"]["workersToDelete"] if expected_safe: # Cleaned up workers to delete assert patched_cpu_workers_to_delete == [] assert patched_gpu_workers_to_delete == [] + assert patched_tpu_workers_to_delete == [] else: # Did not clean up workers to delete assert patched_cpu_workers_to_delete == cpu_workers_to_delete assert patched_gpu_workers_to_delete == gpu_workers_to_delete + assert patched_tpu_workers_to_delete == tpu_workers_to_delete if __name__ == "__main__": From f1704b79c79011bcc93ab37bace3c4ccfc15bcef Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Wed, 8 May 2024 10:37:12 +0000 Subject: [PATCH 04/45] Fix KubeRay spelling Signed-off-by: ryanaoleary --- python/ray/autoscaler/_private/kuberay/node_provider.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/python/ray/autoscaler/_private/kuberay/node_provider.py b/python/ray/autoscaler/_private/kuberay/node_provider.py index 33b6eb1e45822..2f833cf70d148 100644 --- a/python/ray/autoscaler/_private/kuberay/node_provider.py +++ b/python/ray/autoscaler/_private/kuberay/node_provider.py @@ -226,7 +226,8 @@ def _worker_group_replicas(raycluster: Dict[str, Any], group_index: int): return raycluster["spec"]["workerGroupSpecs"][group_index].get("replicas", 1) def _worker_group_num_of_hosts(raycluster: Dict[str, Any], group_index: int): - # Extract NumOfHosts of a worker group. 1 is the default NumOfHosts value used by the Kuberay operator. + # Extract NumOfHosts of a worker group. 1 is the default NumOfHosts value used by + # the KubeRay operator. return raycluster["spec"]["workerGroupSpecs"][group_index].get("numOfHosts", 1) class KubeRayNodeProvider(BatchingNodeProvider): # type: ignore From 544a80b6f23a39198d81bae97f463100f78916b2 Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Fri, 10 May 2024 00:37:44 +0000 Subject: [PATCH 05/45] Make replica_index optional and added KeyError check Signed-off-by: ryanaoleary --- python/ray/autoscaler/_private/autoscaler.py | 8 +++++--- .../ray/autoscaler/batching_node_provider.py | 20 ++++++++++--------- 2 files changed, 16 insertions(+), 12 deletions(-) diff --git a/python/ray/autoscaler/_private/autoscaler.py b/python/ray/autoscaler/_private/autoscaler.py index 14c831c918b46..4db437d5b77f3 100644 --- a/python/ray/autoscaler/_private/autoscaler.py +++ b/python/ray/autoscaler/_private/autoscaler.py @@ -130,13 +130,15 @@ def __init__(self, provider: NodeProvider): self.multi_host_replicas_to_workers = defaultdict(list) for node in self.all_node_ids: - node_kind = provider.node_tags(node)[TAG_RAY_NODE_KIND] + tags = self.provider.node_tags(node) + node_kind = tags[TAG_RAY_NODE_KIND] if node_kind == NODE_KIND_WORKER: self.worker_ids.append(node) elif node_kind == NODE_KIND_HEAD: self.head_id = node - node_multihost_replica = provider.node_tags(node)[TAG_RAY_MULTIHOST_REPLICA] - self.multi_host_replicas_to_workers.append(node) + if TAG_RAY_REPLICA_INDEX in tags: + node_replica_index = tags[TAG_RAY_REPLICA_INDEX] + self.replicas_to_nodes[node_replica_index].append(node) # Note: For typical use-cases, self.all_node_ids == self.worker_ids + # [self.head_id]. The difference being in the case of unmanaged nodes. diff --git a/python/ray/autoscaler/batching_node_provider.py b/python/ray/autoscaler/batching_node_provider.py index 1101b3de8e6a7..bbbb770b83168 100644 --- a/python/ray/autoscaler/batching_node_provider.py +++ b/python/ray/autoscaler/batching_node_provider.py @@ -60,7 +60,7 @@ class NodeData: kind: NodeKind type: NodeType - multihost_replica: string + replica_index: Optional[str] ip: Optional[NodeIP] status: NodeStatus @@ -168,8 +168,10 @@ def non_terminated_nodes(self, tag_filters: Dict[str, str]) -> List[str]: # Initialize multi-host replica to workers map self.multi_host_replicas_to_workers = defaultdict(list) for node_id in all_nodes: - multi_host_replica = self.node_data_dict[node_id].multihost_replica - self.multi_host_replicas_to_workers[multi_host_replica].append(node_id) + replica_index = self.node_data_dict[node_id].replica_index + # Only add node to map if it belongs to a TPU podslice + if replica_index: + self.replicas_to_nodes[replica_index].append(node_id) # Support filtering by TAG_RAY_NODE_KIND, TAG_RAY_NODE_STATUS, and # TAG_RAY_USER_NODE_TYPE. # The autoscaler only uses tag_filters={}, @@ -240,12 +242,12 @@ def terminate_node(self, node_id: str) -> Optional[Dict[str, Any]]: "NodeProvider attempted to request less than 0 workers of type " f"{node_type}. Skipping termination request." ) - - # Scale down entire replica if part of a multi-host group - node_multihost_replica = self.node_data_dict[node_id].multihost_replica - if node_multihost_replica != "": - for worker in self.multi_host_replicas_to_workers[node_multihost_replica]: - # Check if worker has already been scheduled to delete + + # Scale down entire replica if part of a TPU podslice being deleted + node_replica_index = self.node_data_dict[node_id].replica_index + if node_replica_index: + for worker in self.replicas_to_nodes[node_replica_index]: + # Check if node has already been scheduled to delete if node_id not in self.scale_request.workers_to_delete: # Assume all workers in a group are of the same type self.scale_request.desired_num_workers[node_type] -= 1 From 2d3d99029fb2c373255937f6bb7977501fe350fc Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Tue, 21 May 2024 21:29:46 +0000 Subject: [PATCH 06/45] Remove self from tags initialization Signed-off-by: ryanaoleary --- python/ray/autoscaler/_private/autoscaler.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/autoscaler/_private/autoscaler.py b/python/ray/autoscaler/_private/autoscaler.py index 4db437d5b77f3..42b10b770d107 100644 --- a/python/ray/autoscaler/_private/autoscaler.py +++ b/python/ray/autoscaler/_private/autoscaler.py @@ -130,7 +130,7 @@ def __init__(self, provider: NodeProvider): self.multi_host_replicas_to_workers = defaultdict(list) for node in self.all_node_ids: - tags = self.provider.node_tags(node) + tags = provider.node_tags(node) node_kind = tags[TAG_RAY_NODE_KIND] if node_kind == NODE_KIND_WORKER: self.worker_ids.append(node) From f2c227b3e6a8091bb352bb271cbf243082b73a26 Mon Sep 17 00:00:00 2001 From: ryanaoleary <113500783+ryanaoleary@users.noreply.github.com> Date: Fri, 31 May 2024 14:22:07 -0700 Subject: [PATCH 07/45] Update python/ray/autoscaler/batching_node_provider.py Co-authored-by: Kai-Hsun Chen Signed-off-by: ryanaoleary <113500783+ryanaoleary@users.noreply.github.com> --- python/ray/autoscaler/batching_node_provider.py | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/python/ray/autoscaler/batching_node_provider.py b/python/ray/autoscaler/batching_node_provider.py index bbbb770b83168..40535c37602e8 100644 --- a/python/ray/autoscaler/batching_node_provider.py +++ b/python/ray/autoscaler/batching_node_provider.py @@ -44,9 +44,8 @@ class NodeData: Attributes: kind: Whether the node is the head or a worker. type: The user-defined type of the node. - multihost_replica: An identifier for nodes belonging to a replica of a multi-host worker group. - * This value is set as a label by a webhook when NumOfHosts > 1 in a worker group - * This var is empty for pods belonging to single-host worker groups + replica_index: An identifier for nodes in a replica of a TPU worker group. + This value is set as a Pod label by a GKE webhook when TPUs are requested ip: Cluster-internal ip of the node. ip can be None if the ip has not yet been assigned. status: The status of the node. You must adhere to the following semantics From 7d76c403a69cdeb891a9d438eca0d667f481c387 Mon Sep 17 00:00:00 2001 From: ryanaoleary <113500783+ryanaoleary@users.noreply.github.com> Date: Fri, 31 May 2024 14:22:50 -0700 Subject: [PATCH 08/45] Update python/ray/autoscaler/_private/kuberay/node_provider.py Co-authored-by: Kai-Hsun Chen Signed-off-by: ryanaoleary <113500783+ryanaoleary@users.noreply.github.com> --- python/ray/autoscaler/_private/kuberay/node_provider.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/autoscaler/_private/kuberay/node_provider.py b/python/ray/autoscaler/_private/kuberay/node_provider.py index 2f833cf70d148..0a47a43d9b094 100644 --- a/python/ray/autoscaler/_private/kuberay/node_provider.py +++ b/python/ray/autoscaler/_private/kuberay/node_provider.py @@ -109,7 +109,7 @@ def replica_index_label(pod: Dict[str, Any]) -> Optional[str]: """ labels = pod["metadata"]["labels"] if REPLICA_INDEX_KEY in labels: - labels[REPLICA_INDEX_KEY] + return labels[REPLICA_INDEX_KEY] return None From 62f25b2226b3dc8efc12fcf3d562e103a4b22545 Mon Sep 17 00:00:00 2001 From: ryanaoleary <113500783+ryanaoleary@users.noreply.github.com> Date: Fri, 31 May 2024 14:57:16 -0700 Subject: [PATCH 09/45] Update python/ray/autoscaler/_private/kuberay/node_provider.py Co-authored-by: Kai-Hsun Chen Signed-off-by: ryanaoleary <113500783+ryanaoleary@users.noreply.github.com> --- python/ray/autoscaler/_private/kuberay/node_provider.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/autoscaler/_private/kuberay/node_provider.py b/python/ray/autoscaler/_private/kuberay/node_provider.py index 0a47a43d9b094..4d314080d1b38 100644 --- a/python/ray/autoscaler/_private/kuberay/node_provider.py +++ b/python/ray/autoscaler/_private/kuberay/node_provider.py @@ -102,7 +102,7 @@ def multihost_replica_id(pod: Dict[str, Any]) -> string: def replica_index_label(pod: Dict[str, Any]) -> Optional[str]: - """Returns the replicaIndex label for a Pod in a TPU worker group. + """Returns the replicaIndex label for a Pod in a multi-host TPU worker group. The replicaIndex label is set by the GKE TPU Ray webhook and is of the form {$WORKER_GROUP_NAME-$REPLICA_INDEX} where $REPLICA_INDEX is an integer from 0 to Replicas-1. From de6f7f24aaa3291aa5d14eb209aec0d44ccd7bbd Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Fri, 31 May 2024 21:54:50 +0000 Subject: [PATCH 10/45] Move replicas_to_delete and change to set Signed-off-by: Ryan O'Leary --- python/ray/autoscaler/_private/autoscaler.py | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/python/ray/autoscaler/_private/autoscaler.py b/python/ray/autoscaler/_private/autoscaler.py index 42b10b770d107..dff8d3cf7c650 100644 --- a/python/ray/autoscaler/_private/autoscaler.py +++ b/python/ray/autoscaler/_private/autoscaler.py @@ -126,8 +126,9 @@ def __init__(self, provider: NodeProvider): self.worker_ids: List[NodeID] = [] # The head node (node kind "head") self.head_id: Optional[NodeID] = None - # Map of multi-host replica IDs to nodes in each replica - self.multi_host_replicas_to_workers = defaultdict(list) + # Map of replica IDs to worker nodes in each replica. + # A replica ID refers to the index of a multi-host PodSlice created by KubeRay. + self.replicas_to_nodes = defaultdict(list) for node in self.all_node_ids: tags = provider.node_tags(node) @@ -276,6 +277,9 @@ def read_fn(): # Tracks nodes scheduled for termination self.nodes_to_terminate: List[NodeID] = [] + # Tracks replicas scheduled for termination + self.replicas_to_delete = set() + # Disable NodeUpdater threads if true. # Should be set to true in situations where another component, such as # a Kubernetes operator, is responsible for Ray setup on nodes. @@ -415,9 +419,6 @@ def _update(self): # This will accumulate the nodes we need to terminate. self.nodes_to_terminate = [] - # This will accumulate the multi-host replicas we need to terminate. - self.multi_host_replicas_to_delete = [] - # Update running nodes gauge num_workers = len(self.non_terminated_nodes.worker_ids) self.prom_metrics.running_workers.set(num_workers) From e6a73725fe218e799a900a45d3cb5d7a1016459b Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Fri, 31 May 2024 22:11:52 +0000 Subject: [PATCH 11/45] Add assert for self.replicas_to_nodes Signed-off-by: Ryan O'Leary --- python/ray/autoscaler/_private/autoscaler.py | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/python/ray/autoscaler/_private/autoscaler.py b/python/ray/autoscaler/_private/autoscaler.py index dff8d3cf7c650..c30b925d14e80 100644 --- a/python/ray/autoscaler/_private/autoscaler.py +++ b/python/ray/autoscaler/_private/autoscaler.py @@ -647,11 +647,12 @@ def terminate_scheduled_nodes(self): self.prom_metrics.stopped_nodes.inc() # Clean up multi-host replicas to delete tags = self.provider.node_tags(node) - if TAG_RAY_MULTIHOST_REPLICA in tags: - multi_host_replica_id = tags[TAG_RAY_MULTIHOST_REPLICA] - self.multi_host_replicas_to_workers[multi_host_replica_id].remove(node) - if len(self.multi_host_replicas_to_workers[multi_host_replica_id]) == 0: - self.multi_host_replicas_to_delete.remove(multi_host_replica_id) + if TAG_RAY_REPLICA_INDEX in tags: + assert self.replicas_to_nodes + replica_id = tags[TAG_RAY_REPLICA_INDEX] + self.replicas_to_nodes[replica_id].remove(node) + if len(self.replicas_to_nodes[replica_id]) == 0: + self.replicas_to_delete.remove(replica_id) # Update internal node lists self.non_terminated_nodes.remove_terminating_nodes(self.nodes_to_terminate) From 1f1bccc7923ae6db1f5ed895bd1af72ae0873a83 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Fri, 31 May 2024 22:14:25 +0000 Subject: [PATCH 12/45] Move replicas_to_node initialization to constructor Signed-off-by: Ryan O'Leary --- python/ray/autoscaler/batching_node_provider.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/python/ray/autoscaler/batching_node_provider.py b/python/ray/autoscaler/batching_node_provider.py index 40535c37602e8..2a7f8fd5ffb88 100644 --- a/python/ray/autoscaler/batching_node_provider.py +++ b/python/ray/autoscaler/batching_node_provider.py @@ -120,6 +120,9 @@ def __init__( self.scale_request = ScaleRequest() + # Initialize map of replica indices to nodes in that replica + self.replicas_to_nodes = defaultdict(list) + def get_node_data(self) -> Dict[NodeID, NodeData]: """Queries cluster manager for node info. Returns a mapping from node id to NodeData. @@ -164,8 +167,6 @@ def non_terminated_nodes(self, tag_filters: Dict[str, str]) -> List[str]: workers_to_delete=set(), # No workers to delete yet ) all_nodes = list(self.node_data_dict.keys()) - # Initialize multi-host replica to workers map - self.multi_host_replicas_to_workers = defaultdict(list) for node_id in all_nodes: replica_index = self.node_data_dict[node_id].replica_index # Only add node to map if it belongs to a TPU podslice From 2d969c82e95f5b31f214ad5f7c95aefe102e411f Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Fri, 7 Jun 2024 02:27:57 +0000 Subject: [PATCH 13/45] Fixed comments Signed-off-by: Ryan O'Leary --- python/ray/autoscaler/_private/autoscaler.py | 78 +++++++++----------- 1 file changed, 34 insertions(+), 44 deletions(-) diff --git a/python/ray/autoscaler/_private/autoscaler.py b/python/ray/autoscaler/_private/autoscaler.py index c30b925d14e80..0606fd5761fc9 100644 --- a/python/ray/autoscaler/_private/autoscaler.py +++ b/python/ray/autoscaler/_private/autoscaler.py @@ -126,9 +126,6 @@ def __init__(self, provider: NodeProvider): self.worker_ids: List[NodeID] = [] # The head node (node kind "head") self.head_id: Optional[NodeID] = None - # Map of replica IDs to worker nodes in each replica. - # A replica ID refers to the index of a multi-host PodSlice created by KubeRay. - self.replicas_to_nodes = defaultdict(list) for node in self.all_node_ids: tags = provider.node_tags(node) @@ -137,9 +134,6 @@ def __init__(self, provider: NodeProvider): self.worker_ids.append(node) elif node_kind == NODE_KIND_HEAD: self.head_id = node - if TAG_RAY_REPLICA_INDEX in tags: - node_replica_index = tags[TAG_RAY_REPLICA_INDEX] - self.replicas_to_nodes[node_replica_index].append(node) # Note: For typical use-cases, self.all_node_ids == self.worker_ids + # [self.head_id]. The difference being in the case of unmanaged nodes. @@ -278,7 +272,11 @@ def read_fn(): self.nodes_to_terminate: List[NodeID] = [] # Tracks replicas scheduled for termination - self.replicas_to_delete = set() + self.replicas_to_delete: Set[str] = set() + + # Map of replica IDs to worker nodes in each replica. + # A replica ID refers to the index of a multi-host PodSlice created by KubeRay. + self.replicas_to_nodes: Dict[str, List[NodeID]] = defaultdict(List[NodeID]) # Disable NodeUpdater threads if true. # Should be set to true in situations where another component, such as @@ -416,6 +414,14 @@ def _update(self): ) return + # Populate mapping of replica IDs to nodes in that replica. + for node in self.non_terminated_nodes: + tags = self.provider.node_tags(node) + if TAG_RAY_REPLICA_INDEX in tags: + node_replica_index = tags[TAG_RAY_REPLICA_INDEX] + if node not in self.replicas_to_nodes[node_replica_index]: + self.replicas_to_nodes[node_replica_index].append(node) + # This will accumulate the nodes we need to terminate. self.nodes_to_terminate = [] @@ -622,10 +628,10 @@ def schedule_node_termination( self.nodes_to_terminate.append(node_id) # Scale down entire multi-host replica of node being deleted tags = self.provider.node_tags(node_id) - if TAG_RAY_MULTIHOST_REPLICA in tags: - multi_host_replica_id = tags[TAG_RAY_MULTIHOST_REPLICA] - if multi_host_replica_id not in self.multi_host_replicas_to_delete: - self.multi_host_replicas_to_delete.append(multi_host_replica_id) + if TAG_RAY_REPLICA_INDEX in tags: + replica_id = tags[TAG_RAY_REPLICA_INDEX] + if replica_id not in self.replicas_to_delete: + self.replicas_to_delete.add(replica_id) def terminate_scheduled_nodes(self): """Terminate scheduled nodes and clean associated autoscaler state.""" @@ -983,7 +989,9 @@ def _keep_worker_of_node_type( self, node_id: NodeID, node_type_counts: Dict[NodeType, int] ) -> Tuple[KeepOrTerminate, Optional[str]]: """Determines if a worker should be kept based on the min_workers - and max_workers constraint of the worker's node_type. + and max_workers constraint of the worker's node_type. Additionally, + workers belonging to a multi-host replica being deleted are scheduled + to delete as well. Returns KeepOrTerminate.keep when both of the following hold: (a) The worker's node_type is present among the keys of the current @@ -991,11 +999,13 @@ def _keep_worker_of_node_type( (b) Deleting the node would violate the min_workers constraint for that worker's node_type. - Returns KeepOrTerminate.terminate when both the following hold: + Returns KeepOrTerminate.terminate when the following hold: (a) The worker's node_type is not present among the keys of the current config's available_node_types dict. (b) Keeping the node would violate the max_workers constraint for that worker's node_type. + (c) The worker has TAG_RAY_REPLICA_INDEX tag set, and its replica index is + found in replicas_to_delete. Return KeepOrTerminate.decide_later otherwise. @@ -1013,6 +1023,17 @@ def _keep_worker_of_node_type( assert self.provider tags = self.provider.node_tags(node_id) + + if TAG_RAY_REPLICA_INDEX in tags: + replica_id = tags[TAG_RAY_REPLICA_INDEX] + # All nodes in this replica should be deleted, regardless of + # available_node_types. + if replica_id in self.replicas_to_delete: + return ( + KeepOrTerminate.terminate, + f"Node belongs to a replica being deleted: {replica_id}", + ) + if TAG_RAY_USER_NODE_TYPE in tags: node_type = tags[TAG_RAY_USER_NODE_TYPE] @@ -1038,37 +1059,6 @@ def _keep_worker_of_node_type( return KeepOrTerminate.decide_later, None - def _keep_worker_of_multihost_replica(self, node_id: NodeID) -> Tuple[KeepOrTerminate, Optional[str]]: - """Determines if a worker should be kept based on whether any - other nodes in the same multi-host replica have been deleted. - - Returns KeepOrTerminate.terminate when: - (a) The worker belongs to a multi-host replica scheduled to delete. - - Return KeepOrTerminate.keep otherwise. - - Returns: - KeepOrTerminate: keep if the node should be kept, terminate if the - node should be terminated - Optional[str]: reason for termination. Not None on - KeepOrTerminate.terminate, None otherwise. - """ - # For type checking, assert that this object has been instantitiated. - assert self.provider - - tags = self.provider.node_tags(node_id) - if TAG_RAY_MULTIHOST_REPLICA in tags: - multi_host_replica_id = tags[TAG_RAY_MULTIHOST_REPLICA] - - if multi_host_replica_id in self.multi_host_replicas_to_delete: - # All pods in this multi-host replica should be deleted - return ( - KeepOrTerminate.terminate, - f"Pod belongs to a multi-host replica being deleted: {multi_host_replica_id}", - ) - - return KeepOrTerminate.keep, None - def _node_resources(self, node_id): node_type = self.provider.node_tags(node_id).get(TAG_RAY_USER_NODE_TYPE) if self.available_node_types: From 1286434368214d05fe28dd7bf1bc8b1e3c17cbf0 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Fri, 7 Jun 2024 02:27:57 +0000 Subject: [PATCH 14/45] Fixed comments Signed-off-by: Ryan O'Leary --- .../_private/kuberay/node_provider.py | 20 ++++++++----------- 1 file changed, 8 insertions(+), 12 deletions(-) diff --git a/python/ray/autoscaler/_private/kuberay/node_provider.py b/python/ray/autoscaler/_private/kuberay/node_provider.py index 615fcdf4c65be..359307189f1bd 100644 --- a/python/ray/autoscaler/_private/kuberay/node_provider.py +++ b/python/ray/autoscaler/_private/kuberay/node_provider.py @@ -45,8 +45,8 @@ RAY_HEAD_POD_NAME = os.getenv("RAY_HEAD_POD_NAME") -# Key for label that identifies which multi-host relica a pod belongs to -MULTIHOST_REPLICA_KEY = "multihost-replica" +# Key for GKE label that identifies which multi-host replica a pod belongs to +REPLICA_INDEX_KEY = "replicaIndex" # Design: @@ -82,8 +82,10 @@ def node_data_from_pod(pod: Dict[str, Any]) -> NodeData: kind, type = kind_and_type(pod) status = status_tag(pod) ip = pod_ip(pod) - multihost_replica = multihost_replica_id(pod) - return NodeData(kind=kind, type=type, multihost_replica=multihost_replica, status=status, ip=ip) + replica_index = replica_index_label(pod) + return NodeData( + kind=kind, type=type, replica_index=replica_index, status=status, ip=ip + ) def kind_and_type(pod: Dict[str, Any]) -> Tuple[NodeKind, NodeType]: @@ -100,14 +102,6 @@ def kind_and_type(pod: Dict[str, Any]) -> Tuple[NodeKind, NodeType]: return kind, type -def multihost_replica_id(pod: Dict[str, Any]) -> string: - """ Returns the replica identifier for a multi-host worker group. - If the pod belongs to a single-host group, returns an empty string. - """ - labels = pod["metadata"]["labels"] - return labels[MULTIHOST_REPLICA_KEY] - - def replica_index_label(pod: Dict[str, Any]) -> Optional[str]: """Returns the replicaIndex label for a Pod in a multi-host TPU worker group. The replicaIndex label is set by the GKE TPU Ray webhook and is of @@ -234,11 +228,13 @@ def _worker_group_replicas(raycluster: Dict[str, Any], group_index: int): # 1 is the default replicas value used by the KubeRay operator return raycluster["spec"]["workerGroupSpecs"][group_index].get("replicas", 1) + def _worker_group_num_of_hosts(raycluster: Dict[str, Any], group_index: int): # Extract NumOfHosts of a worker group. 1 is the default NumOfHosts value used by # the KubeRay operator. return raycluster["spec"]["workerGroupSpecs"][group_index].get("numOfHosts", 1) + class IKubernetesHttpApiClient(ABC): """ An interface for a Kubernetes HTTP API client. From 60930a4d94f39308d26438913d7584bb7dd735f0 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Fri, 7 Jun 2024 04:18:58 +0000 Subject: [PATCH 15/45] Add back in changes lost in bad rebase Signed-off-by: Ryan O'Leary --- python/ray/autoscaler/_private/autoscaler.py | 11 +++-------- python/ray/autoscaler/batching_node_provider.py | 4 ++-- python/ray/autoscaler/tags.py | 4 ++-- 3 files changed, 7 insertions(+), 12 deletions(-) diff --git a/python/ray/autoscaler/_private/autoscaler.py b/python/ray/autoscaler/_private/autoscaler.py index 204871723d2e6..d22b3bb787627 100644 --- a/python/ray/autoscaler/_private/autoscaler.py +++ b/python/ray/autoscaler/_private/autoscaler.py @@ -74,6 +74,7 @@ TAG_RAY_LAUNCH_CONFIG, TAG_RAY_NODE_KIND, TAG_RAY_NODE_STATUS, + TAG_RAY_REPLICA_INDEX, TAG_RAY_RUNTIME_CONFIG, TAG_RAY_USER_NODE_TYPE, ) @@ -129,8 +130,7 @@ def __init__(self, provider: NodeProvider): self.head_id: Optional[NodeID] = None for node in self.all_node_ids: - tags = provider.node_tags(node) - node_kind = tags[TAG_RAY_NODE_KIND] + node_kind = provider.node_tags(node)[TAG_RAY_NODE_KIND] if node_kind == NODE_KIND_WORKER: self.worker_ids.append(node) elif node_kind == NODE_KIND_HEAD: @@ -498,7 +498,7 @@ def terminate_nodes_to_enforce_config_constraints(self, now: float): (3) Terminates outdated nodes, namely nodes whose configs don't match `node_config` for the relevant node type. - (4) Terminate nodes part of a multi-host replica being deleted. + (4) Terminates nodes part of a multi-host replica being deleted. Avoids terminating non-outdated nodes required by autoscaler.sdk.request_resources(). @@ -545,11 +545,6 @@ def keep_node(node_id: NodeID) -> None: should_keep_or_terminate, reason = self._keep_worker_of_node_type( node_id, node_type_counts ) - if should_keep_or_terminate == KeepOrTerminate.terminate: - self.schedule_node_termination(node_id, reason, logger.info) - continue - # Check if node is part of a multi-host replica that is being deleted - should_keep_or_terminate, reason = self._keep_worker_of_multihost_replica(node_id) if should_keep_or_terminate == KeepOrTerminate.terminate: self.schedule_node_termination(node_id, reason, logger.info) continue diff --git a/python/ray/autoscaler/batching_node_provider.py b/python/ray/autoscaler/batching_node_provider.py index 2a7f8fd5ffb88..83a7fc2bad782 100644 --- a/python/ray/autoscaler/batching_node_provider.py +++ b/python/ray/autoscaler/batching_node_provider.py @@ -14,8 +14,8 @@ NODE_KIND_HEAD, TAG_RAY_NODE_KIND, TAG_RAY_NODE_STATUS, + TAG_RAY_REPLICA_INDEX, TAG_RAY_USER_NODE_TYPE, - TAG_RAY_MULTIHOST_REPLICA, ) provider_exists = False @@ -203,7 +203,7 @@ def node_tags(self, node_id: str) -> Dict[str, str]: TAG_RAY_NODE_KIND: node_data.kind, TAG_RAY_NODE_STATUS: node_data.status, TAG_RAY_USER_NODE_TYPE: node_data.type, - TAG_RAY_MULTIHOST_REPLICA: node_data.multihost_replica, + TAG_RAY_REPLICA_INDEX: node_data.replica_index, } def internal_ip(self, node_id: str) -> str: diff --git a/python/ray/autoscaler/tags.py b/python/ray/autoscaler/tags.py index f2d7514415f40..38d03855040fd 100644 --- a/python/ray/autoscaler/tags.py +++ b/python/ray/autoscaler/tags.py @@ -13,8 +13,8 @@ # Tag for user defined node types (e.g., m4xl_spot). This is used for multi # node type clusters. TAG_RAY_USER_NODE_TYPE = "ray-user-node-type" -# Tag for multi-host replica node belongs to. Used for multi-host worker groups. -TAG_RAY_MULTIHOST_REPLICA = "ray-multihost-replica" +# Tag for index of replica node belongs to. Used for multi-host worker groups. +TAG_RAY_REPLICA_INDEX = "ray-replica-index" # Tag for autofilled node types for legacy cluster yamls without multi # node type defined in the cluster configs. NODE_TYPE_LEGACY_HEAD = "ray-legacy-head-node-type" From 95c3f16fb574f27a73fb46c85ea797d19b872836 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Fri, 7 Jun 2024 07:35:12 +0000 Subject: [PATCH 16/45] Change to iterate through worker_ids Signed-off-by: Ryan O'Leary --- python/ray/autoscaler/_private/autoscaler.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/autoscaler/_private/autoscaler.py b/python/ray/autoscaler/_private/autoscaler.py index d22b3bb787627..a274a01d78b5a 100644 --- a/python/ray/autoscaler/_private/autoscaler.py +++ b/python/ray/autoscaler/_private/autoscaler.py @@ -416,7 +416,7 @@ def _update(self): return # Populate mapping of replica IDs to nodes in that replica. - for node in self.non_terminated_nodes: + for node in self.non_terminated_nodes.worker_ids: tags = self.provider.node_tags(node) if TAG_RAY_REPLICA_INDEX in tags: node_replica_index = tags[TAG_RAY_REPLICA_INDEX] From 73e9517b0c077a2d2ab688b48d7616dab931f4ee Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Fri, 7 Jun 2024 09:51:06 +0000 Subject: [PATCH 17/45] Fix replicas_to_nodes instantiation Signed-off-by: Ryan O'Leary Fix replicas_to_nodes mapping Signed-off-by: Ryan O'Leary --- python/ray/autoscaler/_private/autoscaler.py | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/python/ray/autoscaler/_private/autoscaler.py b/python/ray/autoscaler/_private/autoscaler.py index a274a01d78b5a..ec2d4f76d13f1 100644 --- a/python/ray/autoscaler/_private/autoscaler.py +++ b/python/ray/autoscaler/_private/autoscaler.py @@ -277,7 +277,7 @@ def read_fn(): # Map of replica IDs to worker nodes in each replica. # A replica ID refers to the index of a multi-host PodSlice created by KubeRay. - self.replicas_to_nodes: Dict[str, List[NodeID]] = defaultdict(List[NodeID]) + self.replicas_to_nodes: Dict[str, List[NodeID]] = defaultdict(list) # Disable NodeUpdater threads if true. # Should be set to true in situations where another component, such as @@ -416,12 +416,15 @@ def _update(self): return # Populate mapping of replica IDs to nodes in that replica. - for node in self.non_terminated_nodes.worker_ids: - tags = self.provider.node_tags(node) + for node_id in self.non_terminated_nodes.worker_ids: + tags = self.provider.node_tags(node_id) if TAG_RAY_REPLICA_INDEX in tags: node_replica_index = tags[TAG_RAY_REPLICA_INDEX] - if node not in self.replicas_to_nodes[node_replica_index]: - self.replicas_to_nodes[node_replica_index].append(node) + if node_replica_index in self.replicas_to_nodes: + if node_id not in self.replicas_to_nodes[node_replica_index]: + self.replicas_to_nodes[node_replica_index].append(node_id) + else: + self.replicas_to_nodes[node_replica_index] = list(node_id) # This will accumulate the nodes we need to terminate. self.nodes_to_terminate = [] From 143e03a0e8fe0fdc9898c331c45e220b9985182d Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Sat, 8 Jun 2024 01:57:56 +0000 Subject: [PATCH 18/45] Fix errors causing autoscaler test to fail Signed-off-by: Ryan O'Leary --- python/ray/autoscaler/_private/autoscaler.py | 16 ++++++++++------ python/ray/autoscaler/batching_node_provider.py | 2 +- 2 files changed, 11 insertions(+), 7 deletions(-) diff --git a/python/ray/autoscaler/_private/autoscaler.py b/python/ray/autoscaler/_private/autoscaler.py index ec2d4f76d13f1..724e8745ca438 100644 --- a/python/ray/autoscaler/_private/autoscaler.py +++ b/python/ray/autoscaler/_private/autoscaler.py @@ -645,19 +645,23 @@ def terminate_scheduled_nodes(self): # explicitly disabled. if self.worker_rpc_drain: self.drain_nodes_via_gcs(self.nodes_to_terminate) - # Terminate the nodes - self.provider.terminate_nodes(self.nodes_to_terminate) + # Clean up multi-host replicas to delete for node in self.nodes_to_terminate: - self.node_tracker.untrack(node) - self.prom_metrics.stopped_nodes.inc() - # Clean up multi-host replicas to delete tags = self.provider.node_tags(node) if TAG_RAY_REPLICA_INDEX in tags: assert self.replicas_to_nodes replica_id = tags[TAG_RAY_REPLICA_INDEX] - self.replicas_to_nodes[replica_id].remove(node) + if replica_id in self.replicas_to_nodes: + if node in self.replicas_to_nodes[replica_id]: + self.replicas_to_nodes[replica_id].remove(node) + # remove replica ID once all nodes in replica removed if len(self.replicas_to_nodes[replica_id]) == 0: self.replicas_to_delete.remove(replica_id) + # Terminate the nodes + self.provider.terminate_nodes(self.nodes_to_terminate) + for node in self.nodes_to_terminate: + self.node_tracker.untrack(node) + self.prom_metrics.stopped_nodes.inc() # Update internal node lists self.non_terminated_nodes.remove_terminating_nodes(self.nodes_to_terminate) diff --git a/python/ray/autoscaler/batching_node_provider.py b/python/ray/autoscaler/batching_node_provider.py index 83a7fc2bad782..68ecd6ca5af63 100644 --- a/python/ray/autoscaler/batching_node_provider.py +++ b/python/ray/autoscaler/batching_node_provider.py @@ -59,9 +59,9 @@ class NodeData: kind: NodeKind type: NodeType - replica_index: Optional[str] ip: Optional[NodeIP] status: NodeStatus + replica_index: Optional[str] = None class BatchingNodeProvider(NodeProvider): From 13949356dd26c2eedf830c2d0d74264098f4aa4b Mon Sep 17 00:00:00 2001 From: ryanaoleary <113500783+ryanaoleary@users.noreply.github.com> Date: Tue, 11 Jun 2024 11:22:36 -0700 Subject: [PATCH 19/45] Update python/ray/autoscaler/_private/kuberay/node_provider.py Co-authored-by: Kai-Hsun Chen Signed-off-by: ryanaoleary <113500783+ryanaoleary@users.noreply.github.com> --- python/ray/autoscaler/_private/kuberay/node_provider.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/autoscaler/_private/kuberay/node_provider.py b/python/ray/autoscaler/_private/kuberay/node_provider.py index 359307189f1bd..73e7ca0cd7db8 100644 --- a/python/ray/autoscaler/_private/kuberay/node_provider.py +++ b/python/ray/autoscaler/_private/kuberay/node_provider.py @@ -102,7 +102,7 @@ def kind_and_type(pod: Dict[str, Any]) -> Tuple[NodeKind, NodeType]: return kind, type -def replica_index_label(pod: Dict[str, Any]) -> Optional[str]: +def _replica_index_label(pod: Dict[str, Any]) -> Optional[str]: """Returns the replicaIndex label for a Pod in a multi-host TPU worker group. The replicaIndex label is set by the GKE TPU Ray webhook and is of the form {$WORKER_GROUP_NAME-$REPLICA_INDEX} where $REPLICA_INDEX From dd2259433a2822217caa41167cdb63a99ce6bcd5 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Tue, 11 Jun 2024 21:22:02 +0000 Subject: [PATCH 20/45] Fixed comments Signed-off-by: Ryan O'Leary --- python/ray/autoscaler/_private/autoscaler.py | 10 +++++----- .../ray/autoscaler/_private/kuberay/node_provider.py | 6 ++---- 2 files changed, 7 insertions(+), 9 deletions(-) diff --git a/python/ray/autoscaler/_private/autoscaler.py b/python/ray/autoscaler/_private/autoscaler.py index 724e8745ca438..6a231ee6b5576 100644 --- a/python/ray/autoscaler/_private/autoscaler.py +++ b/python/ray/autoscaler/_private/autoscaler.py @@ -416,6 +416,7 @@ def _update(self): return # Populate mapping of replica IDs to nodes in that replica. + self.replicas_to_nodes.clear() for node_id in self.non_terminated_nodes.worker_ids: tags = self.provider.node_tags(node_id) if TAG_RAY_REPLICA_INDEX in tags: @@ -629,8 +630,7 @@ def schedule_node_termination( tags = self.provider.node_tags(node_id) if TAG_RAY_REPLICA_INDEX in tags: replica_id = tags[TAG_RAY_REPLICA_INDEX] - if replica_id not in self.replicas_to_delete: - self.replicas_to_delete.add(replica_id) + self.replicas_to_delete.add(replica_id) def terminate_scheduled_nodes(self): """Terminate scheduled nodes and clean associated autoscaler state.""" @@ -654,9 +654,9 @@ def terminate_scheduled_nodes(self): if replica_id in self.replicas_to_nodes: if node in self.replicas_to_nodes[replica_id]: self.replicas_to_nodes[replica_id].remove(node) - # remove replica ID once all nodes in replica removed - if len(self.replicas_to_nodes[replica_id]) == 0: - self.replicas_to_delete.remove(replica_id) + # remove replica ID once all nodes in replica removed + if len(self.replicas_to_nodes[replica_id]) == 0: + self.replicas_to_delete.remove(replica_id) # Terminate the nodes self.provider.terminate_nodes(self.nodes_to_terminate) for node in self.nodes_to_terminate: diff --git a/python/ray/autoscaler/_private/kuberay/node_provider.py b/python/ray/autoscaler/_private/kuberay/node_provider.py index 73e7ca0cd7db8..8b3c5caab54b0 100644 --- a/python/ray/autoscaler/_private/kuberay/node_provider.py +++ b/python/ray/autoscaler/_private/kuberay/node_provider.py @@ -82,7 +82,7 @@ def node_data_from_pod(pod: Dict[str, Any]) -> NodeData: kind, type = kind_and_type(pod) status = status_tag(pod) ip = pod_ip(pod) - replica_index = replica_index_label(pod) + replica_index = _replica_index_label(pod) return NodeData( kind=kind, type=type, replica_index=replica_index, status=status, ip=ip ) @@ -109,9 +109,7 @@ def _replica_index_label(pod: Dict[str, Any]) -> Optional[str]: is an integer from 0 to Replicas-1. """ labels = pod["metadata"]["labels"] - if REPLICA_INDEX_KEY in labels: - return labels[REPLICA_INDEX_KEY] - return None + return labels.get(REPLICA_INDEX_KEY, None) def pod_ip(pod: Dict[str, Any]) -> NodeIP: From 473b7cbd8018b046f9b9028ba147095237f7ee46 Mon Sep 17 00:00:00 2001 From: ryanaoleary <113500783+ryanaoleary@users.noreply.github.com> Date: Wed, 12 Jun 2024 14:52:51 -0700 Subject: [PATCH 21/45] Update python/ray/autoscaler/_private/autoscaler.py Co-authored-by: Kai-Hsun Chen Signed-off-by: ryanaoleary <113500783+ryanaoleary@users.noreply.github.com> --- python/ray/autoscaler/_private/autoscaler.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/autoscaler/_private/autoscaler.py b/python/ray/autoscaler/_private/autoscaler.py index 6a231ee6b5576..3307accf65703 100644 --- a/python/ray/autoscaler/_private/autoscaler.py +++ b/python/ray/autoscaler/_private/autoscaler.py @@ -272,7 +272,7 @@ def read_fn(): # Tracks nodes scheduled for termination self.nodes_to_terminate: List[NodeID] = [] - # Tracks replicas scheduled for termination + # A set of replica IDs used to track replicas scheduled for termination. self.replicas_to_delete: Set[str] = set() # Map of replica IDs to worker nodes in each replica. From fa083aaffd8d110005746cce268144c92721f272 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Wed, 12 Jun 2024 21:55:22 +0000 Subject: [PATCH 22/45] Fix replicas_to_nodes creation Signed-off-by: Ryan O'Leary --- python/ray/autoscaler/_private/autoscaler.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/autoscaler/_private/autoscaler.py b/python/ray/autoscaler/_private/autoscaler.py index 3307accf65703..03beb2725a72a 100644 --- a/python/ray/autoscaler/_private/autoscaler.py +++ b/python/ray/autoscaler/_private/autoscaler.py @@ -425,7 +425,7 @@ def _update(self): if node_id not in self.replicas_to_nodes[node_replica_index]: self.replicas_to_nodes[node_replica_index].append(node_id) else: - self.replicas_to_nodes[node_replica_index] = list(node_id) + self.replicas_to_nodes[node_replica_index].append(node_id) # This will accumulate the nodes we need to terminate. self.nodes_to_terminate = [] From 81df1dfb9675e6f3cfdf1988924b1e6da330afdc Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Wed, 12 Jun 2024 22:02:30 +0000 Subject: [PATCH 23/45] Add log statement and make naming consistent Signed-off-by: Ryan O'Leary --- python/ray/autoscaler/_private/autoscaler.py | 26 +++++++++++--------- 1 file changed, 15 insertions(+), 11 deletions(-) diff --git a/python/ray/autoscaler/_private/autoscaler.py b/python/ray/autoscaler/_private/autoscaler.py index 03beb2725a72a..dd063703b1c90 100644 --- a/python/ray/autoscaler/_private/autoscaler.py +++ b/python/ray/autoscaler/_private/autoscaler.py @@ -629,8 +629,12 @@ def schedule_node_termination( # Scale down entire multi-host replica of node being deleted tags = self.provider.node_tags(node_id) if TAG_RAY_REPLICA_INDEX in tags: - replica_id = tags[TAG_RAY_REPLICA_INDEX] - self.replicas_to_delete.add(replica_id) + replica_index = tags[TAG_RAY_REPLICA_INDEX] + self.replicas_to_delete.add(replica_index) + logger_method( + "StandardAutoscaler: " + f"Terminating nodes with replicaIndex {replica_index}." + ) def terminate_scheduled_nodes(self): """Terminate scheduled nodes and clean associated autoscaler state.""" @@ -650,13 +654,13 @@ def terminate_scheduled_nodes(self): tags = self.provider.node_tags(node) if TAG_RAY_REPLICA_INDEX in tags: assert self.replicas_to_nodes - replica_id = tags[TAG_RAY_REPLICA_INDEX] - if replica_id in self.replicas_to_nodes: - if node in self.replicas_to_nodes[replica_id]: - self.replicas_to_nodes[replica_id].remove(node) + replica_index = tags[TAG_RAY_REPLICA_INDEX] + if replica_index in self.replicas_to_nodes: + if node in self.replicas_to_nodes[replica_index]: + self.replicas_to_nodes[replica_index].remove(node) # remove replica ID once all nodes in replica removed - if len(self.replicas_to_nodes[replica_id]) == 0: - self.replicas_to_delete.remove(replica_id) + if len(self.replicas_to_nodes[replica_index]) == 0: + self.replicas_to_delete.remove(replica_index) # Terminate the nodes self.provider.terminate_nodes(self.nodes_to_terminate) for node in self.nodes_to_terminate: @@ -1032,13 +1036,13 @@ def _keep_worker_of_node_type( tags = self.provider.node_tags(node_id) if TAG_RAY_REPLICA_INDEX in tags: - replica_id = tags[TAG_RAY_REPLICA_INDEX] + replica_index = tags[TAG_RAY_REPLICA_INDEX] # All nodes in this replica should be deleted, regardless of # available_node_types. - if replica_id in self.replicas_to_delete: + if replica_index in self.replicas_to_delete: return ( KeepOrTerminate.terminate, - f"Node belongs to a replica being deleted: {replica_id}", + f"Node belongs to a replica being deleted: {replica_index}", ) if TAG_RAY_USER_NODE_TYPE in tags: From c219c22a4c70f23eb8f55c23e59ebf2989b4bc9e Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Wed, 12 Jun 2024 22:10:36 +0000 Subject: [PATCH 24/45] Change all instances of replica ID to index Signed-off-by: Ryan O'Leary --- python/ray/autoscaler/_private/autoscaler.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/python/ray/autoscaler/_private/autoscaler.py b/python/ray/autoscaler/_private/autoscaler.py index dd063703b1c90..b015db5c533f9 100644 --- a/python/ray/autoscaler/_private/autoscaler.py +++ b/python/ray/autoscaler/_private/autoscaler.py @@ -272,11 +272,11 @@ def read_fn(): # Tracks nodes scheduled for termination self.nodes_to_terminate: List[NodeID] = [] - # A set of replica IDs used to track replicas scheduled for termination. + # A set of replica indices used to track replicas scheduled for termination. self.replicas_to_delete: Set[str] = set() - # Map of replica IDs to worker nodes in each replica. - # A replica ID refers to the index of a multi-host PodSlice created by KubeRay. + # Map of replica indices to worker nodes in each replica. + # A replica index refers to a multi-host PodSlice created by KubeRay. self.replicas_to_nodes: Dict[str, List[NodeID]] = defaultdict(list) # Disable NodeUpdater threads if true. @@ -415,7 +415,7 @@ def _update(self): ) return - # Populate mapping of replica IDs to nodes in that replica. + # Populate mapping of replica indices to nodes in that replica. self.replicas_to_nodes.clear() for node_id in self.non_terminated_nodes.worker_ids: tags = self.provider.node_tags(node_id) @@ -658,7 +658,7 @@ def terminate_scheduled_nodes(self): if replica_index in self.replicas_to_nodes: if node in self.replicas_to_nodes[replica_index]: self.replicas_to_nodes[replica_index].remove(node) - # remove replica ID once all nodes in replica removed + # remove replica index once all nodes in replica removed if len(self.replicas_to_nodes[replica_index]) == 0: self.replicas_to_delete.remove(replica_index) # Terminate the nodes From 112a448b6bce7362ba96ca3b3d8894e68bee7511 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Thu, 13 Jun 2024 01:06:18 +0000 Subject: [PATCH 25/45] Remove NumOfHosts check from node provider Signed-off-by: Ryan O'Leary --- .../ray/autoscaler/_private/kuberay/node_provider.py | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/python/ray/autoscaler/_private/kuberay/node_provider.py b/python/ray/autoscaler/_private/kuberay/node_provider.py index 8b3c5caab54b0..823afe34e548e 100644 --- a/python/ray/autoscaler/_private/kuberay/node_provider.py +++ b/python/ray/autoscaler/_private/kuberay/node_provider.py @@ -227,12 +227,6 @@ def _worker_group_replicas(raycluster: Dict[str, Any], group_index: int): return raycluster["spec"]["workerGroupSpecs"][group_index].get("replicas", 1) -def _worker_group_num_of_hosts(raycluster: Dict[str, Any], group_index: int): - # Extract NumOfHosts of a worker group. 1 is the default NumOfHosts value used by - # the KubeRay operator. - return raycluster["spec"]["workerGroupSpecs"][group_index].get("numOfHosts", 1) - - class IKubernetesHttpApiClient(ABC): """ An interface for a Kubernetes HTTP API client. @@ -468,10 +462,6 @@ def _scale_request_to_patch_payload( for node_type, target_replicas in scale_request.desired_num_workers.items(): group_index = _worker_group_index(raycluster, node_type) group_max_replicas = _worker_group_max_replicas(raycluster, group_index) - group_num_of_hosts = _worker_group_num_of_hosts(raycluster, group_index) - # Account for multi-host worker groups - if group_num_of_hosts != 0: - target_replicas /= group_num_of_hosts # Cap the replica count to maxReplicas. if group_max_replicas is not None and group_max_replicas < target_replicas: logger.warning( From 64c5d4b462653575c5907c01239c7b5899025307 Mon Sep 17 00:00:00 2001 From: ryanaoleary <113500783+ryanaoleary@users.noreply.github.com> Date: Thu, 13 Jun 2024 15:35:10 -0700 Subject: [PATCH 26/45] Update python/ray/autoscaler/batching_node_provider.py Co-authored-by: Kai-Hsun Chen Signed-off-by: ryanaoleary <113500783+ryanaoleary@users.noreply.github.com> --- python/ray/autoscaler/batching_node_provider.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/autoscaler/batching_node_provider.py b/python/ray/autoscaler/batching_node_provider.py index 68ecd6ca5af63..42ba841c0b956 100644 --- a/python/ray/autoscaler/batching_node_provider.py +++ b/python/ray/autoscaler/batching_node_provider.py @@ -245,7 +245,7 @@ def terminate_node(self, node_id: str) -> Optional[Dict[str, Any]]: # Scale down entire replica if part of a TPU podslice being deleted node_replica_index = self.node_data_dict[node_id].replica_index - if node_replica_index: + if node_replica_index is not None: for worker in self.replicas_to_nodes[node_replica_index]: # Check if node has already been scheduled to delete if node_id not in self.scale_request.workers_to_delete: From eab706f60663bea116781f9df537f12e3bfad55e Mon Sep 17 00:00:00 2001 From: ryanaoleary <113500783+ryanaoleary@users.noreply.github.com> Date: Thu, 13 Jun 2024 15:45:56 -0700 Subject: [PATCH 27/45] Update python/ray/autoscaler/batching_node_provider.py Co-authored-by: Kai-Hsun Chen Signed-off-by: ryanaoleary <113500783+ryanaoleary@users.noreply.github.com> --- python/ray/autoscaler/batching_node_provider.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/autoscaler/batching_node_provider.py b/python/ray/autoscaler/batching_node_provider.py index 42ba841c0b956..4c4ec2adb26c8 100644 --- a/python/ray/autoscaler/batching_node_provider.py +++ b/python/ray/autoscaler/batching_node_provider.py @@ -170,7 +170,7 @@ def non_terminated_nodes(self, tag_filters: Dict[str, str]) -> List[str]: for node_id in all_nodes: replica_index = self.node_data_dict[node_id].replica_index # Only add node to map if it belongs to a TPU podslice - if replica_index: + if replica_index is not None: self.replicas_to_nodes[replica_index].append(node_id) # Support filtering by TAG_RAY_NODE_KIND, TAG_RAY_NODE_STATUS, and # TAG_RAY_USER_NODE_TYPE. From 96272bb95718a707d98e31ca3b7cabeb7ea0c1e1 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Thu, 13 Jun 2024 22:45:25 +0000 Subject: [PATCH 28/45] Clean up replicas to nodes and fixed var names Signed-off-by: Ryan O'Leary --- python/ray/autoscaler/_private/autoscaler.py | 1 + python/ray/autoscaler/batching_node_provider.py | 17 +++++++++-------- 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/python/ray/autoscaler/_private/autoscaler.py b/python/ray/autoscaler/_private/autoscaler.py index b015db5c533f9..d7ea719c3ba04 100644 --- a/python/ray/autoscaler/_private/autoscaler.py +++ b/python/ray/autoscaler/_private/autoscaler.py @@ -660,6 +660,7 @@ def terminate_scheduled_nodes(self): self.replicas_to_nodes[replica_index].remove(node) # remove replica index once all nodes in replica removed if len(self.replicas_to_nodes[replica_index]) == 0: + self.replicas_to_nodes.remove(replica_index) self.replicas_to_delete.remove(replica_index) # Terminate the nodes self.provider.terminate_nodes(self.nodes_to_terminate) diff --git a/python/ray/autoscaler/batching_node_provider.py b/python/ray/autoscaler/batching_node_provider.py index 4c4ec2adb26c8..1c40050e2977f 100644 --- a/python/ray/autoscaler/batching_node_provider.py +++ b/python/ray/autoscaler/batching_node_provider.py @@ -243,16 +243,17 @@ def terminate_node(self, node_id: str) -> Optional[Dict[str, Any]]: f"{node_type}. Skipping termination request." ) - # Scale down entire replica if part of a TPU podslice being deleted + # Terminate node + self.scale_request.desired_num_workers[node_type] -= 1 + self.scale_request.workers_to_delete.add(node_id) + + # Scale down all nodes in replica if node_id is part of a multi-host podslice node_replica_index = self.node_data_dict[node_id].replica_index if node_replica_index is not None: - for worker in self.replicas_to_nodes[node_replica_index]: - # Check if node has already been scheduled to delete - if node_id not in self.scale_request.workers_to_delete: + for worker_id in self.replicas_to_nodes[node_replica_index]: + # Check if worker has already been scheduled to delete + if worker_id not in self.scale_request.workers_to_delete: # Assume all workers in a group are of the same type self.scale_request.desired_num_workers[node_type] -= 1 - self.scale_request.workers_to_delete.add(node_id) - else: - self.scale_request.desired_num_workers[node_type] -= 1 - self.scale_request.workers_to_delete.add(node_id) + self.scale_request.workers_to_delete.add(worker_id) self.scale_change_needed = True From fa4e28583298f33fff06ddf480c271a299858956 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Thu, 13 Jun 2024 22:59:37 +0000 Subject: [PATCH 29/45] Change from remove to pop Signed-off-by: Ryan O'Leary --- python/ray/autoscaler/_private/autoscaler.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/autoscaler/_private/autoscaler.py b/python/ray/autoscaler/_private/autoscaler.py index d7ea719c3ba04..8410a021b39c9 100644 --- a/python/ray/autoscaler/_private/autoscaler.py +++ b/python/ray/autoscaler/_private/autoscaler.py @@ -660,7 +660,7 @@ def terminate_scheduled_nodes(self): self.replicas_to_nodes[replica_index].remove(node) # remove replica index once all nodes in replica removed if len(self.replicas_to_nodes[replica_index]) == 0: - self.replicas_to_nodes.remove(replica_index) + self.replicas_to_nodes.pop(replica_index) self.replicas_to_delete.remove(replica_index) # Terminate the nodes self.provider.terminate_nodes(self.nodes_to_terminate) From 963525061576517c4418e749cc70428a73492a62 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Fri, 14 Jun 2024 02:28:56 +0000 Subject: [PATCH 30/45] Add None check to replicas_to_delete Signed-off-by: Ryan O'Leary --- python/ray/autoscaler/_private/autoscaler.py | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/python/ray/autoscaler/_private/autoscaler.py b/python/ray/autoscaler/_private/autoscaler.py index 8410a021b39c9..0671dce0ac471 100644 --- a/python/ray/autoscaler/_private/autoscaler.py +++ b/python/ray/autoscaler/_private/autoscaler.py @@ -630,11 +630,12 @@ def schedule_node_termination( tags = self.provider.node_tags(node_id) if TAG_RAY_REPLICA_INDEX in tags: replica_index = tags[TAG_RAY_REPLICA_INDEX] - self.replicas_to_delete.add(replica_index) - logger_method( - "StandardAutoscaler: " - f"Terminating nodes with replicaIndex {replica_index}." - ) + if replica_index is not None: + self.replicas_to_delete.add(replica_index) + logger_method( + "StandardAutoscaler: " + f"Terminating nodes with replicaIndex {replica_index}." + ) def terminate_scheduled_nodes(self): """Terminate scheduled nodes and clean associated autoscaler state.""" From 0504175f6f61f765897cde2c97babc052d397dec Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Fri, 14 Jun 2024 06:02:47 +0000 Subject: [PATCH 31/45] Add more testing for scale down behavior Signed-off-by: Ryan O'Leary --- .../tests/kuberay/test_files/podlist2.yaml | 191 ++++++++++++++++++ .../kuberay/test_kuberay_node_provider.py | 9 + python/ray/tests/test_autoscaler.py | 110 ++++++++++ 3 files changed, 310 insertions(+) diff --git a/python/ray/tests/kuberay/test_files/podlist2.yaml b/python/ray/tests/kuberay/test_files/podlist2.yaml index 56371fb1b76a2..270eea0515862 100644 --- a/python/ray/tests/kuberay/test_files/podlist2.yaml +++ b/python/ray/tests/kuberay/test_files/podlist2.yaml @@ -405,6 +405,197 @@ items: - ip: 10.4.0.6 qosClass: Burstable startTime: "2022-11-14T23:13:47Z" +- apiVersion: v1 + kind: Pod + metadata: + annotations: + key: value + ray.io/ft-enabled: "false" + ray.io/health-state: "" + creationTimestamp: "2022-11-14T23:13:47Z" + generateName: raycluster-autoscaler-worker-fake-tpu-group- + labels: + app.kubernetes.io/created-by: kuberay-operator + app.kubernetes.io/name: kuberay + key: value + ray.io/cluster: raycluster-autoscaler + ray.io/cluster-dashboard: raycluster-autoscaler-dashboard + ray.io/group: fake-tpu-group + ray.io/identifier: raycluster-autoscaler-worker + ray.io/is-ray-node: "yes" + ray.io/node-type: worker + replicaIndex: fake-tpu-group-0 # GKE set label for multi-host podslices + name: raycluster-autoscaler-worker-fake-tpu-group-3rlvy + namespace: default + ownerReferences: + - apiVersion: ray.io/v1alpha1 + blockOwnerDeletion: true + controller: true + kind: RayCluster + name: raycluster-autoscaler + uid: ec79effb-0295-4f40-b08b-8633aa7f786a + resourceVersion: "5805" + uid: f1cadfb8-b76f-467d-9c2b-a1a5eb7d1d3f + spec: + containers: + - args: + - 'ulimit -n 65536; ray start --address=raycluster-autoscaler-head-svc:6379 --metrics-export-port=8080 --num-cpus=1 --memory=536870912 --block --resources="{\"Custom1\": + 1, \"Custom2\": 5, \"TPU\": 4}" ' + command: + - /bin/bash + - -c + - -- + env: + - name: RAY_IP + value: raycluster-autoscaler-head-svc + - name: RAY_PORT + value: "6379" + - name: RAY_ADDRESS + value: raycluster-autoscaler-head-svc:6379 + - name: REDIS_PASSWORD + image: gekho/ray + imagePullPolicy: Always + lifecycle: + preStop: + exec: + command: + - /bin/sh + - -c + - ray stop + name: machine-learning + ports: + - containerPort: 8080 + name: metrics + protocol: TCP + resources: + limits: + cpu: "1" + memory: 512Mi + requests: + cpu: 500m + memory: 256Mi + terminationMessagePath: /dev/termination-log + terminationMessagePolicy: File + volumeMounts: + - mountPath: /dev/shm + name: shared-mem + - mountPath: /var/run/secrets/kubernetes.io/serviceaccount + name: kube-api-access-ghm2j + readOnly: true + dnsPolicy: ClusterFirst + enableServiceLinks: true + initContainers: + - command: + - sh + - -c + - until nslookup $RAY_IP.$(cat /var/run/secrets/kubernetes.io/serviceaccount/namespace).svc.cluster.local; + do echo waiting for myservice; sleep 2; done + env: + - name: RAY_IP + value: raycluster-autoscaler-head-svc + image: busybox:1.28 + imagePullPolicy: IfNotPresent + name: init-myservice + resources: {} + terminationMessagePath: /dev/termination-log + terminationMessagePolicy: File + volumeMounts: + - mountPath: /var/run/secrets/kubernetes.io/serviceaccount + name: kube-api-access-ghm2j + readOnly: true + nodeName: gke-cluster-1-default-pool-a5503908-dpst + preemptionPolicy: PreemptLowerPriority + priority: 0 + restartPolicy: Always + schedulerName: default-scheduler + securityContext: {} + serviceAccount: default + serviceAccountName: default + terminationGracePeriodSeconds: 30 + tolerations: + - effect: NoExecute + key: node.kubernetes.io/not-ready + operator: Exists + tolerationSeconds: 300 + - effect: NoExecute + key: node.kubernetes.io/unreachable + operator: Exists + tolerationSeconds: 300 + volumes: + - emptyDir: + medium: Memory + sizeLimit: 256Mi + name: shared-mem + - name: kube-api-access-ghm2j + projected: + defaultMode: 420 + sources: + - serviceAccountToken: + expirationSeconds: 3607 + path: token + - configMap: + items: + - key: ca.crt + path: ca.crt + name: kube-root-ca.crt + - downwardAPI: + items: + - fieldRef: + apiVersion: v1 + fieldPath: metadata.namespace + path: namespace + status: + conditions: + - lastProbeTime: null + lastTransitionTime: "2022-11-14T23:13:49Z" + status: "True" + type: Initialized + - lastProbeTime: null + lastTransitionTime: "2022-11-14T23:13:50Z" + status: "True" + type: Ready + - lastProbeTime: null + lastTransitionTime: "2022-11-14T23:13:50Z" + status: "True" + type: ContainersReady + - lastProbeTime: null + lastTransitionTime: "2022-11-14T23:13:47Z" + status: "True" + type: PodScheduled + containerStatuses: + - containerID: containerd://b6f4129ac590c62e41cc6db1a18fb051a29e3458fb47296086292a5765c36a6c + image: docker.io/gekho/ray:latest + imageID: docker.io/gekho/ray@sha256:7859a78d1a089bb88691864d5c4a2aad529f5353d7d9c82cc0274842fbda242b + lastState: {} + name: machine-learning + ready: true + restartCount: 0 + started: true + state: + running: + startedAt: "2022-11-14T23:13:49Z" + hostIP: 10.128.0.31 + initContainerStatuses: + - containerID: containerd://5c946a3d443f76d21a3c72244edb4f80fc420b13b223f3c6981e25e689d381cf + image: docker.io/library/busybox:1.28 + imageID: docker.io/library/busybox@sha256:141c253bc4c3fd0a201d32dc1f493bcf3fff003b6df416dea4f41046e0f37d47 + lastState: {} + name: init-myservice + ready: true + restartCount: 0 + state: + terminated: + containerID: containerd://5c946a3d443f76d21a3c72244edb4f80fc420b13b223f3c6981e25e689d381cf + exitCode: 0 + finishedAt: "2022-11-14T23:13:48Z" + reason: Completed + startedAt: "2022-11-14T23:13:48Z" + phase: Running + podIP: 10.4.3.6 + podIPs: + - ip: 10.4.3.6 + qosClass: Burstable + startTime: "2022-11-14T23:13:47Z" - apiVersion: v1 kind: Pod metadata: diff --git a/python/ray/tests/kuberay/test_kuberay_node_provider.py b/python/ray/tests/kuberay/test_kuberay_node_provider.py index 40c5e27a9a32f..fe546214b2e03 100644 --- a/python/ray/tests/kuberay/test_kuberay_node_provider.py +++ b/python/ray/tests/kuberay/test_kuberay_node_provider.py @@ -5,6 +5,7 @@ import jsonpatch import pytest +from collections import defaultdict from ray.autoscaler.batching_node_provider import NodeData from ray.autoscaler._private.kuberay.node_provider import ( _worker_group_index, @@ -160,6 +161,13 @@ def test_create_node_cap_at_max( ip="10.4.0.6", status="up-to-date", ), + "raycluster-autoscaler-worker-fake-tpu-group-3rlvy": NodeData( + kind="worker", + type="fake-tpu-group", + replica_index="fake-tpu-group-0", + ip="10.4.3.6", + status="up-to-date", + ), "raycluster-autoscaler-worker-small-group-dkz2r": NodeData( kind="worker", type="small-group", @@ -195,6 +203,7 @@ def mock_get(node_provider, path): ), mock.patch.object(KubeRayNodeProvider, "_get", mock_get): kr_node_provider = KubeRayNodeProvider(provider_config={}, cluster_name="fake") kr_node_provider.cluster_name = "fake" + kr_node_provider.replicas_to_nodes = defaultdict(list) nodes = kr_node_provider.non_terminated_nodes({}) assert kr_node_provider.node_data_dict == expected_node_data assert set(nodes) == set(expected_node_data.keys()) diff --git a/python/ray/tests/test_autoscaler.py b/python/ray/tests/test_autoscaler.py index 7870d804ead5b..d52415a2b02c8 100644 --- a/python/ray/tests/test_autoscaler.py +++ b/python/ray/tests/test_autoscaler.py @@ -64,6 +64,7 @@ TAG_RAY_NODE_KIND, TAG_RAY_NODE_STATUS, TAG_RAY_USER_NODE_TYPE, + TAG_RAY_REPLICA_INDEX, ) from ray.tests.test_batch_node_provider_unit import ( MockBatchingNodeProvider, @@ -316,10 +317,30 @@ def update_nodes(self): }, } +TYPES_TPU = { + "empty_node": { + "node_config": { + "FooProperty": 42, + "TestProp": 1, + }, + "resources": {}, + "max_workers": 0, + }, + "tpu-worker": { + "node_config": {}, + "resources": {"CPU": 1, "TPU": 4}, + "max_workers": 4, + }, +} + MULTI_WORKER_CLUSTER = dict( SMALL_CLUSTER, **{"available_node_types": TYPES_A, "head_node_type": "empty_node"} ) +TPU_CLUSTER = dict( + SMALL_CLUSTER, **{"available_node_types": TYPES_TPU, "head_node_type": "empty_node"} +) + exc_info = None try: raise Exception("Test exception.") @@ -3574,6 +3595,95 @@ def _test_autoscaler_status_log(self, status_log_enabled_env: int): break assert status_log_found is bool(status_log_enabled_env) + def testTerminateMultiHostReplica(self): + """Test multi-host replica deletion logic for KubeRay. + + Tests manually deleting a node in a multi-host replica + and verifying that the entire replica is scaled down. + Nodes belonging to the same multi-host replica are identified + through a replicaIndex label set by a GKE webhook. + """ + config = copy.deepcopy(TPU_CLUSTER) + config["available_node_types"]["tpu-worker"]["min_workers"] = 4 + config["available_node_types"]["tpu-worker"]["max_workers"] = 4 + config_path = self.write_config(config) + self.provider = MockProvider() + runner = MockProcessRunner() + runner.respond_to_call("json .Config.Env", ["[]" for i in range(5)]) + lm = LoadMetrics() + + get_or_create_head_node( + config, + printable_config_file=config_path, + no_restart=False, + restart_only=False, + yes=True, + override_cluster_name=None, + _provider=self.provider, + _runner=runner, + ) + self.waitForNodes(1) + autoscaler = MockAutoscaler( + config_path, + lm, + MockGcsClient(), + max_failures=0, + max_concurrent_launches=13, + max_launch_batch=13, + process_runner=runner, + update_interval_s=0, + ) + autoscaler.update() + self.waitForNodes(5) + assert autoscaler.pending_launches.value == 0 + assert ( + len( + self.provider.non_terminated_nodes( + {TAG_RAY_NODE_KIND: NODE_KIND_WORKER} + ) + ) + == 4 + ) + + fill_in_raylet_ids(self.provider, lm) + autoscaler.update() + self.waitFor(lambda: autoscaler.pending_launches.value == 0) + self.waitForNodes(4, tag_filters={TAG_RAY_NODE_KIND: NODE_KIND_WORKER}) + assert autoscaler.pending_launches.value == 0 + + # Set replicaIndex node tag + for node_id in NonTerminatedNodes(self.provider).worker_ids: + self.provider.set_node_tags( + node_id, {TAG_RAY_REPLICA_INDEX: "tpu-group-0", **WORKER_FILTER} + ) + + # Manually delete one TPU worker + mock_logger = Mock(spec=logging.Logger("")) + worker_0 = NonTerminatedNodes(self.provider).worker_ids[0] + autoscaler.schedule_node_termination( + worker_0, "deleting TPU worker for test", mock_logger.info + ) + autoscaler.update() + events = autoscaler.event_summarizer.summary() + assert ( + "Removing 4 nodes of type tpu-worker (Node belongs to a replica being " + "deleted: tpu-group-0)." + ) in events + + # We should not be starting/stopping empty_node at all. + for event in events: + assert "empty_node" not in event + + # All nodes in the same replica index have been removed + assert ( + len( + self.provider.non_terminated_nodes( + {TAG_RAY_NODE_KIND: NODE_KIND_WORKER} + ) + ) + == 0 + ) + def test_import(): """This test ensures that all the autoscaler imports work as expected to From fd27aca3160c72ff4ef906147a3fb117c9ed1156 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Fri, 14 Jun 2024 07:13:40 +0000 Subject: [PATCH 32/45] Add more error checking Signed-off-by: Ryan O'Leary --- python/ray/autoscaler/_private/autoscaler.py | 27 ++++++++++---------- 1 file changed, 14 insertions(+), 13 deletions(-) diff --git a/python/ray/autoscaler/_private/autoscaler.py b/python/ray/autoscaler/_private/autoscaler.py index 0671dce0ac471..e5d6f25c65c0e 100644 --- a/python/ray/autoscaler/_private/autoscaler.py +++ b/python/ray/autoscaler/_private/autoscaler.py @@ -654,15 +654,15 @@ def terminate_scheduled_nodes(self): for node in self.nodes_to_terminate: tags = self.provider.node_tags(node) if TAG_RAY_REPLICA_INDEX in tags: - assert self.replicas_to_nodes replica_index = tags[TAG_RAY_REPLICA_INDEX] - if replica_index in self.replicas_to_nodes: - if node in self.replicas_to_nodes[replica_index]: - self.replicas_to_nodes[replica_index].remove(node) - # remove replica index once all nodes in replica removed - if len(self.replicas_to_nodes[replica_index]) == 0: - self.replicas_to_nodes.pop(replica_index) - self.replicas_to_delete.remove(replica_index) + if replica_index is not None: + if replica_index in self.replicas_to_nodes: + if node in self.replicas_to_nodes[replica_index]: + self.replicas_to_nodes[replica_index].remove(node) + # remove replica index once all nodes in replica removed + if len(self.replicas_to_nodes[replica_index]) == 0: + self.replicas_to_nodes.pop(replica_index) + self.replicas_to_delete.remove(replica_index) # Terminate the nodes self.provider.terminate_nodes(self.nodes_to_terminate) for node in self.nodes_to_terminate: @@ -1041,11 +1041,12 @@ def _keep_worker_of_node_type( replica_index = tags[TAG_RAY_REPLICA_INDEX] # All nodes in this replica should be deleted, regardless of # available_node_types. - if replica_index in self.replicas_to_delete: - return ( - KeepOrTerminate.terminate, - f"Node belongs to a replica being deleted: {replica_index}", - ) + if replica_index is not None: + if replica_index in self.replicas_to_delete: + return ( + KeepOrTerminate.terminate, + f"Node belongs to a replica being deleted: {replica_index}", + ) if TAG_RAY_USER_NODE_TYPE in tags: node_type = tags[TAG_RAY_USER_NODE_TYPE] From 47e46ac7aebe90b758af1abe8b30e2d8dfbd2432 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Thu, 20 Jun 2024 19:05:38 +0000 Subject: [PATCH 33/45] Add node_tags None check Signed-off-by: Ryan O'Leary --- python/ray/autoscaler/_private/autoscaler.py | 43 ++++++++----------- .../ray/autoscaler/batching_node_provider.py | 8 +++- 2 files changed, 25 insertions(+), 26 deletions(-) diff --git a/python/ray/autoscaler/_private/autoscaler.py b/python/ray/autoscaler/_private/autoscaler.py index e5d6f25c65c0e..668ece74342f3 100644 --- a/python/ray/autoscaler/_private/autoscaler.py +++ b/python/ray/autoscaler/_private/autoscaler.py @@ -421,11 +421,7 @@ def _update(self): tags = self.provider.node_tags(node_id) if TAG_RAY_REPLICA_INDEX in tags: node_replica_index = tags[TAG_RAY_REPLICA_INDEX] - if node_replica_index in self.replicas_to_nodes: - if node_id not in self.replicas_to_nodes[node_replica_index]: - self.replicas_to_nodes[node_replica_index].append(node_id) - else: - self.replicas_to_nodes[node_replica_index].append(node_id) + self.replicas_to_nodes[node_replica_index].append(node_id) # This will accumulate the nodes we need to terminate. self.nodes_to_terminate = [] @@ -630,12 +626,11 @@ def schedule_node_termination( tags = self.provider.node_tags(node_id) if TAG_RAY_REPLICA_INDEX in tags: replica_index = tags[TAG_RAY_REPLICA_INDEX] - if replica_index is not None: - self.replicas_to_delete.add(replica_index) - logger_method( - "StandardAutoscaler: " - f"Terminating nodes with replicaIndex {replica_index}." - ) + self.replicas_to_delete.add(replica_index) + logger_method( + "StandardAutoscaler: " + f"Terminating nodes with replicaIndex {replica_index}." + ) def terminate_scheduled_nodes(self): """Terminate scheduled nodes and clean associated autoscaler state.""" @@ -655,14 +650,13 @@ def terminate_scheduled_nodes(self): tags = self.provider.node_tags(node) if TAG_RAY_REPLICA_INDEX in tags: replica_index = tags[TAG_RAY_REPLICA_INDEX] - if replica_index is not None: - if replica_index in self.replicas_to_nodes: - if node in self.replicas_to_nodes[replica_index]: - self.replicas_to_nodes[replica_index].remove(node) - # remove replica index once all nodes in replica removed - if len(self.replicas_to_nodes[replica_index]) == 0: - self.replicas_to_nodes.pop(replica_index) - self.replicas_to_delete.remove(replica_index) + if replica_index in self.replicas_to_nodes: + if node in self.replicas_to_nodes[replica_index]: + self.replicas_to_nodes[replica_index].remove(node) + # remove replica index once all nodes in replica removed + if len(self.replicas_to_nodes[replica_index]) == 0: + self.replicas_to_nodes.pop(replica_index) + self.replicas_to_delete.remove(replica_index) # Terminate the nodes self.provider.terminate_nodes(self.nodes_to_terminate) for node in self.nodes_to_terminate: @@ -1041,12 +1035,11 @@ def _keep_worker_of_node_type( replica_index = tags[TAG_RAY_REPLICA_INDEX] # All nodes in this replica should be deleted, regardless of # available_node_types. - if replica_index is not None: - if replica_index in self.replicas_to_delete: - return ( - KeepOrTerminate.terminate, - f"Node belongs to a replica being deleted: {replica_index}", - ) + if replica_index in self.replicas_to_delete: + return ( + KeepOrTerminate.terminate, + f"Node belongs to a replica being deleted: {replica_index}", + ) if TAG_RAY_USER_NODE_TYPE in tags: node_type = tags[TAG_RAY_USER_NODE_TYPE] diff --git a/python/ray/autoscaler/batching_node_provider.py b/python/ray/autoscaler/batching_node_provider.py index 1c40050e2977f..4b12fd505f820 100644 --- a/python/ray/autoscaler/batching_node_provider.py +++ b/python/ray/autoscaler/batching_node_provider.py @@ -199,11 +199,17 @@ def _cur_num_workers(self, node_data_dict: Dict[str, Any]): def node_tags(self, node_id: str) -> Dict[str, str]: node_data = self.node_data_dict[node_id] + if node_data.replica_index is not None: + return { + TAG_RAY_NODE_KIND: node_data.kind, + TAG_RAY_NODE_STATUS: node_data.status, + TAG_RAY_USER_NODE_TYPE: node_data.type, + TAG_RAY_REPLICA_INDEX: node_data.replica_index, + } return { TAG_RAY_NODE_KIND: node_data.kind, TAG_RAY_NODE_STATUS: node_data.status, TAG_RAY_USER_NODE_TYPE: node_data.type, - TAG_RAY_REPLICA_INDEX: node_data.replica_index, } def internal_ip(self, node_id: str) -> str: From a06581c75fbfae32ec5b237c4b40f2aee28af7a1 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Thu, 20 Jun 2024 21:00:57 +0000 Subject: [PATCH 34/45] Change get_node_tags and fix terminate node logic Signed-off-by: Ryan O'Leary --- python/ray/autoscaler/batching_node_provider.py | 14 ++++---------- 1 file changed, 4 insertions(+), 10 deletions(-) diff --git a/python/ray/autoscaler/batching_node_provider.py b/python/ray/autoscaler/batching_node_provider.py index 4b12fd505f820..36594008b79d2 100644 --- a/python/ray/autoscaler/batching_node_provider.py +++ b/python/ray/autoscaler/batching_node_provider.py @@ -199,18 +199,14 @@ def _cur_num_workers(self, node_data_dict: Dict[str, Any]): def node_tags(self, node_id: str) -> Dict[str, str]: node_data = self.node_data_dict[node_id] - if node_data.replica_index is not None: - return { - TAG_RAY_NODE_KIND: node_data.kind, - TAG_RAY_NODE_STATUS: node_data.status, - TAG_RAY_USER_NODE_TYPE: node_data.type, - TAG_RAY_REPLICA_INDEX: node_data.replica_index, - } - return { + tags = { TAG_RAY_NODE_KIND: node_data.kind, TAG_RAY_NODE_STATUS: node_data.status, TAG_RAY_USER_NODE_TYPE: node_data.type, } + if node_data.replica_index is not None: + tags[TAG_RAY_REPLICA_INDEX] = node_data.replica_index + return tags def internal_ip(self, node_id: str) -> str: return self.node_data_dict[node_id].ip @@ -259,7 +255,5 @@ def terminate_node(self, node_id: str) -> Optional[Dict[str, Any]]: for worker_id in self.replicas_to_nodes[node_replica_index]: # Check if worker has already been scheduled to delete if worker_id not in self.scale_request.workers_to_delete: - # Assume all workers in a group are of the same type - self.scale_request.desired_num_workers[node_type] -= 1 self.scale_request.workers_to_delete.add(worker_id) self.scale_change_needed = True From 33d7f0ae0580dafb4f7869061260b6290639d286 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Wed, 26 Jun 2024 21:39:57 +0000 Subject: [PATCH 35/45] Move multi-host scale down logic entirely to batching_node_provider Signed-off-by: Ryan O'Leary --- python/ray/autoscaler/_private/autoscaler.py | 56 +------------------ .../ray/autoscaler/batching_node_provider.py | 7 +-- python/ray/autoscaler/tags.py | 2 - 3 files changed, 4 insertions(+), 61 deletions(-) diff --git a/python/ray/autoscaler/_private/autoscaler.py b/python/ray/autoscaler/_private/autoscaler.py index 668ece74342f3..0c08eb5b4a3b3 100644 --- a/python/ray/autoscaler/_private/autoscaler.py +++ b/python/ray/autoscaler/_private/autoscaler.py @@ -74,7 +74,6 @@ TAG_RAY_LAUNCH_CONFIG, TAG_RAY_NODE_KIND, TAG_RAY_NODE_STATUS, - TAG_RAY_REPLICA_INDEX, TAG_RAY_RUNTIME_CONFIG, TAG_RAY_USER_NODE_TYPE, ) @@ -272,13 +271,6 @@ def read_fn(): # Tracks nodes scheduled for termination self.nodes_to_terminate: List[NodeID] = [] - # A set of replica indices used to track replicas scheduled for termination. - self.replicas_to_delete: Set[str] = set() - - # Map of replica indices to worker nodes in each replica. - # A replica index refers to a multi-host PodSlice created by KubeRay. - self.replicas_to_nodes: Dict[str, List[NodeID]] = defaultdict(list) - # Disable NodeUpdater threads if true. # Should be set to true in situations where another component, such as # a Kubernetes operator, is responsible for Ray setup on nodes. @@ -415,14 +407,6 @@ def _update(self): ) return - # Populate mapping of replica indices to nodes in that replica. - self.replicas_to_nodes.clear() - for node_id in self.non_terminated_nodes.worker_ids: - tags = self.provider.node_tags(node_id) - if TAG_RAY_REPLICA_INDEX in tags: - node_replica_index = tags[TAG_RAY_REPLICA_INDEX] - self.replicas_to_nodes[node_replica_index].append(node_id) - # This will accumulate the nodes we need to terminate. self.nodes_to_terminate = [] @@ -498,7 +482,6 @@ def terminate_nodes_to_enforce_config_constraints(self, now: float): (3) Terminates outdated nodes, namely nodes whose configs don't match `node_config` for the relevant node type. - (4) Terminates nodes part of a multi-host replica being deleted. Avoids terminating non-outdated nodes required by autoscaler.sdk.request_resources(). @@ -622,15 +605,6 @@ def schedule_node_termination( aggregate=operator.add, ) self.nodes_to_terminate.append(node_id) - # Scale down entire multi-host replica of node being deleted - tags = self.provider.node_tags(node_id) - if TAG_RAY_REPLICA_INDEX in tags: - replica_index = tags[TAG_RAY_REPLICA_INDEX] - self.replicas_to_delete.add(replica_index) - logger_method( - "StandardAutoscaler: " - f"Terminating nodes with replicaIndex {replica_index}." - ) def terminate_scheduled_nodes(self): """Terminate scheduled nodes and clean associated autoscaler state.""" @@ -645,18 +619,6 @@ def terminate_scheduled_nodes(self): # explicitly disabled. if self.worker_rpc_drain: self.drain_nodes_via_gcs(self.nodes_to_terminate) - # Clean up multi-host replicas to delete - for node in self.nodes_to_terminate: - tags = self.provider.node_tags(node) - if TAG_RAY_REPLICA_INDEX in tags: - replica_index = tags[TAG_RAY_REPLICA_INDEX] - if replica_index in self.replicas_to_nodes: - if node in self.replicas_to_nodes[replica_index]: - self.replicas_to_nodes[replica_index].remove(node) - # remove replica index once all nodes in replica removed - if len(self.replicas_to_nodes[replica_index]) == 0: - self.replicas_to_nodes.pop(replica_index) - self.replicas_to_delete.remove(replica_index) # Terminate the nodes self.provider.terminate_nodes(self.nodes_to_terminate) for node in self.nodes_to_terminate: @@ -996,9 +958,7 @@ def _keep_worker_of_node_type( self, node_id: NodeID, node_type_counts: Dict[NodeType, int] ) -> Tuple[KeepOrTerminate, Optional[str]]: """Determines if a worker should be kept based on the min_workers - and max_workers constraint of the worker's node_type. Additionally, - workers belonging to a multi-host replica being deleted are scheduled - to delete as well. + and max_workers constraint of the worker's node_type. Returns KeepOrTerminate.keep when both of the following hold: (a) The worker's node_type is present among the keys of the current @@ -1006,13 +966,11 @@ def _keep_worker_of_node_type( (b) Deleting the node would violate the min_workers constraint for that worker's node_type. - Returns KeepOrTerminate.terminate when the following hold: + Returns KeepOrTerminate.terminate when both the following hold: (a) The worker's node_type is not present among the keys of the current config's available_node_types dict. (b) Keeping the node would violate the max_workers constraint for that worker's node_type. - (c) The worker has TAG_RAY_REPLICA_INDEX tag set, and its replica index is - found in replicas_to_delete. Return KeepOrTerminate.decide_later otherwise. @@ -1031,16 +989,6 @@ def _keep_worker_of_node_type( tags = self.provider.node_tags(node_id) - if TAG_RAY_REPLICA_INDEX in tags: - replica_index = tags[TAG_RAY_REPLICA_INDEX] - # All nodes in this replica should be deleted, regardless of - # available_node_types. - if replica_index in self.replicas_to_delete: - return ( - KeepOrTerminate.terminate, - f"Node belongs to a replica being deleted: {replica_index}", - ) - if TAG_RAY_USER_NODE_TYPE in tags: node_type = tags[TAG_RAY_USER_NODE_TYPE] diff --git a/python/ray/autoscaler/batching_node_provider.py b/python/ray/autoscaler/batching_node_provider.py index 36594008b79d2..35201d646e245 100644 --- a/python/ray/autoscaler/batching_node_provider.py +++ b/python/ray/autoscaler/batching_node_provider.py @@ -14,7 +14,6 @@ NODE_KIND_HEAD, TAG_RAY_NODE_KIND, TAG_RAY_NODE_STATUS, - TAG_RAY_REPLICA_INDEX, TAG_RAY_USER_NODE_TYPE, ) @@ -167,6 +166,7 @@ def non_terminated_nodes(self, tag_filters: Dict[str, str]) -> List[str]: workers_to_delete=set(), # No workers to delete yet ) all_nodes = list(self.node_data_dict.keys()) + self.replicas_to_nodes.clear() for node_id in all_nodes: replica_index = self.node_data_dict[node_id].replica_index # Only add node to map if it belongs to a TPU podslice @@ -199,14 +199,11 @@ def _cur_num_workers(self, node_data_dict: Dict[str, Any]): def node_tags(self, node_id: str) -> Dict[str, str]: node_data = self.node_data_dict[node_id] - tags = { + return { TAG_RAY_NODE_KIND: node_data.kind, TAG_RAY_NODE_STATUS: node_data.status, TAG_RAY_USER_NODE_TYPE: node_data.type, } - if node_data.replica_index is not None: - tags[TAG_RAY_REPLICA_INDEX] = node_data.replica_index - return tags def internal_ip(self, node_id: str) -> str: return self.node_data_dict[node_id].ip diff --git a/python/ray/autoscaler/tags.py b/python/ray/autoscaler/tags.py index 38d03855040fd..380b4450d6eca 100644 --- a/python/ray/autoscaler/tags.py +++ b/python/ray/autoscaler/tags.py @@ -13,8 +13,6 @@ # Tag for user defined node types (e.g., m4xl_spot). This is used for multi # node type clusters. TAG_RAY_USER_NODE_TYPE = "ray-user-node-type" -# Tag for index of replica node belongs to. Used for multi-host worker groups. -TAG_RAY_REPLICA_INDEX = "ray-replica-index" # Tag for autofilled node types for legacy cluster yamls without multi # node type defined in the cluster configs. NODE_TYPE_LEGACY_HEAD = "ray-legacy-head-node-type" From 45c5b230f232b0e5a3554e8b409a6dbce3b6a3ae Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Wed, 26 Jun 2024 22:15:09 +0000 Subject: [PATCH 36/45] Add logger info statement for multi-host scaling Signed-off-by: Ryan O'Leary --- python/ray/autoscaler/batching_node_provider.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/python/ray/autoscaler/batching_node_provider.py b/python/ray/autoscaler/batching_node_provider.py index 35201d646e245..194223701533a 100644 --- a/python/ray/autoscaler/batching_node_provider.py +++ b/python/ray/autoscaler/batching_node_provider.py @@ -253,4 +253,8 @@ def terminate_node(self, node_id: str) -> Optional[Dict[str, Any]]: # Check if worker has already been scheduled to delete if worker_id not in self.scale_request.workers_to_delete: self.scale_request.workers_to_delete.add(worker_id) + logger.info( + f"Autoscaler terminating node {node_id}" + f"in multi-host replica {node_replica_index}." + ) self.scale_change_needed = True From 184bc2bc4ac2447d603e289051d2953baaefdc76 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Thu, 27 Jun 2024 03:04:18 +0000 Subject: [PATCH 37/45] Add back in replica index tag Signed-off-by: Ryan O'Leary --- python/ray/autoscaler/batching_node_provider.py | 13 +++++++++---- python/ray/autoscaler/tags.py | 2 ++ 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/python/ray/autoscaler/batching_node_provider.py b/python/ray/autoscaler/batching_node_provider.py index 194223701533a..0de016a88a90a 100644 --- a/python/ray/autoscaler/batching_node_provider.py +++ b/python/ray/autoscaler/batching_node_provider.py @@ -14,6 +14,7 @@ NODE_KIND_HEAD, TAG_RAY_NODE_KIND, TAG_RAY_NODE_STATUS, + TAG_RAY_REPLICA_INDEX, TAG_RAY_USER_NODE_TYPE, ) @@ -169,7 +170,7 @@ def non_terminated_nodes(self, tag_filters: Dict[str, str]) -> List[str]: self.replicas_to_nodes.clear() for node_id in all_nodes: replica_index = self.node_data_dict[node_id].replica_index - # Only add node to map if it belongs to a TPU podslice + # Only add node to map if it belongs to a multi-host podslice if replica_index is not None: self.replicas_to_nodes[replica_index].append(node_id) # Support filtering by TAG_RAY_NODE_KIND, TAG_RAY_NODE_STATUS, and @@ -199,11 +200,14 @@ def _cur_num_workers(self, node_data_dict: Dict[str, Any]): def node_tags(self, node_id: str) -> Dict[str, str]: node_data = self.node_data_dict[node_id] - return { + tags = { TAG_RAY_NODE_KIND: node_data.kind, TAG_RAY_NODE_STATUS: node_data.status, TAG_RAY_USER_NODE_TYPE: node_data.type, } + if node_data.replica_index is not None: + tags[TAG_RAY_REPLICA_INDEX] = node_data.replica_index + return tags def internal_ip(self, node_id: str) -> str: return self.node_data_dict[node_id].ip @@ -247,8 +251,9 @@ def terminate_node(self, node_id: str) -> Optional[Dict[str, Any]]: self.scale_request.workers_to_delete.add(node_id) # Scale down all nodes in replica if node_id is part of a multi-host podslice - node_replica_index = self.node_data_dict[node_id].replica_index - if node_replica_index is not None: + tags = self.node_tags(node_id) + if TAG_RAY_REPLICA_INDEX in tags: + node_replica_index = tags[TAG_RAY_REPLICA_INDEX] for worker_id in self.replicas_to_nodes[node_replica_index]: # Check if worker has already been scheduled to delete if worker_id not in self.scale_request.workers_to_delete: diff --git a/python/ray/autoscaler/tags.py b/python/ray/autoscaler/tags.py index 380b4450d6eca..38d03855040fd 100644 --- a/python/ray/autoscaler/tags.py +++ b/python/ray/autoscaler/tags.py @@ -13,6 +13,8 @@ # Tag for user defined node types (e.g., m4xl_spot). This is used for multi # node type clusters. TAG_RAY_USER_NODE_TYPE = "ray-user-node-type" +# Tag for index of replica node belongs to. Used for multi-host worker groups. +TAG_RAY_REPLICA_INDEX = "ray-replica-index" # Tag for autofilled node types for legacy cluster yamls without multi # node type defined in the cluster configs. NODE_TYPE_LEGACY_HEAD = "ray-legacy-head-node-type" From ef7e0259948e5c3ecf7f479beae0e7ff15bd2f28 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Thu, 27 Jun 2024 03:10:21 +0000 Subject: [PATCH 38/45] BatchingNodeProvider autoscaling test Signed-off-by: Ryan O'Leary --- python/ray/tests/test_autoscaler.py | 118 ++++++++---------- .../tests/test_batch_node_provider_unit.py | 3 + 2 files changed, 56 insertions(+), 65 deletions(-) diff --git a/python/ray/tests/test_autoscaler.py b/python/ray/tests/test_autoscaler.py index d52415a2b02c8..cdc8a096806ea 100644 --- a/python/ray/tests/test_autoscaler.py +++ b/python/ray/tests/test_autoscaler.py @@ -63,8 +63,8 @@ TAG_RAY_CLUSTER_NAME, TAG_RAY_NODE_KIND, TAG_RAY_NODE_STATUS, - TAG_RAY_USER_NODE_TYPE, TAG_RAY_REPLICA_INDEX, + TAG_RAY_USER_NODE_TYPE, ) from ray.tests.test_batch_node_provider_unit import ( MockBatchingNodeProvider, @@ -318,14 +318,6 @@ def update_nodes(self): } TYPES_TPU = { - "empty_node": { - "node_config": { - "FooProperty": 42, - "TestProp": 1, - }, - "resources": {}, - "max_workers": 0, - }, "tpu-worker": { "node_config": {}, "resources": {"CPU": 1, "TPU": 4}, @@ -3606,22 +3598,23 @@ def testTerminateMultiHostReplica(self): config = copy.deepcopy(TPU_CLUSTER) config["available_node_types"]["tpu-worker"]["min_workers"] = 4 config["available_node_types"]["tpu-worker"]["max_workers"] = 4 + config["provider"][FOREGROUND_NODE_LAUNCH_KEY] = True + config["provider"][DISABLE_LAUNCH_CONFIG_CHECK_KEY] = True + config["provider"][DISABLE_NODE_UPDATERS_KEY] = True config_path = self.write_config(config) - self.provider = MockProvider() + self.provider = MockBatchingNodeProvider( + provider_config={ + DISABLE_LAUNCH_CONFIG_CHECK_KEY: True, + DISABLE_NODE_UPDATERS_KEY: True, + FOREGROUND_NODE_LAUNCH_KEY: True, + }, + cluster_name="test-cluster", + _allow_multiple=True, + ) runner = MockProcessRunner() runner.respond_to_call("json .Config.Env", ["[]" for i in range(5)]) lm = LoadMetrics() - get_or_create_head_node( - config, - printable_config_file=config_path, - no_restart=False, - restart_only=False, - yes=True, - override_cluster_name=None, - _provider=self.provider, - _runner=runner, - ) self.waitForNodes(1) autoscaler = MockAutoscaler( config_path, @@ -3633,56 +3626,51 @@ def testTerminateMultiHostReplica(self): process_runner=runner, update_interval_s=0, ) + self.provider.safe_to_scale_flag = True autoscaler.update() - self.waitForNodes(5) - assert autoscaler.pending_launches.value == 0 - assert ( - len( - self.provider.non_terminated_nodes( - {TAG_RAY_NODE_KIND: NODE_KIND_WORKER} - ) - ) - == 4 + # Scale up 4 Ray workers + assert self.num_nodes(tag_filters=WORKER_FILTER) == 4, ( + self.provider.non_terminated_nodes(tag_filters=WORKER_FILTER), + self.provider.non_terminated_nodes(tag_filters={}), ) - fill_in_raylet_ids(self.provider, lm) - autoscaler.update() - self.waitFor(lambda: autoscaler.pending_launches.value == 0) - self.waitForNodes(4, tag_filters={TAG_RAY_NODE_KIND: NODE_KIND_WORKER}) - assert autoscaler.pending_launches.value == 0 - # Set replicaIndex node tag + # Set replica_index in node_data for all workers + index = 0 for node_id in NonTerminatedNodes(self.provider).worker_ids: - self.provider.set_node_tags( - node_id, {TAG_RAY_REPLICA_INDEX: "tpu-group-0", **WORKER_FILTER} - ) - - # Manually delete one TPU worker - mock_logger = Mock(spec=logging.Logger("")) - worker_0 = NonTerminatedNodes(self.provider).worker_ids[0] - autoscaler.schedule_node_termination( - worker_0, "deleting TPU worker for test", mock_logger.info - ) - autoscaler.update() - events = autoscaler.event_summarizer.summary() - assert ( - "Removing 4 nodes of type tpu-worker (Node belongs to a replica being " - "deleted: tpu-group-0)." - ) in events - - # We should not be starting/stopping empty_node at all. - for event in events: - assert "empty_node" not in event - - # All nodes in the same replica index have been removed - assert ( - len( - self.provider.non_terminated_nodes( - {TAG_RAY_NODE_KIND: NODE_KIND_WORKER} - ) - ) - == 0 - ) + if index < 2: + self.provider.set_node_replica_index(node_id, "tpu-group-0") + else: + self.provider.set_node_replica_index(node_id, "tpu-group-1") + index += 1 + + replicaIndexFilter = {TAG_RAY_REPLICA_INDEX: "tpu-group-0"} + assert self.num_nodes(tag_filters=replicaIndexFilter) == 2, ( + self.provider.non_terminated_nodes(tag_filters=replicaIndexFilter), + self.provider.non_terminated_nodes(tag_filters={}), + ) + replicaIndexFilter[TAG_RAY_REPLICA_INDEX] = "tpu-group-1" + assert self.num_nodes(tag_filters=replicaIndexFilter) == 2, ( + self.provider.non_terminated_nodes(tag_filters=replicaIndexFilter), + self.provider.non_terminated_nodes(tag_filters={}), + ) + + # Verify replica_to_nodes mapping has been populated + assert len(self.provider.replicas_to_nodes["tpu-group-0"]) == 2 + assert len(self.provider.replicas_to_nodes["tpu-group-1"]) == 2 + + worker_0 = NonTerminatedNodes(self.provider).worker_ids[0] # tpu-group-0 + worker_2 = NonTerminatedNodes(self.provider).worker_ids[2] # tpu-group-1 + # Manually delete one TPU worker in tpu-group-0 + # BatchingNodeProvider should scale down all nodes in the replica + assert worker_0 in self.provider.node_data_dict + self.provider.terminate_node(worker_0) + assert len(self.provider.scale_request.workers_to_delete) == 2 + + # Scale down the tpu-group-1 replica + assert worker_2 in self.provider.node_data_dict + self.provider.terminate_node(worker_2) + assert len(self.provider.scale_request.workers_to_delete) == 4 def test_import(): diff --git a/python/ray/tests/test_batch_node_provider_unit.py b/python/ray/tests/test_batch_node_provider_unit.py index 55d075be53ad6..c5bb7a057d06e 100644 --- a/python/ray/tests/test_batch_node_provider_unit.py +++ b/python/ray/tests/test_batch_node_provider_unit.py @@ -58,6 +58,9 @@ def get_node_data(self) -> Dict[NodeID, NodeData]: self.num_non_terminated_nodes_calls += 1 return self._node_data_dict + def set_node_replica_index(self, node_id, replica_index): + self._node_data_dict[node_id].replica_index = replica_index + def submit_scale_request(self, scale_request: ScaleRequest) -> None: """Simulate modification of cluster state by an external cluster manager.""" self._scale_request_submitted_count += 1 From 776f8dff64875928a4e8dd054575738bdc904c3c Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Thu, 27 Jun 2024 03:12:37 +0000 Subject: [PATCH 39/45] Remove newline Signed-off-by: Ryan O'Leary --- python/ray/autoscaler/_private/autoscaler.py | 1 - 1 file changed, 1 deletion(-) diff --git a/python/ray/autoscaler/_private/autoscaler.py b/python/ray/autoscaler/_private/autoscaler.py index 0c08eb5b4a3b3..eec513cea9692 100644 --- a/python/ray/autoscaler/_private/autoscaler.py +++ b/python/ray/autoscaler/_private/autoscaler.py @@ -988,7 +988,6 @@ def _keep_worker_of_node_type( assert self.provider tags = self.provider.node_tags(node_id) - if TAG_RAY_USER_NODE_TYPE in tags: node_type = tags[TAG_RAY_USER_NODE_TYPE] From 76f092dd91afd2717131ca0a12ae8978d43d9f8e Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Thu, 27 Jun 2024 22:58:21 +0000 Subject: [PATCH 40/45] Mock BatchNodeProvider directly Signed-off-by: Ryan O'Leary --- .../ray/autoscaler/batching_node_provider.py | 2 +- python/ray/tests/test_autoscaler.py | 98 ------------------- .../tests/test_batch_node_provider_unit.py | 65 ++++++++++++ 3 files changed, 66 insertions(+), 99 deletions(-) diff --git a/python/ray/autoscaler/batching_node_provider.py b/python/ray/autoscaler/batching_node_provider.py index 0de016a88a90a..3d18ac9c2811a 100644 --- a/python/ray/autoscaler/batching_node_provider.py +++ b/python/ray/autoscaler/batching_node_provider.py @@ -259,7 +259,7 @@ def terminate_node(self, node_id: str) -> Optional[Dict[str, Any]]: if worker_id not in self.scale_request.workers_to_delete: self.scale_request.workers_to_delete.add(worker_id) logger.info( - f"Autoscaler terminating node {node_id}" + f"Autoscaler terminating node {node_id} " f"in multi-host replica {node_replica_index}." ) self.scale_change_needed = True diff --git a/python/ray/tests/test_autoscaler.py b/python/ray/tests/test_autoscaler.py index cdc8a096806ea..7870d804ead5b 100644 --- a/python/ray/tests/test_autoscaler.py +++ b/python/ray/tests/test_autoscaler.py @@ -63,7 +63,6 @@ TAG_RAY_CLUSTER_NAME, TAG_RAY_NODE_KIND, TAG_RAY_NODE_STATUS, - TAG_RAY_REPLICA_INDEX, TAG_RAY_USER_NODE_TYPE, ) from ray.tests.test_batch_node_provider_unit import ( @@ -317,22 +316,10 @@ def update_nodes(self): }, } -TYPES_TPU = { - "tpu-worker": { - "node_config": {}, - "resources": {"CPU": 1, "TPU": 4}, - "max_workers": 4, - }, -} - MULTI_WORKER_CLUSTER = dict( SMALL_CLUSTER, **{"available_node_types": TYPES_A, "head_node_type": "empty_node"} ) -TPU_CLUSTER = dict( - SMALL_CLUSTER, **{"available_node_types": TYPES_TPU, "head_node_type": "empty_node"} -) - exc_info = None try: raise Exception("Test exception.") @@ -3587,91 +3574,6 @@ def _test_autoscaler_status_log(self, status_log_enabled_env: int): break assert status_log_found is bool(status_log_enabled_env) - def testTerminateMultiHostReplica(self): - """Test multi-host replica deletion logic for KubeRay. - - Tests manually deleting a node in a multi-host replica - and verifying that the entire replica is scaled down. - Nodes belonging to the same multi-host replica are identified - through a replicaIndex label set by a GKE webhook. - """ - config = copy.deepcopy(TPU_CLUSTER) - config["available_node_types"]["tpu-worker"]["min_workers"] = 4 - config["available_node_types"]["tpu-worker"]["max_workers"] = 4 - config["provider"][FOREGROUND_NODE_LAUNCH_KEY] = True - config["provider"][DISABLE_LAUNCH_CONFIG_CHECK_KEY] = True - config["provider"][DISABLE_NODE_UPDATERS_KEY] = True - config_path = self.write_config(config) - self.provider = MockBatchingNodeProvider( - provider_config={ - DISABLE_LAUNCH_CONFIG_CHECK_KEY: True, - DISABLE_NODE_UPDATERS_KEY: True, - FOREGROUND_NODE_LAUNCH_KEY: True, - }, - cluster_name="test-cluster", - _allow_multiple=True, - ) - runner = MockProcessRunner() - runner.respond_to_call("json .Config.Env", ["[]" for i in range(5)]) - lm = LoadMetrics() - - self.waitForNodes(1) - autoscaler = MockAutoscaler( - config_path, - lm, - MockGcsClient(), - max_failures=0, - max_concurrent_launches=13, - max_launch_batch=13, - process_runner=runner, - update_interval_s=0, - ) - self.provider.safe_to_scale_flag = True - autoscaler.update() - # Scale up 4 Ray workers - assert self.num_nodes(tag_filters=WORKER_FILTER) == 4, ( - self.provider.non_terminated_nodes(tag_filters=WORKER_FILTER), - self.provider.non_terminated_nodes(tag_filters={}), - ) - fill_in_raylet_ids(self.provider, lm) - - # Set replica_index in node_data for all workers - index = 0 - for node_id in NonTerminatedNodes(self.provider).worker_ids: - if index < 2: - self.provider.set_node_replica_index(node_id, "tpu-group-0") - else: - self.provider.set_node_replica_index(node_id, "tpu-group-1") - index += 1 - - replicaIndexFilter = {TAG_RAY_REPLICA_INDEX: "tpu-group-0"} - assert self.num_nodes(tag_filters=replicaIndexFilter) == 2, ( - self.provider.non_terminated_nodes(tag_filters=replicaIndexFilter), - self.provider.non_terminated_nodes(tag_filters={}), - ) - replicaIndexFilter[TAG_RAY_REPLICA_INDEX] = "tpu-group-1" - assert self.num_nodes(tag_filters=replicaIndexFilter) == 2, ( - self.provider.non_terminated_nodes(tag_filters=replicaIndexFilter), - self.provider.non_terminated_nodes(tag_filters={}), - ) - - # Verify replica_to_nodes mapping has been populated - assert len(self.provider.replicas_to_nodes["tpu-group-0"]) == 2 - assert len(self.provider.replicas_to_nodes["tpu-group-1"]) == 2 - - worker_0 = NonTerminatedNodes(self.provider).worker_ids[0] # tpu-group-0 - worker_2 = NonTerminatedNodes(self.provider).worker_ids[2] # tpu-group-1 - # Manually delete one TPU worker in tpu-group-0 - # BatchingNodeProvider should scale down all nodes in the replica - assert worker_0 in self.provider.node_data_dict - self.provider.terminate_node(worker_0) - assert len(self.provider.scale_request.workers_to_delete) == 2 - - # Scale down the tpu-group-1 replica - assert worker_2 in self.provider.node_data_dict - self.provider.terminate_node(worker_2) - assert len(self.provider.scale_request.workers_to_delete) == 4 - def test_import(): """This test ensures that all the autoscaler imports work as expected to diff --git a/python/ray/tests/test_batch_node_provider_unit.py b/python/ray/tests/test_batch_node_provider_unit.py index c5bb7a057d06e..4f2b2bc0e2859 100644 --- a/python/ray/tests/test_batch_node_provider_unit.py +++ b/python/ray/tests/test_batch_node_provider_unit.py @@ -22,6 +22,7 @@ TAG_RAY_USER_NODE_TYPE, TAG_RAY_NODE_KIND, TAG_RAY_NODE_STATUS, + TAG_RAY_REPLICA_INDEX, NODE_KIND_HEAD, NODE_KIND_WORKER, ) @@ -418,6 +419,70 @@ def test_terminate_safeguards(): assert len(nodes) == 1 +@pytest.mark.skipif(sys.platform.startswith("win"), reason="Not relevant on Windows.") +def test_terminate_node_in_multihost_replica(): + """Test multi-host replica deletion logic for KubeRay. + + Tests manually deleting a node in a multi-host replica + and verifying that the entire replica is scaled down. + Nodes belonging to the same multi-host replica are identified + through a replicaIndex label set by a GKE webhook. + """ + # create 4 TPU nodes with MockBatchingNodeProvider + node_provider = MockBatchingNodeProvider( + provider_config={ + DISABLE_LAUNCH_CONFIG_CHECK_KEY: True, + DISABLE_NODE_UPDATERS_KEY: True, + FOREGROUND_NODE_LAUNCH_KEY: True, + }, + cluster_name="test-cluster", + _allow_multiple=True, + ) + + num_tpu_workers = 4 + for i in range(num_tpu_workers): + node_provider._add_node(node_type="TPU", node_kind=NODE_KIND_WORKER) + + # Set replica_index in node_data for all workers + workers = node_provider.non_terminated_nodes( + tag_filters={TAG_RAY_NODE_KIND: NODE_KIND_WORKER} + ) + assert len(workers) == num_tpu_workers + index = 0 + for node_id in workers: + if index < num_tpu_workers / 2: + node_provider.set_node_replica_index(node_id, "tpu-group-0") + else: + node_provider.set_node_replica_index(node_id, "tpu-group-1") + index += 1 + + # Verify RAY_REPLICA_INDEX tag has been set + replicaIndexFilter = {TAG_RAY_REPLICA_INDEX: "tpu-group-0"} + replicaWorkers1 = node_provider.non_terminated_nodes(tag_filters=replicaIndexFilter) + assert len(replicaWorkers1) == num_tpu_workers / 2 + + replicaIndexFilter[TAG_RAY_REPLICA_INDEX] = "tpu-group-1" + replicaWorkers2 = node_provider.non_terminated_nodes(tag_filters=replicaIndexFilter) + assert len(replicaWorkers2) == num_tpu_workers / 2 + + # Verify replica_to_nodes mapping has been populated + assert len(node_provider.replicas_to_nodes["tpu-group-0"]) == num_tpu_workers / 2 + assert len(node_provider.replicas_to_nodes["tpu-group-1"]) == num_tpu_workers / 2 + + worker_0 = replicaWorkers1[0] # tpu-group-0 + worker_2 = replicaWorkers2[0] # tpu-group-1 + # Manually delete one TPU worker in tpu-group-0 + # BatchingNodeProvider should scale down all nodes in the replica + assert worker_0 in node_provider.node_data_dict + node_provider.terminate_node(worker_0) + assert len(node_provider.scale_request.workers_to_delete) == num_tpu_workers / 2 + + # Scale down the tpu-group-1 replica + assert worker_2 in node_provider.node_data_dict + node_provider.terminate_node(worker_2) + assert len(node_provider.scale_request.workers_to_delete) == num_tpu_workers + + if __name__ == "__main__": if os.environ.get("PARALLEL_CI"): From f1037d10b4bd04f73e4d7cf63f15649077d6625a Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Fri, 28 Jun 2024 10:26:16 +0000 Subject: [PATCH 41/45] Update podlist2 with actual TPU pod yaml Signed-off-by: Ryan O'Leary --- .../tests/kuberay/test_files/podlist2.yaml | 210 +++++++++++++----- .../kuberay/test_kuberay_node_provider.py | 8 +- 2 files changed, 157 insertions(+), 61 deletions(-) diff --git a/python/ray/tests/kuberay/test_files/podlist2.yaml b/python/ray/tests/kuberay/test_files/podlist2.yaml index 270eea0515862..92528b27b0fcc 100644 --- a/python/ray/tests/kuberay/test_files/podlist2.yaml +++ b/python/ray/tests/kuberay/test_files/podlist2.yaml @@ -409,51 +409,85 @@ items: kind: Pod metadata: annotations: - key: value ray.io/ft-enabled: "false" - ray.io/health-state: "" - creationTimestamp: "2022-11-14T23:13:47Z" - generateName: raycluster-autoscaler-worker-fake-tpu-group- + creationTimestamp: "2024-06-28T10:11:15Z" + generateName: raycluster-autoscaler-worker-tpu-group- labels: app.kubernetes.io/created-by: kuberay-operator app.kubernetes.io/name: kuberay - key: value ray.io/cluster: raycluster-autoscaler - ray.io/cluster-dashboard: raycluster-autoscaler-dashboard - ray.io/group: fake-tpu-group + ray.io/group: tpu-group ray.io/identifier: raycluster-autoscaler-worker ray.io/is-ray-node: "yes" ray.io/node-type: worker - replicaIndex: fake-tpu-group-0 # GKE set label for multi-host podslices - name: raycluster-autoscaler-worker-fake-tpu-group-3rlvy + replicaIndex: tpu-group-0 + name: raycluster-autoscaler-worker-fake-tpu-group-xtpcl namespace: default ownerReferences: - - apiVersion: ray.io/v1alpha1 + - apiVersion: ray.io/v1 blockOwnerDeletion: true controller: true kind: RayCluster name: raycluster-autoscaler - uid: ec79effb-0295-4f40-b08b-8633aa7f786a - resourceVersion: "5805" - uid: f1cadfb8-b76f-467d-9c2b-a1a5eb7d1d3f + uid: eaac19a2-93e5-420e-98ce-9e47cf9f401f + resourceVersion: "13131412" + uid: a943c7f8-7e93-40c6-b676-9b4d7a0ac8c3 spec: + affinity: + podAffinity: + requiredDuringSchedulingIgnoredDuringExecution: + - labelSelector: + matchExpressions: + - key: replicaIndex + operator: In + values: + - tpu-group-0 + topologyKey: cloud.google.com/gke-nodepool containers: - args: - - 'ulimit -n 65536; ray start --address=raycluster-autoscaler-head-svc:6379 --metrics-export-port=8080 --num-cpus=1 --memory=536870912 --block --resources="{\"Custom1\": - 1, \"Custom2\": 5, \"TPU\": 4}" ' + - 'ulimit -n 65536; ray start --resources="{\"TPU\": 4}" --address=raycluster-autoscaler-head-svc.default.svc.cluster.local:6379 --metrics-export-port=8080 --block --dashboard-agent-listen-port=52365 --num-cpus=1 --memory=40000000000 ' command: - /bin/bash - - -c + - -lc - -- env: + - name: FQ_RAY_IP + value: raycluster-autoscaler-head-svc.default.svc.cluster.local - name: RAY_IP value: raycluster-autoscaler-head-svc + - name: RAY_CLUSTER_NAME + valueFrom: + fieldRef: + apiVersion: v1 + fieldPath: metadata.labels['ray.io/cluster'] + - name: RAY_CLOUD_INSTANCE_ID + valueFrom: + fieldRef: + apiVersion: v1 + fieldPath: metadata.name + - name: RAY_NODE_TYPE_NAME + valueFrom: + fieldRef: + apiVersion: v1 + fieldPath: metadata.labels['ray.io/group'] + - name: KUBERAY_GEN_RAY_START_CMD + value: 'ray start --resources="{\"TPU\": 4}" --address=raycluster-autoscaler-head-svc.default.svc.cluster.local:6379 --metrics-export-port=8080 --block --dashboard-agent-listen-port=52365 --num-cpus=1 --memory=40000000000 ' - name: RAY_PORT value: "6379" - name: RAY_ADDRESS - value: raycluster-autoscaler-head-svc:6379 + value: raycluster-autoscaler-head-svc.default.svc.cluster.local:6379 + - name: RAY_USAGE_STATS_KUBERAY_IN_USE + value: "1" - name: REDIS_PASSWORD - image: gekho/ray + - name: RAY_DASHBOARD_ENABLE_K8S_DISK_USAGE + value: "1" + - name: TPU_WORKER_HOSTNAMES + value: tpu-group-0-0.raycluster-autoscaler-headless-worker-svc,tpu-group-0-1.raycluster-autoscaler-headless-worker-svc + - name: TPU_WORKER_ID + value: "0" + - name: TPU_NAME + value: tpu-group-0 + image: rayproject/ray:2.9.0 imagePullPolicy: Always lifecycle: preStop: @@ -462,48 +496,99 @@ items: - /bin/sh - -c - ray stop - name: machine-learning + livenessProbe: + exec: + command: + - bash + - -c + - wget -T 2 -q -O- http://localhost:52365/api/local_raylet_healthz | grep + success + failureThreshold: 120 + initialDelaySeconds: 30 + periodSeconds: 5 + successThreshold: 1 + timeoutSeconds: 1 + name: ray-worker ports: - containerPort: 8080 name: metrics protocol: TCP + readinessProbe: + exec: + command: + - bash + - -c + - wget -T 2 -q -O- http://localhost:52365/api/local_raylet_healthz | grep + success + failureThreshold: 10 + initialDelaySeconds: 10 + periodSeconds: 5 + successThreshold: 1 + timeoutSeconds: 1 resources: limits: cpu: "1" - memory: 512Mi + ephemeral-storage: 10Gi + google.com/tpu: "4" + memory: 40G requests: - cpu: 500m - memory: 256Mi + cpu: "1" + ephemeral-storage: 10Gi + google.com/tpu: "4" + memory: 40G terminationMessagePath: /dev/termination-log terminationMessagePolicy: File volumeMounts: - mountPath: /dev/shm name: shared-mem - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access-ghm2j + name: kube-api-access-65x9l readOnly: true dnsPolicy: ClusterFirst enableServiceLinks: true + hostname: tpu-group-0-0 initContainers: - - command: - - sh - - -c - - until nslookup $RAY_IP.$(cat /var/run/secrets/kubernetes.io/serviceaccount/namespace).svc.cluster.local; - do echo waiting for myservice; sleep 2; done + - args: + - "\n\t\t\t\t\tSECONDS=0\n\t\t\t\t\twhile true; do\n\t\t\t\t\t\tif (( SECONDS + <= 120 )); then\n\t\t\t\t\t\t\tif ray health-check --address raycluster-autoscaler-head-svc.default.svc.cluster.local:6379 + > /dev/null 2>&1; then\n\t\t\t\t\t\t\t\techo \"GCS is ready.\"\n\t\t\t\t\t\t\t\tbreak\n\t\t\t\t\t\t\tfi\n\t\t\t\t\t\t\techo + \"$SECONDS seconds elapsed: Waiting for GCS to be ready.\"\n\t\t\t\t\t\telse\n\t\t\t\t\t\t\tif + ray health-check --address raycluster-autoscaler-head-svc.default.svc.cluster.local:6379; + then\n\t\t\t\t\t\t\t\techo \"GCS is ready. Any error messages above can be safely + ignored.\"\n\t\t\t\t\t\t\t\tbreak\n\t\t\t\t\t\t\tfi\n\t\t\t\t\t\t\techo \"$SECONDS + seconds elapsed: Still waiting for GCS to be ready. For troubleshooting, refer + to the FAQ at https://github.com/ray-project/kuberay/blob/master/docs/guidance/FAQ.md.\"\n\t\t\t\t\t\tfi\n\t\t\t\t\t\tsleep + 5\t\t\n\t\t\t\t\tdone\n\t\t\t\t" + command: + - /bin/bash + - -lc + - -- env: + - name: FQ_RAY_IP + value: raycluster-autoscaler-head-svc.default.svc.cluster.local - name: RAY_IP value: raycluster-autoscaler-head-svc - image: busybox:1.28 - imagePullPolicy: IfNotPresent - name: init-myservice - resources: {} + image: rayproject/ray:2.9.0 + imagePullPolicy: Always + name: wait-gcs-ready + resources: + limits: + cpu: 200m + memory: 256Mi + requests: + cpu: 200m + memory: 256Mi terminationMessagePath: /dev/termination-log terminationMessagePolicy: File volumeMounts: - mountPath: /var/run/secrets/kubernetes.io/serviceaccount - name: kube-api-access-ghm2j + name: kube-api-access-65x9l readOnly: true - nodeName: gke-cluster-1-default-pool-a5503908-dpst + nodeName: gke-tpu-0bf19815-10mj + nodeSelector: + cloud.google.com/gke-accelerator-count: "4" + cloud.google.com/gke-tpu-accelerator: tpu-v4-podslice + cloud.google.com/gke-tpu-topology: 2x2x2 preemptionPolicy: PreemptLowerPriority priority: 0 restartPolicy: Always @@ -511,6 +596,7 @@ items: securityContext: {} serviceAccount: default serviceAccountName: default + subdomain: raycluster-autoscaler-headless-worker-svc terminationGracePeriodSeconds: 30 tolerations: - effect: NoExecute @@ -521,12 +607,15 @@ items: key: node.kubernetes.io/unreachable operator: Exists tolerationSeconds: 300 + - effect: NoSchedule + key: google.com/tpu + operator: Exists volumes: - emptyDir: medium: Memory - sizeLimit: 256Mi + sizeLimit: 40G name: shared-mem - - name: kube-api-access-ghm2j + - name: kube-api-access-65x9l projected: defaultMode: 420 sources: @@ -547,55 +636,62 @@ items: status: conditions: - lastProbeTime: null - lastTransitionTime: "2022-11-14T23:13:49Z" + lastTransitionTime: "2024-06-28T10:11:48Z" + status: "True" + type: PodReadyToStartContainers + - lastProbeTime: null + lastTransitionTime: "2024-06-28T10:11:57Z" status: "True" type: Initialized - lastProbeTime: null - lastTransitionTime: "2022-11-14T23:13:50Z" + lastTransitionTime: "2024-06-28T10:12:07Z" status: "True" type: Ready - lastProbeTime: null - lastTransitionTime: "2022-11-14T23:13:50Z" + lastTransitionTime: "2024-06-28T10:12:07Z" status: "True" type: ContainersReady - lastProbeTime: null - lastTransitionTime: "2022-11-14T23:13:47Z" + lastTransitionTime: "2024-06-28T10:11:46Z" status: "True" type: PodScheduled containerStatuses: - - containerID: containerd://b6f4129ac590c62e41cc6db1a18fb051a29e3458fb47296086292a5765c36a6c - image: docker.io/gekho/ray:latest - imageID: docker.io/gekho/ray@sha256:7859a78d1a089bb88691864d5c4a2aad529f5353d7d9c82cc0274842fbda242b + - containerID: containerd://1e5d9cef5cb10636d44ef2ab6e557e71861f0960d05135df45d9af0c33a06d97 + image: docker.io/rayproject/ray:2.9.0 + imageID: docker.io/rayproject/ray@sha256:e64546fb5c3233bb0f33608e186e285c52cdd7440cae1af18f7fcde1c04e49f2 lastState: {} - name: machine-learning + name: ray-worker ready: true restartCount: 0 started: true state: running: - startedAt: "2022-11-14T23:13:49Z" - hostIP: 10.128.0.31 + startedAt: "2024-06-28T10:11:57Z" + hostIP: 10.0.0.57 + hostIPs: + - ip: 10.0.0.57 initContainerStatuses: - - containerID: containerd://5c946a3d443f76d21a3c72244edb4f80fc420b13b223f3c6981e25e689d381cf - image: docker.io/library/busybox:1.28 - imageID: docker.io/library/busybox@sha256:141c253bc4c3fd0a201d32dc1f493bcf3fff003b6df416dea4f41046e0f37d47 + - containerID: containerd://40257ec805418def64c50b7ce7b59e5eca79bc91754893beb9bde4d4042f819b + image: docker.io/rayproject/ray:2.9.0 + imageID: docker.io/rayproject/ray@sha256:e64546fb5c3233bb0f33608e186e285c52cdd7440cae1af18f7fcde1c04e49f2 lastState: {} - name: init-myservice + name: wait-gcs-ready ready: true restartCount: 0 + started: false state: terminated: - containerID: containerd://5c946a3d443f76d21a3c72244edb4f80fc420b13b223f3c6981e25e689d381cf + containerID: containerd://40257ec805418def64c50b7ce7b59e5eca79bc91754893beb9bde4d4042f819b exitCode: 0 - finishedAt: "2022-11-14T23:13:48Z" + finishedAt: "2024-06-28T10:11:56Z" reason: Completed - startedAt: "2022-11-14T23:13:48Z" + startedAt: "2024-06-28T10:11:47Z" phase: Running - podIP: 10.4.3.6 + podIP: 10.136.1.29 podIPs: - - ip: 10.4.3.6 - qosClass: Burstable - startTime: "2022-11-14T23:13:47Z" + - ip: 10.136.1.29 + qosClass: Guaranteed + startTime: "2024-06-28T10:11:46Z" - apiVersion: v1 kind: Pod metadata: diff --git a/python/ray/tests/kuberay/test_kuberay_node_provider.py b/python/ray/tests/kuberay/test_kuberay_node_provider.py index fe546214b2e03..0827494ee3d6a 100644 --- a/python/ray/tests/kuberay/test_kuberay_node_provider.py +++ b/python/ray/tests/kuberay/test_kuberay_node_provider.py @@ -161,11 +161,11 @@ def test_create_node_cap_at_max( ip="10.4.0.6", status="up-to-date", ), - "raycluster-autoscaler-worker-fake-tpu-group-3rlvy": NodeData( + "raycluster-autoscaler-worker-fake-tpu-group-xtpcl": NodeData( kind="worker", - type="fake-tpu-group", - replica_index="fake-tpu-group-0", - ip="10.4.3.6", + type="tpu-group", + replica_index="tpu-group-0", + ip="10.136.1.29", status="up-to-date", ), "raycluster-autoscaler-worker-small-group-dkz2r": NodeData( From 302b69a08849801a6c7f4e594c18cb7c880240b9 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Fri, 28 Jun 2024 20:16:13 +0000 Subject: [PATCH 42/45] Fix test comments Signed-off-by: Ryan O'Leary --- python/ray/tests/test_batch_node_provider_unit.py | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/python/ray/tests/test_batch_node_provider_unit.py b/python/ray/tests/test_batch_node_provider_unit.py index 4f2b2bc0e2859..b0423e8ec92e2 100644 --- a/python/ray/tests/test_batch_node_provider_unit.py +++ b/python/ray/tests/test_batch_node_provider_unit.py @@ -448,9 +448,8 @@ def test_terminate_node_in_multihost_replica(): tag_filters={TAG_RAY_NODE_KIND: NODE_KIND_WORKER} ) assert len(workers) == num_tpu_workers - index = 0 - for node_id in workers: - if index < num_tpu_workers / 2: + for index, node_id in enumerate(workers): + if index < num_tpu_workers // 2: node_provider.set_node_replica_index(node_id, "tpu-group-0") else: node_provider.set_node_replica_index(node_id, "tpu-group-1") @@ -459,15 +458,15 @@ def test_terminate_node_in_multihost_replica(): # Verify RAY_REPLICA_INDEX tag has been set replicaIndexFilter = {TAG_RAY_REPLICA_INDEX: "tpu-group-0"} replicaWorkers1 = node_provider.non_terminated_nodes(tag_filters=replicaIndexFilter) - assert len(replicaWorkers1) == num_tpu_workers / 2 + assert len(replicaWorkers1) == num_tpu_workers // 2 replicaIndexFilter[TAG_RAY_REPLICA_INDEX] = "tpu-group-1" replicaWorkers2 = node_provider.non_terminated_nodes(tag_filters=replicaIndexFilter) - assert len(replicaWorkers2) == num_tpu_workers / 2 + assert len(replicaWorkers2) == num_tpu_workers // 2 # Verify replica_to_nodes mapping has been populated - assert len(node_provider.replicas_to_nodes["tpu-group-0"]) == num_tpu_workers / 2 - assert len(node_provider.replicas_to_nodes["tpu-group-1"]) == num_tpu_workers / 2 + assert len(node_provider.replicas_to_nodes["tpu-group-0"]) == num_tpu_workers // 2 + assert len(node_provider.replicas_to_nodes["tpu-group-1"]) == num_tpu_workers // 2 worker_0 = replicaWorkers1[0] # tpu-group-0 worker_2 = replicaWorkers2[0] # tpu-group-1 @@ -475,7 +474,7 @@ def test_terminate_node_in_multihost_replica(): # BatchingNodeProvider should scale down all nodes in the replica assert worker_0 in node_provider.node_data_dict node_provider.terminate_node(worker_0) - assert len(node_provider.scale_request.workers_to_delete) == num_tpu_workers / 2 + assert len(node_provider.scale_request.workers_to_delete) == num_tpu_workers // 2 # Scale down the tpu-group-1 replica assert worker_2 in node_provider.node_data_dict From 14e8f797c8d256314ae15a1dd2cba1190af8d281 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Fri, 28 Jun 2024 23:45:08 +0000 Subject: [PATCH 43/45] Fix batching node provider log statement Signed-off-by: Ryan O'Leary --- python/ray/autoscaler/batching_node_provider.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/autoscaler/batching_node_provider.py b/python/ray/autoscaler/batching_node_provider.py index 3d18ac9c2811a..c9191be78a620 100644 --- a/python/ray/autoscaler/batching_node_provider.py +++ b/python/ray/autoscaler/batching_node_provider.py @@ -259,7 +259,7 @@ def terminate_node(self, node_id: str) -> Optional[Dict[str, Any]]: if worker_id not in self.scale_request.workers_to_delete: self.scale_request.workers_to_delete.add(worker_id) logger.info( - f"Autoscaler terminating node {node_id} " + f"Autoscaler terminating node {worker_id} " f"in multi-host replica {node_replica_index}." ) self.scale_change_needed = True From aa282218463c56fd1d9a842ead42df250a55c859 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Mon, 1 Jul 2024 17:16:07 +0000 Subject: [PATCH 44/45] Remove unused index increment Signed-off-by: Ryan O'Leary --- python/ray/tests/test_batch_node_provider_unit.py | 1 - 1 file changed, 1 deletion(-) diff --git a/python/ray/tests/test_batch_node_provider_unit.py b/python/ray/tests/test_batch_node_provider_unit.py index b0423e8ec92e2..734a445c394bd 100644 --- a/python/ray/tests/test_batch_node_provider_unit.py +++ b/python/ray/tests/test_batch_node_provider_unit.py @@ -453,7 +453,6 @@ def test_terminate_node_in_multihost_replica(): node_provider.set_node_replica_index(node_id, "tpu-group-0") else: node_provider.set_node_replica_index(node_id, "tpu-group-1") - index += 1 # Verify RAY_REPLICA_INDEX tag has been set replicaIndexFilter = {TAG_RAY_REPLICA_INDEX: "tpu-group-0"} From 2aa5a31379d88f540ea2e115fec9110291b36376 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Tue, 2 Jul 2024 00:47:24 +0000 Subject: [PATCH 45/45] Add type annotation and fix nits Signed-off-by: Ryan O'Leary --- python/ray/autoscaler/batching_node_provider.py | 8 ++++---- python/ray/tests/kuberay/test_kuberay_node_provider.py | 2 +- python/ray/tests/test_batch_node_provider_unit.py | 8 ++++++-- 3 files changed, 11 insertions(+), 7 deletions(-) diff --git a/python/ray/autoscaler/batching_node_provider.py b/python/ray/autoscaler/batching_node_provider.py index c9191be78a620..f3a26d085952d 100644 --- a/python/ray/autoscaler/batching_node_provider.py +++ b/python/ray/autoscaler/batching_node_provider.py @@ -121,7 +121,7 @@ def __init__( self.scale_request = ScaleRequest() # Initialize map of replica indices to nodes in that replica - self.replicas_to_nodes = defaultdict(list) + self.replica_index_to_nodes = defaultdict(list[str]) def get_node_data(self) -> Dict[NodeID, NodeData]: """Queries cluster manager for node info. Returns a mapping from node id to @@ -167,12 +167,12 @@ def non_terminated_nodes(self, tag_filters: Dict[str, str]) -> List[str]: workers_to_delete=set(), # No workers to delete yet ) all_nodes = list(self.node_data_dict.keys()) - self.replicas_to_nodes.clear() + self.replica_index_to_nodes.clear() for node_id in all_nodes: replica_index = self.node_data_dict[node_id].replica_index # Only add node to map if it belongs to a multi-host podslice if replica_index is not None: - self.replicas_to_nodes[replica_index].append(node_id) + self.replica_index_to_nodes[replica_index].append(node_id) # Support filtering by TAG_RAY_NODE_KIND, TAG_RAY_NODE_STATUS, and # TAG_RAY_USER_NODE_TYPE. # The autoscaler only uses tag_filters={}, @@ -254,7 +254,7 @@ def terminate_node(self, node_id: str) -> Optional[Dict[str, Any]]: tags = self.node_tags(node_id) if TAG_RAY_REPLICA_INDEX in tags: node_replica_index = tags[TAG_RAY_REPLICA_INDEX] - for worker_id in self.replicas_to_nodes[node_replica_index]: + for worker_id in self.replica_index_to_nodes[node_replica_index]: # Check if worker has already been scheduled to delete if worker_id not in self.scale_request.workers_to_delete: self.scale_request.workers_to_delete.add(worker_id) diff --git a/python/ray/tests/kuberay/test_kuberay_node_provider.py b/python/ray/tests/kuberay/test_kuberay_node_provider.py index 0827494ee3d6a..4b15a1c847315 100644 --- a/python/ray/tests/kuberay/test_kuberay_node_provider.py +++ b/python/ray/tests/kuberay/test_kuberay_node_provider.py @@ -203,7 +203,7 @@ def mock_get(node_provider, path): ), mock.patch.object(KubeRayNodeProvider, "_get", mock_get): kr_node_provider = KubeRayNodeProvider(provider_config={}, cluster_name="fake") kr_node_provider.cluster_name = "fake" - kr_node_provider.replicas_to_nodes = defaultdict(list) + kr_node_provider.replica_index_to_nodes = defaultdict(list[str]) nodes = kr_node_provider.non_terminated_nodes({}) assert kr_node_provider.node_data_dict == expected_node_data assert set(nodes) == set(expected_node_data.keys()) diff --git a/python/ray/tests/test_batch_node_provider_unit.py b/python/ray/tests/test_batch_node_provider_unit.py index 734a445c394bd..86d2b5389e57b 100644 --- a/python/ray/tests/test_batch_node_provider_unit.py +++ b/python/ray/tests/test_batch_node_provider_unit.py @@ -464,8 +464,12 @@ def test_terminate_node_in_multihost_replica(): assert len(replicaWorkers2) == num_tpu_workers // 2 # Verify replica_to_nodes mapping has been populated - assert len(node_provider.replicas_to_nodes["tpu-group-0"]) == num_tpu_workers // 2 - assert len(node_provider.replicas_to_nodes["tpu-group-1"]) == num_tpu_workers // 2 + assert ( + len(node_provider.replica_index_to_nodes["tpu-group-0"]) == num_tpu_workers // 2 + ) + assert ( + len(node_provider.replica_index_to_nodes["tpu-group-1"]) == num_tpu_workers // 2 + ) worker_0 = replicaWorkers1[0] # tpu-group-0 worker_2 = replicaWorkers2[0] # tpu-group-1