From 9f389a6737b1d42797032a0c8d56be10bb3c5727 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Thu, 20 Mar 2025 13:14:33 +0000 Subject: [PATCH 01/22] Initial label based scheduling REP commit Signed-off-by: Ryan O'Leary --- reps/2025-03-18-label-based-scheduling.md | 212 ++++++++++++++++++++++ 1 file changed, 212 insertions(+) create mode 100644 reps/2025-03-18-label-based-scheduling.md diff --git a/reps/2025-03-18-label-based-scheduling.md b/reps/2025-03-18-label-based-scheduling.md new file mode 100644 index 0000000..6973ec6 --- /dev/null +++ b/reps/2025-03-18-label-based-scheduling.md @@ -0,0 +1,212 @@ +## Summary +### General Motivation + +This REP summarizes the current state of the node label scheduling feature enhancement and the remaining work to fully support label based scheduling for Tasks and Actors in Ray. + +### Should this change be within `ray` or outside? + +The change should be within Ray since it's a direct enhancement to the Ray scheduler. + +## Stewardship +### Required Reviewers +@MengjinYan + +### Shepherd of the Proposal (should be a senior committer) +@edoakes + +## Design and Architecture + +### Current implementation state + +Ray currently supports passing labels to a node through `ray start` with the `--labels` flag in Python, parsing labels from a json string with `parse_node_labels_json`, and associates a list of labels with a Node through the `_labels` field. Node information, including labels, are saved in the `GcsNodeInfo` data struct when a node is added. Ray also supports setting default labels on node add, but currently only sets `ray.io/node-id`. + +To pass labels to a Ray node: +```sh +ray start --head --labels='{"gpu_type": "A100", "region": "us"}' +``` + +To access node labels: +```python +ray.nodes()[0]["Labels"] == {"gpu_type": "A100", "region": "us"} +``` + +To schedule nodes based on these labels, users specify `scheduling_strategy=NodeLabelSchedulingStrategy` as follows: +```python + actor = MyActor.options( + scheduling_strategy=NodeLabelSchedulingStrategy({"ray.io/availability-zone": In("us-central2-b")}) +).remote() +``` + +With both hard and soft constraints: +```python +MyActor.options( + actor = MyActor.options( + scheduling_strategy=NodeLabelSchedulingStrategy( + {"gpu_type": NotIn("A100", "T100"), "other_key": DoesNotExist()} + hard={"gpu_type": DoesNotExist()}, + soft={"gpu_type": In("A100")}, + ) + ) +).remote() +``` + +These API are currently [hidden](https://github.com/ray-project/ray/blob/da092abe3d4adfe2c5d94bde64c97a994a2e061b/python/ray/scripts/scripts.py#L628) and not publicly exposed. +The above API is supported through the following internal implementation: + +NodeInfo struct: +```python +message GcsNodeInfo { + ... + // The key-value labels of this node. + map labels = 26; + ... +} +``` + +Add labels from GCS when a Node is added: +```python +void NodeManager::NodeAdded(const GcsNodeInfo &node_info) { + ... + // Set node labels when node added. + absl::flat_hash_map labels(node_info.labels().begin(), + node_info.labels().end()); + cluster_resource_scheduler_->GetClusterResourceManager().SetNodeLabels( + scheduling::NodeID(node_id.Binary()), labels); + ... +} +``` + +Add default labels: +```python +void NodeManagerConfig::AddDefaultLabels(const std::string &self_node_id) { + # Adds the default `ray.io/node-id` label to the label mapping +} +``` + +Get node labels from GCS: +```python +std::unordered_map PythonGetNodeLabels(const rpc::GcsNodeInfo &node_info) { + # Returns the current list of labels from the GcsNodeInfo +} +``` + +And finally a `NodeLabelSchedulingStrategy` Scheduling Policy with the following key functions. This scheduling strategy has not yet been added to the [`SchedulingStrategy` proto](https://github.com/larrylian/ray/blob/66c05338b07f1ef149928d4742b5f70c6c49b138/src/ray/protobuf/common.proto#L72), but an alpha version is public in the [Python worker](https://github.com/ray-project/ray/blob/07cdfec1fd9b63559cb1d47b5197ef5318f4d43e/python/ray/util/scheduling_strategies.py#L40). +```python +scheduling::NodeID NodeLabelSchedulingPolicy::Schedule(...) { + # Filters the feasible nodes - those that satisfy the provided resource request - by the + # hard constraints of the label selectors and conditions, and then creates another list + # of those nodes which satisfy both the hard and soft label conditions. Schedule then returns + # the best node from these two lists. +} + +scheduling::NodeID NodeLabelSchedulingPolicy::SelectBestNode(...) { + # If non-empty, returns a random node from the list of available nodes which satisfy both + # hard and soft constraints. Else, returns a random node from the list of available nodes which + # satify the hard conditions. If there are no available nodes, returns a random feasible node + # from the hard and soft matches, or the hard matches if the former is empty. +} + +NodeLabelSchedulingPolicy::FilterNodesByLabelMatchExpressions(...) { + # Iterates through candidate nodes and returns list of those which satisfy the conditions. +} + +NodeLabelSchedulingPolicy::IsNodeMatchLabelExpression( + const Node &node, const rpc::LabelMatchExpression &expression) const { + # Returns a bool based on whether a node's labels satisfy the given condition. + # Supports exists, not exists, in, and not in conditions. We should also extend + # support to equal and not equal. +} +``` + +### Brief idea +In order to implement full label based scheduling as described in the [public proposal](https://docs.google.com/document/d/1DKhPuZERlLbsU4TIQZZ6POCsm1pVMBgN_yn5r0OmaDI), there are several required changes to the existing API and internal implementation in Ray core. Since most of the core functionality for adding, storing, and retrieving node labels is already implemented, the primary changes proposed here are to update the APIs, support autoscaling, and directly schedule nodes based on label selectors when provided, rather than passing a separate scheduling policy. + + +### API Design + +To pass labels to a Ray node, we will amend the `--labels` argument to `ray start` to accept a string list of key-value pairs. Currently the labels argument accepts a json struct. +```sh +ray start --labels "key1=val1,key2=val2" +``` + +We will also support sourcing labels from a file using bash: +```sh +ray start --labels $(cat labels.txt) +``` + +To then schedule based on these labels, we will support passing a `label_selector` argument to the `@ray.remote` decorator. +```python +@ray.remote(label_selector={"ray.io/accelerator-type": "nvidia-h100"}) +class Actor: + pass +... + +@ray.remote(label_selector={"ray.io/market-type": "spot"}) +def my_task(): + pass +``` + +To schedule placement groups based on labels we will implement support for applying label selectors to placement groups on a per-bundle level. This would require adding a `bundle_label_selector` to the `ray.util.placement_group` constructor. +```python +# Same labels on all bundles +ray.util.placement_group( + bundles=[{"GPU": 1}, {"GPU": 1}], + bundle_label_selector=[{"ray.io/availability-zone": "us-west4-a"} * 2], +) + +# Different bundles require different labels +ray.util.placement_group( + bundles=[{"CPU": 1}] + [{"GPU": 1} * 2], + bundle_label_selector=[{"ray.io/market_type": "spot"}] + [{"ray.io/accelerator-type": "nvidia-h100"} * 2] +) +``` + +Finally, we will implement a `fallback_strategy` API to support soft constraints or multiple deployment options if the initial `label_selector` cannot be satisfied. +```python +@ray.remote( + label_selector={"instance_type": "m5.16xlarge"}, + fallback_strategy=[ + # Fall back to an empty set of labels (no constraints). + # This is equivalent to a "soft" constraint for an m5.16xlarge. + {"label_selector": {}}, + ], +) +``` + +### Implementation plan + +A portion of the internal implementation to save node labels, match based on label conditions, and support node labels in the core Python worker already exists. The primary changes required are to update the current APIs to those described above, move the logic from the `NodeLabelSchedulingStrategy` directly to the [cluster resource scheduler](https://github.com/ray-project/ray/blob/07cdfec1fd9b63559cb1d47b5197ef5318f4d43e/src/ray/raylet/scheduling/cluster_resource_scheduler.cc#L149), and implement support for autoscaling. + +Overview of Ray scheduler steps during label based scheduling: +1. Ray gets a request to schedule nodes based on some resources and labels. +2. Ray filters the feasible nodes by those that satisfy the resource request. A feasible node is one with sufficient total resources to satisfy the request, although those resources may not currently be available. +3. Ray hard matches nodes that satisfy the resource request with those that satisfy the label selector and expression. +4. If no nodes match and a `fallback_strategy` is provided, filter by the provided fallback label selector and return the list of candidate nodes. +5. Ray returns the best schedulable node from the list of available (or feasible if no nodes are available) that satisfy the expressions in steps 3 and/or 4. + +Remaining steps to implement the label based scheduling feature: +- [ ] (P1) Update `--labels` argument to take either a list of strings or read from file and expose this API publicly +- [ ] (P1) Add `label_selector` API to `@ray.remote` decorator to schedule tasks/actors +- [ ] (P1) Add `bundle_label_selector` to the `ray.util.placement_group` constructor to apply a set of `label_selector`s to placement group bundles +- [x] (P1) Ray saves label info associated with a node in `GcsNodeInfo` - already implemented +- [ ] (P1) Update `ClusterResourceScheduler::GetBestSchedulableNode` to enforce `label_selector` conditions when returning list of candidate nodes. This will eventually replace `SchedulingOptions::NodeLabelScheduling(scheduling_strategy)`. +- [ ] (P1) Populate list of default labels automatically, currently only supports `ray.io/node-id` +- [ ] (P1) Adapt Ray V2 Autoscaler to parse labels from K8s Pod Spec and generate a `--labels` arg to `rayStartParams` +- [ ] (P1) Update Autoscaler data model to pass label information by adding a labels field to the `ResourceRequest` message +- [ ] (P1) Update Autoscaler bin packing logic to directly consider label matching +- [ ] (P2) Determine whitelist of K8s labels to always pass to Ray nodes +- [ ] (P2) Implement `fallback_strategy` API to match available/feasible nodes by the provided conditions if `label_selector` returns 0 matches +- [ ] (P2) Update documentation/examples to use updated `label_selector` API +- [ ] (P2) Update library usage of `NodeLabelSchedulingStrategy` with `label_selector` API +- [ ] (P2) Add labels argument to `request_resource()` SDK function used by Ray libraries + +### Autoscaler adaptation +Label based scheduling support should be added to the Ray V2 Autoscaler, only supporting the Kubernetes stack at first. Once the VM stack is also migrated to the V2 autoscaler, we can extend label based scheduling support. In order to inform scaling decisions based on user provided label selectors to Ray tasks/actors, it's necessary to propogate label information at runtime to the autoscaler and GCS. The required changes to the Ray Autoscaler V2 APIs and data model are described above in the implementation plan. + + +## Compatibility, Deprecation, and Migration Plan +As the above APIs are implemented, we can deprecate redundant functionality like `accelerator-type`, but retain NodeAffinitySchedulingStrategy until soft constraints are supported through `fallback_strategy`. We will update libraries, documentation, and examples where appropriate to use the new label selector API. + + +## Test Plan and Acceptance Criteria +All APIs will be rigorously unit tested, ensuring thorough validation of the documented specifications. End-to-end flows will be covered in CI tests. Prior to promoting this API to beta, we will add large-scale tests to assess scalability limits and performance impact on large clusters. End-to-end testing can be added to the KubeRay repo for the K8s stack as well as to the Ray V2 Autoscaler as part of that feature's promotion to beta. From ab4b5331b67f67ada1ad2597c3c5d66b5ca532f9 Mon Sep 17 00:00:00 2001 From: ryanaoleary <113500783+ryanaoleary@users.noreply.github.com> Date: Thu, 20 Mar 2025 12:34:15 -0700 Subject: [PATCH 02/22] Update reps/2025-03-18-label-based-scheduling.md Co-authored-by: Edward Oakes Signed-off-by: ryanaoleary <113500783+ryanaoleary@users.noreply.github.com> --- reps/2025-03-18-label-based-scheduling.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/reps/2025-03-18-label-based-scheduling.md b/reps/2025-03-18-label-based-scheduling.md index 6973ec6..596f942 100644 --- a/reps/2025-03-18-label-based-scheduling.md +++ b/reps/2025-03-18-label-based-scheduling.md @@ -1,7 +1,7 @@ ## Summary ### General Motivation -This REP summarizes the current state of the node label scheduling feature enhancement and the remaining work to fully support label based scheduling for Tasks and Actors in Ray. +This REP summarizes the current state of the node label scheduling feature enhancement and the remaining work to fully support scheduling using label selectors in Ray. ### Should this change be within `ray` or outside? From 5262b733aef12441c1410d34357e68c020be2a8b Mon Sep 17 00:00:00 2001 From: ryanaoleary <113500783+ryanaoleary@users.noreply.github.com> Date: Thu, 20 Mar 2025 12:34:21 -0700 Subject: [PATCH 03/22] Update reps/2025-03-18-label-based-scheduling.md Co-authored-by: Edward Oakes Signed-off-by: ryanaoleary <113500783+ryanaoleary@users.noreply.github.com> --- reps/2025-03-18-label-based-scheduling.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/reps/2025-03-18-label-based-scheduling.md b/reps/2025-03-18-label-based-scheduling.md index 596f942..09bc6a4 100644 --- a/reps/2025-03-18-label-based-scheduling.md +++ b/reps/2025-03-18-label-based-scheduling.md @@ -151,7 +151,7 @@ To schedule placement groups based on labels we will implement support for apply # Same labels on all bundles ray.util.placement_group( bundles=[{"GPU": 1}, {"GPU": 1}], - bundle_label_selector=[{"ray.io/availability-zone": "us-west4-a"} * 2], + bundle_label_selector=[{"ray.io/availability-zone": "us-west4-a"}] * 2, ) # Different bundles require different labels From cf660b8fbba96eb5db5a2d37ce60d6e665ab7356 Mon Sep 17 00:00:00 2001 From: ryanaoleary <113500783+ryanaoleary@users.noreply.github.com> Date: Thu, 20 Mar 2025 12:34:27 -0700 Subject: [PATCH 04/22] Update reps/2025-03-18-label-based-scheduling.md Co-authored-by: Edward Oakes Signed-off-by: ryanaoleary <113500783+ryanaoleary@users.noreply.github.com> --- reps/2025-03-18-label-based-scheduling.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/reps/2025-03-18-label-based-scheduling.md b/reps/2025-03-18-label-based-scheduling.md index 09bc6a4..e6a96d3 100644 --- a/reps/2025-03-18-label-based-scheduling.md +++ b/reps/2025-03-18-label-based-scheduling.md @@ -154,7 +154,7 @@ ray.util.placement_group( bundle_label_selector=[{"ray.io/availability-zone": "us-west4-a"}] * 2, ) -# Different bundles require different labels +# Different bundles requiring different labels ray.util.placement_group( bundles=[{"CPU": 1}] + [{"GPU": 1} * 2], bundle_label_selector=[{"ray.io/market_type": "spot"}] + [{"ray.io/accelerator-type": "nvidia-h100"} * 2] From cd049fcbdfac4c10ba65746cecdfb2bc1e4ea1d8 Mon Sep 17 00:00:00 2001 From: ryanaoleary <113500783+ryanaoleary@users.noreply.github.com> Date: Thu, 20 Mar 2025 12:34:43 -0700 Subject: [PATCH 05/22] Update reps/2025-03-18-label-based-scheduling.md Co-authored-by: Edward Oakes Signed-off-by: ryanaoleary <113500783+ryanaoleary@users.noreply.github.com> --- reps/2025-03-18-label-based-scheduling.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/reps/2025-03-18-label-based-scheduling.md b/reps/2025-03-18-label-based-scheduling.md index e6a96d3..acd40cc 100644 --- a/reps/2025-03-18-label-based-scheduling.md +++ b/reps/2025-03-18-label-based-scheduling.md @@ -156,7 +156,7 @@ ray.util.placement_group( # Different bundles requiring different labels ray.util.placement_group( - bundles=[{"CPU": 1}] + [{"GPU": 1} * 2], + bundles=[{"CPU": 1}] + [{"GPU": 1}] * 2, bundle_label_selector=[{"ray.io/market_type": "spot"}] + [{"ray.io/accelerator-type": "nvidia-h100"} * 2] ) ``` From 4709064c484a39c0048672f3b47777df49728366 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Thu, 20 Mar 2025 20:07:36 +0000 Subject: [PATCH 06/22] Fix comments Signed-off-by: Ryan O'Leary --- reps/2025-03-18-label-based-scheduling.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/reps/2025-03-18-label-based-scheduling.md b/reps/2025-03-18-label-based-scheduling.md index acd40cc..bfec9e6 100644 --- a/reps/2025-03-18-label-based-scheduling.md +++ b/reps/2025-03-18-label-based-scheduling.md @@ -1,7 +1,7 @@ ## Summary ### General Motivation -This REP summarizes the current state of the node label scheduling feature enhancement and the remaining work to fully support scheduling using label selectors in Ray. +This REP summarizes the current state of the node label scheduling feature enhancement and the remaining work to fully support scheduling using label selectors in Ray. This REP supersedes the previous [node affinity feature enhancement REP](https://github.com/ryanaoleary/enhancements/blob/main/reps/2023-02-03-node-affinity-feature-enhancements.md). ### Should this change be within `ray` or outside? @@ -134,7 +134,7 @@ We will also support sourcing labels from a file using bash: ray start --labels $(cat labels.txt) ``` -To then schedule based on these labels, we will support passing a `label_selector` argument to the `@ray.remote` decorator. +To then schedule based on these labels, we will support passing a `label_selector` argument to the `@ray.remote` decorator. Adding this API here, rather than as a task/actor `scheduling_strategy`, will enable users to utilize label selectors in addition to other scheduling strategies. ```python @ray.remote(label_selector={"ray.io/accelerator-type": "nvidia-h100"}) class Actor: From 31699ee545b8d986b893b55df8c09b30ee79b490 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Thu, 20 Mar 2025 20:10:05 +0000 Subject: [PATCH 07/22] Fix bundles line Signed-off-by: Ryan O'Leary --- reps/2025-03-18-label-based-scheduling.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/reps/2025-03-18-label-based-scheduling.md b/reps/2025-03-18-label-based-scheduling.md index bfec9e6..6109092 100644 --- a/reps/2025-03-18-label-based-scheduling.md +++ b/reps/2025-03-18-label-based-scheduling.md @@ -157,7 +157,7 @@ ray.util.placement_group( # Different bundles requiring different labels ray.util.placement_group( bundles=[{"CPU": 1}] + [{"GPU": 1}] * 2, - bundle_label_selector=[{"ray.io/market_type": "spot"}] + [{"ray.io/accelerator-type": "nvidia-h100"} * 2] + bundle_label_selector=[{"ray.io/market_type": "spot"}] + [{"ray.io/accelerator-type": "nvidia-h100"}] * 2 ) ``` From 0555f39b347027ef9922c5d6ea4fbeb01fe2ca7b Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Thu, 20 Mar 2025 20:14:50 +0000 Subject: [PATCH 08/22] Move implementation steps to Ray feature Signed-off-by: Ryan O'Leary --- reps/2025-03-18-label-based-scheduling.md | 17 ++--------------- 1 file changed, 2 insertions(+), 15 deletions(-) diff --git a/reps/2025-03-18-label-based-scheduling.md b/reps/2025-03-18-label-based-scheduling.md index 6109092..11bc131 100644 --- a/reps/2025-03-18-label-based-scheduling.md +++ b/reps/2025-03-18-label-based-scheduling.md @@ -184,21 +184,8 @@ Overview of Ray scheduler steps during label based scheduling: 4. If no nodes match and a `fallback_strategy` is provided, filter by the provided fallback label selector and return the list of candidate nodes. 5. Ray returns the best schedulable node from the list of available (or feasible if no nodes are available) that satisfy the expressions in steps 3 and/or 4. -Remaining steps to implement the label based scheduling feature: -- [ ] (P1) Update `--labels` argument to take either a list of strings or read from file and expose this API publicly -- [ ] (P1) Add `label_selector` API to `@ray.remote` decorator to schedule tasks/actors -- [ ] (P1) Add `bundle_label_selector` to the `ray.util.placement_group` constructor to apply a set of `label_selector`s to placement group bundles -- [x] (P1) Ray saves label info associated with a node in `GcsNodeInfo` - already implemented -- [ ] (P1) Update `ClusterResourceScheduler::GetBestSchedulableNode` to enforce `label_selector` conditions when returning list of candidate nodes. This will eventually replace `SchedulingOptions::NodeLabelScheduling(scheduling_strategy)`. -- [ ] (P1) Populate list of default labels automatically, currently only supports `ray.io/node-id` -- [ ] (P1) Adapt Ray V2 Autoscaler to parse labels from K8s Pod Spec and generate a `--labels` arg to `rayStartParams` -- [ ] (P1) Update Autoscaler data model to pass label information by adding a labels field to the `ResourceRequest` message -- [ ] (P1) Update Autoscaler bin packing logic to directly consider label matching -- [ ] (P2) Determine whitelist of K8s labels to always pass to Ray nodes -- [ ] (P2) Implement `fallback_strategy` API to match available/feasible nodes by the provided conditions if `label_selector` returns 0 matches -- [ ] (P2) Update documentation/examples to use updated `label_selector` API -- [ ] (P2) Update library usage of `NodeLabelSchedulingStrategy` with `label_selector` API -- [ ] (P2) Add labels argument to `request_resource()` SDK function used by Ray libraries +Remaining steps to implement the label based scheduling feature: https://github.com/ray-project/ray/issues/51564 + ### Autoscaler adaptation Label based scheduling support should be added to the Ray V2 Autoscaler, only supporting the Kubernetes stack at first. Once the VM stack is also migrated to the V2 autoscaler, we can extend label based scheduling support. In order to inform scaling decisions based on user provided label selectors to Ray tasks/actors, it's necessary to propogate label information at runtime to the autoscaler and GCS. The required changes to the Ray Autoscaler V2 APIs and data model are described above in the implementation plan. From cff0f362126a5de4f4bfa71df857d6a874362651 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Thu, 20 Mar 2025 20:30:45 +0000 Subject: [PATCH 09/22] Add additional sections Signed-off-by: Ryan O'Leary --- reps/2025-03-18-label-based-scheduling.md | 40 +++++++++++++++++++++++ 1 file changed, 40 insertions(+) diff --git a/reps/2025-03-18-label-based-scheduling.md b/reps/2025-03-18-label-based-scheduling.md index 11bc131..137fee6 100644 --- a/reps/2025-03-18-label-based-scheduling.md +++ b/reps/2025-03-18-label-based-scheduling.md @@ -173,6 +173,46 @@ Finally, we will implement a `fallback_strategy` API to support soft constraints ) ``` +### Label selector requirements +This API is based on [K8s labels and selectors](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels/). Labels are key-value pairs which conform to the same format and restrictions as [Kubernetes](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels/#syntax-and-character-set), with both the key and value required to be 63 characters or less, beginning and ending with an alphanumeric character ([a-z0-9A-Z]) with dashes (-), underscores (_), dots (.), and alphanumerics between. + +Operators replace the label value and define the desired condition of each label. Operators are case insensitive and will support a string-based operator syntax. The initial list of supported operators is as follows: +- Equal: label equals exactly one value + - `{“key”: “value”}` + +- Not Equal: label equals anything but one value + - `{“key”: “!value”}` + +- In: label matches one of the provided values + - `{“key”: “in(val1,val2)”}` + +- Not In: label matches none of the provided values + - `{“key”: “!in(val1,val2)”}` + +To be added later if needed: +- Exists: label exists on the node + - `{“key”: “exists()”}` + +- Does Not Exist: label does not exist on the node + - `{“key”: “!exists()”}` + + +### Default labels +The initial set of supported default labels will be: +- `ray.io/node-id` + - this label is already supported +- `ray.io/accelerator-type` + - Set to "” on CPU-only machines. + - Supports existing accelerator type strings. +- `ray.io/market-type` + - spot or on-demand +- `ray.io/node-group` + - head or worker group name set by autoscaler +- `ray.io/availability-zone` + +These labels will be automatically populated based on the Kubernetes label or from information such as the GCE metadata when necessary. + + ### Implementation plan A portion of the internal implementation to save node labels, match based on label conditions, and support node labels in the core Python worker already exists. The primary changes required are to update the current APIs to those described above, move the logic from the `NodeLabelSchedulingStrategy` directly to the [cluster resource scheduler](https://github.com/ray-project/ray/blob/07cdfec1fd9b63559cb1d47b5197ef5318f4d43e/src/ray/raylet/scheduling/cluster_resource_scheduler.cc#L149), and implement support for autoscaling. From 5057fd00c45cd5289ada6f3c7e4f1cd9f7474d08 Mon Sep 17 00:00:00 2001 From: ryanaoleary <113500783+ryanaoleary@users.noreply.github.com> Date: Thu, 20 Mar 2025 13:58:33 -0700 Subject: [PATCH 10/22] Update reps/2025-03-18-label-based-scheduling.md Co-authored-by: Mengjin Yan Signed-off-by: ryanaoleary <113500783+ryanaoleary@users.noreply.github.com> --- reps/2025-03-18-label-based-scheduling.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/reps/2025-03-18-label-based-scheduling.md b/reps/2025-03-18-label-based-scheduling.md index 137fee6..eba1145 100644 --- a/reps/2025-03-18-label-based-scheduling.md +++ b/reps/2025-03-18-label-based-scheduling.md @@ -218,7 +218,7 @@ These labels will be automatically populated based on the Kubernetes label or fr A portion of the internal implementation to save node labels, match based on label conditions, and support node labels in the core Python worker already exists. The primary changes required are to update the current APIs to those described above, move the logic from the `NodeLabelSchedulingStrategy` directly to the [cluster resource scheduler](https://github.com/ray-project/ray/blob/07cdfec1fd9b63559cb1d47b5197ef5318f4d43e/src/ray/raylet/scheduling/cluster_resource_scheduler.cc#L149), and implement support for autoscaling. Overview of Ray scheduler steps during label based scheduling: -1. Ray gets a request to schedule nodes based on some resources and labels. +1. Ray gets a request to schedule an actor or task based on some resources and labels. 2. Ray filters the feasible nodes by those that satisfy the resource request. A feasible node is one with sufficient total resources to satisfy the request, although those resources may not currently be available. 3. Ray hard matches nodes that satisfy the resource request with those that satisfy the label selector and expression. 4. If no nodes match and a `fallback_strategy` is provided, filter by the provided fallback label selector and return the list of candidate nodes. From 260edcb10d2ffed8338471745131eecfab2bd981 Mon Sep 17 00:00:00 2001 From: ryanaoleary <113500783+ryanaoleary@users.noreply.github.com> Date: Thu, 20 Mar 2025 13:58:46 -0700 Subject: [PATCH 11/22] Update reps/2025-03-18-label-based-scheduling.md Co-authored-by: Mengjin Yan Signed-off-by: ryanaoleary <113500783+ryanaoleary@users.noreply.github.com> --- reps/2025-03-18-label-based-scheduling.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/reps/2025-03-18-label-based-scheduling.md b/reps/2025-03-18-label-based-scheduling.md index eba1145..afdc133 100644 --- a/reps/2025-03-18-label-based-scheduling.md +++ b/reps/2025-03-18-label-based-scheduling.md @@ -221,7 +221,7 @@ Overview of Ray scheduler steps during label based scheduling: 1. Ray gets a request to schedule an actor or task based on some resources and labels. 2. Ray filters the feasible nodes by those that satisfy the resource request. A feasible node is one with sufficient total resources to satisfy the request, although those resources may not currently be available. 3. Ray hard matches nodes that satisfy the resource request with those that satisfy the label selector and expression. -4. If no nodes match and a `fallback_strategy` is provided, filter by the provided fallback label selector and return the list of candidate nodes. +4. If no nodes match and a `fallback_strategy` is provided, filter by the provided fallback label selectors one-by-one until there is a match and return the list of candidate nodes. 5. Ray returns the best schedulable node from the list of available (or feasible if no nodes are available) that satisfy the expressions in steps 3 and/or 4. Remaining steps to implement the label based scheduling feature: https://github.com/ray-project/ray/issues/51564 From ed882e220ac3c02ef5a7f5ccc66312a2342d072c Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Thu, 20 Mar 2025 22:15:27 +0000 Subject: [PATCH 12/22] Fix nits/wording Signed-off-by: Ryan O'Leary --- reps/2025-03-18-label-based-scheduling.md | 34 ++++++++++++++++++----- 1 file changed, 27 insertions(+), 7 deletions(-) diff --git a/reps/2025-03-18-label-based-scheduling.md b/reps/2025-03-18-label-based-scheduling.md index afdc133..07c2ede 100644 --- a/reps/2025-03-18-label-based-scheduling.md +++ b/reps/2025-03-18-label-based-scheduling.md @@ -18,16 +18,16 @@ The change should be within Ray since it's a direct enhancement to the Ray sched ### Current implementation state -Ray currently supports passing labels to a node through `ray start` with the `--labels` flag in Python, parsing labels from a json string with `parse_node_labels_json`, and associates a list of labels with a Node through the `_labels` field. Node information, including labels, are saved in the `GcsNodeInfo` data struct when a node is added. Ray also supports setting default labels on node add, but currently only sets `ray.io/node-id`. +Ray currently supports passing labels to a node through `ray start` with the `--labels` flag in Python, parsing labels from a json string with `parse_node_labels_json`, and associates a list of labels with a remote function through the `_labels` field in task/actor options. Node information, including labels, are saved in the `GcsNodeInfo` data struct when a node is added. Ray also supports setting default labels on node add, but currently only sets `ray.io/node-id`. To pass labels to a Ray node: ```sh -ray start --head --labels='{"gpu_type": "A100", "region": "us"}' +ray start --head --labels='{"ray.io/accelerator-type": "A100", "region": "us"}' ``` To access node labels: ```python -ray.nodes()[0]["Labels"] == {"gpu_type": "A100", "region": "us"} +ray.nodes()[0]["Labels"] == {"ray.io/accelerator-type": "A100", "region": "us"} ``` To schedule nodes based on these labels, users specify `scheduling_strategy=NodeLabelSchedulingStrategy` as follows: @@ -42,9 +42,9 @@ With both hard and soft constraints: MyActor.options( actor = MyActor.options( scheduling_strategy=NodeLabelSchedulingStrategy( - {"gpu_type": NotIn("A100", "T100"), "other_key": DoesNotExist()} - hard={"gpu_type": DoesNotExist()}, - soft={"gpu_type": In("A100")}, + {"ray.io/accelerator-type": NotIn("A100", "T100"), "other_key": DoesNotExist()} + hard={"ray.io/accelerator-type": DoesNotExist()}, + soft={"ray.io/accelerator-type": In("A100")}, ) ) ).remote() @@ -146,7 +146,7 @@ def my_task(): pass ``` -To schedule placement groups based on labels we will implement support for applying label selectors to placement groups on a per-bundle level. This would require adding a `bundle_label_selector` to the `ray.util.placement_group` constructor. +To schedule placement groups based on labels we will implement support for applying label selectors to placement groups on a per-bundle level. This would require adding a `bundle_label_selector` to the `ray.util.placement_group` constructor. The items in `bundle_label_selector` map 1:1 with the items in `bundles`. ```python # Same labels on all bundles ray.util.placement_group( @@ -173,6 +173,26 @@ Finally, we will implement a `fallback_strategy` API to support soft constraints ) ``` +For placement groups: +```python +# Prefer 2 H100s, fall back to 4 A100s, then fall back to 8 V100s: +ray.util.placement_group( + bundles=[{"GPU": 1} * 2], + bundle_label_selector=[{"ray.io/accelerator-type": "nvidia-h100"}] * 2, + fallback_strategy=[ + { + "bundles": [{"GPU": 1}] * 4, + "bundle_label_selector": [{"ray.io/accelerator-type": "nvidia-h100"}] * 4 + }, + { + "bundles": [{"GPU": 1}] * 8, + "bundle_label_selector": [{"ray.io/accelerator-type": "nvidia_v100"}] * 8 + }, + ], +) +``` + + ### Label selector requirements This API is based on [K8s labels and selectors](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels/). Labels are key-value pairs which conform to the same format and restrictions as [Kubernetes](https://kubernetes.io/docs/concepts/overview/working-with-objects/labels/#syntax-and-character-set), with both the key and value required to be 63 characters or less, beginning and ending with an alphanumeric character ([a-z0-9A-Z]) with dashes (-), underscores (_), dots (.), and alphanumerics between. From 42f3524274de3df1bd0bea4afc5f69eb95499d24 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Thu, 20 Mar 2025 22:47:36 +0000 Subject: [PATCH 13/22] Add ray init example Signed-off-by: Ryan O'Leary --- reps/2025-03-18-label-based-scheduling.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/reps/2025-03-18-label-based-scheduling.md b/reps/2025-03-18-label-based-scheduling.md index 07c2ede..2f93120 100644 --- a/reps/2025-03-18-label-based-scheduling.md +++ b/reps/2025-03-18-label-based-scheduling.md @@ -23,6 +23,9 @@ Ray currently supports passing labels to a node through `ray start` with the `-- To pass labels to a Ray node: ```sh ray start --head --labels='{"ray.io/accelerator-type": "A100", "region": "us"}' + +# or with ray init +ray.init(labels='{"ray.io/accelerator-type": "A100", "region": "us"}') ``` To access node labels: From 2a9f044077e5f68161d06c6ce24be93180d7de79 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Thu, 20 Mar 2025 22:50:54 +0000 Subject: [PATCH 14/22] Clarify special case Signed-off-by: Ryan O'Leary --- reps/2025-03-18-label-based-scheduling.md | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/reps/2025-03-18-label-based-scheduling.md b/reps/2025-03-18-label-based-scheduling.md index 2f93120..f58ab41 100644 --- a/reps/2025-03-18-label-based-scheduling.md +++ b/reps/2025-03-18-label-based-scheduling.md @@ -127,12 +127,15 @@ In order to implement full label based scheduling as described in the [public pr ### API Design -To pass labels to a Ray node, we will amend the `--labels` argument to `ray start` to accept a string list of key-value pairs. Currently the labels argument accepts a json struct. +To pass labels to a Ray node, we will amend the `--labels` argument to `ray start` and `ray init` to accept a string list of key-value pairs. Currently the labels argument accepts a json struct. ```sh ray start --labels "key1=val1,key2=val2" + +# or +ray.init(labels="key1=val1,key2=val2") ``` -We will also support sourcing labels from a file using bash: +We will also support sourcing labels from a file using bash for `ray start` only: ```sh ray start --labels $(cat labels.txt) ``` From eb2aab3b527e9e66083a7fc18afaeae6c8964b83 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Thu, 20 Mar 2025 22:54:39 +0000 Subject: [PATCH 15/22] Add labels-from-file Signed-off-by: Ryan O'Leary --- reps/2025-03-18-label-based-scheduling.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/reps/2025-03-18-label-based-scheduling.md b/reps/2025-03-18-label-based-scheduling.md index f58ab41..656f857 100644 --- a/reps/2025-03-18-label-based-scheduling.md +++ b/reps/2025-03-18-label-based-scheduling.md @@ -135,9 +135,9 @@ ray start --labels "key1=val1,key2=val2" ray.init(labels="key1=val1,key2=val2") ``` -We will also support sourcing labels from a file using bash for `ray start` only: +We will also support sourcing labels from a file using bash for `ray start` only. This command will read labels in YAML format to support passing down Pod labels into the Raylet using downward API. The labels passed in from file should be composable with those specified by `--labels`. ```sh -ray start --labels $(cat labels.txt) +ray start --labels-from-file $(cat labels.txt) ``` To then schedule based on these labels, we will support passing a `label_selector` argument to the `@ray.remote` decorator. Adding this API here, rather than as a task/actor `scheduling_strategy`, will enable users to utilize label selectors in addition to other scheduling strategies. From ecc2967090e850660f12be92b28b0ede3d5d649c Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Sat, 22 Mar 2025 00:39:58 +0000 Subject: [PATCH 16/22] Add task/actor options example Signed-off-by: Ryan O'Leary --- reps/2025-03-18-label-based-scheduling.md | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/reps/2025-03-18-label-based-scheduling.md b/reps/2025-03-18-label-based-scheduling.md index 656f857..7869136 100644 --- a/reps/2025-03-18-label-based-scheduling.md +++ b/reps/2025-03-18-label-based-scheduling.md @@ -152,6 +152,15 @@ def my_task(): pass ``` +Or in the Ray task/actor options: +```python +actor_1 = Actor.options( + num_gpus=1, + resources={"custom_resource": 1}, + label_selector={"ray.io/accelerator-type": "nvidia-h100"}, +).remote() +``` + To schedule placement groups based on labels we will implement support for applying label selectors to placement groups on a per-bundle level. This would require adding a `bundle_label_selector` to the `ray.util.placement_group` constructor. The items in `bundle_label_selector` map 1:1 with the items in `bundles`. ```python # Same labels on all bundles From 4f52141efd0eacb66ceb113a673d5c1d8c131ba5 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Sat, 22 Mar 2025 00:45:54 +0000 Subject: [PATCH 17/22] Clarify _labels is not used for scheduling tasks/actors Signed-off-by: Ryan O'Leary --- reps/2025-03-18-label-based-scheduling.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/reps/2025-03-18-label-based-scheduling.md b/reps/2025-03-18-label-based-scheduling.md index 7869136..53eb92b 100644 --- a/reps/2025-03-18-label-based-scheduling.md +++ b/reps/2025-03-18-label-based-scheduling.md @@ -18,7 +18,7 @@ The change should be within Ray since it's a direct enhancement to the Ray sched ### Current implementation state -Ray currently supports passing labels to a node through `ray start` with the `--labels` flag in Python, parsing labels from a json string with `parse_node_labels_json`, and associates a list of labels with a remote function through the `_labels` field in task/actor options. Node information, including labels, are saved in the `GcsNodeInfo` data struct when a node is added. Ray also supports setting default labels on node add, but currently only sets `ray.io/node-id`. +Ray currently supports passing labels to a node through `ray start` with the `--labels` flag in Python and parsing labels from a json string with `parse_node_labels_json`. Node information, including labels, are saved in the `GcsNodeInfo` data struct when a node is added. Ray also supports setting default labels on node add, but currently only sets `ray.io/node-id`. To pass labels to a Ray node: ```sh @@ -122,7 +122,7 @@ NodeLabelSchedulingPolicy::IsNodeMatchLabelExpression( ``` ### Brief idea -In order to implement full label based scheduling as described in the [public proposal](https://docs.google.com/document/d/1DKhPuZERlLbsU4TIQZZ6POCsm1pVMBgN_yn5r0OmaDI), there are several required changes to the existing API and internal implementation in Ray core. Since most of the core functionality for adding, storing, and retrieving node labels is already implemented, the primary changes proposed here are to update the APIs, support autoscaling, and directly schedule nodes based on label selectors when provided, rather than passing a separate scheduling policy. +In order to implement full label based scheduling as described in the [public proposal](https://docs.google.com/document/d/1DKhPuZERlLbsU4TIQZZ6POCsm1pVMBgN_yn5r0OmaDI), there are several required changes to the existing API and internal implementation in Ray core. Since most of the core functionality for adding, storing, and retrieving node labels is already implemented, the primary changes proposed here are to update the APIs, support autoscaling, and directly schedule nodes based on label selectors passed to Ray tasks/actors, rather than requiring a separate scheduling policy. ### API Design From 57f465086295d76ee54e76a80f00224c93a952e4 Mon Sep 17 00:00:00 2001 From: ryanaoleary <113500783+ryanaoleary@users.noreply.github.com> Date: Wed, 26 Mar 2025 13:42:12 -0700 Subject: [PATCH 18/22] Update reps/2025-03-18-label-based-scheduling.md Co-authored-by: Edward Oakes Signed-off-by: ryanaoleary <113500783+ryanaoleary@users.noreply.github.com> --- reps/2025-03-18-label-based-scheduling.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/reps/2025-03-18-label-based-scheduling.md b/reps/2025-03-18-label-based-scheduling.md index 53eb92b..688a6c8 100644 --- a/reps/2025-03-18-label-based-scheduling.md +++ b/reps/2025-03-18-label-based-scheduling.md @@ -137,7 +137,7 @@ ray.init(labels="key1=val1,key2=val2") We will also support sourcing labels from a file using bash for `ray start` only. This command will read labels in YAML format to support passing down Pod labels into the Raylet using downward API. The labels passed in from file should be composable with those specified by `--labels`. ```sh -ray start --labels-from-file $(cat labels.txt) +ray start --labels-file $(cat labels.txt) ``` To then schedule based on these labels, we will support passing a `label_selector` argument to the `@ray.remote` decorator. Adding this API here, rather than as a task/actor `scheduling_strategy`, will enable users to utilize label selectors in addition to other scheduling strategies. From 9ab94a251764f9f0111b35b4dbfa68ae2dff74d8 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Wed, 26 Mar 2025 20:49:12 +0000 Subject: [PATCH 19/22] Fix last comments Signed-off-by: Ryan O'Leary --- reps/2025-03-18-label-based-scheduling.md | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/reps/2025-03-18-label-based-scheduling.md b/reps/2025-03-18-label-based-scheduling.md index 688a6c8..dd5f9a0 100644 --- a/reps/2025-03-18-label-based-scheduling.md +++ b/reps/2025-03-18-label-based-scheduling.md @@ -130,12 +130,14 @@ In order to implement full label based scheduling as described in the [public pr To pass labels to a Ray node, we will amend the `--labels` argument to `ray start` and `ray init` to accept a string list of key-value pairs. Currently the labels argument accepts a json struct. ```sh ray start --labels "key1=val1,key2=val2" +``` -# or -ray.init(labels="key1=val1,key2=val2") +Or with `ray init`: +```python +ray.init(labels={"key1": "val1", "key2": "val2"}) ``` -We will also support sourcing labels from a file using bash for `ray start` only. This command will read labels in YAML format to support passing down Pod labels into the Raylet using downward API. The labels passed in from file should be composable with those specified by `--labels`. +We will also support sourcing labels from a file using bash for `ray start` only. This command will read labels in YAML format to support passing down Pod labels into the Raylet using downward API. The labels passed in from file should be composable with those specified by `--labels`, with the value in `--labels` taking precedence if there is a conflict. ```sh ray start --labels-file $(cat labels.txt) ``` From 66362f4e4effc48beb67bebca88f537425fa1d84 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Fri, 28 Mar 2025 00:42:35 +0000 Subject: [PATCH 20/22] Add reviewer Signed-off-by: Ryan O'Leary --- reps/2025-03-18-label-based-scheduling.md | 1 + 1 file changed, 1 insertion(+) diff --git a/reps/2025-03-18-label-based-scheduling.md b/reps/2025-03-18-label-based-scheduling.md index dd5f9a0..3ea5859 100644 --- a/reps/2025-03-18-label-based-scheduling.md +++ b/reps/2025-03-18-label-based-scheduling.md @@ -10,6 +10,7 @@ The change should be within Ray since it's a direct enhancement to the Ray sched ## Stewardship ### Required Reviewers @MengjinYan +@andrewsykim ### Shepherd of the Proposal (should be a senior committer) @edoakes From 59e6c5d4dfa816352dac60368e56546a607b19d0 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Fri, 28 Mar 2025 00:47:26 +0000 Subject: [PATCH 21/22] remove ray init command Signed-off-by: Ryan O'Leary --- reps/2025-03-18-label-based-scheduling.md | 14 ++++---------- 1 file changed, 4 insertions(+), 10 deletions(-) diff --git a/reps/2025-03-18-label-based-scheduling.md b/reps/2025-03-18-label-based-scheduling.md index 3ea5859..9721faa 100644 --- a/reps/2025-03-18-label-based-scheduling.md +++ b/reps/2025-03-18-label-based-scheduling.md @@ -24,9 +24,6 @@ Ray currently supports passing labels to a node through `ray start` with the `-- To pass labels to a Ray node: ```sh ray start --head --labels='{"ray.io/accelerator-type": "A100", "region": "us"}' - -# or with ray init -ray.init(labels='{"ray.io/accelerator-type": "A100", "region": "us"}') ``` To access node labels: @@ -128,17 +125,12 @@ In order to implement full label based scheduling as described in the [public pr ### API Design -To pass labels to a Ray node, we will amend the `--labels` argument to `ray start` and `ray init` to accept a string list of key-value pairs. Currently the labels argument accepts a json struct. +To pass labels to a Ray node, we will amend the `--labels` argument to `ray start` to accept a string list of key-value pairs. Currently the labels argument accepts a json struct. ```sh ray start --labels "key1=val1,key2=val2" ``` -Or with `ray init`: -```python -ray.init(labels={"key1": "val1", "key2": "val2"}) -``` - -We will also support sourcing labels from a file using bash for `ray start` only. This command will read labels in YAML format to support passing down Pod labels into the Raylet using downward API. The labels passed in from file should be composable with those specified by `--labels`, with the value in `--labels` taking precedence if there is a conflict. +We will also support sourcing labels from a file using bash for `ray start`. This command will read labels in YAML format to support passing down Pod labels into the Raylet using downward API. The labels passed in from file should be composable with those specified by `--labels`, with the value in `--labels` taking precedence if there is a conflict. ```sh ray start --labels-file $(cat labels.txt) ``` @@ -164,6 +156,8 @@ actor_1 = Actor.options( ).remote() ``` +The `label_selector` requirement will be ignored for scheduling when running on a local Ray cluster. A warning indicating this behavior will be logged in this case. + To schedule placement groups based on labels we will implement support for applying label selectors to placement groups on a per-bundle level. This would require adding a `bundle_label_selector` to the `ray.util.placement_group` constructor. The items in `bundle_label_selector` map 1:1 with the items in `bundles`. ```python # Same labels on all bundles From 3c444e71e87ca9582ab92264a112c8193f394f49 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Fri, 28 Mar 2025 23:28:22 +0000 Subject: [PATCH 22/22] Fix labels-file Signed-off-by: Ryan O'Leary --- reps/2025-03-18-label-based-scheduling.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/reps/2025-03-18-label-based-scheduling.md b/reps/2025-03-18-label-based-scheduling.md index 9721faa..f289e26 100644 --- a/reps/2025-03-18-label-based-scheduling.md +++ b/reps/2025-03-18-label-based-scheduling.md @@ -130,9 +130,9 @@ To pass labels to a Ray node, we will amend the `--labels` argument to `ray star ray start --labels "key1=val1,key2=val2" ``` -We will also support sourcing labels from a file using bash for `ray start`. This command will read labels in YAML format to support passing down Pod labels into the Raylet using downward API. The labels passed in from file should be composable with those specified by `--labels`, with the value in `--labels` taking precedence if there is a conflict. +We will also support reading labels from a file passed to `ray start`. This command will read labels in YAML format to support passing down Pod labels into the Raylet using downward API. The labels passed in from file should be composable with those specified by `--labels`, with the value in `--labels` taking precedence if there is a conflict. ```sh -ray start --labels-file $(cat labels.txt) +ray start --labels-file /path/to/labels.yaml ``` To then schedule based on these labels, we will support passing a `label_selector` argument to the `@ray.remote` decorator. Adding this API here, rather than as a task/actor `scheduling_strategy`, will enable users to utilize label selectors in addition to other scheduling strategies.