Skip to content
Open
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
67 commits
Select commit Hold shift + click to select a range
ab6cd99
Add label_selector and bundle_label_selector to Serve API
ryanaoleary Oct 14, 2025
0a449e1
Fix argument order
ryanaoleary Oct 16, 2025
a900e41
Merge branch 'master' into add-label-selector-serve-option
ryanaoleary Nov 3, 2025
6862eee
Merge branch 'master' into add-label-selector-serve-option
zcin Nov 5, 2025
a0c2246
Add fallback strategy to serve options and consider labels during rep…
ryanaoleary Dec 18, 2025
c74a0e8
Merge branch 'master' into add-label-selector-serve-option
ryanaoleary Dec 18, 2025
6a4d241
Add new fields to DeploymentVersion proto
ryanaoleary Dec 19, 2025
a1e5fd1
Add validation check that bundles are provided with bundle_label_sele…
ryanaoleary Dec 19, 2025
0ba9ed7
Correctly handle label constraint for replica actor compaction
ryanaoleary Dec 19, 2025
181156b
Fix fallback strategy type
ryanaoleary Dec 19, 2025
be6fdf1
mock correct flag in test
ryanaoleary Dec 19, 2025
4558c47
Fix test, move some to unit, and remove NodeLabelSchedulingStrategy c…
ryanaoleary Dec 19, 2025
b0eae3a
Update python/ray/serve/_private/config.py
ryanaoleary Dec 22, 2025
8b72f0c
Fix argument names, refactor test fixture, and improve readability
ryanaoleary Dec 30, 2025
248be33
Remove fallback_strategy from Deployment API
ryanaoleary Dec 30, 2025
3ddb0a1
Add new fields to to_dict
ryanaoleary Dec 30, 2025
a9dbec0
Fully remove fallback strategy from deployment API
ryanaoleary Dec 30, 2025
d96f858
Fix var names and tests
ryanaoleary Dec 30, 2025
db162af
Move appropriate tests to deployment_scheduler and remove duplicate test
ryanaoleary Dec 30, 2025
935e084
Handle Label selector set to None
ryanaoleary Dec 30, 2025
d080190
Merge branch 'master' into add-label-selector-serve-option
ryanaoleary Dec 30, 2025
d87f838
Move back actor_options copy to where it was
ryanaoleary Dec 31, 2025
31bd7d8
Merge branch 'master' into add-label-selector-serve-option
ryanaoleary Dec 31, 2025
fe6a1a7
Fix invalid parameter being passed as placement_group_version
ryanaoleary Jan 6, 2026
40c02e0
Add string type check to _filter_nodes_by_labels
ryanaoleary Jan 6, 2026
4305e20
Merge branch 'master' into add-label-selector-serve-option
ryanaoleary Jan 6, 2026
7b13cc5
Fix None check for fallback strategy
ryanaoleary Jan 7, 2026
fe1eb88
pass bundle_label_selector through config override path
ryanaoleary Jan 7, 2026
f4e90a4
Update python/ray/serve/schema.py
ryanaoleary Jan 7, 2026
fc52d16
Add TODO comments
ryanaoleary Jan 7, 2026
b844884
Fix field name that gets popped
ryanaoleary Jan 7, 2026
47ed24d
Refactor schedule function to be more clear
ryanaoleary Jan 7, 2026
4f438f4
Add unit test coverage for bundle_label_selector and fallback_strateg…
ryanaoleary Jan 8, 2026
485dde1
Refactor is_scaled_copy_of for readability and add attribute checks
ryanaoleary Jan 8, 2026
38d4d30
make variable assignments more readable
ryanaoleary Jan 8, 2026
06c23f5
Prevent fallback_strategy from getting overwritten
ryanaoleary Jan 8, 2026
4a771c1
Fix explicit null bundles in fallback causes crash
ryanaoleary Jan 8, 2026
5fc10e7
Add validation test for validate_bundle_label_selector
ryanaoleary Jan 8, 2026
0b294cc
Merge branch 'master' into add-label-selector-serve-option
ryanaoleary Jan 8, 2026
cb78cc2
Update python/ray/serve/_private/deployment_scheduler.py
ryanaoleary Jan 8, 2026
a55af8e
Update python/ray/serve/_private/deployment_scheduler.py
ryanaoleary Jan 8, 2026
5e8f85e
Update python/ray/serve/_private/deployment_scheduler.py
ryanaoleary Jan 8, 2026
3128161
Update python/ray/serve/_private/deployment_scheduler.py
ryanaoleary Jan 8, 2026
51dd705
Update python/ray/serve/_private/deployment_scheduler.py
ryanaoleary Jan 8, 2026
45dd5aa
Update python/ray/serve/_private/deployment_scheduler.py
ryanaoleary Jan 8, 2026
c213408
Update python/ray/serve/_private/deployment_scheduler.py
ryanaoleary Jan 8, 2026
dc59988
Update python/ray/serve/_private/deployment_scheduler.py
ryanaoleary Jan 8, 2026
fc79d5a
Update python/ray/serve/_private/deployment_scheduler.py
ryanaoleary Jan 8, 2026
802bd5e
Fix naming from suggested comments
ryanaoleary Jan 9, 2026
cde9d08
Fix version truthiness check
ryanaoleary Jan 9, 2026
656986d
Merge branch 'master' into add-label-selector-serve-option
ryanaoleary Jan 9, 2026
010827f
Move filter by label selector logic to label_utils
ryanaoleary Jan 9, 2026
1e3a074
Merge branch 'master' into add-label-selector-serve-option
ryanaoleary Jan 9, 2026
89d4390
Fix key names and add NotImplementedError
ryanaoleary Jan 9, 2026
bc76515
Fix test key
ryanaoleary Jan 9, 2026
3c7c07f
Update python/ray/tests/test_label_utils.py
ryanaoleary Jan 13, 2026
206a89d
Trim whitespace around selector and fix invalid test case
ryanaoleary Jan 13, 2026
8cd0b0b
Add more test cases and re-structure under relevant classes
ryanaoleary Jan 14, 2026
21bd155
Enable single bundle_label_selector to apply to all bundles and add t…
ryanaoleary Jan 14, 2026
9f8d570
Merge branch 'master' into add-label-selector-serve-option
ryanaoleary Jan 14, 2026
56f8bdf
Fix validate_bundle_label_selector for single selector case
ryanaoleary Jan 15, 2026
a9f1954
Fix undescriptive normalize function name
ryanaoleary Jan 15, 2026
0a150b8
Update tests and add unhappy path coverage
ryanaoleary Jan 15, 2026
331e8cd
Merge branch 'master' into add-label-selector-serve-option
ryanaoleary Jan 15, 2026
d21d001
Fix test_schema.py for validate bundle_label_selector
ryanaoleary Jan 15, 2026
bc26060
Import match label selector logic from C++ instead
ryanaoleary Jan 16, 2026
523de76
Merge branch 'master' into add-label-selector-serve-option
ryanaoleary Jan 16, 2026
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
14 changes: 14 additions & 0 deletions python/ray/serve/_private/config.py
Original file line number Diff line number Diff line change
Expand Up @@ -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,
):
Expand All @@ -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

Expand Down Expand Up @@ -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

Expand All @@ -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,
):
Expand Down Expand Up @@ -586,6 +591,7 @@ def create(
ray_actor_options,
placement_group_bundles,
placement_group_strategy,
bundle_label_selector,
max_replicas_per_node,
)

Expand All @@ -612,6 +618,7 @@ def _validate_ray_actor_options(self):
"resources",
# Other options
"runtime_env",
"label_selector",
}

for option in self.ray_actor_options:
Expand Down Expand Up @@ -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 = (
Expand Down Expand Up @@ -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,
)

Expand All @@ -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):
Expand Down
10 changes: 9 additions & 1 deletion python/ray/serve/api.py
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand Down Expand Up @@ -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
Expand All @@ -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].
Expand Down Expand Up @@ -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
Expand Down
35 changes: 35 additions & 0 deletions python/ray/serve/tests/conftest.py
Original file line number Diff line number Diff line change
Expand Up @@ -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()
29 changes: 29 additions & 0 deletions python/ray/serve/tests/test_api.py
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand Down
1 change: 1 addition & 0 deletions src/ray/protobuf/serve.proto
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down