diff --git a/python/ray/_private/ray_constants.py b/python/ray/_private/ray_constants.py index c274095b2750..d5a2120e865d 100644 --- a/python/ray/_private/ray_constants.py +++ b/python/ray/_private/ray_constants.py @@ -382,6 +382,15 @@ def env_set_by_user(key): LANGUAGE_WORKER_TYPES = ["python", "java", "cpp"] NOSET_CUDA_VISIBLE_DEVICES_ENV_VAR = "RAY_EXPERIMENTAL_NOSET_CUDA_VISIBLE_DEVICES" +NOSET_XPU_VISIBLE_DEVICES_ENV_VAR = "RAY_EXPERIMENTAL_NOSET_XPU_VISIBLE_DEVICES" + +RAY_DEVICE_XPU_SELECTOR_ENV_VAR = "ONEAPI_DEVICE_SELECTOR" +RAY_DEVICE_XPU_BACKEND_TYPE = "level_zero" +RAY_DEVICE_XPU_DEVICE_TYPE = "gpu" + +RAY_DEVICE_SUPPORT_TYPES = {"CPU", "CUDA", "XPU"} +RAY_ACCELERATOR_DEFAULT = "CUDA" + RAY_WORKER_NICENESS = "RAY_worker_niceness" # Default max_retries option in @ray.remote for non-actor diff --git a/python/ray/_private/resource_spec.py b/python/ray/_private/resource_spec.py index d916a7db43b6..912aba1411b0 100644 --- a/python/ray/_private/resource_spec.py +++ b/python/ray/_private/resource_spec.py @@ -168,35 +168,21 @@ def resolve(self, is_head: bool, node_ip_address: Optional[str] = None): if is_head: resources[HEAD_NODE_RESOURCE_NAME] = 1.0 + # Get cpu num num_cpus = self.num_cpus if num_cpus is None: num_cpus = ray._private.utils.get_num_cpus() - num_gpus = self.num_gpus - gpu_ids = ray._private.utils.get_cuda_visible_devices() - # Check that the number of GPUs that the raylet wants doesn't - # exceed the amount allowed by CUDA_VISIBLE_DEVICES. - if num_gpus is not None and gpu_ids is not None and num_gpus > len(gpu_ids): - raise ValueError( - "Attempting to start raylet with {} GPUs, " - "but CUDA_VISIBLE_DEVICES contains {}.".format(num_gpus, gpu_ids) - ) - if num_gpus is None: - # Try to automatically detect the number of GPUs. - num_gpus = _autodetect_num_gpus() - # Don't use more GPUs than allowed by CUDA_VISIBLE_DEVICES. - if gpu_ids is not None: - num_gpus = min(num_gpus, len(gpu_ids)) - - try: - if importlib.util.find_spec("GPUtil") is not None: - gpu_types = _get_gpu_types_gputil() - else: - info_string = _get_gpu_info_string() - gpu_types = _constraints_from_gpu_info(info_string) + # Get accelerate device info + accelerator = ray._private.utils.get_current_accelerator() + if accelerator == "CUDA": # get cuda device num + num_gpus, gpu_types = _get_cuda_info(self.num_gpus) + resources.update(gpu_types) + elif accelerator == "XPU": # get xpu device num + # here we take xpu as gpu, so no need to develop core's scheduling policy + # If we don't want to take xpu as gpu, ray core need to develop new scheduling policy + num_gpus, gpu_types = _get_xpu_info(self.num_gpus) resources.update(gpu_types) - except Exception: - logger.exception("Could not parse gpu information.") # Choose a default object store size. system_memory = ray._private.utils.get_system_memory() @@ -276,6 +262,76 @@ def resolve(self, is_head: bool, node_ip_address: Optional[str] = None): return spec +def _get_cuda_info(num_gpus): + """ Attemp to process the number and type of GPUs + Notice: + If gpu id not specified in CUDA_VISIBLE_DEVICES, + and num_gpus is defined in task or actor, + this function will return the input num_gpus, not 0 + + Returns: + (num_gpus, gpu_types) + """ + gpu_ids = ray._private.utils.get_cuda_visible_devices() + # Check that the number of GPUs that the raylet wants doesn't + # exceed the amount allowed by CUDA_VISIBLE_DEVICES. + if num_gpus is not None and gpu_ids is not None and num_gpus > len(gpu_ids): + raise ValueError( + "Attempting to start raylet with {} GPUs, " + "but CUDA_VISIBLE_DEVICES contains {}.".format(num_gpus, gpu_ids) + ) + if num_gpus is None: + # Try to automatically detect the number of GPUs. + num_gpus = _autodetect_num_gpus() + # Don't use more GPUs than allowed by CUDA_VISIBLE_DEVICES. + if gpu_ids is not None: + num_gpus = min(num_gpus, len(gpu_ids)) + + gpu_types = "" + try: + if importlib.util.find_spec("GPUtil") is not None: + gpu_types = _get_gpu_types_gputil() + else: + info_string = _get_gpu_info_string() + gpu_types = _constraints_from_gpu_info(info_string) + except Exception: + logger.exception("Could not parse gpu information.") + + return num_gpus, gpu_types + + +def _get_xpu_info(num_xpus): + """Attempt to process the number of XPUs as GPUs + + Here we use `dpctl` to detect XPU device: + Enumrate all device by API dpctl.get_devices + Notice that ONEAPI_DEVICE_SELECTOR environment variable should be unset + Or dpctl.get_devices will only return filtered device set by ONEAPI_DEVICE_SELECTOR + Another method to enumrate XPU device is to use C++ API, maybe can upgrade later + + Returns: + The number of XPUs that detected by dpctl with specific backend and device type + """ + xpu_ids = ray._private.utils.get_xpu_visible_devices() + if num_xpus is not None and xpu_ids is not None and num_xpus > len(xpu_ids): + raise ValueError( + "Attempting to start raylet with {} XPUs, " + "but XPU_VISIBLE_DEVICES contains {}.".format(num_xpus, xpu_ids) + ) + if num_xpus is None: + try: + import dpctl + num_xpus = len(dpctl.get_devices(backend=ray_constants.RAY_DEVICE_XPU_BACKEND_TYPE, + device_type=ray_constants.RAY_DEVICE_XPU_DEVICE_TYPE)) + except ImportError: + num_xpus = 0 + + if xpu_ids is not None: + num_xpus = min(num_xpus, len(xpu_ids)) + xpu_types = {f"{ray_constants.RESOURCE_CONSTRAINT_PREFIX}" "xpu": 1} + return num_xpus, xpu_types + + def _autodetect_num_gpus(): """Attempt to detect the number of GPUs on this machine. diff --git a/python/ray/_private/utils.py b/python/ray/_private/utils.py index ac5d35fd5676..ad56814fa164 100644 --- a/python/ray/_private/utils.py +++ b/python/ray/_private/utils.py @@ -298,6 +298,33 @@ def get_cuda_visible_devices(): last_set_gpu_ids = None +def get_xpu_visible_devices(): + """Get the devices IDs in the XPU_VISIBLE_DEVICES environment variable. + Returns: + devices (List[str]): If XPU_VISIBLE_DEVICES is set, return a + list of strings representing the IDs of the visible XPUs. + If it is not set or is set, returns empty list. + """ + xpu_ids_str = os.environ.get("XPU_VISIBLE_DEVICES", None) + if xpu_ids_str is None: + return None + + if xpu_ids_str == "": + return [] + + return list(xpu_ids_str.split(",")) + + +def get_current_accelerator(): + return os.environ.get("RAY_ACCELERATOR", ray_constants.RAY_ACCELERATOR_DEFAULT) + + +def get_gpu_visible_devices(): + if get_current_accelerator() == "XPU": + return get_xpu_visible_devices() + return get_cuda_visible_devices() + + def set_omp_num_threads_if_unset() -> bool: """Set the OMP_NUM_THREADS to default to num cpus assigned to the worker @@ -338,22 +365,44 @@ def set_omp_num_threads_if_unset() -> bool: return True -def set_cuda_visible_devices(gpu_ids): +def set_cuda_visible_devices(dev_ids): """Set the CUDA_VISIBLE_DEVICES environment variable. Args: - gpu_ids (List[str]): List of strings representing GPU IDs. + dev_ids (List[str]): List of strings representing GPU IDs. """ if os.environ.get(ray_constants.NOSET_CUDA_VISIBLE_DEVICES_ENV_VAR): return global last_set_gpu_ids - if last_set_gpu_ids == gpu_ids: + if last_set_gpu_ids == dev_ids: return # optimization: already set - os.environ["CUDA_VISIBLE_DEVICES"] = ",".join([str(i) for i in gpu_ids]) - last_set_gpu_ids = gpu_ids + os.environ["CUDA_VISIBLE_DEVICES"] = ",".join([str(i) for i in dev_ids]) + last_set_gpu_ids = dev_ids + + +def set_xpu_visible_devices(dev_ids): + """Set the ONEAPI_DEVICE_SELECTOR environment variable. + Args: + dev_ids (List[str]): List of strings representing GPU IDs + """ + + if os.environ.get(ray_constants.NOSET_XPU_VISIBLE_DEVICES_ENV_VAR): + return + + ids_str = ",".join([str(i) for i in dev_ids]) + os.environ["XPU_VISIBLE_DEVICES"] = ids_str + os.environ["ONEAPI_DEVICE_SELECTOR"] = ray_constants.RAY_DEVICE_XPU_BACKEND_TYPE + ":" + ids_str + + +def set_gpu_visible_devices(device_ids): + accelerator = get_current_accelerator() + if accelerator == "XPU": + return set_xpu_visible_devices(device_ids) + elif accelerator == "CUDA": + return set_cuda_visible_devices(device_ids) def resources_from_ray_options(options_dict: Dict[str, Any]) -> Dict[str, Any]: diff --git a/python/ray/_private/worker.py b/python/ray/_private/worker.py index 5b045fe26e80..0bdecc34b4a3 100644 --- a/python/ray/_private/worker.py +++ b/python/ray/_private/worker.py @@ -422,9 +422,10 @@ def __init__(self): self.node = None self.mode = None self.actors = {} - # When the worker is constructed. Record the original value of the - # CUDA_VISIBLE_DEVICES environment variable. - self.original_gpu_ids = ray._private.utils.get_cuda_visible_devices() + # When the worker is constructed. + # Record the original value of the CUDA_VISIBLE_DEVICES environment variable, + # or value of the XPU_VISIBLE_DEVICES environment variable. + self.original_gpu_ids = ray._private.utils.get_gpu_visible_devices() # A dictionary that maps from driver id to SerializationContext # TODO: clean up the SerializationContext once the job finished. self.serialization_context_map = {} @@ -830,9 +831,7 @@ def print_logs(self): subscriber.close() -@PublicAPI -@client_mode_hook -def get_gpu_ids(): +def get_cuda_ids(): """Get the IDs of the GPUs that are available to the worker. If the CUDA_VISIBLE_DEVICES environment variable was set when the worker @@ -883,6 +882,65 @@ def get_gpu_ids(): return assigned_ids +def get_xpu_ids(): + """ Get the IDs of the XPUs that are available to the worker. + + If the XPU_VISIBLE_DEVICES environment variable was set when the worker + started up, then the IDs returned by this method will be a subset of the + IDs in CUDA_VISIBLE_DEVICES. If not, the IDs will fall in the range + [0, NUM_GPUS - 1], where NUM_GPUS is the number of GPUs that the node has. + + Returns: + A list of XPU IDs + """ + worker = global_worker + worker.check_connected() + + if worker.mode != WORKER_MODE: + if log_once("worker_get_gpu_ids_empty_from_driver"): + logger.warning( + "`ray.get_xpu_ids()` will always return the empty list when " + "called from the driver. This is because Ray does not manage " + "XPU allocations to the driver process." + ) + + # Get all resources from global core worker + all_resource_ids = global_worker.core_worker.resource_ids() + assigned_ids = set() + for resource, assignment in all_resource_ids.items(): + # Handle both normal and placement group GPU resources. + # Note: We should only get the GPU ids from the placement + # group resource that does not contain the bundle index! + import re + + if resource == "GPU" or re.match(r"^GPU_group_[0-9A-Za-z]+$", resource): + for resource_id, _ in assignment: + assigned_ids.add(resource_id) + + assigned_ids = list(assigned_ids) + # If the user had already set CUDA_VISIBLE_DEVICES, then respect that (in + # the sense that only GPU IDs that appear in CUDA_VISIBLE_DEVICES should be + # returned). + if global_worker.original_gpu_ids is not None: + assigned_ids = [ + global_worker.original_gpu_ids[gpu_id] for gpu_id in assigned_ids + ] + + return assigned_ids + + +@PublicAPI +@client_mode_hook +def get_gpu_ids(): + accelerator = ray._private.utils.get_current_accelerator() + ids = [] + if accelerator == "CUDA": + ids = get_cuda_ids() + elif accelerator == "XPU": + ids = get_xpu_ids() + return ids + + @Deprecated( message="Use ray.get_runtime_context().get_assigned_resources() instead.", warning=True, diff --git a/python/ray/_raylet.pyx b/python/ray/_raylet.pyx index 6031816fc3a7..1f35a8593511 100644 --- a/python/ray/_raylet.pyx +++ b/python/ray/_raylet.pyx @@ -1693,7 +1693,7 @@ cdef execute_task_with_cancellation_handler( title = f"ray::{task_name}" # Automatically restrict the GPUs available to this task. - ray._private.utils.set_cuda_visible_devices(ray.get_gpu_ids()) + ray._private.utils.set_gpu_visible_devices(ray.get_gpu_ids()) # Automatically configure OMP_NUM_THREADS to the assigned CPU number. # It will be unset after the task execution if it was overwridden here. diff --git a/python/ray/tests/test_actor_resources.py b/python/ray/tests/test_actor_resources.py index 6aa6f01f4fec..ab5e4341db9d 100644 --- a/python/ray/tests/test_actor_resources.py +++ b/python/ray/tests/test_actor_resources.py @@ -8,6 +8,8 @@ import ray import ray.cluster_utils +import ray._private.ray_constants as ray_constants + try: import pytest_timeout except ImportError: @@ -80,7 +82,9 @@ def echo(self, value): @pytest.mark.skipif(sys.platform == "win32", reason="Failing on Windows.") -def test_actor_gpus(ray_start_cluster): +@pytest.mark.parametrize("ACCELERATOR_TYPE", ["CUDA", "XPU"]) +def test_actor_gpus(ray_start_cluster, ACCELERATOR_TYPE): + os.environ["RAY_ACCELERATOR"] = ACCELERATOR_TYPE cluster = ray_start_cluster num_nodes = 3 num_gpus_per_raylet = 4 @@ -123,7 +127,9 @@ def get_location_and_ids(self): assert ready_ids == [] -def test_actor_multiple_gpus(ray_start_cluster): +@pytest.mark.parametrize("ACCELERATOR_TYPE", ["CUDA", "XPU"]) +def test_actor_multiple_gpus(ray_start_cluster, ACCELERATOR_TYPE): + os.environ["RAY_ACCELERATOR"] = ACCELERATOR_TYPE cluster = ray_start_cluster num_nodes = 3 num_gpus_per_raylet = 5 @@ -201,7 +207,9 @@ def get_location_and_ids(self): @pytest.mark.skipif(sys.platform == "win32", reason="Very flaky.") -def test_actor_different_numbers_of_gpus(ray_start_cluster): +@pytest.mark.parametrize("ACCELERATOR_TYPE", ["CUDA", "XPU"]) +def test_actor_different_numbers_of_gpus(ray_start_cluster, ACCELERATOR_TYPE): + os.environ["RAY_ACCELERATOR"] = ACCELERATOR_TYPE # Test that we can create actors on two nodes that have different # numbers of GPUs. cluster = ray_start_cluster @@ -243,7 +251,9 @@ def get_location_and_ids(self): assert ready_ids == [] -def test_actor_multiple_gpus_from_multiple_tasks(ray_start_cluster): +@pytest.mark.parametrize("ACCELERATOR_TYPE", ["CUDA", "XPU"]) +def test_actor_multiple_gpus_from_multiple_tasks(ray_start_cluster, ACCELERATOR_TYPE): + os.environ["RAY_ACCELERATOR"] = ACCELERATOR_TYPE cluster = ray_start_cluster num_nodes = 3 num_gpus_per_raylet = 2 @@ -320,7 +330,9 @@ def get_location_and_ids(self): assert ready_ids == [] -def test_actors_and_tasks_with_gpus(enable_syncer_test, ray_start_cluster): +@pytest.mark.parametrize("ACCELERATOR_TYPE", ["CUDA", "XPU"]) +def test_actors_and_tasks_with_gpus(enable_syncer_test, ray_start_cluster, ACCELERATOR_TYPE): + os.environ["RAY_ACCELERATOR"] = ACCELERATOR_TYPE cluster = ray_start_cluster num_nodes = 3 num_gpus_per_raylet = 2 @@ -439,7 +451,9 @@ def locations_to_intervals_for_many_tasks(): assert len(ready_ids) == 0 -def test_actors_and_tasks_with_gpus_version_two(shutdown_only): +@pytest.mark.parametrize("ACCELERATOR_TYPE", ["CUDA", "XPU"]) +def test_actors_and_tasks_with_gpus_version_two(shutdown_only, ACCELERATOR_TYPE): + os.environ["RAY_ACCELERATOR"] = ACCELERATOR_TYPE # Create tasks and actors that both use GPUs and make sure that they # are given different GPUs num_gpus = 4 @@ -616,7 +630,9 @@ def get_location(self): assert location == custom_resource2_node.unique_id -def test_creating_more_actors_than_resources(shutdown_only): +@pytest.mark.parametrize("ACCELERATOR_TYPE", ["CUDA", "XPU"]) +def test_creating_more_actors_than_resources(shutdown_only, ACCELERATOR_TYPE): + os.environ["RAY_ACCELERATOR"] = ACCELERATOR_TYPE ray.init(num_cpus=10, num_gpus=2, resources={"CustomResource1": 1}) @ray.remote(num_gpus=1) diff --git a/python/ray/tests/test_advanced_2.py b/python/ray/tests/test_advanced_2.py index 8bb2838c61e6..00aa5bc675bb 100644 --- a/python/ray/tests/test_advanced_2.py +++ b/python/ray/tests/test_advanced_2.py @@ -137,7 +137,9 @@ def method(self): assert valid_node.unique_id == ray.get(a.method.remote()) -def test_fractional_resources(shutdown_only): +@pytest.mark.parametrize("ACCELERATOR_TYPE", ["CUDA", "XPU"]) +def test_fractional_resources(shutdown_only, ACCELERATOR_TYPE): + os.environ["RAY_ACCELERATOR"] = ACCELERATOR_TYPE ray.init(num_cpus=6, num_gpus=3, resources={"Custom": 1}) @ray.remote(num_gpus=0.5) diff --git a/python/ray/tests/test_advanced_6.py b/python/ray/tests/test_advanced_6.py index 3e916f7eebcb..27b0c5681672 100644 --- a/python/ray/tests/test_advanced_6.py +++ b/python/ray/tests/test_advanced_6.py @@ -37,8 +37,26 @@ def save_gpu_ids_shutdown_only(): del os.environ["CUDA_VISIBLE_DEVICES"] +@pytest.fixture +def save_xpu_ids_shutdown_only(): + # Record the curent value of this environment variable so that we can + # reset it after the test. + original_xpu_ids = os.environ.get("XPU_VISIBLE_DEVICES", None) + + yield None + + # The code after the yield will run as teardown code. + ray.shutdown() + # Reset the environment variable. + if original_xpu_ids is not None: + os.environ["XPU_VISIBLE_DEVICES"] = original_xpu_ids + else: + del os.environ["XPU_VISIBLE_DEVICES"] + + @pytest.mark.skipif(platform.system() == "Windows", reason="Hangs on Windows") def test_specific_gpus(save_gpu_ids_shutdown_only): + os.environ["RAY_ACCELERATOR"] = "CUDA" allowed_gpu_ids = [4, 5, 6] os.environ["CUDA_VISIBLE_DEVICES"] = ",".join([str(i) for i in allowed_gpu_ids]) ray.init(num_gpus=3) @@ -60,6 +78,30 @@ def g(): ray.get([g.remote() for _ in range(100)]) +@pytest.mark.skipif(platform.system() == "Windows", reason="Hangs on Windows") +def test_specific_xpus(save_xpu_ids_shutdown_only): + os.environ["RAY_ACCELERATOR"] = "XPU" + allowed_xpu_ids = [1, 3, 5] + os.environ["XPU_VISIBLE_DEVICES"] = ",".join([str(i) for i in allowed_xpu_ids]) + ray.init(num_gpus=3) + + @ray.remote(num_gpus=1) + def f(): + xpu_ids = ray.get_gpu_ids() + assert len(xpu_ids) == 1 + assert int(xpu_ids[0]) in allowed_xpu_ids + + @ray.remote(num_gpus=2) + def g(): + xpu_ids = ray.get_gpu_ids() + assert len(xpu_ids) == 2 + assert int(xpu_ids[0]) in allowed_xpu_ids + assert int(xpu_ids[1]) in allowed_xpu_ids + + ray.get([f.remote() for _ in range(100)]) + ray.get([g.remote() for _ in range(100)]) + + def test_local_mode_gpus(save_gpu_ids_shutdown_only): allowed_gpu_ids = [4, 5, 6, 7, 8] os.environ["CUDA_VISIBLE_DEVICES"] = ",".join([str(i) for i in allowed_gpu_ids]) diff --git a/python/ray/tests/test_basic.py b/python/ray/tests/test_basic.py index d7a9a7919e85..a4e289978d35 100644 --- a/python/ray/tests/test_basic.py +++ b/python/ray/tests/test_basic.py @@ -230,7 +230,9 @@ def g(): assert ray.get(f.options(num_cpus=4).remote()) == "1" -def test_submit_api(shutdown_only): +@pytest.mark.parametrize("ACCELERATOR_TYPE", ["CUDA", "XPU"]) +def test_submit_api(shutdown_only, ACCELERATOR_TYPE): + os.environ["RAY_ACCELERATOR"] = ACCELERATOR_TYPE ray.init(num_cpus=2, num_gpus=1, resources={"Custom": 1}) @ray.remote