Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
71 changes: 71 additions & 0 deletions doc/source/ray-core/scheduling/accelerators.rst
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,9 @@ The accelerators natively supported by Ray Core are:
* - Rebellions RBLN
- RBLN
- Experimental, supported by the community
* - METAX GPU
- GPU
- Experimental, supported by the community

Starting Ray nodes with accelerators
------------------------------------
Expand Down Expand Up @@ -130,6 +133,15 @@ If you need to, you can :ref:`override <specify-node-resources>` this.
For example, ``RBLN_DEVICES=1,3 ray start --head --resources='{"RBLN": 2}'``
lets Ray only see devices 1 and 3.

.. tab-item:: METAX GPU
:sync: METAX GPU

.. tip::

You can set the ``CUDA_VISIBLE_DEVICES`` environment variable before starting a Ray node
to limit the METAX GPUs that are visible to Ray.
For example, ``CUDA_VISIBLE_DEVICES=1,3 ray start --head --num-gpus=2``
lets Ray only see devices 1 and 3.
.. note::

There's nothing preventing you from specifying a larger number of
Expand Down Expand Up @@ -457,6 +469,44 @@ and assign accelerators to the task or actor by setting the corresponding enviro
(rbln_task pid=51830) RBLN IDs: [1]
(rbln_task pid=51830) RBLN_DEVICES: 1

.. tab-item:: METAX GPU
:sync: METAX GPU

.. testcode::
:hide:

ray.shutdown()

.. testcode::

import os
import ray

ray.init(num_gpus=2)

@ray.remote(num_gpus=1)
class GPUActor:
def ping(self):
print("GPU IDs: {}".format(ray.get_runtime_context().get_accelerator_ids()["GPU"]))
print("CUDA_VISIBLE_DEVICES: {}".format(os.environ["CUDA_VISIBLE_DEVICES"]))

@ray.remote(num_gpus=1)
def gpu_task():
print("GPU IDs: {}".format(ray.get_runtime_context().get_accelerator_ids()["GPU"]))
print("CUDA_VISIBLE_DEVICES: {}".format(os.environ["CUDA_VISIBLE_DEVICES"]))

gpu_actor = GPUActor.remote()
ray.get(gpu_actor.ping.remote())
# The actor uses the first GPU so the task uses the second one.
ray.get(gpu_task.remote())

.. testoutput::
:options: +MOCK

(GPUActor pid=52420) GPU IDs: [0]
(GPUActor pid=52420) CUDA_VISIBLE_DEVICES: 0
(gpu_task pid=51830) GPU IDs: [1]
(gpu_task pid=51830) CUDA_VISIBLE_DEVICES: 1

Inside a task or actor, :func:`ray.get_runtime_context().get_accelerator_ids() <ray.runtime_context.RuntimeContext.get_accelerator_ids>` returns a
list of accelerator IDs that are available to the task or actor.
Expand Down Expand Up @@ -606,6 +656,27 @@ so multiple tasks and actors can share the same accelerator.

Rebellions RBLN doesn't support fractional resources.

.. tab-item:: METAX GPU
:sync: METAX GPU

.. testcode::
:hide:

ray.shutdown()

.. testcode::

ray.init(num_cpus=4, num_gpus=1)

@ray.remote(num_gpus=0.25)
def f():
import time

time.sleep(1)

# The four tasks created here can execute concurrently
# and share the same GPU.
ray.get([f.remote() for _ in range(4)])

**Note:** It is the user's responsibility to make sure that the individual tasks
don't use more than their share of the accelerator memory.
Expand Down
4 changes: 4 additions & 0 deletions python/ray/_private/accelerators/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -7,6 +7,7 @@
from ray._private.accelerators.amd_gpu import AMDGPUAcceleratorManager
from ray._private.accelerators.hpu import HPUAcceleratorManager
from ray._private.accelerators.intel_gpu import IntelGPUAcceleratorManager
from ray._private.accelerators.metax_gpu import MetaxGPUAcceleratorManager
from ray._private.accelerators.neuron import NeuronAcceleratorManager
from ray._private.accelerators.npu import NPUAcceleratorManager
from ray._private.accelerators.nvidia_gpu import NvidiaGPUAcceleratorManager
Expand All @@ -25,6 +26,7 @@ def get_all_accelerator_managers() -> Set[AcceleratorManager]:
HPUAcceleratorManager,
NPUAcceleratorManager,
RBLNAcceleratorManager,
MetaxGPUAcceleratorManager,
}


Expand Down Expand Up @@ -60,6 +62,8 @@ def get_accelerator_manager_for_resource(
resource_name_to_accelerator_manager["GPU"] = AMDGPUAcceleratorManager
elif IntelGPUAcceleratorManager.get_current_node_num_accelerators() > 0:
resource_name_to_accelerator_manager["GPU"] = IntelGPUAcceleratorManager
elif MetaxGPUAcceleratorManager.get_current_node_num_accelerators() > 0:
resource_name_to_accelerator_manager["GPU"] = MetaxGPUAcceleratorManager
Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Bug: Bug

The MetaxGPUAcceleratorManager class is missing from the module's __all__ list. This prevents direct import, which is inconsistent with other accelerator managers and can cause ImportError for users.

Fix in Cursor Fix in Web

else:
resource_name_to_accelerator_manager["GPU"] = NvidiaGPUAcceleratorManager
get_accelerator_manager_for_resource._resource_name_to_accelerator_manager = (
Expand Down
93 changes: 93 additions & 0 deletions python/ray/_private/accelerators/metax_gpu.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,93 @@
import logging
import os
from typing import List, Optional, Tuple

from ray._private.accelerators.accelerator import AcceleratorManager

logger = logging.getLogger(__name__)

CUDA_VISIBLE_DEVICES_ENV_VAR = "CUDA_VISIBLE_DEVICES"
NOSET_CUDA_VISIBLE_DEVICES_ENV_VAR = "RAY_EXPERIMENTAL_NOSET_CUDA_VISIBLE_DEVICES"


class MetaxGPUAcceleratorManager(AcceleratorManager):
"""Metax GPU accelerators."""

@staticmethod
def get_resource_name() -> str:
return "GPU"

@staticmethod
def get_visible_accelerator_ids_env_var() -> str:
return CUDA_VISIBLE_DEVICES_ENV_VAR

@staticmethod
def get_current_process_visible_accelerator_ids() -> Optional[List[str]]:
cuda_visible_devices = os.environ.get(
MetaxGPUAcceleratorManager.get_visible_accelerator_ids_env_var(), None
)
if cuda_visible_devices is None:
return None

if cuda_visible_devices == "":
return []

if cuda_visible_devices == "NoDevFiles":
return []
Comment on lines +35 to +36
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

medium

The check for cuda_visible_devices == "NoDevFiles" is a special case for NVIDIA GPUs when no devices are found. It's unlikely that Metax GPU drivers have the same behavior. This code seems to be copied from the NVIDIA accelerator manager and might be incorrect or dead code in this context. Please remove this check if it's not applicable to Metax GPUs.


return list(cuda_visible_devices.split(","))

@staticmethod
def get_current_node_num_accelerators() -> int:
try:
import pymxsml.mxsml_extension as pymxsml

try:
pymxsml.mxSmlExInit()
except pymxsml.MXSMLEXError:
return 0
device_count = pymxsml.mxSmlExDeviceGetCount()
pymxsml.mxSmlExShutdown()
return device_count
except Exception as e:
logger.debug("Could not import pymxsml: %s", e)
return 0

@staticmethod
def get_current_node_accelerator_type() -> Optional[str]:
try:
import pymxsml.mxsml_extension as pymxsml

try:
pymxsml.mxSmlExInit()
except pymxsml.MXSMLEXError:
return None
device_name = None
device_count = pymxsml.mxSmlExDeviceGetCount()
if device_count > 0:
handle = pymxsml.mxSmlExDeviceGetHandleByIndex(0)
device_name = pymxsml.mxSmlExDeviceGetName(handle)
if isinstance(device_name, bytes):
device_name = device_name.decode("utf-8")
pymxsml.mxSmlExShutdown()
return device_name
except Exception:
logger.exception("Failed to detect GPU type.")
return None

@staticmethod
def validate_resource_request_quantity(
quantity: float,
) -> Tuple[bool, Optional[str]]:
return (True, None)

@staticmethod
def set_current_process_visible_accelerator_ids(
visible_cuda_devices: List[str],
) -> None:
if os.environ.get(NOSET_CUDA_VISIBLE_DEVICES_ENV_VAR):
return

os.environ[
MetaxGPUAcceleratorManager.get_visible_accelerator_ids_env_var()
] = ",".join([str(i) for i in visible_cuda_devices])
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

medium

The visible_cuda_devices parameter is already a List[str], so the str(i) conversion inside the list comprehension is redundant. You can directly join the list elements.

        ] = ",".join(visible_cuda_devices)

1 change: 1 addition & 0 deletions python/ray/tests/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -526,6 +526,7 @@ py_test_module_list(
"accelerators/test_accelerators.py",
"accelerators/test_amd_gpu.py",
"accelerators/test_intel_gpu.py",
"accelerators/test_metax_gpu.py",
"accelerators/test_npu.py",
"accelerators/test_nvidia_gpu.py",
"accelerators/test_rbln.py",
Expand Down
81 changes: 81 additions & 0 deletions python/ray/tests/accelerators/test_metax_gpu.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,81 @@
import os
import sys
from unittest.mock import patch

import pytest

import ray
from ray._private.accelerators import (
MetaxGPUAcceleratorManager,
get_accelerator_manager_for_resource,
)


@patch(
"ray._private.accelerators.MetaxGPUAcceleratorManager.get_current_node_num_accelerators",
return_value=4,
)
def test_visible_metax_gpu_ids(mock_get_num_accelerators, monkeypatch, shutdown_only):
monkeypatch.setenv("CUDA_VISIBLE_DEVICES", "0,1,2")
del get_accelerator_manager_for_resource._resource_name_to_accelerator_manager
ray.init()
assert mock_get_num_accelerators.called
assert ray.available_resources()["GPU"] == 3


def test_metax_gpu_type(shutdown_only):
with patch(
"ray._private.accelerators.MetaxGPUAcceleratorManager.get_current_node_accelerator_type",
return_value="MXC500",
):
from ray.util import accelerators

ray.init()
result = MetaxGPUAcceleratorManager.get_current_node_accelerator_type()
assert result == accelerators.METAX_C500
Comment on lines +26 to +35
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

high

The test test_metax_gpu_type calls ray.init(), which triggers accelerator detection. This can fail in test environments that do not have Metax GPU hardware or drivers installed because get_current_node_num_accelerators is not mocked. Please mock get_current_node_num_accelerators to make the test more robust and independent of the environment.

Suggested change
def test_metax_gpu_type(shutdown_only):
with patch(
"ray._private.accelerators.MetaxGPUAcceleratorManager.get_current_node_accelerator_type",
return_value="MXC500",
):
from ray.util import accelerators
ray.init()
result = MetaxGPUAcceleratorManager.get_current_node_accelerator_type()
assert result == accelerators.METAX_C500
@patch(
"ray._private.accelerators.MetaxGPUAcceleratorManager.get_current_node_num_accelerators",
return_value=1,
)
def test_metax_gpu_type(mock_get_num_accelerators, shutdown_only):
with patch(
"ray._private.accelerators.MetaxGPUAcceleratorManager.get_current_node_accelerator_type",
return_value="MXC500",
):
from ray.util import accelerators
ray.init()
result = MetaxGPUAcceleratorManager.get_current_node_accelerator_type()
assert result == accelerators.METAX_C500



def test_get_current_process_visible_accelerator_ids(monkeypatch):
monkeypatch.setenv("CUDA_VISIBLE_DEVICES", "0")
assert MetaxGPUAcceleratorManager.get_current_process_visible_accelerator_ids() == [
"0"
]

monkeypatch.setenv("CUDA_VISIBLE_DEVICES", "0,4,7")
assert MetaxGPUAcceleratorManager.get_current_process_visible_accelerator_ids() == [
"0",
"4",
"7",
]

monkeypatch.setenv("CUDA_VISIBLE_DEVICES", "")
assert (
MetaxGPUAcceleratorManager.get_current_process_visible_accelerator_ids() == []
)

del os.environ["CUDA_VISIBLE_DEVICES"]
assert (
MetaxGPUAcceleratorManager.get_current_process_visible_accelerator_ids() is None
)
Comment on lines +38 to +59
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

medium

To improve test coverage and validate the handling of all edge cases, please add a test case for when CUDA_VISIBLE_DEVICES is set to "NoDevFiles". This is especially important given that this behavior is likely specific to NVIDIA drivers and its applicability to Metax GPUs should be confirmed.

def test_get_current_process_visible_accelerator_ids(monkeypatch):
    monkeypatch.setenv("CUDA_VISIBLE_DEVICES", "0")
    assert MetaxGPUAcceleratorManager.get_current_process_visible_accelerator_ids() == [
        "0"
    ]

    monkeypatch.setenv("CUDA_VISIBLE_DEVICES", "0,4,7")
    assert MetaxGPUAcceleratorManager.get_current_process_visible_accelerator_ids() == [
        "0",
        "4",
        "7",
    ]

    monkeypatch.setenv("CUDA_VISIBLE_DEVICES", "")
    assert (
        MetaxGPUAcceleratorManager.get_current_process_visible_accelerator_ids() == []
    )

    monkeypatch.setenv("CUDA_VISIBLE_DEVICES", "NoDevFiles")
    assert (
        MetaxGPUAcceleratorManager.get_current_process_visible_accelerator_ids() == []
    )

    del os.environ["CUDA_VISIBLE_DEVICES"]
    assert (
        MetaxGPUAcceleratorManager.get_current_process_visible_accelerator_ids() is None
    )



def test_set_current_process_visible_accelerator_ids():
MetaxGPUAcceleratorManager.set_current_process_visible_accelerator_ids(["0"])
assert os.environ["CUDA_VISIBLE_DEVICES"] == "0"

MetaxGPUAcceleratorManager.set_current_process_visible_accelerator_ids(["0", "1"])
assert os.environ["CUDA_VISIBLE_DEVICES"] == "0,1"

MetaxGPUAcceleratorManager.set_current_process_visible_accelerator_ids(
["0", "1", "7"]
)
assert os.environ["CUDA_VISIBLE_DEVICES"] == "0,1,7"
Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Bug: CUDA_VISIBLE_DEVICES leaks across tests - fix cleanup

test_set_current_process_visible_accelerator_ids sets CUDA_VISIBLE_DEVICES but never unsets it, causing the env var to leak across the test process and potentially affecting subsequent tests’ GPU detection and resource resolution. Add cleanup at the end (e.g., del os.environ["CUDA_VISIBLE_DEVICES"]) to prevent flakiness.

Fix in Cursor Fix in Web


del os.environ["CUDA_VISIBLE_DEVICES"]


if __name__ == "__main__":
if os.environ.get("PARALLEL_CI"):
sys.exit(pytest.main(["-n", "auto", "--boxed", "-vs", __file__]))
else:
sys.exit(pytest.main(["-sv", __file__]))
4 changes: 4 additions & 0 deletions python/ray/util/accelerators/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@
INTEL_GAUDI,
INTEL_MAX_1100,
INTEL_MAX_1550,
METAX_C500,
METAX_C550,
NVIDIA_A100,
NVIDIA_H100,
NVIDIA_L4,
Expand Down Expand Up @@ -60,6 +62,8 @@
"GOOGLE_TPU_V5P",
"GOOGLE_TPU_V5LITEPOD",
"GOOGLE_TPU_V6E",
"METAX_C500",
"METAX_C550",
# Deprecated
"NVIDIA_TESLA_A100",
]
Expand Down
2 changes: 2 additions & 0 deletions python/ray/util/accelerators/accelerators.py
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,8 @@
GOOGLE_TPU_V6E = "TPU-V6E"
HUAWEI_NPU_910B = "Ascend910B"
HUAWEI_NPU_910B4 = "Ascend910B4"
METAX_C500 = "MXC500"
METAX_C550 = "MXC550"

# Use these instead of NVIDIA_A100 if you need a specific accelerator size. Note that
# these labels are not auto-added to nodes, you'll have to add them manually in
Expand Down