From f1a7c4faa0f0970f7b1d06c79ade580685c0ac3b Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Wed, 28 May 2025 07:29:05 +0000 Subject: [PATCH 01/39] Add default Ray Node labels at Node init Signed-off-by: Ryan O'Leary Add default Ray Node labels at Node init Signed-off-by: Ryan O'Leary Add default Ray Node labels at Node init Signed-off-by: Ryan O'Leary --- python/ray/_private/node.py | 39 ++++++++++++++++++- python/ray/tests/test_node_labels.py | 56 +++++++++++++++++++++++----- 2 files changed, 83 insertions(+), 12 deletions(-) diff --git a/python/ray/_private/node.py b/python/ray/_private/node.py index 00b6599dfc56..79de0b223371 100644 --- a/python/ray/_private/node.py +++ b/python/ray/_private/node.py @@ -1267,6 +1267,8 @@ def start_raylet( create_out=True, create_err=True, ) + resource_spec = self.get_resource_spec() + node_labels = self._add_default_ray_node_labels(resource_spec) process_info = ray._private.services.start_raylet( self.redis_address, self.gcs_address, @@ -1282,7 +1284,7 @@ def start_raylet( self._session_dir, self._runtime_env_dir, self._logs_dir, - self.get_resource_spec(), + resource_spec, plasma_directory, fallback_directory, object_store_memory, @@ -1315,7 +1317,7 @@ def start_raylet( env_updates=self._ray_params.env_vars, node_name=self._ray_params.node_name, webui=self._webui_url, - labels=self.node_labels, + labels=node_labels, resource_isolation_config=self.resource_isolation_config, ) assert ray_constants.PROCESS_TYPE_RAYLET not in self.all_processes @@ -1913,3 +1915,36 @@ def _record_stats(self): # so we truncate it to the first 50 characters # to avoid any issues. record_hardware_usage(cpu_model_name[:50]) + + def _add_default_ray_node_labels(self, resource_spec): + node_labels = self._get_node_labels() + + # Get environment variables populated from K8s Pod Spec + market_type = os.environ.get("RAY_NODE_MARKET_TYPE", "") + node_group = os.environ.get("RAY_NODE_GROUP", "") + availability_zone = os.environ.get("RAY_NODE_ZONE", "") + availability_region = os.environ.get("RAY_NODE_REGION", "") + + # Map environment variables to default ray node labels + if market_type: + node_labels["ray.io/market-type"] = market_type + if node_group: + node_labels["ray.io/node-group"] = node_group + if availability_zone: + node_labels["ray.io/availability-zone"] = availability_zone + if availability_region: + node_labels["ray.io/availability-region"] = availability_region + + # Get accelerator type from ResourceSpec + accelerator_type = None + if resource_spec.resolved(): + for key in resource_spec.resources: + if key.startswith(ray_constants.RESOURCE_CONSTRAINT_PREFIX): + accelerator_type = key.split( + ray_constants.RESOURCE_CONSTRAINT_PREFIX + )[1] + break # Only add one value for ray.io/accelerator-type + if accelerator_type: + node_labels["ray.io/accelerator-type"] = accelerator_type + + return node_labels diff --git a/python/ray/tests/test_node_labels.py b/python/ray/tests/test_node_labels.py index 6a7221ecc446..4a2fc3d9a042 100644 --- a/python/ray/tests/test_node_labels.py +++ b/python/ray/tests/test_node_labels.py @@ -13,7 +13,7 @@ def check_cmd_stderr(cmd): return subprocess.run(cmd, stderr=subprocess.PIPE).stderr.decode("utf-8") -def add_default_labels(node_info, labels): +def add_default_labels_for_test(node_info, labels): labels["ray.io/node_id"] = node_info["NodeID"] return labels @@ -26,7 +26,7 @@ def add_default_labels(node_info, labels): def test_ray_start_set_node_labels_from_json(call_ray_start): ray.init(address=call_ray_start) node_info = ray.nodes()[0] - assert node_info["Labels"] == add_default_labels( + assert node_info["Labels"] == add_default_labels_for_test( node_info, {"gpu_type": "A100", "region": "us"} ) @@ -39,7 +39,7 @@ def test_ray_start_set_node_labels_from_json(call_ray_start): def test_ray_start_set_node_labels_from_string(call_ray_start): ray.init(address=call_ray_start) node_info = ray.nodes()[0] - assert node_info["Labels"] == add_default_labels( + assert node_info["Labels"] == add_default_labels_for_test( node_info, {"gpu_type": "A100", "region": "us"} ) @@ -54,18 +54,18 @@ def test_ray_start_set_node_labels_from_string(call_ray_start): def test_ray_start_set_empty_node_labels(call_ray_start): ray.init(address=call_ray_start) node_info = ray.nodes()[0] - assert node_info["Labels"] == add_default_labels(node_info, {}) + assert node_info["Labels"] == add_default_labels_for_test(node_info, {}) def test_ray_init_set_node_labels(shutdown_only): labels = {"gpu_type": "A100", "region": "us"} ray.init(labels=labels) node_info = ray.nodes()[0] - assert node_info["Labels"] == add_default_labels(node_info, labels) + assert node_info["Labels"] == add_default_labels_for_test(node_info, labels) ray.shutdown() ray.init(labels={}) node_info = ray.nodes()[0] - assert node_info["Labels"] == add_default_labels(node_info, {}) + assert node_info["Labels"] == add_default_labels_for_test(node_info, {}) def test_ray_init_set_node_labels_value_error(ray_start_cluster): @@ -104,14 +104,14 @@ def test_cluster_add_node_with_labels(ray_start_cluster): cluster.wait_for_nodes() ray.init(address=cluster.address) node_info = ray.nodes()[0] - assert node_info["Labels"] == add_default_labels(node_info, labels) + assert node_info["Labels"] == add_default_labels_for_test(node_info, labels) head_node_id = ray.nodes()[0]["NodeID"] cluster.add_node(num_cpus=1, labels={}) cluster.wait_for_nodes() for node in ray.nodes(): if node["NodeID"] != head_node_id: - assert node["Labels"] == add_default_labels(node, {}) + assert node["Labels"] == add_default_labels_for_test(node, {}) @pytest.mark.parametrize("autoscaler_v2", [False, True], ids=["v1", "v2"]) @@ -137,7 +137,9 @@ def test_autoscaler_set_node_labels(autoscaler_v2, shutdown_only): for node in ray.nodes(): if node["Resources"].get("CPU", 0) == 1: - assert node["Labels"] == add_default_labels(node, {"region": "us"}) + assert node["Labels"] == add_default_labels_for_test( + node, {"region": "us"} + ) finally: cluster.shutdown() @@ -152,7 +154,7 @@ def test_ray_start_set_node_labels_from_file(): subprocess.check_call(cmd) ray.init(address="auto") node_info = ray.nodes()[0] - assert node_info["Labels"] == add_default_labels( + assert node_info["Labels"] == add_default_labels_for_test( node_info, {"gpu_type": "A100", "region": "us", "market-type": "spot"} ) finally: @@ -160,5 +162,39 @@ def test_ray_start_set_node_labels_from_file(): os.remove(test_file_path) +# def test_add_default_ray_node_labels_sets_expected_labels(shutdown_only): +# # Set environment variables for test +# os.environ["RAY_NODE_MARKET_TYPE"] = "spot" +# os.environ["RAY_NODE_GROUP"] = "worker-group-1" +# os.environ["RAY_NODE_REGION"] = "us-central2" +# os.environ["RAY_NODE_ZONE"] = "us-central2-b" + +# # Patch accelerator manager to return a known type since test +# # is not actually scheduled on accelerator node. +# with patch.object( +# get_all_accelerator_managers()[0], +# "get_current_node_accelerator_type", +# return_value="A100", +# ): +# ray.init() +# node_info = ray.nodes()[0] +# labels = node_info["Labels"] + +# assert labels.get("ray.io/market-type") == "spot" +# assert labels.get("ray.io/node-group") == "worker-group-1" +# assert labels.get("ray.io/availability-region") == "us-central2" +# assert labels.get("ray.io/availability-zone") == "us-central2-b" +# assert labels.get("ray.io/accelerator-type") == "A100" + +# # Unset env vars +# for k in [ +# "RAY_NODE_MARKET_TYPE", +# "RAY_NODE_GROUP", +# "RAY_NODE_ZONE", +# "RAY_NODE_REGION", +# ]: +# del os.environ[k] + + if __name__ == "__main__": sys.exit(pytest.main(["-sv", __file__])) From fce62f2c3be643ba9b804289530964216887d89e Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Wed, 28 May 2025 09:57:49 +0000 Subject: [PATCH 02/39] Fix test Signed-off-by: Ryan O'Leary --- python/ray/_private/node.py | 2 +- python/ray/tests/test_node_labels.py | 59 +++++++++++++--------------- 2 files changed, 28 insertions(+), 33 deletions(-) diff --git a/python/ray/_private/node.py b/python/ray/_private/node.py index 79de0b223371..0a78d8bc4f75 100644 --- a/python/ray/_private/node.py +++ b/python/ray/_private/node.py @@ -1921,7 +1921,7 @@ def _add_default_ray_node_labels(self, resource_spec): # Get environment variables populated from K8s Pod Spec market_type = os.environ.get("RAY_NODE_MARKET_TYPE", "") - node_group = os.environ.get("RAY_NODE_GROUP", "") + node_group = os.environ.get("RAY_NODE_TYPE_NAME", "") availability_zone = os.environ.get("RAY_NODE_ZONE", "") availability_region = os.environ.get("RAY_NODE_REGION", "") diff --git a/python/ray/tests/test_node_labels.py b/python/ray/tests/test_node_labels.py index 4a2fc3d9a042..99a79177c599 100644 --- a/python/ray/tests/test_node_labels.py +++ b/python/ray/tests/test_node_labels.py @@ -162,38 +162,33 @@ def test_ray_start_set_node_labels_from_file(): os.remove(test_file_path) -# def test_add_default_ray_node_labels_sets_expected_labels(shutdown_only): -# # Set environment variables for test -# os.environ["RAY_NODE_MARKET_TYPE"] = "spot" -# os.environ["RAY_NODE_GROUP"] = "worker-group-1" -# os.environ["RAY_NODE_REGION"] = "us-central2" -# os.environ["RAY_NODE_ZONE"] = "us-central2-b" - -# # Patch accelerator manager to return a known type since test -# # is not actually scheduled on accelerator node. -# with patch.object( -# get_all_accelerator_managers()[0], -# "get_current_node_accelerator_type", -# return_value="A100", -# ): -# ray.init() -# node_info = ray.nodes()[0] -# labels = node_info["Labels"] - -# assert labels.get("ray.io/market-type") == "spot" -# assert labels.get("ray.io/node-group") == "worker-group-1" -# assert labels.get("ray.io/availability-region") == "us-central2" -# assert labels.get("ray.io/availability-zone") == "us-central2-b" -# assert labels.get("ray.io/accelerator-type") == "A100" - -# # Unset env vars -# for k in [ -# "RAY_NODE_MARKET_TYPE", -# "RAY_NODE_GROUP", -# "RAY_NODE_ZONE", -# "RAY_NODE_REGION", -# ]: -# del os.environ[k] +def test_add_default_ray_node_labels(shutdown_only): + # Set environment variables for test + os.environ["RAY_NODE_MARKET_TYPE"] = "spot" + os.environ["RAY_NODE_TYPE_NAME"] = "worker-group-1" + os.environ["RAY_NODE_REGION"] = "us-central2" + os.environ["RAY_NODE_ZONE"] = "us-central2-b" + + ray.init() + node_info = ray.nodes()[0] + labels = node_info["Labels"] + + assert labels.get("ray.io/market-type") == "spot" + assert labels.get("ray.io/node-group") == "worker-group-1" + assert labels.get("ray.io/availability-region") == "us-central2" + assert labels.get("ray.io/availability-zone") == "us-central2-b" + assert ( + labels.get("ray.io/accelerator-type") is None + ) # node is not scheduled on an accelerator + + # Unset env vars + for k in [ + "RAY_NODE_MARKET_TYPE", + "RAY_NODE_TYPE_NAME", + "RAY_NODE_ZONE", + "RAY_NODE_REGION", + ]: + del os.environ[k] if __name__ == "__main__": From 102c2f59347e8f39ac0c5d006f73a73f494a4f59 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Thu, 29 May 2025 10:01:43 +0000 Subject: [PATCH 03/39] Fix ray re-init error Signed-off-by: Ryan O'Leary --- python/ray/tests/test_node_labels.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/tests/test_node_labels.py b/python/ray/tests/test_node_labels.py index 99a79177c599..c1a08e73577e 100644 --- a/python/ray/tests/test_node_labels.py +++ b/python/ray/tests/test_node_labels.py @@ -144,7 +144,7 @@ def test_autoscaler_set_node_labels(autoscaler_v2, shutdown_only): cluster.shutdown() -def test_ray_start_set_node_labels_from_file(): +def test_ray_start_set_node_labels_from_file(shutdown_only): with tempfile.NamedTemporaryFile(mode="w+", delete=False) as test_file: test_file.write('"gpu_type": "A100"\n"region": "us"\n"market-type": "spot"') test_file_path = test_file.name From a0a6763e08b4596813c0b145d3608210e42ba5c5 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Thu, 29 May 2025 10:11:34 +0000 Subject: [PATCH 04/39] Actually check accelerator-type in test Signed-off-by: Ryan O'Leary --- python/ray/tests/test_node_labels.py | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/python/ray/tests/test_node_labels.py b/python/ray/tests/test_node_labels.py index c1a08e73577e..4c3e20cbd069 100644 --- a/python/ray/tests/test_node_labels.py +++ b/python/ray/tests/test_node_labels.py @@ -169,7 +169,10 @@ def test_add_default_ray_node_labels(shutdown_only): os.environ["RAY_NODE_REGION"] = "us-central2" os.environ["RAY_NODE_ZONE"] = "us-central2-b" - ray.init() + # set env var for AcceleratorManager to detect + os.environ["TPU_ACCELERATOR_TYPE"] = "v4-16" + + ray.init(resources={"TPU": 4}) node_info = ray.nodes()[0] labels = node_info["Labels"] @@ -177,9 +180,7 @@ def test_add_default_ray_node_labels(shutdown_only): assert labels.get("ray.io/node-group") == "worker-group-1" assert labels.get("ray.io/availability-region") == "us-central2" assert labels.get("ray.io/availability-zone") == "us-central2-b" - assert ( - labels.get("ray.io/accelerator-type") is None - ) # node is not scheduled on an accelerator + assert labels.get("ray.io/accelerator-type") == "TPU-V4" # Unset env vars for k in [ @@ -187,6 +188,7 @@ def test_add_default_ray_node_labels(shutdown_only): "RAY_NODE_TYPE_NAME", "RAY_NODE_ZONE", "RAY_NODE_REGION", + "TPU_ACCELERATOR_TYPE", ]: del os.environ[k] From c1192c7b1088ce7981c1ebe536db34d5ff353d29 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Fri, 6 Jun 2025 00:19:37 +0000 Subject: [PATCH 05/39] Move Ray node related keys/env vars to constants.h Signed-off-by: Ryan O'Leary --- python/ray/_private/node.py | 18 +++++++++--------- python/ray/includes/common.pxd | 9 +++++++++ python/ray/includes/common.pxi | 21 +++++++++++++++++++++ src/ray/common/constants.h | 11 ++++++++++- 4 files changed, 49 insertions(+), 10 deletions(-) diff --git a/python/ray/_private/node.py b/python/ray/_private/node.py index 0a78d8bc4f75..fde0ee987cbc 100644 --- a/python/ray/_private/node.py +++ b/python/ray/_private/node.py @@ -1920,20 +1920,20 @@ def _add_default_ray_node_labels(self, resource_spec): node_labels = self._get_node_labels() # Get environment variables populated from K8s Pod Spec - market_type = os.environ.get("RAY_NODE_MARKET_TYPE", "") - node_group = os.environ.get("RAY_NODE_TYPE_NAME", "") - availability_zone = os.environ.get("RAY_NODE_ZONE", "") - availability_region = os.environ.get("RAY_NODE_REGION", "") + node_group = os.environ.get(ray._raylet.NODE_TYPE_NAME_ENV, "") + market_type = os.environ.get(ray._raylet.NODE_MARKET_TYPE_ENV, "") + availability_region = os.environ.get(ray._raylet.NODE_REGION_ENV, "") + availability_zone = os.environ.get(ray._raylet.NODE_ZONE_ENV, "") # Map environment variables to default ray node labels if market_type: - node_labels["ray.io/market-type"] = market_type + node_labels[ray._raylet.RAY_NODE_MARKET_TYPE_KEY] = market_type if node_group: - node_labels["ray.io/node-group"] = node_group + node_labels[ray._raylet.RAY_NODE_GROUP_KEY] = node_group if availability_zone: - node_labels["ray.io/availability-zone"] = availability_zone + node_labels[ray._raylet.RAY_NODE_ZONE_KEY] = availability_zone if availability_region: - node_labels["ray.io/availability-region"] = availability_region + node_labels[ray._raylet.RAY_NODE_REGION_KEY] = availability_region # Get accelerator type from ResourceSpec accelerator_type = None @@ -1945,6 +1945,6 @@ def _add_default_ray_node_labels(self, resource_spec): )[1] break # Only add one value for ray.io/accelerator-type if accelerator_type: - node_labels["ray.io/accelerator-type"] = accelerator_type + node_labels[ray._raylet.RAY_NODE_ACCELERATOR_TYPE_KEY] = accelerator_type return node_labels diff --git a/python/ray/includes/common.pxd b/python/ray/includes/common.pxd index f69f80f46e10..b6a010a53aa9 100644 --- a/python/ray/includes/common.pxd +++ b/python/ray/includes/common.pxd @@ -774,3 +774,12 @@ cdef extern from "ray/common/constants.h" nogil: cdef const char[] kGcsAutoscalerV2EnabledKey cdef const char[] kGcsAutoscalerClusterConfigKey cdef const char[] kGcsPidKey + cdef const char[] kNodeTypeNameEnv + cdef const char[] kNodeMarketTypeEnv + cdef const char[] kNodeRegionEnv + cdef const char[] kNodeZoneEnv + cdef const char[] kLabelKeyNodeAcceleratorType + cdef const char[] kLabelKeyNodeMarketType + cdef const char[] kLabelKeyNodeRegion + cdef const char[] kLabelKeyNodeZone + cdef const char[] kLabelKeyNodeGroup diff --git a/python/ray/includes/common.pxi b/python/ray/includes/common.pxi index 3db7b9391d72..b4c257a7b140 100644 --- a/python/ray/includes/common.pxi +++ b/python/ray/includes/common.pxi @@ -14,6 +14,15 @@ from ray.includes.common cimport ( kGcsAutoscalerV2EnabledKey, kGcsAutoscalerClusterConfigKey, kGcsPidKey, + kNodeTypeNameEnv, + kNodeMarketTypeEnv, + kNodeRegionEnv, + kNodeZoneEnv, + kLabelKeyNodeAcceleratorType, + kLabelKeyNodeMarketType, + kLabelKeyNodeRegion, + kLabelKeyNodeZone, + kLabelKeyNodeGroup, ) from ray.exceptions import ( @@ -128,3 +137,15 @@ GCS_AUTOSCALER_STATE_NAMESPACE = kGcsAutoscalerStateNamespace.decode() GCS_AUTOSCALER_V2_ENABLED_KEY = kGcsAutoscalerV2EnabledKey.decode() GCS_AUTOSCALER_CLUSTER_CONFIG_KEY = kGcsAutoscalerClusterConfigKey.decode() GCS_PID_KEY = kGcsPidKey.decode() + +# Ray node label related constants form src/ray/common/constants.h +NODE_TYPE_NAME_ENV = kNodeTypeNameEnv.decode() +NODE_MARKET_TYPE_ENV = kNodeMarketTypeEnv.decode() +NODE_REGION_ENV = kNodeRegionEnv.decode() +NODE_ZONE_ENV = kNodeZoneEnv.decode() + +RAY_NODE_ACCELERATOR_TYPE_KEY = kLabelKeyNodeAcceleratorType.decode() +RAY_NODE_MARKET_TYPE_KEY = kLabelKeyNodeMarketType.decode() +RAY_NODE_REGION_KEY = kLabelKeyNodeRegion.decode() +RAY_NODE_ZONE_KEY = kLabelKeyNodeZone.decode() +RAY_NODE_GROUP_KEY = kLabelKeyNodeGroup.decode() diff --git a/src/ray/common/constants.h b/src/ray/common/constants.h index edac51c437f3..0d0b5028dc0d 100644 --- a/src/ray/common/constants.h +++ b/src/ray/common/constants.h @@ -76,7 +76,11 @@ constexpr char kGcsAutoscalerClusterConfigKey[] = "__autoscaler_cluster_config"; /// Name for cloud instance id env constexpr char kNodeCloudInstanceIdEnv[] = "RAY_CLOUD_INSTANCE_ID"; +// ENV keys for Ray node labels constexpr char kNodeTypeNameEnv[] = "RAY_NODE_TYPE_NAME"; +constexpr char kNodeMarketTypeEnv[] = "RAY_NODE_MARKET_TYPE"; +constexpr char kNodeRegionEnv[] = "RAY_NODE_REGION"; +constexpr char kNodeZoneEnv[] = "RAY_NODE_ZONE"; constexpr char kNodeCloudInstanceTypeNameEnv[] = "RAY_CLOUD_INSTANCE_TYPE_NAME"; @@ -97,8 +101,13 @@ constexpr char kLibraryPathEnvName[] = "LD_LIBRARY_PATH"; #endif #define RAY_LABEL_KEY_PREFIX "ray.io/" -/// Default node label key: node_id +/// Default node label keys populated by the Raylet constexpr char kLabelKeyNodeID[] = RAY_LABEL_KEY_PREFIX "node_id"; +constexpr char kLabelKeyNodeAcceleratorType[] = RAY_LABEL_KEY_PREFIX "accelerator-type"; +constexpr char kLabelKeyNodeMarketType[] = RAY_LABEL_KEY_PREFIX "market-type"; +constexpr char kLabelKeyNodeRegion[] = RAY_LABEL_KEY_PREFIX "availability-region"; +constexpr char kLabelKeyNodeZone[] = RAY_LABEL_KEY_PREFIX "availability-zone"; +constexpr char kLabelKeyNodeGroup[] = RAY_LABEL_KEY_PREFIX "node-group"; #undef RAY_LABEL_KEY_PREFIX /// All nodes implicitly have resources with this prefix and the quantity is 1. From c2894bf86658a3f0e0984c4ff3c534a05e24f554 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Fri, 6 Jun 2025 01:37:46 +0000 Subject: [PATCH 06/39] Add test fixture Signed-off-by: Ryan O'Leary --- python/ray/tests/test_node_labels.py | 39 ++++++++++++++++++---------- 1 file changed, 25 insertions(+), 14 deletions(-) diff --git a/python/ray/tests/test_node_labels.py b/python/ray/tests/test_node_labels.py index 4c3e20cbd069..75b7e6440d41 100644 --- a/python/ray/tests/test_node_labels.py +++ b/python/ray/tests/test_node_labels.py @@ -162,16 +162,37 @@ def test_ray_start_set_node_labels_from_file(shutdown_only): os.remove(test_file_path) -def test_add_default_ray_node_labels(shutdown_only): - # Set environment variables for test +@pytest.fixture +def ray_node_labels_env(): + # Ray default node label keys and TPU accelerator env key + keys = [ + "RAY_NODE_MARKET_TYPE", + "RAY_NODE_TYPE_NAME", + "RAY_NODE_REGION", + "RAY_NODE_ZONE", + "TPU_ACCELERATOR_TYPE", + ] + + # Save original vals for env vars under test + original_env = {k: os.environ.get(k) for k in keys} + + # Set env var values for test os.environ["RAY_NODE_MARKET_TYPE"] = "spot" os.environ["RAY_NODE_TYPE_NAME"] = "worker-group-1" os.environ["RAY_NODE_REGION"] = "us-central2" os.environ["RAY_NODE_ZONE"] = "us-central2-b" - - # set env var for AcceleratorManager to detect os.environ["TPU_ACCELERATOR_TYPE"] = "v4-16" + yield + # Restore original values + for k in keys: + if original_env[k] is None: + os.environ.pop(k, None) + else: + os.environ[k] = original_env[k] + + +def test_add_default_ray_node_labels(shutdown_only, ray_node_labels_env): ray.init(resources={"TPU": 4}) node_info = ray.nodes()[0] labels = node_info["Labels"] @@ -182,16 +203,6 @@ def test_add_default_ray_node_labels(shutdown_only): assert labels.get("ray.io/availability-zone") == "us-central2-b" assert labels.get("ray.io/accelerator-type") == "TPU-V4" - # Unset env vars - for k in [ - "RAY_NODE_MARKET_TYPE", - "RAY_NODE_TYPE_NAME", - "RAY_NODE_ZONE", - "RAY_NODE_REGION", - "TPU_ACCELERATOR_TYPE", - ]: - del os.environ[k] - if __name__ == "__main__": sys.exit(pytest.main(["-sv", __file__])) From 0036699da4e8e3ac827e50a2a6ca0e66e3e2c172 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Thu, 12 Jun 2025 01:22:06 +0000 Subject: [PATCH 07/39] Fix comments Signed-off-by: Ryan O'Leary --- python/ray/_private/node.py | 56 ++++++++++++++++++---------- python/ray/tests/test_node_labels.py | 2 +- 2 files changed, 38 insertions(+), 20 deletions(-) diff --git a/python/ray/_private/node.py b/python/ray/_private/node.py index fde0ee987cbc..72ecde44115d 100644 --- a/python/ray/_private/node.py +++ b/python/ray/_private/node.py @@ -1267,8 +1267,11 @@ def start_raylet( create_out=True, create_err=True, ) - resource_spec = self.get_resource_spec() - node_labels = self._add_default_ray_node_labels(resource_spec) + # Add default labels to Ray node labels, with labels passed in + # from `--labels` taking precedence. + default_labels = self._get_default_ray_node_labels() + node_labels = {**default_labels, **self._get_node_labels()} + process_info = ray._private.services.start_raylet( self.redis_address, self.gcs_address, @@ -1284,7 +1287,7 @@ def start_raylet( self._session_dir, self._runtime_env_dir, self._logs_dir, - resource_spec, + self.get_resource_spec(), plasma_directory, fallback_directory, object_store_memory, @@ -1916,8 +1919,11 @@ def _record_stats(self): # to avoid any issues. record_hardware_usage(cpu_model_name[:50]) - def _add_default_ray_node_labels(self, resource_spec): - node_labels = self._get_node_labels() + # _get_default_ray_node_labels is a helper function to return a dictionary with + # default ray.io/ labels set for this node. + def _get_default_ray_node_labels(self): + default_labels = {} + resource_spec = self.get_resource_spec() # Get environment variables populated from K8s Pod Spec node_group = os.environ.get(ray._raylet.NODE_TYPE_NAME_ENV, "") @@ -1927,24 +1933,36 @@ def _add_default_ray_node_labels(self, resource_spec): # Map environment variables to default ray node labels if market_type: - node_labels[ray._raylet.RAY_NODE_MARKET_TYPE_KEY] = market_type + default_labels[ray._raylet.RAY_NODE_MARKET_TYPE_KEY] = market_type if node_group: - node_labels[ray._raylet.RAY_NODE_GROUP_KEY] = node_group + default_labels[ray._raylet.RAY_NODE_GROUP_KEY] = node_group if availability_zone: - node_labels[ray._raylet.RAY_NODE_ZONE_KEY] = availability_zone + default_labels[ray._raylet.RAY_NODE_ZONE_KEY] = availability_zone if availability_region: - node_labels[ray._raylet.RAY_NODE_REGION_KEY] = availability_region + default_labels[ray._raylet.RAY_NODE_REGION_KEY] = availability_region - # Get accelerator type from ResourceSpec - accelerator_type = None + # Get accelerator type from AcceleratorManager if resource_spec.resolved(): - for key in resource_spec.resources: - if key.startswith(ray_constants.RESOURCE_CONSTRAINT_PREFIX): - accelerator_type = key.split( - ray_constants.RESOURCE_CONSTRAINT_PREFIX - )[1] + # Check first that the resource configuration passed to the Raylet has been set. + # Only check for accelerator-type if resource_spec.resources is not None. + for ( + accelerator_resource_name + ) in ray._private.accelerators.get_all_accelerator_resource_names(): + accelerator_manager = ( + ray._private.accelerators.get_accelerator_manager_for_resource( + accelerator_resource_name + ) + ) + accelerator_type = ( + accelerator_manager.get_current_node_accelerator_type() + ) + num_accelerators = resource_spec.resources.get( + accelerator_resource_name, None + ) + if accelerator_type and num_accelerators: + default_labels[ + ray._raylet.RAY_NODE_ACCELERATOR_TYPE_KEY + ] = accelerator_type break # Only add one value for ray.io/accelerator-type - if accelerator_type: - node_labels[ray._raylet.RAY_NODE_ACCELERATOR_TYPE_KEY] = accelerator_type - return node_labels + return default_labels diff --git a/python/ray/tests/test_node_labels.py b/python/ray/tests/test_node_labels.py index 75b7e6440d41..2804b8b1284f 100644 --- a/python/ray/tests/test_node_labels.py +++ b/python/ray/tests/test_node_labels.py @@ -192,7 +192,7 @@ def ray_node_labels_env(): os.environ[k] = original_env[k] -def test_add_default_ray_node_labels(shutdown_only, ray_node_labels_env): +def test_get_default_ray_node_labels(shutdown_only, ray_node_labels_env): ray.init(resources={"TPU": 4}) node_info = ray.nodes()[0] labels = node_info["Labels"] From 54143c40003c24702afd2774e171cd3e40a7fe18 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Thu, 12 Jun 2025 08:44:06 +0000 Subject: [PATCH 08/39] Fix Ray data test Signed-off-by: Ryan O'Leary --- python/ray/data/tests/test_sql.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/python/ray/data/tests/test_sql.py b/python/ray/data/tests/test_sql.py index ffc9a657913f..20612636fb58 100644 --- a/python/ray/data/tests/test_sql.py +++ b/python/ray/data/tests/test_sql.py @@ -306,7 +306,10 @@ def request_get_mock(url, params=None, **kwargs): assert False, "Invalid request." - with mock.patch("requests.get", request_get_mock), mock.patch( + with mock.patch( + "ray._private.accelerators.tpu.TPUAcceleratorManager.get_current_node_accelerator_type", + return_value=None, + ), mock.patch("requests.get", request_get_mock), mock.patch( "requests.post", request_post_mock ), mock.patch.dict( os.environ, From bbf776921c89cd74b7be11eeea88102e7703f868 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Fri, 13 Jun 2025 12:27:06 +0000 Subject: [PATCH 09/39] Fix exit_logging test and add try block Signed-off-by: Ryan O'Leary --- python/ray/_private/node.py | 33 ++++++++++++++++++++++---------- python/ray/tests/test_basic_5.py | 23 +++++++++++++--------- 2 files changed, 37 insertions(+), 19 deletions(-) diff --git a/python/ray/_private/node.py b/python/ray/_private/node.py index 72ecde44115d..b3140ed140f7 100644 --- a/python/ray/_private/node.py +++ b/python/ray/_private/node.py @@ -1948,18 +1948,31 @@ def _get_default_ray_node_labels(self): for ( accelerator_resource_name ) in ray._private.accelerators.get_all_accelerator_resource_names(): - accelerator_manager = ( - ray._private.accelerators.get_accelerator_manager_for_resource( - accelerator_resource_name + try: + accelerator_manager = ( + ray._private.accelerators.get_accelerator_manager_for_resource( + accelerator_resource_name + ) + ) + num_accelerators = resource_spec.resources.get( + accelerator_resource_name, None + ) + accelerator_type = ( + accelerator_manager.get_current_node_accelerator_type() + ) + if num_accelerators is None: + # Try to automatically detect the number of accelerators. + num_accelerators = ( + accelerator_manager.get_current_node_num_accelerators() + ) + except Exception: + accelerator_type = None + logger.debug( + "Failed to detect accelerator type for default Ray labels during Node init." ) - ) - accelerator_type = ( - accelerator_manager.get_current_node_accelerator_type() - ) - num_accelerators = resource_spec.resources.get( - accelerator_resource_name, None - ) if accelerator_type and num_accelerators: + # Only set label if both accelerator type and the number of + # accelerators were successfully detected. default_labels[ ray._raylet.RAY_NODE_ACCELERATOR_TYPE_KEY ] = accelerator_type diff --git a/python/ray/tests/test_basic_5.py b/python/ray/tests/test_basic_5.py index 765bb4d72ee8..dfe304e63a2a 100644 --- a/python/ray/tests/test_basic_5.py +++ b/python/ray/tests/test_basic_5.py @@ -131,17 +131,22 @@ def test_exit_logging(): log = run_string_as_driver( """ import ray +import unittest.mock -@ray.remote -class A: - def pid(self): - import os - return os.getpid() - +# Patch method that makes calls to AcceleratorManager in Node init +with unittest.mock.patch( + "ray._private.node.Node._get_default_ray_node_labels", + return_value={} +): + @ray.remote + class A: + def pid(self): + import os + return os.getpid() -a = A.remote() -ray.get(a.pid.remote()) - """ + a = A.remote() + ray.get(a.pid.remote()) + """ ) assert "Traceback" not in log From f6a0c26a75f561c58f61b88beb6c033bc3d976f5 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Mon, 23 Jun 2025 19:05:07 +0000 Subject: [PATCH 10/39] Change node id constant Signed-off-by: Ryan O'Leary --- python/ray/tests/test_node_labels.py | 4 ++-- src/ray/common/constants.h | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/python/ray/tests/test_node_labels.py b/python/ray/tests/test_node_labels.py index 2804b8b1284f..04ef07740bb8 100644 --- a/python/ray/tests/test_node_labels.py +++ b/python/ray/tests/test_node_labels.py @@ -14,7 +14,7 @@ def check_cmd_stderr(cmd): def add_default_labels_for_test(node_info, labels): - labels["ray.io/node_id"] = node_info["NodeID"] + labels["ray.io/node-id"] = node_info["NodeID"] return labels @@ -87,7 +87,7 @@ def test_ray_start_set_node_labels_value_error(): assert "Label string is not a key-value pair." in out out = check_cmd_stderr( - ["ray", "start", "--head", '--labels={"ray.io/node_id":"111"}'] + ["ray", "start", "--head", '--labels={"ray.io/node-id":"111"}'] ) assert "Label string is not a key-value pair" in out diff --git a/src/ray/common/constants.h b/src/ray/common/constants.h index 0d0b5028dc0d..85ac66524982 100644 --- a/src/ray/common/constants.h +++ b/src/ray/common/constants.h @@ -102,7 +102,7 @@ constexpr char kLibraryPathEnvName[] = "LD_LIBRARY_PATH"; #define RAY_LABEL_KEY_PREFIX "ray.io/" /// Default node label keys populated by the Raylet -constexpr char kLabelKeyNodeID[] = RAY_LABEL_KEY_PREFIX "node_id"; +constexpr char kLabelKeyNodeID[] = RAY_LABEL_KEY_PREFIX "node-id"; constexpr char kLabelKeyNodeAcceleratorType[] = RAY_LABEL_KEY_PREFIX "accelerator-type"; constexpr char kLabelKeyNodeMarketType[] = RAY_LABEL_KEY_PREFIX "market-type"; constexpr char kLabelKeyNodeRegion[] = RAY_LABEL_KEY_PREFIX "availability-region"; From d717762295916ca05791943aaabf27bb7823954b Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Tue, 24 Jun 2025 01:25:07 +0000 Subject: [PATCH 11/39] Fix tests that use node_id Signed-off-by: Ryan O'Leary --- python/ray/tests/test_node_label_scheduling_strategy.py | 2 +- python/ray/tests/test_state_api.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/python/ray/tests/test_node_label_scheduling_strategy.py b/python/ray/tests/test_node_label_scheduling_strategy.py index 9493cd68acf2..d13fc587ac08 100644 --- a/python/ray/tests/test_node_label_scheduling_strategy.py +++ b/python/ray/tests/test_node_label_scheduling_strategy.py @@ -106,7 +106,7 @@ def test_node_label_scheduling_in_cluster(ray_start_cluster): assert ray.get(actor.get_node_id.remote(), timeout=3) == node_1 actor = MyActor.options( - scheduling_strategy=NodeLabelSchedulingStrategy({"ray.io/node_id": In(node_4)}) + scheduling_strategy=NodeLabelSchedulingStrategy({"ray.io/node-id": In(node_4)}) ).remote() assert ray.get(actor.get_node_id.remote(), timeout=3) == node_4 diff --git a/python/ray/tests/test_state_api.py b/python/ray/tests/test_state_api.py index 1ea400e82a48..77c7b9a72897 100644 --- a/python/ray/tests/test_state_api.py +++ b/python/ray/tests/test_state_api.py @@ -2198,7 +2198,7 @@ def verify(): nodes = list_nodes(detail=True) for node in nodes: assert is_hex(node["node_id"]) - assert node["labels"] == {"ray.io/node_id": node["node_id"]} + assert node["labels"] == {"ray.io/node-id": node["node_id"]} if node["node_name"] == "head_node": assert node["is_head_node"] assert node["state"] == "ALIVE" From 241861f98746793260b15a89d57f3a0108db6d66 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Tue, 24 Jun 2025 21:05:33 +0000 Subject: [PATCH 12/39] Add accelerator util functions Signed-off-by: Ryan O'Leary --- .../accelerators/accelerator_utils.py | 95 ++++++++++++++++ python/ray/_private/node.py | 44 ++------ python/ray/tests/BUILD | 1 + .../accelerators/test_accelerator_utils.py | 105 ++++++++++++++++++ python/ray/tests/test_basic_5.py | 9 +- 5 files changed, 215 insertions(+), 39 deletions(-) create mode 100644 python/ray/_private/accelerators/accelerator_utils.py create mode 100644 python/ray/tests/accelerators/test_accelerator_utils.py diff --git a/python/ray/_private/accelerators/accelerator_utils.py b/python/ray/_private/accelerators/accelerator_utils.py new file mode 100644 index 000000000000..ffc4436a18a6 --- /dev/null +++ b/python/ray/_private/accelerators/accelerator_utils.py @@ -0,0 +1,95 @@ +from typing import Dict, Optional, Tuple + +from ray._common.utils import RESOURCE_CONSTRAINT_PREFIX +from ray._private import accelerators + + +def resolve_and_update_accelerator_resources( + num_gpus: int, resources: Dict[str, float] +) -> int: + """Detect and update accelerator resources on a node.""" + for accelerator_resource_name in accelerators.get_all_accelerator_resource_names(): + accelerator_manager = accelerators.get_accelerator_manager_for_resource( + accelerator_resource_name + ) + # Respect configured value for GPUs if set + num_accelerators = None + if accelerator_resource_name == "GPU": + num_accelerators = num_gpus + else: + num_accelerators = resources.get(accelerator_resource_name, None) + visible_accelerator_ids = ( + accelerator_manager.get_current_process_visible_accelerator_ids() + ) + + # Check that the number of accelerators that the raylet wants doesn't + # exceed the amount allowed by visible accelerator ids. + if ( + num_accelerators is not None + and visible_accelerator_ids is not None + and num_accelerators > len(visible_accelerator_ids) + ): + raise ValueError( + f"Attempting to start raylet with {num_accelerators} " + f"{accelerator_resource_name}, " + f"but {accelerator_manager.get_visible_accelerator_ids_env_var()} " + f"contains {visible_accelerator_ids}." + ) + + if num_accelerators is None: + # Try to automatically detect the number of accelerators. + num_accelerators = accelerator_manager.get_current_node_num_accelerators() + # Don't use more accelerators than allowed by visible accelerator ids. + if visible_accelerator_ids is not None: + num_accelerators = min(num_accelerators, len(visible_accelerator_ids)) + + if num_accelerators: + if accelerator_resource_name == "GPU": + num_gpus = num_accelerators + else: + resources[accelerator_resource_name] = num_accelerators + + accelerator_type = accelerator_manager.get_current_node_accelerator_type() + if accelerator_type: + resources[f"{RESOURCE_CONSTRAINT_PREFIX}{accelerator_type}"] = 1 + + from ray._private.usage import usage_lib + + usage_lib.record_hardware_usage(accelerator_type) + additional_resources = ( + accelerator_manager.get_current_node_additional_resources() + ) + if additional_resources: + resources.update(additional_resources) + + return num_gpus or 0 + + +def get_first_detectable_accelerator_type( + resources: dict, +) -> Tuple[Optional[str], Optional[int]]: + """ + Returns the first detectable accelerator type on this node and num_gpus. + + Args: + resources: A dictionary of resolved resource quantities. + + Returns: + Tuple of: + - Accelerator type string (e.g., 'A100') or None if undetectable + """ + for accelerator_resource_name in accelerators.get_all_accelerator_resource_names(): + accelerator_manager = accelerators.get_accelerator_manager_for_resource( + accelerator_resource_name + ) + + num_accelerators = resources.get(accelerator_resource_name) + if num_accelerators is None: + num_accelerators = accelerator_manager.get_current_node_num_accelerators() + + accelerator_type = accelerator_manager.get_current_node_accelerator_type() + + if accelerator_type and num_accelerators: + return accelerator_type + + return None diff --git a/python/ray/_private/node.py b/python/ray/_private/node.py index b3140ed140f7..8eaa80547f7c 100644 --- a/python/ray/_private/node.py +++ b/python/ray/_private/node.py @@ -24,6 +24,9 @@ import ray._private.services from ray._common.ray_constants import LOGGING_ROTATE_BACKUP_COUNT, LOGGING_ROTATE_BYTES from ray._common.utils import try_to_create_directory +from ray._private.accelerators.accelerator_utils import ( + get_first_detectable_accelerator_type, +) from ray._private.resource_isolation_config import ResourceIsolationConfig from ray._private.resource_spec import ResourceSpec from ray._private.services import get_address, serialize_config @@ -1943,39 +1946,12 @@ def _get_default_ray_node_labels(self): # Get accelerator type from AcceleratorManager if resource_spec.resolved(): - # Check first that the resource configuration passed to the Raylet has been set. - # Only check for accelerator-type if resource_spec.resources is not None. - for ( - accelerator_resource_name - ) in ray._private.accelerators.get_all_accelerator_resource_names(): - try: - accelerator_manager = ( - ray._private.accelerators.get_accelerator_manager_for_resource( - accelerator_resource_name - ) - ) - num_accelerators = resource_spec.resources.get( - accelerator_resource_name, None - ) - accelerator_type = ( - accelerator_manager.get_current_node_accelerator_type() - ) - if num_accelerators is None: - # Try to automatically detect the number of accelerators. - num_accelerators = ( - accelerator_manager.get_current_node_num_accelerators() - ) - except Exception: - accelerator_type = None - logger.debug( - "Failed to detect accelerator type for default Ray labels during Node init." - ) - if accelerator_type and num_accelerators: - # Only set label if both accelerator type and the number of - # accelerators were successfully detected. - default_labels[ - ray._raylet.RAY_NODE_ACCELERATOR_TYPE_KEY - ] = accelerator_type - break # Only add one value for ray.io/accelerator-type + accelerator_type = get_first_detectable_accelerator_type( + resource_spec.resources or {} + ) + if accelerator_type: + default_labels[ + ray._raylet.RAY_NODE_ACCELERATOR_TYPE_KEY + ] = accelerator_type return default_labels diff --git a/python/ray/tests/BUILD b/python/ray/tests/BUILD index 584a331ee6b0..cffddbc39133 100644 --- a/python/ray/tests/BUILD +++ b/python/ray/tests/BUILD @@ -495,6 +495,7 @@ py_test_module_list( py_test_module_list( size = "small", files = [ + "accelerators/test_accelerator_utils.py", "accelerators/test_accelerators.py", "accelerators/test_amd_gpu.py", "accelerators/test_intel_gpu.py", diff --git a/python/ray/tests/accelerators/test_accelerator_utils.py b/python/ray/tests/accelerators/test_accelerator_utils.py new file mode 100644 index 000000000000..cf71a732dc9f --- /dev/null +++ b/python/ray/tests/accelerators/test_accelerator_utils.py @@ -0,0 +1,105 @@ +import pytest +import sys +from unittest.mock import patch, MagicMock + +from ray._private.accelerators.accelerator_utils import ( + resolve_and_update_accelerator_resources, + get_first_detectable_accelerator_type, +) + + +@patch("ray._private.usage.usage_lib.record_hardware_usage") +@patch("ray._private.accelerators.get_all_accelerator_resource_names") +@patch("ray._private.accelerators.get_accelerator_manager_for_resource") +def test_resolve_and_update_accelerators_with_gpu( + mock_get_mgr, mock_get_names, mock_record_usage +): + mock_mgr = MagicMock() + mock_mgr.get_current_process_visible_accelerator_ids.return_value = ["0", "1"] + mock_mgr.get_current_node_num_accelerators.return_value = 2 + mock_mgr.get_current_node_accelerator_type.return_value = "A100" + mock_mgr.get_current_node_additional_resources.return_value = {"tensor_cores": 4} + mock_mgr.get_visible_accelerator_ids_env_var.return_value = "CUDA_VISIBLE_DEVICES" + + mock_get_mgr.return_value = mock_mgr + mock_get_names.return_value = ["GPU"] + + resources = {} + num_gpus = resolve_and_update_accelerator_resources(1, resources) + + assert num_gpus == 1 + assert resources["tensor_cores"] == 4 + + +@patch("ray._private.usage.usage_lib.record_hardware_usage") +@patch("ray._private.accelerators.get_all_accelerator_resource_names") +@patch("ray._private.accelerators.get_accelerator_manager_for_resource") +def test_auto_detect_num_gpus_with_visibility_limit( + mock_get_mgr, mock_get_names, mock_record_usage +): + mock_mgr = MagicMock() + mock_mgr.get_current_process_visible_accelerator_ids.return_value = ["0"] + mock_mgr.get_current_node_num_accelerators.return_value = 4 + mock_mgr.get_current_node_accelerator_type.return_value = "L4" + mock_mgr.get_current_node_additional_resources.return_value = {} + mock_mgr.get_visible_accelerator_ids_env_var.return_value = "CUDA_VISIBLE_DEVICES" + + mock_get_mgr.return_value = mock_mgr + mock_get_names.return_value = ["GPU"] + + resources = {} + num_gpus = resolve_and_update_accelerator_resources(None, resources) + assert num_gpus == 1 + + +@patch("ray._private.accelerators.get_all_accelerator_resource_names") +@patch("ray._private.accelerators.get_accelerator_manager_for_resource") +def test_resolve_and_update_accelerators_too_many_requested( + mock_get_mgr, mock_get_names +): + mock_mgr = MagicMock() + mock_mgr.get_current_process_visible_accelerator_ids.return_value = ["0", "1"] + mock_mgr.get_current_node_num_accelerators.return_value = 2 + mock_mgr.get_current_node_accelerator_type.return_value = None + mock_mgr.get_current_node_additional_resources.return_value = {} + mock_mgr.get_visible_accelerator_ids_env_var.return_value = "CUDA_VISIBLE_DEVICES" + + mock_get_mgr.return_value = mock_mgr + mock_get_names.return_value = ["GPU"] + + resources = {} + + with pytest.raises(ValueError, match="Attempting to start raylet with 3 GPU"): + resolve_and_update_accelerator_resources(3, resources) + + +@patch("ray._private.accelerators.get_all_accelerator_resource_names") +@patch("ray._private.accelerators.get_accelerator_manager_for_resource") +def test_get_first_detectable_accelerator_type(mock_get_mgr, mock_get_names): + mock_mgr = MagicMock() + mock_mgr.get_current_node_num_accelerators.return_value = 2 + mock_mgr.get_current_node_accelerator_type.return_value = "B200" + + mock_get_mgr.return_value = mock_mgr + mock_get_names.return_value = ["GPU"] + + accelerator_type = get_first_detectable_accelerator_type({"GPU": 2}) + assert accelerator_type == "B200" + + +@patch("ray._private.accelerators.get_all_accelerator_resource_names") +@patch("ray._private.accelerators.get_accelerator_manager_for_resource") +def test_detect_non_gpu_accelerator_type(mock_get_mgr, mock_get_names): + mock_mgr = MagicMock() + mock_mgr.get_current_node_num_accelerators.return_value = 4 + mock_mgr.get_current_node_accelerator_type.return_value = "TPU-V6E" + + mock_get_mgr.return_value = mock_mgr + mock_get_names.return_value = ["TPU"] + + accelerator_type = get_first_detectable_accelerator_type({"TPU": 4}) + assert accelerator_type == "TPU-V6E" + + +if __name__ == "__main__": + sys.exit(pytest.main(["-sv", __file__])) diff --git a/python/ray/tests/test_basic_5.py b/python/ray/tests/test_basic_5.py index dfe304e63a2a..44c327c0010f 100644 --- a/python/ray/tests/test_basic_5.py +++ b/python/ray/tests/test_basic_5.py @@ -133,11 +133,10 @@ def test_exit_logging(): import ray import unittest.mock -# Patch method that makes calls to AcceleratorManager in Node init -with unittest.mock.patch( - "ray._private.node.Node._get_default_ray_node_labels", - return_value={} -): +# Patch methods that makes calls to AcceleratorManager in Node init +with unittest.mock.patch("ray._private.accelerators.accelerator_utils.resolve_and_update_accelerator_resources", return_value=0), \\ + unittest.mock.patch("ray._private.accelerators.get_all_accelerator_resource_names", return_value=[]): + @ray.remote class A: def pid(self): From 10a8ad6fe385d81c9e8bcf1f8300069d6dc30b46 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Mon, 14 Jul 2025 21:01:01 +0000 Subject: [PATCH 13/39] Fix comments and add ResourceAndLabelSpec Signed-off-by: Ryan O'Leary --- .../accelerators/accelerator_utils.py | 95 ---------------- python/ray/_private/accelerators/utils.py | 103 ++++++++++++++++++ python/ray/_private/node.py | 100 +++-------------- python/ray/_private/resource_spec.py | 94 ++++++++++++++-- python/ray/_private/services.py | 13 +-- python/ray/tests/BUILD | 2 +- ...est_accelerator_utils.py => test_utils.py} | 82 +++++++------- python/ray/tests/test_basic_5.py | 3 +- src/ray/common/constants.h | 18 ++- 9 files changed, 266 insertions(+), 244 deletions(-) delete mode 100644 python/ray/_private/accelerators/accelerator_utils.py create mode 100644 python/ray/_private/accelerators/utils.py rename python/ray/tests/accelerators/{test_accelerator_utils.py => test_utils.py} (51%) diff --git a/python/ray/_private/accelerators/accelerator_utils.py b/python/ray/_private/accelerators/accelerator_utils.py deleted file mode 100644 index ffc4436a18a6..000000000000 --- a/python/ray/_private/accelerators/accelerator_utils.py +++ /dev/null @@ -1,95 +0,0 @@ -from typing import Dict, Optional, Tuple - -from ray._common.utils import RESOURCE_CONSTRAINT_PREFIX -from ray._private import accelerators - - -def resolve_and_update_accelerator_resources( - num_gpus: int, resources: Dict[str, float] -) -> int: - """Detect and update accelerator resources on a node.""" - for accelerator_resource_name in accelerators.get_all_accelerator_resource_names(): - accelerator_manager = accelerators.get_accelerator_manager_for_resource( - accelerator_resource_name - ) - # Respect configured value for GPUs if set - num_accelerators = None - if accelerator_resource_name == "GPU": - num_accelerators = num_gpus - else: - num_accelerators = resources.get(accelerator_resource_name, None) - visible_accelerator_ids = ( - accelerator_manager.get_current_process_visible_accelerator_ids() - ) - - # Check that the number of accelerators that the raylet wants doesn't - # exceed the amount allowed by visible accelerator ids. - if ( - num_accelerators is not None - and visible_accelerator_ids is not None - and num_accelerators > len(visible_accelerator_ids) - ): - raise ValueError( - f"Attempting to start raylet with {num_accelerators} " - f"{accelerator_resource_name}, " - f"but {accelerator_manager.get_visible_accelerator_ids_env_var()} " - f"contains {visible_accelerator_ids}." - ) - - if num_accelerators is None: - # Try to automatically detect the number of accelerators. - num_accelerators = accelerator_manager.get_current_node_num_accelerators() - # Don't use more accelerators than allowed by visible accelerator ids. - if visible_accelerator_ids is not None: - num_accelerators = min(num_accelerators, len(visible_accelerator_ids)) - - if num_accelerators: - if accelerator_resource_name == "GPU": - num_gpus = num_accelerators - else: - resources[accelerator_resource_name] = num_accelerators - - accelerator_type = accelerator_manager.get_current_node_accelerator_type() - if accelerator_type: - resources[f"{RESOURCE_CONSTRAINT_PREFIX}{accelerator_type}"] = 1 - - from ray._private.usage import usage_lib - - usage_lib.record_hardware_usage(accelerator_type) - additional_resources = ( - accelerator_manager.get_current_node_additional_resources() - ) - if additional_resources: - resources.update(additional_resources) - - return num_gpus or 0 - - -def get_first_detectable_accelerator_type( - resources: dict, -) -> Tuple[Optional[str], Optional[int]]: - """ - Returns the first detectable accelerator type on this node and num_gpus. - - Args: - resources: A dictionary of resolved resource quantities. - - Returns: - Tuple of: - - Accelerator type string (e.g., 'A100') or None if undetectable - """ - for accelerator_resource_name in accelerators.get_all_accelerator_resource_names(): - accelerator_manager = accelerators.get_accelerator_manager_for_resource( - accelerator_resource_name - ) - - num_accelerators = resources.get(accelerator_resource_name) - if num_accelerators is None: - num_accelerators = accelerator_manager.get_current_node_num_accelerators() - - accelerator_type = accelerator_manager.get_current_node_accelerator_type() - - if accelerator_type and num_accelerators: - return accelerator_type - - return None diff --git a/python/ray/_private/accelerators/utils.py b/python/ray/_private/accelerators/utils.py new file mode 100644 index 000000000000..f65ecd08d11b --- /dev/null +++ b/python/ray/_private/accelerators/utils.py @@ -0,0 +1,103 @@ +from typing import Dict, Optional, Tuple + +from ray._common.utils import RESOURCE_CONSTRAINT_PREFIX +from ray._private import accelerators +from ray._private.accelerators import AcceleratorManager + + +def get_current_node_accelerator( + resources: Dict[str, float] +) -> Optional[Tuple[AcceleratorManager, str, int]]: + """ + Returns the AcceleratorManager, resource name, and accelerator count for the + accelerator associated with this node. + + This assumes each node has at most one accelerator type. If no accelerators + are present, returns None. + """ + for resource_name in accelerators.get_all_accelerator_resource_names(): + accelerator_manager = accelerators.get_accelerator_manager_for_resource( + resource_name + ) + accelerator_resource_name = accelerator_manager.get_resource_name() + + num_accelerators = resources.get(accelerator_resource_name) + if num_accelerators is None: + num_accelerators = accelerator_manager.get_current_node_num_accelerators() + visible_accelerator_ids = ( + accelerator_manager.get_current_process_visible_accelerator_ids() + ) + if visible_accelerator_ids is not None: + num_accelerators = min(num_accelerators, len(visible_accelerator_ids)) + + if num_accelerators > 0: + return accelerator_manager, accelerator_resource_name, num_accelerators + + return None + + +def resolve_and_update_accelerator_resources( + num_gpus: int, resources: Dict[str, float] +) -> int: + """Detect and update accelerator resources on a node.""" + accelerator = get_current_node_accelerator(resources) + if not accelerator: + return num_gpus or 0 + + accelerator_manager, accelerator_resource_name, num_accelerators = accelerator + visible_accelerator_ids = ( + accelerator_manager.get_current_process_visible_accelerator_ids() + ) + + # Respect configured value for GPUs if set + if accelerator_resource_name == "GPU" and num_gpus: + num_accelerators = num_gpus + + # Check that the number of accelerators that the raylet wants doesn't + # exceed the amount allowed by visible accelerator ids. + if ( + num_accelerators is not None + and visible_accelerator_ids is not None + and num_accelerators > len(visible_accelerator_ids) + ): + raise ValueError( + f"Attempting to start raylet with {num_accelerators} " + f"{accelerator_resource_name}, " + f"but {accelerator_manager.get_visible_accelerator_ids_env_var()} " + f"contains {visible_accelerator_ids}." + ) + + if num_accelerators: + if accelerator_resource_name == "GPU": + num_gpus = num_accelerators + else: + resources[accelerator_resource_name] = num_accelerators + + accelerator_type = accelerator_manager.get_current_node_accelerator_type() + if accelerator_type: + resources[f"{RESOURCE_CONSTRAINT_PREFIX}{accelerator_type}"] = 1 + + from ray._private.usage import usage_lib + + usage_lib.record_hardware_usage(accelerator_type) + additional_resources = ( + accelerator_manager.get_current_node_additional_resources() + ) + if additional_resources: + resources.update(additional_resources) + + return num_gpus or 0 + + +def get_current_node_accelerator_type(resources: dict) -> Optional[str]: + """ + Returns the accelerator type (e.g. 'A100') if detectable on this node, + or None if no accelerator is present. + """ + accelerator = get_current_node_accelerator(resources) + if accelerator: + accelerator_manager, _, _ = accelerator + accelerator_type = accelerator_manager.get_current_node_accelerator_type() + return accelerator_type or None + + return None diff --git a/python/ray/_private/node.py b/python/ray/_private/node.py index 8eaa80547f7c..64791ba24e15 100644 --- a/python/ray/_private/node.py +++ b/python/ray/_private/node.py @@ -24,11 +24,8 @@ import ray._private.services from ray._common.ray_constants import LOGGING_ROTATE_BACKUP_COUNT, LOGGING_ROTATE_BYTES from ray._common.utils import try_to_create_directory -from ray._private.accelerators.accelerator_utils import ( - get_first_detectable_accelerator_type, -) from ray._private.resource_isolation_config import ResourceIsolationConfig -from ray._private.resource_spec import ResourceSpec +from ray._private.resource_spec import ResourceAndLabelSpec from ray._private.services import get_address, serialize_config from ray._private.utils import ( is_in_test, @@ -140,7 +137,7 @@ def __init__( ), ) - self._resource_spec = None + self._resource_and_label_spec = None self._localhost = socket.gethostbyname("localhost") self._ray_params = ray_params self._config = ray_params._system_config or {} @@ -290,7 +287,7 @@ def __init__( self._ray_params.raylet_socket_name, default_prefix="raylet" ) # Set node labels from RayParams or environment override variables. - self._node_labels = self._get_node_labels() + # self._node_labels = self._get_node_labels() if ( self._ray_params.env_vars is not None and "RAY_OVERRIDE_NODE_ID_FOR_TESTING" in self._ray_params.env_vars @@ -524,43 +521,10 @@ def _init_temp(self): tpu_logs_symlink = os.path.join(self._logs_dir, "tpu_logs") try_to_symlink(tpu_logs_symlink, tpu_log_dir) - def _get_node_labels(self): - def merge_labels(env_override_labels, params_labels): - """Merges two dictionaries, picking from the - first in the event of a conflict. Also emit a warning on every - conflict. - """ - - result = params_labels.copy() - result.update(env_override_labels) - - for key in set(env_override_labels.keys()).intersection( - set(params_labels.keys()) - ): - if params_labels[key] != env_override_labels[key]: - logger.warning( - "Autoscaler is overriding your label:" - f"{key}: {params_labels[key]} to " - f"{key}: {env_override_labels[key]}." - ) - return result - - env_override_labels = {} - env_override_labels_string = os.getenv( - ray_constants.LABELS_ENVIRONMENT_VARIABLE - ) - if env_override_labels_string: - try: - env_override_labels = json.loads(env_override_labels_string) - except Exception: - logger.exception(f"Failed to load {env_override_labels_string}") - raise - logger.info(f"Autoscaler overriding labels: {env_override_labels}.") - - return merge_labels(env_override_labels, self._ray_params.labels or {}) - - def get_resource_spec(self): - """Resolve and return the current resource spec for the node.""" + def get_resource_and_label_spec(self): + """Resolve and return the current resource spec for the node. Also resolve and return + the merged user-input, Ray default, and autoscaler environment override labels from + the LabelSpec for this node.""" def merge_resources(env_dict, params_dict): """Separates special case params and merges two dictionaries, picking from the @@ -583,7 +547,7 @@ def merge_resources(env_dict, params_dict): ) return num_cpus, num_gpus, memory, object_store_memory, result - if not self._resource_spec: + if not self._resource_and_label_spec: env_resources = {} env_string = os.getenv(ray_constants.RESOURCES_ENVIRONMENT_VARIABLE) if env_string: @@ -600,7 +564,7 @@ def merge_resources(env_dict, params_dict): object_store_memory, resources, ) = merge_resources(env_resources, self._ray_params.resources) - self._resource_spec = ResourceSpec( + self._resource_and_label_spec = ResourceAndLabelSpec( self._ray_params.num_cpus if num_cpus is None else num_cpus, self._ray_params.num_gpus if num_gpus is None else num_gpus, self._ray_params.memory if memory is None else memory, @@ -610,8 +574,9 @@ def merge_resources(env_dict, params_dict): else object_store_memory ), resources, + self._ray_params.labels, ).resolve(is_head=self.head, node_ip_address=self.node_ip_address) - return self._resource_spec + return self._resource_and_label_spec @property def node_id(self): @@ -1270,10 +1235,6 @@ def start_raylet( create_out=True, create_err=True, ) - # Add default labels to Ray node labels, with labels passed in - # from `--labels` taking precedence. - default_labels = self._get_default_ray_node_labels() - node_labels = {**default_labels, **self._get_node_labels()} process_info = ray._private.services.start_raylet( self.redis_address, @@ -1290,7 +1251,7 @@ def start_raylet( self._session_dir, self._runtime_env_dir, self._logs_dir, - self.get_resource_spec(), + self.get_resource_and_label_spec(), plasma_directory, fallback_directory, object_store_memory, @@ -1323,7 +1284,6 @@ def start_raylet( env_updates=self._ray_params.env_vars, node_name=self._ray_params.node_name, webui=self._webui_url, - labels=node_labels, resource_isolation_config=self.resource_isolation_config, ) assert ray_constants.PROCESS_TYPE_RAYLET not in self.all_processes @@ -1485,7 +1445,7 @@ def start_ray_processes(self): # Make sure we don't call `determine_plasma_store_config` multiple # times to avoid printing multiple warnings. - resource_spec = self.get_resource_spec() + resource_spec = self.get_resource_and_label_spec() ( plasma_directory, @@ -1921,37 +1881,3 @@ def _record_stats(self): # so we truncate it to the first 50 characters # to avoid any issues. record_hardware_usage(cpu_model_name[:50]) - - # _get_default_ray_node_labels is a helper function to return a dictionary with - # default ray.io/ labels set for this node. - def _get_default_ray_node_labels(self): - default_labels = {} - resource_spec = self.get_resource_spec() - - # Get environment variables populated from K8s Pod Spec - node_group = os.environ.get(ray._raylet.NODE_TYPE_NAME_ENV, "") - market_type = os.environ.get(ray._raylet.NODE_MARKET_TYPE_ENV, "") - availability_region = os.environ.get(ray._raylet.NODE_REGION_ENV, "") - availability_zone = os.environ.get(ray._raylet.NODE_ZONE_ENV, "") - - # Map environment variables to default ray node labels - if market_type: - default_labels[ray._raylet.RAY_NODE_MARKET_TYPE_KEY] = market_type - if node_group: - default_labels[ray._raylet.RAY_NODE_GROUP_KEY] = node_group - if availability_zone: - default_labels[ray._raylet.RAY_NODE_ZONE_KEY] = availability_zone - if availability_region: - default_labels[ray._raylet.RAY_NODE_REGION_KEY] = availability_region - - # Get accelerator type from AcceleratorManager - if resource_spec.resolved(): - accelerator_type = get_first_detectable_accelerator_type( - resource_spec.resources or {} - ) - if accelerator_type: - default_labels[ - ray._raylet.RAY_NODE_ACCELERATOR_TYPE_KEY - ] = accelerator_type - - return default_labels diff --git a/python/ray/_private/resource_spec.py b/python/ray/_private/resource_spec.py index 1f2ffbe0497e..648a7b7acd09 100644 --- a/python/ray/_private/resource_spec.py +++ b/python/ray/_private/resource_spec.py @@ -1,7 +1,9 @@ +import json import logging +import os import sys from collections import namedtuple -from typing import Optional +from typing import Dict, Optional import ray import ray._private.ray_constants as ray_constants @@ -11,22 +13,23 @@ logger = logging.getLogger(__name__) -class ResourceSpec( +class ResourceAndLabelSpec( namedtuple( - "ResourceSpec", + "ResourceAndLabelSpec", [ "num_cpus", "num_gpus", "memory", "object_store_memory", "resources", + "labels", ], ) ): """Represents the resource configuration passed to a raylet. All fields can be None. Before starting services, resolve() should be - called to return a ResourceSpec with unknown values filled in with + called to return a ResourceAndLabelSpec with unknown values filled in with defaults based on the local machine specifications. Attributes: @@ -37,6 +40,8 @@ class ResourceSpec( Note that when calling to_resource_dict(), this will be scaled down by 30% to account for the global plasma LRU reserve. resources: The custom resources allocated for this raylet. + labels: The labels associated with this node. Labels can be used along + with resources for scheduling. """ def __new__( @@ -46,18 +51,20 @@ def __new__( memory=None, object_store_memory=None, resources=None, + labels=None, ): - return super(ResourceSpec, cls).__new__( + return super(ResourceAndLabelSpec, cls).__new__( cls, num_cpus, num_gpus, memory, object_store_memory, resources, + labels, ) def resolved(self): - """Returns if this ResourceSpec has default values filled out.""" + """Returns if this ResourceAndLabelSpec has default values filled out.""" for v in self._asdict().values(): if v is None: return False @@ -122,6 +129,8 @@ def resolve(self, is_head: bool, node_ip_address: Optional[str] = None): is_head: Whether this is the head node. node_ip_address: The IP address of the node that we are on. This is used to automatically create a node id resource. + Returns: + A new ResourceAndLabelSpec with resolved fields (e.g., CPU, GPU, memory). """ resources = (self.resources or {}).copy() @@ -272,12 +281,83 @@ def resolve(self, is_head: bool, node_ip_address: Optional[str] = None): ) ) - spec = ResourceSpec( + spec = ResourceAndLabelSpec( num_cpus, num_gpus, memory, object_store_memory, resources, + self.get_resolved_labels(), ) assert spec.resolved() return spec + + def _load_env_labels(self) -> Dict[str, str]: + env_override_labels = {} + env_override_labels_string = os.getenv( + ray_constants.LABELS_ENVIRONMENT_VARIABLE + ) + if env_override_labels_string: + try: + env_override_labels = json.loads(env_override_labels_string) + except Exception: + logger.exception(f"Failed to load {env_override_labels_string}") + raise + logger.info(f"Autoscaler overriding labels: {env_override_labels}.") + + return env_override_labels + + def _get_default_labels(self) -> Dict[str, str]: + default_labels = {} + + # Get environment variables populated from K8s Pod Spec + node_group = os.environ.get(ray._raylet.NODE_TYPE_NAME_ENV, "") + market_type = os.environ.get(ray._raylet.NODE_MARKET_TYPE_ENV, "") + availability_region = os.environ.get(ray._raylet.NODE_REGION_ENV, "") + availability_zone = os.environ.get(ray._raylet.NODE_ZONE_ENV, "") + + # Map environment variables to default ray node labels + if market_type: + default_labels[ray._raylet.RAY_NODE_MARKET_TYPE_KEY] = market_type + if node_group: + default_labels[ray._raylet.RAY_NODE_GROUP_KEY] = node_group + if availability_zone: + default_labels[ray._raylet.RAY_NODE_ZONE_KEY] = availability_zone + if availability_region: + default_labels[ray._raylet.RAY_NODE_REGION_KEY] = availability_region + + # Get accelerator type from AcceleratorManager + accelerator_type = get_current_node_accelerator_type(self.resources) + if accelerator_type: + default_labels[ray._raylet.RAY_NODE_ACCELERATOR_TYPE_KEY] = accelerator_type + + return default_labels + + def get_resolved_labels(self) -> Dict[str, str]: + """Merge environment override, user-input from params, and Ray default labels in + that order of precedence.""" + + env_labels = self._load_env_labels() + merged = dict(self._get_default_labels() or {}) + + # Merge user-specified labels from Ray params + for key, val in (self.labels or {}).items(): + if key in merged and merged[key] != val: + logger.warning( + f"User label is overriding Ray default label: {key}: " + f"{key}: {merged[key]} to " + f"{key}: {self.labels[key]}." + ) + merged[key] = val + + # Merge autoscaler override labels from environment + for key, val in (env_labels or {}).items(): + if key in merged and merged[key] != val: + logger.warning( + "Autoscaler is overriding your label:" + f"{key}: {merged[key]} to " + f"{key}: {env_labels[key]}." + ) + merged[key] = val + + return merged diff --git a/python/ray/_private/services.py b/python/ray/_private/services.py index ee7ac25403c9..7a21ef875795 100644 --- a/python/ray/_private/services.py +++ b/python/ray/_private/services.py @@ -1538,7 +1538,7 @@ def start_raylet( session_dir: str, resource_dir: str, log_dir: str, - resource_spec, + resource_and_label_spec, plasma_directory: str, fallback_directory: str, object_store_memory: int, @@ -1572,7 +1572,6 @@ def start_raylet( env_updates: Optional[dict] = None, node_name: Optional[str] = None, webui: Optional[str] = None, - labels: Optional[dict] = None, ): """Start a raylet, which is a combined local scheduler and object manager. @@ -1594,7 +1593,7 @@ def start_raylet( session_dir: The path of this session. resource_dir: The path of resource of this session . log_dir: The path of the dir where log files are created. - resource_spec: Resources for this raylet. + resource_and_label_spec: Resources and key-value labels for this raylet. plasma_directory: A directory where the Plasma memory mapped files will be created. fallback_directory: A directory where the Object store fallback files will be created. @@ -1649,7 +1648,6 @@ def start_raylet( env_updates: Environment variable overrides. node_name: The name of the node. webui: The url of the UI. - labels: The key-value labels of the node. Returns: ProcessInfo for the process that was started. """ @@ -1658,8 +1656,9 @@ def start_raylet( if use_valgrind and use_profiler: raise ValueError("Cannot use valgrind and profiler at the same time.") - assert resource_spec.resolved() - static_resources = resource_spec.to_resource_dict() + assert resource_and_label_spec.resolved() + static_resources = resource_and_label_spec.to_resource_dict() + labels = resource_and_label_spec.get_resolved_labels() # Limit the number of workers that can be started in parallel by the # raylet. However, make sure it is at least 1. @@ -1907,7 +1906,7 @@ def start_raylet( if worker_port_list is not None: command.append(f"--worker_port_list={worker_port_list}") command.append( - "--num_prestart_python_workers={}".format(int(resource_spec.num_cpus)) + "--num_prestart_python_workers={}".format(int(resource_and_label_spec.num_cpus)) ) command.append( "--dashboard_agent_command={}".format( diff --git a/python/ray/tests/BUILD b/python/ray/tests/BUILD index cffddbc39133..5b0ca9eca7cd 100644 --- a/python/ray/tests/BUILD +++ b/python/ray/tests/BUILD @@ -495,13 +495,13 @@ py_test_module_list( py_test_module_list( size = "small", files = [ - "accelerators/test_accelerator_utils.py", "accelerators/test_accelerators.py", "accelerators/test_amd_gpu.py", "accelerators/test_intel_gpu.py", "accelerators/test_npu.py", "accelerators/test_nvidia_gpu.py", "accelerators/test_tpu.py", + "accelerators/test_utils.py", "test_actor_lineage_reconstruction.py", "test_actor_out_of_order.py", "test_annotations.py", diff --git a/python/ray/tests/accelerators/test_accelerator_utils.py b/python/ray/tests/accelerators/test_utils.py similarity index 51% rename from python/ray/tests/accelerators/test_accelerator_utils.py rename to python/ray/tests/accelerators/test_utils.py index cf71a732dc9f..7ebd4d497c43 100644 --- a/python/ray/tests/accelerators/test_accelerator_utils.py +++ b/python/ray/tests/accelerators/test_utils.py @@ -2,103 +2,99 @@ import sys from unittest.mock import patch, MagicMock -from ray._private.accelerators.accelerator_utils import ( +from ray._common.utils import RESOURCE_CONSTRAINT_PREFIX +from ray._private.accelerators.utils import ( resolve_and_update_accelerator_resources, - get_first_detectable_accelerator_type, + get_current_node_accelerator_type, ) @patch("ray._private.usage.usage_lib.record_hardware_usage") -@patch("ray._private.accelerators.get_all_accelerator_resource_names") -@patch("ray._private.accelerators.get_accelerator_manager_for_resource") -def test_resolve_and_update_accelerators_with_gpu( - mock_get_mgr, mock_get_names, mock_record_usage -): +@patch("ray._private.accelerators.utils.get_current_node_accelerator") +def test_resolve_and_update_accelerators(mock_get_mgr, mock_record_usage): mock_mgr = MagicMock() + mock_mgr.get_resource_name.return_value = "GPU" mock_mgr.get_current_process_visible_accelerator_ids.return_value = ["0", "1"] mock_mgr.get_current_node_num_accelerators.return_value = 2 mock_mgr.get_current_node_accelerator_type.return_value = "A100" mock_mgr.get_current_node_additional_resources.return_value = {"tensor_cores": 4} mock_mgr.get_visible_accelerator_ids_env_var.return_value = "CUDA_VISIBLE_DEVICES" - mock_get_mgr.return_value = mock_mgr - mock_get_names.return_value = ["GPU"] + mock_get_mgr.return_value = (mock_mgr, "GPU", 2) resources = {} num_gpus = resolve_and_update_accelerator_resources(1, resources) assert num_gpus == 1 assert resources["tensor_cores"] == 4 + assert resources[f"{RESOURCE_CONSTRAINT_PREFIX}A100"] == 1 @patch("ray._private.usage.usage_lib.record_hardware_usage") -@patch("ray._private.accelerators.get_all_accelerator_resource_names") -@patch("ray._private.accelerators.get_accelerator_manager_for_resource") -def test_auto_detect_num_gpus_with_visibility_limit( - mock_get_mgr, mock_get_names, mock_record_usage -): +@patch("ray._private.accelerators.utils.get_current_node_accelerator") +def test_detect_num_gpus_with_visibility_limit(mock_get_mgr, mock_record_usage): mock_mgr = MagicMock() + mock_mgr.get_resource_name.return_value = "GPU" mock_mgr.get_current_process_visible_accelerator_ids.return_value = ["0"] mock_mgr.get_current_node_num_accelerators.return_value = 4 mock_mgr.get_current_node_accelerator_type.return_value = "L4" mock_mgr.get_current_node_additional_resources.return_value = {} mock_mgr.get_visible_accelerator_ids_env_var.return_value = "CUDA_VISIBLE_DEVICES" - mock_get_mgr.return_value = mock_mgr - mock_get_names.return_value = ["GPU"] + mock_get_mgr.return_value = (mock_mgr, "GPU", 1) resources = {} num_gpus = resolve_and_update_accelerator_resources(None, resources) assert num_gpus == 1 + assert resources[f"{RESOURCE_CONSTRAINT_PREFIX}L4"] == 1 -@patch("ray._private.accelerators.get_all_accelerator_resource_names") -@patch("ray._private.accelerators.get_accelerator_manager_for_resource") -def test_resolve_and_update_accelerators_too_many_requested( - mock_get_mgr, mock_get_names -): +@patch("ray._private.accelerators.utils.get_current_node_accelerator") +def test_resolve_and_update_accelerators_over_request(mock_get_mgr): mock_mgr = MagicMock() + mock_mgr.get_resource_name.return_value = "GPU" mock_mgr.get_current_process_visible_accelerator_ids.return_value = ["0", "1"] mock_mgr.get_current_node_num_accelerators.return_value = 2 mock_mgr.get_current_node_accelerator_type.return_value = None mock_mgr.get_current_node_additional_resources.return_value = {} mock_mgr.get_visible_accelerator_ids_env_var.return_value = "CUDA_VISIBLE_DEVICES" - mock_get_mgr.return_value = mock_mgr - mock_get_names.return_value = ["GPU"] + mock_get_mgr.return_value = (mock_mgr, "GPU", 3) + # Expect an error since only 2 GPU devices are visible resources = {} - with pytest.raises(ValueError, match="Attempting to start raylet with 3 GPU"): resolve_and_update_accelerator_resources(3, resources) -@patch("ray._private.accelerators.get_all_accelerator_resource_names") -@patch("ray._private.accelerators.get_accelerator_manager_for_resource") -def test_get_first_detectable_accelerator_type(mock_get_mgr, mock_get_names): +@patch("ray._private.accelerators.utils.get_current_node_accelerator") +def test_get_current_node_accelerator_type_detected(mock_get_mgr): mock_mgr = MagicMock() - mock_mgr.get_current_node_num_accelerators.return_value = 2 - mock_mgr.get_current_node_accelerator_type.return_value = "B200" - - mock_get_mgr.return_value = mock_mgr - mock_get_names.return_value = ["GPU"] + mock_mgr.get_current_node_num_accelerators.return_value = 4 + mock_mgr.get_current_node_accelerator_type.return_value = "TPU-V6E" + mock_get_mgr.return_value = (mock_mgr, "TPU", 4) - accelerator_type = get_first_detectable_accelerator_type({"GPU": 2}) - assert accelerator_type == "B200" + resources = {"TPU": 4} + result = get_current_node_accelerator_type(resources) + assert result == "TPU-V6E" -@patch("ray._private.accelerators.get_all_accelerator_resource_names") -@patch("ray._private.accelerators.get_accelerator_manager_for_resource") -def test_detect_non_gpu_accelerator_type(mock_get_mgr, mock_get_names): +@patch("ray._private.accelerators.utils.get_current_node_accelerator") +def test_get_current_node_accelerator_type_none(mock_get_mgr): mock_mgr = MagicMock() - mock_mgr.get_current_node_num_accelerators.return_value = 4 - mock_mgr.get_current_node_accelerator_type.return_value = "TPU-V6E" + mock_mgr.get_current_node_num_accelerators.return_value = 0 + mock_mgr.get_current_node_accelerator_type.return_value = "" + mock_get_mgr.return_value = (mock_mgr, "TPU", 0) + + result = get_current_node_accelerator_type({}) + assert result is None - mock_get_mgr.return_value = mock_mgr - mock_get_names.return_value = ["TPU"] - accelerator_type = get_first_detectable_accelerator_type({"TPU": 4}) - assert accelerator_type == "TPU-V6E" +@patch("ray._private.accelerators.utils.get_current_node_accelerator") +def test_get_current_node_accelerator_type_no_manager_detected(mock_get_mgr): + mock_get_mgr.return_value = None + result = get_current_node_accelerator_type({}) + assert result is None if __name__ == "__main__": diff --git a/python/ray/tests/test_basic_5.py b/python/ray/tests/test_basic_5.py index 44c327c0010f..ddaa001ffff9 100644 --- a/python/ray/tests/test_basic_5.py +++ b/python/ray/tests/test_basic_5.py @@ -134,8 +134,7 @@ def test_exit_logging(): import unittest.mock # Patch methods that makes calls to AcceleratorManager in Node init -with unittest.mock.patch("ray._private.accelerators.accelerator_utils.resolve_and_update_accelerator_resources", return_value=0), \\ - unittest.mock.patch("ray._private.accelerators.get_all_accelerator_resource_names", return_value=[]): +with unittest.mock.patch("ray._private.accelerators.utils.get_current_node_accelerator", return_value=None): @ray.remote class A: diff --git a/src/ray/common/constants.h b/src/ray/common/constants.h index 85ac66524982..d940741a51f9 100644 --- a/src/ray/common/constants.h +++ b/src/ray/common/constants.h @@ -76,7 +76,7 @@ constexpr char kGcsAutoscalerClusterConfigKey[] = "__autoscaler_cluster_config"; /// Name for cloud instance id env constexpr char kNodeCloudInstanceIdEnv[] = "RAY_CLOUD_INSTANCE_ID"; -// ENV keys for Ray node labels +/// ENV keys for Ray node labels constexpr char kNodeTypeNameEnv[] = "RAY_NODE_TYPE_NAME"; constexpr char kNodeMarketTypeEnv[] = "RAY_NODE_MARKET_TYPE"; constexpr char kNodeRegionEnv[] = "RAY_NODE_REGION"; @@ -100,14 +100,28 @@ constexpr char kLibraryPathEnvName[] = "PATH"; constexpr char kLibraryPathEnvName[] = "LD_LIBRARY_PATH"; #endif -#define RAY_LABEL_KEY_PREFIX "ray.io/" /// Default node label keys populated by the Raylet +#define RAY_LABEL_KEY_PREFIX "ray.io/" + +// The unique ID assigned to this node by the Raylet. constexpr char kLabelKeyNodeID[] = RAY_LABEL_KEY_PREFIX "node-id"; + +// The accelerator type associated with the Ray node (e.g., "A100"). constexpr char kLabelKeyNodeAcceleratorType[] = RAY_LABEL_KEY_PREFIX "accelerator-type"; + +// The market type of the cloud instance this Ray node runs on (e.g., "on-demand" or +// "spot"). constexpr char kLabelKeyNodeMarketType[] = RAY_LABEL_KEY_PREFIX "market-type"; + +// The region of the cloud instance this Ray node runs on (e.g., "us-central2"). constexpr char kLabelKeyNodeRegion[] = RAY_LABEL_KEY_PREFIX "availability-region"; + +// The zone of the cloud instance this Ray node runs on (e.g., "us-central2-b"). constexpr char kLabelKeyNodeZone[] = RAY_LABEL_KEY_PREFIX "availability-zone"; + +// The name of the head or worker group this Ray node is a part of. constexpr char kLabelKeyNodeGroup[] = RAY_LABEL_KEY_PREFIX "node-group"; + #undef RAY_LABEL_KEY_PREFIX /// All nodes implicitly have resources with this prefix and the quantity is 1. From 334dc94116c888d616bcb3faa60f1c6397aad1b8 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Mon, 14 Jul 2025 21:04:24 +0000 Subject: [PATCH 14/39] Fix comment Signed-off-by: Ryan O'Leary --- python/ray/_private/node.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/python/ray/_private/node.py b/python/ray/_private/node.py index 64791ba24e15..e017ed7dc0c3 100644 --- a/python/ray/_private/node.py +++ b/python/ray/_private/node.py @@ -522,9 +522,7 @@ def _init_temp(self): try_to_symlink(tpu_logs_symlink, tpu_log_dir) def get_resource_and_label_spec(self): - """Resolve and return the current resource spec for the node. Also resolve and return - the merged user-input, Ray default, and autoscaler environment override labels from - the LabelSpec for this node.""" + """Resolve and return the current ResourceAndLabelSpec for the node.""" def merge_resources(env_dict, params_dict): """Separates special case params and merges two dictionaries, picking from the From 7dfb82dde67b0f28a5b62776e853486acb0951cb Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Fri, 18 Jul 2025 23:08:17 +0000 Subject: [PATCH 15/39] Fix nits / rename file Signed-off-by: Ryan O'Leary --- python/ray/_private/node.py | 4 +--- .../_private/{resource_spec.py => resource_and_label_spec.py} | 0 python/ray/_private/services.py | 4 ++-- python/ray/serve/tests/test_serve_ha.py | 2 +- 4 files changed, 4 insertions(+), 6 deletions(-) rename python/ray/_private/{resource_spec.py => resource_and_label_spec.py} (100%) diff --git a/python/ray/_private/node.py b/python/ray/_private/node.py index e017ed7dc0c3..f67562bf982a 100644 --- a/python/ray/_private/node.py +++ b/python/ray/_private/node.py @@ -24,8 +24,8 @@ import ray._private.services from ray._common.ray_constants import LOGGING_ROTATE_BACKUP_COUNT, LOGGING_ROTATE_BYTES from ray._common.utils import try_to_create_directory +from ray._private.resource_and_label_spec import ResourceAndLabelSpec from ray._private.resource_isolation_config import ResourceIsolationConfig -from ray._private.resource_spec import ResourceAndLabelSpec from ray._private.services import get_address, serialize_config from ray._private.utils import ( is_in_test, @@ -286,8 +286,6 @@ def __init__( self._raylet_socket_name = self._prepare_socket_file( self._ray_params.raylet_socket_name, default_prefix="raylet" ) - # Set node labels from RayParams or environment override variables. - # self._node_labels = self._get_node_labels() if ( self._ray_params.env_vars is not None and "RAY_OVERRIDE_NODE_ID_FOR_TESTING" in self._ray_params.env_vars diff --git a/python/ray/_private/resource_spec.py b/python/ray/_private/resource_and_label_spec.py similarity index 100% rename from python/ray/_private/resource_spec.py rename to python/ray/_private/resource_and_label_spec.py diff --git a/python/ray/_private/services.py b/python/ray/_private/services.py index 7a21ef875795..e6aa1975bbc1 100644 --- a/python/ray/_private/services.py +++ b/python/ray/_private/services.py @@ -1656,9 +1656,9 @@ def start_raylet( if use_valgrind and use_profiler: raise ValueError("Cannot use valgrind and profiler at the same time.") - assert resource_and_label_spec.resolved() + # Get the static resources and labels from the resolved ResourceAndLabelSpec static_resources = resource_and_label_spec.to_resource_dict() - labels = resource_and_label_spec.get_resolved_labels() + labels = resource_and_label_spec.labels # Limit the number of workers that can be started in parallel by the # raylet. However, make sure it is at least 1. diff --git a/python/ray/serve/tests/test_serve_ha.py b/python/ray/serve/tests/test_serve_ha.py index 608b36ae1c33..a15cff0a3a47 100644 --- a/python/ray/serve/tests/test_serve_ha.py +++ b/python/ray/serve/tests/test_serve_ha.py @@ -109,7 +109,7 @@ def check_for_head_node_come_back_up(): import ray import requests from ray.serve.schema import ServeInstanceDetails -from ray._private.resource_spec import HEAD_NODE_RESOURCE_NAME +from ray._private.resource_and_label_spec import HEAD_NODE_RESOURCE_NAME ray.init(address="auto") head_node_id = ray.get_runtime_context().get_node_id() serve_details = ServeInstanceDetails( From ece86ed7a0a123a5201571d9d3013388948bfb9a Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Fri, 18 Jul 2025 23:44:00 +0000 Subject: [PATCH 16/39] Fix Java test using node_id Signed-off-by: Ryan O'Leary --- .../src/main/java/io/ray/test/NodeLabelSchedulingTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/java/test/src/main/java/io/ray/test/NodeLabelSchedulingTest.java b/java/test/src/main/java/io/ray/test/NodeLabelSchedulingTest.java index b25457e87d6e..f03a6415dc73 100644 --- a/java/test/src/main/java/io/ray/test/NodeLabelSchedulingTest.java +++ b/java/test/src/main/java/io/ray/test/NodeLabelSchedulingTest.java @@ -16,7 +16,7 @@ public void testEmptyNodeLabels() { List nodeInfos = Ray.getRuntimeContext().getAllNodeInfo(); Assert.assertTrue(nodeInfos.size() == 1); Map labels = new HashMap<>(); - labels.put("ray.io/node_id", nodeInfos.get(0).nodeId.toString()); + labels.put("ray.io/node-id", nodeInfos.get(0).nodeId.toString()); Assert.assertEquals(nodeInfos.get(0).labels, labels); } finally { Ray.shutdown(); @@ -30,7 +30,7 @@ public void testSetNodeLabels() { List nodeInfos = Ray.getRuntimeContext().getAllNodeInfo(); Assert.assertTrue(nodeInfos.size() == 1); Map labels = new HashMap<>(); - labels.put("ray.io/node_id", nodeInfos.get(0).nodeId.toString()); + labels.put("ray.io/node-id", nodeInfos.get(0).nodeId.toString()); labels.put("gpu_type", "A100"); labels.put("azone", "azone-1"); Assert.assertEquals(nodeInfos.get(0).labels, labels); From caefdfda21f63ac89abbf9478875641a937cfcd7 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Tue, 22 Jul 2025 03:09:40 +0000 Subject: [PATCH 17/39] Refactor resolve functions Signed-off-by: Ryan O'Leary --- python/ray/_private/accelerators/utils.py | 86 ++----------- .../ray/_private/resource_and_label_spec.py | 119 +++++++++++------- python/ray/tests/accelerators/test_utils.py | 116 +++++++---------- 3 files changed, 124 insertions(+), 197 deletions(-) diff --git a/python/ray/_private/accelerators/utils.py b/python/ray/_private/accelerators/utils.py index f65ecd08d11b..f75e994e5f43 100644 --- a/python/ray/_private/accelerators/utils.py +++ b/python/ray/_private/accelerators/utils.py @@ -1,16 +1,15 @@ from typing import Dict, Optional, Tuple -from ray._common.utils import RESOURCE_CONSTRAINT_PREFIX from ray._private import accelerators from ray._private.accelerators import AcceleratorManager def get_current_node_accelerator( - resources: Dict[str, float] -) -> Optional[Tuple[AcceleratorManager, str, int]]: + num_gpus, resources: Dict[str, float] +) -> Optional[Tuple[AcceleratorManager, int]]: """ - Returns the AcceleratorManager, resource name, and accelerator count for the - accelerator associated with this node. + Returns the AcceleratorManager and accelerator count for the accelerator + associated with this node. This assumes each node has at most one accelerator type. If no accelerators are present, returns None. @@ -19,9 +18,11 @@ def get_current_node_accelerator( accelerator_manager = accelerators.get_accelerator_manager_for_resource( resource_name ) - accelerator_resource_name = accelerator_manager.get_resource_name() + # Respect configured value for GPUs if set + if resource_name == "GPU" and num_gpus: + num_accelerators = num_gpus - num_accelerators = resources.get(accelerator_resource_name) + num_accelerators = resources.get(resource_name) if num_accelerators is None: num_accelerators = accelerator_manager.get_current_node_num_accelerators() visible_accelerator_ids = ( @@ -31,73 +32,6 @@ def get_current_node_accelerator( num_accelerators = min(num_accelerators, len(visible_accelerator_ids)) if num_accelerators > 0: - return accelerator_manager, accelerator_resource_name, num_accelerators + return accelerator_manager, num_accelerators - return None - - -def resolve_and_update_accelerator_resources( - num_gpus: int, resources: Dict[str, float] -) -> int: - """Detect and update accelerator resources on a node.""" - accelerator = get_current_node_accelerator(resources) - if not accelerator: - return num_gpus or 0 - - accelerator_manager, accelerator_resource_name, num_accelerators = accelerator - visible_accelerator_ids = ( - accelerator_manager.get_current_process_visible_accelerator_ids() - ) - - # Respect configured value for GPUs if set - if accelerator_resource_name == "GPU" and num_gpus: - num_accelerators = num_gpus - - # Check that the number of accelerators that the raylet wants doesn't - # exceed the amount allowed by visible accelerator ids. - if ( - num_accelerators is not None - and visible_accelerator_ids is not None - and num_accelerators > len(visible_accelerator_ids) - ): - raise ValueError( - f"Attempting to start raylet with {num_accelerators} " - f"{accelerator_resource_name}, " - f"but {accelerator_manager.get_visible_accelerator_ids_env_var()} " - f"contains {visible_accelerator_ids}." - ) - - if num_accelerators: - if accelerator_resource_name == "GPU": - num_gpus = num_accelerators - else: - resources[accelerator_resource_name] = num_accelerators - - accelerator_type = accelerator_manager.get_current_node_accelerator_type() - if accelerator_type: - resources[f"{RESOURCE_CONSTRAINT_PREFIX}{accelerator_type}"] = 1 - - from ray._private.usage import usage_lib - - usage_lib.record_hardware_usage(accelerator_type) - additional_resources = ( - accelerator_manager.get_current_node_additional_resources() - ) - if additional_resources: - resources.update(additional_resources) - - return num_gpus or 0 - - -def get_current_node_accelerator_type(resources: dict) -> Optional[str]: - """ - Returns the accelerator type (e.g. 'A100') if detectable on this node, - or None if no accelerator is present. - """ - accelerator = get_current_node_accelerator(resources) - if accelerator: - accelerator_manager, _, _ = accelerator - accelerator_type = accelerator_manager.get_current_node_accelerator_type() - return accelerator_type or None - - return None + return None, 0 diff --git a/python/ray/_private/resource_and_label_spec.py b/python/ray/_private/resource_and_label_spec.py index 648a7b7acd09..7688f4c61c6c 100644 --- a/python/ray/_private/resource_and_label_spec.py +++ b/python/ray/_private/resource_and_label_spec.py @@ -9,6 +9,8 @@ import ray._private.ray_constants as ray_constants from ray._common.constants import HEAD_NODE_RESOURCE_NAME, NODE_ID_PREFIX from ray._common.utils import RESOURCE_CONSTRAINT_PREFIX +from ray._private.accelerators import AcceleratorManager +from ray._private.accelerators.utils import get_current_node_accelerator logger = logging.getLogger(__name__) @@ -160,46 +162,14 @@ def resolve(self, is_head: bool, node_ip_address: Optional[str] = None): if num_cpus is None: num_cpus = ray._private.utils.get_num_cpus() - num_gpus = 0 - for ( - accelerator_resource_name - ) in ray._private.accelerators.get_all_accelerator_resource_names(): - accelerator_manager = ( - ray._private.accelerators.get_accelerator_manager_for_resource( - accelerator_resource_name - ) - ) - num_accelerators = None - if accelerator_resource_name == "GPU": - num_accelerators = self.num_gpus - else: - num_accelerators = resources.get(accelerator_resource_name, None) - visible_accelerator_ids = ( - accelerator_manager.get_current_process_visible_accelerator_ids() - ) - # Check that the number of accelerators that the raylet wants doesn't - # exceed the amount allowed by visible accelerator ids. - if ( - num_accelerators is not None - and visible_accelerator_ids is not None - and num_accelerators > len(visible_accelerator_ids) - ): - raise ValueError( - f"Attempting to start raylet with {num_accelerators} " - f"{accelerator_resource_name}, " - f"but {accelerator_manager.get_visible_accelerator_ids_env_var()} " - f"contains {visible_accelerator_ids}." - ) - if num_accelerators is None: - # Try to automatically detect the number of accelerators. - num_accelerators = ( - accelerator_manager.get_current_node_num_accelerators() - ) - # Don't use more accelerators than allowed by visible accelerator ids. - if visible_accelerator_ids is not None: - num_accelerators = min( - num_accelerators, len(visible_accelerator_ids) - ) + # Resolve num_gpus and accelerator resources + accelerator_manager, num_accelerators = get_current_node_accelerator( + self.num_gpus, resources + ) + num_gpus = self.resolve_and_update_accelerator_resources( + accelerator_manager, num_accelerators, resources + ) + labels = self.get_resolved_labels(accelerator_manager) if num_accelerators: if accelerator_resource_name == "GPU": @@ -287,7 +257,7 @@ def resolve(self, is_head: bool, node_ip_address: Optional[str] = None): memory, object_store_memory, resources, - self.get_resolved_labels(), + labels, ) assert spec.resolved() return spec @@ -307,7 +277,9 @@ def _load_env_labels(self) -> Dict[str, str]: return env_override_labels - def _get_default_labels(self) -> Dict[str, str]: + def _get_default_labels( + self, accelerator_manager: Optional[AcceleratorManager] + ) -> Dict[str, str]: default_labels = {} # Get environment variables populated from K8s Pod Spec @@ -327,18 +299,23 @@ def _get_default_labels(self) -> Dict[str, str]: default_labels[ray._raylet.RAY_NODE_REGION_KEY] = availability_region # Get accelerator type from AcceleratorManager - accelerator_type = get_current_node_accelerator_type(self.resources) - if accelerator_type: - default_labels[ray._raylet.RAY_NODE_ACCELERATOR_TYPE_KEY] = accelerator_type + if accelerator_manager: + accelerator_type = accelerator_manager.get_current_node_accelerator_type() + if accelerator_type: + default_labels[ + ray._raylet.RAY_NODE_ACCELERATOR_TYPE_KEY + ] = accelerator_type return default_labels - def get_resolved_labels(self) -> Dict[str, str]: + def get_resolved_labels( + self, accelerator_manager: Optional[AcceleratorManager] + ) -> Dict[str, str]: """Merge environment override, user-input from params, and Ray default labels in that order of precedence.""" env_labels = self._load_env_labels() - merged = dict(self._get_default_labels() or {}) + merged = dict(self._get_default_labels(accelerator_manager) or {}) # Merge user-specified labels from Ray params for key, val in (self.labels or {}).items(): @@ -361,3 +338,51 @@ def get_resolved_labels(self) -> Dict[str, str]: merged[key] = val return merged + + def resolve_and_update_accelerator_resources( + self, accelerator_manager, num_accelerators, resources: Dict[str, float] + ) -> int: + """Detect and update accelerator resources on a node.""" + if not accelerator_manager: + return num_accelerators or 0 + + accelerator_resource_name = accelerator_manager.get_resource_name() + visible_accelerator_ids = ( + accelerator_manager.get_current_process_visible_accelerator_ids() + ) + + # Check that the number of accelerators that the raylet wants doesn't + # exceed the amount allowed by visible accelerator ids. + if ( + num_accelerators is not None + and visible_accelerator_ids is not None + and num_accelerators > len(visible_accelerator_ids) + ): + raise ValueError( + f"Attempting to start raylet with {num_accelerators} " + f"{accelerator_resource_name}, " + f"but {accelerator_manager.get_visible_accelerator_ids_env_var()} " + f"contains {visible_accelerator_ids}." + ) + + num_gpus = self.num_gpus + if num_accelerators: + if accelerator_resource_name == "GPU": + num_gpus = num_accelerators + else: + resources[accelerator_resource_name] = num_accelerators + + accelerator_type = accelerator_manager.get_current_node_accelerator_type() + if accelerator_type: + resources[f"{RESOURCE_CONSTRAINT_PREFIX}{accelerator_type}"] = 1 + + from ray._private.usage import usage_lib + + usage_lib.record_hardware_usage(accelerator_type) + additional_resources = ( + accelerator_manager.get_current_node_additional_resources() + ) + if additional_resources: + resources.update(additional_resources) + + return num_gpus or 0 diff --git a/python/ray/tests/accelerators/test_utils.py b/python/ray/tests/accelerators/test_utils.py index 7ebd4d497c43..c831709238bb 100644 --- a/python/ray/tests/accelerators/test_utils.py +++ b/python/ray/tests/accelerators/test_utils.py @@ -2,99 +2,67 @@ import sys from unittest.mock import patch, MagicMock -from ray._common.utils import RESOURCE_CONSTRAINT_PREFIX -from ray._private.accelerators.utils import ( - resolve_and_update_accelerator_resources, - get_current_node_accelerator_type, -) +from ray._private.accelerators.utils import get_current_node_accelerator -@patch("ray._private.usage.usage_lib.record_hardware_usage") -@patch("ray._private.accelerators.utils.get_current_node_accelerator") -def test_resolve_and_update_accelerators(mock_get_mgr, mock_record_usage): +@patch("ray._private.accelerators.get_accelerator_manager_for_resource") +@patch("ray._private.accelerators.get_all_accelerator_resource_names") +def test_get_current_node_accelerator_auto_detect(mock_all_names, mock_get_mgr): + # Validate GPU num_accelerators in resource dict is detected and returned + mock_all_names.return_value = ["GPU", "TPU"] mock_mgr = MagicMock() - mock_mgr.get_resource_name.return_value = "GPU" - mock_mgr.get_current_process_visible_accelerator_ids.return_value = ["0", "1"] - mock_mgr.get_current_node_num_accelerators.return_value = 2 - mock_mgr.get_current_node_accelerator_type.return_value = "A100" - mock_mgr.get_current_node_additional_resources.return_value = {"tensor_cores": 4} - mock_mgr.get_visible_accelerator_ids_env_var.return_value = "CUDA_VISIBLE_DEVICES" - - mock_get_mgr.return_value = (mock_mgr, "GPU", 2) - - resources = {} - num_gpus = resolve_and_update_accelerator_resources(1, resources) + mock_mgr.get_current_node_num_accelerators.return_value = 4 + mock_mgr.get_current_node_accelerator_type.return_value = "TPU" + mock_mgr.get_current_process_visible_accelerator_ids.return_value = [0, 1, 3, 4] + mock_get_mgr.return_value = mock_mgr - assert num_gpus == 1 - assert resources["tensor_cores"] == 4 - assert resources[f"{RESOURCE_CONSTRAINT_PREFIX}A100"] == 1 + result = get_current_node_accelerator(None, {}) + assert result == (mock_mgr, 4) -@patch("ray._private.usage.usage_lib.record_hardware_usage") -@patch("ray._private.accelerators.utils.get_current_node_accelerator") -def test_detect_num_gpus_with_visibility_limit(mock_get_mgr, mock_record_usage): +@patch("ray._private.accelerators.get_accelerator_manager_for_resource") +@patch("ray._private.accelerators.get_all_accelerator_resource_names") +def test_get_current_node_accelerator_from_resources(mock_all_names, mock_get_mgr): + # Validate GPU num_accelerators in resource dict is detected and returned + mock_all_names.return_value = ["GPU"] mock_mgr = MagicMock() - mock_mgr.get_resource_name.return_value = "GPU" - mock_mgr.get_current_process_visible_accelerator_ids.return_value = ["0"] - mock_mgr.get_current_node_num_accelerators.return_value = 4 - mock_mgr.get_current_node_accelerator_type.return_value = "L4" - mock_mgr.get_current_node_additional_resources.return_value = {} - mock_mgr.get_visible_accelerator_ids_env_var.return_value = "CUDA_VISIBLE_DEVICES" - - mock_get_mgr.return_value = (mock_mgr, "GPU", 1) + mock_get_mgr.return_value = mock_mgr - resources = {} - num_gpus = resolve_and_update_accelerator_resources(None, resources) - assert num_gpus == 1 - assert resources[f"{RESOURCE_CONSTRAINT_PREFIX}L4"] == 1 + resources = {"GPU": 3} + result = get_current_node_accelerator(None, resources) + assert result == (mock_mgr, 3) -@patch("ray._private.accelerators.utils.get_current_node_accelerator") -def test_resolve_and_update_accelerators_over_request(mock_get_mgr): +@patch("ray._private.accelerators.get_accelerator_manager_for_resource") +@patch("ray._private.accelerators.get_all_accelerator_resource_names") +def test_get_current_node_accelerator_with_visibility_limit( + mock_all_names, mock_get_mgr +): + # Check get_current_node_accelerator caps num_accelerators to visible ids + mock_all_names.return_value = ["GPU"] mock_mgr = MagicMock() - mock_mgr.get_resource_name.return_value = "GPU" + mock_mgr.get_current_node_num_accelerators.return_value = 5 mock_mgr.get_current_process_visible_accelerator_ids.return_value = ["0", "1"] - mock_mgr.get_current_node_num_accelerators.return_value = 2 - mock_mgr.get_current_node_accelerator_type.return_value = None - mock_mgr.get_current_node_additional_resources.return_value = {} - mock_mgr.get_visible_accelerator_ids_env_var.return_value = "CUDA_VISIBLE_DEVICES" - - mock_get_mgr.return_value = (mock_mgr, "GPU", 3) + mock_get_mgr.return_value = mock_mgr - # Expect an error since only 2 GPU devices are visible resources = {} - with pytest.raises(ValueError, match="Attempting to start raylet with 3 GPU"): - resolve_and_update_accelerator_resources(3, resources) + result = get_current_node_accelerator(None, resources) + assert result == (mock_mgr, 2) -@patch("ray._private.accelerators.utils.get_current_node_accelerator") -def test_get_current_node_accelerator_type_detected(mock_get_mgr): - mock_mgr = MagicMock() - mock_mgr.get_current_node_num_accelerators.return_value = 4 - mock_mgr.get_current_node_accelerator_type.return_value = "TPU-V6E" - mock_get_mgr.return_value = (mock_mgr, "TPU", 4) - - resources = {"TPU": 4} - result = get_current_node_accelerator_type(resources) - assert result == "TPU-V6E" - - -@patch("ray._private.accelerators.utils.get_current_node_accelerator") -def test_get_current_node_accelerator_type_none(mock_get_mgr): +@patch("ray._private.accelerators.get_accelerator_manager_for_resource") +@patch("ray._private.accelerators.get_all_accelerator_resource_names") +def test_get_current_node_accelerator_none(mock_all_names, mock_get_mgr): + # Check get_current_node_accelerator returns None for num_accelerators == 0 + mock_all_names.return_value = ["GPU", "TPU"] mock_mgr = MagicMock() mock_mgr.get_current_node_num_accelerators.return_value = 0 - mock_mgr.get_current_node_accelerator_type.return_value = "" - mock_get_mgr.return_value = (mock_mgr, "TPU", 0) - - result = get_current_node_accelerator_type({}) - assert result is None + mock_mgr.get_current_process_visible_accelerator_ids.return_value = [] + mock_get_mgr.side_effect = lambda name: mock_mgr - -@patch("ray._private.accelerators.utils.get_current_node_accelerator") -def test_get_current_node_accelerator_type_no_manager_detected(mock_get_mgr): - mock_get_mgr.return_value = None - result = get_current_node_accelerator_type({}) - assert result is None + resources = {} + result = get_current_node_accelerator(None, resources) + assert result[0] is None and result[1] == 0 if __name__ == "__main__": From 6be107c38070ada368e498dd1bd2af3f76466926 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Wed, 23 Jul 2025 06:57:32 +0000 Subject: [PATCH 18/39] Add ResourceAndLabelSpec unit test file and add helpers Signed-off-by: Ryan O'Leary --- python/ray/_private/accelerators/utils.py | 37 -- .../ray/_private/resource_and_label_spec.py | 352 +++++++++--------- python/ray/tests/BUILD | 2 +- python/ray/tests/accelerators/test_utils.py | 69 ---- python/ray/tests/test_basic_5.py | 20 +- .../ray/tests/test_resource_and_label_spec.py | 157 ++++++++ 6 files changed, 346 insertions(+), 291 deletions(-) delete mode 100644 python/ray/_private/accelerators/utils.py delete mode 100644 python/ray/tests/accelerators/test_utils.py create mode 100644 python/ray/tests/test_resource_and_label_spec.py diff --git a/python/ray/_private/accelerators/utils.py b/python/ray/_private/accelerators/utils.py deleted file mode 100644 index f75e994e5f43..000000000000 --- a/python/ray/_private/accelerators/utils.py +++ /dev/null @@ -1,37 +0,0 @@ -from typing import Dict, Optional, Tuple - -from ray._private import accelerators -from ray._private.accelerators import AcceleratorManager - - -def get_current_node_accelerator( - num_gpus, resources: Dict[str, float] -) -> Optional[Tuple[AcceleratorManager, int]]: - """ - Returns the AcceleratorManager and accelerator count for the accelerator - associated with this node. - - This assumes each node has at most one accelerator type. If no accelerators - are present, returns None. - """ - for resource_name in accelerators.get_all_accelerator_resource_names(): - accelerator_manager = accelerators.get_accelerator_manager_for_resource( - resource_name - ) - # Respect configured value for GPUs if set - if resource_name == "GPU" and num_gpus: - num_accelerators = num_gpus - - num_accelerators = resources.get(resource_name) - if num_accelerators is None: - num_accelerators = accelerator_manager.get_current_node_num_accelerators() - visible_accelerator_ids = ( - accelerator_manager.get_current_process_visible_accelerator_ids() - ) - if visible_accelerator_ids is not None: - num_accelerators = min(num_accelerators, len(visible_accelerator_ids)) - - if num_accelerators > 0: - return accelerator_manager, num_accelerators - - return None, 0 diff --git a/python/ray/_private/resource_and_label_spec.py b/python/ray/_private/resource_and_label_spec.py index 7688f4c61c6c..f216ec50c23d 100644 --- a/python/ray/_private/resource_and_label_spec.py +++ b/python/ray/_private/resource_and_label_spec.py @@ -2,75 +2,69 @@ import logging import os import sys -from collections import namedtuple -from typing import Dict, Optional +from typing import Dict, Optional, Tuple import ray import ray._private.ray_constants as ray_constants from ray._common.constants import HEAD_NODE_RESOURCE_NAME, NODE_ID_PREFIX from ray._common.utils import RESOURCE_CONSTRAINT_PREFIX +from ray._private import accelerators from ray._private.accelerators import AcceleratorManager -from ray._private.accelerators.utils import get_current_node_accelerator logger = logging.getLogger(__name__) -class ResourceAndLabelSpec( - namedtuple( - "ResourceAndLabelSpec", - [ - "num_cpus", - "num_gpus", - "memory", - "object_store_memory", - "resources", - "labels", - ], - ) -): +class ResourceAndLabelSpec: """Represents the resource configuration passed to a raylet. All fields can be None. Before starting services, resolve() should be called to return a ResourceAndLabelSpec with unknown values filled in with defaults based on the local machine specifications. - - Attributes: - num_cpus: The CPUs allocated for this raylet. - num_gpus: The GPUs allocated for this raylet. - memory: The memory allocated for this raylet. - object_store_memory: The object store memory allocated for this raylet. - Note that when calling to_resource_dict(), this will be scaled down - by 30% to account for the global plasma LRU reserve. - resources: The custom resources allocated for this raylet. - labels: The labels associated with this node. Labels can be used along - with resources for scheduling. """ - def __new__( - cls, - num_cpus=None, - num_gpus=None, - memory=None, - object_store_memory=None, - resources=None, - labels=None, + def __init__( + self, + num_cpus: Optional[int] = None, + num_gpus: Optional[int] = None, + memory: Optional[float] = None, + object_store_memory: Optional[float] = None, + resources: Optional[Dict[str, float]] = None, + labels: Optional[Dict[str, str]] = None, ): - return super(ResourceAndLabelSpec, cls).__new__( - cls, - num_cpus, - num_gpus, - memory, - object_store_memory, - resources, - labels, - ) + """ + Initialize a ResourceAndLabelSpec - def resolved(self): + Args: + num_cpus: The CPUs allocated for this raylet. + num_gpus: The GPUs allocated for this raylet. + memory: The memory allocated for this raylet. + object_store_memory: The object store memory allocated for this raylet. + Note that when calling to_resource_dict(), this will be scaled down + by 30% to account for the global plasma LRU reserve. + resources: The custom resources allocated for this raylet. + labels: The labels associated with this node. Labels can be used along + with resources for scheduling. + """ + self.num_cpus = num_cpus + self.num_gpus = num_gpus + self.memory = memory + self.object_store_memory = object_store_memory + self.resources = resources + self.labels = labels + + def resolved(self) -> bool: """Returns if this ResourceAndLabelSpec has default values filled out.""" - for v in self._asdict().values(): - if v is None: - return False - return True + return all( + v is not None + for v in ( + self.num_cpus, + self.num_gpus, + self.memory, + self.object_store_memory, + self.resources, + self.labels, + ) + ) def to_resource_dict(self): """Returns a dict suitable to pass to raylet initialization. @@ -125,21 +119,19 @@ def to_resource_dict(self): return resources def resolve(self, is_head: bool, node_ip_address: Optional[str] = None): - """Returns a copy with values filled out with system defaults. + """Fills out this ResourceAndLabelSpec instance with system defaults. Args: is_head: Whether this is the head node. node_ip_address: The IP address of the node that we are on. This is used to automatically create a node id resource. + Returns: - A new ResourceAndLabelSpec with resolved fields (e.g., CPU, GPU, memory). + ResourceAndLabelSpec: This instance with all defaults resolved. """ - resources = (self.resources or {}).copy() - assert "CPU" not in resources, resources - assert "GPU" not in resources, resources - assert "memory" not in resources, resources - assert "object_store_memory" not in resources, resources + if self.resources is None: + self.resources = {} if node_ip_address is None: node_ip_address = ray.util.get_node_ip_address() @@ -147,120 +139,36 @@ def resolve(self, is_head: bool, node_ip_address: Optional[str] = None): # Automatically create a node id resource on each node. This is # queryable with ray._private.state.node_ids() and # ray._private.state.current_node_id(). - resources[NODE_ID_PREFIX + node_ip_address] = 1.0 + self.resources[NODE_ID_PREFIX + node_ip_address] = 1.0 # Automatically create a head node resource. - if HEAD_NODE_RESOURCE_NAME in resources: + if HEAD_NODE_RESOURCE_NAME in self.resources: raise ValueError( f"{HEAD_NODE_RESOURCE_NAME}" " is a reserved resource name, use another name instead." ) if is_head: - resources[HEAD_NODE_RESOURCE_NAME] = 1.0 - - num_cpus = self.num_cpus - if num_cpus is None: - num_cpus = ray._private.utils.get_num_cpus() - - # Resolve num_gpus and accelerator resources - accelerator_manager, num_accelerators = get_current_node_accelerator( - self.num_gpus, resources - ) - num_gpus = self.resolve_and_update_accelerator_resources( - accelerator_manager, num_accelerators, resources - ) - labels = self.get_resolved_labels(accelerator_manager) - - if num_accelerators: - if accelerator_resource_name == "GPU": - num_gpus = num_accelerators - else: - resources[accelerator_resource_name] = num_accelerators - - accelerator_type = ( - accelerator_manager.get_current_node_accelerator_type() - ) - if accelerator_type: - resources[f"{RESOURCE_CONSTRAINT_PREFIX}{accelerator_type}"] = 1 - - from ray._common.usage import usage_lib - - usage_lib.record_hardware_usage(accelerator_type) - additional_resources = ( - accelerator_manager.get_current_node_additional_resources() - ) - if additional_resources: - resources.update(additional_resources) - # Choose a default object store size. - system_memory = ray._common.utils.get_system_memory() - avail_memory = ray._private.utils.estimate_available_memory() - object_store_memory = self.object_store_memory - if object_store_memory is None: - object_store_memory = int( - avail_memory * ray_constants.DEFAULT_OBJECT_STORE_MEMORY_PROPORTION - ) - - # Set the object_store_memory size to 2GB on Mac - # to avoid degraded performance. - # (https://github.com/ray-project/ray/issues/20388) - if sys.platform == "darwin": - object_store_memory = min( - object_store_memory, ray_constants.MAC_DEGRADED_PERF_MMAP_SIZE_LIMIT - ) - - object_store_memory_cap = ( - ray_constants.DEFAULT_OBJECT_STORE_MAX_MEMORY_BYTES - ) + self.resources[HEAD_NODE_RESOURCE_NAME] = 1.0 - # Cap by shm size by default to avoid low performance, but don't - # go lower than REQUIRE_SHM_SIZE_THRESHOLD. - if sys.platform == "linux" or sys.platform == "linux2": - # Multiple by 0.95 to give a bit of wiggle-room. - # https://github.com/ray-project/ray/pull/23034/files - shm_avail = ray._private.utils.get_shared_memory_bytes() * 0.95 - shm_cap = max(ray_constants.REQUIRE_SHM_SIZE_THRESHOLD, shm_avail) + if self.num_cpus is None: + self.num_cpus = ray._private.utils.get_num_cpus() - object_store_memory_cap = min(object_store_memory_cap, shm_cap) + # Resolve accelerator resources + accel_info = self._get_current_node_accelerator() + if accel_info: + accelerator_manager, num_accelerators = accel_info + else: + accelerator_manager, num_accelerators = None, 0 + self._resolve_accelerator_resources(accelerator_manager, num_accelerators) - # Cap memory to avoid memory waste and perf issues on large nodes - if ( - object_store_memory_cap - and object_store_memory > object_store_memory_cap - ): - logger.debug( - "Warning: Capping object memory store to {}GB. ".format( - object_store_memory_cap // 1e9 - ) - + "To increase this further, specify `object_store_memory` " - "when calling ray.init() or ray start." - ) - object_store_memory = object_store_memory_cap + # Resolve node labels + self._resolve_labels(accelerator_manager) - memory = self.memory - if memory is None: - memory = avail_memory - object_store_memory - if memory < 100e6 and memory < 0.05 * system_memory: - raise ValueError( - "After taking into account object store and redis memory " - "usage, the amount of memory on this node available for " - "tasks and actors ({} GB) is less than {}% of total. " - "You can adjust these settings with " - "ray.init(memory=, " - "object_store_memory=).".format( - round(memory / 1e9, 2), int(100 * (memory / system_memory)) - ) - ) + # Resolve memory resources + self._resolve_memory_resources() - spec = ResourceAndLabelSpec( - num_cpus, - num_gpus, - memory, - object_store_memory, - resources, - labels, - ) - assert spec.resolved() - return spec + assert self.resolved() + return self def _load_env_labels(self) -> Dict[str, str]: env_override_labels = {} @@ -308,9 +216,9 @@ def _get_default_labels( return default_labels - def get_resolved_labels( + def _resolve_labels( self, accelerator_manager: Optional[AcceleratorManager] - ) -> Dict[str, str]: + ) -> None: """Merge environment override, user-input from params, and Ray default labels in that order of precedence.""" @@ -337,14 +245,14 @@ def get_resolved_labels( ) merged[key] = val - return merged + self.labels = merged - def resolve_and_update_accelerator_resources( - self, accelerator_manager, num_accelerators, resources: Dict[str, float] - ) -> int: + def _resolve_accelerator_resources(self, accelerator_manager, num_accelerators): """Detect and update accelerator resources on a node.""" if not accelerator_manager: - return num_accelerators or 0 + if self.num_gpus is None: + self.num_gpus = num_accelerators or 0 + return accelerator_resource_name = accelerator_manager.get_resource_name() visible_accelerator_ids = ( @@ -365,16 +273,17 @@ def resolve_and_update_accelerator_resources( f"contains {visible_accelerator_ids}." ) - num_gpus = self.num_gpus if num_accelerators: - if accelerator_resource_name == "GPU": - num_gpus = num_accelerators + if accelerator_resource_name == "GPU" and self.num_gpus is None: + self.num_gpus = num_accelerators else: - resources[accelerator_resource_name] = num_accelerators + self.resources[accelerator_resource_name] = num_accelerators + if self.num_gpus is None: + self.num_gpus = 0 accelerator_type = accelerator_manager.get_current_node_accelerator_type() if accelerator_type: - resources[f"{RESOURCE_CONSTRAINT_PREFIX}{accelerator_type}"] = 1 + self.resources[f"{RESOURCE_CONSTRAINT_PREFIX}{accelerator_type}"] = 1 from ray._private.usage import usage_lib @@ -383,6 +292,105 @@ def resolve_and_update_accelerator_resources( accelerator_manager.get_current_node_additional_resources() ) if additional_resources: - resources.update(additional_resources) + self.resources.update(additional_resources) + + def _resolve_memory_resources(self): + # Choose a default object store size. + system_memory = ray._common.utils.get_system_memory() + avail_memory = ray._private.utils.estimate_available_memory() + object_store_memory = self.object_store_memory + if object_store_memory is None: + object_store_memory = int( + avail_memory * ray_constants.DEFAULT_OBJECT_STORE_MEMORY_PROPORTION + ) + + # Set the object_store_memory size to 2GB on Mac + # to avoid degraded performance. + # (https://github.com/ray-project/ray/issues/20388) + if sys.platform == "darwin": + object_store_memory = min( + object_store_memory, ray_constants.MAC_DEGRADED_PERF_MMAP_SIZE_LIMIT + ) + + object_store_memory_cap = ( + ray_constants.DEFAULT_OBJECT_STORE_MAX_MEMORY_BYTES + ) + + # Cap by shm size by default to avoid low performance, but don't + # go lower than REQUIRE_SHM_SIZE_THRESHOLD. + if sys.platform == "linux" or sys.platform == "linux2": + # Multiple by 0.95 to give a bit of wiggle-room. + # https://github.com/ray-project/ray/pull/23034/files + shm_avail = ray._private.utils.get_shared_memory_bytes() * 0.95 + shm_cap = max(ray_constants.REQUIRE_SHM_SIZE_THRESHOLD, shm_avail) + + object_store_memory_cap = min(object_store_memory_cap, shm_cap) + + # Cap memory to avoid memory waste and perf issues on large nodes + if ( + object_store_memory_cap + and object_store_memory > object_store_memory_cap + ): + logger.debug( + "Warning: Capping object memory store to {}GB. ".format( + object_store_memory_cap // 1e9 + ) + + "To increase this further, specify `object_store_memory` " + "when calling ray.init() or ray start." + ) + object_store_memory = object_store_memory_cap + + memory = self.memory + if memory is None: + memory = avail_memory - object_store_memory + if memory < 100e6 and memory < 0.05 * system_memory: + raise ValueError( + "After taking into account object store and redis memory " + "usage, the amount of memory on this node available for " + "tasks and actors ({} GB) is less than {}% of total. " + "You can adjust these settings with " + "ray.init(memory=, " + "object_store_memory=).".format( + round(memory / 1e9, 2), int(100 * (memory / system_memory)) + ) + ) + + # Set the resolved memory and object_store_memory + self.object_store_memory = object_store_memory + self.memory = memory + + def _get_current_node_accelerator(self) -> Optional[Tuple[AcceleratorManager, int]]: + """ + Returns the AcceleratorManager and accelerator count for the accelerator + associated with this node. + + This assumes each node has at most one accelerator type. If no accelerators + are present, returns None. + """ + for resource_name in accelerators.get_all_accelerator_resource_names(): + accelerator_manager = accelerators.get_accelerator_manager_for_resource( + resource_name + ) + if accelerator_manager is None: + continue + # Respect configured value for GPUs if set + if resource_name == "GPU" and self.num_gpus is not None: + num_accelerators = self.num_gpus + else: + num_accelerators = self.resources.get(resource_name) + if num_accelerators is None: + num_accelerators = ( + accelerator_manager.get_current_node_num_accelerators() + ) + visible_accelerator_ids = ( + accelerator_manager.get_current_process_visible_accelerator_ids() + ) + if visible_accelerator_ids is not None: + num_accelerators = min( + num_accelerators, len(visible_accelerator_ids) + ) + + if num_accelerators > 0: + return accelerator_manager, num_accelerators - return num_gpus or 0 + return None diff --git a/python/ray/tests/BUILD b/python/ray/tests/BUILD index 5b0ca9eca7cd..39e2c47d46f6 100644 --- a/python/ray/tests/BUILD +++ b/python/ray/tests/BUILD @@ -340,6 +340,7 @@ py_test_module_list( "test_ray_init.py", "test_ray_init_2.py", "test_ray_shutdown.py", + "test_resource_and_label_spec.py", "test_resource_metrics.py", "test_runtime_context.py", "test_runtime_env_env_vars.py", @@ -501,7 +502,6 @@ py_test_module_list( "accelerators/test_npu.py", "accelerators/test_nvidia_gpu.py", "accelerators/test_tpu.py", - "accelerators/test_utils.py", "test_actor_lineage_reconstruction.py", "test_actor_out_of_order.py", "test_annotations.py", diff --git a/python/ray/tests/accelerators/test_utils.py b/python/ray/tests/accelerators/test_utils.py deleted file mode 100644 index c831709238bb..000000000000 --- a/python/ray/tests/accelerators/test_utils.py +++ /dev/null @@ -1,69 +0,0 @@ -import pytest -import sys -from unittest.mock import patch, MagicMock - -from ray._private.accelerators.utils import get_current_node_accelerator - - -@patch("ray._private.accelerators.get_accelerator_manager_for_resource") -@patch("ray._private.accelerators.get_all_accelerator_resource_names") -def test_get_current_node_accelerator_auto_detect(mock_all_names, mock_get_mgr): - # Validate GPU num_accelerators in resource dict is detected and returned - mock_all_names.return_value = ["GPU", "TPU"] - mock_mgr = MagicMock() - mock_mgr.get_current_node_num_accelerators.return_value = 4 - mock_mgr.get_current_node_accelerator_type.return_value = "TPU" - mock_mgr.get_current_process_visible_accelerator_ids.return_value = [0, 1, 3, 4] - mock_get_mgr.return_value = mock_mgr - - result = get_current_node_accelerator(None, {}) - assert result == (mock_mgr, 4) - - -@patch("ray._private.accelerators.get_accelerator_manager_for_resource") -@patch("ray._private.accelerators.get_all_accelerator_resource_names") -def test_get_current_node_accelerator_from_resources(mock_all_names, mock_get_mgr): - # Validate GPU num_accelerators in resource dict is detected and returned - mock_all_names.return_value = ["GPU"] - mock_mgr = MagicMock() - mock_get_mgr.return_value = mock_mgr - - resources = {"GPU": 3} - result = get_current_node_accelerator(None, resources) - assert result == (mock_mgr, 3) - - -@patch("ray._private.accelerators.get_accelerator_manager_for_resource") -@patch("ray._private.accelerators.get_all_accelerator_resource_names") -def test_get_current_node_accelerator_with_visibility_limit( - mock_all_names, mock_get_mgr -): - # Check get_current_node_accelerator caps num_accelerators to visible ids - mock_all_names.return_value = ["GPU"] - mock_mgr = MagicMock() - mock_mgr.get_current_node_num_accelerators.return_value = 5 - mock_mgr.get_current_process_visible_accelerator_ids.return_value = ["0", "1"] - mock_get_mgr.return_value = mock_mgr - - resources = {} - result = get_current_node_accelerator(None, resources) - assert result == (mock_mgr, 2) - - -@patch("ray._private.accelerators.get_accelerator_manager_for_resource") -@patch("ray._private.accelerators.get_all_accelerator_resource_names") -def test_get_current_node_accelerator_none(mock_all_names, mock_get_mgr): - # Check get_current_node_accelerator returns None for num_accelerators == 0 - mock_all_names.return_value = ["GPU", "TPU"] - mock_mgr = MagicMock() - mock_mgr.get_current_node_num_accelerators.return_value = 0 - mock_mgr.get_current_process_visible_accelerator_ids.return_value = [] - mock_get_mgr.side_effect = lambda name: mock_mgr - - resources = {} - result = get_current_node_accelerator(None, resources) - assert result[0] is None and result[1] == 0 - - -if __name__ == "__main__": - sys.exit(pytest.main(["-sv", __file__])) diff --git a/python/ray/tests/test_basic_5.py b/python/ray/tests/test_basic_5.py index ddaa001ffff9..10089ab38408 100644 --- a/python/ray/tests/test_basic_5.py +++ b/python/ray/tests/test_basic_5.py @@ -131,20 +131,16 @@ def test_exit_logging(): log = run_string_as_driver( """ import ray -import unittest.mock -# Patch methods that makes calls to AcceleratorManager in Node init -with unittest.mock.patch("ray._private.accelerators.utils.get_current_node_accelerator", return_value=None): - - @ray.remote - class A: - def pid(self): - import os - return os.getpid() +@ray.remote +class A: + def pid(self): + import os + return os.getpid() - a = A.remote() - ray.get(a.pid.remote()) - """ +a = A.remote() +ray.get(a.pid.remote()) + """ ) assert "Traceback" not in log diff --git a/python/ray/tests/test_resource_and_label_spec.py b/python/ray/tests/test_resource_and_label_spec.py new file mode 100644 index 000000000000..a2393ca04569 --- /dev/null +++ b/python/ray/tests/test_resource_and_label_spec.py @@ -0,0 +1,157 @@ +import pytest +import sys +import json +from unittest.mock import patch, MagicMock +from ray._private.resource_and_label_spec import ResourceAndLabelSpec +from ray._common.constants import HEAD_NODE_RESOURCE_NAME, NODE_ID_PREFIX +import ray._private.ray_constants as ray_constants + + +def test_spec_resolved(): + spec = ResourceAndLabelSpec( + num_cpus=2, + num_gpus=1, + memory=1_000_000_000, + object_store_memory=500_000_000, + resources={"TPU": 5}, + labels={}, + ) + assert spec.resolved() + resource_dict = spec.to_resource_dict() + assert resource_dict["CPU"] == 2 + assert resource_dict["GPU"] == 1 + assert "TPU" in resource_dict + + +def test_resolved_false_until_resolve(): + spec = ResourceAndLabelSpec() + assert not spec.resolved() + + # Patch calls that rely on Ray environment for test + with patch("ray._private.utils.get_num_cpus", return_value=4), patch( + "ray.util.get_node_ip_address", return_value="127.0.0.1" + ), patch( + "ray._private.utils.estimate_available_memory", return_value=10000000 + ), patch( + "ray._common.utils.get_system_memory", return_value=20000000 + ), patch( + "ray._private.utils.get_shared_memory_bytes", return_value=50000 + ), patch.object( + ResourceAndLabelSpec, "_get_current_node_accelerator", return_value=None + ), patch( + "ray._private.usage.usage_lib.record_hardware_usage", lambda *_: None + ): + spec.resolve(is_head=True) + + # Values set by default should be filled out + assert spec.resolved() + assert any(key.startswith(NODE_ID_PREFIX) for key in spec.resources.keys()) + assert HEAD_NODE_RESOURCE_NAME in spec.resources + + +def test_to_resource_dict_invalid_types(): + spec = ResourceAndLabelSpec( + num_cpus=1, + num_gpus=1, + memory=1000, + object_store_memory=1000, + resources={"INVALID_RESOURCE": -1}, + labels={}, + ) + assert spec.resolved() + with pytest.raises(ValueError): + spec.to_resource_dict() + + +@patch("ray._private.accelerators.get_accelerator_manager_for_resource") +@patch("ray._private.accelerators.get_all_accelerator_resource_names") +def test_get_current_node_accelerator_auto_detect(mock_all_names, mock_get_mgr): + mock_all_names.return_value = ["GPU", "TPU"] + mock_mgr = MagicMock() + mock_mgr.get_current_node_num_accelerators.return_value = 4 + mock_mgr.get_current_node_accelerator_type.return_value = "TPU" + mock_mgr.get_current_process_visible_accelerator_ids.return_value = [0, 1, 3, 4] + mock_get_mgr.return_value = mock_mgr + + spec = ResourceAndLabelSpec(resources={}) + result = spec._get_current_node_accelerator() + assert result == (mock_mgr, 4) + + +@patch("ray._private.accelerators.get_accelerator_manager_for_resource") +@patch("ray._private.accelerators.get_all_accelerator_resource_names") +def test_get_current_node_accelerator_from_resources(mock_all_names, mock_get_mgr): + mock_all_names.return_value = ["A100"] + mock_mgr = MagicMock() + mock_get_mgr.return_value = mock_mgr + + spec = ResourceAndLabelSpec(resources={"A100": 3}) + result = spec._get_current_node_accelerator() + assert result == (mock_mgr, 3) + + +@patch("ray._private.accelerators.get_accelerator_manager_for_resource") +@patch("ray._private.accelerators.get_all_accelerator_resource_names") +def test_get_current_node_accelerator_with_visibility_limit( + mock_all_names, mock_get_mgr +): + mock_all_names.return_value = ["A100"] + mock_mgr = MagicMock() + mock_mgr.get_current_node_num_accelerators.return_value = 5 + mock_mgr.get_current_process_visible_accelerator_ids.return_value = ["0", "1"] + mock_get_mgr.return_value = mock_mgr + + spec = ResourceAndLabelSpec(resources={}) + result = spec._get_current_node_accelerator() + assert result == (mock_mgr, 2) + + +@patch("ray._private.accelerators.get_accelerator_manager_for_resource") +@patch("ray._private.accelerators.get_all_accelerator_resource_names") +def test_get_current_node_accelerator_none(mock_all_names, mock_get_mgr): + mock_all_names.return_value = ["B200", "TPU-v6e"] + mock_mgr = MagicMock() + mock_mgr.get_current_node_num_accelerators.return_value = 0 + mock_mgr.get_current_process_visible_accelerator_ids.return_value = [] + mock_get_mgr.side_effect = lambda name: mock_mgr + + spec = ResourceAndLabelSpec(resources={}) + result = spec._get_current_node_accelerator() + assert result is None + + +def test_load_override_env_labels_merges_and_logs(monkeypatch): + env_data = {"autoscaler-override-label": "example"} + monkeypatch.setenv(ray_constants.LABELS_ENVIRONMENT_VARIABLE, json.dumps(env_data)) + spec = ResourceAndLabelSpec() + result = spec._load_env_labels() + assert result == env_data + + +@patch("ray._private.usage.usage_lib.record_hardware_usage", lambda *_: None) +def test_resolve_accelerator_resources_sets_num_gpus(): + # num-gpus not passed in but detected from _resolve_accelerator_resources + spec = ResourceAndLabelSpec(resources={}) + mock_mgr = MagicMock() + mock_mgr.get_resource_name.return_value = "GPU" + mock_mgr.get_current_node_accelerator_type.return_value = "A100" + mock_mgr.get_current_process_visible_accelerator_ids.return_value = None + spec._resolve_accelerator_resources(mock_mgr, 2) + assert spec.num_gpus == 2 + assert any("A100" in key for key in spec.resources.keys()) + + +@patch("ray._private.usage.usage_lib.record_hardware_usage", lambda *_: None) +def test_resolve_accelerator_resources_respects_configured_num_gpus(): + # num-gpus value set by user should be respected + spec = ResourceAndLabelSpec(num_gpus=0, resources={}) + mock_mgr = MagicMock() + mock_mgr.get_resource_name.return_value = "GPU" + mock_mgr.get_current_node_accelerator_type.return_value = "H100" + mock_mgr.get_current_process_visible_accelerator_ids.return_value = None + spec._resolve_accelerator_resources(mock_mgr, 4) + assert spec.num_gpus == 0 + + +if __name__ == "__main__": + sys.exit(pytest.main(["-sv", __file__])) From 147dcad455090228edcdfd2f5cb811563d3e48b8 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Wed, 23 Jul 2025 07:00:34 +0000 Subject: [PATCH 19/39] remove change to test_basic_5 Signed-off-by: Ryan O'Leary --- python/ray/tests/test_basic_5.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/ray/tests/test_basic_5.py b/python/ray/tests/test_basic_5.py index 10089ab38408..765bb4d72ee8 100644 --- a/python/ray/tests/test_basic_5.py +++ b/python/ray/tests/test_basic_5.py @@ -138,6 +138,7 @@ def pid(self): import os return os.getpid() + a = A.remote() ray.get(a.pid.remote()) """ From 38cc2504ffcdf75c50bec8fe7a76412a9f4bea14 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> Date: Wed, 23 Jul 2025 14:05:57 -0700 Subject: [PATCH 20/39] Update python/ray/_private/resource_and_label_spec.py Co-authored-by: Jiajun Yao Signed-off-by: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> --- python/ray/_private/resource_and_label_spec.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/_private/resource_and_label_spec.py b/python/ray/_private/resource_and_label_spec.py index f216ec50c23d..3bc99249eb27 100644 --- a/python/ray/_private/resource_and_label_spec.py +++ b/python/ray/_private/resource_and_label_spec.py @@ -15,7 +15,7 @@ class ResourceAndLabelSpec: - """Represents the resource configuration passed to a raylet. + """Represents the resource and label configuration passed to a raylet. All fields can be None. Before starting services, resolve() should be called to return a ResourceAndLabelSpec with unknown values filled in with From 33018f73a45971aa9882501d3227b42d245a1c4b Mon Sep 17 00:00:00 2001 From: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> Date: Wed, 23 Jul 2025 14:06:13 -0700 Subject: [PATCH 21/39] Update python/ray/_private/node.py Co-authored-by: Jiajun Yao Signed-off-by: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> --- python/ray/_private/node.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/_private/node.py b/python/ray/_private/node.py index f67562bf982a..f8ac8091d947 100644 --- a/python/ray/_private/node.py +++ b/python/ray/_private/node.py @@ -1441,7 +1441,7 @@ def start_ray_processes(self): # Make sure we don't call `determine_plasma_store_config` multiple # times to avoid printing multiple warnings. - resource_spec = self.get_resource_and_label_spec() + resource_and_label_spec = self.get_resource_and_label_spec() ( plasma_directory, From d5185496c602bbb91159049843e5a74ab7479d3e Mon Sep 17 00:00:00 2001 From: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> Date: Wed, 23 Jul 2025 14:06:22 -0700 Subject: [PATCH 22/39] Update python/ray/_private/resource_and_label_spec.py Co-authored-by: Jiajun Yao Signed-off-by: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> --- python/ray/_private/resource_and_label_spec.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/python/ray/_private/resource_and_label_spec.py b/python/ray/_private/resource_and_label_spec.py index 3bc99249eb27..5e9ea7a09a35 100644 --- a/python/ray/_private/resource_and_label_spec.py +++ b/python/ray/_private/resource_and_label_spec.py @@ -39,8 +39,6 @@ def __init__( num_gpus: The GPUs allocated for this raylet. memory: The memory allocated for this raylet. object_store_memory: The object store memory allocated for this raylet. - Note that when calling to_resource_dict(), this will be scaled down - by 30% to account for the global plasma LRU reserve. resources: The custom resources allocated for this raylet. labels: The labels associated with this node. Labels can be used along with resources for scheduling. From 8ba4a057f08331480ff7c5db1feb13f670e82d5c Mon Sep 17 00:00:00 2001 From: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> Date: Wed, 23 Jul 2025 14:06:31 -0700 Subject: [PATCH 23/39] Update python/ray/_private/resource_and_label_spec.py Co-authored-by: Jiajun Yao Signed-off-by: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> --- python/ray/_private/resource_and_label_spec.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/_private/resource_and_label_spec.py b/python/ray/_private/resource_and_label_spec.py index 5e9ea7a09a35..03310e4705ca 100644 --- a/python/ray/_private/resource_and_label_spec.py +++ b/python/ray/_private/resource_and_label_spec.py @@ -116,7 +116,7 @@ def to_resource_dict(self): return resources - def resolve(self, is_head: bool, node_ip_address: Optional[str] = None): + def resolve(self, is_head: bool, node_ip_address: Optional[str] = None) -> ResourceAndLabelSpec: """Fills out this ResourceAndLabelSpec instance with system defaults. Args: From 1ba0d6fca91bc61a75cfe1948cfe358106031658 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> Date: Wed, 23 Jul 2025 14:07:01 -0700 Subject: [PATCH 24/39] Update python/ray/_private/resource_and_label_spec.py Co-authored-by: Jiajun Yao Signed-off-by: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> --- python/ray/_private/resource_and_label_spec.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/_private/resource_and_label_spec.py b/python/ray/_private/resource_and_label_spec.py index 03310e4705ca..4cf7d3a3a419 100644 --- a/python/ray/_private/resource_and_label_spec.py +++ b/python/ray/_private/resource_and_label_spec.py @@ -372,7 +372,7 @@ def _get_current_node_accelerator(self) -> Optional[Tuple[AcceleratorManager, in if accelerator_manager is None: continue # Respect configured value for GPUs if set - if resource_name == "GPU" and self.num_gpus is not None: + if resource_name == "GPU": num_accelerators = self.num_gpus else: num_accelerators = self.resources.get(resource_name) From 3116caf63caade3152aa04e226598fb6451d3d70 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Wed, 23 Jul 2025 22:40:30 +0000 Subject: [PATCH 25/39] Update num-gpus being set and fix nits Signed-off-by: Ryan O'Leary --- python/ray/_private/node.py | 2 +- .../ray/_private/resource_and_label_spec.py | 79 +++++++++++-------- .../ray/tests/test_resource_and_label_spec.py | 30 +++---- 3 files changed, 58 insertions(+), 53 deletions(-) diff --git a/python/ray/_private/node.py b/python/ray/_private/node.py index f8ac8091d947..3342044f0a4c 100644 --- a/python/ray/_private/node.py +++ b/python/ray/_private/node.py @@ -1448,7 +1448,7 @@ def start_ray_processes(self): fallback_directory, object_store_memory, ) = ray._private.services.determine_plasma_store_config( - resource_spec.object_store_memory, + resource_and_label_spec.object_store_memory, self._temp_dir, plasma_directory=self._ray_params.plasma_directory, fallback_directory=self._fallback_directory, diff --git a/python/ray/_private/resource_and_label_spec.py b/python/ray/_private/resource_and_label_spec.py index 4cf7d3a3a419..b878b18c4aab 100644 --- a/python/ray/_private/resource_and_label_spec.py +++ b/python/ray/_private/resource_and_label_spec.py @@ -116,7 +116,9 @@ def to_resource_dict(self): return resources - def resolve(self, is_head: bool, node_ip_address: Optional[str] = None) -> ResourceAndLabelSpec: + def resolve( + self, is_head: bool, node_ip_address: Optional[str] = None + ) -> "ResourceAndLabelSpec": """Fills out this ResourceAndLabelSpec instance with system defaults. Args: @@ -151,12 +153,16 @@ def resolve(self, is_head: bool, node_ip_address: Optional[str] = None) -> Resou if self.num_cpus is None: self.num_cpus = ray._private.utils.get_num_cpus() + if self.num_gpus is None: + self.num_gpus = 0 + # Resolve accelerator resources - accel_info = self._get_current_node_accelerator() - if accel_info: - accelerator_manager, num_accelerators = accel_info - else: - accelerator_manager, num_accelerators = None, 0 + ( + accelerator_manager, + num_accelerators, + ) = ResourceAndLabelSpec._get_current_node_accelerator( + self.num_gpus, self.resources + ) self._resolve_accelerator_resources(accelerator_manager, num_accelerators) # Resolve node labels @@ -220,8 +226,8 @@ def _resolve_labels( """Merge environment override, user-input from params, and Ray default labels in that order of precedence.""" - env_labels = self._load_env_labels() - merged = dict(self._get_default_labels(accelerator_manager) or {}) + # Start with a dictionary filled out with Ray default labels + merged = self._get_default_labels(accelerator_manager) # Merge user-specified labels from Ray params for key, val in (self.labels or {}).items(): @@ -234,6 +240,7 @@ def _resolve_labels( merged[key] = val # Merge autoscaler override labels from environment + env_labels = self._load_env_labels() for key, val in (env_labels or {}).items(): if key in merged and merged[key] != val: logger.warning( @@ -248,8 +255,6 @@ def _resolve_labels( def _resolve_accelerator_resources(self, accelerator_manager, num_accelerators): """Detect and update accelerator resources on a node.""" if not accelerator_manager: - if self.num_gpus is None: - self.num_gpus = num_accelerators or 0 return accelerator_resource_name = accelerator_manager.get_resource_name() @@ -271,26 +276,23 @@ def _resolve_accelerator_resources(self, accelerator_manager, num_accelerators): f"contains {visible_accelerator_ids}." ) - if num_accelerators: - if accelerator_resource_name == "GPU" and self.num_gpus is None: - self.num_gpus = num_accelerators - else: - self.resources[accelerator_resource_name] = num_accelerators - if self.num_gpus is None: - self.num_gpus = 0 + if accelerator_resource_name == "GPU": + self.num_gpus = num_accelerators + else: + self.resources[accelerator_resource_name] = num_accelerators - accelerator_type = accelerator_manager.get_current_node_accelerator_type() - if accelerator_type: - self.resources[f"{RESOURCE_CONSTRAINT_PREFIX}{accelerator_type}"] = 1 + accelerator_type = accelerator_manager.get_current_node_accelerator_type() + if accelerator_type: + self.resources[f"{RESOURCE_CONSTRAINT_PREFIX}{accelerator_type}"] = 1 - from ray._private.usage import usage_lib + from ray._private.usage import usage_lib - usage_lib.record_hardware_usage(accelerator_type) - additional_resources = ( - accelerator_manager.get_current_node_additional_resources() - ) - if additional_resources: - self.resources.update(additional_resources) + usage_lib.record_hardware_usage(accelerator_type) + additional_resources = ( + accelerator_manager.get_current_node_additional_resources() + ) + if additional_resources: + self.resources.update(additional_resources) def _resolve_memory_resources(self): # Choose a default object store size. @@ -357,13 +359,22 @@ def _resolve_memory_resources(self): self.object_store_memory = object_store_memory self.memory = memory - def _get_current_node_accelerator(self) -> Optional[Tuple[AcceleratorManager, int]]: + @staticmethod + def _get_current_node_accelerator( + num_gpus: int, resources: Dict[str, float] + ) -> Tuple[AcceleratorManager, int]: """ Returns the AcceleratorManager and accelerator count for the accelerator - associated with this node. + associated with this node. This assumes each node has at most one accelerator type. + If no accelerators are present, returns None. - This assumes each node has at most one accelerator type. If no accelerators - are present, returns None. + Args: + num_gpus: GPU count (if provided by user). + resources: Resource dictionary containing custom resource keys. + + Returns: + Tuple[Optional[AcceleratorManager], int]: A tuple containing the accelerator + manager (or None) and the detected accelerator count (0 if none found). """ for resource_name in accelerators.get_all_accelerator_resource_names(): accelerator_manager = accelerators.get_accelerator_manager_for_resource( @@ -373,9 +384,9 @@ def _get_current_node_accelerator(self) -> Optional[Tuple[AcceleratorManager, in continue # Respect configured value for GPUs if set if resource_name == "GPU": - num_accelerators = self.num_gpus + num_accelerators = num_gpus else: - num_accelerators = self.resources.get(resource_name) + num_accelerators = resources.get(resource_name) if num_accelerators is None: num_accelerators = ( accelerator_manager.get_current_node_num_accelerators() @@ -391,4 +402,4 @@ def _get_current_node_accelerator(self) -> Optional[Tuple[AcceleratorManager, in if num_accelerators > 0: return accelerator_manager, num_accelerators - return None + return None, 0 diff --git a/python/ray/tests/test_resource_and_label_spec.py b/python/ray/tests/test_resource_and_label_spec.py index a2393ca04569..35ab3958a9b9 100644 --- a/python/ray/tests/test_resource_and_label_spec.py +++ b/python/ray/tests/test_resource_and_label_spec.py @@ -37,7 +37,7 @@ def test_resolved_false_until_resolve(): ), patch( "ray._private.utils.get_shared_memory_bytes", return_value=50000 ), patch.object( - ResourceAndLabelSpec, "_get_current_node_accelerator", return_value=None + ResourceAndLabelSpec, "_get_current_node_accelerator", return_value=(None, 0) ), patch( "ray._private.usage.usage_lib.record_hardware_usage", lambda *_: None ): @@ -73,8 +73,7 @@ def test_get_current_node_accelerator_auto_detect(mock_all_names, mock_get_mgr): mock_mgr.get_current_process_visible_accelerator_ids.return_value = [0, 1, 3, 4] mock_get_mgr.return_value = mock_mgr - spec = ResourceAndLabelSpec(resources={}) - result = spec._get_current_node_accelerator() + result = ResourceAndLabelSpec._get_current_node_accelerator(None, resources={}) assert result == (mock_mgr, 4) @@ -85,8 +84,7 @@ def test_get_current_node_accelerator_from_resources(mock_all_names, mock_get_mg mock_mgr = MagicMock() mock_get_mgr.return_value = mock_mgr - spec = ResourceAndLabelSpec(resources={"A100": 3}) - result = spec._get_current_node_accelerator() + result = ResourceAndLabelSpec._get_current_node_accelerator(None, {"A100": 3}) assert result == (mock_mgr, 3) @@ -101,8 +99,7 @@ def test_get_current_node_accelerator_with_visibility_limit( mock_mgr.get_current_process_visible_accelerator_ids.return_value = ["0", "1"] mock_get_mgr.return_value = mock_mgr - spec = ResourceAndLabelSpec(resources={}) - result = spec._get_current_node_accelerator() + result = ResourceAndLabelSpec._get_current_node_accelerator(None, {}) assert result == (mock_mgr, 2) @@ -115,9 +112,8 @@ def test_get_current_node_accelerator_none(mock_all_names, mock_get_mgr): mock_mgr.get_current_process_visible_accelerator_ids.return_value = [] mock_get_mgr.side_effect = lambda name: mock_mgr - spec = ResourceAndLabelSpec(resources={}) - result = spec._get_current_node_accelerator() - assert result is None + result = ResourceAndLabelSpec._get_current_node_accelerator(None, {}) + assert result[0] is None and result[1] == 0 def test_load_override_env_labels_merges_and_logs(monkeypatch): @@ -130,27 +126,25 @@ def test_load_override_env_labels_merges_and_logs(monkeypatch): @patch("ray._private.usage.usage_lib.record_hardware_usage", lambda *_: None) def test_resolve_accelerator_resources_sets_num_gpus(): - # num-gpus not passed in but detected from _resolve_accelerator_resources - spec = ResourceAndLabelSpec(resources={}) + # num-gpus passed in to spec and not detected from AcceleratorManager + spec = ResourceAndLabelSpec(resources={}, num_gpus=2) mock_mgr = MagicMock() - mock_mgr.get_resource_name.return_value = "GPU" - mock_mgr.get_current_node_accelerator_type.return_value = "A100" mock_mgr.get_current_process_visible_accelerator_ids.return_value = None spec._resolve_accelerator_resources(mock_mgr, 2) assert spec.num_gpus == 2 - assert any("A100" in key for key in spec.resources.keys()) + assert not any("GPU" in key for key in spec.resources.keys()) @patch("ray._private.usage.usage_lib.record_hardware_usage", lambda *_: None) -def test_resolve_accelerator_resources_respects_configured_num_gpus(): - # num-gpus value set by user should be respected +def test_resolve_accelerator_resources_with_gpu_auto_detected(): + # num-gpus not passed in but detected from _resolve_accelerator_resources spec = ResourceAndLabelSpec(num_gpus=0, resources={}) mock_mgr = MagicMock() mock_mgr.get_resource_name.return_value = "GPU" mock_mgr.get_current_node_accelerator_type.return_value = "H100" mock_mgr.get_current_process_visible_accelerator_ids.return_value = None spec._resolve_accelerator_resources(mock_mgr, 4) - assert spec.num_gpus == 0 + assert spec.num_gpus == 4 if __name__ == "__main__": From a469b3f44743cad0e9dcb1244edd76f91d97f215 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Fri, 25 Jul 2025 02:29:00 +0000 Subject: [PATCH 26/39] Remove test_sql.py change Signed-off-by: Ryan O'Leary --- python/ray/data/tests/test_sql.py | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/python/ray/data/tests/test_sql.py b/python/ray/data/tests/test_sql.py index 20612636fb58..ffc9a657913f 100644 --- a/python/ray/data/tests/test_sql.py +++ b/python/ray/data/tests/test_sql.py @@ -306,10 +306,7 @@ def request_get_mock(url, params=None, **kwargs): assert False, "Invalid request." - with mock.patch( - "ray._private.accelerators.tpu.TPUAcceleratorManager.get_current_node_accelerator_type", - return_value=None, - ), mock.patch("requests.get", request_get_mock), mock.patch( + with mock.patch("requests.get", request_get_mock), mock.patch( "requests.post", request_post_mock ), mock.patch.dict( os.environ, From ad6895ce5e9f26e9f7b464e1def3963c61ef2da0 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Fri, 25 Jul 2025 06:04:07 +0000 Subject: [PATCH 27/39] Fix GPU tests, use monkeypatch, and fix comment Signed-off-by: Ryan O'Leary --- .../ray/_private/resource_and_label_spec.py | 8 +++- python/ray/tests/test_node_labels.py | 39 ++++--------------- 2 files changed, 14 insertions(+), 33 deletions(-) diff --git a/python/ray/_private/resource_and_label_spec.py b/python/ray/_private/resource_and_label_spec.py index b878b18c4aab..512dfed3d92c 100644 --- a/python/ray/_private/resource_and_label_spec.py +++ b/python/ray/_private/resource_and_label_spec.py @@ -368,13 +368,17 @@ def _get_current_node_accelerator( associated with this node. This assumes each node has at most one accelerator type. If no accelerators are present, returns None. + The resolved accelerator count uses num_gpus (for GPUs) or resources if set, and + otherwise falls back to the count auto-detected by the AcceleratorManager. The + resolved accelerator count is capped by the number of visible accelerators. + Args: num_gpus: GPU count (if provided by user). resources: Resource dictionary containing custom resource keys. Returns: Tuple[Optional[AcceleratorManager], int]: A tuple containing the accelerator - manager (or None) and the detected accelerator count (0 if none found). + manager (or None) the final resolved accelerator count. """ for resource_name in accelerators.get_all_accelerator_resource_names(): accelerator_manager = accelerators.get_accelerator_manager_for_resource( @@ -384,7 +388,7 @@ def _get_current_node_accelerator( continue # Respect configured value for GPUs if set if resource_name == "GPU": - num_accelerators = num_gpus + num_accelerators = num_gpus or None else: num_accelerators = resources.get(resource_name) if num_accelerators is None: diff --git a/python/ray/tests/test_node_labels.py b/python/ray/tests/test_node_labels.py index 04ef07740bb8..0761af137ed8 100644 --- a/python/ray/tests/test_node_labels.py +++ b/python/ray/tests/test_node_labels.py @@ -162,37 +162,14 @@ def test_ray_start_set_node_labels_from_file(shutdown_only): os.remove(test_file_path) -@pytest.fixture -def ray_node_labels_env(): - # Ray default node label keys and TPU accelerator env key - keys = [ - "RAY_NODE_MARKET_TYPE", - "RAY_NODE_TYPE_NAME", - "RAY_NODE_REGION", - "RAY_NODE_ZONE", - "TPU_ACCELERATOR_TYPE", - ] - - # Save original vals for env vars under test - original_env = {k: os.environ.get(k) for k in keys} - - # Set env var values for test - os.environ["RAY_NODE_MARKET_TYPE"] = "spot" - os.environ["RAY_NODE_TYPE_NAME"] = "worker-group-1" - os.environ["RAY_NODE_REGION"] = "us-central2" - os.environ["RAY_NODE_ZONE"] = "us-central2-b" - os.environ["TPU_ACCELERATOR_TYPE"] = "v4-16" - - yield - # Restore original values - for k in keys: - if original_env[k] is None: - os.environ.pop(k, None) - else: - os.environ[k] = original_env[k] - - -def test_get_default_ray_node_labels(shutdown_only, ray_node_labels_env): +def test_get_default_ray_node_labels(shutdown_only, monkeypatch): + # Set env vars for this test + monkeypatch.setenv("RAY_NODE_MARKET_TYPE", "spot") + monkeypatch.setenv("RAY_NODE_TYPE_NAME", "worker-group-1") + monkeypatch.setenv("RAY_NODE_REGION", "us-central2") + monkeypatch.setenv("RAY_NODE_ZONE", "us-central2-b") + monkeypatch.setenv("TPU_ACCELERATOR_TYPE", "v4-16") + ray.init(resources={"TPU": 4}) node_info = ray.nodes()[0] labels = node_info["Labels"] From 775b631ba1a9d8c23b87dbcd271580ca53a3180f Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Fri, 25 Jul 2025 06:33:05 +0000 Subject: [PATCH 28/39] Fix num_gpus edge case and add additional unit tests Signed-off-by: Ryan O'Leary --- .../ray/_private/resource_and_label_spec.py | 11 ++-- .../ray/tests/test_resource_and_label_spec.py | 50 +++++++++++++++++++ 2 files changed, 56 insertions(+), 5 deletions(-) diff --git a/python/ray/_private/resource_and_label_spec.py b/python/ray/_private/resource_and_label_spec.py index 512dfed3d92c..6b81522f6e91 100644 --- a/python/ray/_private/resource_and_label_spec.py +++ b/python/ray/_private/resource_and_label_spec.py @@ -153,9 +153,6 @@ def resolve( if self.num_cpus is None: self.num_cpus = ray._private.utils.get_num_cpus() - if self.num_gpus is None: - self.num_gpus = 0 - # Resolve accelerator resources ( accelerator_manager, @@ -165,6 +162,10 @@ def resolve( ) self._resolve_accelerator_resources(accelerator_manager, num_accelerators) + # Default num_gpus value if unset by user and unable to auto-detect. + if self.num_gpus is None: + self.num_gpus = 0 + # Resolve node labels self._resolve_labels(accelerator_manager) @@ -361,7 +362,7 @@ def _resolve_memory_resources(self): @staticmethod def _get_current_node_accelerator( - num_gpus: int, resources: Dict[str, float] + num_gpus: Optional[int], resources: Dict[str, float] ) -> Tuple[AcceleratorManager, int]: """ Returns the AcceleratorManager and accelerator count for the accelerator @@ -388,7 +389,7 @@ def _get_current_node_accelerator( continue # Respect configured value for GPUs if set if resource_name == "GPU": - num_accelerators = num_gpus or None + num_accelerators = num_gpus else: num_accelerators = resources.get(resource_name) if num_accelerators is None: diff --git a/python/ray/tests/test_resource_and_label_spec.py b/python/ray/tests/test_resource_and_label_spec.py index 35ab3958a9b9..fc2805e223d0 100644 --- a/python/ray/tests/test_resource_and_label_spec.py +++ b/python/ray/tests/test_resource_and_label_spec.py @@ -147,5 +147,55 @@ def test_resolve_accelerator_resources_with_gpu_auto_detected(): assert spec.num_gpus == 4 +@patch("ray._private.usage.usage_lib.record_hardware_usage", lambda *_: None) +def test_resolve_num_gpus_when_unset_and_not_detected(): + # Tests that the ResourceAndLabelSpec defaults num_gpus to 0 when not specified by the user + # and not detected by the AcceleratorManager + spec = ResourceAndLabelSpec(resources={}) + with patch.object( + ResourceAndLabelSpec, "_get_current_node_accelerator", return_value=(None, 0) + ), patch("ray._private.utils.get_num_cpus", return_value=4), patch( + "ray.util.get_node_ip_address", return_value="127.0.0.1" + ), patch( + "ray._private.utils.estimate_available_memory", return_value=10000000 + ), patch( + "ray._common.utils.get_system_memory", return_value=20000000 + ), patch( + "ray._private.utils.get_shared_memory_bytes", return_value=50000 + ): + spec.resolve(is_head=True) + + assert spec.num_gpus == 0 + assert spec.resolved() + + +@patch("ray._private.usage.usage_lib.record_hardware_usage", lambda *_: None) +def test_resolve_accelerator_resources_adds_accelerator_type_label(): + # Verify that accelerator type gets added as a resource + spec = ResourceAndLabelSpec(resources={}) + mock_mgr = MagicMock() + mock_mgr.get_resource_name.return_value = "GPU" + mock_mgr.get_current_node_accelerator_type.return_value = "H100" + mock_mgr.get_current_process_visible_accelerator_ids.return_value = None + + spec._resolve_accelerator_resources(mock_mgr, 4) + + # num_gpus should be set and accelerator type should be added + assert spec.num_gpus == 4 + assert any("H100" in key for key in spec.resources) + + +@patch("ray._private.usage.usage_lib.record_hardware_usage", lambda *_: None) +def test_resolve_accelerator_resources_raises_if_exceeds_visible_devices(): + # Raise a ValueError when requested accelerators > # visible IDs + spec = ResourceAndLabelSpec(resources={}) + mock_mgr = MagicMock() + mock_mgr.get_resource_name.return_value = "GPU" + mock_mgr.get_current_process_visible_accelerator_ids.return_value = ["0", "1"] + + with pytest.raises(ValueError, match="Attempting to start raylet"): + spec._resolve_accelerator_resources(mock_mgr, 3) + + if __name__ == "__main__": sys.exit(pytest.main(["-sv", __file__])) From 2ac985b0b619c2c666551e3293c775c47acfbf46 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Fri, 25 Jul 2025 20:27:09 +0000 Subject: [PATCH 29/39] Move unit tests file, change to call public APIs Signed-off-by: Ryan O'Leary --- python/ray/tests/BUILD | 1 - .../ray/tests/test_resource_and_label_spec.py | 201 ------------------ python/ray/tests/test_runtime_context.py | 19 +- python/ray/tests/unit/BUILD | 5 +- .../unit/test_resource_and_label_spec.py | 147 +++++++++++++ 5 files changed, 160 insertions(+), 213 deletions(-) delete mode 100644 python/ray/tests/test_resource_and_label_spec.py create mode 100644 python/ray/tests/unit/test_resource_and_label_spec.py diff --git a/python/ray/tests/BUILD b/python/ray/tests/BUILD index 39e2c47d46f6..584a331ee6b0 100644 --- a/python/ray/tests/BUILD +++ b/python/ray/tests/BUILD @@ -340,7 +340,6 @@ py_test_module_list( "test_ray_init.py", "test_ray_init_2.py", "test_ray_shutdown.py", - "test_resource_and_label_spec.py", "test_resource_metrics.py", "test_runtime_context.py", "test_runtime_env_env_vars.py", diff --git a/python/ray/tests/test_resource_and_label_spec.py b/python/ray/tests/test_resource_and_label_spec.py deleted file mode 100644 index fc2805e223d0..000000000000 --- a/python/ray/tests/test_resource_and_label_spec.py +++ /dev/null @@ -1,201 +0,0 @@ -import pytest -import sys -import json -from unittest.mock import patch, MagicMock -from ray._private.resource_and_label_spec import ResourceAndLabelSpec -from ray._common.constants import HEAD_NODE_RESOURCE_NAME, NODE_ID_PREFIX -import ray._private.ray_constants as ray_constants - - -def test_spec_resolved(): - spec = ResourceAndLabelSpec( - num_cpus=2, - num_gpus=1, - memory=1_000_000_000, - object_store_memory=500_000_000, - resources={"TPU": 5}, - labels={}, - ) - assert spec.resolved() - resource_dict = spec.to_resource_dict() - assert resource_dict["CPU"] == 2 - assert resource_dict["GPU"] == 1 - assert "TPU" in resource_dict - - -def test_resolved_false_until_resolve(): - spec = ResourceAndLabelSpec() - assert not spec.resolved() - - # Patch calls that rely on Ray environment for test - with patch("ray._private.utils.get_num_cpus", return_value=4), patch( - "ray.util.get_node_ip_address", return_value="127.0.0.1" - ), patch( - "ray._private.utils.estimate_available_memory", return_value=10000000 - ), patch( - "ray._common.utils.get_system_memory", return_value=20000000 - ), patch( - "ray._private.utils.get_shared_memory_bytes", return_value=50000 - ), patch.object( - ResourceAndLabelSpec, "_get_current_node_accelerator", return_value=(None, 0) - ), patch( - "ray._private.usage.usage_lib.record_hardware_usage", lambda *_: None - ): - spec.resolve(is_head=True) - - # Values set by default should be filled out - assert spec.resolved() - assert any(key.startswith(NODE_ID_PREFIX) for key in spec.resources.keys()) - assert HEAD_NODE_RESOURCE_NAME in spec.resources - - -def test_to_resource_dict_invalid_types(): - spec = ResourceAndLabelSpec( - num_cpus=1, - num_gpus=1, - memory=1000, - object_store_memory=1000, - resources={"INVALID_RESOURCE": -1}, - labels={}, - ) - assert spec.resolved() - with pytest.raises(ValueError): - spec.to_resource_dict() - - -@patch("ray._private.accelerators.get_accelerator_manager_for_resource") -@patch("ray._private.accelerators.get_all_accelerator_resource_names") -def test_get_current_node_accelerator_auto_detect(mock_all_names, mock_get_mgr): - mock_all_names.return_value = ["GPU", "TPU"] - mock_mgr = MagicMock() - mock_mgr.get_current_node_num_accelerators.return_value = 4 - mock_mgr.get_current_node_accelerator_type.return_value = "TPU" - mock_mgr.get_current_process_visible_accelerator_ids.return_value = [0, 1, 3, 4] - mock_get_mgr.return_value = mock_mgr - - result = ResourceAndLabelSpec._get_current_node_accelerator(None, resources={}) - assert result == (mock_mgr, 4) - - -@patch("ray._private.accelerators.get_accelerator_manager_for_resource") -@patch("ray._private.accelerators.get_all_accelerator_resource_names") -def test_get_current_node_accelerator_from_resources(mock_all_names, mock_get_mgr): - mock_all_names.return_value = ["A100"] - mock_mgr = MagicMock() - mock_get_mgr.return_value = mock_mgr - - result = ResourceAndLabelSpec._get_current_node_accelerator(None, {"A100": 3}) - assert result == (mock_mgr, 3) - - -@patch("ray._private.accelerators.get_accelerator_manager_for_resource") -@patch("ray._private.accelerators.get_all_accelerator_resource_names") -def test_get_current_node_accelerator_with_visibility_limit( - mock_all_names, mock_get_mgr -): - mock_all_names.return_value = ["A100"] - mock_mgr = MagicMock() - mock_mgr.get_current_node_num_accelerators.return_value = 5 - mock_mgr.get_current_process_visible_accelerator_ids.return_value = ["0", "1"] - mock_get_mgr.return_value = mock_mgr - - result = ResourceAndLabelSpec._get_current_node_accelerator(None, {}) - assert result == (mock_mgr, 2) - - -@patch("ray._private.accelerators.get_accelerator_manager_for_resource") -@patch("ray._private.accelerators.get_all_accelerator_resource_names") -def test_get_current_node_accelerator_none(mock_all_names, mock_get_mgr): - mock_all_names.return_value = ["B200", "TPU-v6e"] - mock_mgr = MagicMock() - mock_mgr.get_current_node_num_accelerators.return_value = 0 - mock_mgr.get_current_process_visible_accelerator_ids.return_value = [] - mock_get_mgr.side_effect = lambda name: mock_mgr - - result = ResourceAndLabelSpec._get_current_node_accelerator(None, {}) - assert result[0] is None and result[1] == 0 - - -def test_load_override_env_labels_merges_and_logs(monkeypatch): - env_data = {"autoscaler-override-label": "example"} - monkeypatch.setenv(ray_constants.LABELS_ENVIRONMENT_VARIABLE, json.dumps(env_data)) - spec = ResourceAndLabelSpec() - result = spec._load_env_labels() - assert result == env_data - - -@patch("ray._private.usage.usage_lib.record_hardware_usage", lambda *_: None) -def test_resolve_accelerator_resources_sets_num_gpus(): - # num-gpus passed in to spec and not detected from AcceleratorManager - spec = ResourceAndLabelSpec(resources={}, num_gpus=2) - mock_mgr = MagicMock() - mock_mgr.get_current_process_visible_accelerator_ids.return_value = None - spec._resolve_accelerator_resources(mock_mgr, 2) - assert spec.num_gpus == 2 - assert not any("GPU" in key for key in spec.resources.keys()) - - -@patch("ray._private.usage.usage_lib.record_hardware_usage", lambda *_: None) -def test_resolve_accelerator_resources_with_gpu_auto_detected(): - # num-gpus not passed in but detected from _resolve_accelerator_resources - spec = ResourceAndLabelSpec(num_gpus=0, resources={}) - mock_mgr = MagicMock() - mock_mgr.get_resource_name.return_value = "GPU" - mock_mgr.get_current_node_accelerator_type.return_value = "H100" - mock_mgr.get_current_process_visible_accelerator_ids.return_value = None - spec._resolve_accelerator_resources(mock_mgr, 4) - assert spec.num_gpus == 4 - - -@patch("ray._private.usage.usage_lib.record_hardware_usage", lambda *_: None) -def test_resolve_num_gpus_when_unset_and_not_detected(): - # Tests that the ResourceAndLabelSpec defaults num_gpus to 0 when not specified by the user - # and not detected by the AcceleratorManager - spec = ResourceAndLabelSpec(resources={}) - with patch.object( - ResourceAndLabelSpec, "_get_current_node_accelerator", return_value=(None, 0) - ), patch("ray._private.utils.get_num_cpus", return_value=4), patch( - "ray.util.get_node_ip_address", return_value="127.0.0.1" - ), patch( - "ray._private.utils.estimate_available_memory", return_value=10000000 - ), patch( - "ray._common.utils.get_system_memory", return_value=20000000 - ), patch( - "ray._private.utils.get_shared_memory_bytes", return_value=50000 - ): - spec.resolve(is_head=True) - - assert spec.num_gpus == 0 - assert spec.resolved() - - -@patch("ray._private.usage.usage_lib.record_hardware_usage", lambda *_: None) -def test_resolve_accelerator_resources_adds_accelerator_type_label(): - # Verify that accelerator type gets added as a resource - spec = ResourceAndLabelSpec(resources={}) - mock_mgr = MagicMock() - mock_mgr.get_resource_name.return_value = "GPU" - mock_mgr.get_current_node_accelerator_type.return_value = "H100" - mock_mgr.get_current_process_visible_accelerator_ids.return_value = None - - spec._resolve_accelerator_resources(mock_mgr, 4) - - # num_gpus should be set and accelerator type should be added - assert spec.num_gpus == 4 - assert any("H100" in key for key in spec.resources) - - -@patch("ray._private.usage.usage_lib.record_hardware_usage", lambda *_: None) -def test_resolve_accelerator_resources_raises_if_exceeds_visible_devices(): - # Raise a ValueError when requested accelerators > # visible IDs - spec = ResourceAndLabelSpec(resources={}) - mock_mgr = MagicMock() - mock_mgr.get_resource_name.return_value = "GPU" - mock_mgr.get_current_process_visible_accelerator_ids.return_value = ["0", "1"] - - with pytest.raises(ValueError, match="Attempting to start raylet"): - spec._resolve_accelerator_resources(mock_mgr, 3) - - -if __name__ == "__main__": - sys.exit(pytest.main(["-sv", __file__])) diff --git a/python/ray/tests/test_runtime_context.py b/python/ray/tests/test_runtime_context.py index 88a2baacb8a8..0d44d1925784 100644 --- a/python/ray/tests/test_runtime_context.py +++ b/python/ray/tests/test_runtime_context.py @@ -422,12 +422,11 @@ def test_get_node_labels(ray_start_cluster_head): resources={"worker1": 1}, num_cpus=1, labels={ - "accelerator-type": "A100", - "region": "us-west4", - "market-type": "spot", + "ray.io/accelerator-type": "A100", + "ray.io/availability-region": "us-west4", + "ray.io/market-type": "spot", }, ) - # ray.init(address=cluster.address) @ray.remote class Actor: @@ -438,20 +437,20 @@ def get_node_labels(self): return ray.get_runtime_context().get_node_labels() expected_node_labels = { - "accelerator-type": "A100", - "region": "us-west4", - "market-type": "spot", + "ray.io/accelerator-type": "A100", + "ray.io/availability-region": "us-west4", + "ray.io/market-type": "spot", } # Check node labels from Actor runtime context - a = Actor.options(label_selector={"accelerator-type": "A100"}).remote() + a = Actor.options(label_selector={"ray.io/accelerator-type": "A100"}).remote() node_labels = ray.get(a.get_node_labels.remote()) - expected_node_labels["ray.io/node_id"] = ray.get(a.get_node_id.remote()) + expected_node_labels["ray.io/node-id"] = ray.get(a.get_node_id.remote()) assert expected_node_labels == node_labels # Check node labels from driver runtime context (none are set except default) driver_labels = ray.get_runtime_context().get_node_labels() - assert {"ray.io/node_id": ray.get_runtime_context().get_node_id()} == driver_labels + assert {"ray.io/node-id": ray.get_runtime_context().get_node_id()} == driver_labels if __name__ == "__main__": diff --git a/python/ray/tests/unit/BUILD b/python/ray/tests/unit/BUILD index e97f92740310..d5990427fd57 100644 --- a/python/ray/tests/unit/BUILD +++ b/python/ray/tests/unit/BUILD @@ -26,7 +26,10 @@ py_test_run_all_subdirectory( py_test( name = "test_runtime_env_validation", size = "small", - srcs = ["test_runtime_env_validation.py"], + srcs = [ + "test_resource_and_label_spec.py", + "test_runtime_env_validation.py", + ], data = glob([ "test_runtime_env_validation_*_schema.json", ]), diff --git a/python/ray/tests/unit/test_resource_and_label_spec.py b/python/ray/tests/unit/test_resource_and_label_spec.py new file mode 100644 index 000000000000..9f93820ff97e --- /dev/null +++ b/python/ray/tests/unit/test_resource_and_label_spec.py @@ -0,0 +1,147 @@ +import sys +import json +import pytest +from unittest.mock import patch +from ray._common.constants import HEAD_NODE_RESOURCE_NAME, NODE_ID_PREFIX +import ray._private.ray_constants as ray_constants +from ray._private.resource_and_label_spec import ResourceAndLabelSpec + + +def test_resource_and_label_spec_resolves(monkeypatch): + """Validate that ResourceAndLabelSpec resolve() fills out defaults.""" + monkeypatch.setenv(ray_constants.LABELS_ENVIRONMENT_VARIABLE, "{}") + + with patch("ray._private.utils.get_num_cpus", return_value=4), patch( + "ray.util.get_node_ip_address", return_value="127.0.0.1" + ), patch( + "ray._private.utils.estimate_available_memory", return_value=1_000_000_000 + ), patch( + "ray._common.utils.get_system_memory", return_value=2_000_000_000 + ), patch( + "ray._private.utils.get_shared_memory_bytes", return_value=50_000 + ): + spec = ResourceAndLabelSpec() + spec.resolve(is_head=True) + + assert spec.resolved() + assert spec.num_cpus == 4 + assert spec.num_gpus == 0 + assert isinstance(spec.labels, dict) + assert HEAD_NODE_RESOURCE_NAME in spec.resources + assert any(key.startswith(NODE_ID_PREFIX) for key in spec.resources.keys()) + + +def test_to_resource_dict_with_invalid_types(): + """Validate malformed resource values raise ValueError from to_resource_dict().""" + spec = ResourceAndLabelSpec( + num_cpus=1, + num_gpus=1, + memory=1_000, + object_store_memory=1_000, + resources={"INVALID": -5}, # Invalid + labels={}, + ) + spec.resolve(is_head=True, node_ip_address="127.0.0.1") + with pytest.raises(ValueError): + spec.to_resource_dict() + + +def test_label_spec_resolve_merged_env_labels(monkeypatch): + """Validate that LABELS_ENVIRONMENT_VARIABLE is merged into final labels.""" + override_labels = {"autoscaler-override-label": "example"} + monkeypatch.setenv( + ray_constants.LABELS_ENVIRONMENT_VARIABLE, json.dumps(override_labels) + ) + + with patch("ray._private.utils.get_num_cpus", return_value=1), patch( + "ray.util.get_node_ip_address", return_value="127.0.0.1" + ), patch( + "ray._private.utils.estimate_available_memory", return_value=1_000_000 + ), patch( + "ray._common.utils.get_system_memory", return_value=2_000_000 + ), patch( + "ray._private.utils.get_shared_memory_bytes", return_value=50_000 + ): + spec = ResourceAndLabelSpec() + spec.resolve(is_head=True) + + assert any(key == "autoscaler-override-label" for key in spec.labels) + + +@patch("ray._private.usage.usage_lib.record_hardware_usage", lambda *_: None) +def test_resolve_sets_accelerator_resources(monkeypatch): + """Verify that GPUs/TPU values are auto-detected and assigned properly.""" + monkeypatch.setenv(ray_constants.LABELS_ENVIRONMENT_VARIABLE, "{}") + + # Mock a node with GPUs with 4 visible IDs + mock_mgr = ( + patch("ray._private.accelerators.get_accelerator_manager_for_resource") + .start() + .return_value + ) + patch( + "ray._private.accelerators.get_all_accelerator_resource_names", + return_value=["GPU"], + ).start() + mock_mgr.get_current_node_num_accelerators.return_value = 4 + mock_mgr.get_current_process_visible_accelerator_ids.return_value = [ + "0", + "1", + "2", + "3", + ] + mock_mgr.get_resource_name.return_value = "GPU" + mock_mgr.get_current_node_accelerator_type.return_value = "A100" + + with patch("ray._private.utils.get_num_cpus", return_value=2), patch( + "ray.util.get_node_ip_address", return_value="127.0.0.1" + ), patch("ray._private.utils.estimate_available_memory", return_value=10000), patch( + "ray._common.utils.get_system_memory", return_value=20000 + ), patch( + "ray._private.utils.get_shared_memory_bytes", return_value=50000 + ): + spec = ResourceAndLabelSpec() + spec.resolve(is_head=False) + + assert spec.num_gpus == 4 + assert any("A100" in key for key in spec.resources) + + +def test_resolve_raises_if_exceeds_visible_devices(monkeypatch): + """Check that ValueError is raised when requested accelerators exceed visible IDs.""" + monkeypatch.setenv(ray_constants.LABELS_ENVIRONMENT_VARIABLE, "{}") + + mock_mgr = ( + patch("ray._private.accelerators.get_accelerator_manager_for_resource") + .start() + .return_value + ) + patch( + "ray._private.accelerators.get_all_accelerator_resource_names", + return_value=["GPU"], + ).start() + mock_mgr.get_current_node_num_accelerators.return_value = 5 + mock_mgr.get_current_process_visible_accelerator_ids.return_value = [ + "0", + "1", + ] # 2 visible IDs + mock_mgr.get_resource_name.return_value = "GPU" + + spec = ResourceAndLabelSpec() + spec.num_gpus = 3 # request 3 GPUs + + with patch("ray._private.utils.get_num_cpus", return_value=2), patch( + "ray.util.get_node_ip_address", return_value="127.0.0.1" + ), patch( + "ray._private.utils.estimate_available_memory", return_value=1_000_000 + ), patch( + "ray._common.utils.get_system_memory", return_value=2_000_000 + ), patch( + "ray._private.utils.get_shared_memory_bytes", return_value=50_000 + ): + with pytest.raises(ValueError, match="Attempting to start raylet"): + spec.resolve(is_head=False) + + +if __name__ == "__main__": + sys.exit(pytest.main(["-sv", __file__])) From 683f5cbc42e7f40ea7a14bd11d2df35fae9818a8 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> Date: Fri, 25 Jul 2025 15:29:23 -0700 Subject: [PATCH 30/39] Update python/ray/_private/resource_and_label_spec.py Co-authored-by: Mengjin Yan Signed-off-by: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> --- python/ray/_private/resource_and_label_spec.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/_private/resource_and_label_spec.py b/python/ray/_private/resource_and_label_spec.py index 6b81522f6e91..116cb69cd8cd 100644 --- a/python/ray/_private/resource_and_label_spec.py +++ b/python/ray/_private/resource_and_label_spec.py @@ -119,7 +119,7 @@ def to_resource_dict(self): def resolve( self, is_head: bool, node_ip_address: Optional[str] = None ) -> "ResourceAndLabelSpec": - """Fills out this ResourceAndLabelSpec instance with system defaults. + """Fills out this ResourceAndLabelSpec instance with merged values from system defaults and user specification. Args: is_head: Whether this is the head node. From cd39761090ad84e46ca35802ca903bf90d26e381 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> Date: Fri, 25 Jul 2025 15:29:33 -0700 Subject: [PATCH 31/39] Update python/ray/_private/resource_and_label_spec.py Co-authored-by: Mengjin Yan Signed-off-by: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> --- python/ray/_private/resource_and_label_spec.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/_private/resource_and_label_spec.py b/python/ray/_private/resource_and_label_spec.py index 116cb69cd8cd..b4405f4a91c3 100644 --- a/python/ray/_private/resource_and_label_spec.py +++ b/python/ray/_private/resource_and_label_spec.py @@ -127,7 +127,7 @@ def resolve( This is used to automatically create a node id resource. Returns: - ResourceAndLabelSpec: This instance with all defaults resolved. + ResourceAndLabelSpec: This instance with all fields resolved. """ if self.resources is None: From 3363509bf4f21b1d5bde5846bb4cac21b05f6c93 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Sat, 26 Jul 2025 08:29:06 +0000 Subject: [PATCH 32/39] Add additional unit tests and move resource merge/load from env to ResourceAndLabelSpec Signed-off-by: Ryan O'Leary --- python/ray/_private/node.py | 52 +-- .../ray/_private/resource_and_label_spec.py | 78 ++++- python/ray/_private/worker.py | 8 +- .../unit/test_resource_and_label_spec.py | 320 +++++++++++++++--- 4 files changed, 345 insertions(+), 113 deletions(-) diff --git a/python/ray/_private/node.py b/python/ray/_private/node.py index 3342044f0a4c..9ec589adb602 100644 --- a/python/ray/_private/node.py +++ b/python/ray/_private/node.py @@ -521,55 +521,13 @@ def _init_temp(self): def get_resource_and_label_spec(self): """Resolve and return the current ResourceAndLabelSpec for the node.""" - - def merge_resources(env_dict, params_dict): - """Separates special case params and merges two dictionaries, picking from the - first in the event of a conflict. Also emit a warning on every - conflict. - """ - num_cpus = env_dict.pop("CPU", None) - num_gpus = env_dict.pop("GPU", None) - memory = env_dict.pop("memory", None) - object_store_memory = env_dict.pop("object_store_memory", None) - - result = params_dict.copy() - result.update(env_dict) - - for key in set(env_dict.keys()).intersection(set(params_dict.keys())): - if params_dict[key] != env_dict[key]: - logger.warning( - "Autoscaler is overriding your resource:" - f"{key}: {params_dict[key]} with {env_dict[key]}." - ) - return num_cpus, num_gpus, memory, object_store_memory, result - if not self._resource_and_label_spec: - env_resources = {} - env_string = os.getenv(ray_constants.RESOURCES_ENVIRONMENT_VARIABLE) - if env_string: - try: - env_resources = json.loads(env_string) - except Exception: - logger.exception(f"Failed to load {env_string}") - raise - logger.debug(f"Autoscaler overriding resources: {env_resources}.") - ( - num_cpus, - num_gpus, - memory, - object_store_memory, - resources, - ) = merge_resources(env_resources, self._ray_params.resources) self._resource_and_label_spec = ResourceAndLabelSpec( - self._ray_params.num_cpus if num_cpus is None else num_cpus, - self._ray_params.num_gpus if num_gpus is None else num_gpus, - self._ray_params.memory if memory is None else memory, - ( - self._ray_params.object_store_memory - if object_store_memory is None - else object_store_memory - ), - resources, + self._ray_params.num_cpus, + self._ray_params.num_gpus, + self._ray_params.memory, + self._ray_params.object_store_memory, + self._ray_params.resources, self._ray_params.labels, ).resolve(is_head=self.head, node_ip_address=self.node_ip_address) return self._resource_and_label_spec diff --git a/python/ray/_private/resource_and_label_spec.py b/python/ray/_private/resource_and_label_spec.py index b4405f4a91c3..831858c5cb17 100644 --- a/python/ray/_private/resource_and_label_spec.py +++ b/python/ray/_private/resource_and_label_spec.py @@ -19,7 +19,7 @@ class ResourceAndLabelSpec: All fields can be None. Before starting services, resolve() should be called to return a ResourceAndLabelSpec with unknown values filled in with - defaults based on the local machine specifications. + merged values based on the local machine and user specifications. """ def __init__( @@ -49,9 +49,15 @@ def __init__( self.object_store_memory = object_store_memory self.resources = resources self.labels = labels + self._is_resolved = False def resolved(self) -> bool: - """Returns if this ResourceAndLabelSpec has default values filled out.""" + """Returns if resolve() has been called for this ResourceAndLabelSpec + and default values are filled out.""" + return self._is_resolved + + def all_fields_set(self) -> bool: + """Returns whether all fields in this ResourceAndLabelSpec are not None.""" return all( v is not None for v in ( @@ -130,8 +136,25 @@ def resolve( ResourceAndLabelSpec: This instance with all fields resolved. """ - if self.resources is None: - self.resources = {} + # Load environment override resources and merge with resources passed + # in from Ray Params. Separates special case params if found in env. + env_resources = self._load_env_resources() + ( + num_cpus, + num_gpus, + memory, + object_store_memory, + merged_resources, + ) = self._merge_resources(env_resources, self.resources or {}) + self.num_cpus = self.num_cpus if num_cpus is None else num_cpus + self.num_gpus = self.num_gpus if num_gpus is None else num_gpus + self.memory = self.memory if memory is None else memory + self.object_store_memory = ( + self.object_store_memory + if object_store_memory is None + else object_store_memory + ) + self.resources = merged_resources if node_ip_address is None: node_ip_address = ray.util.get_node_ip_address() @@ -166,15 +189,52 @@ def resolve( if self.num_gpus is None: self.num_gpus = 0 - # Resolve node labels - self._resolve_labels(accelerator_manager) + # Resolve and merge node labels from all sources (params, env, and default). + self._merge_labels(accelerator_manager) # Resolve memory resources self._resolve_memory_resources() - assert self.resolved() + self._is_resolved = True + assert self.all_fields_set() return self + def _load_env_resources(self) -> Dict[str, float]: + """Load resource overrides from the environment, if present.""" + env_resources = {} + env_string = os.getenv(ray_constants.RESOURCES_ENVIRONMENT_VARIABLE) + if env_string: + try: + env_resources = json.loads(env_string) + except Exception: + logger.exception(f"Failed to load {env_string}") + raise + logger.debug(f"Autoscaler overriding resources: {env_resources}.") + return env_resources + + def _merge_resources( + self, env_dict: Dict[str, float], params_dict: Dict[str, float] + ): + """Merge environment and Ray param-provided resources, with env values taking precedence. + Returns separated special case params (CPU/GPU/memory) and the merged resource dict. + """ + num_cpus = env_dict.pop("CPU", None) + num_gpus = env_dict.pop("GPU", None) + memory = env_dict.pop("memory", None) + object_store_memory = env_dict.pop("object_store_memory", None) + + result = params_dict.copy() + result.update(env_dict) + + for key in set(env_dict.keys()).intersection(params_dict or {}): + if params_dict[key] != env_dict[key]: + logger.warning( + f"Autoscaler is overriding your resource: {key}: " + f"{params_dict[key]} with {env_dict[key]}." + ) + + return num_cpus, num_gpus, memory, object_store_memory, result + def _load_env_labels(self) -> Dict[str, str]: env_override_labels = {} env_override_labels_string = os.getenv( @@ -221,9 +281,7 @@ def _get_default_labels( return default_labels - def _resolve_labels( - self, accelerator_manager: Optional[AcceleratorManager] - ) -> None: + def _merge_labels(self, accelerator_manager: Optional[AcceleratorManager]) -> None: """Merge environment override, user-input from params, and Ray default labels in that order of precedence.""" diff --git a/python/ray/_private/worker.py b/python/ray/_private/worker.py index 1d97f8b02f07..6d253a761354 100644 --- a/python/ray/_private/worker.py +++ b/python/ray/_private/worker.py @@ -1054,10 +1054,12 @@ def get_accelerator_ids_for_accelerator_resource( # Give all accelerator ids in local_mode. if self.mode == LOCAL_MODE: if resource_name == ray_constants.GPU: - max_accelerators = self.node.get_resource_spec().num_gpus + max_accelerators = self.node.get_resource_and_label_spec().num_gpus else: - max_accelerators = self.node.get_resource_spec().resources.get( - resource_name, None + max_accelerators = ( + self.node.get_resource_and_label_spec().resources.get( + resource_name, None + ) ) if max_accelerators: assigned_ids = original_ids[:max_accelerators] diff --git a/python/ray/tests/unit/test_resource_and_label_spec.py b/python/ray/tests/unit/test_resource_and_label_spec.py index 9f93820ff97e..7a5fb7d2dae3 100644 --- a/python/ray/tests/unit/test_resource_and_label_spec.py +++ b/python/ray/tests/unit/test_resource_and_label_spec.py @@ -7,29 +7,102 @@ from ray._private.resource_and_label_spec import ResourceAndLabelSpec -def test_resource_and_label_spec_resolves(monkeypatch): - """Validate that ResourceAndLabelSpec resolve() fills out defaults.""" +def test_resource_and_label_spec_resolves_with_params(monkeypatch): + """Validate that ResourceAndLabelSpec resolve() respects passed in + Ray Params rather than overriding with auto-detection/system defaults.""" + # Create ResourceAndLabelSpec with args from RayParams. + spec = ResourceAndLabelSpec( + num_cpus=8, + num_gpus=2, + memory=10 * 1024**3, + object_store_memory=5 * 1024**3, + resources={"TPU": 42}, + labels={"ray.io/market-type": "spot"}, + ) + + spec.resolve(is_head=False) + + # Verify that explicit Ray Params values are preserved. + assert spec.num_cpus == 8 + assert spec.num_gpus == 2 + assert spec.memory == 10 * 1024**3 + assert spec.object_store_memory == 5 * 1024**3 + assert spec.resources["TPU"] == 42 + assert any(key.startswith(NODE_ID_PREFIX) for key in spec.resources) + assert spec.labels["ray.io/market-type"] == "spot" + + assert spec.resolved() + + +def test_resource_and_label_spec_resolves_auto_detect(monkeypatch): + """Validate that ResourceAndLabelSpec resolve() fills out defaults detected from + system when Params not passed.""" monkeypatch.setenv(ray_constants.LABELS_ENVIRONMENT_VARIABLE, "{}") - with patch("ray._private.utils.get_num_cpus", return_value=4), patch( - "ray.util.get_node_ip_address", return_value="127.0.0.1" - ), patch( - "ray._private.utils.estimate_available_memory", return_value=1_000_000_000 - ), patch( - "ray._common.utils.get_system_memory", return_value=2_000_000_000 - ), patch( - "ray._private.utils.get_shared_memory_bytes", return_value=50_000 - ): - spec = ResourceAndLabelSpec() - spec.resolve(is_head=True) + monkeypatch.setattr("ray._private.utils.get_num_cpus", lambda: 4) # 4 cpus + monkeypatch.setattr( + "ray._common.utils.get_system_memory", lambda: 16 * 1024**3 + ) # 16GB + monkeypatch.setattr( + "ray._private.utils.estimate_available_memory", lambda: 8 * 1024**3 + ) # 8GB + monkeypatch.setattr( + "ray._private.utils.get_shared_memory_bytes", lambda: 4 * 1024**3 + ) # 4GB + + spec = ResourceAndLabelSpec() + spec.resolve(is_head=True) assert spec.resolved() + + # Validate all fields are set based on defaults or calls to system. assert spec.num_cpus == 4 assert spec.num_gpus == 0 assert isinstance(spec.labels, dict) assert HEAD_NODE_RESOURCE_NAME in spec.resources assert any(key.startswith(NODE_ID_PREFIX) for key in spec.resources.keys()) + # object_store_memory = 8GB * DEFAULT_OBJECT_STORE_MEMORY_PROPORTION + expected_object_store = int( + 8 * 1024**3 * ray_constants.DEFAULT_OBJECT_STORE_MEMORY_PROPORTION + ) + assert spec.object_store_memory == expected_object_store + + # memory is total available memory - object_store_memory + expected_memory = 8 * 1024**3 - expected_object_store + assert spec.memory == expected_memory + + +def test_env_resource_overrides_with_conflict(monkeypatch, caplog): + """Validate that RESOURCES_ENVIRONMENT_VARIABLE overrides Ray Param resources.""" + # Prepare environment overrides + env_resources = { + "CPU": 8, + "GPU": 4, + "TPU": 4, + } + monkeypatch.setenv( + ray_constants.RESOURCES_ENVIRONMENT_VARIABLE, json.dumps(env_resources) + ) + + ray_params_resources = {"TPU": 8, "B200": 4} + + # num_cpus, num_gpus, and conflicting resources should override + spec = ResourceAndLabelSpec( + num_cpus=2, + num_gpus=1, + resources=ray_params_resources, + labels={}, + ) + + spec.resolve(is_head=True) + + # Environment overrides values take precedence after resolve + assert spec.num_cpus == 8 + assert spec.num_gpus == 4 + assert spec.resources["TPU"] == 4 + assert spec.resources["B200"] == 4 + def test_to_resource_dict_with_invalid_types(): """Validate malformed resource values raise ValueError from to_resource_dict().""" @@ -46,33 +119,148 @@ def test_to_resource_dict_with_invalid_types(): spec.to_resource_dict() +def test_resolve_memory_resources(monkeypatch): + """Validate that resolve correctly sets system object_store memory and + raises ValueError when configured memory is too low.""" + # object_store_memory capped at 95% of shm size to avoid low performance. + monkeypatch.setattr( + "ray._common.utils.get_system_memory", lambda: 2 * 1024**3 + ) # 2 GB + monkeypatch.setattr( + "ray._private.utils.estimate_available_memory", lambda: 1 * 1024**3 + ) # 2 GB + monkeypatch.setattr( + "ray._private.utils.get_shared_memory_bytes", lambda: 512 * 1024**2 + ) # 512 MB + + spec1 = ResourceAndLabelSpec() + spec1.resolve(is_head=False) + + max_shm = 512 * 1024**2 * 0.95 + assert spec1.object_store_memory <= max_shm + assert spec1.memory > 0 + + # Low available memory for tasks/actors triggers ValueError. + monkeypatch.setattr( + "ray._common.utils.get_system_memory", lambda: 2 * 1024**3 + ) # 2 GB + monkeypatch.setattr( + "ray._private.utils.estimate_available_memory", lambda: 100 * 1024**2 + ) # 100 MB + monkeypatch.setattr( + "ray._private.utils.get_shared_memory_bytes", lambda: 50 * 1024**2 + ) # 50 MB + + spec2 = ResourceAndLabelSpec() + with pytest.raises(ValueError, match="available for tasks and actors"): + spec2.resolve(is_head=False) + + +def test_resolve_raises_on_reserved_head_resource(): + """resolve should raise a ValueError if HEAD_NODE_RESOURCE_NAME is set in resources.""" + spec = ResourceAndLabelSpec(resources={HEAD_NODE_RESOURCE_NAME: 1}, labels={}) + with pytest.raises(ValueError, match=HEAD_NODE_RESOURCE_NAME): + spec.resolve(is_head=True) + + +def test_resolve_handles_no_accelerators(monkeypatch): + """Check resolve() is able to handle the no accelerators detected case.""" + # No accelerators are returned. + patch( + "ray._private.accelerators.get_all_accelerator_resource_names", + return_value=[], + ).start() + + spec = ResourceAndLabelSpec() + spec.resolve(is_head=False, node_ip_address="test") + + # With no accelerators detected or num_gpus, GPU count should default to 0 + # and the resources dictionary is unchanged. + assert spec.num_gpus == 0 + assert spec.resources == {"node:test": 1} + assert spec.resolved() + + def test_label_spec_resolve_merged_env_labels(monkeypatch): """Validate that LABELS_ENVIRONMENT_VARIABLE is merged into final labels.""" override_labels = {"autoscaler-override-label": "example"} monkeypatch.setenv( ray_constants.LABELS_ENVIRONMENT_VARIABLE, json.dumps(override_labels) ) - - with patch("ray._private.utils.get_num_cpus", return_value=1), patch( - "ray.util.get_node_ip_address", return_value="127.0.0.1" - ), patch( - "ray._private.utils.estimate_available_memory", return_value=1_000_000 - ), patch( - "ray._common.utils.get_system_memory", return_value=2_000_000 - ), patch( - "ray._private.utils.get_shared_memory_bytes", return_value=50_000 - ): - spec = ResourceAndLabelSpec() - spec.resolve(is_head=True) + spec = ResourceAndLabelSpec() + spec.resolve(is_head=True) assert any(key == "autoscaler-override-label" for key in spec.labels) @patch("ray._private.usage.usage_lib.record_hardware_usage", lambda *_: None) -def test_resolve_sets_accelerator_resources(monkeypatch): - """Verify that GPUs/TPU values are auto-detected and assigned properly.""" +def test_merge_labels_populates_defaults(monkeypatch): + """Ensure default labels (node type, market type, region, zone, accelerator) populate correctly.""" + # Patch Ray K8s label environment vars monkeypatch.setenv(ray_constants.LABELS_ENVIRONMENT_VARIABLE, "{}") + monkeypatch.setenv("RAY_NODE_TYPE_NAME", "worker-group-1") + monkeypatch.setenv("RAY_NODE_MARKET_TYPE", "spot") + monkeypatch.setenv("RAY_NODE_REGION", "us-west1") + monkeypatch.setenv("RAY_NODE_ZONE", "us-west1-a") + + # Patch accelerator manager to return an accelerator type + mock_mgr = ( + patch("ray._private.accelerators.get_accelerator_manager_for_resource") + .start() + .return_value + ) + patch( + "ray._private.accelerators.get_all_accelerator_resource_names", + return_value=["GPU"], + ).start() + mock_mgr.get_current_node_num_accelerators.return_value = 1 + mock_mgr.get_current_process_visible_accelerator_ids.return_value = ["0"] + mock_mgr.get_resource_name.return_value = "GPU" + mock_mgr.get_current_node_accelerator_type.return_value = "A100" + mock_mgr.get_current_node_additional_resources.return_value = {} + spec = ResourceAndLabelSpec() + spec.resolve(is_head=False) + + # Verify all default labels are present + assert spec.labels.get("ray.io/node-group") == "worker-group-1" + assert spec.labels.get("ray.io/market-type") == "spot" + assert spec.labels.get("ray.io/availability-region") == "us-west1" + assert spec.labels.get("ray.io/availability-zone") == "us-west1-a" + assert spec.labels.get("ray.io/accelerator-type") == "A100" + assert spec.resolved() + + +def test_resolve_raises_if_exceeds_visible_devices(monkeypatch): + """Check that ValueError is raised when requested accelerators exceed visible IDs.""" + monkeypatch.setenv(ray_constants.LABELS_ENVIRONMENT_VARIABLE, "{}") + + mock_mgr = ( + patch("ray._private.accelerators.get_accelerator_manager_for_resource") + .start() + .return_value + ) + patch( + "ray._private.accelerators.get_all_accelerator_resource_names", + return_value=["GPU"], + ).start() + mock_mgr.get_current_node_num_accelerators.return_value = 5 + mock_mgr.get_current_process_visible_accelerator_ids.return_value = [ + "0", + "1", + ] # 2 visible IDs + mock_mgr.get_resource_name.return_value = "GPU" + + spec = ResourceAndLabelSpec() + spec.num_gpus = 3 # request 3 GPUs + + with pytest.raises(ValueError, match="Attempting to start raylet"): + spec.resolve(is_head=False) + + +@patch("ray._private.usage.usage_lib.record_hardware_usage", lambda *_: None) +def test_resolve_sets_accelerator_resources(monkeypatch): + """Verify that GPUs/TPU values are auto-detected and assigned properly.""" # Mock a node with GPUs with 4 visible IDs mock_mgr = ( patch("ray._private.accelerators.get_accelerator_manager_for_resource") @@ -93,24 +281,19 @@ def test_resolve_sets_accelerator_resources(monkeypatch): mock_mgr.get_resource_name.return_value = "GPU" mock_mgr.get_current_node_accelerator_type.return_value = "A100" - with patch("ray._private.utils.get_num_cpus", return_value=2), patch( - "ray.util.get_node_ip_address", return_value="127.0.0.1" - ), patch("ray._private.utils.estimate_available_memory", return_value=10000), patch( - "ray._common.utils.get_system_memory", return_value=20000 - ), patch( - "ray._private.utils.get_shared_memory_bytes", return_value=50000 - ): - spec = ResourceAndLabelSpec() - spec.resolve(is_head=False) + spec = ResourceAndLabelSpec() + spec.resolve(is_head=False) assert spec.num_gpus == 4 - assert any("A100" in key for key in spec.resources) + assert spec.resources.get("accelerator_type:A100") == 1 -def test_resolve_raises_if_exceeds_visible_devices(monkeypatch): - """Check that ValueError is raised when requested accelerators exceed visible IDs.""" +@patch("ray._private.usage.usage_lib.record_hardware_usage", lambda *_: None) +def test_respect_configured_num_gpus(monkeypatch): + """Ensure manually set num_gpus overrides differing auto-detected accelerator value.""" monkeypatch.setenv(ray_constants.LABELS_ENVIRONMENT_VARIABLE, "{}") + # Mock GPU detection to return 4 GPUs by default. mock_mgr = ( patch("ray._private.accelerators.get_accelerator_manager_for_resource") .start() @@ -120,27 +303,58 @@ def test_resolve_raises_if_exceeds_visible_devices(monkeypatch): "ray._private.accelerators.get_all_accelerator_resource_names", return_value=["GPU"], ).start() - mock_mgr.get_current_node_num_accelerators.return_value = 5 + mock_mgr.get_current_node_num_accelerators.return_value = 4 mock_mgr.get_current_process_visible_accelerator_ids.return_value = [ "0", "1", - ] # 2 visible IDs + "2", + "3", + ] mock_mgr.get_resource_name.return_value = "GPU" + mock_mgr.get_current_node_accelerator_type.return_value = "A100" + + # Create a ResourceAndLabelSpec with num_gpus=2 from Ray Params. + spec = ResourceAndLabelSpec(num_gpus=2) + spec.resolve(is_head=False) + + assert spec.num_gpus == 2, ( + f"Expected manually set num_gpus=2 to take precedence over auto-detected value, " + f"but got {spec.num_gpus}" + ) + # Accelerator type key should be set in resources. + assert spec.resources.get("accelerator_type:A100") == 1 + + +@patch("ray._private.usage.usage_lib.record_hardware_usage", lambda *_: None) +def test_resolve_sets_non_gpu_accelerator(monkeypatch): + """Verify that non-GPU accelerators are added to resources. Non-GPU accelerators + should not alter the value of num_gpus.""" + # Mock accelerator manager to return a TPU accelerator + mock_mgr = ( + patch("ray._private.accelerators.get_accelerator_manager_for_resource") + .start() + .return_value + ) + patch( + "ray._private.accelerators.get_all_accelerator_resource_names", + return_value=["TPU"], + ).start() + mock_mgr.get_current_node_num_accelerators.return_value = 2 + mock_mgr.get_current_process_visible_accelerator_ids.return_value = ["0", "1"] + mock_mgr.get_resource_name.return_value = "TPU" + mock_mgr.get_current_node_accelerator_type.return_value = "TPU-v6e" + mock_mgr.get_current_node_additional_resources.return_value = {"TPU-v6e-8-HEAD": 1} spec = ResourceAndLabelSpec() - spec.num_gpus = 3 # request 3 GPUs + spec.resolve(is_head=False) - with patch("ray._private.utils.get_num_cpus", return_value=2), patch( - "ray.util.get_node_ip_address", return_value="127.0.0.1" - ), patch( - "ray._private.utils.estimate_available_memory", return_value=1_000_000 - ), patch( - "ray._common.utils.get_system_memory", return_value=2_000_000 - ), patch( - "ray._private.utils.get_shared_memory_bytes", return_value=50_000 - ): - with pytest.raises(ValueError, match="Attempting to start raylet"): - spec.resolve(is_head=False) + # num_gpus should default to 0 + assert spec.num_gpus == 0 + assert spec.resources["TPU"] == 2 + assert spec.resources["TPU-v6e-8-HEAD"] == 1 + # Accelerator type label is present + assert spec.labels.get("ray.io/accelerator-type") == "TPU-v6e" + assert spec.resolved() if __name__ == "__main__": From a5a68cc99926a8a8fa7bf223ce52c2954fce9769 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Sat, 26 Jul 2025 08:42:03 +0000 Subject: [PATCH 33/39] Make functions that don't use self static Signed-off-by: Ryan O'Leary --- .../ray/_private/resource_and_label_spec.py | 32 +++++++++++-------- 1 file changed, 18 insertions(+), 14 deletions(-) diff --git a/python/ray/_private/resource_and_label_spec.py b/python/ray/_private/resource_and_label_spec.py index 831858c5cb17..18e45f0a6da3 100644 --- a/python/ray/_private/resource_and_label_spec.py +++ b/python/ray/_private/resource_and_label_spec.py @@ -138,14 +138,14 @@ def resolve( # Load environment override resources and merge with resources passed # in from Ray Params. Separates special case params if found in env. - env_resources = self._load_env_resources() + env_resources = ResourceAndLabelSpec._load_env_resources() ( num_cpus, num_gpus, memory, object_store_memory, merged_resources, - ) = self._merge_resources(env_resources, self.resources or {}) + ) = ResourceAndLabelSpec._merge_resources(env_resources, self.resources or {}) self.num_cpus = self.num_cpus if num_cpus is None else num_cpus self.num_gpus = self.num_gpus if num_gpus is None else num_gpus self.memory = self.memory if memory is None else memory @@ -190,7 +190,7 @@ def resolve( self.num_gpus = 0 # Resolve and merge node labels from all sources (params, env, and default). - self._merge_labels(accelerator_manager) + self._resolve_labels(accelerator_manager) # Resolve memory resources self._resolve_memory_resources() @@ -199,7 +199,8 @@ def resolve( assert self.all_fields_set() return self - def _load_env_resources(self) -> Dict[str, float]: + @staticmethod + def _load_env_resources() -> Dict[str, float]: """Load resource overrides from the environment, if present.""" env_resources = {} env_string = os.getenv(ray_constants.RESOURCES_ENVIRONMENT_VARIABLE) @@ -212,9 +213,8 @@ def _load_env_resources(self) -> Dict[str, float]: logger.debug(f"Autoscaler overriding resources: {env_resources}.") return env_resources - def _merge_resources( - self, env_dict: Dict[str, float], params_dict: Dict[str, float] - ): + @staticmethod + def _merge_resources(env_dict: Dict[str, float], params_dict: Dict[str, float]): """Merge environment and Ray param-provided resources, with env values taking precedence. Returns separated special case params (CPU/GPU/memory) and the merged resource dict. """ @@ -235,7 +235,8 @@ def _merge_resources( return num_cpus, num_gpus, memory, object_store_memory, result - def _load_env_labels(self) -> Dict[str, str]: + @staticmethod + def _load_env_labels() -> Dict[str, str]: env_override_labels = {} env_override_labels_string = os.getenv( ray_constants.LABELS_ENVIRONMENT_VARIABLE @@ -250,8 +251,9 @@ def _load_env_labels(self) -> Dict[str, str]: return env_override_labels + @staticmethod def _get_default_labels( - self, accelerator_manager: Optional[AcceleratorManager] + accelerator_manager: Optional[AcceleratorManager], ) -> Dict[str, str]: default_labels = {} @@ -281,12 +283,14 @@ def _get_default_labels( return default_labels - def _merge_labels(self, accelerator_manager: Optional[AcceleratorManager]) -> None: - """Merge environment override, user-input from params, and Ray default labels in - that order of precedence.""" + def _resolve_labels( + self, accelerator_manager: Optional[AcceleratorManager] + ) -> None: + """Resolve and merge environment override, user-input from params, and Ray default + labels in that order of precedence.""" # Start with a dictionary filled out with Ray default labels - merged = self._get_default_labels(accelerator_manager) + merged = ResourceAndLabelSpec._get_default_labels(accelerator_manager) # Merge user-specified labels from Ray params for key, val in (self.labels or {}).items(): @@ -299,7 +303,7 @@ def _merge_labels(self, accelerator_manager: Optional[AcceleratorManager]) -> No merged[key] = val # Merge autoscaler override labels from environment - env_labels = self._load_env_labels() + env_labels = ResourceAndLabelSpec._load_env_labels() for key, val in (env_labels or {}).items(): if key in merged and merged[key] != val: logger.warning( From 9df1245a3ed37cf0ea4b5e23285e8e3779310a66 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Sat, 26 Jul 2025 08:43:45 +0000 Subject: [PATCH 34/39] Remove change to unit/BUILD Signed-off-by: Ryan O'Leary --- python/ray/tests/unit/BUILD | 1 - 1 file changed, 1 deletion(-) diff --git a/python/ray/tests/unit/BUILD b/python/ray/tests/unit/BUILD index d5990427fd57..42e3573f43c5 100644 --- a/python/ray/tests/unit/BUILD +++ b/python/ray/tests/unit/BUILD @@ -27,7 +27,6 @@ py_test( name = "test_runtime_env_validation", size = "small", srcs = [ - "test_resource_and_label_spec.py", "test_runtime_env_validation.py", ], data = glob([ From fd43e7195a8fcc32753b6ba7a0697f0baeafdc9c Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Sat, 26 Jul 2025 08:44:35 +0000 Subject: [PATCH 35/39] Format Signed-off-by: Ryan O'Leary --- python/ray/tests/unit/BUILD | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/python/ray/tests/unit/BUILD b/python/ray/tests/unit/BUILD index 42e3573f43c5..e97f92740310 100644 --- a/python/ray/tests/unit/BUILD +++ b/python/ray/tests/unit/BUILD @@ -26,9 +26,7 @@ py_test_run_all_subdirectory( py_test( name = "test_runtime_env_validation", size = "small", - srcs = [ - "test_runtime_env_validation.py", - ], + srcs = ["test_runtime_env_validation.py"], data = glob([ "test_runtime_env_validation_*_schema.json", ]), From e0f6f9de414dd406eb27731b04f50034e7f5397a Mon Sep 17 00:00:00 2001 From: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> Date: Mon, 28 Jul 2025 13:12:25 -0700 Subject: [PATCH 36/39] Update python/ray/_private/resource_and_label_spec.py Co-authored-by: Jiajun Yao Signed-off-by: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> --- python/ray/_private/resource_and_label_spec.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/_private/resource_and_label_spec.py b/python/ray/_private/resource_and_label_spec.py index 18e45f0a6da3..c9c3d8df4523 100644 --- a/python/ray/_private/resource_and_label_spec.py +++ b/python/ray/_private/resource_and_label_spec.py @@ -56,7 +56,7 @@ def resolved(self) -> bool: and default values are filled out.""" return self._is_resolved - def all_fields_set(self) -> bool: + def _all_fields_set(self) -> bool: """Returns whether all fields in this ResourceAndLabelSpec are not None.""" return all( v is not None From ee4b20d2016b366145862e2f5d90d8abb853006a Mon Sep 17 00:00:00 2001 From: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> Date: Mon, 28 Jul 2025 13:12:47 -0700 Subject: [PATCH 37/39] Update python/ray/_private/resource_and_label_spec.py Co-authored-by: Jiajun Yao Signed-off-by: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> --- python/ray/_private/resource_and_label_spec.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/_private/resource_and_label_spec.py b/python/ray/_private/resource_and_label_spec.py index c9c3d8df4523..e8c80fe402f7 100644 --- a/python/ray/_private/resource_and_label_spec.py +++ b/python/ray/_private/resource_and_label_spec.py @@ -74,7 +74,7 @@ def to_resource_dict(self): """Returns a dict suitable to pass to raylet initialization. This renames num_cpus / num_gpus to "CPU" / "GPU", - translates memory from bytes into 100MB memory units, and checks types. + and check types and values. """ assert self.resolved() From 5805a54bd8955f12a639f4617f85036990baefb3 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Tue, 29 Jul 2025 04:29:03 +0000 Subject: [PATCH 38/39] Change tests to use FakeAcceleratorManager, add _resolve_resources, and move record hardware usage to node.py Signed-off-by: Ryan O'Leary --- python/ray/_private/node.py | 10 + .../ray/_private/resource_and_label_spec.py | 96 ++++---- .../unit/test_resource_and_label_spec.py | 214 +++++++++--------- 3 files changed, 163 insertions(+), 157 deletions(-) diff --git a/python/ray/_private/node.py b/python/ray/_private/node.py index 9ec589adb602..206c342b25f8 100644 --- a/python/ray/_private/node.py +++ b/python/ray/_private/node.py @@ -1400,6 +1400,16 @@ def start_ray_processes(self): # Make sure we don't call `determine_plasma_store_config` multiple # times to avoid printing multiple warnings. resource_and_label_spec = self.get_resource_and_label_spec() + if resource_and_label_spec.labels.get( + ray._raylet.RAY_NODE_ACCELERATOR_TYPE_KEY + ): + from ray._private.usage import usage_lib + + usage_lib.record_hardware_usage( + resource_and_label_spec.labels.get( + ray._raylet.RAY_NODE_ACCELERATOR_TYPE_KEY + ) + ) ( plasma_directory, diff --git a/python/ray/_private/resource_and_label_spec.py b/python/ray/_private/resource_and_label_spec.py index e8c80fe402f7..f33b7d0a45fe 100644 --- a/python/ray/_private/resource_and_label_spec.py +++ b/python/ray/_private/resource_and_label_spec.py @@ -136,47 +136,9 @@ def resolve( ResourceAndLabelSpec: This instance with all fields resolved. """ - # Load environment override resources and merge with resources passed - # in from Ray Params. Separates special case params if found in env. - env_resources = ResourceAndLabelSpec._load_env_resources() - ( - num_cpus, - num_gpus, - memory, - object_store_memory, - merged_resources, - ) = ResourceAndLabelSpec._merge_resources(env_resources, self.resources or {}) - self.num_cpus = self.num_cpus if num_cpus is None else num_cpus - self.num_gpus = self.num_gpus if num_gpus is None else num_gpus - self.memory = self.memory if memory is None else memory - self.object_store_memory = ( - self.object_store_memory - if object_store_memory is None - else object_store_memory - ) - self.resources = merged_resources - - if node_ip_address is None: - node_ip_address = ray.util.get_node_ip_address() + self._resolve_resources(is_head=is_head, node_ip_address=node_ip_address) - # Automatically create a node id resource on each node. This is - # queryable with ray._private.state.node_ids() and - # ray._private.state.current_node_id(). - self.resources[NODE_ID_PREFIX + node_ip_address] = 1.0 - - # Automatically create a head node resource. - if HEAD_NODE_RESOURCE_NAME in self.resources: - raise ValueError( - f"{HEAD_NODE_RESOURCE_NAME}" - " is a reserved resource name, use another name instead." - ) - if is_head: - self.resources[HEAD_NODE_RESOURCE_NAME] = 1.0 - - if self.num_cpus is None: - self.num_cpus = ray._private.utils.get_num_cpus() - - # Resolve accelerator resources + # Resolve accelerator-specific resources ( accelerator_manager, num_accelerators, @@ -196,7 +158,7 @@ def resolve( self._resolve_memory_resources() self._is_resolved = True - assert self.all_fields_set() + assert self._all_fields_set() return self @staticmethod @@ -235,6 +197,54 @@ def _merge_resources(env_dict: Dict[str, float], params_dict: Dict[str, float]): return num_cpus, num_gpus, memory, object_store_memory, result + def _resolve_resources( + self, is_head: bool, node_ip_address: Optional[str] = None + ) -> None: + """Resolve CPU, GPU, and custom resources. Merges resources from environment, + Ray params, and defaults in that order of precedence.""" + + # Load environment override resources and merge with resources passed + # in from Ray Params. Separates special case params if found in env. + env_resources = ResourceAndLabelSpec._load_env_resources() + ( + num_cpus, + num_gpus, + memory, + object_store_memory, + merged_resources, + ) = ResourceAndLabelSpec._merge_resources(env_resources, self.resources or {}) + + self.num_cpus = self.num_cpus if num_cpus is None else num_cpus + self.num_gpus = self.num_gpus if num_gpus is None else num_gpus + self.memory = self.memory if memory is None else memory + self.object_store_memory = ( + self.object_store_memory + if object_store_memory is None + else object_store_memory + ) + self.resources = merged_resources + + if node_ip_address is None: + node_ip_address = ray.util.get_node_ip_address() + + # Automatically create a node id resource on each node. This is + # queryable with ray._private.state.node_ids() and + # ray._private.state.current_node_id(). + self.resources[NODE_ID_PREFIX + node_ip_address] = 1.0 + + # Automatically create a head node resource. + if HEAD_NODE_RESOURCE_NAME in self.resources: + raise ValueError( + f"{HEAD_NODE_RESOURCE_NAME}" + " is a reserved resource name, use another name instead." + ) + if is_head: + self.resources[HEAD_NODE_RESOURCE_NAME] = 1.0 + + # Auto-detect CPU count if not explicitly set + if self.num_cpus is None: + self.num_cpus = ray._private.utils.get_num_cpus() + @staticmethod def _load_env_labels() -> Dict[str, str]: env_override_labels = {} @@ -347,10 +357,6 @@ def _resolve_accelerator_resources(self, accelerator_manager, num_accelerators): accelerator_type = accelerator_manager.get_current_node_accelerator_type() if accelerator_type: self.resources[f"{RESOURCE_CONSTRAINT_PREFIX}{accelerator_type}"] = 1 - - from ray._private.usage import usage_lib - - usage_lib.record_hardware_usage(accelerator_type) additional_resources = ( accelerator_manager.get_current_node_additional_resources() ) diff --git a/python/ray/tests/unit/test_resource_and_label_spec.py b/python/ray/tests/unit/test_resource_and_label_spec.py index 7a5fb7d2dae3..76568b5b4fe0 100644 --- a/python/ray/tests/unit/test_resource_and_label_spec.py +++ b/python/ray/tests/unit/test_resource_and_label_spec.py @@ -4,10 +4,56 @@ from unittest.mock import patch from ray._common.constants import HEAD_NODE_RESOURCE_NAME, NODE_ID_PREFIX import ray._private.ray_constants as ray_constants +from ray._private.accelerators import AcceleratorManager from ray._private.resource_and_label_spec import ResourceAndLabelSpec -def test_resource_and_label_spec_resolves_with_params(monkeypatch): +class FakeAcceleratorManager(AcceleratorManager): + """Minimal fake Acceleratormanager for testing.""" + + # Configure these values to test different resource resolution paths. + def __init__( + self, + resource_name, + accelerator_type, + num_accelerators, + additional_resources=None, + visible_ids=None, + ): + self._resource_name = resource_name + self._accelerator_type = accelerator_type + self._num_accelerators = num_accelerators + self._additional_resources = additional_resources + self._visible_ids = visible_ids + + def get_current_node_num_accelerators(self) -> int: + return self._num_accelerators + + def get_current_process_visible_accelerator_ids(self): + if self._visible_ids is not None: + return [str(i) for i in range(self._visible_ids)] + return [str(i) for i in range(self._num_accelerators)] + + def get_resource_name(self) -> str: + return self._resource_name + + def get_current_node_accelerator_type(self) -> str: + return self._accelerator_type + + def get_visible_accelerator_ids_env_var(self) -> str: + return "CUDA_VISIBLE_DEVICES" + + def get_current_node_additional_resources(self): + return self._additional_resources or {} + + def set_current_process_visible_accelerator_ids(self, ids): + pass + + def validate_resource_request_quantity(self, quantity: int) -> None: + pass + + +def test_resource_and_label_spec_resolves_with_params(): """Validate that ResourceAndLabelSpec resolve() respects passed in Ray Params rather than overriding with auto-detection/system defaults.""" # Create ResourceAndLabelSpec with args from RayParams. @@ -37,8 +83,6 @@ def test_resource_and_label_spec_resolves_with_params(monkeypatch): def test_resource_and_label_spec_resolves_auto_detect(monkeypatch): """Validate that ResourceAndLabelSpec resolve() fills out defaults detected from system when Params not passed.""" - monkeypatch.setenv(ray_constants.LABELS_ENVIRONMENT_VARIABLE, "{}") - monkeypatch.setattr("ray._private.utils.get_num_cpus", lambda: 4) # 4 cpus monkeypatch.setattr( "ray._common.utils.get_system_memory", lambda: 16 * 1024**3 @@ -73,7 +117,7 @@ def test_resource_and_label_spec_resolves_auto_detect(monkeypatch): assert spec.memory == expected_memory -def test_env_resource_overrides_with_conflict(monkeypatch, caplog): +def test_env_resource_overrides_with_conflict(monkeypatch): """Validate that RESOURCES_ENVIRONMENT_VARIABLE overrides Ray Param resources.""" # Prepare environment overrides env_resources = { @@ -163,16 +207,15 @@ def test_resolve_raises_on_reserved_head_resource(): spec.resolve(is_head=True) -def test_resolve_handles_no_accelerators(monkeypatch): +def test_resolve_handles_no_accelerators(): """Check resolve() is able to handle the no accelerators detected case.""" + spec = ResourceAndLabelSpec() # No accelerators are returned. - patch( + with patch( "ray._private.accelerators.get_all_accelerator_resource_names", return_value=[], - ).start() - - spec = ResourceAndLabelSpec() - spec.resolve(is_head=False, node_ip_address="test") + ): + spec.resolve(is_head=False, node_ip_address="test") # With no accelerators detected or num_gpus, GPU count should default to 0 # and the resources dictionary is unchanged. @@ -193,7 +236,6 @@ def test_label_spec_resolve_merged_env_labels(monkeypatch): assert any(key == "autoscaler-override-label" for key in spec.labels) -@patch("ray._private.usage.usage_lib.record_hardware_usage", lambda *_: None) def test_merge_labels_populates_defaults(monkeypatch): """Ensure default labels (node type, market type, region, zone, accelerator) populate correctly.""" # Patch Ray K8s label environment vars @@ -203,24 +245,17 @@ def test_merge_labels_populates_defaults(monkeypatch): monkeypatch.setenv("RAY_NODE_REGION", "us-west1") monkeypatch.setenv("RAY_NODE_ZONE", "us-west1-a") - # Patch accelerator manager to return an accelerator type - mock_mgr = ( - patch("ray._private.accelerators.get_accelerator_manager_for_resource") - .start() - .return_value - ) - patch( + spec = ResourceAndLabelSpec() + + # AcceleratorManager for node with 1 GPU + with patch( + "ray._private.accelerators.get_accelerator_manager_for_resource", + return_value=FakeAcceleratorManager("GPU", "A100", 1), + ), patch( "ray._private.accelerators.get_all_accelerator_resource_names", return_value=["GPU"], - ).start() - mock_mgr.get_current_node_num_accelerators.return_value = 1 - mock_mgr.get_current_process_visible_accelerator_ids.return_value = ["0"] - mock_mgr.get_resource_name.return_value = "GPU" - mock_mgr.get_current_node_accelerator_type.return_value = "A100" - mock_mgr.get_current_node_additional_resources.return_value = {} - - spec = ResourceAndLabelSpec() - spec.resolve(is_head=False) + ): + spec.resolve(is_head=False) # Verify all default labels are present assert spec.labels.get("ray.io/node-group") == "worker-group-1" @@ -231,91 +266,55 @@ def test_merge_labels_populates_defaults(monkeypatch): assert spec.resolved() -def test_resolve_raises_if_exceeds_visible_devices(monkeypatch): +def test_resolve_raises_if_exceeds_visible_devices(): """Check that ValueError is raised when requested accelerators exceed visible IDs.""" - monkeypatch.setenv(ray_constants.LABELS_ENVIRONMENT_VARIABLE, "{}") - - mock_mgr = ( - patch("ray._private.accelerators.get_accelerator_manager_for_resource") - .start() - .return_value - ) - patch( - "ray._private.accelerators.get_all_accelerator_resource_names", - return_value=["GPU"], - ).start() - mock_mgr.get_current_node_num_accelerators.return_value = 5 - mock_mgr.get_current_process_visible_accelerator_ids.return_value = [ - "0", - "1", - ] # 2 visible IDs - mock_mgr.get_resource_name.return_value = "GPU" - spec = ResourceAndLabelSpec() spec.num_gpus = 3 # request 3 GPUs - with pytest.raises(ValueError, match="Attempting to start raylet"): - spec.resolve(is_head=False) + with patch( + "ray._private.accelerators.get_accelerator_manager_for_resource", + return_value=FakeAcceleratorManager( + "GPU", "A100", num_accelerators=5, visible_ids=2 + ), + ), patch( + "ray._private.accelerators.get_all_accelerator_resource_names", + return_value=["GPU"], + ): + with pytest.raises(ValueError, match="Attempting to start raylet"): + spec.resolve(is_head=False) -@patch("ray._private.usage.usage_lib.record_hardware_usage", lambda *_: None) -def test_resolve_sets_accelerator_resources(monkeypatch): +def test_resolve_sets_accelerator_resources(): """Verify that GPUs/TPU values are auto-detected and assigned properly.""" + spec = ResourceAndLabelSpec() + # Mock a node with GPUs with 4 visible IDs - mock_mgr = ( - patch("ray._private.accelerators.get_accelerator_manager_for_resource") - .start() - .return_value - ) - patch( + with patch( + "ray._private.accelerators.get_accelerator_manager_for_resource", + return_value=FakeAcceleratorManager("GPU", "A100", 4), + ), patch( "ray._private.accelerators.get_all_accelerator_resource_names", return_value=["GPU"], - ).start() - mock_mgr.get_current_node_num_accelerators.return_value = 4 - mock_mgr.get_current_process_visible_accelerator_ids.return_value = [ - "0", - "1", - "2", - "3", - ] - mock_mgr.get_resource_name.return_value = "GPU" - mock_mgr.get_current_node_accelerator_type.return_value = "A100" - - spec = ResourceAndLabelSpec() - spec.resolve(is_head=False) + ): + spec.resolve(is_head=False) assert spec.num_gpus == 4 assert spec.resources.get("accelerator_type:A100") == 1 -@patch("ray._private.usage.usage_lib.record_hardware_usage", lambda *_: None) -def test_respect_configured_num_gpus(monkeypatch): +def test_respect_configured_num_gpus(): """Ensure manually set num_gpus overrides differing auto-detected accelerator value.""" - monkeypatch.setenv(ray_constants.LABELS_ENVIRONMENT_VARIABLE, "{}") - - # Mock GPU detection to return 4 GPUs by default. - mock_mgr = ( - patch("ray._private.accelerators.get_accelerator_manager_for_resource") - .start() - .return_value - ) - patch( - "ray._private.accelerators.get_all_accelerator_resource_names", - return_value=["GPU"], - ).start() - mock_mgr.get_current_node_num_accelerators.return_value = 4 - mock_mgr.get_current_process_visible_accelerator_ids.return_value = [ - "0", - "1", - "2", - "3", - ] - mock_mgr.get_resource_name.return_value = "GPU" - mock_mgr.get_current_node_accelerator_type.return_value = "A100" - # Create a ResourceAndLabelSpec with num_gpus=2 from Ray Params. spec = ResourceAndLabelSpec(num_gpus=2) - spec.resolve(is_head=False) + # Mock a node with GPUs with 4 visible IDs + with patch( + "ray._private.accelerators.get_accelerator_manager_for_resource", + return_value=FakeAcceleratorManager("GPU", "A100", 4), + ), patch( + "ray._private.accelerators.get_all_accelerator_resource_names", + return_value=["GPU"], + ): + spec.resolve(is_head=False) assert spec.num_gpus == 2, ( f"Expected manually set num_gpus=2 to take precedence over auto-detected value, " @@ -325,28 +324,19 @@ def test_respect_configured_num_gpus(monkeypatch): assert spec.resources.get("accelerator_type:A100") == 1 -@patch("ray._private.usage.usage_lib.record_hardware_usage", lambda *_: None) -def test_resolve_sets_non_gpu_accelerator(monkeypatch): +def test_resolve_sets_non_gpu_accelerator(): """Verify that non-GPU accelerators are added to resources. Non-GPU accelerators should not alter the value of num_gpus.""" - # Mock accelerator manager to return a TPU accelerator - mock_mgr = ( - patch("ray._private.accelerators.get_accelerator_manager_for_resource") - .start() - .return_value - ) - patch( + spec = ResourceAndLabelSpec() + # Mock accelerator manager to return a TPU v6e accelerator + with patch( + "ray._private.accelerators.get_accelerator_manager_for_resource", + return_value=FakeAcceleratorManager("TPU", "TPU-v6e", 2, {"TPU-v6e-8-HEAD": 1}), + ), patch( "ray._private.accelerators.get_all_accelerator_resource_names", return_value=["TPU"], - ).start() - mock_mgr.get_current_node_num_accelerators.return_value = 2 - mock_mgr.get_current_process_visible_accelerator_ids.return_value = ["0", "1"] - mock_mgr.get_resource_name.return_value = "TPU" - mock_mgr.get_current_node_accelerator_type.return_value = "TPU-v6e" - mock_mgr.get_current_node_additional_resources.return_value = {"TPU-v6e-8-HEAD": 1} - - spec = ResourceAndLabelSpec() - spec.resolve(is_head=False) + ): + spec.resolve(is_head=False) # num_gpus should default to 0 assert spec.num_gpus == 0 From 4148cc0b1dac78e98b41aed348ff3267cc623042 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary Date: Tue, 29 Jul 2025 17:50:21 +0000 Subject: [PATCH 39/39] Fix import of usage_lib Signed-off-by: Ryan O'Leary --- python/ray/_private/node.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/_private/node.py b/python/ray/_private/node.py index 206c342b25f8..5a9cae459933 100644 --- a/python/ray/_private/node.py +++ b/python/ray/_private/node.py @@ -1403,7 +1403,7 @@ def start_ray_processes(self): if resource_and_label_spec.labels.get( ray._raylet.RAY_NODE_ACCELERATOR_TYPE_KEY ): - from ray._private.usage import usage_lib + from ray._common.usage import usage_lib usage_lib.record_hardware_usage( resource_and_label_spec.labels.get(