Movatterモバイル変換


[0]ホーム

URL:


Skip to content

Navigation Menu

Search code, repositories, users, issues, pull requests...

Provide feedback

We read every piece of feedback, and take your input very seriously.

Saved searches

Use saved searches to filter your results more quickly

Sign up
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to ourterms of service andprivacy statement. We’ll occasionally send you account related emails.

Already on GitHub?Sign in to your account

[Compiled Graph] Enhance Compile Graph with Multi-Device Support#51032

Open
hipudding wants to merge1 commit intoray-project:master
base:master
Choose a base branch
Loading
fromhipudding:cupystream

Conversation

hipudding
Copy link

@hipuddinghipudding commentedMar 3, 2025
edited
Loading

Why are these changes needed?

This PR improves multi-device support in Compile Graph, which significantly reduces Tensor transmission latency by utilizing out-of-band communication. Currently, this feature only supports CUDA’s NCCL. Since Ray already supports multiple accelerators, it is necessary to extend Compile Graph to support multi-device as well.

This PR mainly introduces two key changes:
1.Removed dependency on cupy.cuda.ExternalStream – Since this library only supports CUDA devices, we replaced it with a more general stream context manager to accommodate various accelerators. The new implementation uses torch.{device}.StreamContext.
2.Replaced hardcoded torch.cuda.xxx calls with AcceleratorRuntime – This allows automatic detection of the accelerator type and invokes the appropriate device-specific functions.

Steps required to add a new backend for CG:

1. Implement the new backend’s Communicator and AcceleratorRuntime.

--- base.py     2025-03-25 15:24:48.989298863 +0800+++ xpu.py      2025-03-25 15:24:58.701399896 +0800@@ -1,3 +1,7 @@ # xccl_group.py(Refer to the implementation in nccl_group.py for guidance.)+class Xccl_Group(Communicator):+    pass # utils.py(Refer to the implementation of CudaRuntime)+class XpuRuntime(AcceleratorRuntime)+    pass

2. Add the new device to the Device class and incorporate the automatic selection feature for the new device.

--- base.py     2025-03-25 15:53:02.182771033 +0800+++ xpu.py      2025-03-25 15:52:35.510496063 +0800@@ -1,40 +1,45 @@ class AcceleratorRuntime:     def get(cls):        if cls._instance is None:            if len([str(id) for id in ray.get_gpu_ids()]) > 0:                cls._instance = CudaRuntime()+           elif [str(id) for id in ray.get_runtime_context().get_accelerator_ids()[ray_constants.XPU]:+               cls._instance = XpuRuntime()            else device.type == "cpu":                cls._instance = CpuRuntime()        return cls._instance # Add the device type. class Device(Enum):     DEFAULT = "default"     CPU = "cpu"     GPU = "gpu"     CUDA = "cuda"+    XPU = "xpu"

Related issue number

This PR is the main part of Task 2 in#51574
It would better to set the function name more general, such as changing requires_nccl to require_communicator. This is implemented in#51061.

Checks

  • I've signed off every commit(by using the -s flag, i.e.,git commit -s) in this PR.
  • I've runscripts/format.sh to lint the changes in this PR.
  • I've included any doc changes needed forhttps://docs.ray.io/en/master/.
    • I've added any new APIs to the API Reference. For example, if I added a
      method in Tune, I've added it indoc/source/tune/api/ under the
      corresponding.rst file.
  • I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures athttps://flakey-tests.ray.io/
  • Testing Strategy
    • Unit tests
    • Release tests
    • This PR is not tested :(

@hipuddinghipudding marked this pull request as draftMarch 3, 2025 11:47
@hipuddinghipuddingforce-pushed thecupystream branch 2 times, most recently fromf63223c to27215f3CompareMarch 4, 2025 02:55
@hipudding
Copy link
Author

@ruisearch42 Good Day. Could you please review this PR? Thanks.

@hipuddinghipudding marked this pull request as ready for reviewMarch 4, 2025 06:16
@hipuddinghipudding changed the title[WIP] Replacement for cupy.cuda.ExternalStream for hardware diversityReplacement for cupy.cuda.ExternalStream for hardware diversityMar 4, 2025
@hipuddinghipudding changed the titleReplacement for cupy.cuda.ExternalStream for hardware diversity[aDAG] Replacement for cupy.cuda.ExternalStream for hardware diversityMar 4, 2025
"""
raise NotImplementedError
return None
Copy link
Contributor

Choose a reason for hiding this comment

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

If the method returns None, is it still necessary to mark it as an abstract method?

Copy link
Author

@hipuddinghipuddingMar 4, 2025
edited
Loading

Choose a reason for hiding this comment

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

Keep this class abstrct.

@hipuddinghipuddingforce-pushed thecupystream branch 2 times, most recently froma4d06c9 to705a10dCompareMarch 4, 2025 11:18
@jcotant1jcotant1 added the coreIssues that should be addressed in Ray Core labelMar 4, 2025
@hipuddinghipudding changed the title[aDAG] Replacement for cupy.cuda.ExternalStream for hardware diversity[aDAG] Change for acclerator diversity and add HCCL communication libMar 5, 2025
@ruisearch42
Copy link
Contributor

Hi@hipudding , is this ready for review? btw, aDAG was renamed to Compiled Graph:https://docs.ray.io/en/latest/ray-core/compiled-graph/ray-compiled-graph.html

@hipudding
Copy link
Author

hipudding commentedMar 5, 2025
edited
Loading

Hi@hipudding , is this ready for review? btw, aDAG was renamed to Compiled Graph:https://docs.ray.io/en/latest/ray-core/compiled-graph/ray-compiled-graph.html

Yes, I think the main functionality of this PR is ready, I may make some minor adjustments later. Thanks for your review.
I'vd noticed that there's some test cases failed. I will fix them ASAP.

@hipuddinghipudding changed the title[aDAG] Change for acclerator diversity and add HCCL communication lib[Compiled Graph] Change for acclerator diversity and add HCCL communication libMar 5, 2025
@ruisearch42
Copy link
Contributor

Yes, I think the main functionality of this PR is ready, I may make some minor adjustments later. Thanks for your review. I'vd noticed that there's some test cases failed. I will fix them ASAP.

Thanks. I will review tomorrow!

hipudding reacted with hooray emoji

Copy link
Contributor

@ruisearch42ruisearch42 left a comment

Choose a reason for hiding this comment

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

Trying to better understand the PR

hipudding reacted with thumbs up emoji
Comment on lines 775 to 787
with self._send_stream:
return self._write()
Copy link
Contributor

Choose a reason for hiding this comment

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

Why don't we want to use context manager?

Copy link
Author

@hipuddinghipuddingMar 6, 2025
edited
Loading

Choose a reason for hiding this comment

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

This stream context manager is provided by cupy, it only support CUDA devices, So I just remote context manager and specified stream in read and write.
But I think I can implement a stream context manager in Ray, so cupy is not needed. Which is better?

Copy link
Author

@hipuddinghipuddingMar 6, 2025
edited
Loading

Choose a reason for hiding this comment

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

Done. use torch StreamContext instead to avoid too many changes.

else:
future = ResolvedFuture(val)
self._intermediate_future = future

def reset_and_wait_intermediate_future(self) -> Any:
def reset_and_wait_intermediate_future(self, stream: Any = None) -> Any:
Copy link
Contributor

Choose a reason for hiding this comment

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

update doc for arg

Copy link
Author

@hipuddinghipuddingMar 6, 2025
edited
Loading

Choose a reason for hiding this comment

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

Done. reverted this part.

self,
val: Any,
wrap_in_gpu_future: bool,
stream: Any = None,
Copy link
Contributor

Choose a reason for hiding this comment

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

update doc for stream

Copy link
Author

@hipuddinghipuddingMar 6, 2025
edited
Loading

Choose a reason for hiding this comment

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

Done. reverted this part.

Comment on lines 10 to 24
class _DriverGroupHolder(Communicator):
"""
Communicator place holder for Driver, Since driver may has no
Accelerators, TorchDeviceManager cannot be used.
"""
Copy link
Contributor

Choose a reason for hiding this comment

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

Can you elaborate a bit why this is needed?

Copy link
Author

Choose a reason for hiding this comment

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

I replace all torch.cuda.xxx by TorchDeviceManager, including create a _NcclGroup. But driver will keep a communicator instance itself. Driver will not join the group, but need record some information like world size and actors.
But Driver may has no GPU ( or other accelerators), it will choose the wrong TorchDeviceManager (CPUTorchDeviceManager), and get_communicator will return None.
Since Driver will not join the group, it use Communicator only for store some information. So I create a Hardware-independent class for driver.

Copy link
Author

Choose a reason for hiding this comment

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

Comment updated.

comm_id: int,
rank: Optional[int],
actor_handles: List["ray.actor.ActorHandle"],
acl_stream: Optional[Any],
Copy link
Contributor

Choose a reason for hiding this comment

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

What is acl_tream? doc for args

Copy link
Author

Choose a reason for hiding this comment

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

acl_stream is the raw stream in Ascend NPU. Just like cuda_stream to GPU.
What about name this stream torch_stream? since acl_stream and cuda_stream are both torch_stream now.

Copy link
Author

Choose a reason for hiding this comment

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

Done. I changed it to npu_stream for better understanding.

def get_world_size(self) -> int:
return self._world_size

def send(self, buf: "torch.Tensor", peer_rank: int) -> None:
Copy link
Contributor

Choose a reason for hiding this comment

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

many methods in this class look like code duplication from_NcclGroup with minor changes. Should they be unified?

Copy link
Author

Choose a reason for hiding this comment

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

Yes, we should indeed reduce duplicate code. I will think about how to design it.

Copy link
Author

Choose a reason for hiding this comment

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

Indeed, these two files share logical similarities and contain a lot of duplicate code. However, there are several differences in the details. For example, torch.cuda.xxx needs to be replaced with torch.npu.xxx, hccl_group requires an additional StreamContext, and the set device operation must be performed before get_unique_id and creating the HCCL group. Therefore, I haven’t found a good way to merge these two files. Do you have any suggestions?

@hipuddinghipuddingforce-pushed thecupystream branch 2 times, most recently from1e80d9f tod682702CompareMarch 6, 2025 08:03
@hipuddinghipudding changed the title[Compiled Graph] Change for acclerator diversity and add HCCL communication lib[Compiled Graph] Enhance Compile Graph with Multi-Device Support and HCCL IntegrationMar 7, 2025
@hipuddinghipudding requested a review froma team as acode ownerMarch 7, 2025 06:26
@hipudding
Copy link
Author

hipudding commentedMar 7, 2025
edited
Loading

Hi@ruisearch42, Could you please review this PR again?

BTW, I fixed test case fail in doc_test_cgraph_nccl. It blocked this PR.

@hipudding
Copy link
Author

hipudding commentedMar 25, 2025
edited
Loading

@liuxsh9@Bye-legumes@noemotiovon Good day. Could you please review this PR? Thanks.
This PR only includes the generalization of device handling in CG, and the HCCL part has been removed. If you want to check the historical HCCL implementation, please clickthis link

@hipuddinghipuddingforce-pushed thecupystream branch 3 times, most recently from11e5d68 to966d687CompareMarch 27, 2025 01:05
Copy link
Contributor

@noemotiovonnoemotiovon left a comment

Choose a reason for hiding this comment

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

The overall architecture LGTM. Here are some minor suggestions.


returntorch.device("cuda:0")
returngenerate_uid_id()
Copy link
Contributor

Choose a reason for hiding this comment

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

Could this method be directly inlined? Or alternatively, could theget_unique_id method in the base class be defined as str(uuid.uuid4())

Copy link
Author

Choose a reason for hiding this comment

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

Fixed.

def __init__(
self,
world_size: int,
comm_id: int,
Copy link
Contributor

Choose a reason for hiding this comment

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

This parameter is unused.

Copy link
Author

Choose a reason for hiding this comment

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

Done.

@liuxsh9
Copy link
Contributor

liuxsh9 commentedMar 27, 2025
edited
Loading

@liuxsh9@Bye-legumes@noemotiovon Good day. Could you please review this PR? Thanks. This PR only includes the generalization of device handling in CG, and the HCCL part has been removed. If you want to check the historical HCCL implementation, please clickthis link

The refactoring looks good to me! It seems that CG will be easily to support more accelerators.

hipudding and noemotiovon reacted with thumbs up emoji

@hipuddinghipuddingforce-pushed thecupystream branch 3 times, most recently fromedbc7a2 toc13c2bfCompareMarch 28, 2025 03:55
@hipuddinghipudding marked this pull request as ready for reviewMarch 28, 2025 03:55
@hipudding
Copy link
Author

@ruisearch42 Hello, this PR is now ready for review. According to the issue#51574 description, this PR only includes modifications for multi-backend support and has temporarily removed the HCCL-related logic. The HCCL logic will be implemented in a separate PR. My colleague has already completed the review and fixed the review comments.
Could you please review it again? Thanks.

ruisearch42 reacted with thumbs up emoji

@ruisearch42ruisearch42 self-assigned thisMar 28, 2025
@ruisearch42
Copy link
Contributor

@ruisearch42 Hello, this PR is now ready for review. According to the issue#51574 description, this PR only includes modifications for multi-backend support and has temporarily removed the HCCL-related logic. The HCCL logic will be implemented in a separate PR. My colleague has already completed the review and fixed the review comments. Could you please review it again? Thanks.

Thanks. Will take a look!

hipudding reacted with thumbs up emoji

This commit introduces multi-device support in Compile Graph,extending beyond CUDA's NCCL to accommodate various accelerators.The key changes include:1. Removed dependency on `cupy.cuda.ExternalStream`, Replaced with   `torch.{device}.StreamContext` to support diverse hardware.2. Replaced hardcoded `torch.cuda.xxx` calls with `AcceleratorRuntime`   Enables automatic detection and invocation of device-specific   functions.Signed-off-by: hipudding <huafengchun@gmail.com>
@stephanie-wang
Copy link
Contributor

Thanks for this contribution! I'm still looking through the code, but regarding the API changes - can we make it so that you don't need to modify the Device enum to use a custom accelerator type? It would be better if we can only keep the NVIDIA backend in the main codebase and allow the user to provide a custom Communicator and AcceleratorRuntime. This would help maintainability of the project, as we also don't currently have a good way to test all different device types.

Copy link
Contributor

@ruisearch42ruisearch42 left a comment

Choose a reason for hiding this comment

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

Thanks for the PR. Please address Stephanie's comment as well.


Assumes that `CUDA_VISIBLE_DEVICES` is set and is a
superset of the `ray.get_gpu_ids()`.
class AcceleratorRuntime:
Copy link
Contributor

Choose a reason for hiding this comment

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

These abstractions should be put into a different file rather thanutils.py

stephanie-wang reacted with thumbs up emoji
(CudaRuntime, or CpuRuntime).
"""

if cls._instance is None:
Copy link
Contributor

Choose a reason for hiding this comment

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

This is not thread-safe. Need to think a bit more carefully if we need locking, or just add a comment.

Check if an accelerator communicator is available for the specified device.
If no device is provided, the function determines the default non-CPU device.
Args:
device (Optional[Union["torch.device", str]]):
Copy link
Contributor

Choose a reason for hiding this comment

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

arg does not match function signature

def has_communicator(self) -> bool:
"""
Check if an accelerator communicator is available for the specified device.
If no device is provided, the function determines the default non-CPU device.
Copy link
Contributor

Choose a reason for hiding this comment

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

Can you please rephrase this to be clearer?

"""
raise NotImplementedError

def get_device_context(self, device):
Copy link
Contributor

Choose a reason for hiding this comment

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

Can we have a type hint for the return value?

stephanie-wang reacted with thumbs up emoji
Comment on lines +182 to +184
def get_unique_id(self) -> str:
"""
Generates a unique identifier for communication purposes.
Copy link
Contributor

Choose a reason for hiding this comment

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

Why not call itgenerate_unique_id orcreate_unique_id?


def get_device_type(self) -> str:
"""
Retrieves the device type for the accelerator.
Copy link
Contributor

Choose a reason for hiding this comment

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

If this is device type from PyTorch, mention that

Comment on lines +83 to +90
def get_transport_name(self) -> str:
raise NotImplementedError

def recv_stream(self):
raise NotImplementedError

def send_stream(self):
raise NotImplementedError
Copy link
Contributor

Choose a reason for hiding this comment

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

you don't need these, right?
Also for other methods that already haveraise NotImplementedError in the base class

import torch


class CommunicatorInfoHolder(Communicator):
Copy link
Contributor

Choose a reason for hiding this comment

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

Can we improve the naming a bit? I'm thinking aboutCommunicatorMetadata, but it's still not accurate enough

Copy link
Contributor

Choose a reason for hiding this comment

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

Yes, let's call this something different. How about CommunicatorHandle to match ActorHandle?


Assumes that `CUDA_VISIBLE_DEVICES` is set and is a
superset of the `ray.get_gpu_ids()`.
class AcceleratorRuntime:
Copy link
Contributor

Choose a reason for hiding this comment

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

How aboutAcceleratorContext?

@hipudding
Copy link
Author

Thanks@stephanie-wang@ruisearch42. I will fix comment issue ASAP.

@hipudding
Copy link
Author

hipudding commentedApr 1, 2025
edited
Loading

Thanks for this contribution! I'm still looking through the code, but regarding the API changes - can we make it so that you don't need to modify the Device enum to use a custom accelerator type? It would be better if we can only keep the NVIDIA backend in the main codebase and allow the user to provide a custom Communicator and AcceleratorRuntime. This would help maintainability of the project, as we also don't currently have a good way to test all different device types.

Does this mean that the backend connects to new devices by registering Communicator and AcceleratorRuntime?

I have considered integrating via with_tensor_transport(custom_Communicator), but this requires specifying the list of participating actors when creating the Communicator, and still need provide custome Acclerator runtime functions.

However, I have a question to ask@stephanie-wang: where should the backend Communicator code be archived? Since this Communicator will only be used within Ray, would it be appropriate to place it in a separate folder (e.g., custom_communicators), so that user programs can import the required Communicator(or auto select)? The quality of these Communicators would be ensured by the backend vendors.

For Device class, we can set a unified custom backend name, such as customer_accelerator, and dynamically modify the device name when AcceleratorRuntime loads—just like PyTorch’s privateuse1 mechanism.

@hainesmichaelchainesmichaelc added the community-contributionContributed by the community labelApr 4, 2025
@stephanie-wang
Copy link
Contributor

Thanks for this contribution! I'm still looking through the code, but regarding the API changes - can we make it so that you don't need to modify the Device enum to use a custom accelerator type? It would be better if we can only keep the NVIDIA backend in the main codebase and allow the user to provide a custom Communicator and AcceleratorRuntime. This would help maintainability of the project, as we also don't currently have a good way to test all different device types.

Does this mean that the backend connects to new devices by registering Communicator and AcceleratorRuntime?

Yes, what I am imagining is to have some API call likeray.register_accelerator_runtime("xpu", XpuAcceleratorRuntime, XpuCommunicator). So that way custom AcceleratorRuntimes can live outside of the Ray codebase.

I have considered integrating via with_tensor_transport(custom_Communicator), but this requires specifying the list of participating actors when creating the Communicator, and still need provide custome Acclerator runtime functions.

However, I have a question to ask@stephanie-wang: where should the backend Communicator code be archived? Since this Communicator will only be used within Ray, would it be appropriate to place it in a separate folder (e.g., custom_communicators), so that user programs can import the required Communicator(or auto select)? The quality of these Communicators would be ensured by the backend vendors.

Ideally I would like to support CPU and CUDA runtimes out of the box and other Communicators can live outside of the Ray codebase for now. The problem with other accelerators is that right now we don't have a way to test them in CI.

For Device class, we can set a unified custom backend name, such as customer_accelerator, and dynamically modify the device name when AcceleratorRuntime loads—just like PyTorch’s privateuse1 mechanism.

Hmm I'm not sure if I understand this idea. Is it an alternative to theray.register_accelerator_runtime call? Can it support multiple AcceleratorRuntimes?

hipudding reacted with eyes emoji

Copy link
Contributor

@stephanie-wangstephanie-wang left a comment

Choose a reason for hiding this comment

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

Thanks for this contribution! At a high level, I think it makes a lot of sense to try to abstract out the AcceleratorRuntime.

However, I believe need some iteration on the API calls. It would be useful if you can write a short doc that summarizes all of the APIs that need to be specialized to a specific device and shows how the rest of the code could call them. I think we'll be able to iterate faster on such a doc instead of the code. Happy to discuss synchronously too.

Comment on lines -594 to -595
self._send_stream: Union["cp.cuda.Stream", nullcontext] = nullcontext()
self._recv_stream: Union["cp.cuda.Stream", nullcontext] = nullcontext()
Copy link
Contributor

Choose a reason for hiding this comment

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

Let's still keep a generic type to indicate that this is a context?

on when the future is resolved. If None, the current stream is used.
"""
import cupy as cp
from ray.experimental.channel.utils import AcceleratorRuntime
Copy link
Contributor

Choose a reason for hiding this comment

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

Let's import at top-level. Importingcupy here was only necessary because we don't want a hard dependency on it.

if self._event is None:
return

cp.cuda.runtime.eventDestroy(self._event.ptr)
Copy link
Contributor

Choose a reason for hiding this comment

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

Can you bring this code back? Without this, the CUDA event can get destroyed before the stream and it may segfault.

Comment on lines +137 to +138
Check if an accelerator communicator is available for the specified device.
If no device is provided, the function determines the default non-CPU device.
Copy link
Contributor

Choose a reason for hiding this comment

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

I am a bit confused about what this method is supposed to do. Why does it always return True for CUDA and False for CPU?

"""
raise NotImplementedError

def is_available(self) -> bool:
Copy link
Contributor

Choose a reason for hiding this comment

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

It is a little confusing that the runtime can be created but not available. I guess this is meant to return whether the device is available for individual actors? Does it ever return anything different fromlen(get_devices()) > 0?

"""
raise NotImplementedError

def get_unique_id(self) -> str:
Copy link
Contributor

Choose a reason for hiding this comment

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

Suggested change
defget_unique_id(self)->str:
defgenerate_communicator_id(self)->str:

Copy link
Contributor

Choose a reason for hiding this comment

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

(Let's put communicator in the name)

import torch


class CommunicatorInfoHolder(Communicator):
Copy link
Contributor

Choose a reason for hiding this comment

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

Yes, let's call this something different. How about CommunicatorHandle to match ActorHandle?

Comment on lines +12 to +23
The driver maintains a communicator instance but does not participate
in the group. However, it still needs to store certain information,
such as world size and actor details.

Since the driver may not have a GPU (or other accelerators), it could
mistakenly select the wrong `TorchDeviceManager`
(e.g., `CPUTorchDeviceManager`), causing `get_communicator`
to return `None`.

Because the driver does not actively join the group and only uses the
communicator for metadata storage, Introduced a hardware-independent
class specifically for the driver.
Copy link
Contributor

Choose a reason for hiding this comment

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

This comment is written in a bit of a confusing way because it is trying to justify why we need this class, so it only makes sense in the context of this PR.

Instead, can you just write what this class is responsible for?

def get_self_rank(self) -> Optional[int]:
return None

def send(self, tensor: "torch.Tensor", peer_rank: int):
Copy link
Contributor

Choose a reason for hiding this comment

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

Should we remove these? They seem unnecessary from the driver's perspective?

return "cuda"


class CpuRuntime(AcceleratorRuntime):
Copy link
Contributor

Choose a reason for hiding this comment

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

I am a bit confused how the CpuRuntime fits in? Is it meant to be used as a default runtime? What happens if an actor requests 1 CPU and 1 GPU, will it have both a CpuRuntime and a CudaRuntime?

@hipudding
Copy link
Author

hipudding commentedApr 7, 2025
edited
Loading

Thanks@stephanie-wang, I will update this PR soon.

I will first write a document describing the overall work of the PR and the integration method for third-party backends. We’ll discuss based on the document first, and then move on to the code implementation.

@hipudding
Copy link
Author

@stephanie-wang@ruisearch42 Good day. Here's thedoc of this feature, feel free to add any comments on it.

Sign up for freeto join this conversation on GitHub. Already have an account?Sign in to comment
Reviewers

@stephanie-wangstephanie-wangstephanie-wang left review comments

@noemotiovonnoemotiovonnoemotiovon approved these changes

@ruisearch42ruisearch42ruisearch42 left review comments

At least 1 approving review is required to merge this pull request.

Labels
community-contributionContributed by the communitycoreIssues that should be addressed in Ray Core
Projects
None yet
Milestone
No milestone
Development

Successfully merging this pull request may close these issues.

8 participants
@hipudding@ruisearch42@noemotiovon@Bye-legumes@liuxsh9@stephanie-wang@hainesmichaelc@jcotant1

[8]ページ先頭

©2009-2025 Movatter.jp