From ab6cd992bb7202979c719a0e86ca57d3365886db Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Tue, 14 Oct 2025 06:56:12 +0000 Subject: [PATCH 01/55] Add label_selector and bundle_label_selector to Serve API Signed-off-by: Ryan O'Leary --- python/ray/serve/_private/config.py | 14 ++++++++++++ python/ray/serve/api.py | 10 ++++++++- python/ray/serve/tests/conftest.py | 35 +++++++++++++++++++++++++++++ python/ray/serve/tests/test_api.py | 29 ++++++++++++++++++++++++ src/ray/protobuf/serve.proto | 1 + 5 files changed, 88 insertions(+), 1 deletion(-) diff --git a/python/ray/serve/_private/config.py b/python/ray/serve/_private/config.py index 2f642a385d64..25c8a17835c1 100644 --- a/python/ray/serve/_private/config.py +++ b/python/ray/serve/_private/config.py @@ -459,6 +459,7 @@ def __init__( ray_actor_options: Dict, placement_group_bundles: Optional[List[Dict[str, float]]] = None, placement_group_strategy: Optional[str] = None, + bundle_label_selector: Optional[List[Dict[str, str]]] = None, max_replicas_per_node: Optional[int] = None, needs_pickle: bool = True, ): @@ -484,6 +485,7 @@ def __init__( self.placement_group_bundles = placement_group_bundles self.placement_group_strategy = placement_group_strategy + self.bundle_label_selector = bundle_label_selector self.max_replicas_per_node = max_replicas_per_node @@ -514,12 +516,14 @@ def update( ray_actor_options: dict, placement_group_bundles: Optional[List[Dict[str, float]]] = None, placement_group_strategy: Optional[str] = None, + bundle_label_selector: Optional[List[Dict[str, str]]] = None, max_replicas_per_node: Optional[int] = None, ): self.ray_actor_options = ray_actor_options self.placement_group_bundles = placement_group_bundles self.placement_group_strategy = placement_group_strategy + self.bundle_label_selector = bundle_label_selector self.max_replicas_per_node = max_replicas_per_node @@ -536,6 +540,7 @@ def create( ray_actor_options: Optional[Dict] = None, placement_group_bundles: Optional[List[Dict[str, float]]] = None, placement_group_strategy: Optional[str] = None, + bundle_label_selector: Optional[List[Dict[str, str]]] = None, max_replicas_per_node: Optional[int] = None, deployment_def_name: Optional[str] = None, ): @@ -586,6 +591,7 @@ def create( ray_actor_options, placement_group_bundles, placement_group_strategy, + bundle_label_selector, max_replicas_per_node, ) @@ -612,6 +618,7 @@ def _validate_ray_actor_options(self): "resources", # Other options "runtime_env", + "label_selector", } for option in self.ray_actor_options: @@ -658,6 +665,7 @@ def _validate_placement_group_options(self) -> None: bundles=self.placement_group_bundles, strategy=self.placement_group_strategy or "PACK", lifetime="detached", + bundle_label_selector=self.bundle_label_selector, ) resource_error_prefix = ( @@ -763,6 +771,9 @@ def from_proto(cls, proto: ReplicaConfigProto, needs_pickle: bool = True): if proto.placement_group_strategy != "" else None, proto.max_replicas_per_node if proto.max_replicas_per_node else None, + json.loads(proto.bundle_label_selector) + if proto.bundle_label_selector + else None, needs_pickle, ) @@ -785,6 +796,9 @@ def to_proto(self): max_replicas_per_node=self.max_replicas_per_node if self.max_replicas_per_node is not None else 0, + bundle_label_selector=json.dumps(self.bundle_label_selector) + if self.bundle_label_selector is not None + else "", ) def to_proto_bytes(self): diff --git a/python/ray/serve/api.py b/python/ray/serve/api.py index 25fcaa7b76bc..50c1c9501793 100644 --- a/python/ray/serve/api.py +++ b/python/ray/serve/api.py @@ -339,6 +339,7 @@ def deployment( ray_actor_options: Default[Dict] = DEFAULT.VALUE, placement_group_bundles: Default[List[Dict[str, float]]] = DEFAULT.VALUE, placement_group_strategy: Default[str] = DEFAULT.VALUE, + bundle_label_selector: Default[List[Dict[str, str]]] = DEFAULT.VALUE, max_replicas_per_node: Default[int] = DEFAULT.VALUE, user_config: Default[Optional[Any]] = DEFAULT.VALUE, max_ongoing_requests: Default[int] = DEFAULT.VALUE, @@ -378,7 +379,7 @@ class MyDeployment: route_prefix: Route prefix for HTTP requests. Defaults to '/'. Deprecated. ray_actor_options: Options to pass to the Ray Actor decorator, such as resource requirements. Valid options are: `accelerator_type`, `memory`, - `num_cpus`, `num_gpus`, `resources`, and `runtime_env`. + `num_cpus`, `num_gpus`, `resources`, `runtime_env`, and `label_selector`. placement_group_bundles: Defines a set of placement group bundles to be scheduled *for each replica* of this deployment. The replica actor will be scheduled in the first bundle provided, so the resources specified in @@ -389,6 +390,8 @@ class MyDeployment: This cannot be set together with max_replicas_per_node. placement_group_strategy: Strategy to use for the replica placement group specified via `placement_group_bundles`. Defaults to `PACK`. + bundle_label_selector: A list of label selectors to apply to the + placement group on a per-bundle level. max_replicas_per_node: The max number of replicas of this deployment that can run on a single node. Valid values are None (default, no limit) or an integer in the range of [1, 100]. @@ -507,6 +510,11 @@ def decorator(_func_or_class): if placement_group_strategy is not DEFAULT.VALUE else None ), + bundle_label_selector=( + bundle_label_selector + if bundle_label_selector is not DEFAULT.VALUE + else None + ), max_replicas_per_node=( max_replicas_per_node if max_replicas_per_node is not DEFAULT.VALUE diff --git a/python/ray/serve/tests/conftest.py b/python/ray/serve/tests/conftest.py index 69e7392e4b91..e0e7587a2a25 100644 --- a/python/ray/serve/tests/conftest.py +++ b/python/ray/serve/tests/conftest.py @@ -344,3 +344,38 @@ def metrics_start_shutdown(request): serve.shutdown() ray.shutdown() reset_ray_address() + + +# Helper function to return the node ID of a remote worker. +@ray.remote(num_cpus=0) +def _get_node_id(): + return ray.get_runtime_context().get_node_id() + + +# Test fixture to start a Serve instance in a RayCluster with two labelled nodes +@pytest.fixture +def serve_instance_with_labeled_nodes(): + cluster = Cluster() + node1_config = { + "num_cpus": 1, + "resources": {"worker1": 1}, + "labels": {"region": "us-west", "gpu-type": "A100"}, + } + cluster.add_node(**node1_config) + ray.init(address=cluster.address) + node_1_id = ray.get(_get_node_id.options(resources={"worker1": 1}).remote()) + + node2_config = { + "num_cpus": 1, + "resources": {"worker2": 1}, + "labels": {"region": "us-east", "gpu-type": "H100"}, + } + cluster.add_node(**node2_config) + node_2_id = ray.get(_get_node_id.options(resources={"worker2": 1}).remote()) + + serve.start() + yield _get_global_client(), node_1_id, node_2_id + + serve.shutdown() + ray.shutdown() + cluster.shutdown() diff --git a/python/ray/serve/tests/test_api.py b/python/ray/serve/tests/test_api.py index ad6d8fe670ed..f9771c0e4e57 100644 --- a/python/ray/serve/tests/test_api.py +++ b/python/ray/serve/tests/test_api.py @@ -1219,6 +1219,35 @@ def __init__(self, counter): wait_for_condition(lambda: ray.get(counter.get_count.remote()) == 9) +async def test_serve_label_selector_api(serve_instance_with_labeled_nodes): + serve_instance, node_1_id, node_2_id = serve_instance_with_labeled_nodes + + # Test `label_selector` in `ray_actor_options` targets the desired node. + @serve.deployment(ray_actor_options={"label_selector": {"region": "us-west"}}) + class Deployment1: + def get_node_id(self): + return ray.get_runtime_context().get_node_id() + + handle1 = serve.run(Deployment1.bind(), name="app1") + replica_node_id_1 = await handle1.get_node_id.remote() + assert replica_node_id_1 == node_1_id + serve.delete("app1") + + # Test `bundle_label_selector` targets the desired node. + @serve.deployment( + placement_group_bundles=[{"CPU": 1}], + bundle_label_selector=[{"gpu-type": "H100"}], + ) + class Deployment2: + def get_node_id(self): + return ray.get_runtime_context().get_node_id() + + handle2 = serve.run(Deployment2.bind(), name="app2") + replica_node_id_2 = await handle2.get_node_id.remote() + assert replica_node_id_2 == node_2_id + serve.delete("app2") + + if __name__ == "__main__": import sys diff --git a/src/ray/protobuf/serve.proto b/src/ray/protobuf/serve.proto index 69eea2cbc6dc..4fd4d22255c0 100644 --- a/src/ray/protobuf/serve.proto +++ b/src/ray/protobuf/serve.proto @@ -255,6 +255,7 @@ message ReplicaConfig { string placement_group_bundles = 6; string placement_group_strategy = 7; int32 max_replicas_per_node = 8; + string bundle_label_selector = 9; } enum TargetCapacityDirection { From 0a449e131533bf5f42cc26619eb2bbe70d0c4f16 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Thu, 16 Oct 2025 04:10:59 +0000 Subject: [PATCH 02/55] Fix argument order Signed-off-by: Ryan O'Leary --- python/ray/serve/_private/config.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/python/ray/serve/_private/config.py b/python/ray/serve/_private/config.py index 25c8a17835c1..b8fc9e5c2cf3 100644 --- a/python/ray/serve/_private/config.py +++ b/python/ray/serve/_private/config.py @@ -770,10 +770,10 @@ def from_proto(cls, proto: ReplicaConfigProto, needs_pickle: bool = True): proto.placement_group_strategy if proto.placement_group_strategy != "" else None, - proto.max_replicas_per_node if proto.max_replicas_per_node else None, json.loads(proto.bundle_label_selector) if proto.bundle_label_selector else None, + proto.max_replicas_per_node if proto.max_replicas_per_node else None, needs_pickle, ) @@ -793,12 +793,12 @@ def to_proto(self): if self.placement_group_bundles is not None else "", placement_group_strategy=self.placement_group_strategy, - max_replicas_per_node=self.max_replicas_per_node - if self.max_replicas_per_node is not None - else 0, bundle_label_selector=json.dumps(self.bundle_label_selector) if self.bundle_label_selector is not None else "", + max_replicas_per_node=self.max_replicas_per_node + if self.max_replicas_per_node is not None + else 0, ) def to_proto_bytes(self): From a0c224608199e4ea133ec961d227b8a0b80486cb Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Thu, 18 Dec 2025 17:46:15 +0000 Subject: [PATCH 03/55] Add fallback strategy to serve options and consider labels during replica compaction Signed-off-by: ryanaoleary --- .../serve/_private/cluster_node_info_cache.py | 4 + python/ray/serve/_private/common.py | 2 + python/ray/serve/_private/config.py | 51 ++++- python/ray/serve/_private/default_impl.py | 1 + .../serve/_private/deployment_scheduler.py | 143 ++++++++++++- python/ray/serve/_private/deployment_state.py | 14 ++ python/ray/serve/_private/version.py | 34 +++ python/ray/serve/api.py | 6 + python/ray/serve/deployment.py | 10 + python/ray/serve/schema.py | 43 ++++ python/ray/serve/tests/conftest.py | 10 +- python/ray/serve/tests/test_api.py | 29 --- .../serve/tests/test_deployment_scheduler.py | 201 ++++++++++++++++++ .../serve/tests/test_deployment_version.py | 42 ++++ python/ray/serve/tests/test_standalone.py | 97 +++++++++ src/ray/protobuf/serve.proto | 1 + 16 files changed, 640 insertions(+), 48 deletions(-) diff --git a/python/ray/serve/_private/cluster_node_info_cache.py b/python/ray/serve/_private/cluster_node_info_cache.py index ae0121dd309d..ff6166190de7 100644 --- a/python/ray/serve/_private/cluster_node_info_cache.py +++ b/python/ray/serve/_private/cluster_node_info_cache.py @@ -89,6 +89,10 @@ def get_available_resources_per_node(self) -> Dict[str, Union[float, Dict]]: return self._cached_available_resources_per_node + def get_node_labels(self, node_id: str) -> Dict[str, str]: + """Get the labels for a specific node from the cache.""" + return self._cached_node_labels.get(node_id, {}) + class DefaultClusterNodeInfoCache(ClusterNodeInfoCache): def __init__(self, gcs_client: GcsClient): diff --git a/python/ray/serve/_private/common.py b/python/ray/serve/_private/common.py index 0d73a5cba64c..7768e7b9f2a0 100644 --- a/python/ray/serve/_private/common.py +++ b/python/ray/serve/_private/common.py @@ -782,6 +782,8 @@ class CreatePlacementGroupRequest: target_node_id: str name: str runtime_env: Optional[str] = None + bundle_label_selector: Optional[List[Dict[str, str]]] = None + fallback_strategy: Optional[List[Dict[str, Any]]] = None # This error is used to raise when a by-value DeploymentResponse is converted to an diff --git a/python/ray/serve/_private/config.py b/python/ray/serve/_private/config.py index 9c9a2167d065..626e89b07d87 100644 --- a/python/ray/serve/_private/config.py +++ b/python/ray/serve/_private/config.py @@ -477,6 +477,7 @@ def __init__( placement_group_bundles: Optional[List[Dict[str, float]]] = None, placement_group_strategy: Optional[str] = None, bundle_label_selector: Optional[List[Dict[str, str]]] = None, + fallback_strategy: Optional[List[Dict[str, Any]]] = None, max_replicas_per_node: Optional[int] = None, needs_pickle: bool = True, ): @@ -503,6 +504,7 @@ def __init__( self.placement_group_bundles = placement_group_bundles self.placement_group_strategy = placement_group_strategy self.bundle_label_selector = bundle_label_selector + self.fallback_strategy = fallback_strategy self.max_replicas_per_node = max_replicas_per_node @@ -534,6 +536,7 @@ def update( placement_group_bundles: Optional[List[Dict[str, float]]] = None, placement_group_strategy: Optional[str] = None, bundle_label_selector: Optional[List[Dict[str, str]]] = None, + fallback_strategy: Optional[List[Dict[str, Any]]] = None, max_replicas_per_node: Optional[int] = None, ): self.ray_actor_options = ray_actor_options @@ -541,6 +544,7 @@ def update( self.placement_group_bundles = placement_group_bundles self.placement_group_strategy = placement_group_strategy self.bundle_label_selector = bundle_label_selector + self.fallback_strategy = fallback_strategy self.max_replicas_per_node = max_replicas_per_node @@ -558,6 +562,7 @@ def create( placement_group_bundles: Optional[List[Dict[str, float]]] = None, placement_group_strategy: Optional[str] = None, bundle_label_selector: Optional[List[Dict[str, str]]] = None, + fallback_strategy: Optional[List[Dict[str, Any]]] = None, max_replicas_per_node: Optional[int] = None, deployment_def_name: Optional[str] = None, ): @@ -598,18 +603,23 @@ def create( deployment_def_name = deployment_def.__name__ config = cls( - deployment_def_name, - pickle_dumps( + deployment_def_name=deployment_def_name, + serialized_deployment_def=pickle_dumps( deployment_def, f"Could not serialize the deployment {repr(deployment_def)}", ), - pickle_dumps(init_args, "Could not serialize the deployment init args"), - pickle_dumps(init_kwargs, "Could not serialize the deployment init kwargs"), - ray_actor_options, - placement_group_bundles, - placement_group_strategy, - bundle_label_selector, - max_replicas_per_node, + serialized_init_args=pickle_dumps( + init_args, "Could not serialize the deployment init args" + ), + serialized_init_kwargs=pickle_dumps( + init_kwargs, "Could not serialize the deployment init kwargs" + ), + ray_actor_options=ray_actor_options, + placement_group_bundles=placement_group_bundles, + placement_group_strategy=placement_group_strategy, + bundle_label_selector=bundle_label_selector, + fallback_strategy=fallback_strategy, + max_replicas_per_node=max_replicas_per_node, ) config._deployment_def = deployment_def @@ -636,6 +646,7 @@ def _validate_ray_actor_options(self): # Other options "runtime_env", "label_selector", + "fallback_strategy", } for option in self.ray_actor_options: @@ -677,6 +688,24 @@ def _validate_placement_group_options(self) -> None: "`placement_group_bundles` must also be provided." ) + if self.fallback_strategy is not None: + if self.placement_group_bundles is None: + raise ValueError( + "If `fallback_strategy` is provided, " + "`placement_group_bundles` must also be provided." + ) + if not isinstance(self.fallback_strategy, list): + raise TypeError( + "fallback_strategy must be a list of dictionaries. " + f"Got: {type(self.fallback_strategy)}." + ) + for i, strategy in enumerate(self.fallback_strategy): + if not isinstance(strategy, dict): + raise TypeError( + f"fallback_strategy entry at index {i} must be a dictionary. " + f"Got: {type(strategy)}." + ) + if self.placement_group_bundles is not None: validate_placement_group( bundles=self.placement_group_bundles, @@ -790,6 +819,7 @@ def from_proto(cls, proto: ReplicaConfigProto, needs_pickle: bool = True): json.loads(proto.bundle_label_selector) if proto.bundle_label_selector else None, + json.loads(proto.fallback_strategy) if proto.fallback_strategy else None, proto.max_replicas_per_node if proto.max_replicas_per_node else None, needs_pickle, ) @@ -813,6 +843,9 @@ def to_proto(self): bundle_label_selector=json.dumps(self.bundle_label_selector) if self.bundle_label_selector is not None else "", + fallback_strategy=json.dumps(self.fallback_strategy) + if self.fallback_strategy is not None + else "", max_replicas_per_node=self.max_replicas_per_node if self.max_replicas_per_node is not None else 0, diff --git a/python/ray/serve/_private/default_impl.py b/python/ray/serve/_private/default_impl.py index 047960abcdf6..5ec7f49cdc2d 100644 --- a/python/ray/serve/_private/default_impl.py +++ b/python/ray/serve/_private/default_impl.py @@ -63,6 +63,7 @@ def _default_create_placement_group( _soft_target_node_id=request.target_node_id, name=request.name, lifetime="detached", + bundle_label_selector=request.bundle_label_selector, ) diff --git a/python/ray/serve/_private/deployment_scheduler.py b/python/ray/serve/_private/deployment_scheduler.py index 424a6dfea529..d0448cde76cf 100644 --- a/python/ray/serve/_private/deployment_scheduler.py +++ b/python/ray/serve/_private/deployment_scheduler.py @@ -24,7 +24,6 @@ from ray.util.scheduling_strategies import ( LabelMatchExpressionsT, NodeAffinitySchedulingStrategy, - NodeLabelSchedulingStrategy, PlacementGroupSchedulingStrategy, ) @@ -153,6 +152,8 @@ class ReplicaSchedulingRequest: # These are optional: by default replicas do not have a placement group. placement_group_bundles: Optional[List[Dict[str, float]]] = None placement_group_strategy: Optional[str] = None + placement_group_bundle_label_selector: Optional[List[Dict[str, str]]] = None + placement_group_fallback_strategy: Optional[List[Dict[str, Any]]] = None max_replicas_per_node: Optional[int] = None @property @@ -556,6 +557,7 @@ def _schedule_replica( deployment_id = replica_id.deployment_id placement_group = None + actor_options = copy.deepcopy(scheduling_request.actor_options) scheduling_strategy = default_scheduling_strategy if scheduling_request.placement_group_bundles is not None: placement_group_strategy = ( @@ -570,6 +572,7 @@ def _schedule_replica( strategy=placement_group_strategy, target_node_id=target_node_id, name=scheduling_request.actor_options["name"], + bundle_label_selector=scheduling_request.placement_group_bundle_label_selector, ) ) except Exception: @@ -594,12 +597,10 @@ def _schedule_replica( ) target_labels = None elif target_labels is not None: - scheduling_strategy = NodeLabelSchedulingStrategy( - hard={}, soft=target_labels - ) + actor_options["label_selector"] = target_labels + actor_options["fallback_strategy"] = [{"label_selector": {}}] target_node_id = None - actor_options = copy.deepcopy(scheduling_request.actor_options) if scheduling_request.max_replicas_per_node is not None: if "resources" not in actor_options: actor_options["resources"] = {} @@ -678,13 +679,72 @@ def schedule( reverse=True, ) + # Fetch node labels for active nodes. + active_nodes = self._cluster_node_info_cache.get_active_node_ids() + all_node_labels = { + node_id: self._cluster_node_info_cache.get_node_labels(node_id) + for node_id in active_nodes + } + # Schedule each replica for scheduling_request in all_scheduling_requests: - target_node = self._find_best_available_node( - scheduling_request.required_resources, - self._get_available_resources_per_node(), + # Collect a list of required resources and labels to try to schedule to + # support replica compaction when fallback strategies are provided. + strategies_to_try = [] + + primary_labels = [] + primary_bundles = scheduling_request.placement_group_bundles + + if primary_bundles: + # PG: Use PG bundle_label_selector + if scheduling_request.placement_group_bundle_label_selector: + primary_labels = ( + scheduling_request.placement_group_bundle_label_selector + ) + else: + # Actor: Use Actor label selector + if "label_selector" in scheduling_request.actor_options: + primary_labels = [ + scheduling_request.actor_options["label_selector"] + ] + + strategies_to_try.append( + (scheduling_request.required_resources, primary_labels) ) + if scheduling_request.placement_group_fallback_strategy: + # Fallback strategy provided for placement group. + for ( + fallback + ) in scheduling_request.placement_group_fallback_strategy: + fallback_bundles = fallback.get("bundles", primary_bundles) + req_resources = sum( + [Resources(b) for b in fallback_bundles], Resources() + ) + + fallback_labels = fallback.get("bundle_label_selector", []) + strategies_to_try.append((req_resources, fallback_labels)) + elif "fallback_strategy" in scheduling_request.actor_options: + # Fallback strategy provided for Ray Actor. + for fallback in scheduling_request.actor_options[ + "fallback_strategy" + ]: + fallback_labels = [fallback.get("label_selector", {})] + strategies_to_try.append( + (scheduling_request.required_resources, fallback_labels) + ) + + target_node = None + for res, labels in strategies_to_try: + target_node = self._find_best_available_node( + res, + self._get_available_resources_per_node(), + required_labels_list=labels, + node_labels=all_node_labels, + ) + if target_node: + break + self._schedule_replica( scheduling_request, default_scheduling_strategy="DEFAULT", @@ -772,10 +832,68 @@ def key(node_and_num_running_replicas_of_all_deployments): return replicas_to_stop + def _filter_nodes_by_labels( + self, + available_nodes: Dict[str, Resources], + required_labels: Dict[str, str], + node_labels: Dict[str, Dict[str, str]], + ) -> Dict[str, Resources]: + """Filters available nodes based on label selector constraints. + + Supports Ray's label syntax where values are strings: + - Equality: {"key": "value"} + - Not Equal: {"key": "!value"} + - In: {"key": "in(v1, v2, ...)"} + - Not In: {"key": "!in(v1, v2, ...)"} + """ + filtered_nodes = {} + for node_id, resources in available_nodes.items(): + labels = node_labels.get(node_id, {}) + is_match = True + + for key, req_val in required_labels.items(): + node_val = labels.get(key) + + # !in operator + if req_val.startswith("!in(") and req_val.endswith(")"): + content = req_val[4:-1] + values = [v.strip() for v in content.split(",")] + if node_val is not None and node_val in values: + is_match = False + break + + # in operator + elif req_val.startswith("in(") and req_val.endswith(")"): + content = req_val[3:-1] + values = [v.strip() for v in content.split(",")] + if node_val not in values: + is_match = False + break + + # not equal operator + elif req_val.startswith("!"): + target_val = req_val[1:] + if node_val == target_val: + is_match = False + break + + # equals operator + else: + if node_val != req_val: + is_match = False + break + + if is_match: + filtered_nodes[node_id] = resources + + return filtered_nodes + def _find_best_available_node( self, required_resources: Resources, available_resources_per_node: Dict[str, Resources], + required_labels_list: Optional[List[Dict[str, str]]] = None, + node_labels: Optional[Dict[str, Dict[str, str]]] = None, ) -> Optional[str]: """Chooses best available node to schedule the required resources. @@ -784,6 +902,15 @@ def _find_best_available_node( over idle nodes. """ + # Filter feasible nodes by provided label selectors if provided. + if required_labels_list and node_labels: + for required_labels in required_labels_list: + available_resources_per_node = self._filter_nodes_by_labels( + available_resources_per_node, required_labels, node_labels + ) + if not available_resources_per_node: + return None + node_to_running_replicas = self._get_node_to_running_replicas() non_idle_nodes = { diff --git a/python/ray/serve/_private/deployment_state.py b/python/ray/serve/_private/deployment_state.py index 7c5e27e4e164..37e4769fa729 100644 --- a/python/ray/serve/_private/deployment_state.py +++ b/python/ray/serve/_private/deployment_state.py @@ -134,6 +134,10 @@ def create( placement_group_strategy=info.replica_config.placement_group_strategy, max_replicas_per_node=info.replica_config.max_replicas_per_node, route_prefix=info.route_prefix, + placement_group_bundle_label_selector=( + info.replica_config.bundle_label_selector + ), + placement_group_fallback_strategy=(info.replica_config.fallback_strategy), ) return cls(info, target_num_replicas, version, deleting) @@ -159,6 +163,10 @@ def is_scaled_copy_of(self, other_target_state: "DeploymentTargetState") -> bool == other_target_state.info.replica_config.placement_group_bundles, self.info.replica_config.placement_group_strategy == other_target_state.info.replica_config.placement_group_strategy, + self.info.replica_config.bundle_label_selector + == other_target_state.info.replica_config.bundle_label_selector, + self.info.replica_config.fallback_strategy + == other_target_state.info.replica_config.fallback_strategy, self.info.replica_config.max_replicas_per_node == other_target_state.info.replica_config.max_replicas_per_node, self.info.deployment_config.dict(exclude={"num_replicas"}) @@ -559,6 +567,12 @@ def start( placement_group_strategy=( deployment_info.replica_config.placement_group_strategy ), + placement_group_bundle_label_selector=( + deployment_info.replica_config.bundle_label_selector + ), + placement_group_fallback_strategy=( + deployment_info.replica_config.fallback_strategy + ), max_replicas_per_node=( deployment_info.replica_config.max_replicas_per_node ), diff --git a/python/ray/serve/_private/version.py b/python/ray/serve/_private/version.py index 1c064a9a9dc7..20c326bfcd7c 100644 --- a/python/ray/serve/_private/version.py +++ b/python/ray/serve/_private/version.py @@ -21,6 +21,8 @@ def __init__( ray_actor_options: Optional[Dict], placement_group_bundles: Optional[List[Dict[str, float]]] = None, placement_group_strategy: Optional[str] = None, + placement_group_bundle_label_selector: Optional[List[Dict[str, str]]] = None, + placement_group_fallback_strategy: Optional[List[Dict[str, Any]]] = None, max_replicas_per_node: Optional[int] = None, route_prefix: Optional[str] = None, ): @@ -37,6 +39,10 @@ def __init__( self.ray_actor_options = ray_actor_options self.placement_group_bundles = placement_group_bundles self.placement_group_strategy = placement_group_strategy + self.placement_group_bundle_label_selector = ( + placement_group_bundle_label_selector + ) + self.placement_group_fallback_strategy = placement_group_fallback_strategy self.max_replicas_per_node = max_replicas_per_node self.route_prefix = route_prefix self.compute_hashes() @@ -96,6 +102,14 @@ def compute_hashes(self): combined_placement_group_options["bundles"] = self.placement_group_bundles if self.placement_group_strategy is not None: combined_placement_group_options["strategy"] = self.placement_group_strategy + if self.placement_group_bundle_label_selector is not None: + combined_placement_group_options[ + "bundle_label_selector" + ] = self.placement_group_bundle_label_selector + if self.placement_group_fallback_strategy is not None: + combined_placement_group_options[ + "fallback_strategy" + ] = self.placement_group_fallback_strategy serialized_placement_group_options = _serialize( combined_placement_group_options ) @@ -141,6 +155,16 @@ def to_proto(self) -> bytes: placement_group_strategy=self.placement_group_strategy if self.placement_group_strategy is not None else "", + placement_group_bundle_label_selector=json.dumps( + self.placement_group_bundle_label_selector + ) + if self.placement_group_bundle_label_selector is not None + else "", + placement_group_fallback_strategy=json.dumps( + self.placement_group_fallback_strategy + ) + if self.placement_group_fallback_strategy is not None + else "", max_replicas_per_node=self.max_replicas_per_node if self.max_replicas_per_node is not None else 0, @@ -157,6 +181,16 @@ def from_proto(cls, proto: DeploymentVersionProto): if proto.placement_group_bundles else None ), + placement_group_bundle_label_selector=( + json.loads(proto.placement_group_bundle_label_selector) + if proto.placement_group_bundle_label_selector + else None + ), + placement_group_fallback_strategy=( + json.loads(proto.placement_group_fallback_strategy) + if proto.placement_group_fallback_strategy + else None + ), placement_group_version=( proto.placement_group_version if proto.placement_group_version else None ), diff --git a/python/ray/serve/api.py b/python/ray/serve/api.py index 10ae615e2fab..8e107971b492 100644 --- a/python/ray/serve/api.py +++ b/python/ray/serve/api.py @@ -327,6 +327,7 @@ def deployment( placement_group_bundles: Default[List[Dict[str, float]]] = DEFAULT.VALUE, placement_group_strategy: Default[str] = DEFAULT.VALUE, bundle_label_selector: Default[List[Dict[str, str]]] = DEFAULT.VALUE, + fallback_strategy: Default[List[Dict[str, Any]]] = DEFAULT.VALUE, max_replicas_per_node: Default[int] = DEFAULT.VALUE, user_config: Default[Optional[Any]] = DEFAULT.VALUE, max_ongoing_requests: Default[int] = DEFAULT.VALUE, @@ -379,6 +380,8 @@ class MyDeployment: specified via `placement_group_bundles`. Defaults to `PACK`. bundle_label_selector: A list of label selectors to apply to the placement group on a per-bundle level. + fallback_strategy: If specified, expresses soft constraints through a list + of decorator options to fall back on when scheduling on a node. max_replicas_per_node: The max number of replicas of this deployment that can run on a single node. Valid values are None (default, no limit) or an integer in the range of [1, 100]. @@ -502,6 +505,9 @@ def decorator(_func_or_class): if bundle_label_selector is not DEFAULT.VALUE else None ), + fallback_strategy=( + fallback_strategy if fallback_strategy is not DEFAULT.VALUE else None + ), max_replicas_per_node=( max_replicas_per_node if max_replicas_per_node is not DEFAULT.VALUE diff --git a/python/ray/serve/deployment.py b/python/ray/serve/deployment.py index 31505c8ef70d..f54798f194eb 100644 --- a/python/ray/serve/deployment.py +++ b/python/ray/serve/deployment.py @@ -215,6 +215,8 @@ def options( ray_actor_options: Default[Optional[Dict]] = DEFAULT.VALUE, placement_group_bundles: Default[List[Dict[str, float]]] = DEFAULT.VALUE, placement_group_strategy: Default[str] = DEFAULT.VALUE, + bundle_label_selector: Default[List[Dict[str, str]]] = DEFAULT.VALUE, + fallback_strategy: Default[List[Dict[str, Any]]] = DEFAULT.VALUE, max_replicas_per_node: Default[int] = DEFAULT.VALUE, user_config: Default[Optional[Any]] = DEFAULT.VALUE, max_ongoing_requests: Default[int] = DEFAULT.VALUE, @@ -341,6 +343,12 @@ def options( if placement_group_strategy is DEFAULT.VALUE: placement_group_strategy = self._replica_config.placement_group_strategy + if bundle_label_selector is DEFAULT.VALUE: + bundle_label_selector = self._replica_config.bundle_label_selector + + if fallback_strategy is DEFAULT.VALUE: + fallback_strategy = self._replica_config.fallback_strategy + if max_replicas_per_node is DEFAULT.VALUE: max_replicas_per_node = self._replica_config.max_replicas_per_node @@ -378,6 +386,8 @@ def options( ray_actor_options=ray_actor_options, placement_group_bundles=placement_group_bundles, placement_group_strategy=placement_group_strategy, + bundle_label_selector=bundle_label_selector, + fallback_strategy=fallback_strategy, max_replicas_per_node=max_replicas_per_node, ) diff --git a/python/ray/serve/schema.py b/python/ray/serve/schema.py index 9d5d9176259d..7315c37428a2 100644 --- a/python/ray/serve/schema.py +++ b/python/ray/serve/schema.py @@ -254,6 +254,19 @@ class RayActorOptionsSchema(BaseModel): "See :ref:`accelerator types `." ), ) + label_selector: Dict[str, str] = Field( + default=None, + description=( + "If specified, requires that the actor run on a node with the specified labels." + ), + ) + fallback_strategy: Union[str, List[Dict[str, Any]]] = Field( + default=None, + description=( + "If specified, expresses soft constraints through a list of decorator " + "options to fall back on when scheduling on a node." + ), + ) @validator("runtime_env") def runtime_env_contains_remote_uris(cls, v): @@ -395,6 +408,14 @@ class DeploymentSchema(BaseModel, allow_population_by_field_name=True): ), ) + bundle_label_selector: List[Dict[str, str]] = Field( + default=DEFAULT.VALUE, + description=( + "A list of label selectors to apply to the placement group " + "on a per-bundle level." + ), + ) + max_replicas_per_node: int = Field( default=DEFAULT.VALUE, description=( @@ -450,6 +471,28 @@ def validate_max_replicas_per_node_and_placement_group_bundles(cls, values): return values + @root_validator + def validate_bundle_label_selector(cls, values): + placement_group_bundles = values.get("placement_group_bundles", None) + bundle_label_selector = values.get("bundle_label_selector", None) + + if bundle_label_selector not in [DEFAULT.VALUE, None]: + if placement_group_bundles in [DEFAULT.VALUE, None]: + raise ValueError( + "Setting bundle_label_selector is not allowed when " + "placement_group_bundles is not provided." + ) + + if len(bundle_label_selector) != len(placement_group_bundles): + raise ValueError( + f"If bundle_label_selector is provided, it must have the same " + f"length as placement_group_bundles. Got " + f"{len(bundle_label_selector)} label selectors and " + f"{len(placement_group_bundles)} bundles." + ) + + return values + @root_validator def validate_max_queued_requests(cls, values): max_queued_requests = values.get("max_queued_requests", None) diff --git a/python/ray/serve/tests/conftest.py b/python/ray/serve/tests/conftest.py index a19ad296102c..1724e7ba1b31 100644 --- a/python/ray/serve/tests/conftest.py +++ b/python/ray/serve/tests/conftest.py @@ -353,10 +353,16 @@ def _get_node_id(): return ray.get_runtime_context().get_node_id() -# Test fixture to start a Serve instance in a RayCluster with two labelled nodes +# Test fixture to start a Serve instance in a RayCluster with two labeled nodes @pytest.fixture def serve_instance_with_labeled_nodes(): cluster = Cluster() + # Unlabeled default node. + node0_config = { + "num_cpus": 1, + "resources": {"worker0": 1}, + } + cluster.add_node(**node0_config) node1_config = { "num_cpus": 1, "resources": {"worker1": 1}, @@ -367,7 +373,7 @@ def serve_instance_with_labeled_nodes(): node_1_id = ray.get(_get_node_id.options(resources={"worker1": 1}).remote()) node2_config = { - "num_cpus": 1, + "num_cpus": 2, "resources": {"worker2": 1}, "labels": {"region": "us-east", "gpu-type": "H100"}, } diff --git a/python/ray/serve/tests/test_api.py b/python/ray/serve/tests/test_api.py index 39b64bee9250..80955eeff12f 100644 --- a/python/ray/serve/tests/test_api.py +++ b/python/ray/serve/tests/test_api.py @@ -1219,35 +1219,6 @@ def __init__(self, counter): wait_for_condition(lambda: ray.get(counter.get_count.remote()) == 9) -async def test_serve_label_selector_api(serve_instance_with_labeled_nodes): - serve_instance, node_1_id, node_2_id = serve_instance_with_labeled_nodes - - # Test `label_selector` in `ray_actor_options` targets the desired node. - @serve.deployment(ray_actor_options={"label_selector": {"region": "us-west"}}) - class Deployment1: - def get_node_id(self): - return ray.get_runtime_context().get_node_id() - - handle1 = serve.run(Deployment1.bind(), name="app1") - replica_node_id_1 = await handle1.get_node_id.remote() - assert replica_node_id_1 == node_1_id - serve.delete("app1") - - # Test `bundle_label_selector` targets the desired node. - @serve.deployment( - placement_group_bundles=[{"CPU": 1}], - bundle_label_selector=[{"gpu-type": "H100"}], - ) - class Deployment2: - def get_node_id(self): - return ray.get_runtime_context().get_node_id() - - handle2 = serve.run(Deployment2.bind(), name="app2") - replica_node_id_2 = await handle2.get_node_id.remote() - assert replica_node_id_2 == node_2_id - serve.delete("app2") - - if __name__ == "__main__": import sys diff --git a/python/ray/serve/tests/test_deployment_scheduler.py b/python/ray/serve/tests/test_deployment_scheduler.py index b22cd5dfd79b..f53be16919f8 100644 --- a/python/ray/serve/tests/test_deployment_scheduler.py +++ b/python/ray/serve/tests/test_deployment_scheduler.py @@ -1,4 +1,5 @@ import sys +from unittest import mock import pytest @@ -11,6 +12,7 @@ from ray.serve._private.constants import RAY_SERVE_USE_COMPACT_SCHEDULING_STRATEGY from ray.serve._private.deployment_scheduler import ( ReplicaSchedulingRequest, + Resources, SpreadDeploymentSchedulingPolicy, ) from ray.serve._private.test_utils import check_apps_running, get_node_id @@ -286,5 +288,204 @@ def test_e2e_custom_resources(self, ray_cluster, use_pg): serve.shutdown() +class TestSchedulerUnit: + def test_schedule_passes_placement_group_options(self): + """Test that bundle_label_selector is passed to CreatePlacementGroupRequest.""" + cluster_node_info_cache = default_impl.create_cluster_node_info_cache( + GcsClient(address=ray.get_runtime_context().gcs_address) + ) + + captured_requests = [] + + def mock_create_pg(request): + captured_requests.append(request) + + class MockPG: + def wait(self, *args): + return True + + return MockPG() + + scheduler = default_impl.create_deployment_scheduler( + cluster_node_info_cache, + get_head_node_id(), + create_placement_group_fn_override=mock_create_pg, + ) + + dep_id = DeploymentID(name="pg_options_test") + # Use Spread policy here, but the logic is shared across policies. + scheduler.on_deployment_created(dep_id, SpreadDeploymentSchedulingPolicy()) + + test_labels = [{"region": "us-west"}] + # Create a request with the new options + req = ReplicaSchedulingRequest( + replica_id=ReplicaID("r1", dep_id), + actor_def=Replica, + actor_resources={"CPU": 1}, + actor_options={"name": "r1"}, + actor_init_args=(), + on_scheduled=lambda *args: None, + placement_group_bundles=[{"CPU": 1}], + placement_group_bundle_label_selector=test_labels, + ) + + scheduler.schedule(upscales={dep_id: [req]}, downscales={}) + + # Verify the PlacementGroupSchedulingRequest is created. + assert len(captured_requests) == 1 + pg_request = captured_requests[0] + + # bundle_label_selector should be passed to request. + assert pg_request.bundle_label_selector == test_labels + + def test_filter_nodes_by_labels(self): + """Test _filter_nodes_by_labels logic used by _find_best_available_node + when bin-packing, such that label constraints are enforced for the preferred node.""" + + class MockScheduler(default_impl.DefaultDeploymentScheduler): + def __init__(self): + pass + + scheduler = MockScheduler() + + nodes = { + "n1": Resources(), + "n2": Resources(), + "n3": Resources(), + } + node_labels = { + "n1": {"region": "us-west", "gpu": "T4", "env": "prod"}, + "n2": {"region": "us-east", "gpu": "A100", "env": "dev"}, + "n3": {"region": "me-central", "env": "staging"}, # No GPU label + } + + # equals operator + filtered = scheduler._filter_nodes_by_labels( + nodes, {"region": "us-west"}, node_labels + ) + assert set(filtered.keys()) == {"n1"} + + # not equals operator + filtered = scheduler._filter_nodes_by_labels( + nodes, {"region": "!us-west"}, node_labels + ) + assert set(filtered.keys()) == {"n2", "n3"} + + # in operator + filtered = scheduler._filter_nodes_by_labels( + nodes, {"region": "in(us-west, us-east)"}, node_labels + ) + assert set(filtered.keys()) == {"n1", "n2"} + + # !in operator + filtered = scheduler._filter_nodes_by_labels( + nodes, {"env": "!in(dev, staging)"}, node_labels + ) + assert set(filtered.keys()) == {"n1"} + + # Missing labels treated as not a match for equality. + filtered = scheduler._filter_nodes_by_labels( + nodes, {"gpu": "A100"}, node_labels + ) + assert set(filtered.keys()) == {"n2"} + + # Not equal should match node with missing labels. + filtered = scheduler._filter_nodes_by_labels(nodes, {"gpu": "!T4"}, node_labels) + assert set(filtered.keys()) == {"n2", "n3"} + + # Validate we handle whitespace. + filtered = scheduler._filter_nodes_by_labels( + nodes, {"region": "in( us-west , us-east )"}, node_labels + ) + assert set(filtered.keys()) == {"n1", "n2"} + + @mock.patch( + "ray.serve._private.deployment_scheduler.RAY_SERVE_USE_COMPACT_SCHEDULING_STRATEGY", + True, + ) + def test_compact_schedule_respects_labels_and_fallback(self): + """Test that scheduler respects labels and tries fallback strategies + when selecting the best node for bin-packing.""" + + N1_ID = "1" * 56 + N2_ID = "2" * 56 + N3_ID = "3" * 56 + + # Mock some active nodes with varying resources and labels. + class MockActor: + def options(self, **kwargs): + return self + + def remote(self, *args, **kwargs): + return "mock_handle" + + class MockCache: + def get_active_node_ids(self): + return {N1_ID, N2_ID, N3_ID} + + def get_node_labels(self, node_id): + return { + N1_ID: {"region": "us-west"}, + N2_ID: {"region": "us-east"}, + N3_ID: {"region": "eu-central"}, + }[node_id] + + def get_available_resources_per_node(self): + return { + N1_ID: {"CPU": 2}, + N2_ID: {"CPU": 10}, + N3_ID: {"CPU": 4}, + } + + def get_total_resources_per_node(self): + return self.get_available_resources_per_node() + + scheduler = default_impl.create_deployment_scheduler( + MockCache(), "head_node_id" + ) + + # Validate the label_selector constraint is respected. + # For a request of 1 CPU and "us-west", we pick the node that matches the label + # (n2) even though a better fit exists (n1). + dep_id = DeploymentID(name="test_labels") + scheduler.on_deployment_created(dep_id, SpreadDeploymentSchedulingPolicy()) + scheduler._deployments[dep_id].actor_resources = Resources({"CPU": 1}) + + req = ReplicaSchedulingRequest( + replica_id=ReplicaID("r1", dep_id), + actor_def=Replica, + actor_resources={"CPU": 1}, + actor_options={"label_selector": {"region": "us-west"}, "name": "r1"}, + actor_init_args=(), + on_scheduled=lambda *args, **kwargs: None, + ) + + scheduler.schedule(upscales={dep_id: [req]}, downscales={}) + launch_info = scheduler._launching_replicas[dep_id][req.replica_id] + assert launch_info.target_node_id == N1_ID + + # Validate fallback strategy is used when selecting a node. If the + # label constraints and resources of the primary strategy are infeasible, + # we try each fallback strategy in-order. + req_fallback = ReplicaSchedulingRequest( + replica_id=ReplicaID("r2", dep_id), + actor_def=Replica, + actor_resources={"CPU": 1}, + actor_options={ + "label_selector": {"region": "us-north"}, # Invalid + "fallback_strategy": [ + {"label_selector": {"region": "eu-central"}} + ], # Valid (n3) + "name": "r2", + }, + actor_init_args=(), + on_scheduled=lambda *args, **kwargs: None, + ) + + scheduler.schedule(upscales={dep_id: [req_fallback]}, downscales={}) + launch_info = scheduler._launching_replicas[dep_id][req_fallback.replica_id] + assert launch_info.target_node_id == N3_ID + + if __name__ == "__main__": sys.exit(pytest.main(["-v", "-s", __file__])) diff --git a/python/ray/serve/tests/test_deployment_version.py b/python/ray/serve/tests/test_deployment_version.py index b1202aee3134..7fabdbef2c5d 100644 --- a/python/ray/serve/tests/test_deployment_version.py +++ b/python/ray/serve/tests/test_deployment_version.py @@ -38,6 +38,48 @@ def test_route_prefix_changes_trigger_reconfigure_hash(): assert v1.requires_actor_reconfigure(v2) +def test_placement_group_options_trigger_restart(): + """Test that changing a PG label selector or fallback strategy triggers an actor restart.""" + cfg = DeploymentConfig() + + # Initial deployment. + v1 = DeploymentVersion( + code_version="1", + deployment_config=cfg, + ray_actor_options={}, + placement_group_bundles=[{"CPU": 1}], + placement_group_strategy="PACK", + ) + + # Change bundle_label_selector. + v2 = DeploymentVersion( + code_version="1", + deployment_config=cfg, + ray_actor_options={}, + placement_group_bundles=[{"CPU": 1}], + placement_group_strategy="PACK", + placement_group_bundle_label_selector=[{"region": "us-west"}], + ) + + # Validate actor restart occurs due to differing hash. + assert v1.placement_group_options_hash != v2.placement_group_options_hash + assert v1.requires_actor_restart(v2) + + # Change fallback_strategy. + v3 = DeploymentVersion( + code_version="1", + deployment_config=cfg, + ray_actor_options={}, + placement_group_bundles=[{"CPU": 1}], + placement_group_strategy="PACK", + placement_group_fallback_strategy=[{"placement_group_bundles": [{"CPU": 1}]}], + ) + + # Validate actor restart occurs due to differing hash. + assert v1.placement_group_options_hash != v3.placement_group_options_hash + assert v1.requires_actor_restart(v3) + + if __name__ == "__main__": import sys diff --git a/python/ray/serve/tests/test_standalone.py b/python/ray/serve/tests/test_standalone.py index dfa84420c30f..4ee066adddeb 100644 --- a/python/ray/serve/tests/test_standalone.py +++ b/python/ray/serve/tests/test_standalone.py @@ -8,6 +8,7 @@ import socket import sys import time +from unittest import mock import httpx import pytest @@ -951,5 +952,101 @@ def test_serve_start_proxy_location(ray_shutdown, options): assert ray.get(client._controller.get_http_config.remote()) == expected_options +@pytest.mark.asyncio +@pytest.mark.parametrize("use_compaction", ["0", "1"]) +async def test_serve_label_selector_api( + serve_instance_with_labeled_nodes, use_compaction +): + """ + Verifies that label selectors work correctly for both Actors and Placement Groups. + This test also verifies that label selectors are respected when scheduling with a + preferred node ID for resource compaction. + """ + serve_instance, node_1_id, node_2_id = serve_instance_with_labeled_nodes + + # Restart Serve with the specific scheduler feature flag for this iteration + serve.shutdown() + with mock.patch.dict( + os.environ, {"RAY_SERVE_USE_COMPACT_SCHEDULING_STRATEGY": use_compaction} + ): + serve.start() + + # Validate a Serve deplyoment utilizes a label_selector when passed to the Ray Actor options. + @serve.deployment(ray_actor_options={"label_selector": {"region": "us-west"}}) + class DeploymentActor: + def get_node_id(self): + return ray.get_runtime_context().get_node_id() + + handle = serve.run(DeploymentActor.bind(), name="actor_app") + assert await handle.get_node_id.remote() == node_1_id + serve.delete("actor_app") + + # Validate placement_group scheduling strategy with bundle_label_selector + # and PACK strategy. + @serve.deployment( + placement_group_bundles=[{"CPU": 1}], + placement_group_strategy="PACK", + bundle_label_selector=[{"gpu-type": "H100"}], + ) + class DeploymentPGPack: + def get_node_id(self): + return ray.get_runtime_context().get_node_id() + + handle_pack = serve.run(DeploymentPGPack.bind(), name="pg_pack_app") + assert await handle_pack.get_node_id.remote() == node_2_id + serve.delete("pg_pack_app") + + # Validate placement_group scheduling strategy with bundle_label_selector + # and SPREAD strategy. + @serve.deployment( + placement_group_bundles=[{"CPU": 1}], + placement_group_strategy="SPREAD", + bundle_label_selector=[{"gpu-type": "H100"}], + ) + class DeploymentPGSpread: + def get_node_id(self): + return ray.get_runtime_context().get_node_id() + + handle_spread = serve.run(DeploymentPGSpread.bind(), name="pg_spread_app") + assert await handle_spread.get_node_id.remote() == node_2_id + serve.delete("pg_spread_app") + + +@pytest.mark.asyncio +@pytest.mark.parametrize("use_compaction", ["0", "1"]) +async def test_serve_fallback_strategy_api( + serve_instance_with_labeled_nodes, use_compaction +): + """ + Verifies that fallback strategies allow scheduling on alternative nodes when + primary constraints fail. + """ + serve_instance, node_1_id, node_2_id = serve_instance_with_labeled_nodes + + serve.shutdown() + with mock.patch.dict( + os.environ, {"RAY_SERVE_USE_COMPACT_SCHEDULING_STRATEGY": use_compaction} + ): + serve.start() + + # Fallback strategy specified for Ray Actor in Serve deployment. + @serve.deployment( + ray_actor_options={ + "label_selector": {"region": "unavailable"}, + "fallback_strategy": [{"label_selector": {"gpu-type": "H100"}}], + } + ) + class FallbackDeployment: + def get_node_id(self): + return ray.get_runtime_context().get_node_id() + + # TODO (ryanaoleary@): Add a test for fallback_strategy in placement group options + # when support is added. + + handle = serve.run(FallbackDeployment.bind(), name="fallback_app") + assert await handle.get_node_id.remote() == node_2_id + serve.delete("fallback_app") + + if __name__ == "__main__": sys.exit(pytest.main(["-v", "-s", __file__])) diff --git a/src/ray/protobuf/serve.proto b/src/ray/protobuf/serve.proto index 4fd4d22255c0..68ea77bc0cd6 100644 --- a/src/ray/protobuf/serve.proto +++ b/src/ray/protobuf/serve.proto @@ -256,6 +256,7 @@ message ReplicaConfig { string placement_group_strategy = 7; int32 max_replicas_per_node = 8; string bundle_label_selector = 9; + string fallback_strategy = 10; } enum TargetCapacityDirection { From 6a4d24175fcec3b39ce1404b5ab08ec65db175ba Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Fri, 19 Dec 2025 03:20:14 +0000 Subject: [PATCH 04/55] Add new fields to DeploymentVersion proto Signed-off-by: ryanaoleary --- src/ray/protobuf/serve.proto | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/ray/protobuf/serve.proto b/src/ray/protobuf/serve.proto index 8d54c2ad57d0..21c46c546e25 100644 --- a/src/ray/protobuf/serve.proto +++ b/src/ray/protobuf/serve.proto @@ -244,6 +244,8 @@ message DeploymentVersion { string placement_group_bundles = 4; string placement_group_strategy = 5; int32 max_replicas_per_node = 6; + string placement_group_bundle_label_selector = 7; + string placement_group_fallback_strategy = 8; } message ReplicaConfig { From a1e5fd1c2f68f38b1e848a2a451b3f55b84a1ba5 Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Fri, 19 Dec 2025 03:30:50 +0000 Subject: [PATCH 05/55] Add validation check that bundles are provided with bundle_label_selector Signed-off-by: ryanaoleary --- python/ray/serve/_private/config.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/python/ray/serve/_private/config.py b/python/ray/serve/_private/config.py index 10f263c3f421..1679d34c8783 100644 --- a/python/ray/serve/_private/config.py +++ b/python/ray/serve/_private/config.py @@ -710,6 +710,13 @@ def _validate_placement_group_options(self) -> None: f"Got: {type(strategy)}." ) + if self.bundle_label_selector is not None: + if self.placement_group_bundles is None: + raise ValueError( + "If `bundle_label_selector` is provided, " + "`placement_group_bundles` must also be provided." + ) + if self.placement_group_bundles is not None: validate_placement_group( bundles=self.placement_group_bundles, From 0ba9ed7765c24a87dba420f12b0c35ae3ed4e133 Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Fri, 19 Dec 2025 03:35:24 +0000 Subject: [PATCH 06/55] Correctly handle label constraint for replica actor compaction Signed-off-by: ryanaoleary --- .../ray/serve/_private/deployment_scheduler.py | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/python/ray/serve/_private/deployment_scheduler.py b/python/ray/serve/_private/deployment_scheduler.py index 2b585ad08fda..87056040b0c2 100644 --- a/python/ray/serve/_private/deployment_scheduler.py +++ b/python/ray/serve/_private/deployment_scheduler.py @@ -708,9 +708,21 @@ def schedule( if primary_bundles: # PG: Use PG bundle_label_selector if scheduling_request.placement_group_bundle_label_selector: - primary_labels = ( - scheduling_request.placement_group_bundle_label_selector + pg_strategy = ( + scheduling_request.placement_group_strategy or "PACK" ) + if pg_strategy == "STRICT_PACK": + # All bundle_label_selectors must be satisfied on same node. + primary_labels = ( + scheduling_request.placement_group_bundle_label_selector + ) + else: + # Only the first bundle (where the replica actor lives) must be satisfied. + primary_labels = [ + scheduling_request.placement_group_bundle_label_selector[ + 0 + ] + ] else: # Actor: Use Actor label selector if "label_selector" in scheduling_request.actor_options: From 181156b610263ee529246bbbd261f336506a8dc5 Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Fri, 19 Dec 2025 03:36:50 +0000 Subject: [PATCH 07/55] Fix fallback strategy type Signed-off-by: ryanaoleary --- python/ray/serve/schema.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/serve/schema.py b/python/ray/serve/schema.py index 4b98f3d43a02..d9d4f7ccc80e 100644 --- a/python/ray/serve/schema.py +++ b/python/ray/serve/schema.py @@ -260,7 +260,7 @@ class RayActorOptionsSchema(BaseModel): "If specified, requires that the actor run on a node with the specified labels." ), ) - fallback_strategy: Union[str, List[Dict[str, Any]]] = Field( + fallback_strategy: List[Dict[str, Any]] = Field( default=None, description=( "If specified, expresses soft constraints through a list of decorator " From be6fdf11b20318a41143dfae282f550b6d611c20 Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Fri, 19 Dec 2025 03:50:39 +0000 Subject: [PATCH 08/55] mock correct flag in test Signed-off-by: ryanaoleary --- python/ray/serve/tests/test_deployment_scheduler.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/serve/tests/test_deployment_scheduler.py b/python/ray/serve/tests/test_deployment_scheduler.py index 858e25b12936..d1a24f4855b0 100644 --- a/python/ray/serve/tests/test_deployment_scheduler.py +++ b/python/ray/serve/tests/test_deployment_scheduler.py @@ -400,7 +400,7 @@ def __init__(self): assert set(filtered.keys()) == {"n1", "n2"} @mock.patch( - "ray.serve._private.deployment_scheduler.RAY_SERVE_USE_COMPACT_SCHEDULING_STRATEGY", + "ray.serve._private.deployment_scheduler.RAY_SERVE_USE_PACK_SCHEDULING_STRATEGY", True, ) def test_compact_schedule_respects_labels_and_fallback(self): From 4558c4761f755a6edcd5ef081f8759018ebb0b4f Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Fri, 19 Dec 2025 21:20:19 +0000 Subject: [PATCH 09/55] Fix test, move some to unit, and remove NodeLabelSchedulingStrategy change Signed-off-by: ryanaoleary --- .../serve/_private/deployment_scheduler.py | 6 +- python/ray/serve/_private/test_utils.py | 3 + .../serve/tests/test_deployment_scheduler.py | 119 +----------------- .../tests/unit/test_deployment_scheduler.py | 107 ++++++++++++++++ 4 files changed, 115 insertions(+), 120 deletions(-) diff --git a/python/ray/serve/_private/deployment_scheduler.py b/python/ray/serve/_private/deployment_scheduler.py index 87056040b0c2..806e21fe70c9 100644 --- a/python/ray/serve/_private/deployment_scheduler.py +++ b/python/ray/serve/_private/deployment_scheduler.py @@ -26,6 +26,7 @@ from ray.util.scheduling_strategies import ( LabelMatchExpressionsT, NodeAffinitySchedulingStrategy, + NodeLabelSchedulingStrategy, PlacementGroupSchedulingStrategy, ) @@ -599,8 +600,9 @@ def _schedule_replica( ) target_labels = None elif target_labels is not None: - actor_options["label_selector"] = target_labels - actor_options["fallback_strategy"] = [{"label_selector": {}}] + scheduling_strategy = NodeLabelSchedulingStrategy( + hard={}, soft=target_labels + ) target_node_id = None if scheduling_request.max_replicas_per_node is not None: diff --git a/python/ray/serve/_private/test_utils.py b/python/ray/serve/_private/test_utils.py index db8cd4def7ca..1e50346adee7 100644 --- a/python/ray/serve/_private/test_utils.py +++ b/python/ray/serve/_private/test_utils.py @@ -153,6 +153,9 @@ def add_node(self, node_id: str, resources: Dict = None, labels: Dict = None): def set_available_resources_per_node(self, node_id: str, resources: Dict): self.available_resources_per_node[node_id] = deepcopy(resources) + def get_node_labels(self, node_id: str): + return self.node_labels.get(node_id, {}) + class FakeRemoteFunction: def remote(self): diff --git a/python/ray/serve/tests/test_deployment_scheduler.py b/python/ray/serve/tests/test_deployment_scheduler.py index d1a24f4855b0..c52cc2400958 100644 --- a/python/ray/serve/tests/test_deployment_scheduler.py +++ b/python/ray/serve/tests/test_deployment_scheduler.py @@ -1,5 +1,4 @@ import sys -from unittest import mock import pytest @@ -287,123 +286,7 @@ def test_e2e_custom_resources(self, ray_cluster, use_pg): serve.shutdown() - -class TestSchedulerUnit: - def test_schedule_passes_placement_group_options(self): - """Test that bundle_label_selector is passed to CreatePlacementGroupRequest.""" - cluster_node_info_cache = default_impl.create_cluster_node_info_cache( - GcsClient(address=ray.get_runtime_context().gcs_address) - ) - - captured_requests = [] - - def mock_create_pg(request): - captured_requests.append(request) - - class MockPG: - def wait(self, *args): - return True - - return MockPG() - - scheduler = default_impl.create_deployment_scheduler( - cluster_node_info_cache, - get_head_node_id(), - create_placement_group_fn_override=mock_create_pg, - ) - - dep_id = DeploymentID(name="pg_options_test") - # Use Spread policy here, but the logic is shared across policies. - scheduler.on_deployment_created(dep_id, SpreadDeploymentSchedulingPolicy()) - - test_labels = [{"region": "us-west"}] - # Create a request with the new options - req = ReplicaSchedulingRequest( - replica_id=ReplicaID("r1", dep_id), - actor_def=Replica, - actor_resources={"CPU": 1}, - actor_options={"name": "r1"}, - actor_init_args=(), - on_scheduled=lambda *args: None, - placement_group_bundles=[{"CPU": 1}], - placement_group_bundle_label_selector=test_labels, - ) - - scheduler.schedule(upscales={dep_id: [req]}, downscales={}) - - # Verify the PlacementGroupSchedulingRequest is created. - assert len(captured_requests) == 1 - pg_request = captured_requests[0] - - # bundle_label_selector should be passed to request. - assert pg_request.bundle_label_selector == test_labels - - def test_filter_nodes_by_labels(self): - """Test _filter_nodes_by_labels logic used by _find_best_available_node - when bin-packing, such that label constraints are enforced for the preferred node.""" - - class MockScheduler(default_impl.DefaultDeploymentScheduler): - def __init__(self): - pass - - scheduler = MockScheduler() - - nodes = { - "n1": Resources(), - "n2": Resources(), - "n3": Resources(), - } - node_labels = { - "n1": {"region": "us-west", "gpu": "T4", "env": "prod"}, - "n2": {"region": "us-east", "gpu": "A100", "env": "dev"}, - "n3": {"region": "me-central", "env": "staging"}, # No GPU label - } - - # equals operator - filtered = scheduler._filter_nodes_by_labels( - nodes, {"region": "us-west"}, node_labels - ) - assert set(filtered.keys()) == {"n1"} - - # not equals operator - filtered = scheduler._filter_nodes_by_labels( - nodes, {"region": "!us-west"}, node_labels - ) - assert set(filtered.keys()) == {"n2", "n3"} - - # in operator - filtered = scheduler._filter_nodes_by_labels( - nodes, {"region": "in(us-west, us-east)"}, node_labels - ) - assert set(filtered.keys()) == {"n1", "n2"} - - # !in operator - filtered = scheduler._filter_nodes_by_labels( - nodes, {"env": "!in(dev, staging)"}, node_labels - ) - assert set(filtered.keys()) == {"n1"} - - # Missing labels treated as not a match for equality. - filtered = scheduler._filter_nodes_by_labels( - nodes, {"gpu": "A100"}, node_labels - ) - assert set(filtered.keys()) == {"n2"} - - # Not equal should match node with missing labels. - filtered = scheduler._filter_nodes_by_labels(nodes, {"gpu": "!T4"}, node_labels) - assert set(filtered.keys()) == {"n2", "n3"} - - # Validate we handle whitespace. - filtered = scheduler._filter_nodes_by_labels( - nodes, {"region": "in( us-west , us-east )"}, node_labels - ) - assert set(filtered.keys()) == {"n1", "n2"} - - @mock.patch( - "ray.serve._private.deployment_scheduler.RAY_SERVE_USE_PACK_SCHEDULING_STRATEGY", - True, - ) - def test_compact_schedule_respects_labels_and_fallback(self): + def test_e2e_labels_and_fallback_strategy(self): """Test that scheduler respects labels and tries fallback strategies when selecting the best node for bin-packing.""" diff --git a/python/ray/serve/tests/unit/test_deployment_scheduler.py b/python/ray/serve/tests/unit/test_deployment_scheduler.py index 7b502e24e44f..ea36fbe2f7af 100644 --- a/python/ray/serve/tests/unit/test_deployment_scheduler.py +++ b/python/ray/serve/tests/unit/test_deployment_scheduler.py @@ -867,6 +867,113 @@ def test_downscale_single_deployment(): scheduler.on_deployment_deleted(dep_id) +def test_schedule_passes_placement_group_options(): + """Test that bundle_label_selector is passed to CreatePlacementGroupRequest.""" + cluster_node_info_cache = MockClusterNodeInfoCache() + captured_requests = [] + + def mock_create_pg(request): + captured_requests.append(request) + + class MockPG: + def wait(self, *args): + return True + + return MockPG() + + scheduler = default_impl.create_deployment_scheduler( + cluster_node_info_cache, + head_node_id_override="fake-head-node-id", + create_placement_group_fn_override=mock_create_pg, + ) + + dep_id = DeploymentID(name="pg_options_test") + # Use Spread policy here, but the logic is shared across policies. + scheduler.on_deployment_created(dep_id, SpreadDeploymentSchedulingPolicy()) + + test_labels = [{"region": "us-west"}] + # Create a request with the new options + req = ReplicaSchedulingRequest( + replica_id=ReplicaID("r1", dep_id), + actor_def=MockActorClass(), + actor_resources={"CPU": 1}, + actor_options={"name": "r1"}, + actor_init_args=(), + on_scheduled=lambda *args, **kwargs: None, + placement_group_bundles=[{"CPU": 1}], + placement_group_bundle_label_selector=test_labels, + ) + + scheduler.schedule(upscales={dep_id: [req]}, downscales={}) + + # Verify the PlacementGroupSchedulingRequest is created. + assert len(captured_requests) == 1 + pg_request = captured_requests[0] + + # bundle_label_selector should be passed to request. + assert pg_request.bundle_label_selector == test_labels + + +def test_filter_nodes_by_labels(): + """Test _filter_nodes_by_labels logic used by _find_best_available_node + when bin-packing, such that label constraints are enforced for the preferred node.""" + + class MockScheduler(default_impl.DefaultDeploymentScheduler): + def __init__(self): + pass + + scheduler = MockScheduler() + + nodes = { + "n1": Resources(), + "n2": Resources(), + "n3": Resources(), + } + node_labels = { + "n1": {"region": "us-west", "gpu": "T4", "env": "prod"}, + "n2": {"region": "us-east", "gpu": "A100", "env": "dev"}, + "n3": {"region": "me-central", "env": "staging"}, # No GPU label + } + + # equals operator + filtered = scheduler._filter_nodes_by_labels( + nodes, {"region": "us-west"}, node_labels + ) + assert set(filtered.keys()) == {"n1"} + + # not equals operator + filtered = scheduler._filter_nodes_by_labels( + nodes, {"region": "!us-west"}, node_labels + ) + assert set(filtered.keys()) == {"n2", "n3"} + + # in operator + filtered = scheduler._filter_nodes_by_labels( + nodes, {"region": "in(us-west, us-east)"}, node_labels + ) + assert set(filtered.keys()) == {"n1", "n2"} + + # !in operator + filtered = scheduler._filter_nodes_by_labels( + nodes, {"env": "!in(dev, staging)"}, node_labels + ) + assert set(filtered.keys()) == {"n1"} + + # Missing labels treated as not a match for equality. + filtered = scheduler._filter_nodes_by_labels(nodes, {"gpu": "A100"}, node_labels) + assert set(filtered.keys()) == {"n2"} + + # Not equal should match node with missing labels. + filtered = scheduler._filter_nodes_by_labels(nodes, {"gpu": "!T4"}, node_labels) + assert set(filtered.keys()) == {"n2", "n3"} + + # Validate we handle whitespace. + filtered = scheduler._filter_nodes_by_labels( + nodes, {"region": "in( us-west , us-east )"}, node_labels + ) + assert set(filtered.keys()) == {"n1", "n2"} + + @pytest.mark.skipif( not RAY_SERVE_USE_PACK_SCHEDULING_STRATEGY, reason="Needs pack strategy." ) From b0eae3a375ef422a178399da6901fe50949cc6d8 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> Date: Mon, 22 Dec 2025 09:57:44 -0800 Subject: [PATCH 10/55] Update python/ray/serve/_private/config.py Co-authored-by: Abrar Sheikh Signed-off-by: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> --- python/ray/serve/_private/config.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/ray/serve/_private/config.py b/python/ray/serve/_private/config.py index 1679d34c8783..83faf0c522e4 100644 --- a/python/ray/serve/_private/config.py +++ b/python/ray/serve/_private/config.py @@ -480,8 +480,8 @@ def __init__( ray_actor_options: Dict, placement_group_bundles: Optional[List[Dict[str, float]]] = None, placement_group_strategy: Optional[str] = None, - bundle_label_selector: Optional[List[Dict[str, str]]] = None, - fallback_strategy: Optional[List[Dict[str, Any]]] = None, + placement_group_bundle_label_selectors: Optional[List[Dict[str, str]]] = None, + placement_group_fallback_strategys: Optional[List[Dict[str, Any]]] = None, max_replicas_per_node: Optional[int] = None, needs_pickle: bool = True, ): From 8b72f0c726ef89bc5ac2954cdc406d6d58c389fb Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Tue, 30 Dec 2025 03:48:45 +0000 Subject: [PATCH 11/55] Fix argument names, refactor test fixture, and improve readability Signed-off-by: ryanaoleary --- python/ray/serve/_private/config.py | 136 +++++++++++------- python/ray/serve/_private/deployment_state.py | 18 +-- python/ray/serve/api.py | 22 +-- python/ray/serve/deployment.py | 24 ++-- python/ray/serve/tests/conftest.py | 16 ++- python/ray/serve/tests/test_standalone.py | 136 ++++++++---------- src/ray/protobuf/serve.proto | 4 +- 7 files changed, 196 insertions(+), 160 deletions(-) diff --git a/python/ray/serve/_private/config.py b/python/ray/serve/_private/config.py index 83faf0c522e4..2f7b3bc30d7b 100644 --- a/python/ray/serve/_private/config.py +++ b/python/ray/serve/_private/config.py @@ -480,8 +480,8 @@ def __init__( ray_actor_options: Dict, placement_group_bundles: Optional[List[Dict[str, float]]] = None, placement_group_strategy: Optional[str] = None, - placement_group_bundle_label_selectors: Optional[List[Dict[str, str]]] = None, - placement_group_fallback_strategys: Optional[List[Dict[str, Any]]] = None, + placement_group_bundle_label_selector: Optional[List[Dict[str, str]]] = None, + placement_group_fallback_strategy: Optional[List[Dict[str, Any]]] = None, max_replicas_per_node: Optional[int] = None, needs_pickle: bool = True, ): @@ -507,8 +507,10 @@ def __init__( self.placement_group_bundles = placement_group_bundles self.placement_group_strategy = placement_group_strategy - self.bundle_label_selector = bundle_label_selector - self.fallback_strategy = fallback_strategy + self.placement_group_bundle_label_selector = ( + placement_group_bundle_label_selector + ) + self.placement_group_fallback_strategy = placement_group_fallback_strategy self.max_replicas_per_node = max_replicas_per_node @@ -539,16 +541,18 @@ def update( ray_actor_options: dict, placement_group_bundles: Optional[List[Dict[str, float]]] = None, placement_group_strategy: Optional[str] = None, - bundle_label_selector: Optional[List[Dict[str, str]]] = None, - fallback_strategy: Optional[List[Dict[str, Any]]] = None, + placement_group_bundle_label_selector: Optional[List[Dict[str, str]]] = None, + placement_group_fallback_strategy: Optional[List[Dict[str, Any]]] = None, max_replicas_per_node: Optional[int] = None, ): self.ray_actor_options = ray_actor_options self.placement_group_bundles = placement_group_bundles self.placement_group_strategy = placement_group_strategy - self.bundle_label_selector = bundle_label_selector - self.fallback_strategy = fallback_strategy + self.placement_group_bundle_label_selector = ( + placement_group_bundle_label_selector + ) + self.placement_group_fallback_strategy = placement_group_fallback_strategy self.max_replicas_per_node = max_replicas_per_node @@ -565,8 +569,8 @@ def create( ray_actor_options: Optional[Dict] = None, placement_group_bundles: Optional[List[Dict[str, float]]] = None, placement_group_strategy: Optional[str] = None, - bundle_label_selector: Optional[List[Dict[str, str]]] = None, - fallback_strategy: Optional[List[Dict[str, Any]]] = None, + placement_group_bundle_label_selector: Optional[List[Dict[str, str]]] = None, + placement_group_fallback_strategy: Optional[List[Dict[str, Any]]] = None, max_replicas_per_node: Optional[int] = None, deployment_def_name: Optional[str] = None, ): @@ -621,8 +625,8 @@ def create( ray_actor_options=ray_actor_options, placement_group_bundles=placement_group_bundles, placement_group_strategy=placement_group_strategy, - bundle_label_selector=bundle_label_selector, - fallback_strategy=fallback_strategy, + placement_group_bundle_label_selector=placement_group_bundle_label_selector, + placement_group_fallback_strategy=placement_group_fallback_strategy, max_replicas_per_node=max_replicas_per_node, ) @@ -650,7 +654,7 @@ def _validate_ray_actor_options(self): # Other options "runtime_env", "label_selector", - "fallback_strategy", + "placement_group_fallback_strategy", } for option in self.ray_actor_options: @@ -692,28 +696,28 @@ def _validate_placement_group_options(self) -> None: "`placement_group_bundles` must also be provided." ) - if self.fallback_strategy is not None: + if self.placement_group_fallback_strategy is not None: if self.placement_group_bundles is None: raise ValueError( - "If `fallback_strategy` is provided, " + "If `placement_group_fallback_strategy` is provided, " "`placement_group_bundles` must also be provided." ) - if not isinstance(self.fallback_strategy, list): + if not isinstance(self.placement_group_fallback_strategy, list): raise TypeError( - "fallback_strategy must be a list of dictionaries. " - f"Got: {type(self.fallback_strategy)}." + "placement_group_fallback_strategy must be a list of dictionaries. " + f"Got: {type(self.placement_group_fallback_strategy)}." ) - for i, strategy in enumerate(self.fallback_strategy): + for i, strategy in enumerate(self.placement_group_fallback_strategy): if not isinstance(strategy, dict): raise TypeError( - f"fallback_strategy entry at index {i} must be a dictionary. " + f"placement_group_fallback_strategy entry at index {i} must be a dictionary. " f"Got: {type(strategy)}." ) - if self.bundle_label_selector is not None: + if self.placement_group_bundle_label_selector is not None: if self.placement_group_bundles is None: raise ValueError( - "If `bundle_label_selector` is provided, " + "If `placement_group_bundle_label_selector` is provided, " "`placement_group_bundles` must also be provided." ) @@ -722,7 +726,7 @@ def _validate_placement_group_options(self) -> None: bundles=self.placement_group_bundles, strategy=self.placement_group_strategy or "PACK", lifetime="detached", - bundle_label_selector=self.bundle_label_selector, + placement_group_bundle_label_selector=self.placement_group_bundle_label_selector, ) resource_error_prefix = ( @@ -816,23 +820,37 @@ def init_kwargs(self) -> Optional[Tuple[Any]]: @classmethod def from_proto(cls, proto: ReplicaConfigProto, needs_pickle: bool = True): return ReplicaConfig( - proto.deployment_def_name, - proto.deployment_def, - proto.init_args if proto.init_args != b"" else None, - proto.init_kwargs if proto.init_kwargs != b"" else None, - json.loads(proto.ray_actor_options), - json.loads(proto.placement_group_bundles) - if proto.placement_group_bundles - else None, - proto.placement_group_strategy - if proto.placement_group_strategy != "" - else None, - json.loads(proto.bundle_label_selector) - if proto.bundle_label_selector - else None, - json.loads(proto.fallback_strategy) if proto.fallback_strategy else None, - proto.max_replicas_per_node if proto.max_replicas_per_node else None, - needs_pickle, + deployment_def_name=proto.deployment_def_name, + serialized_deployment_def=proto.deployment_def, + serialized_init_args=(proto.init_args if proto.init_args != b"" else None), + serialized_init_kwargs=( + proto.init_kwargs if proto.init_kwargs != b"" else None + ), + ray_actor_options=json.loads(proto.ray_actor_options), + placement_group_bundles=( + json.loads(proto.placement_group_bundles) + if proto.placement_group_bundles + else None + ), + placement_group_strategy=( + proto.placement_group_strategy + if proto.placement_group_strategy != "" + else None + ), + placement_group_bundle_label_selector=( + json.loads(proto.placement_group_bundle_label_selector) + if proto.placement_group_bundle_label_selector + else None + ), + placement_group_fallback_strategy=( + json.loads(proto.placement_group_fallback_strategy) + if proto.placement_group_fallback_strategy + else None + ), + max_replicas_per_node=( + proto.max_replicas_per_node if proto.max_replicas_per_node else None + ), + needs_pickle=needs_pickle, ) @classmethod @@ -841,25 +859,39 @@ def from_proto_bytes(cls, proto_bytes: bytes, needs_pickle: bool = True): return cls.from_proto(proto, needs_pickle) def to_proto(self): + placement_group_bundles = ( + json.dumps(self.placement_group_bundles) + if self.placement_group_bundles is not None + else "" + ) + + bundle_label_selector = ( + json.dumps(self.placement_group_bundle_label_selector) + if self.placement_group_bundle_label_selector is not None + else "" + ) + + fallback_strategy = ( + json.dumps(self.placement_group_fallback_strategy) + if self.placement_group_fallback_strategy is not None + else "" + ) + + max_replicas_per_node = ( + self.max_replicas_per_node if self.max_replicas_per_node is not None else 0 + ) + return ReplicaConfigProto( deployment_def_name=self.deployment_def_name, deployment_def=self.serialized_deployment_def, init_args=self.serialized_init_args, init_kwargs=self.serialized_init_kwargs, ray_actor_options=json.dumps(self.ray_actor_options), - placement_group_bundles=json.dumps(self.placement_group_bundles) - if self.placement_group_bundles is not None - else "", + placement_group_bundles=placement_group_bundles, placement_group_strategy=self.placement_group_strategy, - bundle_label_selector=json.dumps(self.bundle_label_selector) - if self.bundle_label_selector is not None - else "", - fallback_strategy=json.dumps(self.fallback_strategy) - if self.fallback_strategy is not None - else "", - max_replicas_per_node=self.max_replicas_per_node - if self.max_replicas_per_node is not None - else 0, + placement_group_bundle_label_selector=bundle_label_selector, + placement_group_fallback_strategy=fallback_strategy, + max_replicas_per_node=max_replicas_per_node, ) def to_proto_bytes(self): diff --git a/python/ray/serve/_private/deployment_state.py b/python/ray/serve/_private/deployment_state.py index 945c61cabc1d..daa6cc0f591d 100644 --- a/python/ray/serve/_private/deployment_state.py +++ b/python/ray/serve/_private/deployment_state.py @@ -139,9 +139,11 @@ def create( max_replicas_per_node=info.replica_config.max_replicas_per_node, route_prefix=info.route_prefix, placement_group_bundle_label_selector=( - info.replica_config.bundle_label_selector + info.replica_config.placement_group_bundle_label_selector + ), + placement_group_fallback_strategy=( + info.replica_config.placement_group_fallback_strategy ), - placement_group_fallback_strategy=(info.replica_config.fallback_strategy), ) return cls(info, target_num_replicas, version, deleting) @@ -167,10 +169,10 @@ def is_scaled_copy_of(self, other_target_state: "DeploymentTargetState") -> bool == other_target_state.info.replica_config.placement_group_bundles, self.info.replica_config.placement_group_strategy == other_target_state.info.replica_config.placement_group_strategy, - self.info.replica_config.bundle_label_selector - == other_target_state.info.replica_config.bundle_label_selector, - self.info.replica_config.fallback_strategy - == other_target_state.info.replica_config.fallback_strategy, + self.info.replica_config.placement_group_bundle_label_selector + == other_target_state.info.replica_config.placement_group_bundle_label_selector, + self.info.replica_config.placement_group_fallback_strategy + == other_target_state.info.replica_config.placement_group_fallback_strategy, self.info.replica_config.max_replicas_per_node == other_target_state.info.replica_config.max_replicas_per_node, self.info.deployment_config.dict(exclude={"num_replicas"}) @@ -604,10 +606,10 @@ def start( deployment_info.replica_config.placement_group_strategy ), placement_group_bundle_label_selector=( - deployment_info.replica_config.bundle_label_selector + deployment_info.replica_config.placement_group_bundle_label_selector ), placement_group_fallback_strategy=( - deployment_info.replica_config.fallback_strategy + deployment_info.replica_config.placement_group_fallback_strategy ), max_replicas_per_node=( deployment_info.replica_config.max_replicas_per_node diff --git a/python/ray/serve/api.py b/python/ray/serve/api.py index 62eb66ee690e..0e04b481eaae 100644 --- a/python/ray/serve/api.py +++ b/python/ray/serve/api.py @@ -327,8 +327,10 @@ def deployment( ray_actor_options: Default[Dict] = DEFAULT.VALUE, placement_group_bundles: Default[List[Dict[str, float]]] = DEFAULT.VALUE, placement_group_strategy: Default[str] = DEFAULT.VALUE, - bundle_label_selector: Default[List[Dict[str, str]]] = DEFAULT.VALUE, - fallback_strategy: Default[List[Dict[str, Any]]] = DEFAULT.VALUE, + placement_group_bundle_label_selector: Default[ + List[Dict[str, str]] + ] = DEFAULT.VALUE, + placement_group_fallback_strategy: Default[List[Dict[str, Any]]] = DEFAULT.VALUE, max_replicas_per_node: Default[int] = DEFAULT.VALUE, user_config: Default[Optional[Any]] = DEFAULT.VALUE, max_ongoing_requests: Default[int] = DEFAULT.VALUE, @@ -379,9 +381,9 @@ class MyDeployment: This cannot be set together with max_replicas_per_node. placement_group_strategy: Strategy to use for the replica placement group specified via `placement_group_bundles`. Defaults to `PACK`. - bundle_label_selector: A list of label selectors to apply to the + placement_group_bundle_label_selector: A list of label selectors to apply to the placement group on a per-bundle level. - fallback_strategy: If specified, expresses soft constraints through a list + placement_group_fallback_strategy: If specified, expresses soft constraints through a list of decorator options to fall back on when scheduling on a node. max_replicas_per_node: The max number of replicas of this deployment that can run on a single node. Valid values are None (default, no limit) @@ -501,13 +503,15 @@ def decorator(_func_or_class): if placement_group_strategy is not DEFAULT.VALUE else None ), - bundle_label_selector=( - bundle_label_selector - if bundle_label_selector is not DEFAULT.VALUE + placement_group_bundle_label_selector=( + placement_group_bundle_label_selector + if placement_group_bundle_label_selector is not DEFAULT.VALUE else None ), - fallback_strategy=( - fallback_strategy if fallback_strategy is not DEFAULT.VALUE else None + placement_group_fallback_strategy=( + placement_group_fallback_strategy + if placement_group_fallback_strategy is not DEFAULT.VALUE + else None ), max_replicas_per_node=( max_replicas_per_node diff --git a/python/ray/serve/deployment.py b/python/ray/serve/deployment.py index f54798f194eb..196bf60fb43f 100644 --- a/python/ray/serve/deployment.py +++ b/python/ray/serve/deployment.py @@ -215,8 +215,12 @@ def options( ray_actor_options: Default[Optional[Dict]] = DEFAULT.VALUE, placement_group_bundles: Default[List[Dict[str, float]]] = DEFAULT.VALUE, placement_group_strategy: Default[str] = DEFAULT.VALUE, - bundle_label_selector: Default[List[Dict[str, str]]] = DEFAULT.VALUE, - fallback_strategy: Default[List[Dict[str, Any]]] = DEFAULT.VALUE, + placement_group_bundle_label_selector: Default[ + List[Dict[str, str]] + ] = DEFAULT.VALUE, + placement_group_fallback_strategy: Default[ + List[Dict[str, Any]] + ] = DEFAULT.VALUE, max_replicas_per_node: Default[int] = DEFAULT.VALUE, user_config: Default[Optional[Any]] = DEFAULT.VALUE, max_ongoing_requests: Default[int] = DEFAULT.VALUE, @@ -343,11 +347,15 @@ def options( if placement_group_strategy is DEFAULT.VALUE: placement_group_strategy = self._replica_config.placement_group_strategy - if bundle_label_selector is DEFAULT.VALUE: - bundle_label_selector = self._replica_config.bundle_label_selector + if placement_group_bundle_label_selector is DEFAULT.VALUE: + placement_group_bundle_label_selector = ( + self._replica_config.placement_group_bundle_label_selector + ) - if fallback_strategy is DEFAULT.VALUE: - fallback_strategy = self._replica_config.fallback_strategy + if placement_group_fallback_strategy is DEFAULT.VALUE: + placement_group_fallback_strategy = ( + self._replica_config.placement_group_fallback_strategy + ) if max_replicas_per_node is DEFAULT.VALUE: max_replicas_per_node = self._replica_config.max_replicas_per_node @@ -386,8 +394,8 @@ def options( ray_actor_options=ray_actor_options, placement_group_bundles=placement_group_bundles, placement_group_strategy=placement_group_strategy, - bundle_label_selector=bundle_label_selector, - fallback_strategy=fallback_strategy, + placement_group_bundle_label_selector=placement_group_bundle_label_selector, + placement_group_fallback_strategy=placement_group_fallback_strategy, max_replicas_per_node=max_replicas_per_node, ) diff --git a/python/ray/serve/tests/conftest.py b/python/ray/serve/tests/conftest.py index 1724e7ba1b31..e088338932a9 100644 --- a/python/ray/serve/tests/conftest.py +++ b/python/ray/serve/tests/conftest.py @@ -4,6 +4,7 @@ import tempfile from contextlib import contextmanager from copy import deepcopy +from unittest import mock import httpx import pytest @@ -355,7 +356,7 @@ def _get_node_id(): # Test fixture to start a Serve instance in a RayCluster with two labeled nodes @pytest.fixture -def serve_instance_with_labeled_nodes(): +def serve_instance_with_labeled_nodes(request): cluster = Cluster() # Unlabeled default node. node0_config = { @@ -380,9 +381,16 @@ def serve_instance_with_labeled_nodes(): cluster.add_node(**node2_config) node_2_id = ray.get(_get_node_id.options(resources={"worker2": 1}).remote()) - serve.start() - yield _get_global_client(), node_1_id, node_2_id + # Check if the test passed a parameter to the fixture. + use_compaction = getattr(request, "param", None) + env_vars = {} + if use_compaction is not None: + env_vars = {"RAY_SERVE_USE_COMPACT_SCHEDULING_STRATEGY": use_compaction} + + with mock.patch.dict(os.environ, env_vars): + serve.start() + yield _get_global_client(), node_1_id, node_2_id + serve.shutdown() - serve.shutdown() ray.shutdown() cluster.shutdown() diff --git a/python/ray/serve/tests/test_standalone.py b/python/ray/serve/tests/test_standalone.py index 4ee066adddeb..ce581eff0396 100644 --- a/python/ray/serve/tests/test_standalone.py +++ b/python/ray/serve/tests/test_standalone.py @@ -8,7 +8,6 @@ import socket import sys import time -from unittest import mock import httpx import pytest @@ -953,99 +952,82 @@ def test_serve_start_proxy_location(ray_shutdown, options): @pytest.mark.asyncio -@pytest.mark.parametrize("use_compaction", ["0", "1"]) -async def test_serve_label_selector_api( - serve_instance_with_labeled_nodes, use_compaction -): +@pytest.mark.parametrize("serve_instance_with_labeled_nodes", ["0", "1"], indirect=True) +async def test_serve_label_selector_api(serve_instance_with_labeled_nodes): """ Verifies that label selectors work correctly for both Actors and Placement Groups. This test also verifies that label selectors are respected when scheduling with a preferred node ID for resource compaction. """ - serve_instance, node_1_id, node_2_id = serve_instance_with_labeled_nodes + serve_instance, us_west_node_id, us_east_node_id = serve_instance_with_labeled_nodes - # Restart Serve with the specific scheduler feature flag for this iteration - serve.shutdown() - with mock.patch.dict( - os.environ, {"RAY_SERVE_USE_COMPACT_SCHEDULING_STRATEGY": use_compaction} - ): - serve.start() - - # Validate a Serve deplyoment utilizes a label_selector when passed to the Ray Actor options. - @serve.deployment(ray_actor_options={"label_selector": {"region": "us-west"}}) - class DeploymentActor: - def get_node_id(self): - return ray.get_runtime_context().get_node_id() - - handle = serve.run(DeploymentActor.bind(), name="actor_app") - assert await handle.get_node_id.remote() == node_1_id - serve.delete("actor_app") - - # Validate placement_group scheduling strategy with bundle_label_selector - # and PACK strategy. - @serve.deployment( - placement_group_bundles=[{"CPU": 1}], - placement_group_strategy="PACK", - bundle_label_selector=[{"gpu-type": "H100"}], - ) - class DeploymentPGPack: - def get_node_id(self): - return ray.get_runtime_context().get_node_id() - - handle_pack = serve.run(DeploymentPGPack.bind(), name="pg_pack_app") - assert await handle_pack.get_node_id.remote() == node_2_id - serve.delete("pg_pack_app") - - # Validate placement_group scheduling strategy with bundle_label_selector - # and SPREAD strategy. - @serve.deployment( - placement_group_bundles=[{"CPU": 1}], - placement_group_strategy="SPREAD", - bundle_label_selector=[{"gpu-type": "H100"}], - ) - class DeploymentPGSpread: - def get_node_id(self): - return ray.get_runtime_context().get_node_id() + # Validate a Serve deplyoment utilizes a label_selector when passed to the Ray Actor options. + @serve.deployment(ray_actor_options={"label_selector": {"region": "us-west"}}) + class DeploymentActor: + def get_node_id(self): + return ray.get_runtime_context().get_node_id() - handle_spread = serve.run(DeploymentPGSpread.bind(), name="pg_spread_app") - assert await handle_spread.get_node_id.remote() == node_2_id - serve.delete("pg_spread_app") + handle = serve.run(DeploymentActor.bind(), name="actor_app") + assert await handle.get_node_id.remote() == us_west_node_id + serve.delete("actor_app") + + # Validate placement_group scheduling strategy with placement_group_bundle_label_selector + # and PACK strategy. + @serve.deployment( + placement_group_bundles=[{"CPU": 1}], + placement_group_strategy="PACK", + placement_group_bundle_label_selector=[{"gpu-type": "H100"}], + ) + class DeploymentPGPack: + def get_node_id(self): + return ray.get_runtime_context().get_node_id() + + handle_pack = serve.run(DeploymentPGPack.bind(), name="pg_pack_app") + assert await handle_pack.get_node_id.remote() == us_east_node_id + serve.delete("pg_pack_app") + + # Validate placement_group scheduling strategy with placement_group_bundle_label_selector + # and SPREAD strategy. + @serve.deployment( + placement_group_bundles=[{"CPU": 1}], + placement_group_strategy="SPREAD", + placement_group_bundle_label_selector=[{"gpu-type": "H100"}], + ) + class DeploymentPGSpread: + def get_node_id(self): + return ray.get_runtime_context().get_node_id() + + handle_spread = serve.run(DeploymentPGSpread.bind(), name="pg_spread_app") + assert await handle_spread.get_node_id.remote() == us_east_node_id + serve.delete("pg_spread_app") @pytest.mark.asyncio -@pytest.mark.parametrize("use_compaction", ["0", "1"]) -async def test_serve_fallback_strategy_api( - serve_instance_with_labeled_nodes, use_compaction -): +@pytest.mark.parametrize("serve_instance_with_labeled_nodes", ["0", "1"], indirect=True) +async def test_serve_fallback_strategy_api(serve_instance_with_labeled_nodes): """ Verifies that fallback strategies allow scheduling on alternative nodes when primary constraints fail. """ - serve_instance, node_1_id, node_2_id = serve_instance_with_labeled_nodes + serve_instance, _, h100_node_id = serve_instance_with_labeled_nodes - serve.shutdown() - with mock.patch.dict( - os.environ, {"RAY_SERVE_USE_COMPACT_SCHEDULING_STRATEGY": use_compaction} - ): - serve.start() - - # Fallback strategy specified for Ray Actor in Serve deployment. - @serve.deployment( - ray_actor_options={ - "label_selector": {"region": "unavailable"}, - "fallback_strategy": [{"label_selector": {"gpu-type": "H100"}}], - } - ) - class FallbackDeployment: - def get_node_id(self): - return ray.get_runtime_context().get_node_id() + # Fallback strategy specified for Ray Actor in Serve deployment. + @serve.deployment( + ray_actor_options={ + "label_selector": {"region": "unavailable"}, + "fallback_strategy": [{"label_selector": {"gpu-type": "H100"}}], + } + ) + class FallbackDeployment: + def get_node_id(self): + return ray.get_runtime_context().get_node_id() - # TODO (ryanaoleary@): Add a test for fallback_strategy in placement group options - # when support is added. + # TODO (ryanaoleary@): Add a test for fallback_strategy in placement group options + # when support is added. - handle = serve.run(FallbackDeployment.bind(), name="fallback_app") - assert await handle.get_node_id.remote() == node_2_id - serve.delete("fallback_app") + handle = serve.run(FallbackDeployment.bind(), name="fallback_app") + assert await handle.get_node_id.remote() == h100_node_id + serve.delete("fallback_app") if __name__ == "__main__": diff --git a/src/ray/protobuf/serve.proto b/src/ray/protobuf/serve.proto index 21c46c546e25..bd38183b1c61 100644 --- a/src/ray/protobuf/serve.proto +++ b/src/ray/protobuf/serve.proto @@ -257,8 +257,8 @@ message ReplicaConfig { string placement_group_bundles = 6; string placement_group_strategy = 7; int32 max_replicas_per_node = 8; - string bundle_label_selector = 9; - string fallback_strategy = 10; + string placement_group_bundle_label_selector = 9; + string placement_group_fallback_strategy = 10; } enum TargetCapacityDirection { From 248be33069c1383d0b3193a76d73635fab051824 Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Tue, 30 Dec 2025 03:52:59 +0000 Subject: [PATCH 12/55] Remove fallback_strategy from Deployment API Signed-off-by: ryanaoleary --- python/ray/serve/deployment.py | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/python/ray/serve/deployment.py b/python/ray/serve/deployment.py index 196bf60fb43f..36b37b381324 100644 --- a/python/ray/serve/deployment.py +++ b/python/ray/serve/deployment.py @@ -218,9 +218,6 @@ def options( placement_group_bundle_label_selector: Default[ List[Dict[str, str]] ] = DEFAULT.VALUE, - placement_group_fallback_strategy: Default[ - List[Dict[str, Any]] - ] = DEFAULT.VALUE, max_replicas_per_node: Default[int] = DEFAULT.VALUE, user_config: Default[Optional[Any]] = DEFAULT.VALUE, max_ongoing_requests: Default[int] = DEFAULT.VALUE, @@ -352,10 +349,9 @@ def options( self._replica_config.placement_group_bundle_label_selector ) - if placement_group_fallback_strategy is DEFAULT.VALUE: - placement_group_fallback_strategy = ( - self._replica_config.placement_group_fallback_strategy - ) + placement_group_fallback_strategy = ( + self._replica_config.placement_group_fallback_strategy + ) if max_replicas_per_node is DEFAULT.VALUE: max_replicas_per_node = self._replica_config.max_replicas_per_node From 3ddb0a1eb99c7073490909aecda8b2a65d2efdba Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Tue, 30 Dec 2025 03:53:53 +0000 Subject: [PATCH 13/55] Add new fields to to_dict Signed-off-by: ryanaoleary --- python/ray/serve/_private/config.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/python/ray/serve/_private/config.py b/python/ray/serve/_private/config.py index 2f7b3bc30d7b..994a367e380a 100644 --- a/python/ray/serve/_private/config.py +++ b/python/ray/serve/_private/config.py @@ -904,6 +904,8 @@ def to_dict(self): "ray_actor_options": self.ray_actor_options, "placement_group_bundles": self.placement_group_bundles, "placement_group_strategy": self.placement_group_strategy, + "placement_group_bundle_label_selector": self.placement_group_bundle_label_selector, + "placement_group_fallback_strategy": self.placement_group_fallback_strategy, "max_replicas_per_node": self.max_replicas_per_node, } From a9dbec0422c59ddedc2ed13fb35fe123a5b032f1 Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Tue, 30 Dec 2025 03:55:03 +0000 Subject: [PATCH 14/55] Fully remove fallback strategy from deployment API Signed-off-by: ryanaoleary --- python/ray/serve/api.py | 8 -------- 1 file changed, 8 deletions(-) diff --git a/python/ray/serve/api.py b/python/ray/serve/api.py index 0e04b481eaae..586d2cb4c491 100644 --- a/python/ray/serve/api.py +++ b/python/ray/serve/api.py @@ -330,7 +330,6 @@ def deployment( placement_group_bundle_label_selector: Default[ List[Dict[str, str]] ] = DEFAULT.VALUE, - placement_group_fallback_strategy: Default[List[Dict[str, Any]]] = DEFAULT.VALUE, max_replicas_per_node: Default[int] = DEFAULT.VALUE, user_config: Default[Optional[Any]] = DEFAULT.VALUE, max_ongoing_requests: Default[int] = DEFAULT.VALUE, @@ -383,8 +382,6 @@ class MyDeployment: specified via `placement_group_bundles`. Defaults to `PACK`. placement_group_bundle_label_selector: A list of label selectors to apply to the placement group on a per-bundle level. - placement_group_fallback_strategy: If specified, expresses soft constraints through a list - of decorator options to fall back on when scheduling on a node. max_replicas_per_node: The max number of replicas of this deployment that can run on a single node. Valid values are None (default, no limit) or an integer in the range of [1, 100]. @@ -508,11 +505,6 @@ def decorator(_func_or_class): if placement_group_bundle_label_selector is not DEFAULT.VALUE else None ), - placement_group_fallback_strategy=( - placement_group_fallback_strategy - if placement_group_fallback_strategy is not DEFAULT.VALUE - else None - ), max_replicas_per_node=( max_replicas_per_node if max_replicas_per_node is not DEFAULT.VALUE From d96f85841b32a737bfa98b00a01cfb892d0122a9 Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Tue, 30 Dec 2025 21:58:04 +0000 Subject: [PATCH 15/55] Fix var names and tests Signed-off-by: ryanaoleary --- python/ray/serve/_private/config.py | 4 +- python/ray/serve/tests/conftest.py | 16 +--- .../serve/tests/test_deployment_scheduler.py | 95 ++++++------------- python/ray/serve/tests/test_standalone.py | 2 - 4 files changed, 33 insertions(+), 84 deletions(-) diff --git a/python/ray/serve/_private/config.py b/python/ray/serve/_private/config.py index 994a367e380a..b0dc17edca58 100644 --- a/python/ray/serve/_private/config.py +++ b/python/ray/serve/_private/config.py @@ -654,7 +654,7 @@ def _validate_ray_actor_options(self): # Other options "runtime_env", "label_selector", - "placement_group_fallback_strategy", + "fallback_strategy", } for option in self.ray_actor_options: @@ -726,7 +726,7 @@ def _validate_placement_group_options(self) -> None: bundles=self.placement_group_bundles, strategy=self.placement_group_strategy or "PACK", lifetime="detached", - placement_group_bundle_label_selector=self.placement_group_bundle_label_selector, + bundle_label_selector=self.placement_group_bundle_label_selector, ) resource_error_prefix = ( diff --git a/python/ray/serve/tests/conftest.py b/python/ray/serve/tests/conftest.py index e088338932a9..1724e7ba1b31 100644 --- a/python/ray/serve/tests/conftest.py +++ b/python/ray/serve/tests/conftest.py @@ -4,7 +4,6 @@ import tempfile from contextlib import contextmanager from copy import deepcopy -from unittest import mock import httpx import pytest @@ -356,7 +355,7 @@ def _get_node_id(): # Test fixture to start a Serve instance in a RayCluster with two labeled nodes @pytest.fixture -def serve_instance_with_labeled_nodes(request): +def serve_instance_with_labeled_nodes(): cluster = Cluster() # Unlabeled default node. node0_config = { @@ -381,16 +380,9 @@ def serve_instance_with_labeled_nodes(request): cluster.add_node(**node2_config) node_2_id = ray.get(_get_node_id.options(resources={"worker2": 1}).remote()) - # Check if the test passed a parameter to the fixture. - use_compaction = getattr(request, "param", None) - env_vars = {} - if use_compaction is not None: - env_vars = {"RAY_SERVE_USE_COMPACT_SCHEDULING_STRATEGY": use_compaction} - - with mock.patch.dict(os.environ, env_vars): - serve.start() - yield _get_global_client(), node_1_id, node_2_id - serve.shutdown() + serve.start() + yield _get_global_client(), node_1_id, node_2_id + serve.shutdown() ray.shutdown() cluster.shutdown() diff --git a/python/ray/serve/tests/test_deployment_scheduler.py b/python/ray/serve/tests/test_deployment_scheduler.py index c52cc2400958..111e794bb946 100644 --- a/python/ray/serve/tests/test_deployment_scheduler.py +++ b/python/ray/serve/tests/test_deployment_scheduler.py @@ -11,7 +11,6 @@ from ray.serve._private.constants import RAY_SERVE_USE_PACK_SCHEDULING_STRATEGY from ray.serve._private.deployment_scheduler import ( ReplicaSchedulingRequest, - Resources, SpreadDeploymentSchedulingPolicy, ) from ray.serve._private.test_utils import check_apps_running, get_node_id @@ -286,88 +285,48 @@ def test_e2e_custom_resources(self, ray_cluster, use_pg): serve.shutdown() - def test_e2e_labels_and_fallback_strategy(self): + def test_e2e_labels_and_fallback_strategy(self, serve_instance_with_labeled_nodes): """Test that scheduler respects labels and tries fallback strategies when selecting the best node for bin-packing.""" - N1_ID = "1" * 56 - N2_ID = "2" * 56 - N3_ID = "3" * 56 - - # Mock some active nodes with varying resources and labels. - class MockActor: - def options(self, **kwargs): - return self - - def remote(self, *args, **kwargs): - return "mock_handle" - - class MockCache: - def get_active_node_ids(self): - return {N1_ID, N2_ID, N3_ID} - - def get_node_labels(self, node_id): - return { - N1_ID: {"region": "us-west"}, - N2_ID: {"region": "us-east"}, - N3_ID: {"region": "eu-central"}, - }[node_id] - - def get_available_resources_per_node(self): - return { - N1_ID: {"CPU": 2}, - N2_ID: {"CPU": 10}, - N3_ID: {"CPU": 4}, - } - - def get_total_resources_per_node(self): - return self.get_available_resources_per_node() + ( + serve_client, + us_east_node_id, + us_west_node_id, + ) = serve_instance_with_labeled_nodes - scheduler = default_impl.create_deployment_scheduler( - MockCache(), "head_node_id" - ) + @serve.deployment + def GetNodeId(): + return ray.get_runtime_context().get_node_id() # Validate the label_selector constraint is respected. # For a request of 1 CPU and "us-west", we pick the node that matches the label # (n2) even though a better fit exists (n1). - dep_id = DeploymentID(name="test_labels") - scheduler.on_deployment_created(dep_id, SpreadDeploymentSchedulingPolicy()) - scheduler._deployments[dep_id].actor_resources = Resources({"CPU": 1}) - - req = ReplicaSchedulingRequest( - replica_id=ReplicaID("r1", dep_id), - actor_def=Replica, - actor_resources={"CPU": 1}, - actor_options={"label_selector": {"region": "us-west"}, "name": "r1"}, - actor_init_args=(), - on_scheduled=lambda *args, **kwargs: None, - ) + app_primary = GetNodeId.options( + ray_actor_options={"label_selector": {"region": "us-west"}} + ).bind() - scheduler.schedule(upscales={dep_id: [req]}, downscales={}) - launch_info = scheduler._launching_replicas[dep_id][req.replica_id] - assert launch_info.target_node_id == N1_ID + handle_primary = serve.run( + app_primary, name="app_primary", route_prefix="/primary" + ) + assert handle_primary.remote().result() == us_west_node_id # Validate fallback strategy is used when selecting a node. If the # label constraints and resources of the primary strategy are infeasible, # we try each fallback strategy in-order. - req_fallback = ReplicaSchedulingRequest( - replica_id=ReplicaID("r2", dep_id), - actor_def=Replica, - actor_resources={"CPU": 1}, - actor_options={ - "label_selector": {"region": "us-north"}, # Invalid + app_fallback = GetNodeId.options( + ray_actor_options={ + "label_selector": {"region": "us-north"}, # Invalid label "fallback_strategy": [ - {"label_selector": {"region": "eu-central"}} - ], # Valid (n3) - "name": "r2", - }, - actor_init_args=(), - on_scheduled=lambda *args, **kwargs: None, - ) + {"label_selector": {"region": "us-east"}} # Valid fallback + ], + } + ).bind() - scheduler.schedule(upscales={dep_id: [req_fallback]}, downscales={}) - launch_info = scheduler._launching_replicas[dep_id][req_fallback.replica_id] - assert launch_info.target_node_id == N3_ID + handle_fallback = serve.run( + app_fallback, name="app_fallback", route_prefix="/fallback" + ) + assert handle_fallback.remote().result() == us_west_node_id if __name__ == "__main__": diff --git a/python/ray/serve/tests/test_standalone.py b/python/ray/serve/tests/test_standalone.py index ce581eff0396..01e4d502acbe 100644 --- a/python/ray/serve/tests/test_standalone.py +++ b/python/ray/serve/tests/test_standalone.py @@ -952,7 +952,6 @@ def test_serve_start_proxy_location(ray_shutdown, options): @pytest.mark.asyncio -@pytest.mark.parametrize("serve_instance_with_labeled_nodes", ["0", "1"], indirect=True) async def test_serve_label_selector_api(serve_instance_with_labeled_nodes): """ Verifies that label selectors work correctly for both Actors and Placement Groups. @@ -1003,7 +1002,6 @@ def get_node_id(self): @pytest.mark.asyncio -@pytest.mark.parametrize("serve_instance_with_labeled_nodes", ["0", "1"], indirect=True) async def test_serve_fallback_strategy_api(serve_instance_with_labeled_nodes): """ Verifies that fallback strategies allow scheduling on alternative nodes when From db162afa322ea158f03eeadadc03d13ab3e26b52 Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Tue, 30 Dec 2025 22:18:49 +0000 Subject: [PATCH 16/55] Move appropriate tests to deployment_scheduler and remove duplicate test Signed-off-by: ryanaoleary --- .../serve/tests/test_deployment_scheduler.py | 108 ++++++++++++------ python/ray/serve/tests/test_standalone.py | 77 ------------- 2 files changed, 71 insertions(+), 114 deletions(-) diff --git a/python/ray/serve/tests/test_deployment_scheduler.py b/python/ray/serve/tests/test_deployment_scheduler.py index 111e794bb946..8cc084436baf 100644 --- a/python/ray/serve/tests/test_deployment_scheduler.py +++ b/python/ray/serve/tests/test_deployment_scheduler.py @@ -285,48 +285,82 @@ def test_e2e_custom_resources(self, ray_cluster, use_pg): serve.shutdown() - def test_e2e_labels_and_fallback_strategy(self, serve_instance_with_labeled_nodes): - """Test that scheduler respects labels and tries fallback strategies - when selecting the best node for bin-packing.""" - ( - serve_client, - us_east_node_id, - us_west_node_id, - ) = serve_instance_with_labeled_nodes +@pytest.mark.asyncio +async def test_e2e_serve_label_selector(serve_instance_with_labeled_nodes): + """ + Verifies that label selectors work correctly for both Actors and Placement Groups. + This test also verifies that label selectors are respected when scheduling with a + preferred node ID for resource compaction. + """ + serve_instance, us_west_node_id, us_east_node_id = serve_instance_with_labeled_nodes + + # Validate a Serve deplyoment utilizes a label_selector when passed to the Ray Actor options. + @serve.deployment(ray_actor_options={"label_selector": {"region": "us-west"}}) + class DeploymentActor: + def get_node_id(self): + return ray.get_runtime_context().get_node_id() - @serve.deployment - def GetNodeId(): + handle = serve.run(DeploymentActor.bind(), name="actor_app") + assert await handle.get_node_id.remote() == us_west_node_id + serve.delete("actor_app") + + # Validate placement_group scheduling strategy with placement_group_bundle_label_selector + # and PACK strategy. + @serve.deployment( + placement_group_bundles=[{"CPU": 1}], + placement_group_strategy="PACK", + placement_group_bundle_label_selector=[{"gpu-type": "H100"}], + ) + class DeploymentPGPack: + def get_node_id(self): return ray.get_runtime_context().get_node_id() - # Validate the label_selector constraint is respected. - # For a request of 1 CPU and "us-west", we pick the node that matches the label - # (n2) even though a better fit exists (n1). - app_primary = GetNodeId.options( - ray_actor_options={"label_selector": {"region": "us-west"}} - ).bind() + handle_pack = serve.run(DeploymentPGPack.bind(), name="pg_pack_app") + assert await handle_pack.get_node_id.remote() == us_east_node_id + serve.delete("pg_pack_app") - handle_primary = serve.run( - app_primary, name="app_primary", route_prefix="/primary" - ) - assert handle_primary.remote().result() == us_west_node_id - - # Validate fallback strategy is used when selecting a node. If the - # label constraints and resources of the primary strategy are infeasible, - # we try each fallback strategy in-order. - app_fallback = GetNodeId.options( - ray_actor_options={ - "label_selector": {"region": "us-north"}, # Invalid label - "fallback_strategy": [ - {"label_selector": {"region": "us-east"}} # Valid fallback - ], - } - ).bind() - - handle_fallback = serve.run( - app_fallback, name="app_fallback", route_prefix="/fallback" - ) - assert handle_fallback.remote().result() == us_west_node_id + # Validate placement_group scheduling strategy with placement_group_bundle_label_selector + # and SPREAD strategy. + @serve.deployment( + placement_group_bundles=[{"CPU": 1}], + placement_group_strategy="SPREAD", + placement_group_bundle_label_selector=[{"gpu-type": "H100"}], + ) + class DeploymentPGSpread: + def get_node_id(self): + return ray.get_runtime_context().get_node_id() + + handle_spread = serve.run(DeploymentPGSpread.bind(), name="pg_spread_app") + assert await handle_spread.get_node_id.remote() == us_east_node_id + serve.delete("pg_spread_app") + + +@pytest.mark.asyncio +async def test_e2e_serve_fallback_strategy(serve_instance_with_labeled_nodes): + """ + Verifies that fallback strategies allow scheduling on alternative nodes when + primary constraints fail. + """ + serve_instance, _, h100_node_id = serve_instance_with_labeled_nodes + + # Fallback strategy specified for Ray Actor in Serve deployment. + @serve.deployment( + ray_actor_options={ + "label_selector": {"region": "unavailable"}, + "fallback_strategy": [{"label_selector": {"gpu-type": "H100"}}], + } + ) + class FallbackDeployment: + def get_node_id(self): + return ray.get_runtime_context().get_node_id() + + # TODO (ryanaoleary@): Add a test for fallback_strategy in placement group options + # when support is added. + + handle = serve.run(FallbackDeployment.bind(), name="fallback_app") + assert await handle.get_node_id.remote() == h100_node_id + serve.delete("fallback_app") if __name__ == "__main__": diff --git a/python/ray/serve/tests/test_standalone.py b/python/ray/serve/tests/test_standalone.py index 01e4d502acbe..dfa84420c30f 100644 --- a/python/ray/serve/tests/test_standalone.py +++ b/python/ray/serve/tests/test_standalone.py @@ -951,82 +951,5 @@ def test_serve_start_proxy_location(ray_shutdown, options): assert ray.get(client._controller.get_http_config.remote()) == expected_options -@pytest.mark.asyncio -async def test_serve_label_selector_api(serve_instance_with_labeled_nodes): - """ - Verifies that label selectors work correctly for both Actors and Placement Groups. - This test also verifies that label selectors are respected when scheduling with a - preferred node ID for resource compaction. - """ - serve_instance, us_west_node_id, us_east_node_id = serve_instance_with_labeled_nodes - - # Validate a Serve deplyoment utilizes a label_selector when passed to the Ray Actor options. - @serve.deployment(ray_actor_options={"label_selector": {"region": "us-west"}}) - class DeploymentActor: - def get_node_id(self): - return ray.get_runtime_context().get_node_id() - - handle = serve.run(DeploymentActor.bind(), name="actor_app") - assert await handle.get_node_id.remote() == us_west_node_id - serve.delete("actor_app") - - # Validate placement_group scheduling strategy with placement_group_bundle_label_selector - # and PACK strategy. - @serve.deployment( - placement_group_bundles=[{"CPU": 1}], - placement_group_strategy="PACK", - placement_group_bundle_label_selector=[{"gpu-type": "H100"}], - ) - class DeploymentPGPack: - def get_node_id(self): - return ray.get_runtime_context().get_node_id() - - handle_pack = serve.run(DeploymentPGPack.bind(), name="pg_pack_app") - assert await handle_pack.get_node_id.remote() == us_east_node_id - serve.delete("pg_pack_app") - - # Validate placement_group scheduling strategy with placement_group_bundle_label_selector - # and SPREAD strategy. - @serve.deployment( - placement_group_bundles=[{"CPU": 1}], - placement_group_strategy="SPREAD", - placement_group_bundle_label_selector=[{"gpu-type": "H100"}], - ) - class DeploymentPGSpread: - def get_node_id(self): - return ray.get_runtime_context().get_node_id() - - handle_spread = serve.run(DeploymentPGSpread.bind(), name="pg_spread_app") - assert await handle_spread.get_node_id.remote() == us_east_node_id - serve.delete("pg_spread_app") - - -@pytest.mark.asyncio -async def test_serve_fallback_strategy_api(serve_instance_with_labeled_nodes): - """ - Verifies that fallback strategies allow scheduling on alternative nodes when - primary constraints fail. - """ - serve_instance, _, h100_node_id = serve_instance_with_labeled_nodes - - # Fallback strategy specified for Ray Actor in Serve deployment. - @serve.deployment( - ray_actor_options={ - "label_selector": {"region": "unavailable"}, - "fallback_strategy": [{"label_selector": {"gpu-type": "H100"}}], - } - ) - class FallbackDeployment: - def get_node_id(self): - return ray.get_runtime_context().get_node_id() - - # TODO (ryanaoleary@): Add a test for fallback_strategy in placement group options - # when support is added. - - handle = serve.run(FallbackDeployment.bind(), name="fallback_app") - assert await handle.get_node_id.remote() == h100_node_id - serve.delete("fallback_app") - - if __name__ == "__main__": sys.exit(pytest.main(["-v", "-s", __file__])) From 935e084f16cdf02492eaeaa0ce882d5f8243988c Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Tue, 30 Dec 2025 22:26:48 +0000 Subject: [PATCH 17/55] Handle Label selector set to None Signed-off-by: ryanaoleary --- python/ray/serve/_private/deployment_scheduler.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/ray/serve/_private/deployment_scheduler.py b/python/ray/serve/_private/deployment_scheduler.py index 806e21fe70c9..efcd448a7487 100644 --- a/python/ray/serve/_private/deployment_scheduler.py +++ b/python/ray/serve/_private/deployment_scheduler.py @@ -729,7 +729,7 @@ def schedule( # Actor: Use Actor label selector if "label_selector" in scheduling_request.actor_options: primary_labels = [ - scheduling_request.actor_options["label_selector"] + scheduling_request.actor_options["label_selector"] or {} ] strategies_to_try.append( @@ -753,7 +753,7 @@ def schedule( for fallback in scheduling_request.actor_options[ "fallback_strategy" ]: - fallback_labels = [fallback.get("label_selector", {})] + fallback_labels = [fallback.get("label_selector", {}) or {}] strategies_to_try.append( (scheduling_request.required_resources, fallback_labels) ) From d87f8383e46d58c4b95984ecdd14484dac76b7dd Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Wed, 31 Dec 2025 00:49:52 +0000 Subject: [PATCH 18/55] Move back actor_options copy to where it was Signed-off-by: ryanaoleary --- python/ray/serve/_private/deployment_scheduler.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/serve/_private/deployment_scheduler.py b/python/ray/serve/_private/deployment_scheduler.py index efcd448a7487..33e1d2a356c7 100644 --- a/python/ray/serve/_private/deployment_scheduler.py +++ b/python/ray/serve/_private/deployment_scheduler.py @@ -560,7 +560,6 @@ def _schedule_replica( deployment_id = replica_id.deployment_id placement_group = None - actor_options = copy.deepcopy(scheduling_request.actor_options) scheduling_strategy = default_scheduling_strategy if scheduling_request.placement_group_bundles is not None: placement_group_strategy = ( @@ -605,6 +604,7 @@ def _schedule_replica( ) target_node_id = None + actor_options = copy.deepcopy(scheduling_request.actor_options) if scheduling_request.max_replicas_per_node is not None: if "resources" not in actor_options: actor_options["resources"] = {} From fe6a1a79f1c70bdc75338e1b715e04bd8adf984f Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Tue, 6 Jan 2026 20:19:37 +0000 Subject: [PATCH 19/55] Fix invalid parameter being passed as placement_group_version Signed-off-by: ryanaoleary --- python/ray/serve/_private/version.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/python/ray/serve/_private/version.py b/python/ray/serve/_private/version.py index 20c326bfcd7c..be375355db8a 100644 --- a/python/ray/serve/_private/version.py +++ b/python/ray/serve/_private/version.py @@ -191,8 +191,10 @@ def from_proto(cls, proto: DeploymentVersionProto): if proto.placement_group_fallback_strategy else None ), - placement_group_version=( - proto.placement_group_version if proto.placement_group_version else None + placement_group_strategy=( + proto.placement_group_strategy + if proto.placement_group_strategy + else None ), max_replicas_per_node=( proto.max_replicas_per_node if proto.max_replicas_per_node else None From 40c02e0efd15e8e3379ddced8aa73acbc4789e80 Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Tue, 6 Jan 2026 20:25:33 +0000 Subject: [PATCH 20/55] Add string type check to _filter_nodes_by_labels Signed-off-by: ryanaoleary --- python/ray/serve/_private/deployment_scheduler.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/python/ray/serve/_private/deployment_scheduler.py b/python/ray/serve/_private/deployment_scheduler.py index 33e1d2a356c7..4b5b46b3454d 100644 --- a/python/ray/serve/_private/deployment_scheduler.py +++ b/python/ray/serve/_private/deployment_scheduler.py @@ -885,6 +885,11 @@ def _filter_nodes_by_labels( for key, req_val in required_labels.items(): node_val = labels.get(key) + # We only support string labels. + if not isinstance(req_val, str): + is_match = False + break + # !in operator if req_val.startswith("!in(") and req_val.endswith(")"): content = req_val[4:-1] From 7b13cc5e3b46eff55fef0724647e72038c609f82 Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Wed, 7 Jan 2026 00:15:17 +0000 Subject: [PATCH 21/55] Fix None check for fallback strategy Signed-off-by: ryanaoleary --- python/ray/serve/_private/deployment_scheduler.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/serve/_private/deployment_scheduler.py b/python/ray/serve/_private/deployment_scheduler.py index 4b5b46b3454d..c9f1a3c79882 100644 --- a/python/ray/serve/_private/deployment_scheduler.py +++ b/python/ray/serve/_private/deployment_scheduler.py @@ -748,7 +748,7 @@ def schedule( fallback_labels = fallback.get("bundle_label_selector", []) strategies_to_try.append((req_resources, fallback_labels)) - elif "fallback_strategy" in scheduling_request.actor_options: + elif scheduling_request.actor_options.get("fallback_strategy"): # Fallback strategy provided for Ray Actor. for fallback in scheduling_request.actor_options[ "fallback_strategy" From fe1eb88ef519f637b95f949520d76ed53d9c1ee7 Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Wed, 7 Jan 2026 02:59:10 +0000 Subject: [PATCH 22/55] pass bundle_label_selector through config override path Signed-off-by: ryanaoleary --- .../ray/serve/_private/application_state.py | 5 +++++ .../tests/unit/test_application_state.py | 21 +++++++++++++++++++ 2 files changed, 26 insertions(+) diff --git a/python/ray/serve/_private/application_state.py b/python/ray/serve/_private/application_state.py index f3912f6599ad..629ccf08de3e 100644 --- a/python/ray/serve/_private/application_state.py +++ b/python/ray/serve/_private/application_state.py @@ -1707,6 +1707,10 @@ def override_deployment_info( override_max_replicas_per_node = options.pop( "max_replicas_per_node", replica_config.max_replicas_per_node ) + override_bundle_label_selector = options.pop( + "bundle_label_selector", + replica_config.placement_group_bundle_label_selector, + ) # Record telemetry for container runtime env feature at deployment level if override_actor_options.get("runtime_env") and ( @@ -1725,6 +1729,7 @@ def override_deployment_info( placement_group_bundles=override_placement_group_bundles, placement_group_strategy=override_placement_group_strategy, max_replicas_per_node=override_max_replicas_per_node, + placement_group_bundle_label_selector=override_bundle_label_selector, ) override_options["replica_config"] = replica_config diff --git a/python/ray/serve/tests/unit/test_application_state.py b/python/ray/serve/tests/unit/test_application_state.py index 67f8c57ab852..0ae3f966cf30 100644 --- a/python/ray/serve/tests/unit/test_application_state.py +++ b/python/ray/serve/tests/unit/test_application_state.py @@ -1496,6 +1496,27 @@ def test_override_ray_actor_options_5(self): == "s3://B" ) + def test_override_bundle_label_selector(self, info): + """Test bundle_label_selector is propagated from config.""" + config = ServeApplicationSchema( + name="default", + import_path="test.import.path", + deployments=[ + DeploymentSchema( + name="A", + placement_group_bundles=[{"CPU": 1}], + bundle_label_selector=[{"accelerator-type": "A100"}], + ) + ], + ) + + updated_infos = override_deployment_info({"A": info}, config) + updated_info = updated_infos["A"] + + assert updated_info.replica_config.placement_group_bundle_label_selector == [ + {"accelerator-type": "A100"} + ] + class TestAutoscale: def test_autoscale(self, mocked_application_state_manager): From f4e90a459555f6a28120dfe7b1f98ba1b3a17c9d Mon Sep 17 00:00:00 2001 From: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> Date: Tue, 6 Jan 2026 19:00:52 -0800 Subject: [PATCH 23/55] Update python/ray/serve/schema.py Co-authored-by: Abrar Sheikh Signed-off-by: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> --- python/ray/serve/schema.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/serve/schema.py b/python/ray/serve/schema.py index 73fb77c1a402..99cb1ac6c1aa 100644 --- a/python/ray/serve/schema.py +++ b/python/ray/serve/schema.py @@ -408,7 +408,7 @@ class DeploymentSchema(BaseModel, allow_population_by_field_name=True): ), ) - bundle_label_selector: List[Dict[str, str]] = Field( + placement_group_bundle_label_selector: List[Dict[str, str]] = Field( default=DEFAULT.VALUE, description=( "A list of label selectors to apply to the placement group " From fc52d16b8ace734bca42c7697b2291d70112a6c1 Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Wed, 7 Jan 2026 03:46:54 +0000 Subject: [PATCH 24/55] Add TODO comments Signed-off-by: ryanaoleary --- python/ray/serve/api.py | 2 ++ python/ray/serve/deployment.py | 2 ++ python/ray/serve/schema.py | 3 +++ 3 files changed, 7 insertions(+) diff --git a/python/ray/serve/api.py b/python/ray/serve/api.py index ed597689e22a..025b2e90a5a2 100644 --- a/python/ray/serve/api.py +++ b/python/ray/serve/api.py @@ -505,6 +505,8 @@ def decorator(_func_or_class): if placement_group_bundle_label_selector is not DEFAULT.VALUE else None ), + # TODO(ryanaoleary@): add placement_group_fallback_strategy when + # fallback_strategy support is added to placement group options. max_replicas_per_node=( max_replicas_per_node if max_replicas_per_node is not DEFAULT.VALUE diff --git a/python/ray/serve/deployment.py b/python/ray/serve/deployment.py index 36b37b381324..f3dc5db45073 100644 --- a/python/ray/serve/deployment.py +++ b/python/ray/serve/deployment.py @@ -349,6 +349,8 @@ def options( self._replica_config.placement_group_bundle_label_selector ) + # TODO(ryanaoleary@): Add conditional check once fallback_strategy is + # added to placement group options. placement_group_fallback_strategy = ( self._replica_config.placement_group_fallback_strategy ) diff --git a/python/ray/serve/schema.py b/python/ray/serve/schema.py index 99cb1ac6c1aa..33185a38bcc4 100644 --- a/python/ray/serve/schema.py +++ b/python/ray/serve/schema.py @@ -416,6 +416,9 @@ class DeploymentSchema(BaseModel, allow_population_by_field_name=True): ), ) + # TODO(ryanaoleary@): Support placement_group_fallback_strategy here when + # support is added for that field to placement group options. + max_replicas_per_node: int = Field( default=DEFAULT.VALUE, description=( From b84488495a3ad63d4f83c3857faa9300b5a8d9e3 Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Wed, 7 Jan 2026 03:51:27 +0000 Subject: [PATCH 25/55] Fix field name that gets popped Signed-off-by: ryanaoleary --- python/ray/serve/_private/application_state.py | 2 +- python/ray/serve/schema.py | 4 +++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/python/ray/serve/_private/application_state.py b/python/ray/serve/_private/application_state.py index 629ccf08de3e..a5f25a9f6a40 100644 --- a/python/ray/serve/_private/application_state.py +++ b/python/ray/serve/_private/application_state.py @@ -1708,7 +1708,7 @@ def override_deployment_info( "max_replicas_per_node", replica_config.max_replicas_per_node ) override_bundle_label_selector = options.pop( - "bundle_label_selector", + "placement_group_bundle_label_selector", replica_config.placement_group_bundle_label_selector, ) diff --git a/python/ray/serve/schema.py b/python/ray/serve/schema.py index 33185a38bcc4..881b72dbb0ec 100644 --- a/python/ray/serve/schema.py +++ b/python/ray/serve/schema.py @@ -477,7 +477,9 @@ def validate_max_replicas_per_node_and_placement_group_bundles(cls, values): @root_validator def validate_bundle_label_selector(cls, values): placement_group_bundles = values.get("placement_group_bundles", None) - bundle_label_selector = values.get("bundle_label_selector", None) + bundle_label_selector = values.get( + "placement_group_bundle_label_selector", None + ) if bundle_label_selector not in [DEFAULT.VALUE, None]: if placement_group_bundles in [DEFAULT.VALUE, None]: From 47ed24d29124323e660037bff49f1095d188d82b Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Wed, 7 Jan 2026 05:51:38 +0000 Subject: [PATCH 26/55] Refactor schedule function to be more clear Signed-off-by: ryanaoleary --- .../serve/_private/deployment_scheduler.py | 232 ++++++++++-------- .../tests/unit/test_deployment_scheduler.py | 87 +++++++ 2 files changed, 215 insertions(+), 104 deletions(-) diff --git a/python/ray/serve/_private/deployment_scheduler.py b/python/ray/serve/_private/deployment_scheduler.py index c9f1a3c79882..108fb3977811 100644 --- a/python/ray/serve/_private/deployment_scheduler.py +++ b/python/ray/serve/_private/deployment_scheduler.py @@ -664,16 +664,14 @@ def schedule( Returns: The IDs of replicas to stop for each deployment. """ + # Update pending replicas from upscales. for upscale in upscales.values(): for scheduling_request in upscale: replica_id = scheduling_request.replica_id deployment_id = replica_id.deployment_id self._pending_replicas[deployment_id][replica_id] = scheduling_request - non_strict_pack_pgs_exist = any( - d.is_non_strict_pack_pg() for d in self._deployments.values() - ) - # Schedule replicas using compact strategy. + # Check for deprecated environment variable usage if RAY_SERVE_USE_COMPACT_SCHEDULING_STRATEGY: warnings.warn( "The environment variable 'RAY_SERVE_USE_COMPACT_SCHEDULING_STRATEGY' " @@ -682,110 +680,21 @@ def schedule( DeprecationWarning, stacklevel=2, ) - if RAY_SERVE_USE_PACK_SCHEDULING_STRATEGY and not non_strict_pack_pgs_exist: - # Flatten dict of deployment replicas into all replicas, - # then sort by decreasing resource size - all_scheduling_requests = sorted( - _flatten(self._pending_replicas).values(), - key=lambda r: r.required_resources, - reverse=True, - ) - # Fetch node labels for active nodes. - active_nodes = self._cluster_node_info_cache.get_active_node_ids() - all_node_labels = { - node_id: self._cluster_node_info_cache.get_node_labels(node_id) - for node_id in active_nodes - } - - # Schedule each replica - for scheduling_request in all_scheduling_requests: - # Collect a list of required resources and labels to try to schedule to - # support replica compaction when fallback strategies are provided. - strategies_to_try = [] - - primary_labels = [] - primary_bundles = scheduling_request.placement_group_bundles - - if primary_bundles: - # PG: Use PG bundle_label_selector - if scheduling_request.placement_group_bundle_label_selector: - pg_strategy = ( - scheduling_request.placement_group_strategy or "PACK" - ) - if pg_strategy == "STRICT_PACK": - # All bundle_label_selectors must be satisfied on same node. - primary_labels = ( - scheduling_request.placement_group_bundle_label_selector - ) - else: - # Only the first bundle (where the replica actor lives) must be satisfied. - primary_labels = [ - scheduling_request.placement_group_bundle_label_selector[ - 0 - ] - ] - else: - # Actor: Use Actor label selector - if "label_selector" in scheduling_request.actor_options: - primary_labels = [ - scheduling_request.actor_options["label_selector"] or {} - ] - - strategies_to_try.append( - (scheduling_request.required_resources, primary_labels) - ) - - if scheduling_request.placement_group_fallback_strategy: - # Fallback strategy provided for placement group. - for ( - fallback - ) in scheduling_request.placement_group_fallback_strategy: - fallback_bundles = fallback.get("bundles", primary_bundles) - req_resources = sum( - [Resources(b) for b in fallback_bundles], Resources() - ) - - fallback_labels = fallback.get("bundle_label_selector", []) - strategies_to_try.append((req_resources, fallback_labels)) - elif scheduling_request.actor_options.get("fallback_strategy"): - # Fallback strategy provided for Ray Actor. - for fallback in scheduling_request.actor_options[ - "fallback_strategy" - ]: - fallback_labels = [fallback.get("label_selector", {}) or {}] - strategies_to_try.append( - (scheduling_request.required_resources, fallback_labels) - ) - - target_node = None - for res, labels in strategies_to_try: - target_node = self._find_best_available_node( - res, - self._get_available_resources_per_node(), - required_labels_list=labels, - node_labels=all_node_labels, - ) - if target_node: - break - - self._schedule_replica( - scheduling_request, - default_scheduling_strategy="DEFAULT", - target_node_id=target_node, - ) + # Determine scheduling strategy + non_strict_pack_pgs_exist = any( + d.is_non_strict_pack_pg() for d in self._deployments.values() + ) + use_pack_strategy = ( + RAY_SERVE_USE_PACK_SCHEDULING_STRATEGY and not non_strict_pack_pgs_exist + ) + if use_pack_strategy: + self._schedule_with_pack_strategy() else: - for pending_replicas in self._pending_replicas.values(): - if not pending_replicas: - continue - - for scheduling_request in list(pending_replicas.values()): - self._schedule_replica( - scheduling_request=scheduling_request, - default_scheduling_strategy="SPREAD", - ) + self._schedule_with_spread_strategy() + # Handle downscales deployment_to_replicas_to_stop = {} for downscale in downscales.values(): deployment_to_replicas_to_stop[ @@ -796,6 +705,121 @@ def schedule( return deployment_to_replicas_to_stop + def _schedule_with_pack_strategy(self): + """Tries to schedule pending replicas using PACK strategy.""" + # Flatten dict of deployment replicas into all replicas, + # then sort by decreasing resource size + all_scheduling_requests = sorted( + _flatten(self._pending_replicas).values(), + key=lambda r: r.required_resources, + reverse=True, + ) + + # Fetch node labels for active nodes. + active_nodes = self._cluster_node_info_cache.get_active_node_ids() + all_node_labels = { + node_id: self._cluster_node_info_cache.get_node_labels(node_id) + for node_id in active_nodes + } + + for scheduling_request in all_scheduling_requests: + self._process_single_request(scheduling_request, all_node_labels) + + def _schedule_with_spread_strategy(self): + """Tries to schedule pending replicas using the SPREAD strategy.""" + for pending_replicas in self._pending_replicas.values(): + if not pending_replicas: + continue + + for scheduling_request in list(pending_replicas.values()): + self._schedule_replica( + scheduling_request=scheduling_request, + default_scheduling_strategy="SPREAD", + ) + + def _process_single_request( + self, + scheduling_request: ReplicaSchedulingRequest, + all_node_labels: Dict[str, Dict[str, str]], + ): + """Attempts to schedule a single request on the best available node.""" + + strategies_to_try = self._get_strategies_to_try(scheduling_request) + + target_node = None + for res, labels in strategies_to_try: + target_node = self._find_best_available_node( + res, + self._get_available_resources_per_node(), + required_labels_list=labels, + node_labels=all_node_labels, + ) + if target_node: + break + + self._schedule_replica( + scheduling_request, + default_scheduling_strategy="DEFAULT", + target_node_id=target_node, + ) + + def _get_strategies_to_try( + self, scheduling_request: ReplicaSchedulingRequest + ) -> List[Tuple[Resources, List[Dict[str, str]]]]: + """Returns a list of (resources, labels) tuples to attempt for scheduling.""" + + # Collect a list of required resources and labels to try to schedule to + # support replica compaction when fallback strategies are provided. + strategies_to_try = [] + primary_labels = [] + primary_bundles = scheduling_request.placement_group_bundles + + if primary_bundles: + # PG: Use PG bundle_label_selector + if scheduling_request.placement_group_bundle_label_selector: + pg_strategy = scheduling_request.placement_group_strategy or "PACK" + if pg_strategy == "STRICT_PACK": + # All bundle_label_selectors must be satisfied on same node. + primary_labels = ( + scheduling_request.placement_group_bundle_label_selector + ) + else: + # Only the first bundle (where the replica actor lives) must be satisfied. + primary_labels = [ + scheduling_request.placement_group_bundle_label_selector[0] + ] + else: + # Actor: Use Actor label selector + if "label_selector" in scheduling_request.actor_options: + primary_labels = [ + scheduling_request.actor_options["label_selector"] or {} + ] + + strategies_to_try.append( + (scheduling_request.required_resources, primary_labels) + ) + + if scheduling_request.placement_group_fallback_strategy: + # Fallback strategy provided for placement group. + for fallback in scheduling_request.placement_group_fallback_strategy: + fallback_bundles = fallback.get("bundles", primary_bundles) + req_resources = sum( + [Resources(b) for b in fallback_bundles], Resources() + ) + + fallback_labels = fallback.get("bundle_label_selector", []) + strategies_to_try.append((req_resources, fallback_labels)) + + elif scheduling_request.actor_options.get("fallback_strategy"): + # Fallback strategy provided for Ray Actor. + for fallback in scheduling_request.actor_options["fallback_strategy"]: + fallback_labels = [fallback.get("label_selector", {}) or {}] + strategies_to_try.append( + (scheduling_request.required_resources, fallback_labels) + ) + + return strategies_to_try + def _get_replicas_to_stop( self, deployment_id: DeploymentID, max_num_to_stop: int ) -> Set[ReplicaID]: diff --git a/python/ray/serve/tests/unit/test_deployment_scheduler.py b/python/ray/serve/tests/unit/test_deployment_scheduler.py index ea36fbe2f7af..4ba438d8e7f8 100644 --- a/python/ray/serve/tests/unit/test_deployment_scheduler.py +++ b/python/ray/serve/tests/unit/test_deployment_scheduler.py @@ -974,6 +974,93 @@ def __init__(self): assert set(filtered.keys()) == {"n1", "n2"} +def test_get_strategies_to_try(): + """Test strategy generation logic in DefaultDeploymentScheduler._get_strategies_to_try, + verifying that the scheduler correctly generates a list of (resources, labels) tuples to + attempt for scheduling.""" + + # Setup scheduler with mocks + cluster_node_info_cache = MockClusterNodeInfoCache() + scheduler = default_impl.create_deployment_scheduler( + cluster_node_info_cache, + head_node_id_override="head_node", + create_placement_group_fn_override=None, + ) + + # Basic Ray Actor + req_basic = ReplicaSchedulingRequest( + replica_id=ReplicaID("r1", DeploymentID(name="d1")), + actor_def=MockActorClass(), + actor_resources={"CPU": 1}, + actor_options={}, + actor_init_args=(), + on_scheduled=Mock(), + ) + strategies = scheduler._get_strategies_to_try(req_basic) + assert len(strategies) == 1 + assert strategies[0][0] == {"CPU": 1} + assert strategies[0][1] == [] + + # Actor with label_selector and fallback_strategy + req_fallback = ReplicaSchedulingRequest( + replica_id=ReplicaID("r2", DeploymentID(name="d1")), + actor_def=MockActorClass(), + actor_resources={"CPU": 1}, + actor_options={ + "label_selector": {"region": "us-west"}, + "fallback_strategy": [{"label_selector": {"region": "us-east"}}], + }, + actor_init_args=(), + on_scheduled=Mock(), + ) + strategies = scheduler._get_strategies_to_try(req_fallback) + assert len(strategies) == 2 + + assert strategies[0][0] == {"CPU": 1} + assert strategies[0][1] == [{"region": "us-west"}] + assert strategies[1][0] == {"CPU": 1} + assert strategies[1][1] == [{"region": "us-east"}] + + # Scheduling replica with placement group PACK strategy and bundle_label_selector + req_pack = ReplicaSchedulingRequest( + replica_id=ReplicaID("r4", DeploymentID(name="d1")), + actor_def=MockActorClass(), + actor_resources={"CPU": 0.1}, + actor_options={}, + actor_init_args=(), + on_scheduled=Mock(), + placement_group_bundles=[{"CPU": 5}], + placement_group_strategy="PACK", + placement_group_bundle_label_selector=[ + {"accelerator-type": "H100"}, + {"accelerator-type": "H100"}, + ], + ) + strategies = scheduler._get_strategies_to_try(req_pack) + assert len(strategies) == 1 + + assert strategies[0][0] == {"CPU": 0.1} + assert strategies[0][1] == [{"accelerator-type": "H100"}] + + # Scheduling replica with placement group STRICT_PACK strategy and bundle_label_selector + req_pg = ReplicaSchedulingRequest( + replica_id=ReplicaID("r3", DeploymentID(name="d1")), + actor_def=MockActorClass(), + actor_resources={}, + actor_options={}, + actor_init_args=(), + on_scheduled=Mock(), + placement_group_bundles=[{"CPU": 2}], + placement_group_strategy="STRICT_PACK", + placement_group_bundle_label_selector=[{"accelerator-type": "A100"}], + ) + strategies = scheduler._get_strategies_to_try(req_pg) + assert len(strategies) == 1 + + assert strategies[0][0] == {"CPU": 2} + assert strategies[0][1] == [{"accelerator-type": "A100"}] + + @pytest.mark.skipif( not RAY_SERVE_USE_PACK_SCHEDULING_STRATEGY, reason="Needs pack strategy." ) From 4f438f46a502538b2f151cc94464cdfc2b1675d1 Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Thu, 8 Jan 2026 01:02:57 +0000 Subject: [PATCH 27/55] Add unit test coverage for bundle_label_selector and fallback_strategy validation Signed-off-by: ryanaoleary --- python/ray/serve/tests/unit/test_config.py | 81 ++++++++++++++++++++++ 1 file changed, 81 insertions(+) diff --git a/python/ray/serve/tests/unit/test_config.py b/python/ray/serve/tests/unit/test_config.py index f348cf1d27ca..fb0b59f43d5a 100644 --- a/python/ray/serve/tests/unit/test_config.py +++ b/python/ray/serve/tests/unit/test_config.py @@ -580,6 +580,87 @@ def f(): assert config.init_args == tuple() assert config.init_kwargs == dict() + def test_placement_group_bundle_label_selector_validation(self): + class Class: + pass + + # Label selector provided without bundles + with pytest.raises( + ValueError, + match="If `placement_group_bundle_label_selector` is provided, `placement_group_bundles` must also be provided.", + ): + ReplicaConfig.create( + Class, + tuple(), + dict(), + placement_group_bundle_label_selector=[{"gpu": "T4"}], + ) + + # Valid config + config = ReplicaConfig.create( + Class, + tuple(), + dict(), + placement_group_bundles=[{"CPU": 1}], + placement_group_bundle_label_selector=[{"gpu": "T4"}], + ) + assert config.placement_group_bundle_label_selector == [{"gpu": "T4"}] + + def test_placement_group_fallback_strategy_validation(self): + class Class: + pass + + # Validate that fallback strategy provided without bundles raises error. + with pytest.raises( + ValueError, + match="If `placement_group_fallback_strategy` is provided, `placement_group_bundles` must also be provided.", + ): + ReplicaConfig.create( + Class, + tuple(), + dict(), + placement_group_fallback_strategy=[{"bundles": [{"CPU": 1}]}], + ) + + # Validate that fallback strategy is a list + with pytest.raises( + TypeError, + match="placement_group_fallback_strategy must be a list of dictionaries.", + ): + ReplicaConfig.create( + Class, + tuple(), + dict(), + placement_group_bundles=[{"CPU": 1}], + placement_group_fallback_strategy="not_a_list", + ) + + # Fallback strategy list contains non-dict items + with pytest.raises( + TypeError, + match="placement_group_fallback_strategy entry at index 1 must be a dictionary.", + ): + ReplicaConfig.create( + Class, + tuple(), + dict(), + placement_group_bundles=[{"CPU": 1}], + placement_group_fallback_strategy=[ + {"bundles": [{"CPU": 1}]}, + "invalid_entry", + ], + ) + + # Valid config + config = ReplicaConfig.create( + Class, + tuple(), + dict(), + placement_group_bundles=[{"CPU": 1}], + placement_group_fallback_strategy=[{"bundles": [{"CPU": 1}]}], + ) + assert config.placement_group_fallback_strategy == [{"bundles": [{"CPU": 1}]}] + class TestAutoscalingConfig: def test_target_ongoing_requests(self): From 485dde184561ad6aa440942cca8157d969a53c9a Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Thu, 8 Jan 2026 01:16:59 +0000 Subject: [PATCH 28/55] Refactor is_scaled_copy_of for readability and add attribute checks Signed-off-by: ryanaoleary --- python/ray/serve/_private/deployment_state.py | 78 ++++++++++++++----- 1 file changed, 58 insertions(+), 20 deletions(-) diff --git a/python/ray/serve/_private/deployment_state.py b/python/ray/serve/_private/deployment_state.py index 32f8f62f1a3d..4b58350879ea 100644 --- a/python/ray/serve/_private/deployment_state.py +++ b/python/ray/serve/_private/deployment_state.py @@ -162,28 +162,66 @@ def is_scaled_copy_of(self, other_target_state: "DeploymentTargetState") -> bool if other_target_state.info is None: return False + if self.info is None: + return False + + actor_options_match = ( + self.info.replica_config.ray_actor_options + == other_target_state.info.replica_config.ray_actor_options + ) + bundles_match = ( + self.info.replica_config.placement_group_bundles + == other_target_state.info.replica_config.placement_group_bundles + ) + strategy_match = ( + self.info.replica_config.placement_group_strategy + == other_target_state.info.replica_config.placement_group_strategy + ) + max_replicas_match = ( + self.info.replica_config.max_replicas_per_node + == other_target_state.info.replica_config.max_replicas_per_node + ) + deployment_config_match = self.info.deployment_config.dict( + exclude={"num_replicas"} + ) == other_target_state.info.deployment_config.dict(exclude={"num_replicas"}) + + # Backward compatibility check for older versions of Ray without these fields. + current_bundle_label_selector = getattr( + self.info.replica_config, "placement_group_bundle_label_selector", None + ) + other_bundle_label_selector = getattr( + other_target_state.info.replica_config, + "placement_group_bundle_label_selector", + None, + ) + bundle_label_selector_match = ( + current_bundle_label_selector == other_bundle_label_selector + ) + + current_fallback = getattr( + self.info.replica_config, "placement_group_fallback_strategy", None + ) + other_fallback = getattr( + other_target_state.info.replica_config, + "placement_group_fallback_strategy", + None, + ) + fallback_match = current_fallback == other_fallback + + # TODO(zcin): version can be None, this is from an outdated codepath. + # We should remove outdated code, so version can never be None. + version_match = self.version == other_target_state.version + return all( [ - self.info.replica_config.ray_actor_options - == other_target_state.info.replica_config.ray_actor_options, - self.info.replica_config.placement_group_bundles - == other_target_state.info.replica_config.placement_group_bundles, - self.info.replica_config.placement_group_strategy - == other_target_state.info.replica_config.placement_group_strategy, - self.info.replica_config.placement_group_bundle_label_selector - == other_target_state.info.replica_config.placement_group_bundle_label_selector, - self.info.replica_config.placement_group_fallback_strategy - == other_target_state.info.replica_config.placement_group_fallback_strategy, - self.info.replica_config.max_replicas_per_node - == other_target_state.info.replica_config.max_replicas_per_node, - self.info.deployment_config.dict(exclude={"num_replicas"}) - == other_target_state.info.deployment_config.dict( - exclude={"num_replicas"} - ), - # TODO(zcin): version can be None, this is from an outdated codepath. - # We should remove outdated code, so version can never be None. - self.version, - self.version == other_target_state.version, + actor_options_match, + bundles_match, + strategy_match, + bundle_label_selector_match, + fallback_match, + max_replicas_match, + deployment_config_match, + version_match, ] ) From 38d4d30e3754878122f898994aa6737953412fb7 Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Thu, 8 Jan 2026 02:10:52 +0000 Subject: [PATCH 29/55] make variable assignments more readable Signed-off-by: ryanaoleary --- python/ray/serve/_private/version.py | 52 ++++++++++++++++++---------- 1 file changed, 33 insertions(+), 19 deletions(-) diff --git a/python/ray/serve/_private/version.py b/python/ray/serve/_private/version.py index be375355db8a..e9c2c080f80b 100644 --- a/python/ray/serve/_private/version.py +++ b/python/ray/serve/_private/version.py @@ -145,29 +145,43 @@ def compute_hashes(self): def to_proto(self) -> bytes: # TODO(simon): enable cross language user config + + placement_group_bundles = ( + json.dumps(self.placement_group_bundles) + if self.placement_group_bundles is not None + else "" + ) + + placement_group_bundle_label_selector = ( + json.dumps(self.placement_group_bundle_label_selector) + if self.placement_group_bundle_label_selector is not None + else "" + ) + + placement_group_fallback_strategy = ( + json.dumps(self.placement_group_fallback_strategy) + if self.placement_group_fallback_strategy is not None + else "" + ) + + placement_group_strategy = ( + self.placement_group_strategy + if self.placement_group_strategy is not None + else "" + ) + max_replicas_per_node = ( + self.max_replicas_per_node if self.max_replicas_per_node is not None else 0 + ) + return DeploymentVersionProto( code_version=self.code_version, deployment_config=self.deployment_config.to_proto(), ray_actor_options=json.dumps(self.ray_actor_options), - placement_group_bundles=json.dumps(self.placement_group_bundles) - if self.placement_group_bundles is not None - else "", - placement_group_strategy=self.placement_group_strategy - if self.placement_group_strategy is not None - else "", - placement_group_bundle_label_selector=json.dumps( - self.placement_group_bundle_label_selector - ) - if self.placement_group_bundle_label_selector is not None - else "", - placement_group_fallback_strategy=json.dumps( - self.placement_group_fallback_strategy - ) - if self.placement_group_fallback_strategy is not None - else "", - max_replicas_per_node=self.max_replicas_per_node - if self.max_replicas_per_node is not None - else 0, + placement_group_bundles=placement_group_bundles, + placement_group_strategy=placement_group_strategy, + placement_group_bundle_label_selector=placement_group_bundle_label_selector, + placement_group_fallback_strategy=placement_group_fallback_strategy, + max_replicas_per_node=max_replicas_per_node, ) @classmethod From 06c23f52f132dc08cb8eb776ff860b07fdda55d2 Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Thu, 8 Jan 2026 03:06:39 +0000 Subject: [PATCH 30/55] Prevent fallback_strategy from getting overwritten Signed-off-by: ryanaoleary --- .../ray/serve/_private/application_state.py | 5 ++ .../tests/unit/test_application_state.py | 47 ++++++++++++++++++- 2 files changed, 50 insertions(+), 2 deletions(-) diff --git a/python/ray/serve/_private/application_state.py b/python/ray/serve/_private/application_state.py index a5f25a9f6a40..ca1e443d56d2 100644 --- a/python/ray/serve/_private/application_state.py +++ b/python/ray/serve/_private/application_state.py @@ -1711,6 +1711,10 @@ def override_deployment_info( "placement_group_bundle_label_selector", replica_config.placement_group_bundle_label_selector, ) + override_fallback_strategy = options.pop( + "placement_group_fallback_strategy", + replica_config.placement_group_fallback_strategy, + ) # Record telemetry for container runtime env feature at deployment level if override_actor_options.get("runtime_env") and ( @@ -1730,6 +1734,7 @@ def override_deployment_info( placement_group_strategy=override_placement_group_strategy, max_replicas_per_node=override_max_replicas_per_node, placement_group_bundle_label_selector=override_bundle_label_selector, + placement_group_fallback_strategy=override_fallback_strategy, ) override_options["replica_config"] = replica_config diff --git a/python/ray/serve/tests/unit/test_application_state.py b/python/ray/serve/tests/unit/test_application_state.py index 0ae3f966cf30..3510930f5419 100644 --- a/python/ray/serve/tests/unit/test_application_state.py +++ b/python/ray/serve/tests/unit/test_application_state.py @@ -1497,7 +1497,7 @@ def test_override_ray_actor_options_5(self): ) def test_override_bundle_label_selector(self, info): - """Test bundle_label_selector is propagated from config.""" + """Test placement_group_bundle_label_selector is propagated from config.""" config = ServeApplicationSchema( name="default", import_path="test.import.path", @@ -1505,7 +1505,9 @@ def test_override_bundle_label_selector(self, info): DeploymentSchema( name="A", placement_group_bundles=[{"CPU": 1}], - bundle_label_selector=[{"accelerator-type": "A100"}], + placement_group_bundle_label_selector=[ + {"accelerator-type": "A100"} + ], ) ], ) @@ -1517,6 +1519,47 @@ def test_override_bundle_label_selector(self, info): {"accelerator-type": "A100"} ] + def test_override_fallback_strategy(self, info): + """Test placement_group_fallback_strategy is preserved when config is updated. + + placement_group_fallback_strategy is not yet part of the public DeploymentSchema, + so we cannot set it via the override config. Instead, we verify that an existing + value in the ReplicaConfig is preserved when other fields are updated via the config. + """ + initial_info = DeploymentInfo( + route_prefix="/", + version="123", + deployment_config=DeploymentConfig(num_replicas=1), + replica_config=ReplicaConfig.create( + lambda x: x, + placement_group_bundles=[{"CPU": 1}], + placement_group_fallback_strategy=[ + {"placement_group_bundles": [{"CPU": 1}]} + ], + ), + start_time_ms=0, + deployer_job_id="", + ) + + config = ServeApplicationSchema( + name="default", + import_path="test.import.path", + deployments=[ + DeploymentSchema( + name="A", + num_replicas=5, # Update a different field + ) + ], + ) + + updated_infos = override_deployment_info({"A": initial_info}, config) + updated_info = updated_infos["A"] + + assert updated_info.deployment_config.num_replicas == 5 + assert updated_info.replica_config.placement_group_fallback_strategy == [ + {"placement_group_bundles": [{"CPU": 1}]} + ] + class TestAutoscale: def test_autoscale(self, mocked_application_state_manager): From 4a771c1f468317d0da0556eb667b914e43fa1b65 Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Thu, 8 Jan 2026 03:07:08 +0000 Subject: [PATCH 31/55] Fix explicit null bundles in fallback causes crash Signed-off-by: ryanaoleary --- python/ray/serve/_private/deployment_scheduler.py | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/python/ray/serve/_private/deployment_scheduler.py b/python/ray/serve/_private/deployment_scheduler.py index 108fb3977811..d4821318b9b9 100644 --- a/python/ray/serve/_private/deployment_scheduler.py +++ b/python/ray/serve/_private/deployment_scheduler.py @@ -802,7 +802,13 @@ def _get_strategies_to_try( if scheduling_request.placement_group_fallback_strategy: # Fallback strategy provided for placement group. for fallback in scheduling_request.placement_group_fallback_strategy: - fallback_bundles = fallback.get("bundles", primary_bundles) + fallback_bundles = fallback.get("bundles") + if fallback_bundles is None: + fallback_bundles = primary_bundles + + if fallback_bundles is None: + fallback_bundles = [] + req_resources = sum( [Resources(b) for b in fallback_bundles], Resources() ) From 5fc10e7007d13d691164a530ca27128783ab52a3 Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Thu, 8 Jan 2026 03:14:51 +0000 Subject: [PATCH 32/55] Add validation test for validate_bundle_label_selector Signed-off-by: ryanaoleary --- python/ray/serve/tests/unit/test_schema.py | 29 ++++++++++++++++++++++ 1 file changed, 29 insertions(+) diff --git a/python/ray/serve/tests/unit/test_schema.py b/python/ray/serve/tests/unit/test_schema.py index 2cbda98eac8e..b6c7881b595c 100644 --- a/python/ray/serve/tests/unit/test_schema.py +++ b/python/ray/serve/tests/unit/test_schema.py @@ -382,6 +382,35 @@ def test_num_replicas_nullable(self): } DeploymentSchema.parse_obj(deployment_options) + def test_validate_bundle_label_selector(self): + """Test validation for placement_group_bundle_label_selector.""" + + deployment_schema = self.get_minimal_deployment_schema() + + # Validate bundle_label_selector provided without bundles raises. + deployment_schema["placement_group_bundle_label_selector"] = [{"a": "b"}] + with pytest.raises( + ValidationError, + match="Setting bundle_label_selector is not allowed when placement_group_bundles is not provided", + ): + DeploymentSchema.parse_obj(deployment_schema) + + # Validate mismatched lengths for bundles and bundle_label_selector raises. + deployment_schema["placement_group_bundles"] = [{"CPU": 1}, {"CPU": 1}] + deployment_schema["placement_group_bundle_label_selector"] = [{"a": "b"}] + with pytest.raises( + ValidationError, + match="must have the same length as placement_group_bundles", + ): + DeploymentSchema.parse_obj(deployment_schema) + + # Valid config + deployment_schema["placement_group_bundle_label_selector"] = [ + {"a": "b"}, + {"c": "d"}, + ] + DeploymentSchema.parse_obj(deployment_schema) + class TestServeApplicationSchema: def get_valid_serve_application_schema(self): From cb78cc24fdfa7be627b5c84680844dd70b023eef Mon Sep 17 00:00:00 2001 From: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> Date: Wed, 7 Jan 2026 23:41:15 -0800 Subject: [PATCH 33/55] Update python/ray/serve/_private/deployment_scheduler.py Co-authored-by: Abrar Sheikh Signed-off-by: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> --- python/ray/serve/_private/deployment_scheduler.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/serve/_private/deployment_scheduler.py b/python/ray/serve/_private/deployment_scheduler.py index d4821318b9b9..bd0005720404 100644 --- a/python/ray/serve/_private/deployment_scheduler.py +++ b/python/ray/serve/_private/deployment_scheduler.py @@ -954,7 +954,7 @@ def _filter_nodes_by_labels( return filtered_nodes - def _find_best_available_node( + def _find_best_fit_node_for_pack( self, required_resources: Resources, available_resources_per_node: Dict[str, Resources], From a55af8e8841ccbdca0fd2254f8294584dac2556f Mon Sep 17 00:00:00 2001 From: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> Date: Wed, 7 Jan 2026 23:41:26 -0800 Subject: [PATCH 34/55] Update python/ray/serve/_private/deployment_scheduler.py Co-authored-by: Abrar Sheikh Signed-off-by: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> --- python/ray/serve/_private/deployment_scheduler.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/serve/_private/deployment_scheduler.py b/python/ray/serve/_private/deployment_scheduler.py index bd0005720404..fa71811b8485 100644 --- a/python/ray/serve/_private/deployment_scheduler.py +++ b/python/ray/serve/_private/deployment_scheduler.py @@ -893,7 +893,7 @@ def key(node_and_num_running_replicas_of_all_deployments): return replicas_to_stop - def _filter_nodes_by_labels( + def _filter_nodes_by_label_selector( self, available_nodes: Dict[str, Resources], required_labels: Dict[str, str], From 5e8f85ec2970e0788a7c85320596a1a540598ef9 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> Date: Wed, 7 Jan 2026 23:41:46 -0800 Subject: [PATCH 35/55] Update python/ray/serve/_private/deployment_scheduler.py Co-authored-by: Abrar Sheikh Signed-off-by: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> --- python/ray/serve/_private/deployment_scheduler.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/python/ray/serve/_private/deployment_scheduler.py b/python/ray/serve/_private/deployment_scheduler.py index fa71811b8485..8e00452341ad 100644 --- a/python/ray/serve/_private/deployment_scheduler.py +++ b/python/ray/serve/_private/deployment_scheduler.py @@ -690,6 +690,11 @@ def schedule( ) if use_pack_strategy: + # This branch is only reached if each deployment either: + # 1. Use STRICT_PACK placement group strategy, or + # 2. Do not use placement groups at all. + # This ensures Serve's best-fit node selection is respected by Ray Core + # (since _soft_target_node_id only works with STRICT_PACK). self._schedule_with_pack_strategy() else: self._schedule_with_spread_strategy() From 3128161c869686b0789a9678080667eabd6eb9da Mon Sep 17 00:00:00 2001 From: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> Date: Wed, 7 Jan 2026 23:41:54 -0800 Subject: [PATCH 36/55] Update python/ray/serve/_private/deployment_scheduler.py Co-authored-by: Abrar Sheikh Signed-off-by: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> --- python/ray/serve/_private/deployment_scheduler.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/serve/_private/deployment_scheduler.py b/python/ray/serve/_private/deployment_scheduler.py index 8e00452341ad..9fcc1c882e34 100644 --- a/python/ray/serve/_private/deployment_scheduler.py +++ b/python/ray/serve/_private/deployment_scheduler.py @@ -752,7 +752,7 @@ def _process_single_request( strategies_to_try = self._get_strategies_to_try(scheduling_request) target_node = None - for res, labels in strategies_to_try: + for required_resources, required_labels in placement_candidates: target_node = self._find_best_available_node( res, self._get_available_resources_per_node(), From 51dd7054deb462365702f9d8be0918e04ad417a1 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> Date: Thu, 8 Jan 2026 03:03:26 -0800 Subject: [PATCH 37/55] Update python/ray/serve/_private/deployment_scheduler.py Co-authored-by: Abrar Sheikh Signed-off-by: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> --- python/ray/serve/_private/deployment_scheduler.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/serve/_private/deployment_scheduler.py b/python/ray/serve/_private/deployment_scheduler.py index 9fcc1c882e34..145a3894335f 100644 --- a/python/ray/serve/_private/deployment_scheduler.py +++ b/python/ray/serve/_private/deployment_scheduler.py @@ -782,7 +782,7 @@ def _get_strategies_to_try( if primary_bundles: # PG: Use PG bundle_label_selector if scheduling_request.placement_group_bundle_label_selector: - pg_strategy = scheduling_request.placement_group_strategy or "PACK" + pg_strategy = scheduling_request.placement_group_strategy or None if pg_strategy == "STRICT_PACK": # All bundle_label_selectors must be satisfied on same node. primary_labels = ( From 45dd5aad9eaa5066f92d19173182c8743c020ffb Mon Sep 17 00:00:00 2001 From: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> Date: Thu, 8 Jan 2026 03:03:42 -0800 Subject: [PATCH 38/55] Update python/ray/serve/_private/deployment_scheduler.py Co-authored-by: Abrar Sheikh Signed-off-by: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> --- python/ray/serve/_private/deployment_scheduler.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/ray/serve/_private/deployment_scheduler.py b/python/ray/serve/_private/deployment_scheduler.py index 145a3894335f..428e0772f7f3 100644 --- a/python/ray/serve/_private/deployment_scheduler.py +++ b/python/ray/serve/_private/deployment_scheduler.py @@ -800,6 +800,7 @@ def _get_strategies_to_try( scheduling_request.actor_options["label_selector"] or {} ] + # If PG is defined on scheduling request, then `required_resources` represents the sum across all bundles. strategies_to_try.append( (scheduling_request.required_resources, primary_labels) ) From c2134082bfa73ec513b151b1fb82f04257b2746e Mon Sep 17 00:00:00 2001 From: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> Date: Thu, 8 Jan 2026 03:04:09 -0800 Subject: [PATCH 39/55] Update python/ray/serve/_private/deployment_scheduler.py Co-authored-by: Abrar Sheikh Signed-off-by: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> --- python/ray/serve/_private/deployment_scheduler.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/serve/_private/deployment_scheduler.py b/python/ray/serve/_private/deployment_scheduler.py index 428e0772f7f3..06130f770ea3 100644 --- a/python/ray/serve/_private/deployment_scheduler.py +++ b/python/ray/serve/_private/deployment_scheduler.py @@ -742,7 +742,7 @@ def _schedule_with_spread_strategy(self): default_scheduling_strategy="SPREAD", ) - def _process_single_request( + def _pack_schedule_replica( self, scheduling_request: ReplicaSchedulingRequest, all_node_labels: Dict[str, Dict[str, str]], From dc5998814b42f6ed2bdf2975cb4d1a6521071dd0 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> Date: Thu, 8 Jan 2026 03:04:16 -0800 Subject: [PATCH 40/55] Update python/ray/serve/_private/deployment_scheduler.py Co-authored-by: Abrar Sheikh Signed-off-by: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> --- python/ray/serve/_private/deployment_scheduler.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/serve/_private/deployment_scheduler.py b/python/ray/serve/_private/deployment_scheduler.py index 06130f770ea3..c9b957549988 100644 --- a/python/ray/serve/_private/deployment_scheduler.py +++ b/python/ray/serve/_private/deployment_scheduler.py @@ -768,7 +768,7 @@ def _pack_schedule_replica( target_node_id=target_node, ) - def _get_strategies_to_try( + def _build_pack_placement_candidates( self, scheduling_request: ReplicaSchedulingRequest ) -> List[Tuple[Resources, List[Dict[str, str]]]]: """Returns a list of (resources, labels) tuples to attempt for scheduling.""" From fc79d5ad92872e550ae2539a58a059942dd22e82 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> Date: Thu, 8 Jan 2026 03:04:21 -0800 Subject: [PATCH 41/55] Update python/ray/serve/_private/deployment_scheduler.py Co-authored-by: Abrar Sheikh Signed-off-by: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> --- python/ray/serve/_private/deployment_scheduler.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/serve/_private/deployment_scheduler.py b/python/ray/serve/_private/deployment_scheduler.py index c9b957549988..3e1b855f6628 100644 --- a/python/ray/serve/_private/deployment_scheduler.py +++ b/python/ray/serve/_private/deployment_scheduler.py @@ -749,7 +749,7 @@ def _pack_schedule_replica( ): """Attempts to schedule a single request on the best available node.""" - strategies_to_try = self._get_strategies_to_try(scheduling_request) + placement_candidates = self._build_pack_placement_candidates(scheduling_request) target_node = None for required_resources, required_labels in placement_candidates: From 802bd5e984538603646a1e1bb0e701b84538656e Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Fri, 9 Jan 2026 00:52:00 +0000 Subject: [PATCH 42/55] Fix naming from suggested comments Signed-off-by: ryanaoleary --- .../serve/_private/deployment_scheduler.py | 20 +++++------ .../tests/unit/test_deployment_scheduler.py | 34 +++++++++++-------- 2 files changed, 29 insertions(+), 25 deletions(-) diff --git a/python/ray/serve/_private/deployment_scheduler.py b/python/ray/serve/_private/deployment_scheduler.py index 3e1b855f6628..9f395ddbe6be 100644 --- a/python/ray/serve/_private/deployment_scheduler.py +++ b/python/ray/serve/_private/deployment_scheduler.py @@ -728,7 +728,7 @@ def _schedule_with_pack_strategy(self): } for scheduling_request in all_scheduling_requests: - self._process_single_request(scheduling_request, all_node_labels) + self._pack_schedule_replica(scheduling_request, all_node_labels) def _schedule_with_spread_strategy(self): """Tries to schedule pending replicas using the SPREAD strategy.""" @@ -753,10 +753,10 @@ def _pack_schedule_replica( target_node = None for required_resources, required_labels in placement_candidates: - target_node = self._find_best_available_node( - res, + target_node = self._find_best_fit_node_for_pack( + required_resources, self._get_available_resources_per_node(), - required_labels_list=labels, + required_labels_list=required_labels, node_labels=all_node_labels, ) if target_node: @@ -775,7 +775,7 @@ def _build_pack_placement_candidates( # Collect a list of required resources and labels to try to schedule to # support replica compaction when fallback strategies are provided. - strategies_to_try = [] + placement_candidates = [] primary_labels = [] primary_bundles = scheduling_request.placement_group_bundles @@ -801,7 +801,7 @@ def _build_pack_placement_candidates( ] # If PG is defined on scheduling request, then `required_resources` represents the sum across all bundles. - strategies_to_try.append( + placement_candidates.append( (scheduling_request.required_resources, primary_labels) ) @@ -820,17 +820,17 @@ def _build_pack_placement_candidates( ) fallback_labels = fallback.get("bundle_label_selector", []) - strategies_to_try.append((req_resources, fallback_labels)) + placement_candidates.append((req_resources, fallback_labels)) elif scheduling_request.actor_options.get("fallback_strategy"): # Fallback strategy provided for Ray Actor. for fallback in scheduling_request.actor_options["fallback_strategy"]: fallback_labels = [fallback.get("label_selector", {}) or {}] - strategies_to_try.append( + placement_candidates.append( (scheduling_request.required_resources, fallback_labels) ) - return strategies_to_try + return placement_candidates def _get_replicas_to_stop( self, deployment_id: DeploymentID, max_num_to_stop: int @@ -977,7 +977,7 @@ def _find_best_fit_node_for_pack( # Filter feasible nodes by provided label selectors if provided. if required_labels_list and node_labels: for required_labels in required_labels_list: - available_resources_per_node = self._filter_nodes_by_labels( + available_resources_per_node = self._filter_nodes_by_label_selector( available_resources_per_node, required_labels, node_labels ) if not available_resources_per_node: diff --git a/python/ray/serve/tests/unit/test_deployment_scheduler.py b/python/ray/serve/tests/unit/test_deployment_scheduler.py index 4ba438d8e7f8..ad7a3d04f12f 100644 --- a/python/ray/serve/tests/unit/test_deployment_scheduler.py +++ b/python/ray/serve/tests/unit/test_deployment_scheduler.py @@ -914,8 +914,8 @@ def wait(self, *args): assert pg_request.bundle_label_selector == test_labels -def test_filter_nodes_by_labels(): - """Test _filter_nodes_by_labels logic used by _find_best_available_node +def test_filter_nodes_by_label_selector(): + """Test _filter_nodes_by_label_selector logic used by _find_best_fit_node_for_pack when bin-packing, such that label constraints are enforced for the preferred node.""" class MockScheduler(default_impl.DefaultDeploymentScheduler): @@ -936,46 +936,50 @@ def __init__(self): } # equals operator - filtered = scheduler._filter_nodes_by_labels( + filtered = scheduler._filter_nodes_by_label_selector( nodes, {"region": "us-west"}, node_labels ) assert set(filtered.keys()) == {"n1"} # not equals operator - filtered = scheduler._filter_nodes_by_labels( + filtered = scheduler._filter_nodes_by_label_selector( nodes, {"region": "!us-west"}, node_labels ) assert set(filtered.keys()) == {"n2", "n3"} # in operator - filtered = scheduler._filter_nodes_by_labels( + filtered = scheduler._filter_nodes_by_label_selector( nodes, {"region": "in(us-west, us-east)"}, node_labels ) assert set(filtered.keys()) == {"n1", "n2"} # !in operator - filtered = scheduler._filter_nodes_by_labels( + filtered = scheduler._filter_nodes_by_label_selector( nodes, {"env": "!in(dev, staging)"}, node_labels ) assert set(filtered.keys()) == {"n1"} # Missing labels treated as not a match for equality. - filtered = scheduler._filter_nodes_by_labels(nodes, {"gpu": "A100"}, node_labels) + filtered = scheduler._filter_nodes_by_label_selector( + nodes, {"gpu": "A100"}, node_labels + ) assert set(filtered.keys()) == {"n2"} # Not equal should match node with missing labels. - filtered = scheduler._filter_nodes_by_labels(nodes, {"gpu": "!T4"}, node_labels) + filtered = scheduler._filter_nodes_by_label_selector( + nodes, {"gpu": "!T4"}, node_labels + ) assert set(filtered.keys()) == {"n2", "n3"} # Validate we handle whitespace. - filtered = scheduler._filter_nodes_by_labels( + filtered = scheduler._filter_nodes_by_label_selector( nodes, {"region": "in( us-west , us-east )"}, node_labels ) assert set(filtered.keys()) == {"n1", "n2"} -def test_get_strategies_to_try(): - """Test strategy generation logic in DefaultDeploymentScheduler._get_strategies_to_try, +def test_build_pack_placement_candidates(): + """Test strategy generation logic in DefaultDeploymentScheduler._build_pack_placement_candidates, verifying that the scheduler correctly generates a list of (resources, labels) tuples to attempt for scheduling.""" @@ -996,7 +1000,7 @@ def test_get_strategies_to_try(): actor_init_args=(), on_scheduled=Mock(), ) - strategies = scheduler._get_strategies_to_try(req_basic) + strategies = scheduler._build_pack_placement_candidates(req_basic) assert len(strategies) == 1 assert strategies[0][0] == {"CPU": 1} assert strategies[0][1] == [] @@ -1013,7 +1017,7 @@ def test_get_strategies_to_try(): actor_init_args=(), on_scheduled=Mock(), ) - strategies = scheduler._get_strategies_to_try(req_fallback) + strategies = scheduler._build_pack_placement_candidates(req_fallback) assert len(strategies) == 2 assert strategies[0][0] == {"CPU": 1} @@ -1036,7 +1040,7 @@ def test_get_strategies_to_try(): {"accelerator-type": "H100"}, ], ) - strategies = scheduler._get_strategies_to_try(req_pack) + strategies = scheduler._build_pack_placement_candidates(req_pack) assert len(strategies) == 1 assert strategies[0][0] == {"CPU": 0.1} @@ -1054,7 +1058,7 @@ def test_get_strategies_to_try(): placement_group_strategy="STRICT_PACK", placement_group_bundle_label_selector=[{"accelerator-type": "A100"}], ) - strategies = scheduler._get_strategies_to_try(req_pg) + strategies = scheduler._build_pack_placement_candidates(req_pg) assert len(strategies) == 1 assert strategies[0][0] == {"CPU": 2} From cde9d08de72002f8593c1b2753be1537424a3bf0 Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Fri, 9 Jan 2026 00:52:24 +0000 Subject: [PATCH 43/55] Fix version truthiness check Signed-off-by: ryanaoleary --- python/ray/serve/_private/deployment_state.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/python/ray/serve/_private/deployment_state.py b/python/ray/serve/_private/deployment_state.py index 5f476b0d7d6c..73e96f0d4258 100644 --- a/python/ray/serve/_private/deployment_state.py +++ b/python/ray/serve/_private/deployment_state.py @@ -209,7 +209,9 @@ def is_scaled_copy_of(self, other_target_state: "DeploymentTargetState") -> bool # TODO(zcin): version can be None, this is from an outdated codepath. # We should remove outdated code, so version can never be None. - version_match = self.version == other_target_state.version + version_match = ( + self.version is not None and self.version == other_target_state.version + ) return all( [ From 010827fdd30535ba4a1854e812bb7ec47bfe34c8 Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Fri, 9 Jan 2026 01:55:23 +0000 Subject: [PATCH 44/55] Move filter by label selector logic to label_utils Signed-off-by: ryanaoleary --- python/ray/_private/label_utils.py | 42 +++++++++++++ .../serve/_private/deployment_scheduler.py | 61 +++---------------- python/ray/tests/test_label_utils.py | 61 +++++++++++++++++++ 3 files changed, 110 insertions(+), 54 deletions(-) diff --git a/python/ray/_private/label_utils.py b/python/ray/_private/label_utils.py index 6f6de4597624..92cb0ba75129 100644 --- a/python/ray/_private/label_utils.py +++ b/python/ray/_private/label_utils.py @@ -228,3 +228,45 @@ def validate_fallback_strategy( return error_message return None + + +def match_label_selector_value(node_value: Optional[str], selector_value: str) -> bool: + """Evaluates if a node's label value matches a selector expression. + + Supports: + - Equality: "value" matches if node_value == "value" + - Not Equal: "!value" matches if node_value != "value" + - In: "in(v1, v2)" matches if node_value in ["v1", "v2"] + - Not In: "!in(v1, v2)" matches if node_value not in ["v1", "v2"] + """ + if not isinstance(selector_value, str): + return False + + # !in operator + if selector_value.startswith("!in(") and selector_value.endswith(")"): + content = selector_value[4:-1] + values = [v.strip() for v in content.split(",")] + return node_value is None or node_value not in values + + # in operator + if selector_value.startswith("in(") and selector_value.endswith(")"): + content = selector_value[3:-1] + values = [v.strip() for v in content.split(",")] + return node_value in values + + # not equal operator + if selector_value.startswith("!"): + target_val = selector_value[1:] + return node_value != target_val + + # equals operator + return node_value == selector_value + + +def match_label_selector(node_labels: Dict[str, str], selector: Dict[str, str]) -> bool: + """Returns True if node_labels satisfy all selector constraints.""" + for key, selector_value in selector.items(): + node_value = node_labels.get(key) + if not match_label_selector_value(node_value, selector_value): + return False + return True diff --git a/python/ray/serve/_private/deployment_scheduler.py b/python/ray/serve/_private/deployment_scheduler.py index 9f395ddbe6be..47a4cb6b65f7 100644 --- a/python/ray/serve/_private/deployment_scheduler.py +++ b/python/ray/serve/_private/deployment_scheduler.py @@ -10,6 +10,7 @@ from typing import Any, Callable, Dict, List, Optional, Set, Tuple import ray +from ray._private.label_utils import match_label_selector from ray.serve._private.cluster_node_info_cache import ClusterNodeInfoCache from ray.serve._private.common import ( CreatePlacementGroupRequest, @@ -905,60 +906,12 @@ def _filter_nodes_by_label_selector( required_labels: Dict[str, str], node_labels: Dict[str, Dict[str, str]], ) -> Dict[str, Resources]: - """Filters available nodes based on label selector constraints. - - Supports Ray's label syntax where values are strings: - - Equality: {"key": "value"} - - Not Equal: {"key": "!value"} - - In: {"key": "in(v1, v2, ...)"} - - Not In: {"key": "!in(v1, v2, ...)"} - """ - filtered_nodes = {} - for node_id, resources in available_nodes.items(): - labels = node_labels.get(node_id, {}) - is_match = True - - for key, req_val in required_labels.items(): - node_val = labels.get(key) - - # We only support string labels. - if not isinstance(req_val, str): - is_match = False - break - - # !in operator - if req_val.startswith("!in(") and req_val.endswith(")"): - content = req_val[4:-1] - values = [v.strip() for v in content.split(",")] - if node_val is not None and node_val in values: - is_match = False - break - - # in operator - elif req_val.startswith("in(") and req_val.endswith(")"): - content = req_val[3:-1] - values = [v.strip() for v in content.split(",")] - if node_val not in values: - is_match = False - break - - # not equal operator - elif req_val.startswith("!"): - target_val = req_val[1:] - if node_val == target_val: - is_match = False - break - - # equals operator - else: - if node_val != req_val: - is_match = False - break - - if is_match: - filtered_nodes[node_id] = resources - - return filtered_nodes + """Filters available nodes based on label selector constraints.""" + return { + node_id: resources + for node_id, resources in available_nodes.items() + if match_label_selector(node_labels.get(node_id, {}), required_labels) + } def _find_best_fit_node_for_pack( self, diff --git a/python/ray/tests/test_label_utils.py b/python/ray/tests/test_label_utils.py index 85770a7f4248..9fe390d70804 100644 --- a/python/ray/tests/test_label_utils.py +++ b/python/ray/tests/test_label_utils.py @@ -8,6 +8,8 @@ import pytest from ray._private.label_utils import ( + match_label_selector, + match_label_selector_value, parse_node_labels_from_yaml_file, parse_node_labels_json, parse_node_labels_string, @@ -368,5 +370,64 @@ def test_validate_fallback_strategy(fallback_strategy, expected_error): assert result is None +@pytest.mark.parametrize( + "node_value, selector_value, expected", + [ + # Equals operator + ("us-west", "us-west", True), + ("us-east", "us-west", False), + (None, "us-west", False), + # Not equals (!) operator + ("us-west", "!us-east", True), + ("us-east", "!us-east", False), + (None, "!us-east", True), + # In operator + ("A100", "in(A100, H100)", True), + ("T4", "in(A100, H100)", False), + (None, "in(A100, H100)", False), + ("value", "in(value)", True), + # Not in operator + ("T4", "!in(A100, H100)", True), + ("A100", "!in(A100, H100)", False), + (None, "!in(A100, H100)", True), + # Invalid types + ("A100", None, False), + ("A100", 123, False), + ], +) +def test_match_label_selector_value(node_value, selector_value, expected): + assert match_label_selector_value(node_value, selector_value) == expected + + +@pytest.mark.parametrize( + "node_labels, selector, expected", + [ + # Match all + ( + {"region": "us-west", "gpu": "A100"}, + {"region": "us-west", "gpu": "A100"}, + True, + ), + # Partial match + ( + {"region": "us-west", "gpu": "T4"}, + {"region": "us-west", "gpu": "A100"}, + False, + ), + # Multiple operators + ( + {"region": "us-west", "env": "prod"}, + {"region": "in(us-west, us-east)", "env": "!dev"}, + True, + ), + # Missing keys in node labels + ({"region": "us-west"}, {"gpu": "!in(H100)"}, True), + ({"region": "us-west"}, {"gpu": "in(H100)"}, False), + ], +) +def test_match_label_selector(node_labels, selector, expected): + assert match_label_selector(node_labels, selector) == expected + + if __name__ == "__main__": sys.exit(pytest.main(["-sv", "-vv", __file__])) From 89d43900392cfc688666f25a6f44696807a81cdb Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Fri, 9 Jan 2026 02:31:48 +0000 Subject: [PATCH 45/55] Fix key names and add NotImplementedError Signed-off-by: ryanaoleary --- .../serve/_private/deployment_scheduler.py | 28 ++++++------------- .../serve/tests/test_deployment_version.py | 2 +- .../tests/unit/test_application_state.py | 4 +-- .../tests/unit/test_deployment_scheduler.py | 7 ++--- 4 files changed, 14 insertions(+), 27 deletions(-) diff --git a/python/ray/serve/_private/deployment_scheduler.py b/python/ray/serve/_private/deployment_scheduler.py index 47a4cb6b65f7..dc618eeffae4 100644 --- a/python/ray/serve/_private/deployment_scheduler.py +++ b/python/ray/serve/_private/deployment_scheduler.py @@ -790,10 +790,11 @@ def _build_pack_placement_candidates( scheduling_request.placement_group_bundle_label_selector ) else: - # Only the first bundle (where the replica actor lives) must be satisfied. - primary_labels = [ - scheduling_request.placement_group_bundle_label_selector[0] - ] + # TODO(ryanaoleary@): Support PACK strategy with bundle label selectors. + raise NotImplementedError( + "Placement Group strategy 'PACK' with bundle_label_selector " + "is not yet supported in the Serve scheduler." + ) else: # Actor: Use Actor label selector if "label_selector" in scheduling_request.actor_options: @@ -807,21 +808,10 @@ def _build_pack_placement_candidates( ) if scheduling_request.placement_group_fallback_strategy: - # Fallback strategy provided for placement group. - for fallback in scheduling_request.placement_group_fallback_strategy: - fallback_bundles = fallback.get("bundles") - if fallback_bundles is None: - fallback_bundles = primary_bundles - - if fallback_bundles is None: - fallback_bundles = [] - - req_resources = sum( - [Resources(b) for b in fallback_bundles], Resources() - ) - - fallback_labels = fallback.get("bundle_label_selector", []) - placement_candidates.append((req_resources, fallback_labels)) + # TODO(ryanaoleary@): Add support for placement group fallback_strategy when it's added to options. + raise NotImplementedError( + "Placement Group fallback strategies are not yet supported in the Serve scheduler." + ) elif scheduling_request.actor_options.get("fallback_strategy"): # Fallback strategy provided for Ray Actor. diff --git a/python/ray/serve/tests/test_deployment_version.py b/python/ray/serve/tests/test_deployment_version.py index 7fabdbef2c5d..b52170949b10 100644 --- a/python/ray/serve/tests/test_deployment_version.py +++ b/python/ray/serve/tests/test_deployment_version.py @@ -72,7 +72,7 @@ def test_placement_group_options_trigger_restart(): ray_actor_options={}, placement_group_bundles=[{"CPU": 1}], placement_group_strategy="PACK", - placement_group_fallback_strategy=[{"placement_group_bundles": [{"CPU": 1}]}], + placement_group_fallback_strategy=[{"bundles": [{"CPU": 1}]}], ) # Validate actor restart occurs due to differing hash. diff --git a/python/ray/serve/tests/unit/test_application_state.py b/python/ray/serve/tests/unit/test_application_state.py index 3510930f5419..e117de28c7cd 100644 --- a/python/ray/serve/tests/unit/test_application_state.py +++ b/python/ray/serve/tests/unit/test_application_state.py @@ -1533,9 +1533,7 @@ def test_override_fallback_strategy(self, info): replica_config=ReplicaConfig.create( lambda x: x, placement_group_bundles=[{"CPU": 1}], - placement_group_fallback_strategy=[ - {"placement_group_bundles": [{"CPU": 1}]} - ], + placement_group_fallback_strategy=[{"bundles ": [{"CPU": 1}]}], ), start_time_ms=0, deployer_job_id="", diff --git a/python/ray/serve/tests/unit/test_deployment_scheduler.py b/python/ray/serve/tests/unit/test_deployment_scheduler.py index ad7a3d04f12f..c9aee65569b4 100644 --- a/python/ray/serve/tests/unit/test_deployment_scheduler.py +++ b/python/ray/serve/tests/unit/test_deployment_scheduler.py @@ -902,6 +902,7 @@ def wait(self, *args): on_scheduled=lambda *args, **kwargs: None, placement_group_bundles=[{"CPU": 1}], placement_group_bundle_label_selector=test_labels, + placement_group_strategy="STRICT_PACK", ) scheduler.schedule(upscales={dep_id: [req]}, downscales={}) @@ -1040,11 +1041,9 @@ def test_build_pack_placement_candidates(): {"accelerator-type": "H100"}, ], ) - strategies = scheduler._build_pack_placement_candidates(req_pack) - assert len(strategies) == 1 - assert strategies[0][0] == {"CPU": 0.1} - assert strategies[0][1] == [{"accelerator-type": "H100"}] + with pytest.raises(NotImplementedError): + scheduler._build_pack_placement_candidates(req_pack) # Scheduling replica with placement group STRICT_PACK strategy and bundle_label_selector req_pg = ReplicaSchedulingRequest( From bc765150aa4905efcfb395c56890e0cab05da60e Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Fri, 9 Jan 2026 03:38:35 +0000 Subject: [PATCH 46/55] Fix test key Signed-off-by: ryanaoleary --- python/ray/serve/tests/unit/test_application_state.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/ray/serve/tests/unit/test_application_state.py b/python/ray/serve/tests/unit/test_application_state.py index e117de28c7cd..974d164858b8 100644 --- a/python/ray/serve/tests/unit/test_application_state.py +++ b/python/ray/serve/tests/unit/test_application_state.py @@ -1533,7 +1533,7 @@ def test_override_fallback_strategy(self, info): replica_config=ReplicaConfig.create( lambda x: x, placement_group_bundles=[{"CPU": 1}], - placement_group_fallback_strategy=[{"bundles ": [{"CPU": 1}]}], + placement_group_fallback_strategy=[{"bundles": [{"CPU": 1}]}], ), start_time_ms=0, deployer_job_id="", @@ -1555,7 +1555,7 @@ def test_override_fallback_strategy(self, info): assert updated_info.deployment_config.num_replicas == 5 assert updated_info.replica_config.placement_group_fallback_strategy == [ - {"placement_group_bundles": [{"CPU": 1}]} + {"bundles": [{"CPU": 1}]} ] From 3c7c07f1217fc2727b3d18996eb9ba9366c76f30 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> Date: Tue, 13 Jan 2026 09:18:49 -0800 Subject: [PATCH 47/55] Update python/ray/tests/test_label_utils.py Co-authored-by: Abrar Sheikh Signed-off-by: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> --- python/ray/tests/test_label_utils.py | 87 +++++++++++++++++++++++++++- 1 file changed, 85 insertions(+), 2 deletions(-) diff --git a/python/ray/tests/test_label_utils.py b/python/ray/tests/test_label_utils.py index 9fe390d70804..9a98b18c8826 100644 --- a/python/ray/tests/test_label_utils.py +++ b/python/ray/tests/test_label_utils.py @@ -377,22 +377,57 @@ def test_validate_fallback_strategy(fallback_strategy, expected_error): ("us-west", "us-west", True), ("us-east", "us-west", False), (None, "us-west", False), + # Equals with empty strings + ("", "", True), + ("value", "", False), + ("", "value", False), # Not equals (!) operator ("us-west", "!us-east", True), ("us-east", "!us-east", False), (None, "!us-east", True), + # Not equals with empty string + ("value", "!", True), # ! means not equal to empty + ("", "!", False), # empty string equals empty, so !empty is False + ("", "!value", True), # empty string != "value" # In operator ("A100", "in(A100, H100)", True), ("T4", "in(A100, H100)", False), (None, "in(A100, H100)", False), ("value", "in(value)", True), + # In operator without spaces + ("A100", "in(A100,H100)", True), + ("H100", "in(A100,H100)", True), + ("T4", "in(A100,H100)", False), + # In operator with many values + ("c", "in(a, b, c, d, e)", True), + ("z", "in(a, b, c, d, e)", False), + # In operator with empty string value not in list + ("", "in(A100, H100)", False), # Not in operator ("T4", "!in(A100, H100)", True), ("A100", "!in(A100, H100)", False), (None, "!in(A100, H100)", True), - # Invalid types + # Not in operator without spaces + ("T4", "!in(A100,H100)", True), + ("A100", "!in(A100,H100)", False), + # Not in with single value + ("T4", "!in(A100)", True), + ("A100", "!in(A100)", False), + # Not in with empty string + ("", "!in(A100, H100)", True), # empty string not in list + # Invalid selector types ("A100", None, False), ("A100", 123, False), + (None, None, False), + ("value", [], False), + ("value", {}, False), + # Node value that looks like operator syntax (edge case) + ("in(x)", "in(x)", True), + ("!value", "!value", True), + # Case sensitivity + ("A100", "a100", False), + ("A100", "in(a100, h100)", False), + ("A100", "!a100", True), ], ) def test_match_label_selector_value(node_value, selector_value, expected): @@ -402,13 +437,21 @@ def test_match_label_selector_value(node_value, selector_value, expected): @pytest.mark.parametrize( "node_labels, selector, expected", [ + # Empty cases + ({}, {}, True), # Empty selector matches everything + ({"region": "us-west"}, {}, True), # Empty selector matches any node + ({}, {"region": "us-west"}, False), # Missing key with equals fails + ({}, {"region": "!us-west"}, True), # Missing key with not-equals succeeds # Match all ( {"region": "us-west", "gpu": "A100"}, {"region": "us-west", "gpu": "A100"}, True, ), - # Partial match + # Single key match + ({"region": "us-west"}, {"region": "us-west"}, True), + ({"region": "us-east"}, {"region": "us-west"}, False), + # Partial match (one key matches, one doesn't) ( {"region": "us-west", "gpu": "T4"}, {"region": "us-west", "gpu": "A100"}, @@ -420,9 +463,49 @@ def test_match_label_selector_value(node_value, selector_value, expected): {"region": "in(us-west, us-east)", "env": "!dev"}, True, ), + # Multiple operators - failure case + ( + {"region": "us-west", "env": "dev"}, + {"region": "in(us-west, us-east)", "env": "!dev"}, + False, + ), # Missing keys in node labels ({"region": "us-west"}, {"gpu": "!in(H100)"}, True), ({"region": "us-west"}, {"gpu": "in(H100)"}, False), + ({"region": "us-west"}, {"gpu": "!H100"}, True), # Missing key with ! succeeds + # Extra keys in node_labels that selector doesn't care about + ( + {"region": "us-west", "gpu": "A100", "env": "prod", "zone": "a"}, + {"region": "us-west"}, + True, + ), + # All keys matching with mixed operators + ( + {"region": "us-west", "gpu": "A100", "env": "prod"}, + {"region": "us-west", "gpu": "in(A100, H100)", "env": "!dev"}, + True, + ), + # First key matches, second doesn't + ( + {"region": "us-west", "gpu": "T4"}, + {"region": "in(us-west, us-east)", "gpu": "in(A100, H100)"}, + False, + ), + # First key doesn't match + ( + {"region": "eu-west", "gpu": "A100"}, + {"region": "in(us-west, us-east)", "gpu": "in(A100, H100)"}, + False, + ), + # Empty string values + ({"region": ""}, {"region": ""}, True), + ({"region": "us-west"}, {"region": ""}, False), + # Complex multi-key scenario + ( + {"cloud": "aws", "region": "us-west", "gpu": "A100", "spot": "true"}, + {"cloud": "!gcp", "region": "in(us-west, us-east)", "gpu": "A100"}, + True, + ), ], ) def test_match_label_selector(node_labels, selector, expected): From 206a89d32d323a46973ee8614371baff84e57f5a Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Tue, 13 Jan 2026 17:20:53 +0000 Subject: [PATCH 48/55] Trim whitespace around selector and fix invalid test case Signed-off-by: ryanaoleary Trim whitepsace and fix invalid test case Signed-off-by: ryanaoleary --- python/ray/_private/label_utils.py | 3 +++ python/ray/tests/test_label_utils.py | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/python/ray/_private/label_utils.py b/python/ray/_private/label_utils.py index 92cb0ba75129..82a6153e0a09 100644 --- a/python/ray/_private/label_utils.py +++ b/python/ray/_private/label_utils.py @@ -242,6 +242,9 @@ def match_label_selector_value(node_value: Optional[str], selector_value: str) - if not isinstance(selector_value, str): return False + # Trim whitespace + selector_value = selector_value.strip() + # !in operator if selector_value.startswith("!in(") and selector_value.endswith(")"): content = selector_value[4:-1] diff --git a/python/ray/tests/test_label_utils.py b/python/ray/tests/test_label_utils.py index 9a98b18c8826..4e3d964ff291 100644 --- a/python/ray/tests/test_label_utils.py +++ b/python/ray/tests/test_label_utils.py @@ -422,7 +422,7 @@ def test_validate_fallback_strategy(fallback_strategy, expected_error): ("value", [], False), ("value", {}, False), # Node value that looks like operator syntax (edge case) - ("in(x)", "in(x)", True), + ("inference", "inference", True), # parentheses not allowed in label value ("!value", "!value", True), # Case sensitivity ("A100", "a100", False), From 8cd0b0bcd0534733b27734162ac49dcab566b85f Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Wed, 14 Jan 2026 02:22:13 +0000 Subject: [PATCH 49/55] Add more test cases and re-structure under relevant classes Signed-off-by: ryanaoleary --- python/ray/serve/tests/conftest.py | 6 +- .../serve/tests/test_deployment_scheduler.py | 172 +++++++++++++++--- .../tests/unit/test_deployment_scheduler.py | 30 +++ 3 files changed, 175 insertions(+), 33 deletions(-) diff --git a/python/ray/serve/tests/conftest.py b/python/ray/serve/tests/conftest.py index 1724e7ba1b31..9d33c5995d48 100644 --- a/python/ray/serve/tests/conftest.py +++ b/python/ray/serve/tests/conftest.py @@ -355,7 +355,7 @@ def _get_node_id(): # Test fixture to start a Serve instance in a RayCluster with two labeled nodes @pytest.fixture -def serve_instance_with_labeled_nodes(): +def serve_instance_with_labeled_nodes(ray_shutdown): cluster = Cluster() # Unlabeled default node. node0_config = { @@ -382,7 +382,3 @@ def serve_instance_with_labeled_nodes(): serve.start() yield _get_global_client(), node_1_id, node_2_id - - serve.shutdown() - ray.shutdown() - cluster.shutdown() diff --git a/python/ray/serve/tests/test_deployment_scheduler.py b/python/ray/serve/tests/test_deployment_scheduler.py index 8cc084436baf..9410d0d3d433 100644 --- a/python/ray/serve/tests/test_deployment_scheduler.py +++ b/python/ray/serve/tests/test_deployment_scheduler.py @@ -135,6 +135,24 @@ def on_scheduled(actor_handle, placement_group): scheduler.on_replica_stopping(r2_id) scheduler.on_deployment_deleted(dep_id) + @pytest.mark.asyncio + async def test_spread_serve_strict_spread_pg(self, serve_instance): + """ + Verifies STRICT_SPREAD PG strategy runs successfully in the Spread Scheduler. + """ + + @serve.deployment( + placement_group_bundles=[{"CPU": 1}], + placement_group_strategy="STRICT_SPREAD", + ) + class StrictSpread: + def get_node_id(self): + return ray.get_runtime_context().get_node_id() + + handle = serve.run(StrictSpread.bind(), name="strict_spread_app") + assert await handle.get_node_id.remote() + serve.delete("strict_spread_app") + @serve.deployment def A(): @@ -285,13 +303,138 @@ def test_e2e_custom_resources(self, ray_cluster, use_pg): serve.shutdown() + @pytest.mark.asyncio + async def test_e2e_serve_fallback_strategy(self, serve_instance_with_labeled_nodes): + """ + Verifies that fallback strategies allow scheduling on alternative nodes when + primary constraints fail. Fallbacks are currently only supported in the Pack Scheduler. + """ + serve_instance, _, h100_node_id = serve_instance_with_labeled_nodes + + # Fallback strategy specified for Ray Actor in Serve deployment. + @serve.deployment( + ray_actor_options={ + "label_selector": {"region": "unavailable"}, + "fallback_strategy": [{"label_selector": {"gpu-type": "H100"}}], + } + ) + class FallbackDeployment: + def get_node_id(self): + return ray.get_runtime_context().get_node_id() + + # TODO (ryanaoleary@): Add a test for fallback_strategy in placement group options + # when support is added. + + handle = serve.run(FallbackDeployment.bind(), name="fallback_app") + assert await handle.get_node_id.remote() == h100_node_id + serve.delete("fallback_app") + + @pytest.mark.asyncio + async def test_e2e_serve_strict_pack_pg_label_selector( + self, serve_instance_with_labeled_nodes + ): + """ + Verifies STRICT_PACK strategy with placement_group_bundle_label_selector in Pack Scheduling Mode. + """ + serve_instance, _, us_east_node_id = serve_instance_with_labeled_nodes + + @serve.deployment( + placement_group_bundles=[{"CPU": 1}], + placement_group_strategy="STRICT_PACK", + placement_group_bundle_label_selector=[{"gpu-type": "H100"}], + ) + class StrictPackSelector: + def get_node_id(self): + return ray.get_runtime_context().get_node_id() + + handle = serve.run(StrictPackSelector.bind(), name="strict_pack_app") + assert await handle.get_node_id.remote() == us_east_node_id + serve.delete("strict_pack_app") + + @pytest.mark.asyncio + async def test_e2e_serve_pack_pg_forces_spread( + self, serve_instance_with_labeled_nodes + ): + """ + Verifies that using non-strict PACK PG strategy with label selectors works. + + STRICT_PACK throws NotImplementedError for selectors. However, 'PACK' is considered a + 'Non-Strict' strategy which forces the scheduler to fall back to 'Spread Mode'. + """ + serve_instance, _, us_east_node_id = serve_instance_with_labeled_nodes + + @serve.deployment( + placement_group_bundles=[{"CPU": 1}], + placement_group_strategy="PACK", + placement_group_bundle_label_selector=[{"gpu-type": "H100"}], + ) + class PackSelector: + def get_node_id(self): + return ray.get_runtime_context().get_node_id() + + # If this stayed in the Pack Scheduler, it would raise NotImplementedError. + # Because it forces Spread Mode, it succeeds. + handle = serve.run(PackSelector.bind(), name="pack_selector_app") + assert await handle.get_node_id.remote() == us_east_node_id + serve.delete("pack_selector_app") + + @pytest.mark.asyncio + async def test_e2e_serve_multiple_bundles_selector( + self, serve_instance_with_labeled_nodes + ): + """Verifies multiple bundles with bundle_label_selector are applied correctly.""" + serve_instance, _, us_east_node_id = serve_instance_with_labeled_nodes + + @serve.deployment( + placement_group_bundles=[{"CPU": 1}, {"CPU": 1}], + placement_group_strategy="STRICT_PACK", + placement_group_bundle_label_selector=[ + {"gpu-type": "H100"}, + {"gpu-type": "H100"}, + ], + ) + class MultiBundleSelector: + def get_node_id(self): + return ray.get_runtime_context().get_node_id() + + handle = serve.run(MultiBundleSelector.bind(), name="multi_bundle_app") + assert await handle.get_node_id.remote() == us_east_node_id + serve.delete("multi_bundle_app") + + @pytest.mark.asyncio + async def test_e2e_serve_actor_multiple_fallbacks( + self, serve_instance_with_labeled_nodes + ): + """ + Verifies that the scheduler can iterate through a label selector and multiple fallback options. + """ + serve_instance, us_west_node_id, _ = serve_instance_with_labeled_nodes + + @serve.deployment( + ray_actor_options={ + "label_selector": {"region": "invalid-label-1"}, + "fallback_strategy": [ + {"label_selector": {"region": "invalid-label-2"}}, + {"label_selector": {"region": "us-west"}}, # Should match + ], + } + ) + class MultiFallbackActor: + def get_node_id(self): + return ray.get_runtime_context().get_node_id() + + handle = serve.run(MultiFallbackActor.bind(), name="multi_fallback_app") + assert await handle.get_node_id.remote() == us_west_node_id + serve.delete("multi_fallback_app") + @pytest.mark.asyncio async def test_e2e_serve_label_selector(serve_instance_with_labeled_nodes): """ Verifies that label selectors work correctly for both Actors and Placement Groups. This test also verifies that label selectors are respected when scheduling with a - preferred node ID for resource compaction. + preferred node ID for resource compaction. This test verifies both the Pack and + Spread scheduler paths. """ serve_instance, us_west_node_id, us_east_node_id = serve_instance_with_labeled_nodes @@ -336,32 +479,5 @@ def get_node_id(self): serve.delete("pg_spread_app") -@pytest.mark.asyncio -async def test_e2e_serve_fallback_strategy(serve_instance_with_labeled_nodes): - """ - Verifies that fallback strategies allow scheduling on alternative nodes when - primary constraints fail. - """ - serve_instance, _, h100_node_id = serve_instance_with_labeled_nodes - - # Fallback strategy specified for Ray Actor in Serve deployment. - @serve.deployment( - ray_actor_options={ - "label_selector": {"region": "unavailable"}, - "fallback_strategy": [{"label_selector": {"gpu-type": "H100"}}], - } - ) - class FallbackDeployment: - def get_node_id(self): - return ray.get_runtime_context().get_node_id() - - # TODO (ryanaoleary@): Add a test for fallback_strategy in placement group options - # when support is added. - - handle = serve.run(FallbackDeployment.bind(), name="fallback_app") - assert await handle.get_node_id.remote() == h100_node_id - serve.delete("fallback_app") - - if __name__ == "__main__": sys.exit(pytest.main(["-v", "-s", __file__])) diff --git a/python/ray/serve/tests/unit/test_deployment_scheduler.py b/python/ray/serve/tests/unit/test_deployment_scheduler.py index c9aee65569b4..928bb7309c96 100644 --- a/python/ray/serve/tests/unit/test_deployment_scheduler.py +++ b/python/ray/serve/tests/unit/test_deployment_scheduler.py @@ -1064,6 +1064,36 @@ def test_build_pack_placement_candidates(): assert strategies[0][1] == [{"accelerator-type": "A100"}] +def test_build_pack_placement_candidates_pg_fallback_error(): + """ + Test that providing placement_group_fallback_strategy raises NotImplementedError. + """ + cluster_node_info_cache = MockClusterNodeInfoCache() + scheduler = default_impl.create_deployment_scheduler( + cluster_node_info_cache, + head_node_id_override="head_node", + create_placement_group_fn_override=None, + ) + + # Create a request with placement_group_fallback_strategy defined. + req = ReplicaSchedulingRequest( + replica_id=ReplicaID("r1", DeploymentID(name="d1")), + actor_def=MockActorClass(), + actor_resources={}, + actor_options={}, + actor_init_args=(), + on_scheduled=Mock(), + placement_group_bundles=[{"CPU": 1}], + placement_group_strategy="STRICT_PACK", + # Raises NotImplementedError since not added to placement group options yet. + placement_group_fallback_strategy=[{"label_selector": {"zone": "us-east-1a"}}], + ) + + # Verify the scheduler raises the expected error + with pytest.raises(NotImplementedError, match="not yet supported"): + scheduler._build_pack_placement_candidates(req) + + @pytest.mark.skipif( not RAY_SERVE_USE_PACK_SCHEDULING_STRATEGY, reason="Needs pack strategy." ) From 21bd155066593923119912506f4c471ac08bb953 Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Wed, 14 Jan 2026 03:37:40 +0000 Subject: [PATCH 50/55] Enable single bundle_label_selector to apply to all bundles and add tests for this Signed-off-by: ryanaoleary --- python/ray/serve/_private/config.py | 17 +++++ python/ray/serve/api.py | 3 +- .../serve/tests/test_deployment_scheduler.py | 73 ++++++++++++++++++- python/ray/serve/tests/unit/test_config.py | 36 ++++++++- 4 files changed, 121 insertions(+), 8 deletions(-) diff --git a/python/ray/serve/_private/config.py b/python/ray/serve/_private/config.py index b0dc17edca58..1f5b40695a1c 100644 --- a/python/ray/serve/_private/config.py +++ b/python/ray/serve/_private/config.py @@ -514,6 +514,7 @@ def __init__( self.max_replicas_per_node = max_replicas_per_node + self._normalize() self._validate() # Create resource_dict. This contains info about the replica's resource @@ -522,6 +523,21 @@ def __init__( self.resource_dict = resources_from_ray_options(self.ray_actor_options) self.needs_pickle = needs_pickle + def _normalize(self): + """Normalizes config values before validation.""" + # If a single placement_group_bundle_label_selector is provided for multiple bundles, + # apply it uniformly to all bundles. + if ( + self.placement_group_bundles + and self.placement_group_bundle_label_selector + and len(self.placement_group_bundle_label_selector) == 1 + and len(self.placement_group_bundles) > 1 + ): + self.placement_group_bundle_label_selector = ( + self.placement_group_bundle_label_selector + * len(self.placement_group_bundles) + ) + def _validate(self): self._validate_ray_actor_options() self._validate_placement_group_options() @@ -556,6 +572,7 @@ def update( self.max_replicas_per_node = max_replicas_per_node + self._normalize() self._validate() self.resource_dict = resources_from_ray_options(self.ray_actor_options) diff --git a/python/ray/serve/api.py b/python/ray/serve/api.py index 025b2e90a5a2..64e251dbda89 100644 --- a/python/ray/serve/api.py +++ b/python/ray/serve/api.py @@ -381,7 +381,8 @@ class MyDeployment: placement_group_strategy: Strategy to use for the replica placement group specified via `placement_group_bundles`. Defaults to `PACK`. placement_group_bundle_label_selector: A list of label selectors to apply to the - placement group on a per-bundle level. + placement group on a per-bundle level. If a single label selector is provided, + it is applied to all bundles. Otherwise, the length must match `placement_group_bundles`. max_replicas_per_node: The max number of replicas of this deployment that can run on a single node. Valid values are None (default, no limit) or an integer in the range of [1, 100]. diff --git a/python/ray/serve/tests/test_deployment_scheduler.py b/python/ray/serve/tests/test_deployment_scheduler.py index 9410d0d3d433..3f0f0bde7584 100644 --- a/python/ray/serve/tests/test_deployment_scheduler.py +++ b/python/ray/serve/tests/test_deployment_scheduler.py @@ -383,22 +383,87 @@ async def test_e2e_serve_multiple_bundles_selector( self, serve_instance_with_labeled_nodes ): """Verifies multiple bundles with bundle_label_selector are applied correctly.""" + ( + serve_instance, + us_west_node_id, + us_east_node_id, + ) = serve_instance_with_labeled_nodes + + # Helper task to return the node ID it's running on + @ray.remote(num_cpus=1) + def get_task_node_id(): + return ray.get_runtime_context().get_node_id() + + @serve.deployment( + placement_group_bundles=[{"CPU": 1}, {"CPU": 1}], + placement_group_strategy="SPREAD", + placement_group_bundle_label_selector=[ + {"gpu-type": "H100"}, # matches us-east node + {"gpu-type": "A100"}, # matches us-west node + ], + ) + class MultiBundleSelector: + def get_bundle_0_node_id(self): + # The actor should be scheduled on bundle 0. + return ray.get_runtime_context().get_node_id() + + async def get_bundle_1_node_id(self): + pg = ray.util.get_current_placement_group() + + # Schedule a task to bundle index 1 to get node ID. + return await get_task_node_id.options( + scheduling_strategy=ray.util.scheduling_strategies.PlacementGroupSchedulingStrategy( + placement_group=pg, + placement_group_bundle_index=1, # Target the second bundle + ) + ).remote() + + handle = serve.run(MultiBundleSelector.bind(), name="multi_bundle_app") + + # Verify bundles are scheduled to expected nodes based on label selectors. + assert await handle.get_bundle_0_node_id.remote() == us_east_node_id + assert await handle.get_bundle_1_node_id.remote() == us_west_node_id + serve.delete("multi_bundle_app") + + @pytest.mark.asyncio + async def test_e2e_serve_multiple_bundles_single_bundle_label_selector( + self, serve_instance_with_labeled_nodes + ): + """ + Verifies that when only one bundle_label_selector is provided for multiple bundles, + the label_selector is applied to each bundle uniformly. + """ serve_instance, _, us_east_node_id = serve_instance_with_labeled_nodes + @ray.remote(num_cpus=1) + def get_task_node_id(): + return ray.get_runtime_context().get_node_id() + @serve.deployment( placement_group_bundles=[{"CPU": 1}, {"CPU": 1}], - placement_group_strategy="STRICT_PACK", + # Use SPREAD to verify the label constraint forces them to same node. + placement_group_strategy="SPREAD", placement_group_bundle_label_selector=[ {"gpu-type": "H100"}, - {"gpu-type": "H100"}, ], ) class MultiBundleSelector: - def get_node_id(self): + def get_bundle_0_node_id(self): + # Verify actor bundle location. return ray.get_runtime_context().get_node_id() + async def get_bundle_1_node_id(self): + # Verify the second bundle's location + pg = ray.util.get_current_placement_group() + return await get_task_node_id.options( + scheduling_strategy=ray.util.scheduling_strategies.PlacementGroupSchedulingStrategy( + placement_group=pg, placement_group_bundle_index=1 + ) + ).remote() + handle = serve.run(MultiBundleSelector.bind(), name="multi_bundle_app") - assert await handle.get_node_id.remote() == us_east_node_id + assert await handle.get_bundle_0_node_id.remote() == us_east_node_id + assert await handle.get_bundle_1_node_id.remote() == us_east_node_id serve.delete("multi_bundle_app") @pytest.mark.asyncio diff --git a/python/ray/serve/tests/unit/test_config.py b/python/ray/serve/tests/unit/test_config.py index fb0b59f43d5a..4c0ef478aebe 100644 --- a/python/ray/serve/tests/unit/test_config.py +++ b/python/ray/serve/tests/unit/test_config.py @@ -596,15 +596,45 @@ class Class: placement_group_bundle_label_selector=[{"gpu": "T4"}], ) - # Valid config + # bundle_label_selector list does not match bundles list length + with pytest.raises( + ValueError, + match="The length of `bundle_label_selector` should equal the length of `bundles`", + ): + ReplicaConfig.create( + Class, + tuple(), + dict(), + placement_group_bundles=[{"CPU": 1}, {"CPU": 1}, {"CPU": 1}], + placement_group_bundle_label_selector=[{"gpu": "T4"}, {"gpu": "L4"}], + ) + + # Valid config - multiple bundles provided for one bundle_label_selector. config = ReplicaConfig.create( Class, tuple(), dict(), - placement_group_bundles=[{"CPU": 1}], + placement_group_bundles=[{"CPU": 1}, {"CPU": 1}, {"CPU": 1}], placement_group_bundle_label_selector=[{"gpu": "T4"}], ) - assert config.placement_group_bundle_label_selector == [{"gpu": "T4"}] + assert config.placement_group_bundle_label_selector == [ + {"gpu": "T4"}, + {"gpu": "T4"}, + {"gpu": "T4"}, + ] + + # Valid config - multiple bundles and an equal number of bundle label selectors. + config = ReplicaConfig.create( + Class, + tuple(), + dict(), + placement_group_bundles=[{"CPU": 1}, {"CPU": 1}], + placement_group_bundle_label_selector=[{"gpu": "T4"}, {"gpu": "L4"}], + ) + assert config.placement_group_bundle_label_selector == [ + {"gpu": "T4"}, + {"gpu": "L4"}, + ] def test_placement_group_fallback_strategy_validation(self): class Class: From 56f8bdfc062a9aa9314276ecf917cd571cc87d3e Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Thu, 15 Jan 2026 01:52:22 +0000 Subject: [PATCH 51/55] Fix validate_bundle_label_selector for single selector case Signed-off-by: ryanaoleary --- python/ray/serve/schema.py | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/python/ray/serve/schema.py b/python/ray/serve/schema.py index 881b72dbb0ec..46d8f997ec9a 100644 --- a/python/ray/serve/schema.py +++ b/python/ray/serve/schema.py @@ -488,12 +488,14 @@ def validate_bundle_label_selector(cls, values): "placement_group_bundles is not provided." ) - if len(bundle_label_selector) != len(placement_group_bundles): + if len(bundle_label_selector) != 1 and len(bundle_label_selector) != len( + placement_group_bundles + ): raise ValueError( - f"If bundle_label_selector is provided, it must have the same " - f"length as placement_group_bundles. Got " - f"{len(bundle_label_selector)} label selectors and " - f"{len(placement_group_bundles)} bundles." + f"The `placement_group_bundle_label_selector` list must contain either " + f"a single selector (to apply to all bundles) or match the number of " + f"`placement_group_bundles`. Got {len(bundle_label_selector)} " + f"selectors for {len(placement_group_bundles)} bundles." ) return values From a9f1954ecad75a96c338cbae8c3e60156dd519cd Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Thu, 15 Jan 2026 01:59:25 +0000 Subject: [PATCH 52/55] Fix undescriptive normalize function name Signed-off-by: ryanaoleary --- python/ray/serve/_private/config.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/python/ray/serve/_private/config.py b/python/ray/serve/_private/config.py index 1f5b40695a1c..623f74721dc2 100644 --- a/python/ray/serve/_private/config.py +++ b/python/ray/serve/_private/config.py @@ -514,7 +514,7 @@ def __init__( self.max_replicas_per_node = max_replicas_per_node - self._normalize() + self._normalize_bundle_label_selector() self._validate() # Create resource_dict. This contains info about the replica's resource @@ -523,10 +523,10 @@ def __init__( self.resource_dict = resources_from_ray_options(self.ray_actor_options) self.needs_pickle = needs_pickle - def _normalize(self): - """Normalizes config values before validation.""" - # If a single placement_group_bundle_label_selector is provided for multiple bundles, - # apply it uniformly to all bundles. + def _normalize_bundle_label_selector(self): + """If a single selector is provided for multiple bundles, it is broadcasted + uniformly to all bundles. + """ if ( self.placement_group_bundles and self.placement_group_bundle_label_selector @@ -572,7 +572,7 @@ def update( self.max_replicas_per_node = max_replicas_per_node - self._normalize() + self._normalize_bundle_label_selector() self._validate() self.resource_dict = resources_from_ray_options(self.ray_actor_options) From 0a150b867ec98cf5299ea2fc0fd482aeefd401d0 Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Thu, 15 Jan 2026 10:08:39 +0000 Subject: [PATCH 53/55] Update tests and add unhappy path coverage Signed-off-by: ryanaoleary --- python/ray/serve/tests/conftest.py | 44 +-- .../serve/tests/test_deployment_scheduler.py | 315 ++++++++++++++---- 2 files changed, 270 insertions(+), 89 deletions(-) diff --git a/python/ray/serve/tests/conftest.py b/python/ray/serve/tests/conftest.py index 9d33c5995d48..32eef2bbebe0 100644 --- a/python/ray/serve/tests/conftest.py +++ b/python/ray/serve/tests/conftest.py @@ -355,30 +355,32 @@ def _get_node_id(): # Test fixture to start a Serve instance in a RayCluster with two labeled nodes @pytest.fixture -def serve_instance_with_labeled_nodes(ray_shutdown): - cluster = Cluster() +def serve_instance_with_labeled_nodes(ray_cluster): + cluster = ray_cluster + # Unlabeled default node. - node0_config = { - "num_cpus": 1, - "resources": {"worker0": 1}, - } - cluster.add_node(**node0_config) - node1_config = { - "num_cpus": 1, - "resources": {"worker1": 1}, - "labels": {"region": "us-west", "gpu-type": "A100"}, - } - cluster.add_node(**node1_config) + cluster.add_node(num_cpus=3, resources={"worker0": 1}) + + # Node 1 - labeled A100 node in us-west. + cluster.add_node( + num_cpus=3, + resources={"worker1": 1}, + labels={"region": "us-west", "gpu-type": "A100"}, + ) + + # Node 2 - labeled H100 node in us-east. + cluster.add_node( + num_cpus=3, + resources={"worker2": 1}, + labels={"region": "us-east", "gpu-type": "H100"}, + ) + + cluster.wait_for_nodes() ray.init(address=cluster.address) - node_1_id = ray.get(_get_node_id.options(resources={"worker1": 1}).remote()) - node2_config = { - "num_cpus": 2, - "resources": {"worker2": 1}, - "labels": {"region": "us-east", "gpu-type": "H100"}, - } - cluster.add_node(**node2_config) + node_1_id = ray.get(_get_node_id.options(resources={"worker1": 1}).remote()) node_2_id = ray.get(_get_node_id.options(resources={"worker2": 1}).remote()) serve.start() - yield _get_global_client(), node_1_id, node_2_id + + yield _get_global_client(), node_1_id, node_2_id, cluster diff --git a/python/ray/serve/tests/test_deployment_scheduler.py b/python/ray/serve/tests/test_deployment_scheduler.py index 3f0f0bde7584..acb567dadd5f 100644 --- a/python/ray/serve/tests/test_deployment_scheduler.py +++ b/python/ray/serve/tests/test_deployment_scheduler.py @@ -136,22 +136,45 @@ def on_scheduled(actor_handle, placement_group): scheduler.on_deployment_deleted(dep_id) @pytest.mark.asyncio - async def test_spread_serve_strict_spread_pg(self, serve_instance): + async def test_spread_serve_strict_spread_pg(self, ray_cluster): """ - Verifies STRICT_SPREAD PG strategy runs successfully in the Spread Scheduler. + Verifies STRICT_SPREAD PG strategy runs successfully in the Spread Scheduler + and spreads bundles across distinct nodes. """ + cluster = ray_cluster + cluster.add_node(num_cpus=3) + cluster.add_node(num_cpus=3) + cluster.wait_for_nodes() + ray.init(address=cluster.address) + serve.start() + + @ray.remote(num_cpus=0) + def get_task_node_id(): + return ray.get_runtime_context().get_node_id() @serve.deployment( - placement_group_bundles=[{"CPU": 1}], + placement_group_bundles=[{"CPU": 1}, {"CPU": 1}], placement_group_strategy="STRICT_SPREAD", ) class StrictSpread: - def get_node_id(self): - return ray.get_runtime_context().get_node_id() + async def get_bundle_node_id(self, bundle_index: int): + pg = ray.util.get_current_placement_group() + return await get_task_node_id.options( + scheduling_strategy=ray.util.scheduling_strategies.PlacementGroupSchedulingStrategy( + placement_group=pg, + placement_group_bundle_index=bundle_index, + ) + ).remote() handle = serve.run(StrictSpread.bind(), name="strict_spread_app") - assert await handle.get_node_id.remote() + + node_0 = await handle.get_bundle_node_id.remote(0) + node_1 = await handle.get_bundle_node_id.remote(1) + + assert node_0 != node_1 + serve.delete("strict_spread_app") + serve.shutdown() @serve.deployment @@ -303,52 +326,45 @@ def test_e2e_custom_resources(self, ray_cluster, use_pg): serve.shutdown() - @pytest.mark.asyncio - async def test_e2e_serve_fallback_strategy(self, serve_instance_with_labeled_nodes): - """ - Verifies that fallback strategies allow scheduling on alternative nodes when - primary constraints fail. Fallbacks are currently only supported in the Pack Scheduler. - """ - serve_instance, _, h100_node_id = serve_instance_with_labeled_nodes - - # Fallback strategy specified for Ray Actor in Serve deployment. - @serve.deployment( - ray_actor_options={ - "label_selector": {"region": "unavailable"}, - "fallback_strategy": [{"label_selector": {"gpu-type": "H100"}}], - } - ) - class FallbackDeployment: - def get_node_id(self): - return ray.get_runtime_context().get_node_id() - - # TODO (ryanaoleary@): Add a test for fallback_strategy in placement group options - # when support is added. - - handle = serve.run(FallbackDeployment.bind(), name="fallback_app") - assert await handle.get_node_id.remote() == h100_node_id - serve.delete("fallback_app") - @pytest.mark.asyncio async def test_e2e_serve_strict_pack_pg_label_selector( self, serve_instance_with_labeled_nodes ): """ Verifies STRICT_PACK strategy with placement_group_bundle_label_selector in Pack Scheduling Mode. + + Since the strategy is STRICT_PACK, both bundles must be scheduled on the same node, + and that node must satisfy the label constraints in each selector. """ - serve_instance, _, us_east_node_id = serve_instance_with_labeled_nodes + _, _, us_east_node_id, _ = serve_instance_with_labeled_nodes + + @ray.remote(num_cpus=0) + def get_task_node_id(): + return ray.get_runtime_context().get_node_id() @serve.deployment( - placement_group_bundles=[{"CPU": 1}], + placement_group_bundles=[{"CPU": 1}, {"CPU": 1}], placement_group_strategy="STRICT_PACK", - placement_group_bundle_label_selector=[{"gpu-type": "H100"}], + placement_group_bundle_label_selector=[ + {"gpu-type": "H100", "region": "us-east"} + ], ) class StrictPackSelector: - def get_node_id(self): - return ray.get_runtime_context().get_node_id() + async def get_bundle_node_id(self, bundle_index: int): + pg = ray.util.get_current_placement_group() + return await get_task_node_id.options( + scheduling_strategy=ray.util.scheduling_strategies.PlacementGroupSchedulingStrategy( + placement_group=pg, + placement_group_bundle_index=bundle_index, + ) + ).remote() handle = serve.run(StrictPackSelector.bind(), name="strict_pack_app") - assert await handle.get_node_id.remote() == us_east_node_id + + # Both bundles are scheduled to the same node which matches the label constraints. + assert await handle.get_bundle_node_id.remote(0) == us_east_node_id + assert await handle.get_bundle_node_id.remote(1) == us_east_node_id + serve.delete("strict_pack_app") @pytest.mark.asyncio @@ -361,7 +377,7 @@ async def test_e2e_serve_pack_pg_forces_spread( STRICT_PACK throws NotImplementedError for selectors. However, 'PACK' is considered a 'Non-Strict' strategy which forces the scheduler to fall back to 'Spread Mode'. """ - serve_instance, _, us_east_node_id = serve_instance_with_labeled_nodes + _, _, us_east_node_id, _ = serve_instance_with_labeled_nodes @serve.deployment( placement_group_bundles=[{"CPU": 1}], @@ -383,14 +399,10 @@ async def test_e2e_serve_multiple_bundles_selector( self, serve_instance_with_labeled_nodes ): """Verifies multiple bundles with bundle_label_selector are applied correctly.""" - ( - serve_instance, - us_west_node_id, - us_east_node_id, - ) = serve_instance_with_labeled_nodes + _, us_west_node_id, us_east_node_id, _ = serve_instance_with_labeled_nodes # Helper task to return the node ID it's running on - @ray.remote(num_cpus=1) + @ray.remote(num_cpus=0) def get_task_node_id(): return ray.get_runtime_context().get_node_id() @@ -403,26 +415,20 @@ def get_task_node_id(): ], ) class MultiBundleSelector: - def get_bundle_0_node_id(self): - # The actor should be scheduled on bundle 0. - return ray.get_runtime_context().get_node_id() - - async def get_bundle_1_node_id(self): + async def get_bundle_node_id(self, bundle_index: int): pg = ray.util.get_current_placement_group() - - # Schedule a task to bundle index 1 to get node ID. return await get_task_node_id.options( scheduling_strategy=ray.util.scheduling_strategies.PlacementGroupSchedulingStrategy( placement_group=pg, - placement_group_bundle_index=1, # Target the second bundle + placement_group_bundle_index=bundle_index, ) ).remote() handle = serve.run(MultiBundleSelector.bind(), name="multi_bundle_app") # Verify bundles are scheduled to expected nodes based on label selectors. - assert await handle.get_bundle_0_node_id.remote() == us_east_node_id - assert await handle.get_bundle_1_node_id.remote() == us_west_node_id + assert await handle.get_bundle_node_id.remote(0) == us_east_node_id + assert await handle.get_bundle_node_id.remote(1) == us_west_node_id serve.delete("multi_bundle_app") @pytest.mark.asyncio @@ -433,9 +439,9 @@ async def test_e2e_serve_multiple_bundles_single_bundle_label_selector( Verifies that when only one bundle_label_selector is provided for multiple bundles, the label_selector is applied to each bundle uniformly. """ - serve_instance, _, us_east_node_id = serve_instance_with_labeled_nodes + _, _, us_east_node_id, _ = serve_instance_with_labeled_nodes - @ray.remote(num_cpus=1) + @ray.remote(num_cpus=0) def get_task_node_id(): return ray.get_runtime_context().get_node_id() @@ -448,22 +454,18 @@ def get_task_node_id(): ], ) class MultiBundleSelector: - def get_bundle_0_node_id(self): - # Verify actor bundle location. - return ray.get_runtime_context().get_node_id() - - async def get_bundle_1_node_id(self): - # Verify the second bundle's location + async def get_bundle_node_id(self, bundle_index: int): pg = ray.util.get_current_placement_group() return await get_task_node_id.options( scheduling_strategy=ray.util.scheduling_strategies.PlacementGroupSchedulingStrategy( - placement_group=pg, placement_group_bundle_index=1 + placement_group=pg, + placement_group_bundle_index=bundle_index, ) ).remote() handle = serve.run(MultiBundleSelector.bind(), name="multi_bundle_app") - assert await handle.get_bundle_0_node_id.remote() == us_east_node_id - assert await handle.get_bundle_1_node_id.remote() == us_east_node_id + assert await handle.get_bundle_node_id.remote(0) == us_east_node_id + assert await handle.get_bundle_node_id.remote(1) == us_east_node_id serve.delete("multi_bundle_app") @pytest.mark.asyncio @@ -473,7 +475,7 @@ async def test_e2e_serve_actor_multiple_fallbacks( """ Verifies that the scheduler can iterate through a label selector and multiple fallback options. """ - serve_instance, us_west_node_id, _ = serve_instance_with_labeled_nodes + _, us_west_node_id, _, _ = serve_instance_with_labeled_nodes @serve.deployment( ray_actor_options={ @@ -497,11 +499,12 @@ def get_node_id(self): async def test_e2e_serve_label_selector(serve_instance_with_labeled_nodes): """ Verifies that label selectors work correctly for both Actors and Placement Groups. + This test also verifies that label selectors are respected when scheduling with a preferred node ID for resource compaction. This test verifies both the Pack and Spread scheduler paths. """ - serve_instance, us_west_node_id, us_east_node_id = serve_instance_with_labeled_nodes + _, us_west_node_id, us_east_node_id, _ = serve_instance_with_labeled_nodes # Validate a Serve deplyoment utilizes a label_selector when passed to the Ray Actor options. @serve.deployment(ray_actor_options={"label_selector": {"region": "us-west"}}) @@ -544,5 +547,181 @@ def get_node_id(self): serve.delete("pg_spread_app") +@pytest.mark.asyncio +async def test_e2e_serve_fallback_strategy(serve_instance_with_labeled_nodes): + """ + Verifies that fallback strategies allow scheduling on alternative nodes when + primary constraints fail. + """ + _, _, h100_node_id, _ = serve_instance_with_labeled_nodes + + # Fallback strategy specified for Ray Actor in Serve deployment. + @serve.deployment( + ray_actor_options={ + "label_selector": {"region": "unavailable"}, + "fallback_strategy": [{"label_selector": {"gpu-type": "H100"}}], + } + ) + class FallbackDeployment: + def get_node_id(self): + return ray.get_runtime_context().get_node_id() + + # TODO (ryanaoleary@): Add a test for fallback_strategy in placement group options + # when support is added. + + handle = serve.run(FallbackDeployment.bind(), name="fallback_app") + assert await handle.get_node_id.remote() == h100_node_id + serve.delete("fallback_app") + + +@pytest.mark.asyncio +@pytest.mark.parametrize( + "use_pg,strategy", + [ + (False, None), # Actor-level label_selector. + (True, "PACK"), # PG bundle_label_selector with PACK strategy. + (True, "SPREAD"), # PG bundle_label_selector with SPREAD strategy. + ( + True, + "STRICT_SPREAD", + ), # PG bundle_label_selector with STRICT_SPREAD strategy. + ], +) +async def test_e2e_serve_label_selector_unschedulable( + serve_instance_with_labeled_nodes, use_pg, strategy +): + """ + Verifies the interaction between unschedulable a placement_group_bundle_label_selector + and different scheduling strategies in the Pack and Spread Serve scheduler. + """ + _, _, _, cluster = serve_instance_with_labeled_nodes + + @serve.deployment + def A(): + return ray.get_runtime_context().get_node_id() + + # Cluster in fixture only contains us-west and us-east. + target_label = {"region": "eu-central"} + + if use_pg: + app = A.options( + num_replicas=1, + placement_group_bundles=[{"CPU": 1}], + placement_group_strategy=strategy, + placement_group_bundle_label_selector=[target_label], + ).bind() + else: + app = A.options( + num_replicas=1, + ray_actor_options={"label_selector": target_label}, + ).bind() + + handle = serve._run(app, name="unschedulable_label_app", _blocking=False) + + def check_status(expected_status): + try: + status_info = serve.status().applications["unschedulable_label_app"] + return status_info.status == expected_status + except KeyError: + return False + + def verify_resource_request_stuck(): + """Verifies that the underlying resource request is pending.""" + # Serve deployment should be stuck DEPLOYING. + if not check_status("DEPLOYING"): + return False + + # Check PG/Actor is actually pending. + if use_pg: + pgs = ray.util.state.list_placement_groups() + return any(pg["state"] == "PENDING" for pg in pgs) + else: + actors = ray.util.state.list_actors() + return any(a["state"] == "PENDING_CREATION" for a in actors) + + # Serve deployment should remain stuck in deploying because Actor/PG can't be scheduled. + wait_for_condition(verify_resource_request_stuck, timeout=10) + assert not check_status("RUNNING"), ( + "Test setup failed: The deployment became RUNNING before the required " + "node was added. The label selector constraint was ignored." + ) + + # Add a suitable node to the cluster. + cluster.add_node(num_cpus=2, labels=target_label, resources={"target_node": 1}) + cluster.wait_for_nodes() + expected_node_id = ray.get( + get_node_id.options(resources={"target_node": 1}).remote() + ) + + # Validate deployment can now be scheduled since label selector is satisfied. + wait_for_condition(lambda: check_status("RUNNING"), timeout=10) + assert await handle.remote() == expected_node_id + + serve.delete("unschedulable_label_app") + + +@pytest.mark.asyncio +async def test_e2e_serve_fallback_strategy_unschedulable( + serve_instance_with_labeled_nodes, +): + """ + Verifies that an unschedulable fallback_strategy causes the Serve deployment to wait + until a suitable node is added to the cluster. + """ + _, _, _, cluster = serve_instance_with_labeled_nodes + + @serve.deployment + def A(): + return ray.get_runtime_context().get_node_id() + + fallback_label = {"region": "me-central2"} + + app = A.options( + num_replicas=1, + ray_actor_options={ + "label_selector": {"region": "non-existant"}, + "fallback_strategy": [{"label_selector": fallback_label}], + }, + ).bind() + + handle = serve._run(app, name="unschedulable_fallback_app", _blocking=False) + + def check_status(expected_status): + try: + status_info = serve.status().applications["unschedulable_fallback_app"] + return status_info.status == expected_status + except KeyError: + return False + + def verify_resource_request_stuck(): + """Verifies that the underlying resource request is pending.""" + # Serve deployment should be stuck DEPLOYING. + if not check_status("DEPLOYING"): + return False + + actors = ray.util.state.list_actors() + return any(a["state"] == "PENDING_CREATION" for a in actors) + + # Serve deployment should remain stuck in deploying because Actor/PG can't be scheduled. + wait_for_condition(verify_resource_request_stuck, timeout=10) + assert not check_status("RUNNING"), ( + "Test setup failed: The deployment became RUNNING before the required " + "node was added. The label selector constraint was ignored." + ) + + # Add a node that matches the fallback. + cluster.add_node(num_cpus=2, labels=fallback_label, resources={"fallback_node": 1}) + cluster.wait_for_nodes() + expected_node_id = ray.get( + get_node_id.options(resources={"fallback_node": 1}).remote() + ) + + # The serve deployment should recover and start running on the fallback node. + wait_for_condition(lambda: check_status("RUNNING"), timeout=10) + assert await handle.remote() == expected_node_id + + serve.delete("unschedulable_fallback_app") + + if __name__ == "__main__": sys.exit(pytest.main(["-v", "-s", __file__])) From d21d0012b8ebda86360754de9c7a488f7ba3db04 Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Thu, 15 Jan 2026 22:39:27 +0000 Subject: [PATCH 54/55] Fix test_schema.py for validate bundle_label_selector Signed-off-by: ryanaoleary --- python/ray/serve/tests/unit/test_schema.py | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/python/ray/serve/tests/unit/test_schema.py b/python/ray/serve/tests/unit/test_schema.py index b6c7881b595c..d0f0ebd00d30 100644 --- a/python/ray/serve/tests/unit/test_schema.py +++ b/python/ray/serve/tests/unit/test_schema.py @@ -397,20 +397,30 @@ def test_validate_bundle_label_selector(self): # Validate mismatched lengths for bundles and bundle_label_selector raises. deployment_schema["placement_group_bundles"] = [{"CPU": 1}, {"CPU": 1}] - deployment_schema["placement_group_bundle_label_selector"] = [{"a": "b"}] + deployment_schema["placement_group_bundle_label_selector"] = [ + {"a": "b"}, + {"c": "d"}, + {"e": "f"}, + ] with pytest.raises( ValidationError, - match="must have the same length as placement_group_bundles", + match=r"list must contain either a single selector \(to apply to all bundles\) or match the number of `placement_group_bundles`", ): DeploymentSchema.parse_obj(deployment_schema) - # Valid config + # Valid config - 2 bundles and 2 placement_group_bundle_label_selector. deployment_schema["placement_group_bundle_label_selector"] = [ {"a": "b"}, {"c": "d"}, ] DeploymentSchema.parse_obj(deployment_schema) + # Valid config - single placement_group_bundle_label_selector. + deployment_schema["placement_group_bundle_label_selector"] = [ + {"a": "b"}, + ] + DeploymentSchema.parse_obj(deployment_schema) + class TestServeApplicationSchema: def get_valid_serve_application_schema(self): From bc26060ee9b73d63449eac374566c768d4e0f742 Mon Sep 17 00:00:00 2001 From: ryanaoleary Date: Fri, 16 Jan 2026 00:26:16 +0000 Subject: [PATCH 55/55] Import match label selector logic from C++ instead Signed-off-by: ryanaoleary --- python/ray/_private/label_utils.py | 45 ------ python/ray/_raylet.pyx | 32 ++++ python/ray/includes/common.pxd | 6 + .../serve/_private/deployment_scheduler.py | 4 +- python/ray/tests/test_label_utils.py | 144 ------------------ 5 files changed, 40 insertions(+), 191 deletions(-) diff --git a/python/ray/_private/label_utils.py b/python/ray/_private/label_utils.py index 82a6153e0a09..6f6de4597624 100644 --- a/python/ray/_private/label_utils.py +++ b/python/ray/_private/label_utils.py @@ -228,48 +228,3 @@ def validate_fallback_strategy( return error_message return None - - -def match_label_selector_value(node_value: Optional[str], selector_value: str) -> bool: - """Evaluates if a node's label value matches a selector expression. - - Supports: - - Equality: "value" matches if node_value == "value" - - Not Equal: "!value" matches if node_value != "value" - - In: "in(v1, v2)" matches if node_value in ["v1", "v2"] - - Not In: "!in(v1, v2)" matches if node_value not in ["v1", "v2"] - """ - if not isinstance(selector_value, str): - return False - - # Trim whitespace - selector_value = selector_value.strip() - - # !in operator - if selector_value.startswith("!in(") and selector_value.endswith(")"): - content = selector_value[4:-1] - values = [v.strip() for v in content.split(",")] - return node_value is None or node_value not in values - - # in operator - if selector_value.startswith("in(") and selector_value.endswith(")"): - content = selector_value[3:-1] - values = [v.strip() for v in content.split(",")] - return node_value in values - - # not equal operator - if selector_value.startswith("!"): - target_val = selector_value[1:] - return node_value != target_val - - # equals operator - return node_value == selector_value - - -def match_label_selector(node_labels: Dict[str, str], selector: Dict[str, str]) -> bool: - """Returns True if node_labels satisfy all selector constraints.""" - for key, selector_value in selector.items(): - node_value = node_labels.get(key) - if not match_label_selector_value(node_value, selector_value): - return False - return True diff --git a/python/ray/_raylet.pyx b/python/ray/_raylet.pyx index 9e2b5e580c17..e9226a437f3c 100644 --- a/python/ray/_raylet.pyx +++ b/python/ray/_raylet.pyx @@ -103,6 +103,8 @@ from ray.includes.common cimport ( CLabelMatchExpression, CLabelIn, CLabelNotIn, + CLabelSelector, + CNodeResources, CRayFunction, CWorkerType, CJobConfig, @@ -597,6 +599,36 @@ cdef int prepare_label_selector( return 0 +cdef extern from * nogil: + """ + #include "ray/common/scheduling/cluster_resource_data.h" + + // Helper function convert a std::unordered_map to the absl::flat_hash_map used by NodeResources. + void SetNodeResourcesLabels(ray::NodeResources& resources, const std::unordered_map& labels) { + for (const auto& pair : labels) { + resources.labels[pair.first] = pair.second; + } + } + """ + void SetNodeResourcesLabels(CNodeResources& resources, const unordered_map[c_string, c_string]& labels) + +def node_labels_match_selector(node_labels: Dict[str, str], selector: Dict[str, str]) -> bool: + """ + Checks if the given node labels satisfy the label selector. This helper function exposes + the C++ logic for determining if a node satisfies a label selector to the Python layer. + """ + cdef: + CNodeResources c_node_resources + CLabelSelector c_label_selector + unordered_map[c_string, c_string] c_labels_map + + prepare_labels(node_labels, &c_labels_map) + SetNodeResourcesLabels(c_node_resources, c_labels_map) + prepare_label_selector(selector, &c_label_selector) + + # Return whether the node resources satisfy the label constraint. + return c_node_resources.HasRequiredLabels(c_label_selector) + cdef int prepare_fallback_strategy( list fallback_strategy, c_vector[CFallbackOption] *fallback_strategy_vector) except -1: diff --git a/python/ray/includes/common.pxd b/python/ray/includes/common.pxd index 75278ab76d80..7c863d900344 100644 --- a/python/ray/includes/common.pxd +++ b/python/ray/includes/common.pxd @@ -275,6 +275,12 @@ cdef extern from "src/ray/protobuf/common.pb.h" nogil: CLineageReconstructionTask() const c_string &SerializeAsString() const +cdef extern from "ray/common/scheduling/cluster_resource_data.h" namespace "ray" nogil: + cdef cppclass CNodeResources "ray::NodeResources": + CNodeResources() + unordered_map[c_string, c_string] labels + c_bool HasRequiredLabels(const CLabelSelector &label_selector) const + cdef extern from "ray/common/scheduling/label_selector.h" namespace "ray": cdef cppclass CLabelSelector "ray::LabelSelector": CLabelSelector() nogil except + diff --git a/python/ray/serve/_private/deployment_scheduler.py b/python/ray/serve/_private/deployment_scheduler.py index dc618eeffae4..cc72afb36f95 100644 --- a/python/ray/serve/_private/deployment_scheduler.py +++ b/python/ray/serve/_private/deployment_scheduler.py @@ -10,7 +10,7 @@ from typing import Any, Callable, Dict, List, Optional, Set, Tuple import ray -from ray._private.label_utils import match_label_selector +from ray._raylet import node_labels_match_selector from ray.serve._private.cluster_node_info_cache import ClusterNodeInfoCache from ray.serve._private.common import ( CreatePlacementGroupRequest, @@ -900,7 +900,7 @@ def _filter_nodes_by_label_selector( return { node_id: resources for node_id, resources in available_nodes.items() - if match_label_selector(node_labels.get(node_id, {}), required_labels) + if node_labels_match_selector(node_labels.get(node_id, {}), required_labels) } def _find_best_fit_node_for_pack( diff --git a/python/ray/tests/test_label_utils.py b/python/ray/tests/test_label_utils.py index 4e3d964ff291..85770a7f4248 100644 --- a/python/ray/tests/test_label_utils.py +++ b/python/ray/tests/test_label_utils.py @@ -8,8 +8,6 @@ import pytest from ray._private.label_utils import ( - match_label_selector, - match_label_selector_value, parse_node_labels_from_yaml_file, parse_node_labels_json, parse_node_labels_string, @@ -370,147 +368,5 @@ def test_validate_fallback_strategy(fallback_strategy, expected_error): assert result is None -@pytest.mark.parametrize( - "node_value, selector_value, expected", - [ - # Equals operator - ("us-west", "us-west", True), - ("us-east", "us-west", False), - (None, "us-west", False), - # Equals with empty strings - ("", "", True), - ("value", "", False), - ("", "value", False), - # Not equals (!) operator - ("us-west", "!us-east", True), - ("us-east", "!us-east", False), - (None, "!us-east", True), - # Not equals with empty string - ("value", "!", True), # ! means not equal to empty - ("", "!", False), # empty string equals empty, so !empty is False - ("", "!value", True), # empty string != "value" - # In operator - ("A100", "in(A100, H100)", True), - ("T4", "in(A100, H100)", False), - (None, "in(A100, H100)", False), - ("value", "in(value)", True), - # In operator without spaces - ("A100", "in(A100,H100)", True), - ("H100", "in(A100,H100)", True), - ("T4", "in(A100,H100)", False), - # In operator with many values - ("c", "in(a, b, c, d, e)", True), - ("z", "in(a, b, c, d, e)", False), - # In operator with empty string value not in list - ("", "in(A100, H100)", False), - # Not in operator - ("T4", "!in(A100, H100)", True), - ("A100", "!in(A100, H100)", False), - (None, "!in(A100, H100)", True), - # Not in operator without spaces - ("T4", "!in(A100,H100)", True), - ("A100", "!in(A100,H100)", False), - # Not in with single value - ("T4", "!in(A100)", True), - ("A100", "!in(A100)", False), - # Not in with empty string - ("", "!in(A100, H100)", True), # empty string not in list - # Invalid selector types - ("A100", None, False), - ("A100", 123, False), - (None, None, False), - ("value", [], False), - ("value", {}, False), - # Node value that looks like operator syntax (edge case) - ("inference", "inference", True), # parentheses not allowed in label value - ("!value", "!value", True), - # Case sensitivity - ("A100", "a100", False), - ("A100", "in(a100, h100)", False), - ("A100", "!a100", True), - ], -) -def test_match_label_selector_value(node_value, selector_value, expected): - assert match_label_selector_value(node_value, selector_value) == expected - - -@pytest.mark.parametrize( - "node_labels, selector, expected", - [ - # Empty cases - ({}, {}, True), # Empty selector matches everything - ({"region": "us-west"}, {}, True), # Empty selector matches any node - ({}, {"region": "us-west"}, False), # Missing key with equals fails - ({}, {"region": "!us-west"}, True), # Missing key with not-equals succeeds - # Match all - ( - {"region": "us-west", "gpu": "A100"}, - {"region": "us-west", "gpu": "A100"}, - True, - ), - # Single key match - ({"region": "us-west"}, {"region": "us-west"}, True), - ({"region": "us-east"}, {"region": "us-west"}, False), - # Partial match (one key matches, one doesn't) - ( - {"region": "us-west", "gpu": "T4"}, - {"region": "us-west", "gpu": "A100"}, - False, - ), - # Multiple operators - ( - {"region": "us-west", "env": "prod"}, - {"region": "in(us-west, us-east)", "env": "!dev"}, - True, - ), - # Multiple operators - failure case - ( - {"region": "us-west", "env": "dev"}, - {"region": "in(us-west, us-east)", "env": "!dev"}, - False, - ), - # Missing keys in node labels - ({"region": "us-west"}, {"gpu": "!in(H100)"}, True), - ({"region": "us-west"}, {"gpu": "in(H100)"}, False), - ({"region": "us-west"}, {"gpu": "!H100"}, True), # Missing key with ! succeeds - # Extra keys in node_labels that selector doesn't care about - ( - {"region": "us-west", "gpu": "A100", "env": "prod", "zone": "a"}, - {"region": "us-west"}, - True, - ), - # All keys matching with mixed operators - ( - {"region": "us-west", "gpu": "A100", "env": "prod"}, - {"region": "us-west", "gpu": "in(A100, H100)", "env": "!dev"}, - True, - ), - # First key matches, second doesn't - ( - {"region": "us-west", "gpu": "T4"}, - {"region": "in(us-west, us-east)", "gpu": "in(A100, H100)"}, - False, - ), - # First key doesn't match - ( - {"region": "eu-west", "gpu": "A100"}, - {"region": "in(us-west, us-east)", "gpu": "in(A100, H100)"}, - False, - ), - # Empty string values - ({"region": ""}, {"region": ""}, True), - ({"region": "us-west"}, {"region": ""}, False), - # Complex multi-key scenario - ( - {"cloud": "aws", "region": "us-west", "gpu": "A100", "spot": "true"}, - {"cloud": "!gcp", "region": "in(us-west, us-east)", "gpu": "A100"}, - True, - ), - ], -) -def test_match_label_selector(node_labels, selector, expected): - assert match_label_selector(node_labels, selector) == expected - - if __name__ == "__main__": sys.exit(pytest.main(["-sv", "-vv", __file__]))