From 18ea0e2f3109ea00b719e1ddbb0c93061ea8b8d9 Mon Sep 17 00:00:00 2001 From: Hao Zhang Date: Tue, 24 Nov 2020 23:33:20 -0500 Subject: [PATCH 01/51] scaffold of the code --- python/ray/util/collective/__init__.py | 1 + python/ray/util/collective/collective.py | 127 ++++++++++++++++++ .../collective/collective_group/__init__.py | 2 + .../collective_group/base_collective_group.py | 4 + .../collective_group/mpi_collective_group.py | 6 + .../collective_group/nccl_collective_group.py | 9 ++ .../collective/collective_group/nccl_util.py | 5 + python/ray/util/collective/requirements.txt | 1 + .../collective/scratch/test_named_actor.py | 37 +++++ 9 files changed, 192 insertions(+) create mode 100644 python/ray/util/collective/__init__.py create mode 100644 python/ray/util/collective/collective.py create mode 100644 python/ray/util/collective/collective_group/__init__.py create mode 100644 python/ray/util/collective/collective_group/base_collective_group.py create mode 100644 python/ray/util/collective/collective_group/mpi_collective_group.py create mode 100644 python/ray/util/collective/collective_group/nccl_collective_group.py create mode 100644 python/ray/util/collective/collective_group/nccl_util.py create mode 100644 python/ray/util/collective/requirements.txt create mode 100644 python/ray/util/collective/scratch/test_named_actor.py diff --git a/python/ray/util/collective/__init__.py b/python/ray/util/collective/__init__.py new file mode 100644 index 000000000000..92744a8b85e9 --- /dev/null +++ b/python/ray/util/collective/__init__.py @@ -0,0 +1 @@ +from collective.collective_group.base_collective_group import * \ No newline at end of file diff --git a/python/ray/util/collective/collective.py b/python/ray/util/collective/collective.py new file mode 100644 index 000000000000..aba512b62db0 --- /dev/null +++ b/python/ray/util/collective/collective.py @@ -0,0 +1,127 @@ +"""APIs exposed under the namespace ray.util.collective.""" +import ray + +# Get the availability information first by importing information +_MPI_AVAILABLE = True +_NCCL_AVAILABLE = True + +try: + from collective.collectivegoup.mpi_collective_group import MPICollectiveGroup +except ImportError: + _MPI_AVAILABLE = False + +try: + from collective.collectivegoup.nccl_collective_group import NCCLCollectiveGroup +except ImportError: + _NCCL_AVAILABLE = False + + +def nccl_available(): + return _NCCL_AVAILABLE + +def mpi_available(): + return _MPI_AVAILABLE + + +@ray.remote +class NCCLUniqueIDStore(object): + """NCCLUniqueID. This class should be used as a named actor.""" + def __init__(self): + self.nccl_id = None + + def set_id(self, id): + self.nccl_id = id + + def get_id(self): + return self.nccl_id + + +class GroupManager(object): + """ + Use this class to manage the collective groups we created so far; + + """ + def __init__(self): + """Put some necessary meta information here.""" + self._default_group = None + + def get_default_group(self): + pass + + def set_default_group(self): + pass + + def create_collective_group(self, + backend, + group_name, + world_size, + rank): + """ + The only entry to create new collective groups, construct CollectiveGroup here. + + Put the registration and the group information into the manager metadata as well. + """ + pass + + def destroy_collective_group(self, name): + """Group destructor.""" + pass + +_group_mgr = GroupManager() + + +# Frontend API #1: +# This API is supposed to work within the actor or task program: +# See the RFC for an example. +def init_collective_group(backend, + group_name='default', + world_size=-1, + rank=-1): + # do some check on the validaty of the arguments. + # see: https://github.com/pytorch/pytorch/blob/master/torch/distributed/distributed_c10d.py + if backend == 'mpi': + if not mpi_available(): + raise RuntimeError() + raise NotImplementedError() + elif backend == 'nccl': + if not nccl_available(): + raise RuntimeError() + + global _group_mgr + _group_mgr.create_collective_group(group_name, world_size, rank) + + +# Frontend API #2: +# This API is supported to work in the driver program - the users declare a list of actors as a collective group +# @Dacheng: This API is not in the right shape, need to work with ray.remote(), please figure out. +def declare_collective_group(actors, group_options): + pass + + +#collective APIs: +def send(tensor, dst, group_name='default'): + pass + +def recv(tensor, src, group_name='default'): + pass + +def broadcast(tensor, src, group_name='default'): + pass + +def allreduce(tensor, op=SUM, group_name='default'): + pass + +def reduce(tensor, dst, op=SUM, group_name='default'): + pass + +def allgather(tensor_list, tensor, gropu_name='default'): + pass + +def gather(tensor, gather_list=None, dst=0, group_name='default'): + pass + +def scatter(tensor, scatter_list=None, src=0, group_name='default'): + pass + +def barrier(group_name='default'): + pass diff --git a/python/ray/util/collective/collective_group/__init__.py b/python/ray/util/collective/collective_group/__init__.py new file mode 100644 index 000000000000..0663625d56ab --- /dev/null +++ b/python/ray/util/collective/collective_group/__init__.py @@ -0,0 +1,2 @@ +from .nccl_collective_group import NCCLGroup +from .mpi_collective_group import MPIGroup diff --git a/python/ray/util/collective/collective_group/base_collective_group.py b/python/ray/util/collective/collective_group/base_collective_group.py new file mode 100644 index 000000000000..cb43bd503a5a --- /dev/null +++ b/python/ray/util/collective/collective_group/base_collective_group.py @@ -0,0 +1,4 @@ +"""Abstract class for collective groups.""" +class BaseGroup: + def __init__(self, world_size, rank, group_name): + pass diff --git a/python/ray/util/collective/collective_group/mpi_collective_group.py b/python/ray/util/collective/collective_group/mpi_collective_group.py new file mode 100644 index 000000000000..b16f2ac22673 --- /dev/null +++ b/python/ray/util/collective/collective_group/mpi_collective_group.py @@ -0,0 +1,6 @@ +from collective.collective_group.base_collective_group import BaseGroup + +# TODO(Dacheng): implement this +class MPIGroup(BaseGroup): + def __init__(self, world_size, rank, group_name): + BaseGroup.__init__(self, world_size, rank, group_name) diff --git a/python/ray/util/collective/collective_group/nccl_collective_group.py b/python/ray/util/collective/collective_group/nccl_collective_group.py new file mode 100644 index 000000000000..5e88bd956780 --- /dev/null +++ b/python/ray/util/collective/collective_group/nccl_collective_group.py @@ -0,0 +1,9 @@ +from collective.collective_group.nccl_util import * +from collective.collective_group.base_collective_group import BaseGroup + +# TODO(Hao): implement this +class NCCLGroup(BaseGroup): + def __init__(self, world_size, rank, group_name): + BaseGroup.__init__(self, world_size, rank, group_name) + + def diff --git a/python/ray/util/collective/collective_group/nccl_util.py b/python/ray/util/collective/collective_group/nccl_util.py new file mode 100644 index 000000000000..065ad0fc7471 --- /dev/null +++ b/python/ray/util/collective/collective_group/nccl_util.py @@ -0,0 +1,5 @@ +try: + from cupy.cuda.nccl import get_unique_id + from cupy.cuda.nccl import get_version +except ImportError: + raise diff --git a/python/ray/util/collective/requirements.txt b/python/ray/util/collective/requirements.txt new file mode 100644 index 000000000000..ce5057b221f1 --- /dev/null +++ b/python/ray/util/collective/requirements.txt @@ -0,0 +1 @@ +cupy-cuda100 \ No newline at end of file diff --git a/python/ray/util/collective/scratch/test_named_actor.py b/python/ray/util/collective/scratch/test_named_actor.py new file mode 100644 index 000000000000..555e2ce611fd --- /dev/null +++ b/python/ray/util/collective/scratch/test_named_actor.py @@ -0,0 +1,37 @@ +import ray + +@ray.remote +class NamedActor: + def __init__(self): + self.id = '' + + def set_id(self, id): + self.id = id + return self.id + + def get_id(self): + return self.id + + +@ray.remote +class Worker: + def __init__(self, rank): + self.rank = rank + + def do_work(self): + if self.rank == 0: + id = '123' + ac = NamedActor.options(name='uniquename', lifetime="detached").remote() + ray.wait([ac.set_id.remote('123')]) + else: + ac = ray.get_actor('uniquename') + id = ray.get(ac.get_id.remote()) + print(id) + return id + + +ray.init() + +workers = [Worker.remote(rank) for rank in range(4)] +ret = ray.get(workers[0].do_work.remote()) +m = ray.get([workers[i].do_work.remote() for i in range(1, 4)]) From bf1051c9499e4cd7c5ec2792bf9612d0e2cb380c Mon Sep 17 00:00:00 2001 From: YLJALDC Date: Thu, 26 Nov 2020 01:43:27 -0500 Subject: [PATCH 02/51] some scratch and options change --- python/ray/actor.py | 18 +++-- python/ray/util/collective/collective.py | 23 +++--- .../ray/util/collective/scratch/test_nccl.py | 81 +++++++++++++++++++ .../util/collective/scratch/test_options.py | 14 ++++ 4 files changed, 121 insertions(+), 15 deletions(-) create mode 100644 python/ray/util/collective/scratch/test_nccl.py create mode 100644 python/ray/util/collective/scratch/test_options.py diff --git a/python/ray/actor.py b/python/ray/actor.py index d3fa34ff8b84..1a9531e3aec1 100644 --- a/python/ray/actor.py +++ b/python/ray/actor.py @@ -419,7 +419,8 @@ def options(self, placement_group=None, placement_group_bundle_index=-1, placement_group_capture_child_tasks=None, - override_environment_variables=None): + override_environment_variables=None, + collective=None): """Configures and overrides the actor instantiation parameters. The arguments are the same as those that can be passed @@ -461,7 +462,8 @@ def remote(self, *args, **kwargs): placement_group_capture_child_tasks=( placement_group_capture_child_tasks), override_environment_variables=( - override_environment_variables)) + override_environment_variables), + collective=collective) return ActorOptionWrapper() @@ -482,7 +484,8 @@ def _remote(self, placement_group=None, placement_group_bundle_index=-1, placement_group_capture_child_tasks=None, - override_environment_variables=None): + override_environment_variables=None, + collective=None): """Create an actor. This method allows more flexibility than the remote method because @@ -522,6 +525,7 @@ def _remote(self, override_environment_variables: Environment variables to override and/or introduce for this actor. This is a dictionary mapping variable names to their values. + collective: what colletive configuration to use Returns: A handle to the newly created actor. @@ -681,7 +685,8 @@ def _remote(self, actor_method_cpu, meta.actor_creation_function_descriptor, worker.current_session_and_job, - original_handle=True) + original_handle=True, + collective=collective) return actor_handle @@ -725,7 +730,8 @@ def __init__(self, actor_method_cpus, actor_creation_function_descriptor, session_and_job, - original_handle=False): + original_handle=False, + collective=None): self._ray_actor_language = language self._ray_actor_id = actor_id self._ray_original_handle = original_handle @@ -738,7 +744,7 @@ def __init__(self, self._ray_actor_creation_function_descriptor = \ actor_creation_function_descriptor self._ray_function_descriptor = {} - + self._collective = collective if not self._ray_is_cross_language: assert isinstance(actor_creation_function_descriptor, PythonFunctionDescriptor) diff --git a/python/ray/util/collective/collective.py b/python/ray/util/collective/collective.py index aba512b62db0..dfbd5d33e03b 100644 --- a/python/ray/util/collective/collective.py +++ b/python/ray/util/collective/collective.py @@ -6,12 +6,12 @@ _NCCL_AVAILABLE = True try: - from collective.collectivegoup.mpi_collective_group import MPICollectiveGroup + from collective.collectivegoup.mpi_collective_group import MPIGroup except ImportError: _MPI_AVAILABLE = False try: - from collective.collectivegoup.nccl_collective_group import NCCLCollectiveGroup + from collective.collectivegoup.nccl_collective_group import NCCLGroup except ImportError: _NCCL_AVAILABLE = False @@ -22,6 +22,14 @@ def nccl_available(): def mpi_available(): return _MPI_AVAILABLE +def backend_check(backend): + if backend == 'mpi': + if not mpi_available(): + raise RuntimeError() + raise NotImplementedError() + elif backend == 'nccl': + if not nccl_available(): + raise RuntimeError() @ray.remote class NCCLUniqueIDStore(object): @@ -79,13 +87,7 @@ def init_collective_group(backend, rank=-1): # do some check on the validaty of the arguments. # see: https://github.com/pytorch/pytorch/blob/master/torch/distributed/distributed_c10d.py - if backend == 'mpi': - if not mpi_available(): - raise RuntimeError() - raise NotImplementedError() - elif backend == 'nccl': - if not nccl_available(): - raise RuntimeError() + backend_check(backend) global _group_mgr _group_mgr.create_collective_group(group_name, world_size, rank) @@ -95,6 +97,9 @@ def init_collective_group(backend, # This API is supported to work in the driver program - the users declare a list of actors as a collective group # @Dacheng: This API is not in the right shape, need to work with ray.remote(), please figure out. def declare_collective_group(actors, group_options): + backend = group_options["backend"] + backend_check(backend) + uid = None pass diff --git a/python/ray/util/collective/scratch/test_nccl.py b/python/ray/util/collective/scratch/test_nccl.py new file mode 100644 index 000000000000..b903308cf6c9 --- /dev/null +++ b/python/ray/util/collective/scratch/test_nccl.py @@ -0,0 +1,81 @@ +import ray +import cupy.cuda.nccl as nccl +from collections import defaultdict +import cupy as cp + +@ray.remote(num_gpus=0.5) +class Actor: + def __init(self): + cp.cuda.Stream.null.synchronize() + self.send = cp.ones((10,), dtype=cp.float32) + self.recv = cp.zeros((10,), dtype=cp.float32) + cp.cuda.Stream.null.synchronize() + + def send_getter(self): + return self.send + + def send_setter(self, val): + self.send = val + + def recv_getter(self): + return self.recv + + def recv_setter(self, val): + self.recv = val + + def compute(self): + pass + +class GroupManager(Object): + def __init__(self): + self.group = defaultdict([]) + + def create_colletive_group(self, + backend, + group_name, + world_size, + rank, + actor, + uid=None): + + self.group[group_name].append({actor: nccl.NcclCommunicator(world_size, uid, rank)}) + + + +def declare_collective_group(actors, group_options): + # sort actors by rank + ranks = group_options["rank"] + if len(actors) != len(ranks) or len(actors) != group_options["world_size"]: + raise ValueError() + + #sorted_actors = [x for _,x in sorted(zip(ranks, actors)), key=lambda pair: pair[0]] + uid = nccl.get_unique_id() + for i in range(0, len(ranks): + _group_mgr.create_collective_group(group_options["backend"], + group_options["name"], + group_options["world_size"], + ranks, + actors[i], + uid) + +def allreduce(group_name): + for (actor, comm) in _group_mgr.group[group_name]: + dummy = self.recv = cp.zeros((10,), dtype=cp.float32).ptr + comm.allReduce(ray.get(actor.send_getter()).ptr, dummy, 10, cp.cuda.nccl.NCCL_FLOAT32, 1, cp.cuda.Stream.null.ptr) + actor.set_recv(dummy) + cp.cuda.Stream.null.synchronize() + +_group_mgr = GroupManager() + +group_options = {"name" : "haha", + "backend" : "nccl", + "world_size" : 4, + "rank" : [0,1,2,3]} + +actors = [Actor().remote() for i in range(4)] +declare_collective_group(actors, group_options}) + +allreduce("haha") + +for i in range(4): + print(ray.get(actors[i].recv_getter())) diff --git a/python/ray/util/collective/scratch/test_options.py b/python/ray/util/collective/scratch/test_options.py new file mode 100644 index 000000000000..5d99eed91b29 --- /dev/null +++ b/python/ray/util/collective/scratch/test_options.py @@ -0,0 +1,14 @@ +import ray +ray.init() + +@ray.remote(num_gpus=1) +class Actor: + def __init__(self): + pass + + def compute(self): + pass + +# might work? +worker = Actor.options(collective={})# .remote() +print(worker._collective) From 3c5628aaaff32e696438f8f73a3aff5857f557f8 Mon Sep 17 00:00:00 2001 From: Hao Zhang Date: Mon, 30 Nov 2020 23:23:41 -0500 Subject: [PATCH 03/51] NCCL mostly done, supporting API#1 --- python/ray/util/collective/collective.py | 197 +++++++++++++----- .../collective_group/base_collective_group.py | 79 ++++++- .../collective_group/mpi_collective_group.py | 11 +- .../collective_group/nccl_collective_group.py | 125 ++++++++++- .../collective/collective_group/nccl_util.py | 109 +++++++++- .../util/collective/tests/test_nccl_group.py | 0 python/ray/util/collective/types.py | 57 +++++ 7 files changed, 515 insertions(+), 63 deletions(-) create mode 100644 python/ray/util/collective/tests/test_nccl_group.py create mode 100644 python/ray/util/collective/types.py diff --git a/python/ray/util/collective/collective.py b/python/ray/util/collective/collective.py index dfbd5d33e03b..829cec6a142a 100644 --- a/python/ray/util/collective/collective.py +++ b/python/ray/util/collective/collective.py @@ -1,5 +1,8 @@ """APIs exposed under the namespace ray.util.collective.""" +import logging + import ray +from python.ray.util.collective import types # Get the availability information first by importing information _MPI_AVAILABLE = True @@ -19,10 +22,12 @@ def nccl_available(): return _NCCL_AVAILABLE + def mpi_available(): return _MPI_AVAILABLE -def backend_check(backend): + +def _backend_check(backend): if backend == 'mpi': if not mpi_available(): raise RuntimeError() @@ -31,16 +36,21 @@ def backend_check(backend): if not nccl_available(): raise RuntimeError() + @ray.remote class NCCLUniqueIDStore(object): """NCCLUniqueID. This class should be used as a named actor.""" - def __init__(self): + def __init__(self, name): + self.name = name self.nccl_id = None - def set_id(self, id): - self.nccl_id = id + def set_id(self, uid): + self.nccl_id = uid + return self.nccl_id def get_id(self): + if not self.nccl_id: + logging.warning('The NCCL ID has not been set yet for store {}'.format(self.name)) return self.nccl_id @@ -51,82 +61,167 @@ class GroupManager(object): """ def __init__(self): """Put some necessary meta information here.""" - self._default_group = None - - def get_default_group(self): - pass - - def set_default_group(self): - pass + self._name_group_map = {} + self._group_name_map = {} def create_collective_group(self, backend, - group_name, world_size, - rank): + rank, + group_name): """ - The only entry to create new collective groups, construct CollectiveGroup here. + The only entry to create new collective groups and register to the manager. Put the registration and the group information into the manager metadata as well. """ - pass - - def destroy_collective_group(self, name): + if backend == 'mpi': + raise NotImplementedError() + elif backend == 'nccl': + # create the ncclUniqueID + if rank == 0: + import cupy.cuda.nccl as nccl + group_uid = nccl.get_unique_id() + store_name = group_name + types.named_actor_suffix + store = NCCLUniqueIDStore\ + .options(name=store_name, lifetime="detached")\ + .remote() + ray.wait([store.set_id.remote(group_uid)]) + + g = NCCLGroup(world_size, rank, group_name) + self._name_group_map[group_name] = g + self._group_name_map[g] = group_name + return self._name_group_map[group_name] + + def is_group_exist(self, group_name): + if group_name in self._name_group_map: + return True + return False + + def get_group_by_name(self, group_name): + """Get the collective group handle by its name.""" + if group_name not in self._name_group_map: + return None + return self._name_group_map[group_name] + + def destroy_collective_group(self, group_name): """Group destructor.""" - pass + if group_name not in self._name_group_map: + logging.warning('The group {} does not exist'.format(group_name)) + return + + # release the collective group resource + g = self._name_group_map[group_name] + + rank = g.rank + backend = g.backend() + + # clean up the dicts + del self._group_name_map[g] + del self._name_group_map[group_name] + + if backend == 'nccl': + # release the named actor + if rank == 0: + store_name = group_name + types.named_actor_suffix + store = ray.get_actor(store_name) + ray.wait([store.__ray_terminate__.remote()]) + ray.kill(store) + g.destroy() _group_mgr = GroupManager() -# Frontend API #1: -# This API is supposed to work within the actor or task program: -# See the RFC for an example. def init_collective_group(backend, - group_name='default', - world_size=-1, - rank=-1): - # do some check on the validaty of the arguments. - # see: https://github.com/pytorch/pytorch/blob/master/torch/distributed/distributed_c10d.py - backend_check(backend) + world_size, + rank, + group_name='default'): + """ + Initialize a collective group inside an actor process. + + This is an + Args: + backend: + world_size: + rank: + group_name: + Returns: + + """ + _backend_check(backend) global _group_mgr - _group_mgr.create_collective_group(group_name, world_size, rank) + # TODO(Hao): implement a group auto-counter. + if not group_name: + raise ValueError('group_name: {}, needs to be a string.'.format(group_name)) + + if _group_mgr.is_group_exist(group_name): + raise RuntimeError('Trying to initialize a gropu twice.') + + assert(world_size > 0) + assert(rank >= 0 and rank < world_size) + + _group_mgr.create_collective_group(backend, world_size, rank, group_name) -# Frontend API #2: -# This API is supported to work in the driver program - the users declare a list of actors as a collective group -# @Dacheng: This API is not in the right shape, need to work with ray.remote(), please figure out. def declare_collective_group(actors, group_options): + """ + # Frontend API #2: + # This API is supported to work in the driver program - the users declare a list of actors as a collective group + # @Dacheng: This API is not in the right shape, need to work with ray.remote(), please figure out. + Args: + actors: + group_options: + + Returns: + + """ backend = group_options["backend"] - backend_check(backend) + _backend_check(backend) uid = None pass -#collective APIs: -def send(tensor, dst, group_name='default'): - pass +def allreduce(tensor, + group_name, + op=types.ReduceOp.SUM): + """ + Collective allreduce the tensor across the group with name group_name. -def recv(tensor, src, group_name='default'): - pass + Args: + tensor: + group_name (string): + op: -def broadcast(tensor, src, group_name='default'): - pass + Returns: + None + """ + g = _check_and_get_group(group_name) + opts = types.AllReduceOptions + opts.reduceOp = op + g.allreduce(tensor, opts) -def allreduce(tensor, op=SUM, group_name='default'): - pass -def reduce(tensor, dst, op=SUM, group_name='default'): - pass +def barrier(group_name): + """ + Barrier all collective process in the group with name group_name. -def allgather(tensor_list, tensor, gropu_name='default'): - pass + Args: + group_name (string): -def gather(tensor, gather_list=None, dst=0, group_name='default'): - pass + Returns: + None + """ + g = _check_and_get_group(group_name) + g.barrier() + + +def _check_and_get_group(group_name): + """Check the existence and return the group handle.""" + global _group_mgr + if not _group_mgr.is_group_exist(group_name): + raise ValueError('The collective group {} is not initialized.'.format(group_name)) + # TODO(Hao): check if this rank is in the group. + g = _group_mgr.get_group_by_name(group_name) + return g -def scatter(tensor, scatter_list=None, src=0, group_name='default'): - pass -def barrier(group_name='default'): - pass diff --git a/python/ray/util/collective/collective_group/base_collective_group.py b/python/ray/util/collective/collective_group/base_collective_group.py index cb43bd503a5a..c5717b4a1776 100644 --- a/python/ray/util/collective/collective_group/base_collective_group.py +++ b/python/ray/util/collective/collective_group/base_collective_group.py @@ -1,4 +1,81 @@ """Abstract class for collective groups.""" -class BaseGroup: +from abc import ABCMeta +from abc import abstractmethod +import six + +from python.ray.util.collective.types import AllReduceOptions, BarrierOptions + + +class BaseGroup(six.with_metaclass(ABCMeta)): def __init__(self, world_size, rank, group_name): + """ + Init the process group with basic information. + + Args: + world_size: + rank: + group_name: + """ + self._world_size = world_size + self._rank = rank + self._group_name = group_name + + @property + def rank(self): + """Return the rank of the current process.""" + return self._rank + + @property + def world_size(self): + """Return the number of processes in this group.""" + return self._world_size + + def destroy_group(self): + """GC the communicators.""" pass + + @abstractmethod + @classmethod + def backend(cls): + """The backend of this collective group.""" + raise NotImplementedError() + + @abstractmethod + def allreduce(self, tensor, allreduce_options=AllReduceOptions()): + raise NotImplementedError() + + @abstractmethod + def barrier(self, barrier_options=BarrierOptions()): + raise NotImplementedError() + + # @abstractmethod + # def reduce(self, tensor, dst, op=SUM, group_name='default'): + # raise NotImplementedError() + # + # @abstractmethod + # def allgather(self, tensor_list, tensor, gropu_name='default'): + # raise NotImplementedError() + # + # @abstractmethod + # def gather(self, tensor, gather_list=None, dst=0, group_name='default'): + # raise NotImplementedError() + # + # @abstractmethod + # def scatter(self, tensor, scatter_list=None, src=0, group_name='default'): + # raise NotImplementedError() + # + # @abstractmethod + # def barrier(self, group_name='default'): + # raise NotImplementedError() + + # @abstractmethod + # def send(self): + # raise NotImplementedError() + # + # @abstractmethod + # def recv(self): + # raise NotImplementedError() + # + # @abstractmethod + # def broadcast(self): + # raise NotImplementedError() diff --git a/python/ray/util/collective/collective_group/mpi_collective_group.py b/python/ray/util/collective/collective_group/mpi_collective_group.py index b16f2ac22673..58e9b7279e33 100644 --- a/python/ray/util/collective/collective_group/mpi_collective_group.py +++ b/python/ray/util/collective/collective_group/mpi_collective_group.py @@ -1,6 +1,7 @@ -from collective.collective_group.base_collective_group import BaseGroup +"""Implementation of the MPI collective group.""" +try: + import mpi4py +except ImportError: + raise -# TODO(Dacheng): implement this -class MPIGroup(BaseGroup): - def __init__(self, world_size, rank, group_name): - BaseGroup.__init__(self, world_size, rank, group_name) +raise NotImplementedError() diff --git a/python/ray/util/collective/collective_group/nccl_collective_group.py b/python/ray/util/collective/collective_group/nccl_collective_group.py index 5e88bd956780..08c28733fcbc 100644 --- a/python/ray/util/collective/collective_group/nccl_collective_group.py +++ b/python/ray/util/collective/collective_group/nccl_collective_group.py @@ -1,9 +1,124 @@ -from collective.collective_group.nccl_util import * -from collective.collective_group.base_collective_group import BaseGroup +import logging +import ray +import cupy + +from python.ray.util.collective.collective_group import nccl_util +from python.ray.util.collective.collective_group.base_collective_group import BaseGroup +from python.ray.util.collective.types import AllReduceOptions, BarrierOptions, named_actor_suffix + + +# TODO(Hao): +# (1) stream management, instead of using the default stream, using a dedicate stream +# (2) communicator management, adding a caching mechanism to enable + -# TODO(Hao): implement this class NCCLGroup(BaseGroup): def __init__(self, world_size, rank, group_name): - BaseGroup.__init__(self, world_size, rank, group_name) + """Init an NCCL collective group.""" + super(NCCLGroup, self).__init__(self, world_size, rank, group_name) + self._nccl_uid_store = None + self._nccl_uid = None + + # TODO(Hao): change this to a be a cache + self._nccl_comm = None + + # Check NCCL version + if nccl_util.get_nccl_build_version() < 2000: + raise RuntimeError('NCCL in Ray requires NCCL>=2.0.') + + # TODO(Hao): check version here + if nccl_util.get_nccl_runtime_version() < 2704: + logging.warning('NCCL send/recv calls requires NCCL>=2.7.4') + + # Setup the nccl uid using the store + self._init_nccl_unique_id() + + # Setup a tensor for barrier calls + self._barrier_tensor = cupy.array([1]) + + def _init_nccl_unique_id(self): + """Init the NCCL unique ID required for setting up NCCL communicator.""" + # using group_name to query the UniqueIDActor + unique_actor_name = self.group_name + named_actor_suffix + + # Assuming this named actor has been created. + self._nccl_uid_store = ray.get_actor(unique_actor_name) + self._nccl_uid = ray.get(self._nccl_uid_store.get_id.remote()) + + @property + def nccl_uid(self): + return self._nccl_uid + + def destory_group(self): + """Destroy the group and release the NCCL communicators safely.""" + if self._nccl_comm is not None: + self.barrier() + # We also need a barrier call here. + stream = self._get_cuda_stream() + stream.synchronize() + # destroy the communicator + self._nccl_comm.destory() + self._nccl_comm = None + super(NCCLGroup, self).destroy_group() + + @classmethod + def backend(cls): + return 'nccl' + + def allreduce(self, tensor, allreduce_options=AllReduceOptions()): + """ + AllReduce a list of tensors following options. + + Args: + tensor: the tensor to be reduced, each tensor locates on a GPU + allreduce_options: + + Returns: + """ + # obtain the communicator + comm = self._get_nccl_communicator() + # obtain the stream: using default stream by now + # TODO(Hao): implement a simple stream manager here + stream = self._get_cuda_stream() + + dtype = nccl_util.get_nccl_tensor_dtype(tensor) + ptr = nccl_util.get_tensor_ptr(tensor) + n_elems = nccl_util.get_tensor_n_elements(tensor) + reduce_op = nccl_util.get_nccl_reduce_op(allreduce_options.reduceOp) + + # in-place allreduce + comm.allReduce(ptr, + ptr, + n_elems, + dtype, + reduce_op, + stream.ptr) + + def barrier(self, barrier_options=BarrierOptions()): + """ + Blocks until all processes in the communicator have reached this barrier call. + + Args: + barrier_options: + + Returns: + """ + self.allreduce(self._barrier_tensor) + + def _get_nccl_communicator(self): + """Create a new NCCL communicator for the collective task, or using a cached communicator.""" + # TODO(Hao): later change this to use device keys and query from cache. + # TODO(Hao): implement a thin wrapper + if not self._nccl_comm: + self._nccl_comm = nccl_util.create_nccl_communicator( + self.world_size, self.nccl_uid, self.rank) + return self._nccl_comm + + def _get_cuda_stream(self): + """Obtain an idle stream from a stream pool for the collective task.""" + # TODO: implement a simple stream manager. + return cupy.cuda.Stream.null - def + # def _collective_call(self, *args): + # """Private method to encapsulate all collective calls""" + # pass diff --git a/python/ray/util/collective/collective_group/nccl_util.py b/python/ray/util/collective/collective_group/nccl_util.py index 065ad0fc7471..67c595dee930 100644 --- a/python/ray/util/collective/collective_group/nccl_util.py +++ b/python/ray/util/collective/collective_group/nccl_util.py @@ -1,5 +1,112 @@ +"""Code to wrap some NCCL API calls.""" +import numpy try: + import cupy + from cupy.cuda import nccl from cupy.cuda.nccl import get_unique_id from cupy.cuda.nccl import get_version + from cupy.cuda.nccl import get_build_version + from cupy.cuda.nccl import NcclCommunicator + from cupy.cuda.nccl import NcclError except ImportError: - raise + raise ImportError('NCCL in Ray requires Cupy being available!') + +from python.ray.util.collective.types import ReduceOp, torch_available + + +NCCL_REDUCE_OP_MAP = { + ReduceOp.SUM: nccl.NCCL_SUM, + ReduceOp.PRODUCT: nccl.NCCL_PROD, + ReduceOp.MIN: nccl.NCCL_MIN, + ReduceOp.MAX: nccl.NCCL_MAX, +} + + +# cupy types are the same with numpy types +NUMPY_NCCL_DTYPE_MAP = { + numpy.bool: nccl.NCCL_UINT8, + numpy.float16: nccl.NCCL_FLOAT16, + numpy.float32: nccl.NCCL_FLOAT32, + numpy.float64: nccl.NCCL_FLOAT64, +} + +if torch_available(): + import torch + TORCH_NCCL_DTYPE_MAP = { + torch.bool: nccl.NCCL_UINT8, + torch.float16: nccl.NCCL_FLOAT16, + torch.float32: nccl.NCCL_FLOAT32, + torch.float64: nccl.NCCL_FLOAT64, + } + + +def get_nccl_build_version(): + return get_build_version() + + +def get_nccl_runtime_version(): + return get_build_version() + + +def create_nccl_communicator(world_size, nccl_unique_id, rank): + """ + Create an NCCL communicator using NCCL APIs. + + TODO(Hao): verify that the call has big overhead. + + Returns: + comm(type): an NCCL communicator. + + """ + # TODO(Hao): make this inside the NCCLComm class, and implement the abort method. + # Make it RAII + comm = NcclCommunicator(world_size, nccl_unique_id, rank) + return comm + + +def get_nccl_reduce_op(reduce_op): + """ + Map the reduce op to NCCL reduce op type. + + Returns: + Nccl_op (nccl.ncclRedOp_t) + """ + if reduce_op not in NCCL_REDUCE_OP_MAP: + raise RuntimeError('NCCL does not support ReduceOp: {}'.format(reduce_op)) + return NCCL_REDUCE_OP_MAP[reduce_op] + + +def get_nccl_tensor_dtype(tensor): + """Return the corresponded NCCL dtype given a tensor.""" + if isinstance(tensor, cupy.ndarray): + return NUMPY_NCCL_DTYPE_MAP[tensor.dtype] + if torch_available(): + if isinstance(tensor, torch.Tensor): + return TORCH_NCCL_DTYPE_MAP[tensor.dtype] + raise ValueError('Unsupported tensor type') + + +def get_tensor_ptr(tensor): + """Return the pointer to the underlying memory storage of a tensor.""" + if not tensor: + raise RuntimeError('None Tensor.') + if isinstance(tensor, cupy.ndarray): + return tensor.data.ptr + if isinstance(tensor, numpy.ndarray): + return tensor.data + if torch_available(): + if isinstance(tensor, torch.Tensor): + return tensor.data_ptr() + raise ValueError('Unsupported tensor type') + + +def get_tensor_n_elements(tensor): + """Return the number of elements in a tensor.""" + if not tensor: + raise RuntimeError('None Tensor.') + if isinstance(tensor, cupy.ndarray) or isinstance(tensor, numpy.ndarray): + return tensor.size + if torch_available(): + if isinstance(tensor, torch.Tensor): + return torch.numel(tensor) + raise ValueError('Unsupported tensor type') diff --git a/python/ray/util/collective/tests/test_nccl_group.py b/python/ray/util/collective/tests/test_nccl_group.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/python/ray/util/collective/types.py b/python/ray/util/collective/types.py new file mode 100644 index 000000000000..366f8f805c11 --- /dev/null +++ b/python/ray/util/collective/types.py @@ -0,0 +1,57 @@ +"""Types conversion between different backends.""" +from enum import Enum +from datetime import timedelta + +_NUMPY_AVAILABLE = True +_TORCH_AVAILABLE = True +_CUPY_AVAILABLE = True + +try: + import numpy +except ImportError: + _NUMPY_AVAILABLE = False + +try: + import torch +except ImportError: + _TORCH_AVAILABLE = False + +try: + import cupy +except ImportError: + _CUPY_AVAILABLE = False + + +def numpy_available(): + return _NUMPY_AVAILABLE + + +def cupy_available(): + return _CUPY_AVAILABLE + + +def torch_available(): + return _TORCH_AVAILABLE + + +# TODO(Hao): extend this to support more MPI types +class ReduceOp(Enum): + SUM = 0 + PRODUCT = 1 + MIN = 2 + MAX = 3 + + +unset_timeout = timedelta(milliseconds=-1) + + +class AllReduceOptions: + reduceOp = ReduceOp.SUM + timeout = unset_timeout + + +class BarrierOptions: + timeout = unset_timeout + + +named_actor_suffix = '_unique_id_actor' \ No newline at end of file From 0714c4a7804edd16cc812dd00eb56c42137eb5e1 Mon Sep 17 00:00:00 2001 From: Dacheng Li Date: Tue, 1 Dec 2020 01:46:26 -0500 Subject: [PATCH 04/51] interface 2.1 2.2 scratch --- python/ray/actor.py | 8 +- python/ray/util/__init__.py | 2 +- python/ray/util/collective/collective.py | 94 ++++++++++++++++++++++-- 3 files changed, 97 insertions(+), 7 deletions(-) diff --git a/python/ray/actor.py b/python/ray/actor.py index 1a9531e3aec1..d8c3ac9e1406 100644 --- a/python/ray/actor.py +++ b/python/ray/actor.py @@ -8,7 +8,7 @@ import ray.worker from ray.util.placement_group import ( PlacementGroup, check_placement_group_index, get_current_placement_group) - +from ray.util.collective import collective as col from ray import ActorClassID, Language from ray._raylet import PythonFunctionDescriptor from ray import cross_language @@ -675,6 +675,12 @@ def _remote(self, extension_data=str(actor_method_cpu), override_environment_variables=override_environment_variables or dict()) + + if collective: + col.init_collective_group(backend=collective["backend"], + world_size=collective["world_size"], + rank=collective["rank"], + group_name=collective["group_name"]) actor_handle = ActorHandle( meta.language, diff --git a/python/ray/util/__init__.py b/python/ray/util/__init__.py index 2a6d0a0290ec..e4a1e83c93de 100644 --- a/python/ray/util/__init__.py +++ b/python/ray/util/__init__.py @@ -9,5 +9,5 @@ __all__ = [ "ActorPool", "disable_log_once_globally", "enable_periodic_logging", "iter", "log_once", "pdb", "placement_group", "placement_group_table", - "remove_placement_group" + "remove_placement_group", "collective" ] diff --git a/python/ray/util/collective/collective.py b/python/ray/util/collective/collective.py index 829cec6a142a..17bcbd9f5e3b 100644 --- a/python/ray/util/collective/collective.py +++ b/python/ray/util/collective/collective.py @@ -128,8 +128,80 @@ def destroy_collective_group(self, group_name): ray.kill(store) g.destroy() -_group_mgr = GroupManager() +class GroupManager_2(object): + """ + Use this class to manage the collective groups we created so far; + For interface 2.2 + + """ + def __init__(self): + """Put some necessary meta information here.""" + self._name_group_map = {} + self._group_name_map = {} + + def create_collective_group(self, + backend, + world_size, + rank, + group_name): + """ + The only entry to create new collective groups and register to the manager. + + Put the registration and the group information into the manager metadata as well. + """ + if backend == 'mpi': + raise NotImplementedError() + elif backend == 'nccl': + # create the ncclUniqueID + #if rank == 0: + import cupy.cuda.nccl as nccl + group_uid = nccl.get_unique_id() + + for r in rank: + g = NCCLGroup(world_size, r, group_name) + self._name_group_map[group_name] = g + self._group_name_map[g] = group_name + return self._name_group_map[group_name] + + def is_group_exist(self, group_name): + if group_name in self._name_group_map: + return True + return False + + def get_group_by_name(self, group_name): + """Get the collective group handle by its name.""" + if group_name not in self._name_group_map: + return None + return self._name_group_map[group_name] + + def destroy_collective_group(self, group_name): + """Group destructor.""" + if group_name not in self._name_group_map: + logging.warning('The group {} does not exist'.format(group_name)) + return + + # release the collective group resource + g = self._name_group_map[group_name] + + rank = g.rank + backend = g.backend() + + # clean up the dicts + del self._group_name_map[g] + del self._name_group_map[group_name] + + if backend == 'nccl': + # release the named actor + if rank == 0: + store_name = group_name + types.named_actor_suffix + store = ray.get_actor(store_name) + ray.wait([store.__ray_terminate__.remote()]) + ray.kill(store) + g.destroy() + +_group_mgr = GroupManager() +_group_mgr2 = GroupMagager_2() def init_collective_group(backend, world_size, @@ -155,7 +227,7 @@ def init_collective_group(backend, raise ValueError('group_name: {}, needs to be a string.'.format(group_name)) if _group_mgr.is_group_exist(group_name): - raise RuntimeError('Trying to initialize a gropu twice.') + raise RuntimeError('Trying to initialize a group twice.') assert(world_size > 0) assert(rank >= 0 and rank < world_size) @@ -175,11 +247,23 @@ def declare_collective_group(actors, group_options): Returns: """ - backend = group_options["backend"] + global _group_mgr_2 + try: + group_name = group_options["group_name"] + world_size = group_options["world_size"] + rank = group_options["rank"] + backend = group_options["backend"] + except: + raise ValueError("group options incomplete") + _backend_check(backend) - uid = None - pass + if _group_mgr_2.is_group_exist(group_name): + raise RuntimeError('Trying to initialize a group twice.') + + assert(world_size > 0) + assert(rank >= 0 and rank < world_size) + _group_mgr_2.create_collective_group(backend, world_size, rank, group_name, actors) def allreduce(tensor, group_name, From 20df179f98118554a55cd6f095cfedfcfb8d7c76 Mon Sep 17 00:00:00 2001 From: Hao Zhang Date: Tue, 1 Dec 2020 02:41:22 -0500 Subject: [PATCH 05/51] put code into ray and fix some importing issues --- python/ray/util/collective/__init__.py | 2 +- python/ray/util/collective/collective.py | 15 ++++---- .../collective/collective_group/__init__.py | 1 - .../collective_group/base_collective_group.py | 8 +++-- .../collective_group/mpi_collective_group.py | 2 -- .../collective_group/nccl_collective_group.py | 10 +++--- .../collective/collective_group/nccl_util.py | 2 +- .../examples/nccl_allreduce_example.py | 35 +++++++++++++++++++ 8 files changed, 56 insertions(+), 19 deletions(-) create mode 100644 python/ray/util/collective/examples/nccl_allreduce_example.py diff --git a/python/ray/util/collective/__init__.py b/python/ray/util/collective/__init__.py index 92744a8b85e9..976ed700cbb8 100644 --- a/python/ray/util/collective/__init__.py +++ b/python/ray/util/collective/__init__.py @@ -1 +1 @@ -from collective.collective_group.base_collective_group import * \ No newline at end of file +from .collective import init_collective_group, allreduce, barrier \ No newline at end of file diff --git a/python/ray/util/collective/collective.py b/python/ray/util/collective/collective.py index 17bcbd9f5e3b..ef62695eecb9 100644 --- a/python/ray/util/collective/collective.py +++ b/python/ray/util/collective/collective.py @@ -2,19 +2,19 @@ import logging import ray -from python.ray.util.collective import types +from ray.util.collective import types # Get the availability information first by importing information _MPI_AVAILABLE = True _NCCL_AVAILABLE = True try: - from collective.collectivegoup.mpi_collective_group import MPIGroup + from ray.util.collective.collective_group.mpi_collective_group import MPIGroup except ImportError: _MPI_AVAILABLE = False try: - from collective.collectivegoup.nccl_collective_group import NCCLGroup + from ray.util.collective.collective_group.nccl_collective_group import NCCLGroup except ImportError: _NCCL_AVAILABLE = False @@ -82,11 +82,10 @@ def create_collective_group(self, import cupy.cuda.nccl as nccl group_uid = nccl.get_unique_id() store_name = group_name + types.named_actor_suffix - store = NCCLUniqueIDStore\ - .options(name=store_name, lifetime="detached")\ - .remote() + store = NCCLUniqueIDStore.options(name=store_name, lifetime="detached").remote() ray.wait([store.set_id.remote(group_uid)]) + logging.info('creating NCCL group: {}'.format(group_name)) g = NCCLGroup(world_size, rank, group_name) self._name_group_map[group_name] = g self._group_name_map[g] = group_name @@ -230,8 +229,8 @@ def init_collective_group(backend, raise RuntimeError('Trying to initialize a group twice.') assert(world_size > 0) - assert(rank >= 0 and rank < world_size) - + assert(rank >= 0 ) + assert(rank < world_size) _group_mgr.create_collective_group(backend, world_size, rank, group_name) diff --git a/python/ray/util/collective/collective_group/__init__.py b/python/ray/util/collective/collective_group/__init__.py index 0663625d56ab..b2b3ab3ee7ee 100644 --- a/python/ray/util/collective/collective_group/__init__.py +++ b/python/ray/util/collective/collective_group/__init__.py @@ -1,2 +1 @@ from .nccl_collective_group import NCCLGroup -from .mpi_collective_group import MPIGroup diff --git a/python/ray/util/collective/collective_group/base_collective_group.py b/python/ray/util/collective/collective_group/base_collective_group.py index c5717b4a1776..55835e6430cd 100644 --- a/python/ray/util/collective/collective_group/base_collective_group.py +++ b/python/ray/util/collective/collective_group/base_collective_group.py @@ -3,7 +3,7 @@ from abc import abstractmethod import six -from python.ray.util.collective.types import AllReduceOptions, BarrierOptions +from ray.util.collective.types import AllReduceOptions, BarrierOptions class BaseGroup(six.with_metaclass(ABCMeta)): @@ -30,11 +30,15 @@ def world_size(self): """Return the number of processes in this group.""" return self._world_size + @property + def group_name(self): + """Return the group name of this group.""" + return self._group_name + def destroy_group(self): """GC the communicators.""" pass - @abstractmethod @classmethod def backend(cls): """The backend of this collective group.""" diff --git a/python/ray/util/collective/collective_group/mpi_collective_group.py b/python/ray/util/collective/collective_group/mpi_collective_group.py index 58e9b7279e33..92e40ca7847b 100644 --- a/python/ray/util/collective/collective_group/mpi_collective_group.py +++ b/python/ray/util/collective/collective_group/mpi_collective_group.py @@ -3,5 +3,3 @@ import mpi4py except ImportError: raise - -raise NotImplementedError() diff --git a/python/ray/util/collective/collective_group/nccl_collective_group.py b/python/ray/util/collective/collective_group/nccl_collective_group.py index 08c28733fcbc..5df26979ade2 100644 --- a/python/ray/util/collective/collective_group/nccl_collective_group.py +++ b/python/ray/util/collective/collective_group/nccl_collective_group.py @@ -2,9 +2,9 @@ import ray import cupy -from python.ray.util.collective.collective_group import nccl_util -from python.ray.util.collective.collective_group.base_collective_group import BaseGroup -from python.ray.util.collective.types import AllReduceOptions, BarrierOptions, named_actor_suffix +from ray.util.collective.collective_group import nccl_util +from ray.util.collective.collective_group.base_collective_group import BaseGroup +from ray.util.collective.types import AllReduceOptions, BarrierOptions, named_actor_suffix # TODO(Hao): @@ -15,7 +15,7 @@ class NCCLGroup(BaseGroup): def __init__(self, world_size, rank, group_name): """Init an NCCL collective group.""" - super(NCCLGroup, self).__init__(self, world_size, rank, group_name) + super(NCCLGroup, self).__init__(world_size, rank, group_name) self._nccl_uid_store = None self._nccl_uid = None @@ -42,7 +42,9 @@ def _init_nccl_unique_id(self): unique_actor_name = self.group_name + named_actor_suffix # Assuming this named actor has been created. + print('reach here...1') self._nccl_uid_store = ray.get_actor(unique_actor_name) + print('reach here...1') self._nccl_uid = ray.get(self._nccl_uid_store.get_id.remote()) @property diff --git a/python/ray/util/collective/collective_group/nccl_util.py b/python/ray/util/collective/collective_group/nccl_util.py index 67c595dee930..eef3e2cf4a04 100644 --- a/python/ray/util/collective/collective_group/nccl_util.py +++ b/python/ray/util/collective/collective_group/nccl_util.py @@ -11,7 +11,7 @@ except ImportError: raise ImportError('NCCL in Ray requires Cupy being available!') -from python.ray.util.collective.types import ReduceOp, torch_available +from ray.util.collective.types import ReduceOp, torch_available NCCL_REDUCE_OP_MAP = { diff --git a/python/ray/util/collective/examples/nccl_allreduce_example.py b/python/ray/util/collective/examples/nccl_allreduce_example.py new file mode 100644 index 000000000000..8dc444d0c035 --- /dev/null +++ b/python/ray/util/collective/examples/nccl_allreduce_example.py @@ -0,0 +1,35 @@ + +import ray +import cupy as cp + +import ray.util.collective as collective + +@ray.remote(num_gpus=1) +class Worker: + def __init__(self): + self.send = cp.ones((4,), dtype=cp.float32) + self.recv = cp.zeros((4,), dtype=cp.float32) + + def setup(self, world_size, rank): + collective.init_collective_group('nccl', world_size, rank, 'default') + return True + + def compute(self): + collective.allreduce(self.send, 'default') + print(self.send) + return self.send + +if __name__ == "__main__": + ray.init(num_gpus=2) + + num_workers = 2 + workers = [] + init_rets = [] + for i in range(num_workers): + w = Worker.remote() + workers.append(w) + init_rets.append(w.setup.remote(num_workers, i)) + m = ray.get(init_rets) + # results = ray.get([w.compute.remote() for w in workers]) + # print(results) + ray.shutdown() \ No newline at end of file From 5267df1e94942aa5d208f59c52c9521f61c1d543 Mon Sep 17 00:00:00 2001 From: Hao Zhang Date: Tue, 1 Dec 2020 13:53:21 -0500 Subject: [PATCH 06/51] add an addtional Rendezvous class to safely meet at named actor --- python/ray/util/collective/collective.py | 12 ++-- .../collective_group/nccl_collective_group.py | 62 +++++++++++++++---- 2 files changed, 57 insertions(+), 17 deletions(-) diff --git a/python/ray/util/collective/collective.py b/python/ray/util/collective/collective.py index ef62695eecb9..bd4c63bc66bd 100644 --- a/python/ray/util/collective/collective.py +++ b/python/ray/util/collective/collective.py @@ -18,6 +18,7 @@ except ImportError: _NCCL_AVAILABLE = False +logging.getLogger().setLevel(logging.DEBUG) def nccl_available(): return _NCCL_AVAILABLE @@ -82,10 +83,11 @@ def create_collective_group(self, import cupy.cuda.nccl as nccl group_uid = nccl.get_unique_id() store_name = group_name + types.named_actor_suffix - store = NCCLUniqueIDStore.options(name=store_name, lifetime="detached").remote() + + store = NCCLUniqueIDStore.options(name=store_name, lifetime="detached").remote(store_name) ray.wait([store.set_id.remote(group_uid)]) - logging.info('creating NCCL group: {}'.format(group_name)) + logging.debug('creating NCCL group: {}'.format(group_name)) g = NCCLGroup(world_size, rank, group_name) self._name_group_map[group_name] = g self._group_name_map[g] = group_name @@ -199,8 +201,9 @@ def destroy_collective_group(self, group_name): ray.kill(store) g.destroy() + _group_mgr = GroupManager() -_group_mgr2 = GroupMagager_2() +_group_mgr2 = GroupManager_2() def init_collective_group(backend, world_size, @@ -264,6 +267,7 @@ def declare_collective_group(actors, group_options): _group_mgr_2.create_collective_group(backend, world_size, rank, group_name, actors) + def allreduce(tensor, group_name, op=types.ReduceOp.SUM): @@ -306,5 +310,3 @@ def _check_and_get_group(group_name): # TODO(Hao): check if this rank is in the group. g = _group_mgr.get_group_by_name(group_name) return g - - diff --git a/python/ray/util/collective/collective_group/nccl_collective_group.py b/python/ray/util/collective/collective_group/nccl_collective_group.py index 5df26979ade2..99253ba30054 100644 --- a/python/ray/util/collective/collective_group/nccl_collective_group.py +++ b/python/ray/util/collective/collective_group/nccl_collective_group.py @@ -1,4 +1,7 @@ import logging +import datetime +import time + import ray import cupy @@ -11,25 +14,66 @@ # (1) stream management, instead of using the default stream, using a dedicate stream # (2) communicator management, adding a caching mechanism to enable +class Rendezvous: + def __init__(self, group_name): + if not group_name: + raise ValueError('Empty meeting point.') + self._group_name = group_name + self._store_name = None + self._store = None + + def meet_at_store(self, timeout=180): + """Meet at the named actor store.""" + if timeout is not None and timeout < 0: + raise ValueError("The 'timeout' argument must be nonnegative. " + f"Received {timeout}") + self._store_name = self._group_name + named_actor_suffix + timeout_delta = datetime.timedelta(seconds=timeout) + elapsed = datetime.timedelta(seconds=0) + start_time = datetime.datetime.now() + while elapsed < timeout_delta: + try: + logging.debug("Trying to meet at the store '{}'".format(self._store_name)) + self._store = ray.get_actor(self._store_name) + except ValueError: + logging.debug("Failed to meet at the store '{}'." + "Trying again...".format(self._store_name)) + time.sleep(1) + elapsed = datetime.datetime.now() - start_time + continue + break + if not self._store: + raise RuntimeError("Unable to meet other processes " + "at the rendezvous store.") + + @property + def store(self): + return self._store + + def get_nccl_id(self): + if not self._store: + raise ValueError("Rendezvous store is not setup.") + uid = ray.get(self._store.get_id.remote()) + return uid class NCCLGroup(BaseGroup): def __init__(self, world_size, rank, group_name): """Init an NCCL collective group.""" super(NCCLGroup, self).__init__(world_size, rank, group_name) - self._nccl_uid_store = None self._nccl_uid = None # TODO(Hao): change this to a be a cache self._nccl_comm = None - # Check NCCL version if nccl_util.get_nccl_build_version() < 2000: raise RuntimeError('NCCL in Ray requires NCCL>=2.0.') - # TODO(Hao): check version here if nccl_util.get_nccl_runtime_version() < 2704: logging.warning('NCCL send/recv calls requires NCCL>=2.7.4') + self._rendezvous = Rendezvous(self.group_name) + self._rendezvous.meet_at_store() + # Setup the nccl uid using the store self._init_nccl_unique_id() @@ -38,20 +82,14 @@ def __init__(self, world_size, rank, group_name): def _init_nccl_unique_id(self): """Init the NCCL unique ID required for setting up NCCL communicator.""" - # using group_name to query the UniqueIDActor - unique_actor_name = self.group_name + named_actor_suffix + self._nccl_uid = self._rendezvous.get_nccl_id() - # Assuming this named actor has been created. - print('reach here...1') - self._nccl_uid_store = ray.get_actor(unique_actor_name) - print('reach here...1') - self._nccl_uid = ray.get(self._nccl_uid_store.get_id.remote()) @property def nccl_uid(self): return self._nccl_uid - def destory_group(self): + def destroy_group(self): """Destroy the group and release the NCCL communicators safely.""" if self._nccl_comm is not None: self.barrier() @@ -59,7 +97,7 @@ def destory_group(self): stream = self._get_cuda_stream() stream.synchronize() # destroy the communicator - self._nccl_comm.destory() + self._nccl_comm.destroy() self._nccl_comm = None super(NCCLGroup, self).destroy_group() From 8ff63ad5e27d119dc6599533f087a66e07dfe3d5 Mon Sep 17 00:00:00 2001 From: Hao Zhang Date: Tue, 1 Dec 2020 14:11:48 -0500 Subject: [PATCH 07/51] fix some small bugs in nccl_util --- .../ray/util/collective/collective_group/nccl_util.py | 6 +----- .../util/collective/examples/nccl_allreduce_example.py | 10 ++++++++-- 2 files changed, 9 insertions(+), 7 deletions(-) diff --git a/python/ray/util/collective/collective_group/nccl_util.py b/python/ray/util/collective/collective_group/nccl_util.py index eef3e2cf4a04..ec23218af9c1 100644 --- a/python/ray/util/collective/collective_group/nccl_util.py +++ b/python/ray/util/collective/collective_group/nccl_util.py @@ -79,7 +79,7 @@ def get_nccl_reduce_op(reduce_op): def get_nccl_tensor_dtype(tensor): """Return the corresponded NCCL dtype given a tensor.""" if isinstance(tensor, cupy.ndarray): - return NUMPY_NCCL_DTYPE_MAP[tensor.dtype] + return NUMPY_NCCL_DTYPE_MAP[tensor.dtype.type] if torch_available(): if isinstance(tensor, torch.Tensor): return TORCH_NCCL_DTYPE_MAP[tensor.dtype] @@ -88,8 +88,6 @@ def get_nccl_tensor_dtype(tensor): def get_tensor_ptr(tensor): """Return the pointer to the underlying memory storage of a tensor.""" - if not tensor: - raise RuntimeError('None Tensor.') if isinstance(tensor, cupy.ndarray): return tensor.data.ptr if isinstance(tensor, numpy.ndarray): @@ -102,8 +100,6 @@ def get_tensor_ptr(tensor): def get_tensor_n_elements(tensor): """Return the number of elements in a tensor.""" - if not tensor: - raise RuntimeError('None Tensor.') if isinstance(tensor, cupy.ndarray) or isinstance(tensor, numpy.ndarray): return tensor.size if torch_available(): diff --git a/python/ray/util/collective/examples/nccl_allreduce_example.py b/python/ray/util/collective/examples/nccl_allreduce_example.py index 8dc444d0c035..818d46f6ee06 100644 --- a/python/ray/util/collective/examples/nccl_allreduce_example.py +++ b/python/ray/util/collective/examples/nccl_allreduce_example.py @@ -19,7 +19,13 @@ def compute(self): print(self.send) return self.send + def destroy(self): + collective.destroy_group('') + if __name__ == "__main__": + + send = cp.ones((4, ), dtype=cp.float32) + ray.init(num_gpus=2) num_workers = 2 @@ -29,7 +35,7 @@ def compute(self): w = Worker.remote() workers.append(w) init_rets.append(w.setup.remote(num_workers, i)) - m = ray.get(init_rets) - # results = ray.get([w.compute.remote() for w in workers]) + # m = ray.get(init_rets) + results = ray.get([w.compute.remote() for w in workers]) # print(results) ray.shutdown() \ No newline at end of file From 88fbea17c181f7940b82a9e2117ccbe1776105e0 Mon Sep 17 00:00:00 2001 From: Hao Zhang Date: Wed, 2 Dec 2020 01:07:40 -0500 Subject: [PATCH 08/51] some small fix --- python/ray/util/collective/collective.py | 7 ++-- .../collective_group/base_collective_group.py | 32 ------------------- .../collective_group/nccl_collective_group.py | 15 ++++++--- python/ray/util/collective/const.py | 6 ++++ python/ray/util/collective/types.py | 3 -- 5 files changed, 20 insertions(+), 43 deletions(-) create mode 100644 python/ray/util/collective/const.py diff --git a/python/ray/util/collective/collective.py b/python/ray/util/collective/collective.py index bd4c63bc66bd..3cb697b78f09 100644 --- a/python/ray/util/collective/collective.py +++ b/python/ray/util/collective/collective.py @@ -3,6 +3,7 @@ import ray from ray.util.collective import types +from ray.util.collective.const import NAMED_ACTOR_STORE_SUFFIX # Get the availability information first by importing information _MPI_AVAILABLE = True @@ -82,7 +83,7 @@ def create_collective_group(self, if rank == 0: import cupy.cuda.nccl as nccl group_uid = nccl.get_unique_id() - store_name = group_name + types.named_actor_suffix + store_name = group_name + NAMED_ACTOR_STORE_SUFFIX store = NCCLUniqueIDStore.options(name=store_name, lifetime="detached").remote(store_name) ray.wait([store.set_id.remote(group_uid)]) @@ -123,7 +124,7 @@ def destroy_collective_group(self, group_name): if backend == 'nccl': # release the named actor if rank == 0: - store_name = group_name + types.named_actor_suffix + store_name = group_name + NAMED_ACTOR_STORE_SUFFIX store = ray.get_actor(store_name) ray.wait([store.__ray_terminate__.remote()]) ray.kill(store) @@ -195,7 +196,7 @@ def destroy_collective_group(self, group_name): if backend == 'nccl': # release the named actor if rank == 0: - store_name = group_name + types.named_actor_suffix + store_name = group_name + NAMED_ACTOR_STORE_SUFFIX store = ray.get_actor(store_name) ray.wait([store.__ray_terminate__.remote()]) ray.kill(store) diff --git a/python/ray/util/collective/collective_group/base_collective_group.py b/python/ray/util/collective/collective_group/base_collective_group.py index 55835e6430cd..fa8ea410d3ff 100644 --- a/python/ray/util/collective/collective_group/base_collective_group.py +++ b/python/ray/util/collective/collective_group/base_collective_group.py @@ -51,35 +51,3 @@ def allreduce(self, tensor, allreduce_options=AllReduceOptions()): @abstractmethod def barrier(self, barrier_options=BarrierOptions()): raise NotImplementedError() - - # @abstractmethod - # def reduce(self, tensor, dst, op=SUM, group_name='default'): - # raise NotImplementedError() - # - # @abstractmethod - # def allgather(self, tensor_list, tensor, gropu_name='default'): - # raise NotImplementedError() - # - # @abstractmethod - # def gather(self, tensor, gather_list=None, dst=0, group_name='default'): - # raise NotImplementedError() - # - # @abstractmethod - # def scatter(self, tensor, scatter_list=None, src=0, group_name='default'): - # raise NotImplementedError() - # - # @abstractmethod - # def barrier(self, group_name='default'): - # raise NotImplementedError() - - # @abstractmethod - # def send(self): - # raise NotImplementedError() - # - # @abstractmethod - # def recv(self): - # raise NotImplementedError() - # - # @abstractmethod - # def broadcast(self): - # raise NotImplementedError() diff --git a/python/ray/util/collective/collective_group/nccl_collective_group.py b/python/ray/util/collective/collective_group/nccl_collective_group.py index 99253ba30054..14fc99b25adc 100644 --- a/python/ray/util/collective/collective_group/nccl_collective_group.py +++ b/python/ray/util/collective/collective_group/nccl_collective_group.py @@ -7,9 +7,12 @@ from ray.util.collective.collective_group import nccl_util from ray.util.collective.collective_group.base_collective_group import BaseGroup -from ray.util.collective.types import AllReduceOptions, BarrierOptions, named_actor_suffix +from ray.util.collective.types import AllReduceOptions, BarrierOptions +from ray.util.collective.const import NAMED_ACTOR_STORE_SUFFIX +logger = logging.getLogger(__name__) + # TODO(Hao): # (1) stream management, instead of using the default stream, using a dedicate stream # (2) communicator management, adding a caching mechanism to enable @@ -27,20 +30,21 @@ def meet_at_store(self, timeout=180): if timeout is not None and timeout < 0: raise ValueError("The 'timeout' argument must be nonnegative. " f"Received {timeout}") - self._store_name = self._group_name + named_actor_suffix + self._store_name = self._group_name + NAMED_ACTOR_STORE_SUFFIX timeout_delta = datetime.timedelta(seconds=timeout) elapsed = datetime.timedelta(seconds=0) start_time = datetime.datetime.now() while elapsed < timeout_delta: try: - logging.debug("Trying to meet at the store '{}'".format(self._store_name)) + logger.debug("Trying to meet at the store '{}'".format(self._store_name)) self._store = ray.get_actor(self._store_name) except ValueError: - logging.debug("Failed to meet at the store '{}'." + logger.debug("Failed to meet at the store '{}'." "Trying again...".format(self._store_name)) time.sleep(1) elapsed = datetime.datetime.now() - start_time continue + logger.debug("Successful rendezvous!") break if not self._store: raise RuntimeError("Unable to meet other processes " @@ -56,6 +60,7 @@ def get_nccl_id(self): uid = ray.get(self._store.get_id.remote()) return uid + class NCCLGroup(BaseGroup): def __init__(self, world_size, rank, group_name): """Init an NCCL collective group.""" @@ -69,7 +74,7 @@ def __init__(self, world_size, rank, group_name): raise RuntimeError('NCCL in Ray requires NCCL>=2.0.') # TODO(Hao): check version here if nccl_util.get_nccl_runtime_version() < 2704: - logging.warning('NCCL send/recv calls requires NCCL>=2.7.4') + logger.warning('NCCL send/recv calls requires NCCL>=2.7.4') self._rendezvous = Rendezvous(self.group_name) self._rendezvous.meet_at_store() diff --git a/python/ray/util/collective/const.py b/python/ray/util/collective/const.py new file mode 100644 index 000000000000..3be8ebac6541 --- /dev/null +++ b/python/ray/util/collective/const.py @@ -0,0 +1,6 @@ +""" +Constants. + +Contains constants used to setup collective groups. +""" +NAMED_ACTOR_STORE_SUFFIX = '_unique_id_actor' diff --git a/python/ray/util/collective/types.py b/python/ray/util/collective/types.py index 366f8f805c11..19c68bd65bd0 100644 --- a/python/ray/util/collective/types.py +++ b/python/ray/util/collective/types.py @@ -52,6 +52,3 @@ class AllReduceOptions: class BarrierOptions: timeout = unset_timeout - - -named_actor_suffix = '_unique_id_actor' \ No newline at end of file From 1e663545db20b68a63516173e7b882fa8903b57d Mon Sep 17 00:00:00 2001 From: Hao Zhang Date: Tue, 24 Nov 2020 23:33:20 -0500 Subject: [PATCH 09/51] scaffold of the code --- python/ray/util/collective/__init__.py | 1 + python/ray/util/collective/collective.py | 127 ++++++++++++++++++ .../collective/collective_group/__init__.py | 2 + .../collective_group/base_collective_group.py | 4 + .../collective_group/mpi_collective_group.py | 6 + .../collective_group/nccl_collective_group.py | 9 ++ .../collective/collective_group/nccl_util.py | 5 + python/ray/util/collective/requirements.txt | 1 + .../collective/scratch/test_named_actor.py | 37 +++++ 9 files changed, 192 insertions(+) create mode 100644 python/ray/util/collective/__init__.py create mode 100644 python/ray/util/collective/collective.py create mode 100644 python/ray/util/collective/collective_group/__init__.py create mode 100644 python/ray/util/collective/collective_group/base_collective_group.py create mode 100644 python/ray/util/collective/collective_group/mpi_collective_group.py create mode 100644 python/ray/util/collective/collective_group/nccl_collective_group.py create mode 100644 python/ray/util/collective/collective_group/nccl_util.py create mode 100644 python/ray/util/collective/requirements.txt create mode 100644 python/ray/util/collective/scratch/test_named_actor.py diff --git a/python/ray/util/collective/__init__.py b/python/ray/util/collective/__init__.py new file mode 100644 index 000000000000..92744a8b85e9 --- /dev/null +++ b/python/ray/util/collective/__init__.py @@ -0,0 +1 @@ +from collective.collective_group.base_collective_group import * \ No newline at end of file diff --git a/python/ray/util/collective/collective.py b/python/ray/util/collective/collective.py new file mode 100644 index 000000000000..aba512b62db0 --- /dev/null +++ b/python/ray/util/collective/collective.py @@ -0,0 +1,127 @@ +"""APIs exposed under the namespace ray.util.collective.""" +import ray + +# Get the availability information first by importing information +_MPI_AVAILABLE = True +_NCCL_AVAILABLE = True + +try: + from collective.collectivegoup.mpi_collective_group import MPICollectiveGroup +except ImportError: + _MPI_AVAILABLE = False + +try: + from collective.collectivegoup.nccl_collective_group import NCCLCollectiveGroup +except ImportError: + _NCCL_AVAILABLE = False + + +def nccl_available(): + return _NCCL_AVAILABLE + +def mpi_available(): + return _MPI_AVAILABLE + + +@ray.remote +class NCCLUniqueIDStore(object): + """NCCLUniqueID. This class should be used as a named actor.""" + def __init__(self): + self.nccl_id = None + + def set_id(self, id): + self.nccl_id = id + + def get_id(self): + return self.nccl_id + + +class GroupManager(object): + """ + Use this class to manage the collective groups we created so far; + + """ + def __init__(self): + """Put some necessary meta information here.""" + self._default_group = None + + def get_default_group(self): + pass + + def set_default_group(self): + pass + + def create_collective_group(self, + backend, + group_name, + world_size, + rank): + """ + The only entry to create new collective groups, construct CollectiveGroup here. + + Put the registration and the group information into the manager metadata as well. + """ + pass + + def destroy_collective_group(self, name): + """Group destructor.""" + pass + +_group_mgr = GroupManager() + + +# Frontend API #1: +# This API is supposed to work within the actor or task program: +# See the RFC for an example. +def init_collective_group(backend, + group_name='default', + world_size=-1, + rank=-1): + # do some check on the validaty of the arguments. + # see: https://github.com/pytorch/pytorch/blob/master/torch/distributed/distributed_c10d.py + if backend == 'mpi': + if not mpi_available(): + raise RuntimeError() + raise NotImplementedError() + elif backend == 'nccl': + if not nccl_available(): + raise RuntimeError() + + global _group_mgr + _group_mgr.create_collective_group(group_name, world_size, rank) + + +# Frontend API #2: +# This API is supported to work in the driver program - the users declare a list of actors as a collective group +# @Dacheng: This API is not in the right shape, need to work with ray.remote(), please figure out. +def declare_collective_group(actors, group_options): + pass + + +#collective APIs: +def send(tensor, dst, group_name='default'): + pass + +def recv(tensor, src, group_name='default'): + pass + +def broadcast(tensor, src, group_name='default'): + pass + +def allreduce(tensor, op=SUM, group_name='default'): + pass + +def reduce(tensor, dst, op=SUM, group_name='default'): + pass + +def allgather(tensor_list, tensor, gropu_name='default'): + pass + +def gather(tensor, gather_list=None, dst=0, group_name='default'): + pass + +def scatter(tensor, scatter_list=None, src=0, group_name='default'): + pass + +def barrier(group_name='default'): + pass diff --git a/python/ray/util/collective/collective_group/__init__.py b/python/ray/util/collective/collective_group/__init__.py new file mode 100644 index 000000000000..0663625d56ab --- /dev/null +++ b/python/ray/util/collective/collective_group/__init__.py @@ -0,0 +1,2 @@ +from .nccl_collective_group import NCCLGroup +from .mpi_collective_group import MPIGroup diff --git a/python/ray/util/collective/collective_group/base_collective_group.py b/python/ray/util/collective/collective_group/base_collective_group.py new file mode 100644 index 000000000000..cb43bd503a5a --- /dev/null +++ b/python/ray/util/collective/collective_group/base_collective_group.py @@ -0,0 +1,4 @@ +"""Abstract class for collective groups.""" +class BaseGroup: + def __init__(self, world_size, rank, group_name): + pass diff --git a/python/ray/util/collective/collective_group/mpi_collective_group.py b/python/ray/util/collective/collective_group/mpi_collective_group.py new file mode 100644 index 000000000000..b16f2ac22673 --- /dev/null +++ b/python/ray/util/collective/collective_group/mpi_collective_group.py @@ -0,0 +1,6 @@ +from collective.collective_group.base_collective_group import BaseGroup + +# TODO(Dacheng): implement this +class MPIGroup(BaseGroup): + def __init__(self, world_size, rank, group_name): + BaseGroup.__init__(self, world_size, rank, group_name) diff --git a/python/ray/util/collective/collective_group/nccl_collective_group.py b/python/ray/util/collective/collective_group/nccl_collective_group.py new file mode 100644 index 000000000000..5e88bd956780 --- /dev/null +++ b/python/ray/util/collective/collective_group/nccl_collective_group.py @@ -0,0 +1,9 @@ +from collective.collective_group.nccl_util import * +from collective.collective_group.base_collective_group import BaseGroup + +# TODO(Hao): implement this +class NCCLGroup(BaseGroup): + def __init__(self, world_size, rank, group_name): + BaseGroup.__init__(self, world_size, rank, group_name) + + def diff --git a/python/ray/util/collective/collective_group/nccl_util.py b/python/ray/util/collective/collective_group/nccl_util.py new file mode 100644 index 000000000000..065ad0fc7471 --- /dev/null +++ b/python/ray/util/collective/collective_group/nccl_util.py @@ -0,0 +1,5 @@ +try: + from cupy.cuda.nccl import get_unique_id + from cupy.cuda.nccl import get_version +except ImportError: + raise diff --git a/python/ray/util/collective/requirements.txt b/python/ray/util/collective/requirements.txt new file mode 100644 index 000000000000..ce5057b221f1 --- /dev/null +++ b/python/ray/util/collective/requirements.txt @@ -0,0 +1 @@ +cupy-cuda100 \ No newline at end of file diff --git a/python/ray/util/collective/scratch/test_named_actor.py b/python/ray/util/collective/scratch/test_named_actor.py new file mode 100644 index 000000000000..555e2ce611fd --- /dev/null +++ b/python/ray/util/collective/scratch/test_named_actor.py @@ -0,0 +1,37 @@ +import ray + +@ray.remote +class NamedActor: + def __init__(self): + self.id = '' + + def set_id(self, id): + self.id = id + return self.id + + def get_id(self): + return self.id + + +@ray.remote +class Worker: + def __init__(self, rank): + self.rank = rank + + def do_work(self): + if self.rank == 0: + id = '123' + ac = NamedActor.options(name='uniquename', lifetime="detached").remote() + ray.wait([ac.set_id.remote('123')]) + else: + ac = ray.get_actor('uniquename') + id = ray.get(ac.get_id.remote()) + print(id) + return id + + +ray.init() + +workers = [Worker.remote(rank) for rank in range(4)] +ret = ray.get(workers[0].do_work.remote()) +m = ray.get([workers[i].do_work.remote() for i in range(1, 4)]) From c41f046e46cd2e80272aa65d510a8630b365c451 Mon Sep 17 00:00:00 2001 From: YLJALDC Date: Thu, 26 Nov 2020 01:43:27 -0500 Subject: [PATCH 10/51] some scratch and options change --- python/ray/actor.py | 18 +++-- python/ray/util/collective/collective.py | 23 +++--- .../ray/util/collective/scratch/test_nccl.py | 81 +++++++++++++++++++ .../util/collective/scratch/test_options.py | 14 ++++ 4 files changed, 121 insertions(+), 15 deletions(-) create mode 100644 python/ray/util/collective/scratch/test_nccl.py create mode 100644 python/ray/util/collective/scratch/test_options.py diff --git a/python/ray/actor.py b/python/ray/actor.py index b8981ca3d23e..0d39a92c69a9 100644 --- a/python/ray/actor.py +++ b/python/ray/actor.py @@ -423,7 +423,8 @@ def options(self, placement_group=None, placement_group_bundle_index=-1, placement_group_capture_child_tasks=None, - override_environment_variables=None): + override_environment_variables=None, + collective=None): """Configures and overrides the actor instantiation parameters. The arguments are the same as those that can be passed @@ -465,7 +466,8 @@ def remote(self, *args, **kwargs): placement_group_capture_child_tasks=( placement_group_capture_child_tasks), override_environment_variables=( - override_environment_variables)) + override_environment_variables), + collective=collective) return ActorOptionWrapper() @@ -486,7 +488,8 @@ def _remote(self, placement_group=None, placement_group_bundle_index=-1, placement_group_capture_child_tasks=None, - override_environment_variables=None): + override_environment_variables=None, + collective=None): """Create an actor. This method allows more flexibility than the remote method because @@ -526,6 +529,7 @@ def _remote(self, override_environment_variables: Environment variables to override and/or introduce for this actor. This is a dictionary mapping variable names to their values. + collective: what colletive configuration to use Returns: A handle to the newly created actor. @@ -685,7 +689,8 @@ def _remote(self, actor_method_cpu, meta.actor_creation_function_descriptor, worker.current_session_and_job, - original_handle=True) + original_handle=True, + collective=collective) return actor_handle @@ -729,7 +734,8 @@ def __init__(self, actor_method_cpus, actor_creation_function_descriptor, session_and_job, - original_handle=False): + original_handle=False, + collective=None): self._ray_actor_language = language self._ray_actor_id = actor_id self._ray_original_handle = original_handle @@ -742,7 +748,7 @@ def __init__(self, self._ray_actor_creation_function_descriptor = \ actor_creation_function_descriptor self._ray_function_descriptor = {} - + self._collective = collective if not self._ray_is_cross_language: assert isinstance(actor_creation_function_descriptor, PythonFunctionDescriptor) diff --git a/python/ray/util/collective/collective.py b/python/ray/util/collective/collective.py index aba512b62db0..dfbd5d33e03b 100644 --- a/python/ray/util/collective/collective.py +++ b/python/ray/util/collective/collective.py @@ -6,12 +6,12 @@ _NCCL_AVAILABLE = True try: - from collective.collectivegoup.mpi_collective_group import MPICollectiveGroup + from collective.collectivegoup.mpi_collective_group import MPIGroup except ImportError: _MPI_AVAILABLE = False try: - from collective.collectivegoup.nccl_collective_group import NCCLCollectiveGroup + from collective.collectivegoup.nccl_collective_group import NCCLGroup except ImportError: _NCCL_AVAILABLE = False @@ -22,6 +22,14 @@ def nccl_available(): def mpi_available(): return _MPI_AVAILABLE +def backend_check(backend): + if backend == 'mpi': + if not mpi_available(): + raise RuntimeError() + raise NotImplementedError() + elif backend == 'nccl': + if not nccl_available(): + raise RuntimeError() @ray.remote class NCCLUniqueIDStore(object): @@ -79,13 +87,7 @@ def init_collective_group(backend, rank=-1): # do some check on the validaty of the arguments. # see: https://github.com/pytorch/pytorch/blob/master/torch/distributed/distributed_c10d.py - if backend == 'mpi': - if not mpi_available(): - raise RuntimeError() - raise NotImplementedError() - elif backend == 'nccl': - if not nccl_available(): - raise RuntimeError() + backend_check(backend) global _group_mgr _group_mgr.create_collective_group(group_name, world_size, rank) @@ -95,6 +97,9 @@ def init_collective_group(backend, # This API is supported to work in the driver program - the users declare a list of actors as a collective group # @Dacheng: This API is not in the right shape, need to work with ray.remote(), please figure out. def declare_collective_group(actors, group_options): + backend = group_options["backend"] + backend_check(backend) + uid = None pass diff --git a/python/ray/util/collective/scratch/test_nccl.py b/python/ray/util/collective/scratch/test_nccl.py new file mode 100644 index 000000000000..b903308cf6c9 --- /dev/null +++ b/python/ray/util/collective/scratch/test_nccl.py @@ -0,0 +1,81 @@ +import ray +import cupy.cuda.nccl as nccl +from collections import defaultdict +import cupy as cp + +@ray.remote(num_gpus=0.5) +class Actor: + def __init(self): + cp.cuda.Stream.null.synchronize() + self.send = cp.ones((10,), dtype=cp.float32) + self.recv = cp.zeros((10,), dtype=cp.float32) + cp.cuda.Stream.null.synchronize() + + def send_getter(self): + return self.send + + def send_setter(self, val): + self.send = val + + def recv_getter(self): + return self.recv + + def recv_setter(self, val): + self.recv = val + + def compute(self): + pass + +class GroupManager(Object): + def __init__(self): + self.group = defaultdict([]) + + def create_colletive_group(self, + backend, + group_name, + world_size, + rank, + actor, + uid=None): + + self.group[group_name].append({actor: nccl.NcclCommunicator(world_size, uid, rank)}) + + + +def declare_collective_group(actors, group_options): + # sort actors by rank + ranks = group_options["rank"] + if len(actors) != len(ranks) or len(actors) != group_options["world_size"]: + raise ValueError() + + #sorted_actors = [x for _,x in sorted(zip(ranks, actors)), key=lambda pair: pair[0]] + uid = nccl.get_unique_id() + for i in range(0, len(ranks): + _group_mgr.create_collective_group(group_options["backend"], + group_options["name"], + group_options["world_size"], + ranks, + actors[i], + uid) + +def allreduce(group_name): + for (actor, comm) in _group_mgr.group[group_name]: + dummy = self.recv = cp.zeros((10,), dtype=cp.float32).ptr + comm.allReduce(ray.get(actor.send_getter()).ptr, dummy, 10, cp.cuda.nccl.NCCL_FLOAT32, 1, cp.cuda.Stream.null.ptr) + actor.set_recv(dummy) + cp.cuda.Stream.null.synchronize() + +_group_mgr = GroupManager() + +group_options = {"name" : "haha", + "backend" : "nccl", + "world_size" : 4, + "rank" : [0,1,2,3]} + +actors = [Actor().remote() for i in range(4)] +declare_collective_group(actors, group_options}) + +allreduce("haha") + +for i in range(4): + print(ray.get(actors[i].recv_getter())) diff --git a/python/ray/util/collective/scratch/test_options.py b/python/ray/util/collective/scratch/test_options.py new file mode 100644 index 000000000000..5d99eed91b29 --- /dev/null +++ b/python/ray/util/collective/scratch/test_options.py @@ -0,0 +1,14 @@ +import ray +ray.init() + +@ray.remote(num_gpus=1) +class Actor: + def __init__(self): + pass + + def compute(self): + pass + +# might work? +worker = Actor.options(collective={})# .remote() +print(worker._collective) From 912bd0f28f719b3168e1c0945cce53ce687dd56c Mon Sep 17 00:00:00 2001 From: Hao Zhang Date: Mon, 30 Nov 2020 23:23:41 -0500 Subject: [PATCH 11/51] NCCL mostly done, supporting API#1 --- python/ray/util/collective/collective.py | 197 +++++++++++++----- .../collective_group/base_collective_group.py | 79 ++++++- .../collective_group/mpi_collective_group.py | 11 +- .../collective_group/nccl_collective_group.py | 125 ++++++++++- .../collective/collective_group/nccl_util.py | 109 +++++++++- .../util/collective/tests/test_nccl_group.py | 0 python/ray/util/collective/types.py | 57 +++++ 7 files changed, 515 insertions(+), 63 deletions(-) create mode 100644 python/ray/util/collective/tests/test_nccl_group.py create mode 100644 python/ray/util/collective/types.py diff --git a/python/ray/util/collective/collective.py b/python/ray/util/collective/collective.py index dfbd5d33e03b..829cec6a142a 100644 --- a/python/ray/util/collective/collective.py +++ b/python/ray/util/collective/collective.py @@ -1,5 +1,8 @@ """APIs exposed under the namespace ray.util.collective.""" +import logging + import ray +from python.ray.util.collective import types # Get the availability information first by importing information _MPI_AVAILABLE = True @@ -19,10 +22,12 @@ def nccl_available(): return _NCCL_AVAILABLE + def mpi_available(): return _MPI_AVAILABLE -def backend_check(backend): + +def _backend_check(backend): if backend == 'mpi': if not mpi_available(): raise RuntimeError() @@ -31,16 +36,21 @@ def backend_check(backend): if not nccl_available(): raise RuntimeError() + @ray.remote class NCCLUniqueIDStore(object): """NCCLUniqueID. This class should be used as a named actor.""" - def __init__(self): + def __init__(self, name): + self.name = name self.nccl_id = None - def set_id(self, id): - self.nccl_id = id + def set_id(self, uid): + self.nccl_id = uid + return self.nccl_id def get_id(self): + if not self.nccl_id: + logging.warning('The NCCL ID has not been set yet for store {}'.format(self.name)) return self.nccl_id @@ -51,82 +61,167 @@ class GroupManager(object): """ def __init__(self): """Put some necessary meta information here.""" - self._default_group = None - - def get_default_group(self): - pass - - def set_default_group(self): - pass + self._name_group_map = {} + self._group_name_map = {} def create_collective_group(self, backend, - group_name, world_size, - rank): + rank, + group_name): """ - The only entry to create new collective groups, construct CollectiveGroup here. + The only entry to create new collective groups and register to the manager. Put the registration and the group information into the manager metadata as well. """ - pass - - def destroy_collective_group(self, name): + if backend == 'mpi': + raise NotImplementedError() + elif backend == 'nccl': + # create the ncclUniqueID + if rank == 0: + import cupy.cuda.nccl as nccl + group_uid = nccl.get_unique_id() + store_name = group_name + types.named_actor_suffix + store = NCCLUniqueIDStore\ + .options(name=store_name, lifetime="detached")\ + .remote() + ray.wait([store.set_id.remote(group_uid)]) + + g = NCCLGroup(world_size, rank, group_name) + self._name_group_map[group_name] = g + self._group_name_map[g] = group_name + return self._name_group_map[group_name] + + def is_group_exist(self, group_name): + if group_name in self._name_group_map: + return True + return False + + def get_group_by_name(self, group_name): + """Get the collective group handle by its name.""" + if group_name not in self._name_group_map: + return None + return self._name_group_map[group_name] + + def destroy_collective_group(self, group_name): """Group destructor.""" - pass + if group_name not in self._name_group_map: + logging.warning('The group {} does not exist'.format(group_name)) + return + + # release the collective group resource + g = self._name_group_map[group_name] + + rank = g.rank + backend = g.backend() + + # clean up the dicts + del self._group_name_map[g] + del self._name_group_map[group_name] + + if backend == 'nccl': + # release the named actor + if rank == 0: + store_name = group_name + types.named_actor_suffix + store = ray.get_actor(store_name) + ray.wait([store.__ray_terminate__.remote()]) + ray.kill(store) + g.destroy() _group_mgr = GroupManager() -# Frontend API #1: -# This API is supposed to work within the actor or task program: -# See the RFC for an example. def init_collective_group(backend, - group_name='default', - world_size=-1, - rank=-1): - # do some check on the validaty of the arguments. - # see: https://github.com/pytorch/pytorch/blob/master/torch/distributed/distributed_c10d.py - backend_check(backend) + world_size, + rank, + group_name='default'): + """ + Initialize a collective group inside an actor process. + + This is an + Args: + backend: + world_size: + rank: + group_name: + Returns: + + """ + _backend_check(backend) global _group_mgr - _group_mgr.create_collective_group(group_name, world_size, rank) + # TODO(Hao): implement a group auto-counter. + if not group_name: + raise ValueError('group_name: {}, needs to be a string.'.format(group_name)) + + if _group_mgr.is_group_exist(group_name): + raise RuntimeError('Trying to initialize a gropu twice.') + + assert(world_size > 0) + assert(rank >= 0 and rank < world_size) + + _group_mgr.create_collective_group(backend, world_size, rank, group_name) -# Frontend API #2: -# This API is supported to work in the driver program - the users declare a list of actors as a collective group -# @Dacheng: This API is not in the right shape, need to work with ray.remote(), please figure out. def declare_collective_group(actors, group_options): + """ + # Frontend API #2: + # This API is supported to work in the driver program - the users declare a list of actors as a collective group + # @Dacheng: This API is not in the right shape, need to work with ray.remote(), please figure out. + Args: + actors: + group_options: + + Returns: + + """ backend = group_options["backend"] - backend_check(backend) + _backend_check(backend) uid = None pass -#collective APIs: -def send(tensor, dst, group_name='default'): - pass +def allreduce(tensor, + group_name, + op=types.ReduceOp.SUM): + """ + Collective allreduce the tensor across the group with name group_name. -def recv(tensor, src, group_name='default'): - pass + Args: + tensor: + group_name (string): + op: -def broadcast(tensor, src, group_name='default'): - pass + Returns: + None + """ + g = _check_and_get_group(group_name) + opts = types.AllReduceOptions + opts.reduceOp = op + g.allreduce(tensor, opts) -def allreduce(tensor, op=SUM, group_name='default'): - pass -def reduce(tensor, dst, op=SUM, group_name='default'): - pass +def barrier(group_name): + """ + Barrier all collective process in the group with name group_name. -def allgather(tensor_list, tensor, gropu_name='default'): - pass + Args: + group_name (string): -def gather(tensor, gather_list=None, dst=0, group_name='default'): - pass + Returns: + None + """ + g = _check_and_get_group(group_name) + g.barrier() + + +def _check_and_get_group(group_name): + """Check the existence and return the group handle.""" + global _group_mgr + if not _group_mgr.is_group_exist(group_name): + raise ValueError('The collective group {} is not initialized.'.format(group_name)) + # TODO(Hao): check if this rank is in the group. + g = _group_mgr.get_group_by_name(group_name) + return g -def scatter(tensor, scatter_list=None, src=0, group_name='default'): - pass -def barrier(group_name='default'): - pass diff --git a/python/ray/util/collective/collective_group/base_collective_group.py b/python/ray/util/collective/collective_group/base_collective_group.py index cb43bd503a5a..c5717b4a1776 100644 --- a/python/ray/util/collective/collective_group/base_collective_group.py +++ b/python/ray/util/collective/collective_group/base_collective_group.py @@ -1,4 +1,81 @@ """Abstract class for collective groups.""" -class BaseGroup: +from abc import ABCMeta +from abc import abstractmethod +import six + +from python.ray.util.collective.types import AllReduceOptions, BarrierOptions + + +class BaseGroup(six.with_metaclass(ABCMeta)): def __init__(self, world_size, rank, group_name): + """ + Init the process group with basic information. + + Args: + world_size: + rank: + group_name: + """ + self._world_size = world_size + self._rank = rank + self._group_name = group_name + + @property + def rank(self): + """Return the rank of the current process.""" + return self._rank + + @property + def world_size(self): + """Return the number of processes in this group.""" + return self._world_size + + def destroy_group(self): + """GC the communicators.""" pass + + @abstractmethod + @classmethod + def backend(cls): + """The backend of this collective group.""" + raise NotImplementedError() + + @abstractmethod + def allreduce(self, tensor, allreduce_options=AllReduceOptions()): + raise NotImplementedError() + + @abstractmethod + def barrier(self, barrier_options=BarrierOptions()): + raise NotImplementedError() + + # @abstractmethod + # def reduce(self, tensor, dst, op=SUM, group_name='default'): + # raise NotImplementedError() + # + # @abstractmethod + # def allgather(self, tensor_list, tensor, gropu_name='default'): + # raise NotImplementedError() + # + # @abstractmethod + # def gather(self, tensor, gather_list=None, dst=0, group_name='default'): + # raise NotImplementedError() + # + # @abstractmethod + # def scatter(self, tensor, scatter_list=None, src=0, group_name='default'): + # raise NotImplementedError() + # + # @abstractmethod + # def barrier(self, group_name='default'): + # raise NotImplementedError() + + # @abstractmethod + # def send(self): + # raise NotImplementedError() + # + # @abstractmethod + # def recv(self): + # raise NotImplementedError() + # + # @abstractmethod + # def broadcast(self): + # raise NotImplementedError() diff --git a/python/ray/util/collective/collective_group/mpi_collective_group.py b/python/ray/util/collective/collective_group/mpi_collective_group.py index b16f2ac22673..58e9b7279e33 100644 --- a/python/ray/util/collective/collective_group/mpi_collective_group.py +++ b/python/ray/util/collective/collective_group/mpi_collective_group.py @@ -1,6 +1,7 @@ -from collective.collective_group.base_collective_group import BaseGroup +"""Implementation of the MPI collective group.""" +try: + import mpi4py +except ImportError: + raise -# TODO(Dacheng): implement this -class MPIGroup(BaseGroup): - def __init__(self, world_size, rank, group_name): - BaseGroup.__init__(self, world_size, rank, group_name) +raise NotImplementedError() diff --git a/python/ray/util/collective/collective_group/nccl_collective_group.py b/python/ray/util/collective/collective_group/nccl_collective_group.py index 5e88bd956780..08c28733fcbc 100644 --- a/python/ray/util/collective/collective_group/nccl_collective_group.py +++ b/python/ray/util/collective/collective_group/nccl_collective_group.py @@ -1,9 +1,124 @@ -from collective.collective_group.nccl_util import * -from collective.collective_group.base_collective_group import BaseGroup +import logging +import ray +import cupy + +from python.ray.util.collective.collective_group import nccl_util +from python.ray.util.collective.collective_group.base_collective_group import BaseGroup +from python.ray.util.collective.types import AllReduceOptions, BarrierOptions, named_actor_suffix + + +# TODO(Hao): +# (1) stream management, instead of using the default stream, using a dedicate stream +# (2) communicator management, adding a caching mechanism to enable + -# TODO(Hao): implement this class NCCLGroup(BaseGroup): def __init__(self, world_size, rank, group_name): - BaseGroup.__init__(self, world_size, rank, group_name) + """Init an NCCL collective group.""" + super(NCCLGroup, self).__init__(self, world_size, rank, group_name) + self._nccl_uid_store = None + self._nccl_uid = None + + # TODO(Hao): change this to a be a cache + self._nccl_comm = None + + # Check NCCL version + if nccl_util.get_nccl_build_version() < 2000: + raise RuntimeError('NCCL in Ray requires NCCL>=2.0.') + + # TODO(Hao): check version here + if nccl_util.get_nccl_runtime_version() < 2704: + logging.warning('NCCL send/recv calls requires NCCL>=2.7.4') + + # Setup the nccl uid using the store + self._init_nccl_unique_id() + + # Setup a tensor for barrier calls + self._barrier_tensor = cupy.array([1]) + + def _init_nccl_unique_id(self): + """Init the NCCL unique ID required for setting up NCCL communicator.""" + # using group_name to query the UniqueIDActor + unique_actor_name = self.group_name + named_actor_suffix + + # Assuming this named actor has been created. + self._nccl_uid_store = ray.get_actor(unique_actor_name) + self._nccl_uid = ray.get(self._nccl_uid_store.get_id.remote()) + + @property + def nccl_uid(self): + return self._nccl_uid + + def destory_group(self): + """Destroy the group and release the NCCL communicators safely.""" + if self._nccl_comm is not None: + self.barrier() + # We also need a barrier call here. + stream = self._get_cuda_stream() + stream.synchronize() + # destroy the communicator + self._nccl_comm.destory() + self._nccl_comm = None + super(NCCLGroup, self).destroy_group() + + @classmethod + def backend(cls): + return 'nccl' + + def allreduce(self, tensor, allreduce_options=AllReduceOptions()): + """ + AllReduce a list of tensors following options. + + Args: + tensor: the tensor to be reduced, each tensor locates on a GPU + allreduce_options: + + Returns: + """ + # obtain the communicator + comm = self._get_nccl_communicator() + # obtain the stream: using default stream by now + # TODO(Hao): implement a simple stream manager here + stream = self._get_cuda_stream() + + dtype = nccl_util.get_nccl_tensor_dtype(tensor) + ptr = nccl_util.get_tensor_ptr(tensor) + n_elems = nccl_util.get_tensor_n_elements(tensor) + reduce_op = nccl_util.get_nccl_reduce_op(allreduce_options.reduceOp) + + # in-place allreduce + comm.allReduce(ptr, + ptr, + n_elems, + dtype, + reduce_op, + stream.ptr) + + def barrier(self, barrier_options=BarrierOptions()): + """ + Blocks until all processes in the communicator have reached this barrier call. + + Args: + barrier_options: + + Returns: + """ + self.allreduce(self._barrier_tensor) + + def _get_nccl_communicator(self): + """Create a new NCCL communicator for the collective task, or using a cached communicator.""" + # TODO(Hao): later change this to use device keys and query from cache. + # TODO(Hao): implement a thin wrapper + if not self._nccl_comm: + self._nccl_comm = nccl_util.create_nccl_communicator( + self.world_size, self.nccl_uid, self.rank) + return self._nccl_comm + + def _get_cuda_stream(self): + """Obtain an idle stream from a stream pool for the collective task.""" + # TODO: implement a simple stream manager. + return cupy.cuda.Stream.null - def + # def _collective_call(self, *args): + # """Private method to encapsulate all collective calls""" + # pass diff --git a/python/ray/util/collective/collective_group/nccl_util.py b/python/ray/util/collective/collective_group/nccl_util.py index 065ad0fc7471..67c595dee930 100644 --- a/python/ray/util/collective/collective_group/nccl_util.py +++ b/python/ray/util/collective/collective_group/nccl_util.py @@ -1,5 +1,112 @@ +"""Code to wrap some NCCL API calls.""" +import numpy try: + import cupy + from cupy.cuda import nccl from cupy.cuda.nccl import get_unique_id from cupy.cuda.nccl import get_version + from cupy.cuda.nccl import get_build_version + from cupy.cuda.nccl import NcclCommunicator + from cupy.cuda.nccl import NcclError except ImportError: - raise + raise ImportError('NCCL in Ray requires Cupy being available!') + +from python.ray.util.collective.types import ReduceOp, torch_available + + +NCCL_REDUCE_OP_MAP = { + ReduceOp.SUM: nccl.NCCL_SUM, + ReduceOp.PRODUCT: nccl.NCCL_PROD, + ReduceOp.MIN: nccl.NCCL_MIN, + ReduceOp.MAX: nccl.NCCL_MAX, +} + + +# cupy types are the same with numpy types +NUMPY_NCCL_DTYPE_MAP = { + numpy.bool: nccl.NCCL_UINT8, + numpy.float16: nccl.NCCL_FLOAT16, + numpy.float32: nccl.NCCL_FLOAT32, + numpy.float64: nccl.NCCL_FLOAT64, +} + +if torch_available(): + import torch + TORCH_NCCL_DTYPE_MAP = { + torch.bool: nccl.NCCL_UINT8, + torch.float16: nccl.NCCL_FLOAT16, + torch.float32: nccl.NCCL_FLOAT32, + torch.float64: nccl.NCCL_FLOAT64, + } + + +def get_nccl_build_version(): + return get_build_version() + + +def get_nccl_runtime_version(): + return get_build_version() + + +def create_nccl_communicator(world_size, nccl_unique_id, rank): + """ + Create an NCCL communicator using NCCL APIs. + + TODO(Hao): verify that the call has big overhead. + + Returns: + comm(type): an NCCL communicator. + + """ + # TODO(Hao): make this inside the NCCLComm class, and implement the abort method. + # Make it RAII + comm = NcclCommunicator(world_size, nccl_unique_id, rank) + return comm + + +def get_nccl_reduce_op(reduce_op): + """ + Map the reduce op to NCCL reduce op type. + + Returns: + Nccl_op (nccl.ncclRedOp_t) + """ + if reduce_op not in NCCL_REDUCE_OP_MAP: + raise RuntimeError('NCCL does not support ReduceOp: {}'.format(reduce_op)) + return NCCL_REDUCE_OP_MAP[reduce_op] + + +def get_nccl_tensor_dtype(tensor): + """Return the corresponded NCCL dtype given a tensor.""" + if isinstance(tensor, cupy.ndarray): + return NUMPY_NCCL_DTYPE_MAP[tensor.dtype] + if torch_available(): + if isinstance(tensor, torch.Tensor): + return TORCH_NCCL_DTYPE_MAP[tensor.dtype] + raise ValueError('Unsupported tensor type') + + +def get_tensor_ptr(tensor): + """Return the pointer to the underlying memory storage of a tensor.""" + if not tensor: + raise RuntimeError('None Tensor.') + if isinstance(tensor, cupy.ndarray): + return tensor.data.ptr + if isinstance(tensor, numpy.ndarray): + return tensor.data + if torch_available(): + if isinstance(tensor, torch.Tensor): + return tensor.data_ptr() + raise ValueError('Unsupported tensor type') + + +def get_tensor_n_elements(tensor): + """Return the number of elements in a tensor.""" + if not tensor: + raise RuntimeError('None Tensor.') + if isinstance(tensor, cupy.ndarray) or isinstance(tensor, numpy.ndarray): + return tensor.size + if torch_available(): + if isinstance(tensor, torch.Tensor): + return torch.numel(tensor) + raise ValueError('Unsupported tensor type') diff --git a/python/ray/util/collective/tests/test_nccl_group.py b/python/ray/util/collective/tests/test_nccl_group.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/python/ray/util/collective/types.py b/python/ray/util/collective/types.py new file mode 100644 index 000000000000..366f8f805c11 --- /dev/null +++ b/python/ray/util/collective/types.py @@ -0,0 +1,57 @@ +"""Types conversion between different backends.""" +from enum import Enum +from datetime import timedelta + +_NUMPY_AVAILABLE = True +_TORCH_AVAILABLE = True +_CUPY_AVAILABLE = True + +try: + import numpy +except ImportError: + _NUMPY_AVAILABLE = False + +try: + import torch +except ImportError: + _TORCH_AVAILABLE = False + +try: + import cupy +except ImportError: + _CUPY_AVAILABLE = False + + +def numpy_available(): + return _NUMPY_AVAILABLE + + +def cupy_available(): + return _CUPY_AVAILABLE + + +def torch_available(): + return _TORCH_AVAILABLE + + +# TODO(Hao): extend this to support more MPI types +class ReduceOp(Enum): + SUM = 0 + PRODUCT = 1 + MIN = 2 + MAX = 3 + + +unset_timeout = timedelta(milliseconds=-1) + + +class AllReduceOptions: + reduceOp = ReduceOp.SUM + timeout = unset_timeout + + +class BarrierOptions: + timeout = unset_timeout + + +named_actor_suffix = '_unique_id_actor' \ No newline at end of file From 5db388f72a1d4f4629fc0b92a9f6621e5bff54fb Mon Sep 17 00:00:00 2001 From: Dacheng Li Date: Tue, 1 Dec 2020 01:46:26 -0500 Subject: [PATCH 12/51] interface 2.1 2.2 scratch --- python/ray/actor.py | 8 +- python/ray/util/__init__.py | 2 +- python/ray/util/collective/collective.py | 94 ++++++++++++++++++++++-- 3 files changed, 97 insertions(+), 7 deletions(-) diff --git a/python/ray/actor.py b/python/ray/actor.py index 0d39a92c69a9..513180d9cf6d 100644 --- a/python/ray/actor.py +++ b/python/ray/actor.py @@ -8,7 +8,7 @@ import ray.worker from ray.util.placement_group import ( PlacementGroup, check_placement_group_index, get_current_placement_group) - +from ray.util.collective import collective as col from ray import ActorClassID, Language from ray._raylet import PythonFunctionDescriptor from ray import cross_language @@ -679,6 +679,12 @@ def _remote(self, extension_data=str(actor_method_cpu), override_environment_variables=override_environment_variables or dict()) + + if collective: + col.init_collective_group(backend=collective["backend"], + world_size=collective["world_size"], + rank=collective["rank"], + group_name=collective["group_name"]) actor_handle = ActorHandle( meta.language, diff --git a/python/ray/util/__init__.py b/python/ray/util/__init__.py index 2a6d0a0290ec..e4a1e83c93de 100644 --- a/python/ray/util/__init__.py +++ b/python/ray/util/__init__.py @@ -9,5 +9,5 @@ __all__ = [ "ActorPool", "disable_log_once_globally", "enable_periodic_logging", "iter", "log_once", "pdb", "placement_group", "placement_group_table", - "remove_placement_group" + "remove_placement_group", "collective" ] diff --git a/python/ray/util/collective/collective.py b/python/ray/util/collective/collective.py index 829cec6a142a..17bcbd9f5e3b 100644 --- a/python/ray/util/collective/collective.py +++ b/python/ray/util/collective/collective.py @@ -128,8 +128,80 @@ def destroy_collective_group(self, group_name): ray.kill(store) g.destroy() -_group_mgr = GroupManager() +class GroupManager_2(object): + """ + Use this class to manage the collective groups we created so far; + For interface 2.2 + + """ + def __init__(self): + """Put some necessary meta information here.""" + self._name_group_map = {} + self._group_name_map = {} + + def create_collective_group(self, + backend, + world_size, + rank, + group_name): + """ + The only entry to create new collective groups and register to the manager. + + Put the registration and the group information into the manager metadata as well. + """ + if backend == 'mpi': + raise NotImplementedError() + elif backend == 'nccl': + # create the ncclUniqueID + #if rank == 0: + import cupy.cuda.nccl as nccl + group_uid = nccl.get_unique_id() + + for r in rank: + g = NCCLGroup(world_size, r, group_name) + self._name_group_map[group_name] = g + self._group_name_map[g] = group_name + return self._name_group_map[group_name] + + def is_group_exist(self, group_name): + if group_name in self._name_group_map: + return True + return False + + def get_group_by_name(self, group_name): + """Get the collective group handle by its name.""" + if group_name not in self._name_group_map: + return None + return self._name_group_map[group_name] + + def destroy_collective_group(self, group_name): + """Group destructor.""" + if group_name not in self._name_group_map: + logging.warning('The group {} does not exist'.format(group_name)) + return + + # release the collective group resource + g = self._name_group_map[group_name] + + rank = g.rank + backend = g.backend() + + # clean up the dicts + del self._group_name_map[g] + del self._name_group_map[group_name] + + if backend == 'nccl': + # release the named actor + if rank == 0: + store_name = group_name + types.named_actor_suffix + store = ray.get_actor(store_name) + ray.wait([store.__ray_terminate__.remote()]) + ray.kill(store) + g.destroy() + +_group_mgr = GroupManager() +_group_mgr2 = GroupMagager_2() def init_collective_group(backend, world_size, @@ -155,7 +227,7 @@ def init_collective_group(backend, raise ValueError('group_name: {}, needs to be a string.'.format(group_name)) if _group_mgr.is_group_exist(group_name): - raise RuntimeError('Trying to initialize a gropu twice.') + raise RuntimeError('Trying to initialize a group twice.') assert(world_size > 0) assert(rank >= 0 and rank < world_size) @@ -175,11 +247,23 @@ def declare_collective_group(actors, group_options): Returns: """ - backend = group_options["backend"] + global _group_mgr_2 + try: + group_name = group_options["group_name"] + world_size = group_options["world_size"] + rank = group_options["rank"] + backend = group_options["backend"] + except: + raise ValueError("group options incomplete") + _backend_check(backend) - uid = None - pass + if _group_mgr_2.is_group_exist(group_name): + raise RuntimeError('Trying to initialize a group twice.') + + assert(world_size > 0) + assert(rank >= 0 and rank < world_size) + _group_mgr_2.create_collective_group(backend, world_size, rank, group_name, actors) def allreduce(tensor, group_name, From bd91da97d4a7ff7267653efadf5b0b90a99a768d Mon Sep 17 00:00:00 2001 From: Hao Zhang Date: Tue, 1 Dec 2020 02:41:22 -0500 Subject: [PATCH 13/51] put code into ray and fix some importing issues --- python/ray/util/collective/__init__.py | 2 +- python/ray/util/collective/collective.py | 15 ++++---- .../collective/collective_group/__init__.py | 1 - .../collective_group/base_collective_group.py | 8 +++-- .../collective_group/mpi_collective_group.py | 2 -- .../collective_group/nccl_collective_group.py | 10 +++--- .../collective/collective_group/nccl_util.py | 2 +- .../examples/nccl_allreduce_example.py | 35 +++++++++++++++++++ 8 files changed, 56 insertions(+), 19 deletions(-) create mode 100644 python/ray/util/collective/examples/nccl_allreduce_example.py diff --git a/python/ray/util/collective/__init__.py b/python/ray/util/collective/__init__.py index 92744a8b85e9..976ed700cbb8 100644 --- a/python/ray/util/collective/__init__.py +++ b/python/ray/util/collective/__init__.py @@ -1 +1 @@ -from collective.collective_group.base_collective_group import * \ No newline at end of file +from .collective import init_collective_group, allreduce, barrier \ No newline at end of file diff --git a/python/ray/util/collective/collective.py b/python/ray/util/collective/collective.py index 17bcbd9f5e3b..ef62695eecb9 100644 --- a/python/ray/util/collective/collective.py +++ b/python/ray/util/collective/collective.py @@ -2,19 +2,19 @@ import logging import ray -from python.ray.util.collective import types +from ray.util.collective import types # Get the availability information first by importing information _MPI_AVAILABLE = True _NCCL_AVAILABLE = True try: - from collective.collectivegoup.mpi_collective_group import MPIGroup + from ray.util.collective.collective_group.mpi_collective_group import MPIGroup except ImportError: _MPI_AVAILABLE = False try: - from collective.collectivegoup.nccl_collective_group import NCCLGroup + from ray.util.collective.collective_group.nccl_collective_group import NCCLGroup except ImportError: _NCCL_AVAILABLE = False @@ -82,11 +82,10 @@ def create_collective_group(self, import cupy.cuda.nccl as nccl group_uid = nccl.get_unique_id() store_name = group_name + types.named_actor_suffix - store = NCCLUniqueIDStore\ - .options(name=store_name, lifetime="detached")\ - .remote() + store = NCCLUniqueIDStore.options(name=store_name, lifetime="detached").remote() ray.wait([store.set_id.remote(group_uid)]) + logging.info('creating NCCL group: {}'.format(group_name)) g = NCCLGroup(world_size, rank, group_name) self._name_group_map[group_name] = g self._group_name_map[g] = group_name @@ -230,8 +229,8 @@ def init_collective_group(backend, raise RuntimeError('Trying to initialize a group twice.') assert(world_size > 0) - assert(rank >= 0 and rank < world_size) - + assert(rank >= 0 ) + assert(rank < world_size) _group_mgr.create_collective_group(backend, world_size, rank, group_name) diff --git a/python/ray/util/collective/collective_group/__init__.py b/python/ray/util/collective/collective_group/__init__.py index 0663625d56ab..b2b3ab3ee7ee 100644 --- a/python/ray/util/collective/collective_group/__init__.py +++ b/python/ray/util/collective/collective_group/__init__.py @@ -1,2 +1 @@ from .nccl_collective_group import NCCLGroup -from .mpi_collective_group import MPIGroup diff --git a/python/ray/util/collective/collective_group/base_collective_group.py b/python/ray/util/collective/collective_group/base_collective_group.py index c5717b4a1776..55835e6430cd 100644 --- a/python/ray/util/collective/collective_group/base_collective_group.py +++ b/python/ray/util/collective/collective_group/base_collective_group.py @@ -3,7 +3,7 @@ from abc import abstractmethod import six -from python.ray.util.collective.types import AllReduceOptions, BarrierOptions +from ray.util.collective.types import AllReduceOptions, BarrierOptions class BaseGroup(six.with_metaclass(ABCMeta)): @@ -30,11 +30,15 @@ def world_size(self): """Return the number of processes in this group.""" return self._world_size + @property + def group_name(self): + """Return the group name of this group.""" + return self._group_name + def destroy_group(self): """GC the communicators.""" pass - @abstractmethod @classmethod def backend(cls): """The backend of this collective group.""" diff --git a/python/ray/util/collective/collective_group/mpi_collective_group.py b/python/ray/util/collective/collective_group/mpi_collective_group.py index 58e9b7279e33..92e40ca7847b 100644 --- a/python/ray/util/collective/collective_group/mpi_collective_group.py +++ b/python/ray/util/collective/collective_group/mpi_collective_group.py @@ -3,5 +3,3 @@ import mpi4py except ImportError: raise - -raise NotImplementedError() diff --git a/python/ray/util/collective/collective_group/nccl_collective_group.py b/python/ray/util/collective/collective_group/nccl_collective_group.py index 08c28733fcbc..5df26979ade2 100644 --- a/python/ray/util/collective/collective_group/nccl_collective_group.py +++ b/python/ray/util/collective/collective_group/nccl_collective_group.py @@ -2,9 +2,9 @@ import ray import cupy -from python.ray.util.collective.collective_group import nccl_util -from python.ray.util.collective.collective_group.base_collective_group import BaseGroup -from python.ray.util.collective.types import AllReduceOptions, BarrierOptions, named_actor_suffix +from ray.util.collective.collective_group import nccl_util +from ray.util.collective.collective_group.base_collective_group import BaseGroup +from ray.util.collective.types import AllReduceOptions, BarrierOptions, named_actor_suffix # TODO(Hao): @@ -15,7 +15,7 @@ class NCCLGroup(BaseGroup): def __init__(self, world_size, rank, group_name): """Init an NCCL collective group.""" - super(NCCLGroup, self).__init__(self, world_size, rank, group_name) + super(NCCLGroup, self).__init__(world_size, rank, group_name) self._nccl_uid_store = None self._nccl_uid = None @@ -42,7 +42,9 @@ def _init_nccl_unique_id(self): unique_actor_name = self.group_name + named_actor_suffix # Assuming this named actor has been created. + print('reach here...1') self._nccl_uid_store = ray.get_actor(unique_actor_name) + print('reach here...1') self._nccl_uid = ray.get(self._nccl_uid_store.get_id.remote()) @property diff --git a/python/ray/util/collective/collective_group/nccl_util.py b/python/ray/util/collective/collective_group/nccl_util.py index 67c595dee930..eef3e2cf4a04 100644 --- a/python/ray/util/collective/collective_group/nccl_util.py +++ b/python/ray/util/collective/collective_group/nccl_util.py @@ -11,7 +11,7 @@ except ImportError: raise ImportError('NCCL in Ray requires Cupy being available!') -from python.ray.util.collective.types import ReduceOp, torch_available +from ray.util.collective.types import ReduceOp, torch_available NCCL_REDUCE_OP_MAP = { diff --git a/python/ray/util/collective/examples/nccl_allreduce_example.py b/python/ray/util/collective/examples/nccl_allreduce_example.py new file mode 100644 index 000000000000..8dc444d0c035 --- /dev/null +++ b/python/ray/util/collective/examples/nccl_allreduce_example.py @@ -0,0 +1,35 @@ + +import ray +import cupy as cp + +import ray.util.collective as collective + +@ray.remote(num_gpus=1) +class Worker: + def __init__(self): + self.send = cp.ones((4,), dtype=cp.float32) + self.recv = cp.zeros((4,), dtype=cp.float32) + + def setup(self, world_size, rank): + collective.init_collective_group('nccl', world_size, rank, 'default') + return True + + def compute(self): + collective.allreduce(self.send, 'default') + print(self.send) + return self.send + +if __name__ == "__main__": + ray.init(num_gpus=2) + + num_workers = 2 + workers = [] + init_rets = [] + for i in range(num_workers): + w = Worker.remote() + workers.append(w) + init_rets.append(w.setup.remote(num_workers, i)) + m = ray.get(init_rets) + # results = ray.get([w.compute.remote() for w in workers]) + # print(results) + ray.shutdown() \ No newline at end of file From d971237e2d89315e72ad537297f0e23b864fd5c0 Mon Sep 17 00:00:00 2001 From: Hao Zhang Date: Tue, 1 Dec 2020 13:53:21 -0500 Subject: [PATCH 14/51] add an addtional Rendezvous class to safely meet at named actor --- python/ray/util/collective/collective.py | 12 ++-- .../collective_group/nccl_collective_group.py | 62 +++++++++++++++---- 2 files changed, 57 insertions(+), 17 deletions(-) diff --git a/python/ray/util/collective/collective.py b/python/ray/util/collective/collective.py index ef62695eecb9..bd4c63bc66bd 100644 --- a/python/ray/util/collective/collective.py +++ b/python/ray/util/collective/collective.py @@ -18,6 +18,7 @@ except ImportError: _NCCL_AVAILABLE = False +logging.getLogger().setLevel(logging.DEBUG) def nccl_available(): return _NCCL_AVAILABLE @@ -82,10 +83,11 @@ def create_collective_group(self, import cupy.cuda.nccl as nccl group_uid = nccl.get_unique_id() store_name = group_name + types.named_actor_suffix - store = NCCLUniqueIDStore.options(name=store_name, lifetime="detached").remote() + + store = NCCLUniqueIDStore.options(name=store_name, lifetime="detached").remote(store_name) ray.wait([store.set_id.remote(group_uid)]) - logging.info('creating NCCL group: {}'.format(group_name)) + logging.debug('creating NCCL group: {}'.format(group_name)) g = NCCLGroup(world_size, rank, group_name) self._name_group_map[group_name] = g self._group_name_map[g] = group_name @@ -199,8 +201,9 @@ def destroy_collective_group(self, group_name): ray.kill(store) g.destroy() + _group_mgr = GroupManager() -_group_mgr2 = GroupMagager_2() +_group_mgr2 = GroupManager_2() def init_collective_group(backend, world_size, @@ -264,6 +267,7 @@ def declare_collective_group(actors, group_options): _group_mgr_2.create_collective_group(backend, world_size, rank, group_name, actors) + def allreduce(tensor, group_name, op=types.ReduceOp.SUM): @@ -306,5 +310,3 @@ def _check_and_get_group(group_name): # TODO(Hao): check if this rank is in the group. g = _group_mgr.get_group_by_name(group_name) return g - - diff --git a/python/ray/util/collective/collective_group/nccl_collective_group.py b/python/ray/util/collective/collective_group/nccl_collective_group.py index 5df26979ade2..99253ba30054 100644 --- a/python/ray/util/collective/collective_group/nccl_collective_group.py +++ b/python/ray/util/collective/collective_group/nccl_collective_group.py @@ -1,4 +1,7 @@ import logging +import datetime +import time + import ray import cupy @@ -11,25 +14,66 @@ # (1) stream management, instead of using the default stream, using a dedicate stream # (2) communicator management, adding a caching mechanism to enable +class Rendezvous: + def __init__(self, group_name): + if not group_name: + raise ValueError('Empty meeting point.') + self._group_name = group_name + self._store_name = None + self._store = None + + def meet_at_store(self, timeout=180): + """Meet at the named actor store.""" + if timeout is not None and timeout < 0: + raise ValueError("The 'timeout' argument must be nonnegative. " + f"Received {timeout}") + self._store_name = self._group_name + named_actor_suffix + timeout_delta = datetime.timedelta(seconds=timeout) + elapsed = datetime.timedelta(seconds=0) + start_time = datetime.datetime.now() + while elapsed < timeout_delta: + try: + logging.debug("Trying to meet at the store '{}'".format(self._store_name)) + self._store = ray.get_actor(self._store_name) + except ValueError: + logging.debug("Failed to meet at the store '{}'." + "Trying again...".format(self._store_name)) + time.sleep(1) + elapsed = datetime.datetime.now() - start_time + continue + break + if not self._store: + raise RuntimeError("Unable to meet other processes " + "at the rendezvous store.") + + @property + def store(self): + return self._store + + def get_nccl_id(self): + if not self._store: + raise ValueError("Rendezvous store is not setup.") + uid = ray.get(self._store.get_id.remote()) + return uid class NCCLGroup(BaseGroup): def __init__(self, world_size, rank, group_name): """Init an NCCL collective group.""" super(NCCLGroup, self).__init__(world_size, rank, group_name) - self._nccl_uid_store = None self._nccl_uid = None # TODO(Hao): change this to a be a cache self._nccl_comm = None - # Check NCCL version if nccl_util.get_nccl_build_version() < 2000: raise RuntimeError('NCCL in Ray requires NCCL>=2.0.') - # TODO(Hao): check version here if nccl_util.get_nccl_runtime_version() < 2704: logging.warning('NCCL send/recv calls requires NCCL>=2.7.4') + self._rendezvous = Rendezvous(self.group_name) + self._rendezvous.meet_at_store() + # Setup the nccl uid using the store self._init_nccl_unique_id() @@ -38,20 +82,14 @@ def __init__(self, world_size, rank, group_name): def _init_nccl_unique_id(self): """Init the NCCL unique ID required for setting up NCCL communicator.""" - # using group_name to query the UniqueIDActor - unique_actor_name = self.group_name + named_actor_suffix + self._nccl_uid = self._rendezvous.get_nccl_id() - # Assuming this named actor has been created. - print('reach here...1') - self._nccl_uid_store = ray.get_actor(unique_actor_name) - print('reach here...1') - self._nccl_uid = ray.get(self._nccl_uid_store.get_id.remote()) @property def nccl_uid(self): return self._nccl_uid - def destory_group(self): + def destroy_group(self): """Destroy the group and release the NCCL communicators safely.""" if self._nccl_comm is not None: self.barrier() @@ -59,7 +97,7 @@ def destory_group(self): stream = self._get_cuda_stream() stream.synchronize() # destroy the communicator - self._nccl_comm.destory() + self._nccl_comm.destroy() self._nccl_comm = None super(NCCLGroup, self).destroy_group() From 3f2f86bb95cce00bfe145ddf0245ebb9d1ba2223 Mon Sep 17 00:00:00 2001 From: Hao Zhang Date: Tue, 1 Dec 2020 14:11:48 -0500 Subject: [PATCH 15/51] fix some small bugs in nccl_util --- .../ray/util/collective/collective_group/nccl_util.py | 6 +----- .../util/collective/examples/nccl_allreduce_example.py | 10 ++++++++-- 2 files changed, 9 insertions(+), 7 deletions(-) diff --git a/python/ray/util/collective/collective_group/nccl_util.py b/python/ray/util/collective/collective_group/nccl_util.py index eef3e2cf4a04..ec23218af9c1 100644 --- a/python/ray/util/collective/collective_group/nccl_util.py +++ b/python/ray/util/collective/collective_group/nccl_util.py @@ -79,7 +79,7 @@ def get_nccl_reduce_op(reduce_op): def get_nccl_tensor_dtype(tensor): """Return the corresponded NCCL dtype given a tensor.""" if isinstance(tensor, cupy.ndarray): - return NUMPY_NCCL_DTYPE_MAP[tensor.dtype] + return NUMPY_NCCL_DTYPE_MAP[tensor.dtype.type] if torch_available(): if isinstance(tensor, torch.Tensor): return TORCH_NCCL_DTYPE_MAP[tensor.dtype] @@ -88,8 +88,6 @@ def get_nccl_tensor_dtype(tensor): def get_tensor_ptr(tensor): """Return the pointer to the underlying memory storage of a tensor.""" - if not tensor: - raise RuntimeError('None Tensor.') if isinstance(tensor, cupy.ndarray): return tensor.data.ptr if isinstance(tensor, numpy.ndarray): @@ -102,8 +100,6 @@ def get_tensor_ptr(tensor): def get_tensor_n_elements(tensor): """Return the number of elements in a tensor.""" - if not tensor: - raise RuntimeError('None Tensor.') if isinstance(tensor, cupy.ndarray) or isinstance(tensor, numpy.ndarray): return tensor.size if torch_available(): diff --git a/python/ray/util/collective/examples/nccl_allreduce_example.py b/python/ray/util/collective/examples/nccl_allreduce_example.py index 8dc444d0c035..818d46f6ee06 100644 --- a/python/ray/util/collective/examples/nccl_allreduce_example.py +++ b/python/ray/util/collective/examples/nccl_allreduce_example.py @@ -19,7 +19,13 @@ def compute(self): print(self.send) return self.send + def destroy(self): + collective.destroy_group('') + if __name__ == "__main__": + + send = cp.ones((4, ), dtype=cp.float32) + ray.init(num_gpus=2) num_workers = 2 @@ -29,7 +35,7 @@ def compute(self): w = Worker.remote() workers.append(w) init_rets.append(w.setup.remote(num_workers, i)) - m = ray.get(init_rets) - # results = ray.get([w.compute.remote() for w in workers]) + # m = ray.get(init_rets) + results = ray.get([w.compute.remote() for w in workers]) # print(results) ray.shutdown() \ No newline at end of file From 135b9ec8e4ce79f9c696e5aa26cc2289161ed091 Mon Sep 17 00:00:00 2001 From: Hao Zhang Date: Wed, 2 Dec 2020 01:07:40 -0500 Subject: [PATCH 16/51] some small fix --- python/ray/util/collective/collective.py | 7 ++-- .../collective_group/base_collective_group.py | 32 ------------------- .../collective_group/nccl_collective_group.py | 15 ++++++--- python/ray/util/collective/const.py | 6 ++++ python/ray/util/collective/types.py | 3 -- 5 files changed, 20 insertions(+), 43 deletions(-) create mode 100644 python/ray/util/collective/const.py diff --git a/python/ray/util/collective/collective.py b/python/ray/util/collective/collective.py index bd4c63bc66bd..3cb697b78f09 100644 --- a/python/ray/util/collective/collective.py +++ b/python/ray/util/collective/collective.py @@ -3,6 +3,7 @@ import ray from ray.util.collective import types +from ray.util.collective.const import NAMED_ACTOR_STORE_SUFFIX # Get the availability information first by importing information _MPI_AVAILABLE = True @@ -82,7 +83,7 @@ def create_collective_group(self, if rank == 0: import cupy.cuda.nccl as nccl group_uid = nccl.get_unique_id() - store_name = group_name + types.named_actor_suffix + store_name = group_name + NAMED_ACTOR_STORE_SUFFIX store = NCCLUniqueIDStore.options(name=store_name, lifetime="detached").remote(store_name) ray.wait([store.set_id.remote(group_uid)]) @@ -123,7 +124,7 @@ def destroy_collective_group(self, group_name): if backend == 'nccl': # release the named actor if rank == 0: - store_name = group_name + types.named_actor_suffix + store_name = group_name + NAMED_ACTOR_STORE_SUFFIX store = ray.get_actor(store_name) ray.wait([store.__ray_terminate__.remote()]) ray.kill(store) @@ -195,7 +196,7 @@ def destroy_collective_group(self, group_name): if backend == 'nccl': # release the named actor if rank == 0: - store_name = group_name + types.named_actor_suffix + store_name = group_name + NAMED_ACTOR_STORE_SUFFIX store = ray.get_actor(store_name) ray.wait([store.__ray_terminate__.remote()]) ray.kill(store) diff --git a/python/ray/util/collective/collective_group/base_collective_group.py b/python/ray/util/collective/collective_group/base_collective_group.py index 55835e6430cd..fa8ea410d3ff 100644 --- a/python/ray/util/collective/collective_group/base_collective_group.py +++ b/python/ray/util/collective/collective_group/base_collective_group.py @@ -51,35 +51,3 @@ def allreduce(self, tensor, allreduce_options=AllReduceOptions()): @abstractmethod def barrier(self, barrier_options=BarrierOptions()): raise NotImplementedError() - - # @abstractmethod - # def reduce(self, tensor, dst, op=SUM, group_name='default'): - # raise NotImplementedError() - # - # @abstractmethod - # def allgather(self, tensor_list, tensor, gropu_name='default'): - # raise NotImplementedError() - # - # @abstractmethod - # def gather(self, tensor, gather_list=None, dst=0, group_name='default'): - # raise NotImplementedError() - # - # @abstractmethod - # def scatter(self, tensor, scatter_list=None, src=0, group_name='default'): - # raise NotImplementedError() - # - # @abstractmethod - # def barrier(self, group_name='default'): - # raise NotImplementedError() - - # @abstractmethod - # def send(self): - # raise NotImplementedError() - # - # @abstractmethod - # def recv(self): - # raise NotImplementedError() - # - # @abstractmethod - # def broadcast(self): - # raise NotImplementedError() diff --git a/python/ray/util/collective/collective_group/nccl_collective_group.py b/python/ray/util/collective/collective_group/nccl_collective_group.py index 99253ba30054..14fc99b25adc 100644 --- a/python/ray/util/collective/collective_group/nccl_collective_group.py +++ b/python/ray/util/collective/collective_group/nccl_collective_group.py @@ -7,9 +7,12 @@ from ray.util.collective.collective_group import nccl_util from ray.util.collective.collective_group.base_collective_group import BaseGroup -from ray.util.collective.types import AllReduceOptions, BarrierOptions, named_actor_suffix +from ray.util.collective.types import AllReduceOptions, BarrierOptions +from ray.util.collective.const import NAMED_ACTOR_STORE_SUFFIX +logger = logging.getLogger(__name__) + # TODO(Hao): # (1) stream management, instead of using the default stream, using a dedicate stream # (2) communicator management, adding a caching mechanism to enable @@ -27,20 +30,21 @@ def meet_at_store(self, timeout=180): if timeout is not None and timeout < 0: raise ValueError("The 'timeout' argument must be nonnegative. " f"Received {timeout}") - self._store_name = self._group_name + named_actor_suffix + self._store_name = self._group_name + NAMED_ACTOR_STORE_SUFFIX timeout_delta = datetime.timedelta(seconds=timeout) elapsed = datetime.timedelta(seconds=0) start_time = datetime.datetime.now() while elapsed < timeout_delta: try: - logging.debug("Trying to meet at the store '{}'".format(self._store_name)) + logger.debug("Trying to meet at the store '{}'".format(self._store_name)) self._store = ray.get_actor(self._store_name) except ValueError: - logging.debug("Failed to meet at the store '{}'." + logger.debug("Failed to meet at the store '{}'." "Trying again...".format(self._store_name)) time.sleep(1) elapsed = datetime.datetime.now() - start_time continue + logger.debug("Successful rendezvous!") break if not self._store: raise RuntimeError("Unable to meet other processes " @@ -56,6 +60,7 @@ def get_nccl_id(self): uid = ray.get(self._store.get_id.remote()) return uid + class NCCLGroup(BaseGroup): def __init__(self, world_size, rank, group_name): """Init an NCCL collective group.""" @@ -69,7 +74,7 @@ def __init__(self, world_size, rank, group_name): raise RuntimeError('NCCL in Ray requires NCCL>=2.0.') # TODO(Hao): check version here if nccl_util.get_nccl_runtime_version() < 2704: - logging.warning('NCCL send/recv calls requires NCCL>=2.7.4') + logger.warning('NCCL send/recv calls requires NCCL>=2.7.4') self._rendezvous = Rendezvous(self.group_name) self._rendezvous.meet_at_store() diff --git a/python/ray/util/collective/const.py b/python/ray/util/collective/const.py new file mode 100644 index 000000000000..3be8ebac6541 --- /dev/null +++ b/python/ray/util/collective/const.py @@ -0,0 +1,6 @@ +""" +Constants. + +Contains constants used to setup collective groups. +""" +NAMED_ACTOR_STORE_SUFFIX = '_unique_id_actor' diff --git a/python/ray/util/collective/types.py b/python/ray/util/collective/types.py index 366f8f805c11..19c68bd65bd0 100644 --- a/python/ray/util/collective/types.py +++ b/python/ray/util/collective/types.py @@ -52,6 +52,3 @@ class AllReduceOptions: class BarrierOptions: timeout = unset_timeout - - -named_actor_suffix = '_unique_id_actor' \ No newline at end of file From 03e49e7d01e07d6547b371f60b001a58ecfe6dbd Mon Sep 17 00:00:00 2001 From: Hao Zhang Date: Wed, 2 Dec 2020 14:52:31 -0500 Subject: [PATCH 17/51] add a Backend class to make Backend string more robust --- python/ray/util/collective/collective.py | 166 +++--------------- .../collective_group/nccl_collective_group.py | 8 +- python/ray/util/collective/types.py | 16 ++ python/ray/util/collective/util.py | 22 +++ 4 files changed, 69 insertions(+), 143 deletions(-) create mode 100644 python/ray/util/collective/util.py diff --git a/python/ray/util/collective/collective.py b/python/ray/util/collective/collective.py index 3cb697b78f09..e944039841fa 100644 --- a/python/ray/util/collective/collective.py +++ b/python/ray/util/collective/collective.py @@ -19,7 +19,9 @@ except ImportError: _NCCL_AVAILABLE = False -logging.getLogger().setLevel(logging.DEBUG) +logger = logging.getLogger(__name__) +logger.setLevel(logging.DEBUG) + def nccl_available(): return _NCCL_AVAILABLE @@ -29,33 +31,6 @@ def mpi_available(): return _MPI_AVAILABLE -def _backend_check(backend): - if backend == 'mpi': - if not mpi_available(): - raise RuntimeError() - raise NotImplementedError() - elif backend == 'nccl': - if not nccl_available(): - raise RuntimeError() - - -@ray.remote -class NCCLUniqueIDStore(object): - """NCCLUniqueID. This class should be used as a named actor.""" - def __init__(self, name): - self.name = name - self.nccl_id = None - - def set_id(self, uid): - self.nccl_id = uid - return self.nccl_id - - def get_id(self): - if not self.nccl_id: - logging.warning('The NCCL ID has not been set yet for store {}'.format(self.name)) - return self.nccl_id - - class GroupManager(object): """ Use this class to manage the collective groups we created so far; @@ -72,23 +47,26 @@ def create_collective_group(self, rank, group_name): """ - The only entry to create new collective groups and register to the manager. + The entry to create new collective groups and register in the manager. Put the registration and the group information into the manager metadata as well. """ - if backend == 'mpi': + backend = types.Backend(backend) + if backend == types.Backend.MPI: raise NotImplementedError() - elif backend == 'nccl': + elif backend == types.Backend.NCCL: # create the ncclUniqueID if rank == 0: import cupy.cuda.nccl as nccl group_uid = nccl.get_unique_id() store_name = group_name + NAMED_ACTOR_STORE_SUFFIX + # Avoid a potential circular dependency in ray/actor.py + from ray.util.collective.util import NCCLUniqueIDStore store = NCCLUniqueIDStore.options(name=store_name, lifetime="detached").remote(store_name) ray.wait([store.set_id.remote(group_uid)]) - logging.debug('creating NCCL group: {}'.format(group_name)) + logger.debug('creating NCCL group: {}'.format(group_name)) g = NCCLGroup(world_size, rank, group_name) self._name_group_map[group_name] = g self._group_name_map[g] = group_name @@ -108,7 +86,7 @@ def get_group_by_name(self, group_name): def destroy_collective_group(self, group_name): """Group destructor.""" if group_name not in self._name_group_map: - logging.warning('The group {} does not exist'.format(group_name)) + logger.warning('The group {} does not exist'.format(group_name)) return # release the collective group resource @@ -121,79 +99,7 @@ def destroy_collective_group(self, group_name): del self._group_name_map[g] del self._name_group_map[group_name] - if backend == 'nccl': - # release the named actor - if rank == 0: - store_name = group_name + NAMED_ACTOR_STORE_SUFFIX - store = ray.get_actor(store_name) - ray.wait([store.__ray_terminate__.remote()]) - ray.kill(store) - g.destroy() - - -class GroupManager_2(object): - """ - Use this class to manage the collective groups we created so far; - For interface 2.2 - - """ - def __init__(self): - """Put some necessary meta information here.""" - self._name_group_map = {} - self._group_name_map = {} - - def create_collective_group(self, - backend, - world_size, - rank, - group_name): - """ - The only entry to create new collective groups and register to the manager. - - Put the registration and the group information into the manager metadata as well. - """ - if backend == 'mpi': - raise NotImplementedError() - elif backend == 'nccl': - # create the ncclUniqueID - #if rank == 0: - import cupy.cuda.nccl as nccl - group_uid = nccl.get_unique_id() - - for r in rank: - g = NCCLGroup(world_size, r, group_name) - self._name_group_map[group_name] = g - self._group_name_map[g] = group_name - return self._name_group_map[group_name] - - def is_group_exist(self, group_name): - if group_name in self._name_group_map: - return True - return False - - def get_group_by_name(self, group_name): - """Get the collective group handle by its name.""" - if group_name not in self._name_group_map: - return None - return self._name_group_map[group_name] - - def destroy_collective_group(self, group_name): - """Group destructor.""" - if group_name not in self._name_group_map: - logging.warning('The group {} does not exist'.format(group_name)) - return - - # release the collective group resource - g = self._name_group_map[group_name] - - rank = g.rank - backend = g.backend() - - # clean up the dicts - del self._group_name_map[g] - del self._name_group_map[group_name] - - if backend == 'nccl': + if backend == types.Backend.NCCL: # release the named actor if rank == 0: store_name = group_name + NAMED_ACTOR_STORE_SUFFIX @@ -204,7 +110,7 @@ def destroy_collective_group(self, group_name): _group_mgr = GroupManager() -_group_mgr2 = GroupManager_2() + def init_collective_group(backend, world_size, @@ -221,9 +127,10 @@ def init_collective_group(backend, group_name: Returns: - + None """ - _backend_check(backend) + backend = types.Backend(backend) + _check_backend_availability(backend) global _group_mgr # TODO(Hao): implement a group auto-counter. if not group_name: @@ -238,37 +145,6 @@ def init_collective_group(backend, _group_mgr.create_collective_group(backend, world_size, rank, group_name) -def declare_collective_group(actors, group_options): - """ - # Frontend API #2: - # This API is supported to work in the driver program - the users declare a list of actors as a collective group - # @Dacheng: This API is not in the right shape, need to work with ray.remote(), please figure out. - Args: - actors: - group_options: - - Returns: - - """ - global _group_mgr_2 - try: - group_name = group_options["group_name"] - world_size = group_options["world_size"] - rank = group_options["rank"] - backend = group_options["backend"] - except: - raise ValueError("group options incomplete") - - _backend_check(backend) - if _group_mgr_2.is_group_exist(group_name): - raise RuntimeError('Trying to initialize a group twice.') - - assert(world_size > 0) - assert(rank >= 0 and rank < world_size) - - _group_mgr_2.create_collective_group(backend, world_size, rank, group_name, actors) - - def allreduce(tensor, group_name, op=types.ReduceOp.SUM): @@ -311,3 +187,13 @@ def _check_and_get_group(group_name): # TODO(Hao): check if this rank is in the group. g = _group_mgr.get_group_by_name(group_name) return g + + +def _check_backend_availability(backend: types.Backend): + """Check whether the backend is available.""" + if backend == types.Backend.MPI: + if not mpi_available(): + raise RuntimeError('MPI is not available.') + elif backend == types.Backend.NCCL: + if not nccl_available(): + raise RuntimeError('NCCL is not available.') diff --git a/python/ray/util/collective/collective_group/nccl_collective_group.py b/python/ray/util/collective/collective_group/nccl_collective_group.py index 14fc99b25adc..3f06d89c09d7 100644 --- a/python/ray/util/collective/collective_group/nccl_collective_group.py +++ b/python/ray/util/collective/collective_group/nccl_collective_group.py @@ -7,7 +7,7 @@ from ray.util.collective.collective_group import nccl_util from ray.util.collective.collective_group.base_collective_group import BaseGroup -from ray.util.collective.types import AllReduceOptions, BarrierOptions +from ray.util.collective.types import AllReduceOptions, BarrierOptions, Backend from ray.util.collective.const import NAMED_ACTOR_STORE_SUFFIX @@ -17,6 +17,7 @@ # (1) stream management, instead of using the default stream, using a dedicate stream # (2) communicator management, adding a caching mechanism to enable + class Rendezvous: def __init__(self, group_name): if not group_name: @@ -108,7 +109,7 @@ def destroy_group(self): @classmethod def backend(cls): - return 'nccl' + return Backend.NCCL def allreduce(self, tensor, allreduce_options=AllReduceOptions()): """ @@ -159,7 +160,8 @@ def _get_nccl_communicator(self): self.world_size, self.nccl_uid, self.rank) return self._nccl_comm - def _get_cuda_stream(self): + @staticmethod + def _get_cuda_stream(): """Obtain an idle stream from a stream pool for the collective task.""" # TODO: implement a simple stream manager. return cupy.cuda.Stream.null diff --git a/python/ray/util/collective/types.py b/python/ray/util/collective/types.py index 19c68bd65bd0..98324c5ed322 100644 --- a/python/ray/util/collective/types.py +++ b/python/ray/util/collective/types.py @@ -34,6 +34,22 @@ def torch_available(): return _TORCH_AVAILABLE +class Backend(object): + """A class to represent different backends, in case the user string is too error-prone.""" + NCCL = 'nccl' + MPI = 'mpi' + UNRECOGNIZED = 'unrecognized' + + def __new__(cls, name: str): + backend = getattr(Backend, name.upper(), Backend.UNRECOGNIZED) + if backend == Backend.UNRECOGNIZED: + raise ValueError("Unrecognized backend: '{}'" + "Only NCCL is supported".format(name)) + if backend == Backend.MPI: + raise NotImplementedError() + return backend + + # TODO(Hao): extend this to support more MPI types class ReduceOp(Enum): SUM = 0 diff --git a/python/ray/util/collective/util.py b/python/ray/util/collective/util.py new file mode 100644 index 000000000000..acd76713afe9 --- /dev/null +++ b/python/ray/util/collective/util.py @@ -0,0 +1,22 @@ +import ray +import logging + + +logger = logging.getLogger(__name__) + + +@ray.remote +class NCCLUniqueIDStore(object): + """NCCLUniqueID Store as a named actor.""" + def __init__(self, name): + self.name = name + self.nccl_id = None + + def set_id(self, uid): + self.nccl_id = uid + return self.nccl_id + + def get_id(self): + if not self.nccl_id: + logger.warning('The NCCL ID has not been set yet for store {}'.format(self.name)) + return self.nccl_id From 558832223788cb8b2de265390f19c06141485fd4 Mon Sep 17 00:00:00 2001 From: Hao Zhang Date: Wed, 2 Dec 2020 18:13:05 -0500 Subject: [PATCH 18/51] add several useful APIs --- python/ray/util/collective/collective.py | 89 ++++++++++++++++++++---- python/ray/util/collective/types.py | 6 +- 2 files changed, 80 insertions(+), 15 deletions(-) diff --git a/python/ray/util/collective/collective.py b/python/ray/util/collective/collective.py index b8f9a80d1af4..cd10c7756d74 100644 --- a/python/ray/util/collective/collective.py +++ b/python/ray/util/collective/collective.py @@ -79,26 +79,25 @@ def is_group_exist(self, group_name): def get_group_by_name(self, group_name): """Get the collective group handle by its name.""" - if group_name not in self._name_group_map: + if not self.is_group_exist(group_name): + logger.warning("The group '{}' is not initialized.".format(group_name)) return None return self._name_group_map[group_name] def destroy_collective_group(self, group_name): """Group destructor.""" - if group_name not in self._name_group_map: - logger.warning('The group {} does not exist'.format(group_name)) + if not self.is_group_exist(group_name): + logger.warning("The group '{}' does not exist.".format(group_name)) return # release the collective group resource g = self._name_group_map[group_name] - rank = g.rank backend = g.backend() # clean up the dicts del self._group_name_map[g] del self._name_group_map[group_name] - if backend == types.Backend.NCCL: # release the named actor if rank == 0: @@ -106,12 +105,20 @@ def destroy_collective_group(self, group_name): store = ray.get_actor(store_name) ray.wait([store.__ray_terminate__.remote()]) ray.kill(store) + # Release the communicator resources g.destroy() _group_mgr = GroupManager() +def is_group_initialized(group_name): + """Check if the group is initialized in this process by the group name.""" + if not _group_mgr.is_group_exist(group_name): + return False + return True + + def init_collective_group(backend, world_size, rank, @@ -134,17 +141,60 @@ def init_collective_group(backend, global _group_mgr # TODO(Hao): implement a group auto-counter. if not group_name: - raise ValueError('group_name: {}, needs to be a string.'.format(group_name)) + raise ValueError("group_name '{}' needs to be a string." + .format(group_name)) if _group_mgr.is_group_exist(group_name): raise RuntimeError('Trying to initialize a group twice.') assert(world_size > 0) - assert(rank >= 0 ) + assert(rank >= 0) assert(rank < world_size) _group_mgr.create_collective_group(backend, world_size, rank, group_name) +def destroy_collective_group(group_name='default'): + """Destroy a collective group given its group name.""" + global _group_mgr + _group_mgr.destroy_collective_group(group_name) + + +def get_rank(group_name='default') -> int: + """ + Return the rank of this process in the given group. + + Args: + group_name (str): the name of the group to query + + Returns: + the rank of this process in the named group, + -1 if the group does not exist or the process does + not belong to the group. + """ + if not is_group_initialized(group_name): + return -1 + g = _group_mgr.get_group_by_name(group_name) + return g.rank + + +def get_world_size(group_name='default') -> int: + """ + Return the size of the collective gropu with the given name. + + Args: + group_name: the name of the group to query + + Returns: + The world size of the collective group, + -1 if the group does not exist or the process does + not belong to the group. + """ + if not is_group_initialized(group_name): + return -1 + g = _group_mgr.get_group_by_name(group_name) + return g.world_size + + def allreduce(tensor, group_name, op=types.ReduceOp.SUM): @@ -153,12 +203,13 @@ def allreduce(tensor, Args: tensor: - group_name (string): + group_name (str): op: Returns: None """ + _check_single_tensor_input(tensor) g = _check_and_get_group(group_name) opts = types.AllReduceOptions opts.reduceOp = op @@ -181,10 +232,9 @@ def barrier(group_name): def _check_and_get_group(group_name): """Check the existence and return the group handle.""" - global _group_mgr - if not _group_mgr.is_group_exist(group_name): - raise ValueError('The collective group {} is not initialized.'.format(group_name)) - # TODO(Hao): check if this rank is in the group. + if not is_group_initialized(group_name): + raise RuntimeError("The collective group '{}' is not " + "initialized in the process.".format(group_name)) g = _group_mgr.get_group_by_name(group_name) return g @@ -197,3 +247,18 @@ def _check_backend_availability(backend: types.Backend): elif backend == types.Backend.NCCL: if not nccl_available(): raise RuntimeError('NCCL is not available.') + + +def _check_single_tensor_input(tensor): + """Check if the tensor is with a supported type.""" + if types.numpy_available(): + if isinstance(tensor, types.np.ndarry): + return + if types.cupy_available(): + if isinstance(tensor, types.cp.ndarray): + return + if types.torch_available(): + if isinstance(tensor, types.th.Tensor): + return + raise RuntimeError("Unrecognized tensor type. Supported types are: " + "np.ndarray, torch.Tensor, cupy.ndarray.") diff --git a/python/ray/util/collective/types.py b/python/ray/util/collective/types.py index 98324c5ed322..566bd69ab0bf 100644 --- a/python/ray/util/collective/types.py +++ b/python/ray/util/collective/types.py @@ -7,17 +7,17 @@ _CUPY_AVAILABLE = True try: - import numpy + import numpy as np except ImportError: _NUMPY_AVAILABLE = False try: - import torch + import torch as th except ImportError: _TORCH_AVAILABLE = False try: - import cupy + import cupy as cp except ImportError: _CUPY_AVAILABLE = False From 49e59a3f55bdbf3486053d77ecd1bbc7242694eb Mon Sep 17 00:00:00 2001 From: Hao Zhang Date: Thu, 3 Dec 2020 19:50:18 -0500 Subject: [PATCH 19/51] add some tests --- python/ray/util/collective/__init__.py | 2 +- python/ray/util/collective/collective.py | 20 +-- .../collective_group/nccl_collective_group.py | 4 +- .../util/collective/scratch/import_outer.py | 6 + .../util/collective/scratch/import_test.py | 5 + .../ray/util/collective/scratch/importer.py | 4 + .../tests/{test_nccl_group.py => __init__.py} | 0 python/ray/util/collective/tests/conftest.py | 19 +++ .../collective/tests/test_collective_api.py | 144 ++++++++++++++++++ 9 files changed, 192 insertions(+), 12 deletions(-) create mode 100644 python/ray/util/collective/scratch/import_outer.py create mode 100644 python/ray/util/collective/scratch/import_test.py create mode 100644 python/ray/util/collective/scratch/importer.py rename python/ray/util/collective/tests/{test_nccl_group.py => __init__.py} (100%) create mode 100644 python/ray/util/collective/tests/conftest.py create mode 100644 python/ray/util/collective/tests/test_collective_api.py diff --git a/python/ray/util/collective/__init__.py b/python/ray/util/collective/__init__.py index 976ed700cbb8..e8b185a0796c 100644 --- a/python/ray/util/collective/__init__.py +++ b/python/ray/util/collective/__init__.py @@ -1 +1 @@ -from .collective import init_collective_group, allreduce, barrier \ No newline at end of file +from .collective import * diff --git a/python/ray/util/collective/collective.py b/python/ray/util/collective/collective.py index cd10c7756d74..89ac945863d8 100644 --- a/python/ray/util/collective/collective.py +++ b/python/ray/util/collective/collective.py @@ -106,7 +106,7 @@ def destroy_collective_group(self, group_name): ray.wait([store.__ray_terminate__.remote()]) ray.kill(store) # Release the communicator resources - g.destroy() + g.destroy_group() _group_mgr = GroupManager() @@ -119,18 +119,18 @@ def is_group_initialized(group_name): return True -def init_collective_group(backend, - world_size, - rank, - group_name='default'): +def init_collective_group(world_size: int, + rank: int, + backend=types.Backend.NCCL, + group_name: str = 'default'): """ Initialize a collective group inside an actor process. This is an Args: - backend: world_size: rank: + backend: group_name: Returns: @@ -144,6 +144,8 @@ def init_collective_group(backend, raise ValueError("group_name '{}' needs to be a string." .format(group_name)) + + if _group_mgr.is_group_exist(group_name): raise RuntimeError('Trying to initialize a group twice.') @@ -153,13 +155,13 @@ def init_collective_group(backend, _group_mgr.create_collective_group(backend, world_size, rank, group_name) -def destroy_collective_group(group_name='default'): +def destroy_collective_group(group_name: str = 'default') -> None: """Destroy a collective group given its group name.""" global _group_mgr _group_mgr.destroy_collective_group(group_name) -def get_rank(group_name='default') -> int: +def get_rank(group_name: str = 'default') -> int: """ Return the rank of this process in the given group. @@ -196,7 +198,7 @@ def get_world_size(group_name='default') -> int: def allreduce(tensor, - group_name, + group_name: str, op=types.ReduceOp.SUM): """ Collective allreduce the tensor across the group with name group_name. diff --git a/python/ray/util/collective/collective_group/nccl_collective_group.py b/python/ray/util/collective/collective_group/nccl_collective_group.py index 3f06d89c09d7..16c21afcc541 100644 --- a/python/ray/util/collective/collective_group/nccl_collective_group.py +++ b/python/ray/util/collective/collective_group/nccl_collective_group.py @@ -26,7 +26,7 @@ def __init__(self, group_name): self._store_name = None self._store = None - def meet_at_store(self, timeout=180): + def meet(self, timeout=180): """Meet at the named actor store.""" if timeout is not None and timeout < 0: raise ValueError("The 'timeout' argument must be nonnegative. " @@ -78,7 +78,7 @@ def __init__(self, world_size, rank, group_name): logger.warning('NCCL send/recv calls requires NCCL>=2.7.4') self._rendezvous = Rendezvous(self.group_name) - self._rendezvous.meet_at_store() + self._rendezvous.meet() # Setup the nccl uid using the store self._init_nccl_unique_id() diff --git a/python/ray/util/collective/scratch/import_outer.py b/python/ray/util/collective/scratch/import_outer.py new file mode 100644 index 000000000000..74678501c81b --- /dev/null +++ b/python/ray/util/collective/scratch/import_outer.py @@ -0,0 +1,6 @@ + +# try: +from import_test import A +# except ImportError: +# print('123') + diff --git a/python/ray/util/collective/scratch/import_test.py b/python/ray/util/collective/scratch/import_test.py new file mode 100644 index 000000000000..3caab93b0d11 --- /dev/null +++ b/python/ray/util/collective/scratch/import_test.py @@ -0,0 +1,5 @@ +import importer + +class A: + def __init__(self): + pass \ No newline at end of file diff --git a/python/ray/util/collective/scratch/importer.py b/python/ray/util/collective/scratch/importer.py new file mode 100644 index 000000000000..9c40b95931f7 --- /dev/null +++ b/python/ray/util/collective/scratch/importer.py @@ -0,0 +1,4 @@ +try: + import torch +except ImportError: + raise ImportError('Torch not installed..') \ No newline at end of file diff --git a/python/ray/util/collective/tests/test_nccl_group.py b/python/ray/util/collective/tests/__init__.py similarity index 100% rename from python/ray/util/collective/tests/test_nccl_group.py rename to python/ray/util/collective/tests/__init__.py diff --git a/python/ray/util/collective/tests/conftest.py b/python/ray/util/collective/tests/conftest.py new file mode 100644 index 000000000000..ac393365d351 --- /dev/null +++ b/python/ray/util/collective/tests/conftest.py @@ -0,0 +1,19 @@ +"""Some fixtures for collective tests.""" +import pytest + +import ray + + +@pytest.fixture +def ray_start_single_node_2_gpus(): + # Please start this fixture in a cluster with 2 GPUs. + address_info = ray.init(num_gpus=2) + yield address_info + # The code after the yield will run as teardown code. + ray.shutdown() + + +# TODO (Hao): implement this one. +@pytest.fixture +def ray_start_distributed_2_nodes_4_gpus(): + pass \ No newline at end of file diff --git a/python/ray/util/collective/tests/test_collective_api.py b/python/ray/util/collective/tests/test_collective_api.py new file mode 100644 index 000000000000..f4eaa395f7eb --- /dev/null +++ b/python/ray/util/collective/tests/test_collective_api.py @@ -0,0 +1,144 @@ +"""Test the collective group APIs.""" +import pytest + +import ray +import ray.util.collective as col +from ray.util.collective.types import Backend + +import cupy as cp + + +@ray.remote(num_gpus=1) +class Worker: + def __init__(self): + self.buffer = cp.ones((10,), dtype=cp.float32) + + def init_group(self, world_size, rank, backend=Backend.NCCL, group_name='default'): + col.init_collective_group(world_size, rank, backend, group_name) + return True + + def do_work(self): + return + + def destroy_group(self, group_name='default'): + col.destroy_collective_group(group_name) + return True + + def report_rank(self, group_name='default'): + rank = col.get_rank(group_name) + return rank + + def report_world_size(self, group_name='default'): + ws = col.get_world_size(group_name) + return ws + + def report_nccl_availability(self): + avail = col.nccl_available() + return avail + + def report_mpi_availability(self): + avail = col.mpi_available() + return avail + + def report_is_group_initialized(self, group_name='default'): + is_init = col.is_group_initialized(group_name) + return is_init + + +def get_actors_group(num_workers=2, group_name='default', backend='nccl'): + actors = [Worker.remote() for i in range(num_workers)] + world_size = num_workers + init_results = ray.get([actor.init_group.remote(world_size, i, backend, group_name) + for i, actor in enumerate(actors)]) + return actors, init_results + + +@pytest.mark.parametrize("group_name", ['default', 'test', '123?34!']) +def test_init_two_actors(ray_start_single_node_2_gpus, group_name): + world_size = 2 + actors, results = get_actors_group(world_size, group_name) + for i in range(world_size): + assert(results[i]) + + +def test_init_multiple_groups(ray_start_single_node_2_gpus): + world_size = 2 + num_groups = 10 + actors = [Worker.remote() for i in range(world_size)] + for i in range(num_groups): + group_name = str(i) + init_results = ray.get([actor.init_group.remote(world_size, i, group_name=group_name) + for i, actor in enumerate(actors)]) + for j in range(world_size): + assert init_results[j] + + +def test_misc_apis_2_actors(ray_start_single_node_2_gpus): + world_size = 2 + actors, _ = get_actors_group(world_size) + # test report rank + actor0_rank = ray.get(actors[0].report_rank.remote()) + assert actor0_rank == 0 + actor1_rank = ray.get(actors[1].report_rank.remote()) + assert actor1_rank == 1 + + # test world size + actor0_world_size = ray.get(actors[0].report_world_size.remote()) + actor1_world_size = ray.get(actors[0].report_world_size.remote()) + assert actor0_world_size == actor1_world_size == world_size + + # see whether the availability is allright + actor0_nccl_availability = ray.get(actors[0].report_nccl_availability.remote()) + assert(actor0_nccl_availability) + actor0_mpi_availability = ray.get(actors[0].report_mpi_availability.remote()) + assert(actor0_mpi_availability == False) + + # check group is_init + actor0_is_init = ray.get(actors[0].report_is_group_initialized.remote()) + assert actor0_is_init + actor0_is_init = ray.get(actors[0].report_is_group_initialized.remote('random')) + assert actor0_is_init == False + actor0_is_init = ray.get(actors[0].report_is_group_initialized.remote('123')) + assert actor0_is_init == False + actor1_is_init = ray.get(actors[0].report_is_group_initialized.remote()) + assert actor1_is_init + actor1_is_init = ray.get(actors[0].report_is_group_initialized.remote('456')) + assert actor1_is_init == False + + # Now destroy the group at actor0 + ray.wait([actors[0].destroy_group.remote()]) + actor0_is_init = ray.get(actors[0].report_is_group_initialized.remote()) + assert actor0_is_init == False + # should go well + ray.wait([actors[0].destroy_group.remote('random')]) + + actor1_is_init = ray.get(actors[1].report_is_group_initialized.remote()) + assert(actor1_is_init) + ray.wait([actors[1].destroy_group.remote('random')]) + actor1_is_init = ray.get(actors[1].report_is_group_initialized.remote()) + assert(actor1_is_init) + ray.wait([actors[1].destroy_group.remote('default')]) + actor1_is_init = ray.get(actors[1].report_is_group_initialized.remote()) + assert(actor1_is_init == False) + + # Now reconstruct the group using the same name + init_results = ray.get([actor.init_group.remote(world_size, i) for i, actor in enumerate(actors)]) + for i in range(world_size): + assert(init_results[i]) + actor0_is_init = ray.get(actors[0].report_is_group_initialized.remote()) + assert actor0_is_init + actor1_is_init = ray.get(actors[0].report_is_group_initialized.remote()) + assert actor1_is_init + +def test_reinit_group(ray_start_single_node_2_gpus): + pass + +# TODO (Dacheng): test the allreduce api with different groups, object size, etc. +def test_allreduce(ray_start_single_node_2_gpus): + pass + + +if __name__ == "__main__": + import pytest + import sys + sys.exit(pytest.main(["-v", "-x", __file__])) From be40e84bbf8fdaae895fa7a8841f95a06254a3a3 Mon Sep 17 00:00:00 2001 From: Dacheng Li Date: Thu, 3 Dec 2020 23:43:53 -0500 Subject: [PATCH 20/51] added allreduce test --- python/ray/util/collective/collective.py | 5 +- .../collective/collective_group/nccl_util.py | 4 +- .../collective/tests/test_collective_api.py | 102 ++++++++++++++++-- 3 files changed, 100 insertions(+), 11 deletions(-) diff --git a/python/ray/util/collective/collective.py b/python/ray/util/collective/collective.py index 89ac945863d8..1cf446029623 100644 --- a/python/ray/util/collective/collective.py +++ b/python/ray/util/collective/collective.py @@ -65,9 +65,10 @@ def create_collective_group(self, from ray.util.collective.util import NCCLUniqueIDStore store = NCCLUniqueIDStore.options(name=store_name, lifetime="detached").remote(store_name) ray.wait([store.set_id.remote(group_uid)]) - + logger.debug('creating NCCL group: {}'.format(group_name)) g = NCCLGroup(world_size, rank, group_name) + self._name_group_map[group_name] = g self._group_name_map[g] = group_name return self._name_group_map[group_name] @@ -254,7 +255,7 @@ def _check_backend_availability(backend: types.Backend): def _check_single_tensor_input(tensor): """Check if the tensor is with a supported type.""" if types.numpy_available(): - if isinstance(tensor, types.np.ndarry): + if isinstance(tensor, types.np.ndarray): return if types.cupy_available(): if isinstance(tensor, types.cp.ndarray): diff --git a/python/ray/util/collective/collective_group/nccl_util.py b/python/ray/util/collective/collective_group/nccl_util.py index ec23218af9c1..6483c6f483b2 100644 --- a/python/ray/util/collective/collective_group/nccl_util.py +++ b/python/ray/util/collective/collective_group/nccl_util.py @@ -24,7 +24,7 @@ # cupy types are the same with numpy types NUMPY_NCCL_DTYPE_MAP = { - numpy.bool: nccl.NCCL_UINT8, + numpy.uint8: nccl.NCCL_UINT8, numpy.float16: nccl.NCCL_FLOAT16, numpy.float32: nccl.NCCL_FLOAT32, numpy.float64: nccl.NCCL_FLOAT64, @@ -33,7 +33,7 @@ if torch_available(): import torch TORCH_NCCL_DTYPE_MAP = { - torch.bool: nccl.NCCL_UINT8, + torch.uint8: nccl.NCCL_UINT8, torch.float16: nccl.NCCL_FLOAT16, torch.float32: nccl.NCCL_FLOAT32, torch.float64: nccl.NCCL_FLOAT64, diff --git a/python/ray/util/collective/tests/test_collective_api.py b/python/ray/util/collective/tests/test_collective_api.py index f4eaa395f7eb..fdd90bb37efa 100644 --- a/python/ray/util/collective/tests/test_collective_api.py +++ b/python/ray/util/collective/tests/test_collective_api.py @@ -1,9 +1,9 @@ """Test the collective group APIs.""" import pytest - +import numpy as np import ray import ray.util.collective as col -from ray.util.collective.types import Backend +from ray.util.collective.types import Backend, ReduceOp import cupy as cp @@ -17,8 +17,12 @@ def init_group(self, world_size, rank, backend=Backend.NCCL, group_name='default col.init_collective_group(world_size, rank, backend, group_name) return True - def do_work(self): - return + def set_buffer(self, data): + self.buffer = data + + def do_work(self, group_name="default", op=ReduceOp.SUM): + col.allreduce(self.buffer, group_name, op) + return self.buffer def destroy_group(self, group_name='default'): col.destroy_collective_group(group_name) @@ -133,10 +137,94 @@ def test_misc_apis_2_actors(ray_start_single_node_2_gpus): def test_reinit_group(ray_start_single_node_2_gpus): pass -# TODO (Dacheng): test the allreduce api with different groups, object size, etc. -def test_allreduce(ray_start_single_node_2_gpus): - pass +@pytest.mark.parametrize("group_name", ['default', 'test', '123?34!']) +def test_allreduce_different_name(ray_start_single_node_2_gpus, group_name): + world_size = 2 + actors, _ = get_actors_group(num_workers=world_size,group_name=group_name) + results = ray.get([a.do_work.remote(group_name) for a in actors]) + assert (results[0] == cp.ones((10,), dtype=cp.float32) * world_size).all() + assert (results[1] == cp.ones((10,), dtype=cp.float32) * world_size).all() + +@pytest.mark.parametrize("array_size", [1, 15, 177]) +def test_allreduce_different_array_size(ray_start_single_node_2_gpus, array_size): + world_size = 2 + actors, _ = get_actors_group(world_size) + ray.wait([a.set_buffer.remote(cp.ones(array_size, dtype=cp.float32)) for a in actors]) + results = ray.get([a.do_work.remote() for a in actors]) + assert (results[0] == cp.ones((array_size,), dtype=cp.float32) * world_size).all() + assert (results[1] == cp.ones((array_size,), dtype=cp.float32) * world_size).all() + +def test_allreduce_destroy(ray_start_single_node_2_gpus, backend="nccl", group_name="default"): + world_size = 2 + actors, _ = get_actors_group(world_size) + results = ray.get([a.do_work.remote() for a in actors]) + assert (results[0] == cp.ones((10,), dtype=cp.float32) * world_size).all() + assert (results[1] == cp.ones((10,), dtype=cp.float32) * world_size).all() + + # destroy the group and try do work, should fail + ray.wait([a.destroy_group.remote() for a in actors]) + with pytest.raises(RuntimeError): + results = ray.get([a.do_work.remote() for a in actors]) + + # reinit the same group and all reduce + ray.get([actor.init_group.remote(world_size, i, backend, group_name) + for i, actor in enumerate(actors)]) + results = ray.get([a.do_work.remote() for a in actors]) + assert (results[0] == cp.ones((10,), dtype=cp.float32) * world_size * 2).all() + assert (results[1] == cp.ones((10,), dtype=cp.float32) * world_size * 2).all() + +def test_allreduce_multiple_group(ray_start_single_node_2_gpus, backend="nccl", num_groups=5): + world_size = 2 + actors, _ = get_actors_group(world_size) + for group_name in range(1, num_groups): + ray.get([actor.init_group.remote(world_size, i, backend, str(group_name)) + for i, actor in enumerate(actors)]) + for i in range(num_groups): + results = ray.get([a.do_work.remote() for a in actors]) + assert (results[0] == cp.ones((10,), dtype=cp.float32) * (world_size ** (i + 1))).all() + +def test_allreduce_different_op(ray_start_single_node_2_gpus): + world_size = 2 + actors, _ = get_actors_group(world_size) + + # check product + ray.wait([a.set_buffer.remote(cp.ones(10, dtype=cp.float32) * (i + 2)) for i, a in enumerate(actors)]) + results = ray.get([a.do_work.remote(op=ReduceOp.PRODUCT) for a in actors]) + assert (results[0] == cp.ones((10,), dtype=cp.float32) * 6).all() + assert (results[1] == cp.ones((10,), dtype=cp.float32) * 6).all() + + # check min + ray.wait([a.set_buffer.remote(cp.ones(10, dtype=cp.float32) * (i + 2)) for i, a in enumerate(actors)]) + results = ray.get([a.do_work.remote(op=ReduceOp.MIN) for a in actors]) + assert (results[0] == cp.ones((10,), dtype=cp.float32) * 2).all() + assert (results[1] == cp.ones((10,), dtype=cp.float32) * 2).all() + + # check max + ray.wait([a.set_buffer.remote(cp.ones(10, dtype=cp.float32) * (i + 2)) for i, a in enumerate(actors)]) + results = ray.get([a.do_work.remote(op=ReduceOp.MAX) for a in actors]) + assert (results[0] == cp.ones((10,), dtype=cp.float32) * 3).all() + assert (results[1] == cp.ones((10,), dtype=cp.float32) * 3).all() + + +@pytest.mark.parametrize("dtype", [cp.uint8, cp.float16, cp.float32, cp.float64]) +def test_allreduce_different_dtype(ray_start_single_node_2_gpus, dtype): + world_size = 2 + actors, _ = get_actors_group(world_size) + ray.wait([a.set_buffer.remote(cp.ones(10, dtype=dtype)) for a in actors]) + results = ray.get([a.do_work.remote() for a in actors]) + assert (results[0] == cp.ones((10,), dtype=dtype) * world_size).all() + assert (results[1] == cp.ones((10,), dtype=dtype) * world_size).all() + +def test_allreduce_different_torch_cupy(ray_start_single_node_2_gpus): + return + import torch + world_size = 2 + actors, _ = get_actors_group(world_size) + ray.wait([actors[0].set_buffer.remote(torch.ones(10,))]) + results = ray.get([a.do_work.remote() for a in actors]) + assert (results[0] == cp.ones((10,)) * world_size).all() + assert (results[1] == cp.ones((10,)) * world_size).all() if __name__ == "__main__": import pytest From 0133c6a23f84e4bf7d70415b363b944533d73df3 Mon Sep 17 00:00:00 2001 From: Dacheng Li Date: Thu, 3 Dec 2020 23:47:34 -0500 Subject: [PATCH 21/51] fix typos --- python/ray/util/collective/collective.py | 5 ++--- python/ray/util/collective/tests/test_collective_api.py | 1 - 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/python/ray/util/collective/collective.py b/python/ray/util/collective/collective.py index 1cf446029623..26eb2a51b6c2 100644 --- a/python/ray/util/collective/collective.py +++ b/python/ray/util/collective/collective.py @@ -65,10 +65,9 @@ def create_collective_group(self, from ray.util.collective.util import NCCLUniqueIDStore store = NCCLUniqueIDStore.options(name=store_name, lifetime="detached").remote(store_name) ray.wait([store.set_id.remote(group_uid)]) - + logger.debug('creating NCCL group: {}'.format(group_name)) - g = NCCLGroup(world_size, rank, group_name) - + g = NCCLGroup(world_size, rank, group_name) self._name_group_map[group_name] = g self._group_name_map[g] = group_name return self._name_group_map[group_name] diff --git a/python/ray/util/collective/tests/test_collective_api.py b/python/ray/util/collective/tests/test_collective_api.py index fdd90bb37efa..e79e98f59e0e 100644 --- a/python/ray/util/collective/tests/test_collective_api.py +++ b/python/ray/util/collective/tests/test_collective_api.py @@ -1,6 +1,5 @@ """Test the collective group APIs.""" import pytest -import numpy as np import ray import ray.util.collective as col from ray.util.collective.types import Backend, ReduceOp From cbeaafe9660a046f17dfeaaec5c0165aa8027330 Mon Sep 17 00:00:00 2001 From: Hao Zhang Date: Fri, 4 Dec 2020 03:23:55 -0500 Subject: [PATCH 22/51] fix several bugs found via unittests --- .../collective_group/nccl_collective_group.py | 19 +++++- .../collective/tests/test_collective_api.py | 67 +++++++++++++------ 2 files changed, 64 insertions(+), 22 deletions(-) diff --git a/python/ray/util/collective/collective_group/nccl_collective_group.py b/python/ray/util/collective/collective_group/nccl_collective_group.py index 16c21afcc541..273b3f01e5bf 100644 --- a/python/ray/util/collective/collective_group/nccl_collective_group.py +++ b/python/ray/util/collective/collective_group/nccl_collective_group.py @@ -41,7 +41,7 @@ def meet(self, timeout=180): self._store = ray.get_actor(self._store_name) except ValueError: logger.debug("Failed to meet at the store '{}'." - "Trying again...".format(self._store_name)) + "Trying again...".format(self._store_name)) time.sleep(1) elapsed = datetime.datetime.now() - start_time continue @@ -55,10 +55,23 @@ def meet(self, timeout=180): def store(self): return self._store - def get_nccl_id(self): + def get_nccl_id(self, timeout=180): + """Get the NCCLUniqueID from the store.""" if not self._store: raise ValueError("Rendezvous store is not setup.") - uid = ray.get(self._store.get_id.remote()) + uid = None + timeout_delta = datetime.timedelta(seconds=timeout) + elapsed = datetime.timedelta(seconds=0) + start_time = datetime.datetime.now() + while elapsed < timeout_delta: + uid = ray.get(self._store.get_id.remote()) + if not uid: + time.sleep(1) + elapsed = datetime.datetime.now() - start_time + continue + break + if not uid: + raise RuntimeError("Unable to get the NCCLUniqueID from the store.") return uid diff --git a/python/ray/util/collective/tests/test_collective_api.py b/python/ray/util/collective/tests/test_collective_api.py index e79e98f59e0e..e0aca86a45a6 100644 --- a/python/ray/util/collective/tests/test_collective_api.py +++ b/python/ray/util/collective/tests/test_collective_api.py @@ -76,75 +76,98 @@ def test_init_multiple_groups(ray_start_single_node_2_gpus): assert init_results[j] -def test_misc_apis_2_actors(ray_start_single_node_2_gpus): +def test_get_rank(ray_start_single_node_2_gpus): world_size = 2 actors, _ = get_actors_group(world_size) - # test report rank actor0_rank = ray.get(actors[0].report_rank.remote()) assert actor0_rank == 0 actor1_rank = ray.get(actors[1].report_rank.remote()) assert actor1_rank == 1 - # test world size + # create a second group with a different name, and different order of ranks. + new_group_name = 'default2' + _ = ray.get([actor.init_group.remote(world_size, world_size - 1 - i, group_name=new_group_name) + for i, actor in enumerate(actors)]) + actor0_rank = ray.get(actors[0].report_rank.remote(new_group_name)) + assert actor0_rank == 1 + actor1_rank = ray.get(actors[1].report_rank.remote(new_group_name)) + assert actor1_rank == 0 + + +def test_get_world_size(ray_start_single_node_2_gpus): + world_size = 2 + actors, _ = get_actors_group(world_size) actor0_world_size = ray.get(actors[0].report_world_size.remote()) - actor1_world_size = ray.get(actors[0].report_world_size.remote()) + actor1_world_size = ray.get(actors[1].report_world_size.remote()) assert actor0_world_size == actor1_world_size == world_size - # see whether the availability is allright + +def test_availability(ray_start_single_node_2_gpus): + world_size = 2 + actors, _ = get_actors_group(world_size) actor0_nccl_availability = ray.get(actors[0].report_nccl_availability.remote()) assert(actor0_nccl_availability) actor0_mpi_availability = ray.get(actors[0].report_mpi_availability.remote()) assert(actor0_mpi_availability == False) + +def test_is_group_initialized(ray_start_single_node_2_gpus): + world_size = 2 + actors, _ = get_actors_group(world_size) # check group is_init actor0_is_init = ray.get(actors[0].report_is_group_initialized.remote()) assert actor0_is_init actor0_is_init = ray.get(actors[0].report_is_group_initialized.remote('random')) - assert actor0_is_init == False + assert not actor0_is_init actor0_is_init = ray.get(actors[0].report_is_group_initialized.remote('123')) - assert actor0_is_init == False + assert not actor0_is_init actor1_is_init = ray.get(actors[0].report_is_group_initialized.remote()) assert actor1_is_init actor1_is_init = ray.get(actors[0].report_is_group_initialized.remote('456')) - assert actor1_is_init == False + assert not actor1_is_init + +def test_destroy_group(ray_start_single_node_2_gpus): + world_size = 2 + actors, _ = get_actors_group(world_size) # Now destroy the group at actor0 ray.wait([actors[0].destroy_group.remote()]) actor0_is_init = ray.get(actors[0].report_is_group_initialized.remote()) - assert actor0_is_init == False - # should go well + assert not actor0_is_init + + # should go well as the group `random` does not exist at all ray.wait([actors[0].destroy_group.remote('random')]) actor1_is_init = ray.get(actors[1].report_is_group_initialized.remote()) - assert(actor1_is_init) + assert actor1_is_init ray.wait([actors[1].destroy_group.remote('random')]) actor1_is_init = ray.get(actors[1].report_is_group_initialized.remote()) - assert(actor1_is_init) + assert actor1_is_init ray.wait([actors[1].destroy_group.remote('default')]) actor1_is_init = ray.get(actors[1].report_is_group_initialized.remote()) - assert(actor1_is_init == False) + assert actor1_is_init == False # Now reconstruct the group using the same name init_results = ray.get([actor.init_group.remote(world_size, i) for i, actor in enumerate(actors)]) for i in range(world_size): - assert(init_results[i]) + assert init_results[i] actor0_is_init = ray.get(actors[0].report_is_group_initialized.remote()) assert actor0_is_init actor1_is_init = ray.get(actors[0].report_is_group_initialized.remote()) assert actor1_is_init -def test_reinit_group(ray_start_single_node_2_gpus): - pass @pytest.mark.parametrize("group_name", ['default', 'test', '123?34!']) +# @pytest.mark.parametrize("group_name", ['123?34!']) def test_allreduce_different_name(ray_start_single_node_2_gpus, group_name): world_size = 2 - actors, _ = get_actors_group(num_workers=world_size,group_name=group_name) + actors, _ = get_actors_group(num_workers=world_size, group_name=group_name) results = ray.get([a.do_work.remote(group_name) for a in actors]) assert (results[0] == cp.ones((10,), dtype=cp.float32) * world_size).all() assert (results[1] == cp.ones((10,), dtype=cp.float32) * world_size).all() -@pytest.mark.parametrize("array_size", [1, 15, 177]) + +@pytest.mark.parametrize("array_size", [2, 2**5, 2**10, 2**15, 2**20]) def test_allreduce_different_array_size(ray_start_single_node_2_gpus, array_size): world_size = 2 actors, _ = get_actors_group(world_size) @@ -153,6 +176,7 @@ def test_allreduce_different_array_size(ray_start_single_node_2_gpus, array_size assert (results[0] == cp.ones((array_size,), dtype=cp.float32) * world_size).all() assert (results[1] == cp.ones((array_size,), dtype=cp.float32) * world_size).all() + def test_allreduce_destroy(ray_start_single_node_2_gpus, backend="nccl", group_name="default"): world_size = 2 actors, _ = get_actors_group(world_size) @@ -173,6 +197,7 @@ def test_allreduce_destroy(ray_start_single_node_2_gpus, backend="nccl", group_n assert (results[0] == cp.ones((10,), dtype=cp.float32) * world_size * 2).all() assert (results[1] == cp.ones((10,), dtype=cp.float32) * world_size * 2).all() + def test_allreduce_multiple_group(ray_start_single_node_2_gpus, backend="nccl", num_groups=5): world_size = 2 actors, _ = get_actors_group(world_size) @@ -180,9 +205,11 @@ def test_allreduce_multiple_group(ray_start_single_node_2_gpus, backend="nccl", ray.get([actor.init_group.remote(world_size, i, backend, str(group_name)) for i, actor in enumerate(actors)]) for i in range(num_groups): - results = ray.get([a.do_work.remote() for a in actors]) + group_name = 'default' if i == 0 else str(i) + results = ray.get([a.do_work.remote(group_name) for a in actors]) assert (results[0] == cp.ones((10,), dtype=cp.float32) * (world_size ** (i + 1))).all() + def test_allreduce_different_op(ray_start_single_node_2_gpus): world_size = 2 actors, _ = get_actors_group(world_size) @@ -215,6 +242,7 @@ def test_allreduce_different_dtype(ray_start_single_node_2_gpus, dtype): assert (results[0] == cp.ones((10,), dtype=dtype) * world_size).all() assert (results[1] == cp.ones((10,), dtype=dtype) * world_size).all() + def test_allreduce_different_torch_cupy(ray_start_single_node_2_gpus): return import torch @@ -225,6 +253,7 @@ def test_allreduce_different_torch_cupy(ray_start_single_node_2_gpus): assert (results[0] == cp.ones((10,)) * world_size).all() assert (results[1] == cp.ones((10,)) * world_size).all() + if __name__ == "__main__": import pytest import sys From 893142d0df2529312a47da77d8e659a8a9ebd9a6 Mon Sep 17 00:00:00 2001 From: Dacheng Li Date: Fri, 4 Dec 2020 13:06:10 -0500 Subject: [PATCH 23/51] fix and update torch test --- python/ray/util/collective/collective_group/nccl_util.py | 2 ++ python/ray/util/collective/tests/test_collective_api.py | 9 ++++++--- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/python/ray/util/collective/collective_group/nccl_util.py b/python/ray/util/collective/collective_group/nccl_util.py index 6483c6f483b2..5ef19a0263ae 100644 --- a/python/ray/util/collective/collective_group/nccl_util.py +++ b/python/ray/util/collective/collective_group/nccl_util.py @@ -94,6 +94,8 @@ def get_tensor_ptr(tensor): return tensor.data if torch_available(): if isinstance(tensor, torch.Tensor): + if not tensor.is_cuda: + raise RuntimeError('torch tensor must be on gpu.') return tensor.data_ptr() raise ValueError('Unsupported tensor type') diff --git a/python/ray/util/collective/tests/test_collective_api.py b/python/ray/util/collective/tests/test_collective_api.py index e0aca86a45a6..6d07fad44eb8 100644 --- a/python/ray/util/collective/tests/test_collective_api.py +++ b/python/ray/util/collective/tests/test_collective_api.py @@ -243,16 +243,19 @@ def test_allreduce_different_dtype(ray_start_single_node_2_gpus, dtype): assert (results[1] == cp.ones((10,), dtype=dtype) * world_size).all() -def test_allreduce_different_torch_cupy(ray_start_single_node_2_gpus): - return +def test_allreduce_torch_cupy(ray_start_single_node_2_gpus): import torch world_size = 2 actors, _ = get_actors_group(world_size) - ray.wait([actors[0].set_buffer.remote(torch.ones(10,))]) + ray.wait([actors[1].set_buffer.remote(torch.ones(10,).cuda())]) results = ray.get([a.do_work.remote() for a in actors]) assert (results[0] == cp.ones((10,)) * world_size).all() assert (results[1] == cp.ones((10,)) * world_size).all() + ray.wait([actors[0].set_buffer.remote(torch.ones(10,))]) + ray.wait([actors[1].set_buffer.remote(cp.ones(10,))]) + with pytest.raises(RuntimeError): + results = ray.get([a.do_work.remote() for a in actors]) if __name__ == "__main__": import pytest From ec1c07a01b8666d45063412b448e6e4fe2002b0b Mon Sep 17 00:00:00 2001 From: Dacheng Li Date: Fri, 4 Dec 2020 13:09:56 -0500 Subject: [PATCH 24/51] changed back actor --- python/ray/actor.py | 26 +++++++------------------- 1 file changed, 7 insertions(+), 19 deletions(-) diff --git a/python/ray/actor.py b/python/ray/actor.py index 513180d9cf6d..b8981ca3d23e 100644 --- a/python/ray/actor.py +++ b/python/ray/actor.py @@ -8,7 +8,7 @@ import ray.worker from ray.util.placement_group import ( PlacementGroup, check_placement_group_index, get_current_placement_group) -from ray.util.collective import collective as col + from ray import ActorClassID, Language from ray._raylet import PythonFunctionDescriptor from ray import cross_language @@ -423,8 +423,7 @@ def options(self, placement_group=None, placement_group_bundle_index=-1, placement_group_capture_child_tasks=None, - override_environment_variables=None, - collective=None): + override_environment_variables=None): """Configures and overrides the actor instantiation parameters. The arguments are the same as those that can be passed @@ -466,8 +465,7 @@ def remote(self, *args, **kwargs): placement_group_capture_child_tasks=( placement_group_capture_child_tasks), override_environment_variables=( - override_environment_variables), - collective=collective) + override_environment_variables)) return ActorOptionWrapper() @@ -488,8 +486,7 @@ def _remote(self, placement_group=None, placement_group_bundle_index=-1, placement_group_capture_child_tasks=None, - override_environment_variables=None, - collective=None): + override_environment_variables=None): """Create an actor. This method allows more flexibility than the remote method because @@ -529,7 +526,6 @@ def _remote(self, override_environment_variables: Environment variables to override and/or introduce for this actor. This is a dictionary mapping variable names to their values. - collective: what colletive configuration to use Returns: A handle to the newly created actor. @@ -679,12 +675,6 @@ def _remote(self, extension_data=str(actor_method_cpu), override_environment_variables=override_environment_variables or dict()) - - if collective: - col.init_collective_group(backend=collective["backend"], - world_size=collective["world_size"], - rank=collective["rank"], - group_name=collective["group_name"]) actor_handle = ActorHandle( meta.language, @@ -695,8 +685,7 @@ def _remote(self, actor_method_cpu, meta.actor_creation_function_descriptor, worker.current_session_and_job, - original_handle=True, - collective=collective) + original_handle=True) return actor_handle @@ -740,8 +729,7 @@ def __init__(self, actor_method_cpus, actor_creation_function_descriptor, session_and_job, - original_handle=False, - collective=None): + original_handle=False): self._ray_actor_language = language self._ray_actor_id = actor_id self._ray_original_handle = original_handle @@ -754,7 +742,7 @@ def __init__(self, self._ray_actor_creation_function_descriptor = \ actor_creation_function_descriptor self._ray_function_descriptor = {} - self._collective = collective + if not self._ray_is_cross_language: assert isinstance(actor_creation_function_descriptor, PythonFunctionDescriptor) From 8f15ba49410cc8fd641ffe25a31c1415bd8c3354 Mon Sep 17 00:00:00 2001 From: Hao Zhang Date: Fri, 4 Dec 2020 19:25:27 -0500 Subject: [PATCH 25/51] rearange a bit before importing distributed test --- python/ray/util/collective/tests/conftest.py | 2 +- .../tests/test_collective_2_nodes_4_gpus.py | 0 ... => test_collective_single_node_2_gpus.py} | 49 ++----------------- python/ray/util/collective/tests/util.py | 47 ++++++++++++++++++ 4 files changed, 53 insertions(+), 45 deletions(-) create mode 100644 python/ray/util/collective/tests/test_collective_2_nodes_4_gpus.py rename python/ray/util/collective/tests/{test_collective_api.py => test_collective_single_node_2_gpus.py} (87%) create mode 100644 python/ray/util/collective/tests/util.py diff --git a/python/ray/util/collective/tests/conftest.py b/python/ray/util/collective/tests/conftest.py index ac393365d351..62f17ce49a1c 100644 --- a/python/ray/util/collective/tests/conftest.py +++ b/python/ray/util/collective/tests/conftest.py @@ -16,4 +16,4 @@ def ray_start_single_node_2_gpus(): # TODO (Hao): implement this one. @pytest.fixture def ray_start_distributed_2_nodes_4_gpus(): - pass \ No newline at end of file + pass diff --git a/python/ray/util/collective/tests/test_collective_2_nodes_4_gpus.py b/python/ray/util/collective/tests/test_collective_2_nodes_4_gpus.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/python/ray/util/collective/tests/test_collective_api.py b/python/ray/util/collective/tests/test_collective_single_node_2_gpus.py similarity index 87% rename from python/ray/util/collective/tests/test_collective_api.py rename to python/ray/util/collective/tests/test_collective_single_node_2_gpus.py index 6d07fad44eb8..dbb0cccf75db 100644 --- a/python/ray/util/collective/tests/test_collective_api.py +++ b/python/ray/util/collective/tests/test_collective_single_node_2_gpus.py @@ -1,51 +1,12 @@ """Test the collective group APIs.""" import pytest import ray -import ray.util.collective as col -from ray.util.collective.types import Backend, ReduceOp +from ray.util.collective.types import ReduceOp import cupy as cp +import torch - -@ray.remote(num_gpus=1) -class Worker: - def __init__(self): - self.buffer = cp.ones((10,), dtype=cp.float32) - - def init_group(self, world_size, rank, backend=Backend.NCCL, group_name='default'): - col.init_collective_group(world_size, rank, backend, group_name) - return True - - def set_buffer(self, data): - self.buffer = data - - def do_work(self, group_name="default", op=ReduceOp.SUM): - col.allreduce(self.buffer, group_name, op) - return self.buffer - - def destroy_group(self, group_name='default'): - col.destroy_collective_group(group_name) - return True - - def report_rank(self, group_name='default'): - rank = col.get_rank(group_name) - return rank - - def report_world_size(self, group_name='default'): - ws = col.get_world_size(group_name) - return ws - - def report_nccl_availability(self): - avail = col.nccl_available() - return avail - - def report_mpi_availability(self): - avail = col.mpi_available() - return avail - - def report_is_group_initialized(self, group_name='default'): - is_init = col.is_group_initialized(group_name) - return is_init +from .util import Worker def get_actors_group(num_workers=2, group_name='default', backend='nccl'): @@ -244,19 +205,19 @@ def test_allreduce_different_dtype(ray_start_single_node_2_gpus, dtype): def test_allreduce_torch_cupy(ray_start_single_node_2_gpus): - import torch + # import torch world_size = 2 actors, _ = get_actors_group(world_size) ray.wait([actors[1].set_buffer.remote(torch.ones(10,).cuda())]) results = ray.get([a.do_work.remote() for a in actors]) assert (results[0] == cp.ones((10,)) * world_size).all() - assert (results[1] == cp.ones((10,)) * world_size).all() ray.wait([actors[0].set_buffer.remote(torch.ones(10,))]) ray.wait([actors[1].set_buffer.remote(cp.ones(10,))]) with pytest.raises(RuntimeError): results = ray.get([a.do_work.remote() for a in actors]) + if __name__ == "__main__": import pytest import sys diff --git a/python/ray/util/collective/tests/util.py b/python/ray/util/collective/tests/util.py new file mode 100644 index 000000000000..e273fa3d2a7e --- /dev/null +++ b/python/ray/util/collective/tests/util.py @@ -0,0 +1,47 @@ +import cupy as cp + +import ray +import ray.util.collective as col +from ray.util.collective.types import Backend, ReduceOp + + +@ray.remote(num_gpus=1) +class Worker: + def __init__(self): + self.buffer = cp.ones((10,), dtype=cp.float32) + + def init_group(self, world_size, rank, backend=Backend.NCCL, group_name='default'): + col.init_collective_group(world_size, rank, backend, group_name) + return True + + def set_buffer(self, data): + self.buffer = data + return self.buffer + + def do_work(self, group_name="default", op=ReduceOp.SUM): + col.allreduce(self.buffer, group_name, op) + return self.buffer + + def destroy_group(self, group_name='default'): + col.destroy_collective_group(group_name) + return True + + def report_rank(self, group_name='default'): + rank = col.get_rank(group_name) + return rank + + def report_world_size(self, group_name='default'): + ws = col.get_world_size(group_name) + return ws + + def report_nccl_availability(self): + avail = col.nccl_available() + return avail + + def report_mpi_availability(self): + avail = col.mpi_available() + return avail + + def report_is_group_initialized(self, group_name='default'): + is_init = col.is_group_initialized(group_name) + return is_init From 5b40ec3daa5b8d3218f9face4a6c50e4b930703b Mon Sep 17 00:00:00 2001 From: Hao Zhang Date: Fri, 4 Dec 2020 23:55:20 -0500 Subject: [PATCH 26/51] add distributed test --- python/ray/util/collective/collective.py | 7 +- .../collective_group/nccl_collective_group.py | 4 +- python/ray/util/collective/const.py | 7 + python/ray/util/collective/tests/conftest.py | 23 +- .../tests/ray_start_distributed_script.sh | 15 ++ .../tests/test_collective_2_nodes_4_gpus.py | 230 ++++++++++++++++++ python/ray/util/collective/util.py | 1 + 7 files changed, 278 insertions(+), 9 deletions(-) create mode 100644 python/ray/util/collective/tests/ray_start_distributed_script.sh diff --git a/python/ray/util/collective/collective.py b/python/ray/util/collective/collective.py index 26eb2a51b6c2..84548aa9f175 100644 --- a/python/ray/util/collective/collective.py +++ b/python/ray/util/collective/collective.py @@ -3,7 +3,7 @@ import ray from ray.util.collective import types -from ray.util.collective.const import NAMED_ACTOR_STORE_SUFFIX +from ray.util.collective.const import get_nccl_store_name # Get the availability information first by importing information _MPI_AVAILABLE = True @@ -21,7 +21,6 @@ logger = logging.getLogger(__name__) -logger.setLevel(logging.DEBUG) def nccl_available(): @@ -60,7 +59,7 @@ def create_collective_group(self, if rank == 0: import cupy.cuda.nccl as nccl group_uid = nccl.get_unique_id() - store_name = group_name + NAMED_ACTOR_STORE_SUFFIX + store_name = get_nccl_store_name(group_name) # Avoid a potential circular dependency in ray/actor.py from ray.util.collective.util import NCCLUniqueIDStore store = NCCLUniqueIDStore.options(name=store_name, lifetime="detached").remote(store_name) @@ -101,7 +100,7 @@ def destroy_collective_group(self, group_name): if backend == types.Backend.NCCL: # release the named actor if rank == 0: - store_name = group_name + NAMED_ACTOR_STORE_SUFFIX + store_name = get_nccl_store_name(group_name) store = ray.get_actor(store_name) ray.wait([store.__ray_terminate__.remote()]) ray.kill(store) diff --git a/python/ray/util/collective/collective_group/nccl_collective_group.py b/python/ray/util/collective/collective_group/nccl_collective_group.py index 273b3f01e5bf..9ec0ee36dfed 100644 --- a/python/ray/util/collective/collective_group/nccl_collective_group.py +++ b/python/ray/util/collective/collective_group/nccl_collective_group.py @@ -8,7 +8,7 @@ from ray.util.collective.collective_group import nccl_util from ray.util.collective.collective_group.base_collective_group import BaseGroup from ray.util.collective.types import AllReduceOptions, BarrierOptions, Backend -from ray.util.collective.const import NAMED_ACTOR_STORE_SUFFIX +from ray.util.collective.const import get_nccl_store_name logger = logging.getLogger(__name__) @@ -31,7 +31,7 @@ def meet(self, timeout=180): if timeout is not None and timeout < 0: raise ValueError("The 'timeout' argument must be nonnegative. " f"Received {timeout}") - self._store_name = self._group_name + NAMED_ACTOR_STORE_SUFFIX + self._store_name = get_nccl_store_name(self._group_name) timeout_delta = datetime.timedelta(seconds=timeout) elapsed = datetime.timedelta(seconds=0) start_time = datetime.datetime.now() diff --git a/python/ray/util/collective/const.py b/python/ray/util/collective/const.py index 3be8ebac6541..c6862437b803 100644 --- a/python/ray/util/collective/const.py +++ b/python/ray/util/collective/const.py @@ -4,3 +4,10 @@ Contains constants used to setup collective groups. """ NAMED_ACTOR_STORE_SUFFIX = '_unique_id_actor' + + +def get_nccl_store_name(group_name): + """Generate the unique name for the NCCLUniqueID store (named actor).""" + if not group_name: + raise ValueError('group_name is None.') + return group_name + NAMED_ACTOR_STORE_SUFFIX diff --git a/python/ray/util/collective/tests/conftest.py b/python/ray/util/collective/tests/conftest.py index 62f17ce49a1c..364df7e5cd09 100644 --- a/python/ray/util/collective/tests/conftest.py +++ b/python/ray/util/collective/tests/conftest.py @@ -2,18 +2,35 @@ import pytest import ray +from ray.util.collective.const import get_nccl_store_name +def clean_up(): + group_names = ['default', 'test', '123?34!', 'default2', 'randome'] + group_names.extend([str(i) for i in range(10)]) + for group_name in group_names: + try: + store_name = get_nccl_store_name(group_name) + actor = ray.get_actor(store_name) + except ValueError: + actor = None + if actor: + ray.kill(actor) + @pytest.fixture def ray_start_single_node_2_gpus(): # Please start this fixture in a cluster with 2 GPUs. address_info = ray.init(num_gpus=2) yield address_info - # The code after the yield will run as teardown code. ray.shutdown() -# TODO (Hao): implement this one. +# Hao: this fixture is a bit tricky. +# I use a bash script to start a ray cluster on +# my own on-premise cluster before run this fixture. @pytest.fixture def ray_start_distributed_2_nodes_4_gpus(): - pass + ray.init('auto') + yield + clean_up() + ray.shutdown() diff --git a/python/ray/util/collective/tests/ray_start_distributed_script.sh b/python/ray/util/collective/tests/ray_start_distributed_script.sh new file mode 100644 index 000000000000..1dda0011b377 --- /dev/null +++ b/python/ray/util/collective/tests/ray_start_distributed_script.sh @@ -0,0 +1,15 @@ +#!/bin/bash +ROOT_DIR=/home/hao.zhang/project/pycharm/ray +echo $ROOT_DIR +MY_IPADDR=10.20.41.115 +source $ROOT_DIR/env/bin/activate +echo $MY_IPADDR + +ray stop +sleep 0.5 +NCCL_DEBUG=INFO NCCL_SOCKET_IFNAME=enp179s0f0 ray start --head --object-manager-port=8076 --resources='{"machine":1}' --object-store-memory=32359738368 +sleep 0.5 + +echo "=> node $i" +ssh -o StrictHostKeyChecking=no -i /home/hao.zhang/.ssh/arion.pem hao.zhang@10.20.41.120 "source $ROOT_DIR/env/bin/activate; ray stop; NCCL_DEBUG=INFO NCCL_SOCKET_IFNAME=enp179s0f0 ray start --address='$MY_IPADDR:6379' --object-manager-port=8076 --resources='{\"machine\":1}' --object-store-memory=32359738368"; +wait diff --git a/python/ray/util/collective/tests/test_collective_2_nodes_4_gpus.py b/python/ray/util/collective/tests/test_collective_2_nodes_4_gpus.py index e69de29bb2d1..942cddf0cd6f 100644 --- a/python/ray/util/collective/tests/test_collective_2_nodes_4_gpus.py +++ b/python/ray/util/collective/tests/test_collective_2_nodes_4_gpus.py @@ -0,0 +1,230 @@ +"""Test the collective group APIs.""" +from random import shuffle +import pytest +import ray +from ray.util.collective.types import ReduceOp + +import cupy as cp +import torch + +from .util import Worker + + +def get_actors_group(num_workers=2, group_name='default', backend='nccl'): + actors = [Worker.remote() for i in range(num_workers)] + world_size = num_workers + init_results = ray.get([actor.init_group.remote(world_size, i, backend, group_name) + for i, actor in enumerate(actors)]) + return actors, init_results + + +@pytest.mark.parametrize("world_size", [2, 3, 4]) +@pytest.mark.parametrize("group_name", ['default', 'test', '123?34!']) +def test_init_two_actors(ray_start_distributed_2_nodes_4_gpus, world_size, group_name): + actors, results = get_actors_group(world_size, group_name) + for i in range(world_size): + assert (results[i]) + + +@pytest.mark.parametrize("world_size", [2, 3, 4]) +def test_init_multiple_groups(ray_start_distributed_2_nodes_4_gpus, world_size): + num_groups = 1 + actors = [Worker.remote() for _ in range(world_size)] + for i in range(num_groups): + group_name = str(i) + init_results = ray.get([actor.init_group.remote(world_size, i, group_name=group_name) + for i, actor in enumerate(actors)]) + for j in range(world_size): + assert init_results[j] + + +@pytest.mark.parametrize("world_size", [2, 3, 4]) +def test_get_rank(ray_start_distributed_2_nodes_4_gpus, world_size): + actors, _ = get_actors_group(world_size) + actor0_rank = ray.get(actors[0].report_rank.remote()) + assert actor0_rank == 0 + actor1_rank = ray.get(actors[1].report_rank.remote()) + assert actor1_rank == 1 + + # create a second group with a different name, and different order of ranks. + new_group_name = 'default2' + ranks = [i for i in range(world_size)] + shuffle(ranks) + _ = ray.get([actor.init_group.remote(world_size, ranks[i], group_name=new_group_name) + for i, actor in enumerate(actors)]) + actor0_rank = ray.get(actors[0].report_rank.remote(new_group_name)) + assert actor0_rank == ranks[0] + actor1_rank = ray.get(actors[1].report_rank.remote(new_group_name)) + assert actor1_rank == ranks[1] + + +@pytest.mark.parametrize("world_size", [2, 3, 4]) +def test_get_world_size(ray_start_distributed_2_nodes_4_gpus, world_size): + actors, _ = get_actors_group(world_size) + actor0_world_size = ray.get(actors[0].report_world_size.remote()) + actor1_world_size = ray.get(actors[1].report_world_size.remote()) + assert actor0_world_size == actor1_world_size == world_size + + +def test_availability(ray_start_distributed_2_nodes_4_gpus): + world_size = 4 + actors, _ = get_actors_group(world_size) + actor0_nccl_availability = ray.get(actors[0].report_nccl_availability.remote()) + assert actor0_nccl_availability + actor0_mpi_availability = ray.get(actors[0].report_mpi_availability.remote()) + assert not actor0_mpi_availability + + +def test_is_group_initialized(ray_start_distributed_2_nodes_4_gpus): + world_size = 4 + actors, _ = get_actors_group(world_size) + # check group is_init + actor0_is_init = ray.get(actors[0].report_is_group_initialized.remote()) + assert actor0_is_init + actor0_is_init = ray.get(actors[0].report_is_group_initialized.remote('random')) + assert not actor0_is_init + actor0_is_init = ray.get(actors[0].report_is_group_initialized.remote('123')) + assert not actor0_is_init + actor1_is_init = ray.get(actors[0].report_is_group_initialized.remote()) + assert actor1_is_init + actor1_is_init = ray.get(actors[0].report_is_group_initialized.remote('456')) + assert not actor1_is_init + + +def test_destroy_group(ray_start_distributed_2_nodes_4_gpus): + world_size = 4 + actors, _ = get_actors_group(world_size) + # Now destroy the group at actor0 + ray.wait([actors[0].destroy_group.remote()]) + actor0_is_init = ray.get(actors[0].report_is_group_initialized.remote()) + assert not actor0_is_init + + # should go well as the group `random` does not exist at all + ray.wait([actors[0].destroy_group.remote('random')]) + + actor1_is_init = ray.get(actors[1].report_is_group_initialized.remote()) + assert actor1_is_init + ray.wait([actors[1].destroy_group.remote('random')]) + actor1_is_init = ray.get(actors[1].report_is_group_initialized.remote()) + assert actor1_is_init + ray.wait([actors[1].destroy_group.remote('default')]) + actor1_is_init = ray.get(actors[1].report_is_group_initialized.remote()) + assert actor1_is_init == False + for i in [2, 3]: + ray.wait([actors[i].destroy_group.remote('default')]) + + # Now reconstruct the group using the same name + init_results = ray.get([actor.init_group.remote(world_size, i) for i, actor in enumerate(actors)]) + for i in range(world_size): + assert init_results[i] + actor0_is_init = ray.get(actors[0].report_is_group_initialized.remote()) + assert actor0_is_init + actor1_is_init = ray.get(actors[0].report_is_group_initialized.remote()) + assert actor1_is_init + + +@pytest.mark.parametrize("group_name", ['default', 'test', '123?34!']) +@pytest.mark.parametrize("world_size", [2, 3, 4]) +def test_allreduce_different_name(ray_start_distributed_2_nodes_4_gpus, group_name, world_size): + world_size = 2 + actors, _ = get_actors_group(num_workers=world_size, group_name=group_name) + results = ray.get([a.do_work.remote(group_name) for a in actors]) + assert (results[0] == cp.ones((10,), dtype=cp.float32) * world_size).all() + assert (results[1] == cp.ones((10,), dtype=cp.float32) * world_size).all() + + +@pytest.mark.parametrize("array_size", [2, 2 ** 5, 2 ** 10, 2 ** 15, 2 ** 20]) +def test_allreduce_different_array_size(ray_start_distributed_2_nodes_4_gpus, array_size): + world_size = 4 + actors, _ = get_actors_group(world_size) + ray.wait([a.set_buffer.remote(cp.ones(array_size, dtype=cp.float32)) for a in actors]) + results = ray.get([a.do_work.remote() for a in actors]) + assert (results[0] == cp.ones((array_size,), dtype=cp.float32) * world_size).all() + assert (results[1] == cp.ones((array_size,), dtype=cp.float32) * world_size).all() + + +def test_allreduce_destroy(ray_start_distributed_2_nodes_4_gpus, backend="nccl", group_name="default"): + world_size = 4 + actors, _ = get_actors_group(world_size) + + results = ray.get([a.do_work.remote() for a in actors]) + assert (results[0] == cp.ones((10,), dtype=cp.float32) * world_size).all() + assert (results[1] == cp.ones((10,), dtype=cp.float32) * world_size).all() + + # destroy the group and try do work, should fail + ray.wait([a.destroy_group.remote() for a in actors]) + with pytest.raises(RuntimeError): + results = ray.get([a.do_work.remote() for a in actors]) + + # reinit the same group and all reduce + ray.get([actor.init_group.remote(world_size, i, backend, group_name) + for i, actor in enumerate(actors)]) + results = ray.get([a.do_work.remote() for a in actors]) + assert (results[0] == cp.ones((10,), dtype=cp.float32) * world_size * world_size).all() + assert (results[1] == cp.ones((10,), dtype=cp.float32) * world_size * world_size).all() + + +def test_allreduce_multiple_group(ray_start_distributed_2_nodes_4_gpus, backend="nccl", num_groups=5): + world_size = 4 + actors, _ = get_actors_group(world_size) + for group_name in range(1, num_groups): + ray.get([actor.init_group.remote(world_size, i, backend, str(group_name)) + for i, actor in enumerate(actors)]) + for i in range(num_groups): + group_name = 'default' if i == 0 else str(i) + results = ray.get([a.do_work.remote(group_name) for a in actors]) + assert (results[0] == cp.ones((10,), dtype=cp.float32) * (world_size ** (i + 1))).all() + + +def test_allreduce_different_op(ray_start_distributed_2_nodes_4_gpus): + world_size = 4 + actors, _ = get_actors_group(world_size) + + # check product + ray.wait([a.set_buffer.remote(cp.ones(10, dtype=cp.float32) * (i + 2)) for i, a in enumerate(actors)]) + results = ray.get([a.do_work.remote(op=ReduceOp.PRODUCT) for a in actors]) + assert (results[0] == cp.ones((10,), dtype=cp.float32) * 120).all() + assert (results[1] == cp.ones((10,), dtype=cp.float32) * 120).all() + + # check min + ray.wait([a.set_buffer.remote(cp.ones(10, dtype=cp.float32) * (i + 2)) for i, a in enumerate(actors)]) + results = ray.get([a.do_work.remote(op=ReduceOp.MIN) for a in actors]) + assert (results[0] == cp.ones((10,), dtype=cp.float32) * 2).all() + assert (results[1] == cp.ones((10,), dtype=cp.float32) * 2).all() + + # check max + ray.wait([a.set_buffer.remote(cp.ones(10, dtype=cp.float32) * (i + 2)) for i, a in enumerate(actors)]) + results = ray.get([a.do_work.remote(op=ReduceOp.MAX) for a in actors]) + assert (results[0] == cp.ones((10,), dtype=cp.float32) * 5).all() + assert (results[1] == cp.ones((10,), dtype=cp.float32) * 5).all() + + +@pytest.mark.parametrize("dtype", [cp.uint8, cp.float16, cp.float32, cp.float64]) +def test_allreduce_different_dtype(ray_start_distributed_2_nodes_4_gpus, dtype): + world_size = 4 + actors, _ = get_actors_group(world_size) + ray.wait([a.set_buffer.remote(cp.ones(10, dtype=dtype)) for a in actors]) + results = ray.get([a.do_work.remote() for a in actors]) + assert (results[0] == cp.ones((10,), dtype=dtype) * world_size).all() + assert (results[1] == cp.ones((10,), dtype=dtype) * world_size).all() + + +def test_allreduce_torch_cupy(ray_start_distributed_2_nodes_4_gpus): + # import torch + world_size = 4 + actors, _ = get_actors_group(world_size) + ray.wait([actors[1].set_buffer.remote(torch.ones(10, ).cuda())]) + results = ray.get([a.do_work.remote() for a in actors]) + assert (results[0] == cp.ones((10,)) * world_size).all() + + ray.wait([actors[0].set_buffer.remote(torch.ones(10, ))]) + ray.wait([actors[1].set_buffer.remote(cp.ones(10, ))]) + with pytest.raises(RuntimeError): + results = ray.get([a.do_work.remote() for a in actors]) + + +if __name__ == "__main__": + import pytest + import sys + + sys.exit(pytest.main(["-v", "-x", __file__])) diff --git a/python/ray/util/collective/util.py b/python/ray/util/collective/util.py index acd76713afe9..91b2c12f4b01 100644 --- a/python/ray/util/collective/util.py +++ b/python/ray/util/collective/util.py @@ -1,3 +1,4 @@ +"""Some utility class for Collectives.""" import ray import logging From 793830c7ea129333649000417ff0064609252f3f Mon Sep 17 00:00:00 2001 From: Hao Zhang Date: Sat, 5 Dec 2020 02:30:44 -0500 Subject: [PATCH 27/51] remove scratch code --- .../util/collective/scratch/import_outer.py | 6 -- .../util/collective/scratch/import_test.py | 5 -- .../ray/util/collective/scratch/importer.py | 4 - python/ray/util/collective/scratch/test_id.py | 19 ----- .../collective/scratch/test_named_actor.py | 37 --------- .../collective/scratch/test_named_actor_2.py | 13 --- .../ray/util/collective/scratch/test_nccl.py | 81 ------------------- .../util/collective/scratch/test_options.py | 14 ---- 8 files changed, 179 deletions(-) delete mode 100644 python/ray/util/collective/scratch/import_outer.py delete mode 100644 python/ray/util/collective/scratch/import_test.py delete mode 100644 python/ray/util/collective/scratch/importer.py delete mode 100644 python/ray/util/collective/scratch/test_id.py delete mode 100644 python/ray/util/collective/scratch/test_named_actor.py delete mode 100644 python/ray/util/collective/scratch/test_named_actor_2.py delete mode 100644 python/ray/util/collective/scratch/test_nccl.py delete mode 100644 python/ray/util/collective/scratch/test_options.py diff --git a/python/ray/util/collective/scratch/import_outer.py b/python/ray/util/collective/scratch/import_outer.py deleted file mode 100644 index 74678501c81b..000000000000 --- a/python/ray/util/collective/scratch/import_outer.py +++ /dev/null @@ -1,6 +0,0 @@ - -# try: -from import_test import A -# except ImportError: -# print('123') - diff --git a/python/ray/util/collective/scratch/import_test.py b/python/ray/util/collective/scratch/import_test.py deleted file mode 100644 index 3caab93b0d11..000000000000 --- a/python/ray/util/collective/scratch/import_test.py +++ /dev/null @@ -1,5 +0,0 @@ -import importer - -class A: - def __init__(self): - pass \ No newline at end of file diff --git a/python/ray/util/collective/scratch/importer.py b/python/ray/util/collective/scratch/importer.py deleted file mode 100644 index 9c40b95931f7..000000000000 --- a/python/ray/util/collective/scratch/importer.py +++ /dev/null @@ -1,4 +0,0 @@ -try: - import torch -except ImportError: - raise ImportError('Torch not installed..') \ No newline at end of file diff --git a/python/ray/util/collective/scratch/test_id.py b/python/ray/util/collective/scratch/test_id.py deleted file mode 100644 index 5e8d427923e2..000000000000 --- a/python/ray/util/collective/scratch/test_id.py +++ /dev/null @@ -1,19 +0,0 @@ -import ray -import ray.worker -ray.init() - -@ray.remote -class Actor: - def __init__(self): - pass - def compute(self): - - worker = ray.worker.global_worker - return (worker.core_worker.get_actor_id()) # 0 #self.__ray_metadata___ - -actor = Actor.remote() -actor2 = Actor.remote() -print(actor._ray_actor_id, actor2._ray_actor_id) -k = actor.compute.remote() -print(ray.get(k)) -print(actor._ray_actor_id) diff --git a/python/ray/util/collective/scratch/test_named_actor.py b/python/ray/util/collective/scratch/test_named_actor.py deleted file mode 100644 index 555e2ce611fd..000000000000 --- a/python/ray/util/collective/scratch/test_named_actor.py +++ /dev/null @@ -1,37 +0,0 @@ -import ray - -@ray.remote -class NamedActor: - def __init__(self): - self.id = '' - - def set_id(self, id): - self.id = id - return self.id - - def get_id(self): - return self.id - - -@ray.remote -class Worker: - def __init__(self, rank): - self.rank = rank - - def do_work(self): - if self.rank == 0: - id = '123' - ac = NamedActor.options(name='uniquename', lifetime="detached").remote() - ray.wait([ac.set_id.remote('123')]) - else: - ac = ray.get_actor('uniquename') - id = ray.get(ac.get_id.remote()) - print(id) - return id - - -ray.init() - -workers = [Worker.remote(rank) for rank in range(4)] -ret = ray.get(workers[0].do_work.remote()) -m = ray.get([workers[i].do_work.remote() for i in range(1, 4)]) diff --git a/python/ray/util/collective/scratch/test_named_actor_2.py b/python/ray/util/collective/scratch/test_named_actor_2.py deleted file mode 100644 index 13a308bc105b..000000000000 --- a/python/ray/util/collective/scratch/test_named_actor_2.py +++ /dev/null @@ -1,13 +0,0 @@ -import ray -ray.init() -@ray.remote -class Actor: - def __init__(self): - self.data = 1 - - def get_data(self): - return self.data - -a = Actor.options(name="GM").remote() -b = Actor.options(name="GM").remote() -print(ray.get(b.get_data.remote())) diff --git a/python/ray/util/collective/scratch/test_nccl.py b/python/ray/util/collective/scratch/test_nccl.py deleted file mode 100644 index b903308cf6c9..000000000000 --- a/python/ray/util/collective/scratch/test_nccl.py +++ /dev/null @@ -1,81 +0,0 @@ -import ray -import cupy.cuda.nccl as nccl -from collections import defaultdict -import cupy as cp - -@ray.remote(num_gpus=0.5) -class Actor: - def __init(self): - cp.cuda.Stream.null.synchronize() - self.send = cp.ones((10,), dtype=cp.float32) - self.recv = cp.zeros((10,), dtype=cp.float32) - cp.cuda.Stream.null.synchronize() - - def send_getter(self): - return self.send - - def send_setter(self, val): - self.send = val - - def recv_getter(self): - return self.recv - - def recv_setter(self, val): - self.recv = val - - def compute(self): - pass - -class GroupManager(Object): - def __init__(self): - self.group = defaultdict([]) - - def create_colletive_group(self, - backend, - group_name, - world_size, - rank, - actor, - uid=None): - - self.group[group_name].append({actor: nccl.NcclCommunicator(world_size, uid, rank)}) - - - -def declare_collective_group(actors, group_options): - # sort actors by rank - ranks = group_options["rank"] - if len(actors) != len(ranks) or len(actors) != group_options["world_size"]: - raise ValueError() - - #sorted_actors = [x for _,x in sorted(zip(ranks, actors)), key=lambda pair: pair[0]] - uid = nccl.get_unique_id() - for i in range(0, len(ranks): - _group_mgr.create_collective_group(group_options["backend"], - group_options["name"], - group_options["world_size"], - ranks, - actors[i], - uid) - -def allreduce(group_name): - for (actor, comm) in _group_mgr.group[group_name]: - dummy = self.recv = cp.zeros((10,), dtype=cp.float32).ptr - comm.allReduce(ray.get(actor.send_getter()).ptr, dummy, 10, cp.cuda.nccl.NCCL_FLOAT32, 1, cp.cuda.Stream.null.ptr) - actor.set_recv(dummy) - cp.cuda.Stream.null.synchronize() - -_group_mgr = GroupManager() - -group_options = {"name" : "haha", - "backend" : "nccl", - "world_size" : 4, - "rank" : [0,1,2,3]} - -actors = [Actor().remote() for i in range(4)] -declare_collective_group(actors, group_options}) - -allreduce("haha") - -for i in range(4): - print(ray.get(actors[i].recv_getter())) diff --git a/python/ray/util/collective/scratch/test_options.py b/python/ray/util/collective/scratch/test_options.py deleted file mode 100644 index 5d99eed91b29..000000000000 --- a/python/ray/util/collective/scratch/test_options.py +++ /dev/null @@ -1,14 +0,0 @@ -import ray -ray.init() - -@ray.remote(num_gpus=1) -class Actor: - def __init__(self): - pass - - def compute(self): - pass - -# might work? -worker = Actor.options(collective={})# .remote() -print(worker._collective) From f8587dfbc80f22b99aec2d6208d97b48942ada2f Mon Sep 17 00:00:00 2001 From: Hao Zhang Date: Sat, 5 Dec 2020 02:34:43 -0500 Subject: [PATCH 28/51] auto-linting --- python/ray/util/collective/collective.py | 28 ++-- .../collective_group/nccl_collective_group.py | 15 +- .../collective/collective_group/nccl_util.py | 5 +- .../examples/nccl_allreduce_example.py | 9 +- python/ray/util/collective/tests/conftest.py | 1 + .../tests/test_collective_2_nodes_4_gpus.py | 140 +++++++++++------ .../test_collective_single_node_2_gpus.py | 148 +++++++++++------- python/ray/util/collective/tests/util.py | 8 +- python/ray/util/collective/util.py | 6 +- 9 files changed, 222 insertions(+), 138 deletions(-) diff --git a/python/ray/util/collective/collective.py b/python/ray/util/collective/collective.py index 84548aa9f175..a22bdde377ae 100644 --- a/python/ray/util/collective/collective.py +++ b/python/ray/util/collective/collective.py @@ -19,7 +19,6 @@ except ImportError: _NCCL_AVAILABLE = False - logger = logging.getLogger(__name__) @@ -36,16 +35,13 @@ class GroupManager(object): Use this class to manage the collective groups we created so far; """ + def __init__(self): """Put some necessary meta information here.""" self._name_group_map = {} self._group_name_map = {} - def create_collective_group(self, - backend, - world_size, - rank, - group_name): + def create_collective_group(self, backend, world_size, rank, group_name): """ The entry to create new collective groups and register in the manager. @@ -62,11 +58,12 @@ def create_collective_group(self, store_name = get_nccl_store_name(group_name) # Avoid a potential circular dependency in ray/actor.py from ray.util.collective.util import NCCLUniqueIDStore - store = NCCLUniqueIDStore.options(name=store_name, lifetime="detached").remote(store_name) + store = NCCLUniqueIDStore.options( + name=store_name, lifetime="detached").remote(store_name) ray.wait([store.set_id.remote(group_uid)]) logger.debug('creating NCCL group: {}'.format(group_name)) - g = NCCLGroup(world_size, rank, group_name) + g = NCCLGroup(world_size, rank, group_name) self._name_group_map[group_name] = g self._group_name_map[g] = group_name return self._name_group_map[group_name] @@ -79,7 +76,8 @@ def is_group_exist(self, group_name): def get_group_by_name(self, group_name): """Get the collective group handle by its name.""" if not self.is_group_exist(group_name): - logger.warning("The group '{}' is not initialized.".format(group_name)) + logger.warning( + "The group '{}' is not initialized.".format(group_name)) return None return self._name_group_map[group_name] @@ -143,14 +141,12 @@ def init_collective_group(world_size: int, raise ValueError("group_name '{}' needs to be a string." .format(group_name)) - - if _group_mgr.is_group_exist(group_name): raise RuntimeError('Trying to initialize a group twice.') - assert(world_size > 0) - assert(rank >= 0) - assert(rank < world_size) + assert (world_size > 0) + assert (rank >= 0) + assert (rank < world_size) _group_mgr.create_collective_group(backend, world_size, rank, group_name) @@ -196,9 +192,7 @@ def get_world_size(group_name='default') -> int: return g.world_size -def allreduce(tensor, - group_name: str, - op=types.ReduceOp.SUM): +def allreduce(tensor, group_name: str, op=types.ReduceOp.SUM): """ Collective allreduce the tensor across the group with name group_name. diff --git a/python/ray/util/collective/collective_group/nccl_collective_group.py b/python/ray/util/collective/collective_group/nccl_collective_group.py index 9ec0ee36dfed..2256485af57b 100644 --- a/python/ray/util/collective/collective_group/nccl_collective_group.py +++ b/python/ray/util/collective/collective_group/nccl_collective_group.py @@ -10,7 +10,6 @@ from ray.util.collective.types import AllReduceOptions, BarrierOptions, Backend from ray.util.collective.const import get_nccl_store_name - logger = logging.getLogger(__name__) # TODO(Hao): @@ -37,7 +36,8 @@ def meet(self, timeout=180): start_time = datetime.datetime.now() while elapsed < timeout_delta: try: - logger.debug("Trying to meet at the store '{}'".format(self._store_name)) + logger.debug("Trying to meet at the store '{}'".format( + self._store_name)) self._store = ray.get_actor(self._store_name) except ValueError: logger.debug("Failed to meet at the store '{}'." @@ -71,7 +71,8 @@ def get_nccl_id(self, timeout=180): continue break if not uid: - raise RuntimeError("Unable to get the NCCLUniqueID from the store.") + raise RuntimeError( + "Unable to get the NCCLUniqueID from the store.") return uid @@ -103,7 +104,6 @@ def _init_nccl_unique_id(self): """Init the NCCL unique ID required for setting up NCCL communicator.""" self._nccl_uid = self._rendezvous.get_nccl_id() - @property def nccl_uid(self): return self._nccl_uid @@ -146,12 +146,7 @@ def allreduce(self, tensor, allreduce_options=AllReduceOptions()): reduce_op = nccl_util.get_nccl_reduce_op(allreduce_options.reduceOp) # in-place allreduce - comm.allReduce(ptr, - ptr, - n_elems, - dtype, - reduce_op, - stream.ptr) + comm.allReduce(ptr, ptr, n_elems, dtype, reduce_op, stream.ptr) def barrier(self, barrier_options=BarrierOptions()): """ diff --git a/python/ray/util/collective/collective_group/nccl_util.py b/python/ray/util/collective/collective_group/nccl_util.py index 5ef19a0263ae..cff6aeb74091 100644 --- a/python/ray/util/collective/collective_group/nccl_util.py +++ b/python/ray/util/collective/collective_group/nccl_util.py @@ -13,7 +13,6 @@ from ray.util.collective.types import ReduceOp, torch_available - NCCL_REDUCE_OP_MAP = { ReduceOp.SUM: nccl.NCCL_SUM, ReduceOp.PRODUCT: nccl.NCCL_PROD, @@ -21,7 +20,6 @@ ReduceOp.MAX: nccl.NCCL_MAX, } - # cupy types are the same with numpy types NUMPY_NCCL_DTYPE_MAP = { numpy.uint8: nccl.NCCL_UINT8, @@ -72,7 +70,8 @@ def get_nccl_reduce_op(reduce_op): Nccl_op (nccl.ncclRedOp_t) """ if reduce_op not in NCCL_REDUCE_OP_MAP: - raise RuntimeError('NCCL does not support ReduceOp: {}'.format(reduce_op)) + raise RuntimeError( + 'NCCL does not support ReduceOp: {}'.format(reduce_op)) return NCCL_REDUCE_OP_MAP[reduce_op] diff --git a/python/ray/util/collective/examples/nccl_allreduce_example.py b/python/ray/util/collective/examples/nccl_allreduce_example.py index 818d46f6ee06..82bf01ed5cbd 100644 --- a/python/ray/util/collective/examples/nccl_allreduce_example.py +++ b/python/ray/util/collective/examples/nccl_allreduce_example.py @@ -1,14 +1,14 @@ - import ray import cupy as cp import ray.util.collective as collective + @ray.remote(num_gpus=1) class Worker: def __init__(self): - self.send = cp.ones((4,), dtype=cp.float32) - self.recv = cp.zeros((4,), dtype=cp.float32) + self.send = cp.ones((4, ), dtype=cp.float32) + self.recv = cp.zeros((4, ), dtype=cp.float32) def setup(self, world_size, rank): collective.init_collective_group('nccl', world_size, rank, 'default') @@ -22,6 +22,7 @@ def compute(self): def destroy(self): collective.destroy_group('') + if __name__ == "__main__": send = cp.ones((4, ), dtype=cp.float32) @@ -38,4 +39,4 @@ def destroy(self): # m = ray.get(init_rets) results = ray.get([w.compute.remote() for w in workers]) # print(results) - ray.shutdown() \ No newline at end of file + ray.shutdown() diff --git a/python/ray/util/collective/tests/conftest.py b/python/ray/util/collective/tests/conftest.py index 364df7e5cd09..ad52e2e39881 100644 --- a/python/ray/util/collective/tests/conftest.py +++ b/python/ray/util/collective/tests/conftest.py @@ -17,6 +17,7 @@ def clean_up(): if actor: ray.kill(actor) + @pytest.fixture def ray_start_single_node_2_gpus(): # Please start this fixture in a cluster with 2 GPUs. diff --git a/python/ray/util/collective/tests/test_collective_2_nodes_4_gpus.py b/python/ray/util/collective/tests/test_collective_2_nodes_4_gpus.py index 942cddf0cd6f..edc735ca2a19 100644 --- a/python/ray/util/collective/tests/test_collective_2_nodes_4_gpus.py +++ b/python/ray/util/collective/tests/test_collective_2_nodes_4_gpus.py @@ -13,27 +13,33 @@ def get_actors_group(num_workers=2, group_name='default', backend='nccl'): actors = [Worker.remote() for i in range(num_workers)] world_size = num_workers - init_results = ray.get([actor.init_group.remote(world_size, i, backend, group_name) - for i, actor in enumerate(actors)]) + init_results = ray.get([ + actor.init_group.remote(world_size, i, backend, group_name) + for i, actor in enumerate(actors) + ]) return actors, init_results @pytest.mark.parametrize("world_size", [2, 3, 4]) @pytest.mark.parametrize("group_name", ['default', 'test', '123?34!']) -def test_init_two_actors(ray_start_distributed_2_nodes_4_gpus, world_size, group_name): +def test_init_two_actors(ray_start_distributed_2_nodes_4_gpus, world_size, + group_name): actors, results = get_actors_group(world_size, group_name) for i in range(world_size): assert (results[i]) @pytest.mark.parametrize("world_size", [2, 3, 4]) -def test_init_multiple_groups(ray_start_distributed_2_nodes_4_gpus, world_size): +def test_init_multiple_groups(ray_start_distributed_2_nodes_4_gpus, + world_size): num_groups = 1 actors = [Worker.remote() for _ in range(world_size)] for i in range(num_groups): group_name = str(i) - init_results = ray.get([actor.init_group.remote(world_size, i, group_name=group_name) - for i, actor in enumerate(actors)]) + init_results = ray.get([ + actor.init_group.remote(world_size, i, group_name=group_name) + for i, actor in enumerate(actors) + ]) for j in range(world_size): assert init_results[j] @@ -50,8 +56,11 @@ def test_get_rank(ray_start_distributed_2_nodes_4_gpus, world_size): new_group_name = 'default2' ranks = [i for i in range(world_size)] shuffle(ranks) - _ = ray.get([actor.init_group.remote(world_size, ranks[i], group_name=new_group_name) - for i, actor in enumerate(actors)]) + _ = ray.get([ + actor.init_group.remote( + world_size, ranks[i], group_name=new_group_name) + for i, actor in enumerate(actors) + ]) actor0_rank = ray.get(actors[0].report_rank.remote(new_group_name)) assert actor0_rank == ranks[0] actor1_rank = ray.get(actors[1].report_rank.remote(new_group_name)) @@ -69,9 +78,11 @@ def test_get_world_size(ray_start_distributed_2_nodes_4_gpus, world_size): def test_availability(ray_start_distributed_2_nodes_4_gpus): world_size = 4 actors, _ = get_actors_group(world_size) - actor0_nccl_availability = ray.get(actors[0].report_nccl_availability.remote()) + actor0_nccl_availability = ray.get( + actors[0].report_nccl_availability.remote()) assert actor0_nccl_availability - actor0_mpi_availability = ray.get(actors[0].report_mpi_availability.remote()) + actor0_mpi_availability = ray.get( + actors[0].report_mpi_availability.remote()) assert not actor0_mpi_availability @@ -81,13 +92,16 @@ def test_is_group_initialized(ray_start_distributed_2_nodes_4_gpus): # check group is_init actor0_is_init = ray.get(actors[0].report_is_group_initialized.remote()) assert actor0_is_init - actor0_is_init = ray.get(actors[0].report_is_group_initialized.remote('random')) + actor0_is_init = ray.get( + actors[0].report_is_group_initialized.remote('random')) assert not actor0_is_init - actor0_is_init = ray.get(actors[0].report_is_group_initialized.remote('123')) + actor0_is_init = ray.get( + actors[0].report_is_group_initialized.remote('123')) assert not actor0_is_init actor1_is_init = ray.get(actors[0].report_is_group_initialized.remote()) assert actor1_is_init - actor1_is_init = ray.get(actors[0].report_is_group_initialized.remote('456')) + actor1_is_init = ray.get( + actors[0].report_is_group_initialized.remote('456')) assert not actor1_is_init @@ -114,7 +128,10 @@ def test_destroy_group(ray_start_distributed_2_nodes_4_gpus): ray.wait([actors[i].destroy_group.remote('default')]) # Now reconstruct the group using the same name - init_results = ray.get([actor.init_group.remote(world_size, i) for i, actor in enumerate(actors)]) + init_results = ray.get([ + actor.init_group.remote(world_size, i) + for i, actor in enumerate(actors) + ]) for i in range(world_size): assert init_results[i] actor0_is_init = ray.get(actors[0].report_is_group_initialized.remote()) @@ -125,31 +142,40 @@ def test_destroy_group(ray_start_distributed_2_nodes_4_gpus): @pytest.mark.parametrize("group_name", ['default', 'test', '123?34!']) @pytest.mark.parametrize("world_size", [2, 3, 4]) -def test_allreduce_different_name(ray_start_distributed_2_nodes_4_gpus, group_name, world_size): +def test_allreduce_different_name(ray_start_distributed_2_nodes_4_gpus, + group_name, world_size): world_size = 2 actors, _ = get_actors_group(num_workers=world_size, group_name=group_name) results = ray.get([a.do_work.remote(group_name) for a in actors]) - assert (results[0] == cp.ones((10,), dtype=cp.float32) * world_size).all() - assert (results[1] == cp.ones((10,), dtype=cp.float32) * world_size).all() + assert (results[0] == cp.ones((10, ), dtype=cp.float32) * world_size).all() + assert (results[1] == cp.ones((10, ), dtype=cp.float32) * world_size).all() -@pytest.mark.parametrize("array_size", [2, 2 ** 5, 2 ** 10, 2 ** 15, 2 ** 20]) -def test_allreduce_different_array_size(ray_start_distributed_2_nodes_4_gpus, array_size): +@pytest.mark.parametrize("array_size", [2, 2**5, 2**10, 2**15, 2**20]) +def test_allreduce_different_array_size(ray_start_distributed_2_nodes_4_gpus, + array_size): world_size = 4 actors, _ = get_actors_group(world_size) - ray.wait([a.set_buffer.remote(cp.ones(array_size, dtype=cp.float32)) for a in actors]) + ray.wait([ + a.set_buffer.remote(cp.ones(array_size, dtype=cp.float32)) + for a in actors + ]) results = ray.get([a.do_work.remote() for a in actors]) - assert (results[0] == cp.ones((array_size,), dtype=cp.float32) * world_size).all() - assert (results[1] == cp.ones((array_size,), dtype=cp.float32) * world_size).all() + assert (results[0] == cp.ones( + (array_size, ), dtype=cp.float32) * world_size).all() + assert (results[1] == cp.ones( + (array_size, ), dtype=cp.float32) * world_size).all() -def test_allreduce_destroy(ray_start_distributed_2_nodes_4_gpus, backend="nccl", group_name="default"): +def test_allreduce_destroy(ray_start_distributed_2_nodes_4_gpus, + backend="nccl", + group_name="default"): world_size = 4 actors, _ = get_actors_group(world_size) results = ray.get([a.do_work.remote() for a in actors]) - assert (results[0] == cp.ones((10,), dtype=cp.float32) * world_size).all() - assert (results[1] == cp.ones((10,), dtype=cp.float32) * world_size).all() + assert (results[0] == cp.ones((10, ), dtype=cp.float32) * world_size).all() + assert (results[1] == cp.ones((10, ), dtype=cp.float32) * world_size).all() # destroy the group and try do work, should fail ray.wait([a.destroy_group.remote() for a in actors]) @@ -157,23 +183,32 @@ def test_allreduce_destroy(ray_start_distributed_2_nodes_4_gpus, backend="nccl", results = ray.get([a.do_work.remote() for a in actors]) # reinit the same group and all reduce - ray.get([actor.init_group.remote(world_size, i, backend, group_name) - for i, actor in enumerate(actors)]) + ray.get([ + actor.init_group.remote(world_size, i, backend, group_name) + for i, actor in enumerate(actors) + ]) results = ray.get([a.do_work.remote() for a in actors]) - assert (results[0] == cp.ones((10,), dtype=cp.float32) * world_size * world_size).all() - assert (results[1] == cp.ones((10,), dtype=cp.float32) * world_size * world_size).all() + assert (results[0] == cp.ones( + (10, ), dtype=cp.float32) * world_size * world_size).all() + assert (results[1] == cp.ones( + (10, ), dtype=cp.float32) * world_size * world_size).all() -def test_allreduce_multiple_group(ray_start_distributed_2_nodes_4_gpus, backend="nccl", num_groups=5): +def test_allreduce_multiple_group(ray_start_distributed_2_nodes_4_gpus, + backend="nccl", + num_groups=5): world_size = 4 actors, _ = get_actors_group(world_size) for group_name in range(1, num_groups): - ray.get([actor.init_group.remote(world_size, i, backend, str(group_name)) - for i, actor in enumerate(actors)]) + ray.get([ + actor.init_group.remote(world_size, i, backend, str(group_name)) + for i, actor in enumerate(actors) + ]) for i in range(num_groups): group_name = 'default' if i == 0 else str(i) results = ray.get([a.do_work.remote(group_name) for a in actors]) - assert (results[0] == cp.ones((10,), dtype=cp.float32) * (world_size ** (i + 1))).all() + assert (results[0] == cp.ones( + (10, ), dtype=cp.float32) * (world_size**(i + 1))).all() def test_allreduce_different_op(ray_start_distributed_2_nodes_4_gpus): @@ -181,32 +216,43 @@ def test_allreduce_different_op(ray_start_distributed_2_nodes_4_gpus): actors, _ = get_actors_group(world_size) # check product - ray.wait([a.set_buffer.remote(cp.ones(10, dtype=cp.float32) * (i + 2)) for i, a in enumerate(actors)]) + ray.wait([ + a.set_buffer.remote(cp.ones(10, dtype=cp.float32) * (i + 2)) + for i, a in enumerate(actors) + ]) results = ray.get([a.do_work.remote(op=ReduceOp.PRODUCT) for a in actors]) - assert (results[0] == cp.ones((10,), dtype=cp.float32) * 120).all() - assert (results[1] == cp.ones((10,), dtype=cp.float32) * 120).all() + assert (results[0] == cp.ones((10, ), dtype=cp.float32) * 120).all() + assert (results[1] == cp.ones((10, ), dtype=cp.float32) * 120).all() # check min - ray.wait([a.set_buffer.remote(cp.ones(10, dtype=cp.float32) * (i + 2)) for i, a in enumerate(actors)]) + ray.wait([ + a.set_buffer.remote(cp.ones(10, dtype=cp.float32) * (i + 2)) + for i, a in enumerate(actors) + ]) results = ray.get([a.do_work.remote(op=ReduceOp.MIN) for a in actors]) - assert (results[0] == cp.ones((10,), dtype=cp.float32) * 2).all() - assert (results[1] == cp.ones((10,), dtype=cp.float32) * 2).all() + assert (results[0] == cp.ones((10, ), dtype=cp.float32) * 2).all() + assert (results[1] == cp.ones((10, ), dtype=cp.float32) * 2).all() # check max - ray.wait([a.set_buffer.remote(cp.ones(10, dtype=cp.float32) * (i + 2)) for i, a in enumerate(actors)]) + ray.wait([ + a.set_buffer.remote(cp.ones(10, dtype=cp.float32) * (i + 2)) + for i, a in enumerate(actors) + ]) results = ray.get([a.do_work.remote(op=ReduceOp.MAX) for a in actors]) - assert (results[0] == cp.ones((10,), dtype=cp.float32) * 5).all() - assert (results[1] == cp.ones((10,), dtype=cp.float32) * 5).all() + assert (results[0] == cp.ones((10, ), dtype=cp.float32) * 5).all() + assert (results[1] == cp.ones((10, ), dtype=cp.float32) * 5).all() -@pytest.mark.parametrize("dtype", [cp.uint8, cp.float16, cp.float32, cp.float64]) -def test_allreduce_different_dtype(ray_start_distributed_2_nodes_4_gpus, dtype): +@pytest.mark.parametrize("dtype", + [cp.uint8, cp.float16, cp.float32, cp.float64]) +def test_allreduce_different_dtype(ray_start_distributed_2_nodes_4_gpus, + dtype): world_size = 4 actors, _ = get_actors_group(world_size) ray.wait([a.set_buffer.remote(cp.ones(10, dtype=dtype)) for a in actors]) results = ray.get([a.do_work.remote() for a in actors]) - assert (results[0] == cp.ones((10,), dtype=dtype) * world_size).all() - assert (results[1] == cp.ones((10,), dtype=dtype) * world_size).all() + assert (results[0] == cp.ones((10, ), dtype=dtype) * world_size).all() + assert (results[1] == cp.ones((10, ), dtype=dtype) * world_size).all() def test_allreduce_torch_cupy(ray_start_distributed_2_nodes_4_gpus): @@ -215,7 +261,7 @@ def test_allreduce_torch_cupy(ray_start_distributed_2_nodes_4_gpus): actors, _ = get_actors_group(world_size) ray.wait([actors[1].set_buffer.remote(torch.ones(10, ).cuda())]) results = ray.get([a.do_work.remote() for a in actors]) - assert (results[0] == cp.ones((10,)) * world_size).all() + assert (results[0] == cp.ones((10, )) * world_size).all() ray.wait([actors[0].set_buffer.remote(torch.ones(10, ))]) ray.wait([actors[1].set_buffer.remote(cp.ones(10, ))]) diff --git a/python/ray/util/collective/tests/test_collective_single_node_2_gpus.py b/python/ray/util/collective/tests/test_collective_single_node_2_gpus.py index dbb0cccf75db..17c876c77370 100644 --- a/python/ray/util/collective/tests/test_collective_single_node_2_gpus.py +++ b/python/ray/util/collective/tests/test_collective_single_node_2_gpus.py @@ -12,8 +12,10 @@ def get_actors_group(num_workers=2, group_name='default', backend='nccl'): actors = [Worker.remote() for i in range(num_workers)] world_size = num_workers - init_results = ray.get([actor.init_group.remote(world_size, i, backend, group_name) - for i, actor in enumerate(actors)]) + init_results = ray.get([ + actor.init_group.remote(world_size, i, backend, group_name) + for i, actor in enumerate(actors) + ]) return actors, init_results @@ -22,7 +24,7 @@ def test_init_two_actors(ray_start_single_node_2_gpus, group_name): world_size = 2 actors, results = get_actors_group(world_size, group_name) for i in range(world_size): - assert(results[i]) + assert (results[i]) def test_init_multiple_groups(ray_start_single_node_2_gpus): @@ -31,8 +33,10 @@ def test_init_multiple_groups(ray_start_single_node_2_gpus): actors = [Worker.remote() for i in range(world_size)] for i in range(num_groups): group_name = str(i) - init_results = ray.get([actor.init_group.remote(world_size, i, group_name=group_name) - for i, actor in enumerate(actors)]) + init_results = ray.get([ + actor.init_group.remote(world_size, i, group_name=group_name) + for i, actor in enumerate(actors) + ]) for j in range(world_size): assert init_results[j] @@ -47,8 +51,11 @@ def test_get_rank(ray_start_single_node_2_gpus): # create a second group with a different name, and different order of ranks. new_group_name = 'default2' - _ = ray.get([actor.init_group.remote(world_size, world_size - 1 - i, group_name=new_group_name) - for i, actor in enumerate(actors)]) + _ = ray.get([ + actor.init_group.remote( + world_size, world_size - 1 - i, group_name=new_group_name) + for i, actor in enumerate(actors) + ]) actor0_rank = ray.get(actors[0].report_rank.remote(new_group_name)) assert actor0_rank == 1 actor1_rank = ray.get(actors[1].report_rank.remote(new_group_name)) @@ -66,10 +73,12 @@ def test_get_world_size(ray_start_single_node_2_gpus): def test_availability(ray_start_single_node_2_gpus): world_size = 2 actors, _ = get_actors_group(world_size) - actor0_nccl_availability = ray.get(actors[0].report_nccl_availability.remote()) - assert(actor0_nccl_availability) - actor0_mpi_availability = ray.get(actors[0].report_mpi_availability.remote()) - assert(actor0_mpi_availability == False) + actor0_nccl_availability = ray.get( + actors[0].report_nccl_availability.remote()) + assert (actor0_nccl_availability) + actor0_mpi_availability = ray.get( + actors[0].report_mpi_availability.remote()) + assert (actor0_mpi_availability == False) def test_is_group_initialized(ray_start_single_node_2_gpus): @@ -78,13 +87,16 @@ def test_is_group_initialized(ray_start_single_node_2_gpus): # check group is_init actor0_is_init = ray.get(actors[0].report_is_group_initialized.remote()) assert actor0_is_init - actor0_is_init = ray.get(actors[0].report_is_group_initialized.remote('random')) + actor0_is_init = ray.get( + actors[0].report_is_group_initialized.remote('random')) assert not actor0_is_init - actor0_is_init = ray.get(actors[0].report_is_group_initialized.remote('123')) + actor0_is_init = ray.get( + actors[0].report_is_group_initialized.remote('123')) assert not actor0_is_init actor1_is_init = ray.get(actors[0].report_is_group_initialized.remote()) assert actor1_is_init - actor1_is_init = ray.get(actors[0].report_is_group_initialized.remote('456')) + actor1_is_init = ray.get( + actors[0].report_is_group_initialized.remote('456')) assert not actor1_is_init @@ -109,7 +121,10 @@ def test_destroy_group(ray_start_single_node_2_gpus): assert actor1_is_init == False # Now reconstruct the group using the same name - init_results = ray.get([actor.init_group.remote(world_size, i) for i, actor in enumerate(actors)]) + init_results = ray.get([ + actor.init_group.remote(world_size, i) + for i, actor in enumerate(actors) + ]) for i in range(world_size): assert init_results[i] actor0_is_init = ray.get(actors[0].report_is_group_initialized.remote()) @@ -124,51 +139,68 @@ def test_allreduce_different_name(ray_start_single_node_2_gpus, group_name): world_size = 2 actors, _ = get_actors_group(num_workers=world_size, group_name=group_name) results = ray.get([a.do_work.remote(group_name) for a in actors]) - assert (results[0] == cp.ones((10,), dtype=cp.float32) * world_size).all() - assert (results[1] == cp.ones((10,), dtype=cp.float32) * world_size).all() + assert (results[0] == cp.ones((10, ), dtype=cp.float32) * world_size).all() + assert (results[1] == cp.ones((10, ), dtype=cp.float32) * world_size).all() @pytest.mark.parametrize("array_size", [2, 2**5, 2**10, 2**15, 2**20]) -def test_allreduce_different_array_size(ray_start_single_node_2_gpus, array_size): +def test_allreduce_different_array_size(ray_start_single_node_2_gpus, + array_size): world_size = 2 actors, _ = get_actors_group(world_size) - ray.wait([a.set_buffer.remote(cp.ones(array_size, dtype=cp.float32)) for a in actors]) + ray.wait([ + a.set_buffer.remote(cp.ones(array_size, dtype=cp.float32)) + for a in actors + ]) results = ray.get([a.do_work.remote() for a in actors]) - assert (results[0] == cp.ones((array_size,), dtype=cp.float32) * world_size).all() - assert (results[1] == cp.ones((array_size,), dtype=cp.float32) * world_size).all() + assert (results[0] == cp.ones( + (array_size, ), dtype=cp.float32) * world_size).all() + assert (results[1] == cp.ones( + (array_size, ), dtype=cp.float32) * world_size).all() -def test_allreduce_destroy(ray_start_single_node_2_gpus, backend="nccl", group_name="default"): +def test_allreduce_destroy(ray_start_single_node_2_gpus, + backend="nccl", + group_name="default"): world_size = 2 actors, _ = get_actors_group(world_size) results = ray.get([a.do_work.remote() for a in actors]) - assert (results[0] == cp.ones((10,), dtype=cp.float32) * world_size).all() - assert (results[1] == cp.ones((10,), dtype=cp.float32) * world_size).all() - - # destroy the group and try do work, should fail + assert (results[0] == cp.ones((10, ), dtype=cp.float32) * world_size).all() + assert (results[1] == cp.ones((10, ), dtype=cp.float32) * world_size).all() + + # destroy the group and try do work, should fail ray.wait([a.destroy_group.remote() for a in actors]) with pytest.raises(RuntimeError): results = ray.get([a.do_work.remote() for a in actors]) - # reinit the same group and all reduce - ray.get([actor.init_group.remote(world_size, i, backend, group_name) - for i, actor in enumerate(actors)]) + # reinit the same group and all reduce + ray.get([ + actor.init_group.remote(world_size, i, backend, group_name) + for i, actor in enumerate(actors) + ]) results = ray.get([a.do_work.remote() for a in actors]) - assert (results[0] == cp.ones((10,), dtype=cp.float32) * world_size * 2).all() - assert (results[1] == cp.ones((10,), dtype=cp.float32) * world_size * 2).all() + assert (results[0] == cp.ones( + (10, ), dtype=cp.float32) * world_size * 2).all() + assert (results[1] == cp.ones( + (10, ), dtype=cp.float32) * world_size * 2).all() -def test_allreduce_multiple_group(ray_start_single_node_2_gpus, backend="nccl", num_groups=5): +def test_allreduce_multiple_group(ray_start_single_node_2_gpus, + backend="nccl", + num_groups=5): world_size = 2 actors, _ = get_actors_group(world_size) for group_name in range(1, num_groups): - ray.get([actor.init_group.remote(world_size, i, backend, str(group_name)) - for i, actor in enumerate(actors)]) + ray.get([ + actor.init_group.remote(world_size, i, backend, str(group_name)) + for i, actor in enumerate(actors) + ]) for i in range(num_groups): group_name = 'default' if i == 0 else str(i) results = ray.get([a.do_work.remote(group_name) for a in actors]) - assert (results[0] == cp.ones((10,), dtype=cp.float32) * (world_size ** (i + 1))).all() + assert (results[0] == cp.ones( + (10, ), dtype=cp.float32) * (world_size**(i + 1))).all() def test_allreduce_different_op(ray_start_single_node_2_gpus): @@ -176,44 +208,54 @@ def test_allreduce_different_op(ray_start_single_node_2_gpus): actors, _ = get_actors_group(world_size) # check product - ray.wait([a.set_buffer.remote(cp.ones(10, dtype=cp.float32) * (i + 2)) for i, a in enumerate(actors)]) + ray.wait([ + a.set_buffer.remote(cp.ones(10, dtype=cp.float32) * (i + 2)) + for i, a in enumerate(actors) + ]) results = ray.get([a.do_work.remote(op=ReduceOp.PRODUCT) for a in actors]) - assert (results[0] == cp.ones((10,), dtype=cp.float32) * 6).all() - assert (results[1] == cp.ones((10,), dtype=cp.float32) * 6).all() - + assert (results[0] == cp.ones((10, ), dtype=cp.float32) * 6).all() + assert (results[1] == cp.ones((10, ), dtype=cp.float32) * 6).all() + # check min - ray.wait([a.set_buffer.remote(cp.ones(10, dtype=cp.float32) * (i + 2)) for i, a in enumerate(actors)]) + ray.wait([ + a.set_buffer.remote(cp.ones(10, dtype=cp.float32) * (i + 2)) + for i, a in enumerate(actors) + ]) results = ray.get([a.do_work.remote(op=ReduceOp.MIN) for a in actors]) - assert (results[0] == cp.ones((10,), dtype=cp.float32) * 2).all() - assert (results[1] == cp.ones((10,), dtype=cp.float32) * 2).all() - + assert (results[0] == cp.ones((10, ), dtype=cp.float32) * 2).all() + assert (results[1] == cp.ones((10, ), dtype=cp.float32) * 2).all() + # check max - ray.wait([a.set_buffer.remote(cp.ones(10, dtype=cp.float32) * (i + 2)) for i, a in enumerate(actors)]) + ray.wait([ + a.set_buffer.remote(cp.ones(10, dtype=cp.float32) * (i + 2)) + for i, a in enumerate(actors) + ]) results = ray.get([a.do_work.remote(op=ReduceOp.MAX) for a in actors]) - assert (results[0] == cp.ones((10,), dtype=cp.float32) * 3).all() - assert (results[1] == cp.ones((10,), dtype=cp.float32) * 3).all() + assert (results[0] == cp.ones((10, ), dtype=cp.float32) * 3).all() + assert (results[1] == cp.ones((10, ), dtype=cp.float32) * 3).all() -@pytest.mark.parametrize("dtype", [cp.uint8, cp.float16, cp.float32, cp.float64]) +@pytest.mark.parametrize("dtype", + [cp.uint8, cp.float16, cp.float32, cp.float64]) def test_allreduce_different_dtype(ray_start_single_node_2_gpus, dtype): world_size = 2 actors, _ = get_actors_group(world_size) ray.wait([a.set_buffer.remote(cp.ones(10, dtype=dtype)) for a in actors]) results = ray.get([a.do_work.remote() for a in actors]) - assert (results[0] == cp.ones((10,), dtype=dtype) * world_size).all() - assert (results[1] == cp.ones((10,), dtype=dtype) * world_size).all() + assert (results[0] == cp.ones((10, ), dtype=dtype) * world_size).all() + assert (results[1] == cp.ones((10, ), dtype=dtype) * world_size).all() def test_allreduce_torch_cupy(ray_start_single_node_2_gpus): # import torch world_size = 2 actors, _ = get_actors_group(world_size) - ray.wait([actors[1].set_buffer.remote(torch.ones(10,).cuda())]) + ray.wait([actors[1].set_buffer.remote(torch.ones(10, ).cuda())]) results = ray.get([a.do_work.remote() for a in actors]) - assert (results[0] == cp.ones((10,)) * world_size).all() + assert (results[0] == cp.ones((10, )) * world_size).all() - ray.wait([actors[0].set_buffer.remote(torch.ones(10,))]) - ray.wait([actors[1].set_buffer.remote(cp.ones(10,))]) + ray.wait([actors[0].set_buffer.remote(torch.ones(10, ))]) + ray.wait([actors[1].set_buffer.remote(cp.ones(10, ))]) with pytest.raises(RuntimeError): results = ray.get([a.do_work.remote() for a in actors]) diff --git a/python/ray/util/collective/tests/util.py b/python/ray/util/collective/tests/util.py index e273fa3d2a7e..a2ef1728e68e 100644 --- a/python/ray/util/collective/tests/util.py +++ b/python/ray/util/collective/tests/util.py @@ -8,9 +8,13 @@ @ray.remote(num_gpus=1) class Worker: def __init__(self): - self.buffer = cp.ones((10,), dtype=cp.float32) + self.buffer = cp.ones((10, ), dtype=cp.float32) - def init_group(self, world_size, rank, backend=Backend.NCCL, group_name='default'): + def init_group(self, + world_size, + rank, + backend=Backend.NCCL, + group_name='default'): col.init_collective_group(world_size, rank, backend, group_name) return True diff --git a/python/ray/util/collective/util.py b/python/ray/util/collective/util.py index 91b2c12f4b01..fefee1373c2a 100644 --- a/python/ray/util/collective/util.py +++ b/python/ray/util/collective/util.py @@ -2,13 +2,13 @@ import ray import logging - logger = logging.getLogger(__name__) @ray.remote class NCCLUniqueIDStore(object): """NCCLUniqueID Store as a named actor.""" + def __init__(self, name): self.name = name self.nccl_id = None @@ -19,5 +19,7 @@ def set_id(self, uid): def get_id(self): if not self.nccl_id: - logger.warning('The NCCL ID has not been set yet for store {}'.format(self.name)) + logger.warning( + 'The NCCL ID has not been set yet for store {}'.format( + self.name)) return self.nccl_id From d7e4aee66cec32f51c4e000b45bb4f36846af932 Mon Sep 17 00:00:00 2001 From: Hao Zhang Date: Sat, 5 Dec 2020 03:04:40 -0500 Subject: [PATCH 29/51] linting 2 --- python/ray/util/collective/__init__.py | 8 ++++- python/ray/util/collective/collective.py | 34 ++++++++++--------- .../collective/collective_group/__init__.py | 2 ++ .../collective_group/mpi_collective_group.py | 2 +- .../collective_group/nccl_collective_group.py | 22 ++++++++---- 5 files changed, 43 insertions(+), 25 deletions(-) diff --git a/python/ray/util/collective/__init__.py b/python/ray/util/collective/__init__.py index e8b185a0796c..eff8b2e33b9e 100644 --- a/python/ray/util/collective/__init__.py +++ b/python/ray/util/collective/__init__.py @@ -1 +1,7 @@ -from .collective import * +from .collective import nccl_available, mpi_available, is_group_initialized, \ + init_collective_group, destroy_collective_group, get_rank, get_world_size, \ + allreduce, barrier + +__all__ = ["nccl_available", "mpi_available", "is_group_initialized", + "init_collective_group", "destroy_collective_group", "get_rank", + "get_world_size", "allreduce", "barrier"] diff --git a/python/ray/util/collective/collective.py b/python/ray/util/collective/collective.py index a22bdde377ae..c3e20423c768 100644 --- a/python/ray/util/collective/collective.py +++ b/python/ray/util/collective/collective.py @@ -6,16 +6,17 @@ from ray.util.collective.const import get_nccl_store_name # Get the availability information first by importing information -_MPI_AVAILABLE = True +_MPI_AVAILABLE = False _NCCL_AVAILABLE = True -try: - from ray.util.collective.collective_group.mpi_collective_group import MPIGroup -except ImportError: - _MPI_AVAILABLE = False +# try: +# from ray.util.collective.collective_group.mpi_collective_group +# import MPIGroup +# except ImportError: +# _MPI_AVAILABLE = False try: - from ray.util.collective.collective_group.nccl_collective_group import NCCLGroup + from ray.util.collective.collective_group import NCCLGroup except ImportError: _NCCL_AVAILABLE = False @@ -32,7 +33,7 @@ def mpi_available(): class GroupManager(object): """ - Use this class to manage the collective groups we created so far; + Use this class to manage the collective groups we created so far. """ @@ -45,7 +46,8 @@ def create_collective_group(self, backend, world_size, rank, group_name): """ The entry to create new collective groups and register in the manager. - Put the registration and the group information into the manager metadata as well. + Put the registration and the group information into the manager + metadata as well. """ backend = types.Backend(backend) if backend == types.Backend.MPI: @@ -62,7 +64,7 @@ def create_collective_group(self, backend, world_size, rank, group_name): name=store_name, lifetime="detached").remote(store_name) ray.wait([store.set_id.remote(group_uid)]) - logger.debug('creating NCCL group: {}'.format(group_name)) + logger.debug("creating NCCL group: '{}'".format(group_name)) g = NCCLGroup(world_size, rank, group_name) self._name_group_map[group_name] = g self._group_name_map[g] = group_name @@ -119,7 +121,7 @@ def is_group_initialized(group_name): def init_collective_group(world_size: int, rank: int, backend=types.Backend.NCCL, - group_name: str = 'default'): + group_name: str = "default"): """ Initialize a collective group inside an actor process. @@ -142,7 +144,7 @@ def init_collective_group(world_size: int, .format(group_name)) if _group_mgr.is_group_exist(group_name): - raise RuntimeError('Trying to initialize a group twice.') + raise RuntimeError("Trying to initialize a group twice.") assert (world_size > 0) assert (rank >= 0) @@ -150,13 +152,13 @@ def init_collective_group(world_size: int, _group_mgr.create_collective_group(backend, world_size, rank, group_name) -def destroy_collective_group(group_name: str = 'default') -> None: +def destroy_collective_group(group_name: str = "default") -> None: """Destroy a collective group given its group name.""" global _group_mgr _group_mgr.destroy_collective_group(group_name) -def get_rank(group_name: str = 'default') -> int: +def get_rank(group_name: str = "default") -> int: """ Return the rank of this process in the given group. @@ -174,7 +176,7 @@ def get_rank(group_name: str = 'default') -> int: return g.rank -def get_world_size(group_name='default') -> int: +def get_world_size(group_name="default") -> int: """ Return the size of the collective gropu with the given name. @@ -238,10 +240,10 @@ def _check_backend_availability(backend: types.Backend): """Check whether the backend is available.""" if backend == types.Backend.MPI: if not mpi_available(): - raise RuntimeError('MPI is not available.') + raise RuntimeError("MPI is not available.") elif backend == types.Backend.NCCL: if not nccl_available(): - raise RuntimeError('NCCL is not available.') + raise RuntimeError("NCCL is not available.") def _check_single_tensor_input(tensor): diff --git a/python/ray/util/collective/collective_group/__init__.py b/python/ray/util/collective/collective_group/__init__.py index b2b3ab3ee7ee..38a1b53a1717 100644 --- a/python/ray/util/collective/collective_group/__init__.py +++ b/python/ray/util/collective/collective_group/__init__.py @@ -1 +1,3 @@ from .nccl_collective_group import NCCLGroup + +__all__ = ['NCCLGroup'] diff --git a/python/ray/util/collective/collective_group/mpi_collective_group.py b/python/ray/util/collective/collective_group/mpi_collective_group.py index 92e40ca7847b..b23ee3178b6e 100644 --- a/python/ray/util/collective/collective_group/mpi_collective_group.py +++ b/python/ray/util/collective/collective_group/mpi_collective_group.py @@ -2,4 +2,4 @@ try: import mpi4py except ImportError: - raise + raise \ No newline at end of file diff --git a/python/ray/util/collective/collective_group/nccl_collective_group.py b/python/ray/util/collective/collective_group/nccl_collective_group.py index 2256485af57b..68cb98f1319f 100644 --- a/python/ray/util/collective/collective_group/nccl_collective_group.py +++ b/python/ray/util/collective/collective_group/nccl_collective_group.py @@ -6,8 +6,10 @@ import cupy from ray.util.collective.collective_group import nccl_util -from ray.util.collective.collective_group.base_collective_group import BaseGroup -from ray.util.collective.types import AllReduceOptions, BarrierOptions, Backend +from ray.util.collective.collective_group.base_collective_group \ + import BaseGroup +from ray.util.collective.types import AllReduceOptions, \ + BarrierOptions, Backend from ray.util.collective.const import get_nccl_store_name logger = logging.getLogger(__name__) @@ -20,7 +22,7 @@ class Rendezvous: def __init__(self, group_name): if not group_name: - raise ValueError('Empty meeting point.') + raise ValueError("Invalid group name.") self._group_name = group_name self._store_name = None self._store = None @@ -86,10 +88,10 @@ def __init__(self, world_size, rank, group_name): self._nccl_comm = None if nccl_util.get_nccl_build_version() < 2000: - raise RuntimeError('NCCL in Ray requires NCCL>=2.0.') + raise RuntimeError("NCCL in Ray requires NCCL >= 2.0.") # TODO(Hao): check version here if nccl_util.get_nccl_runtime_version() < 2704: - logger.warning('NCCL send/recv calls requires NCCL>=2.7.4') + logger.warning("NCCL send/recv calls requires NCCL>=2.7.4") self._rendezvous = Rendezvous(self.group_name) self._rendezvous.meet() @@ -101,7 +103,10 @@ def __init__(self, world_size, rank, group_name): self._barrier_tensor = cupy.array([1]) def _init_nccl_unique_id(self): - """Init the NCCL unique ID required for setting up NCCL communicator.""" + """ + Init the NCCL unique ID required for setting up NCCL communicator. + + """ self._nccl_uid = self._rendezvous.get_nccl_id() @property @@ -109,7 +114,10 @@ def nccl_uid(self): return self._nccl_uid def destroy_group(self): - """Destroy the group and release the NCCL communicators safely.""" + """ + Destroy the group and release the NCCL communicators safely. + + """ if self._nccl_comm is not None: self.barrier() # We also need a barrier call here. From cd62a503c0822344c0cd2a1c7b39cea8bdd8592f Mon Sep 17 00:00:00 2001 From: Hao Zhang Date: Sat, 5 Dec 2020 03:25:34 -0500 Subject: [PATCH 30/51] linting 2 --- python/ray/util/collective/tests/conftest.py | 2 +- .../tests/test_collective_2_nodes_4_gpus.py | 28 ++++++++-------- .../test_collective_single_node_2_gpus.py | 33 ++++++++++--------- python/ray/util/collective/tests/util.py | 10 +++--- python/ray/util/collective/types.py | 16 ++++----- python/ray/util/collective/util.py | 2 +- 6 files changed, 46 insertions(+), 45 deletions(-) diff --git a/python/ray/util/collective/tests/conftest.py b/python/ray/util/collective/tests/conftest.py index ad52e2e39881..23e7041ae8e7 100644 --- a/python/ray/util/collective/tests/conftest.py +++ b/python/ray/util/collective/tests/conftest.py @@ -31,7 +31,7 @@ def ray_start_single_node_2_gpus(): # my own on-premise cluster before run this fixture. @pytest.fixture def ray_start_distributed_2_nodes_4_gpus(): - ray.init('auto') + ray.init("auto") yield clean_up() ray.shutdown() diff --git a/python/ray/util/collective/tests/test_collective_2_nodes_4_gpus.py b/python/ray/util/collective/tests/test_collective_2_nodes_4_gpus.py index edc735ca2a19..5d6c9d767117 100644 --- a/python/ray/util/collective/tests/test_collective_2_nodes_4_gpus.py +++ b/python/ray/util/collective/tests/test_collective_2_nodes_4_gpus.py @@ -10,7 +10,7 @@ from .util import Worker -def get_actors_group(num_workers=2, group_name='default', backend='nccl'): +def get_actors_group(num_workers=2, group_name="default", backend="nccl"): actors = [Worker.remote() for i in range(num_workers)] world_size = num_workers init_results = ray.get([ @@ -21,7 +21,7 @@ def get_actors_group(num_workers=2, group_name='default', backend='nccl'): @pytest.mark.parametrize("world_size", [2, 3, 4]) -@pytest.mark.parametrize("group_name", ['default', 'test', '123?34!']) +@pytest.mark.parametrize("group_name", ["default", "test", "123?34!"]) def test_init_two_actors(ray_start_distributed_2_nodes_4_gpus, world_size, group_name): actors, results = get_actors_group(world_size, group_name) @@ -52,8 +52,9 @@ def test_get_rank(ray_start_distributed_2_nodes_4_gpus, world_size): actor1_rank = ray.get(actors[1].report_rank.remote()) assert actor1_rank == 1 - # create a second group with a different name, and different order of ranks. - new_group_name = 'default2' + # create a second group with a different name, and different + # orders of ranks. + new_group_name = "default2" ranks = [i for i in range(world_size)] shuffle(ranks) _ = ray.get([ @@ -93,15 +94,15 @@ def test_is_group_initialized(ray_start_distributed_2_nodes_4_gpus): actor0_is_init = ray.get(actors[0].report_is_group_initialized.remote()) assert actor0_is_init actor0_is_init = ray.get( - actors[0].report_is_group_initialized.remote('random')) + actors[0].report_is_group_initialized.remote("random")) assert not actor0_is_init actor0_is_init = ray.get( - actors[0].report_is_group_initialized.remote('123')) + actors[0].report_is_group_initialized.remote("123")) assert not actor0_is_init actor1_is_init = ray.get(actors[0].report_is_group_initialized.remote()) assert actor1_is_init actor1_is_init = ray.get( - actors[0].report_is_group_initialized.remote('456')) + actors[0].report_is_group_initialized.remote("456")) assert not actor1_is_init @@ -114,18 +115,18 @@ def test_destroy_group(ray_start_distributed_2_nodes_4_gpus): assert not actor0_is_init # should go well as the group `random` does not exist at all - ray.wait([actors[0].destroy_group.remote('random')]) + ray.wait([actors[0].destroy_group.remote("random")]) actor1_is_init = ray.get(actors[1].report_is_group_initialized.remote()) assert actor1_is_init - ray.wait([actors[1].destroy_group.remote('random')]) + ray.wait([actors[1].destroy_group.remote("random")]) actor1_is_init = ray.get(actors[1].report_is_group_initialized.remote()) assert actor1_is_init - ray.wait([actors[1].destroy_group.remote('default')]) + ray.wait([actors[1].destroy_group.remote("default")]) actor1_is_init = ray.get(actors[1].report_is_group_initialized.remote()) assert actor1_is_init == False for i in [2, 3]: - ray.wait([actors[i].destroy_group.remote('default')]) + ray.wait([actors[i].destroy_group.remote("default")]) # Now reconstruct the group using the same name init_results = ray.get([ @@ -140,11 +141,10 @@ def test_destroy_group(ray_start_distributed_2_nodes_4_gpus): assert actor1_is_init -@pytest.mark.parametrize("group_name", ['default', 'test', '123?34!']) +@pytest.mark.parametrize("group_name", ["default", "test", "123?34!"]) @pytest.mark.parametrize("world_size", [2, 3, 4]) def test_allreduce_different_name(ray_start_distributed_2_nodes_4_gpus, group_name, world_size): - world_size = 2 actors, _ = get_actors_group(num_workers=world_size, group_name=group_name) results = ray.get([a.do_work.remote(group_name) for a in actors]) assert (results[0] == cp.ones((10, ), dtype=cp.float32) * world_size).all() @@ -205,7 +205,7 @@ def test_allreduce_multiple_group(ray_start_distributed_2_nodes_4_gpus, for i, actor in enumerate(actors) ]) for i in range(num_groups): - group_name = 'default' if i == 0 else str(i) + group_name = "default" if i == 0 else str(i) results = ray.get([a.do_work.remote(group_name) for a in actors]) assert (results[0] == cp.ones( (10, ), dtype=cp.float32) * (world_size**(i + 1))).all() diff --git a/python/ray/util/collective/tests/test_collective_single_node_2_gpus.py b/python/ray/util/collective/tests/test_collective_single_node_2_gpus.py index 17c876c77370..267375e29eb9 100644 --- a/python/ray/util/collective/tests/test_collective_single_node_2_gpus.py +++ b/python/ray/util/collective/tests/test_collective_single_node_2_gpus.py @@ -9,8 +9,8 @@ from .util import Worker -def get_actors_group(num_workers=2, group_name='default', backend='nccl'): - actors = [Worker.remote() for i in range(num_workers)] +def get_actors_group(num_workers=2, group_name="default", backend="nccl"): + actors = [Worker.remote() for _ in range(num_workers)] world_size = num_workers init_results = ray.get([ actor.init_group.remote(world_size, i, backend, group_name) @@ -19,7 +19,7 @@ def get_actors_group(num_workers=2, group_name='default', backend='nccl'): return actors, init_results -@pytest.mark.parametrize("group_name", ['default', 'test', '123?34!']) +@pytest.mark.parametrize("group_name", ["default", "test", "123?34!"]) def test_init_two_actors(ray_start_single_node_2_gpus, group_name): world_size = 2 actors, results = get_actors_group(world_size, group_name) @@ -49,8 +49,9 @@ def test_get_rank(ray_start_single_node_2_gpus): actor1_rank = ray.get(actors[1].report_rank.remote()) assert actor1_rank == 1 - # create a second group with a different name, and different order of ranks. - new_group_name = 'default2' + # create a second group with a different name, + # and different order of ranks. + new_group_name = "default2" _ = ray.get([ actor.init_group.remote( world_size, world_size - 1 - i, group_name=new_group_name) @@ -75,10 +76,10 @@ def test_availability(ray_start_single_node_2_gpus): actors, _ = get_actors_group(world_size) actor0_nccl_availability = ray.get( actors[0].report_nccl_availability.remote()) - assert (actor0_nccl_availability) + assert actor0_nccl_availability actor0_mpi_availability = ray.get( actors[0].report_mpi_availability.remote()) - assert (actor0_mpi_availability == False) + assert not actor0_mpi_availability def test_is_group_initialized(ray_start_single_node_2_gpus): @@ -88,15 +89,15 @@ def test_is_group_initialized(ray_start_single_node_2_gpus): actor0_is_init = ray.get(actors[0].report_is_group_initialized.remote()) assert actor0_is_init actor0_is_init = ray.get( - actors[0].report_is_group_initialized.remote('random')) + actors[0].report_is_group_initialized.remote("random")) assert not actor0_is_init actor0_is_init = ray.get( - actors[0].report_is_group_initialized.remote('123')) + actors[0].report_is_group_initialized.remote("123")) assert not actor0_is_init actor1_is_init = ray.get(actors[0].report_is_group_initialized.remote()) assert actor1_is_init actor1_is_init = ray.get( - actors[0].report_is_group_initialized.remote('456')) + actors[0].report_is_group_initialized.remote("456")) assert not actor1_is_init @@ -109,16 +110,16 @@ def test_destroy_group(ray_start_single_node_2_gpus): assert not actor0_is_init # should go well as the group `random` does not exist at all - ray.wait([actors[0].destroy_group.remote('random')]) + ray.wait([actors[0].destroy_group.remote("random")]) actor1_is_init = ray.get(actors[1].report_is_group_initialized.remote()) assert actor1_is_init - ray.wait([actors[1].destroy_group.remote('random')]) + ray.wait([actors[1].destroy_group.remote("random")]) actor1_is_init = ray.get(actors[1].report_is_group_initialized.remote()) assert actor1_is_init - ray.wait([actors[1].destroy_group.remote('default')]) + ray.wait([actors[1].destroy_group.remote("default")]) actor1_is_init = ray.get(actors[1].report_is_group_initialized.remote()) - assert actor1_is_init == False + assert not actor1_is_init # Now reconstruct the group using the same name init_results = ray.get([ @@ -133,7 +134,7 @@ def test_destroy_group(ray_start_single_node_2_gpus): assert actor1_is_init -@pytest.mark.parametrize("group_name", ['default', 'test', '123?34!']) +@pytest.mark.parametrize("group_name", ["default", "test", "123?34!"]) # @pytest.mark.parametrize("group_name", ['123?34!']) def test_allreduce_different_name(ray_start_single_node_2_gpus, group_name): world_size = 2 @@ -197,7 +198,7 @@ def test_allreduce_multiple_group(ray_start_single_node_2_gpus, for i, actor in enumerate(actors) ]) for i in range(num_groups): - group_name = 'default' if i == 0 else str(i) + group_name = "default" if i == 0 else str(i) results = ray.get([a.do_work.remote(group_name) for a in actors]) assert (results[0] == cp.ones( (10, ), dtype=cp.float32) * (world_size**(i + 1))).all() diff --git a/python/ray/util/collective/tests/util.py b/python/ray/util/collective/tests/util.py index a2ef1728e68e..d59294d3f5bd 100644 --- a/python/ray/util/collective/tests/util.py +++ b/python/ray/util/collective/tests/util.py @@ -14,7 +14,7 @@ def init_group(self, world_size, rank, backend=Backend.NCCL, - group_name='default'): + group_name="default"): col.init_collective_group(world_size, rank, backend, group_name) return True @@ -26,15 +26,15 @@ def do_work(self, group_name="default", op=ReduceOp.SUM): col.allreduce(self.buffer, group_name, op) return self.buffer - def destroy_group(self, group_name='default'): + def destroy_group(self, group_name="default"): col.destroy_collective_group(group_name) return True - def report_rank(self, group_name='default'): + def report_rank(self, group_name="default"): rank = col.get_rank(group_name) return rank - def report_world_size(self, group_name='default'): + def report_world_size(self, group_name="default"): ws = col.get_world_size(group_name) return ws @@ -46,6 +46,6 @@ def report_mpi_availability(self): avail = col.mpi_available() return avail - def report_is_group_initialized(self, group_name='default'): + def report_is_group_initialized(self, group_name="default"): is_init = col.is_group_initialized(group_name) return is_init diff --git a/python/ray/util/collective/types.py b/python/ray/util/collective/types.py index 566bd69ab0bf..023a8c4316d5 100644 --- a/python/ray/util/collective/types.py +++ b/python/ray/util/collective/types.py @@ -7,17 +7,17 @@ _CUPY_AVAILABLE = True try: - import numpy as np + import numpy as np # noqa: F401 except ImportError: _NUMPY_AVAILABLE = False try: - import torch as th + import torch as th # noqa: F401 except ImportError: _TORCH_AVAILABLE = False try: - import cupy as cp + import cupy as cp # noqa: F401 except ImportError: _CUPY_AVAILABLE = False @@ -35,15 +35,15 @@ def torch_available(): class Backend(object): - """A class to represent different backends, in case the user string is too error-prone.""" - NCCL = 'nccl' - MPI = 'mpi' - UNRECOGNIZED = 'unrecognized' + """A class to represent different backends.""" + NCCL = "nccl" + MPI = "mpi" + UNRECOGNIZED = "unrecognized" def __new__(cls, name: str): backend = getattr(Backend, name.upper(), Backend.UNRECOGNIZED) if backend == Backend.UNRECOGNIZED: - raise ValueError("Unrecognized backend: '{}'" + raise ValueError("Unrecognized backend: '{}'. " "Only NCCL is supported".format(name)) if backend == Backend.MPI: raise NotImplementedError() diff --git a/python/ray/util/collective/util.py b/python/ray/util/collective/util.py index fefee1373c2a..9dc8d10affae 100644 --- a/python/ray/util/collective/util.py +++ b/python/ray/util/collective/util.py @@ -20,6 +20,6 @@ def set_id(self, uid): def get_id(self): if not self.nccl_id: logger.warning( - 'The NCCL ID has not been set yet for store {}'.format( + "The NCCL ID has not been set yet for store {}".format( self.name)) return self.nccl_id From bdb90de0b7ddde73db795bae7fe662ed5bd9c1b3 Mon Sep 17 00:00:00 2001 From: Hao Zhang Date: Sat, 5 Dec 2020 03:38:28 -0500 Subject: [PATCH 31/51] linting 3 --- python/ray/util/collective/__init__.py | 4 ++-- .../collective/collective_group/__init__.py | 2 +- .../collective_group/mpi_collective_group.py | 4 ++-- .../collective_group/nccl_collective_group.py | 12 ++++++---- .../collective/collective_group/nccl_util.py | 22 +++++++++---------- python/ray/util/collective/const.py | 4 ++-- .../examples/nccl_allreduce_example.py | 6 ++--- python/ray/util/collective/tests/conftest.py | 2 +- .../tests/test_collective_2_nodes_4_gpus.py | 2 +- 9 files changed, 30 insertions(+), 28 deletions(-) diff --git a/python/ray/util/collective/__init__.py b/python/ray/util/collective/__init__.py index eff8b2e33b9e..3215ece9d4fc 100644 --- a/python/ray/util/collective/__init__.py +++ b/python/ray/util/collective/__init__.py @@ -1,6 +1,6 @@ from .collective import nccl_available, mpi_available, is_group_initialized, \ - init_collective_group, destroy_collective_group, get_rank, get_world_size, \ - allreduce, barrier + init_collective_group, destroy_collective_group, get_rank, \ + get_world_size, allreduce, barrier __all__ = ["nccl_available", "mpi_available", "is_group_initialized", "init_collective_group", "destroy_collective_group", "get_rank", diff --git a/python/ray/util/collective/collective_group/__init__.py b/python/ray/util/collective/collective_group/__init__.py index 38a1b53a1717..c8ecc463ea97 100644 --- a/python/ray/util/collective/collective_group/__init__.py +++ b/python/ray/util/collective/collective_group/__init__.py @@ -1,3 +1,3 @@ from .nccl_collective_group import NCCLGroup -__all__ = ['NCCLGroup'] +__all__ = ["NCCLGroup"] diff --git a/python/ray/util/collective/collective_group/mpi_collective_group.py b/python/ray/util/collective/collective_group/mpi_collective_group.py index b23ee3178b6e..e045ac7160db 100644 --- a/python/ray/util/collective/collective_group/mpi_collective_group.py +++ b/python/ray/util/collective/collective_group/mpi_collective_group.py @@ -1,5 +1,5 @@ """Implementation of the MPI collective group.""" try: - import mpi4py + import mpi4py # noqa: F401 except ImportError: - raise \ No newline at end of file + raise diff --git a/python/ray/util/collective/collective_group/nccl_collective_group.py b/python/ray/util/collective/collective_group/nccl_collective_group.py index 68cb98f1319f..0ec9fd004a2b 100644 --- a/python/ray/util/collective/collective_group/nccl_collective_group.py +++ b/python/ray/util/collective/collective_group/nccl_collective_group.py @@ -15,8 +15,9 @@ logger = logging.getLogger(__name__) # TODO(Hao): -# (1) stream management, instead of using the default stream, using a dedicate stream -# (2) communicator management, adding a caching mechanism to enable +# (1) stream management, instead of using the default stream, +# using a dedicate stream +# (2) communicator management and support num_gpus > 2 per actor. class Rendezvous: @@ -158,7 +159,7 @@ def allreduce(self, tensor, allreduce_options=AllReduceOptions()): def barrier(self, barrier_options=BarrierOptions()): """ - Blocks until all processes in the communicator have reached this barrier call. + Blocks until all processes in the communicator have reached this barrier. Args: barrier_options: @@ -168,7 +169,10 @@ def barrier(self, barrier_options=BarrierOptions()): self.allreduce(self._barrier_tensor) def _get_nccl_communicator(self): - """Create a new NCCL communicator for the collective task, or using a cached communicator.""" + """ + Create or use a cached NCCL communicator for the collective task. + + """ # TODO(Hao): later change this to use device keys and query from cache. # TODO(Hao): implement a thin wrapper if not self._nccl_comm: diff --git a/python/ray/util/collective/collective_group/nccl_util.py b/python/ray/util/collective/collective_group/nccl_util.py index cff6aeb74091..48cb5682d078 100644 --- a/python/ray/util/collective/collective_group/nccl_util.py +++ b/python/ray/util/collective/collective_group/nccl_util.py @@ -3,13 +3,11 @@ try: import cupy from cupy.cuda import nccl - from cupy.cuda.nccl import get_unique_id from cupy.cuda.nccl import get_version from cupy.cuda.nccl import get_build_version from cupy.cuda.nccl import NcclCommunicator - from cupy.cuda.nccl import NcclError except ImportError: - raise ImportError('NCCL in Ray requires Cupy being available!') + raise ImportError("NCCL in Ray requires Cupy being available!") from ray.util.collective.types import ReduceOp, torch_available @@ -43,7 +41,7 @@ def get_nccl_build_version(): def get_nccl_runtime_version(): - return get_build_version() + return get_version() def create_nccl_communicator(world_size, nccl_unique_id, rank): @@ -53,11 +51,11 @@ def create_nccl_communicator(world_size, nccl_unique_id, rank): TODO(Hao): verify that the call has big overhead. Returns: - comm(type): an NCCL communicator. + comm(nccl.ncclComm_t): an NCCL communicator. """ - # TODO(Hao): make this inside the NCCLComm class, and implement the abort method. - # Make it RAII + # TODO(Hao): make this inside the NCCLComm class, + # and implement the abort method. Make it RAII. comm = NcclCommunicator(world_size, nccl_unique_id, rank) return comm @@ -71,7 +69,7 @@ def get_nccl_reduce_op(reduce_op): """ if reduce_op not in NCCL_REDUCE_OP_MAP: raise RuntimeError( - 'NCCL does not support ReduceOp: {}'.format(reduce_op)) + "NCCL does not support ReduceOp: '{}'".format(reduce_op)) return NCCL_REDUCE_OP_MAP[reduce_op] @@ -82,7 +80,7 @@ def get_nccl_tensor_dtype(tensor): if torch_available(): if isinstance(tensor, torch.Tensor): return TORCH_NCCL_DTYPE_MAP[tensor.dtype] - raise ValueError('Unsupported tensor type') + raise ValueError("Unsupported tensor type") def get_tensor_ptr(tensor): @@ -94,9 +92,9 @@ def get_tensor_ptr(tensor): if torch_available(): if isinstance(tensor, torch.Tensor): if not tensor.is_cuda: - raise RuntimeError('torch tensor must be on gpu.') + raise RuntimeError("torch tensor must be on gpu.") return tensor.data_ptr() - raise ValueError('Unsupported tensor type') + raise ValueError("Unsupported tensor type.") def get_tensor_n_elements(tensor): @@ -106,4 +104,4 @@ def get_tensor_n_elements(tensor): if torch_available(): if isinstance(tensor, torch.Tensor): return torch.numel(tensor) - raise ValueError('Unsupported tensor type') + raise ValueError("Unsupported tensor type") diff --git a/python/ray/util/collective/const.py b/python/ray/util/collective/const.py index c6862437b803..15fc4b78bee2 100644 --- a/python/ray/util/collective/const.py +++ b/python/ray/util/collective/const.py @@ -3,11 +3,11 @@ Contains constants used to setup collective groups. """ -NAMED_ACTOR_STORE_SUFFIX = '_unique_id_actor' +NAMED_ACTOR_STORE_SUFFIX = "_unique_id_actor" def get_nccl_store_name(group_name): """Generate the unique name for the NCCLUniqueID store (named actor).""" if not group_name: - raise ValueError('group_name is None.') + raise ValueError("group_name is None.") return group_name + NAMED_ACTOR_STORE_SUFFIX diff --git a/python/ray/util/collective/examples/nccl_allreduce_example.py b/python/ray/util/collective/examples/nccl_allreduce_example.py index 82bf01ed5cbd..9b20f4c6a393 100644 --- a/python/ray/util/collective/examples/nccl_allreduce_example.py +++ b/python/ray/util/collective/examples/nccl_allreduce_example.py @@ -11,16 +11,16 @@ def __init__(self): self.recv = cp.zeros((4, ), dtype=cp.float32) def setup(self, world_size, rank): - collective.init_collective_group('nccl', world_size, rank, 'default') + collective.init_collective_group("nccl", world_size, rank, "default") return True def compute(self): - collective.allreduce(self.send, 'default') + collective.allreduce(self.send, "default") print(self.send) return self.send def destroy(self): - collective.destroy_group('') + collective.destroy_group() if __name__ == "__main__": diff --git a/python/ray/util/collective/tests/conftest.py b/python/ray/util/collective/tests/conftest.py index 23e7041ae8e7..b84a01742bf8 100644 --- a/python/ray/util/collective/tests/conftest.py +++ b/python/ray/util/collective/tests/conftest.py @@ -6,7 +6,7 @@ def clean_up(): - group_names = ['default', 'test', '123?34!', 'default2', 'randome'] + group_names = ["default", "test", "123?34!", "default2", "random"] group_names.extend([str(i) for i in range(10)]) for group_name in group_names: try: diff --git a/python/ray/util/collective/tests/test_collective_2_nodes_4_gpus.py b/python/ray/util/collective/tests/test_collective_2_nodes_4_gpus.py index 5d6c9d767117..20b8193906ce 100644 --- a/python/ray/util/collective/tests/test_collective_2_nodes_4_gpus.py +++ b/python/ray/util/collective/tests/test_collective_2_nodes_4_gpus.py @@ -124,7 +124,7 @@ def test_destroy_group(ray_start_distributed_2_nodes_4_gpus): assert actor1_is_init ray.wait([actors[1].destroy_group.remote("default")]) actor1_is_init = ray.get(actors[1].report_is_group_initialized.remote()) - assert actor1_is_init == False + assert not actor1_is_init for i in [2, 3]: ray.wait([actors[i].destroy_group.remote("default")]) From 63973ec00a14f5ece30dc29459ccd265c44378a1 Mon Sep 17 00:00:00 2001 From: Hao Zhang Date: Sat, 5 Dec 2020 03:39:56 -0500 Subject: [PATCH 32/51] linting 4 --- .../util/collective/collective_group/nccl_collective_group.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/util/collective/collective_group/nccl_collective_group.py b/python/ray/util/collective/collective_group/nccl_collective_group.py index 0ec9fd004a2b..404fc79ed261 100644 --- a/python/ray/util/collective/collective_group/nccl_collective_group.py +++ b/python/ray/util/collective/collective_group/nccl_collective_group.py @@ -159,7 +159,7 @@ def allreduce(self, tensor, allreduce_options=AllReduceOptions()): def barrier(self, barrier_options=BarrierOptions()): """ - Blocks until all processes in the communicator have reached this barrier. + Blocks until all processes reach this barrier. Args: barrier_options: From e02789146305628b945f18e6c17e3b1681cbf0db Mon Sep 17 00:00:00 2001 From: Hao Zhang Date: Sat, 5 Dec 2020 03:41:17 -0500 Subject: [PATCH 33/51] linting 5 --- python/ray/util/collective/__init__.py | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/python/ray/util/collective/__init__.py b/python/ray/util/collective/__init__.py index 3215ece9d4fc..68fcb78d444e 100644 --- a/python/ray/util/collective/__init__.py +++ b/python/ray/util/collective/__init__.py @@ -2,6 +2,8 @@ init_collective_group, destroy_collective_group, get_rank, \ get_world_size, allreduce, barrier -__all__ = ["nccl_available", "mpi_available", "is_group_initialized", - "init_collective_group", "destroy_collective_group", "get_rank", - "get_world_size", "allreduce", "barrier"] +__all__ = [ + "nccl_available", "mpi_available", "is_group_initialized", + "init_collective_group", "destroy_collective_group", "get_rank", + "get_world_size", "allreduce", "barrier" +] From 4136fa97e2c8d05c62108294417a4b3eb20e2bde Mon Sep 17 00:00:00 2001 From: Hao Zhang Date: Sat, 5 Dec 2020 03:42:52 -0500 Subject: [PATCH 34/51] linting 6 --- .../ray/util/collective/tests/ray_start_distributed_script.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/util/collective/tests/ray_start_distributed_script.sh b/python/ray/util/collective/tests/ray_start_distributed_script.sh index 1dda0011b377..21d701b82f54 100644 --- a/python/ray/util/collective/tests/ray_start_distributed_script.sh +++ b/python/ray/util/collective/tests/ray_start_distributed_script.sh @@ -10,6 +10,6 @@ sleep 0.5 NCCL_DEBUG=INFO NCCL_SOCKET_IFNAME=enp179s0f0 ray start --head --object-manager-port=8076 --resources='{"machine":1}' --object-store-memory=32359738368 sleep 0.5 -echo "=> node $i" +echo "=> remote node..." ssh -o StrictHostKeyChecking=no -i /home/hao.zhang/.ssh/arion.pem hao.zhang@10.20.41.120 "source $ROOT_DIR/env/bin/activate; ray stop; NCCL_DEBUG=INFO NCCL_SOCKET_IFNAME=enp179s0f0 ray start --address='$MY_IPADDR:6379' --object-manager-port=8076 --resources='{\"machine\":1}' --object-store-memory=32359738368"; wait From ac603adf46ba1475adc4dfa542b3c645968f3137 Mon Sep 17 00:00:00 2001 From: Dacheng Li Date: Sat, 5 Dec 2020 22:54:23 -0500 Subject: [PATCH 35/51] 2.1 2.2 --- python/ray/actor.py | 16 ++++- python/ray/util/collective/collective.py | 65 ++++++++++++++++++- .../examples/nccl_allreduce_example_2_1.py | 33 ++++++++++ .../examples/nccl_allreduce_example_2_2.py | 33 ++++++++++ python/ray/util/collective/util.py | 44 +++++++++++++ 5 files changed, 187 insertions(+), 4 deletions(-) create mode 100644 python/ray/util/collective/examples/nccl_allreduce_example_2_1.py create mode 100644 python/ray/util/collective/examples/nccl_allreduce_example_2_2.py diff --git a/python/ray/actor.py b/python/ray/actor.py index b8981ca3d23e..5cdcb3c69aa6 100644 --- a/python/ray/actor.py +++ b/python/ray/actor.py @@ -8,6 +8,7 @@ import ray.worker from ray.util.placement_group import ( PlacementGroup, check_placement_group_index, get_current_placement_group) +from ray.util.collective.util import collective_to_envs from ray import ActorClassID, Language from ray._raylet import PythonFunctionDescriptor @@ -423,7 +424,8 @@ def options(self, placement_group=None, placement_group_bundle_index=-1, placement_group_capture_child_tasks=None, - override_environment_variables=None): + override_environment_variables=None, + collective=None): """Configures and overrides the actor instantiation parameters. The arguments are the same as those that can be passed @@ -465,7 +467,8 @@ def remote(self, *args, **kwargs): placement_group_capture_child_tasks=( placement_group_capture_child_tasks), override_environment_variables=( - override_environment_variables)) + override_environment_variables), + collective=collective) return ActorOptionWrapper() @@ -486,7 +489,8 @@ def _remote(self, placement_group=None, placement_group_bundle_index=-1, placement_group_capture_child_tasks=None, - override_environment_variables=None): + override_environment_variables=None, + collective=None): """Create an actor. This method allows more flexibility than the remote method because @@ -526,6 +530,7 @@ def _remote(self, override_environment_variables: Environment variables to override and/or introduce for this actor. This is a dictionary mapping variable names to their values. + collective: what colletive configuration to use Returns: A handle to the newly created actor. @@ -656,6 +661,11 @@ def _remote(self, function_signature = meta.method_meta.signatures["__init__"] creation_args = signature.flatten_args(function_signature, args, kwargs) + + if collective: + override_environment_variables = collective_to_envs(collective, + override_environment_variables) + actor_id = worker.core_worker.create_actor( meta.language, meta.actor_creation_function_descriptor, diff --git a/python/ray/util/collective/collective.py b/python/ray/util/collective/collective.py index c3e20423c768..e1f16adcf734 100644 --- a/python/ray/util/collective/collective.py +++ b/python/ray/util/collective/collective.py @@ -1,5 +1,6 @@ """APIs exposed under the namespace ray.util.collective.""" import logging +import os import ray from ray.util.collective import types @@ -151,6 +152,49 @@ def init_collective_group(world_size: int, assert (rank < world_size) _group_mgr.create_collective_group(backend, world_size, rank, group_name) +def declare_collective_group(actors, group_options): + """ + Declare a list of actors in a collective group with group options. This function + should be called in a driver process. + Args: + actors (list): a list of actors to be set in a collective group. + group_options (dict): a dictionary that contains group_name(str), world_size(int), + rank(list of int, e.g. [0,1] means the first actor is rank 0, and + the second actor is rank 1), backend(str) + """ + try: + group_name = group_options["group_name"] + world_size = group_options["world_size"] + rank = group_options["rank"] + backend = group_options["backend"] + except: + raise ValueError("group options incomplete.") + + backend = types.Backend(backend) + _check_backend_availability(backend) + + name = "info" + group_name + try: + ray.get_actor(name) + raise RuntimeError('Trying to initialize a group twice.') + except: + pass + + if len(rank) != len(actors): + raise RuntimeError("Each actor should correspond to one rank.") + + if set(rank) != set(range(len(rank))): + raise RuntimeError("Rank must be a permutation from 0 to len-1.") + + assert world_size > 0 + assert all(rank) >= 0 and all(rank) < world_size + + from ray.util.collective.util import Info + # store the information into a NamedActor that can be accessed later/ + name = "info" + group_name + actors_id = [a._ray_actor_id for a in actors] + info = Info.options(name=name, lifetime="detached").remote() + ray.wait([info.set_info.remote(actors_id, world_size, rank, backend)]) def destroy_collective_group(group_name: str = "default") -> None: """Destroy a collective group given its group name.""" @@ -229,8 +273,27 @@ def barrier(group_name): def _check_and_get_group(group_name): """Check the existence and return the group handle.""" + global _group_mgr if not is_group_initialized(group_name): - raise RuntimeError("The collective group '{}' is not " + # try loading from remote info store + try: + # if the information is stored in an Info object, get and create the group. + name = "info" + group_name + mgr = ray.get_actor(name=name) + ids, world_size, rank, backend = ray.get(mgr.get_info.remote()) + worker = ray.worker.global_worker + id_ = worker.core_worker.get_actor_id() + r = rank[ids.index(id_)] + _group_mgr.create_collective_group(backend, world_size, r, group_name) + except: + # check if this group is initialized using options() + if os.environ["collective_group_name"] == group_name: + rank = int(os.environ["collective_rank"]) + world_size = int(os.environ["collective_world_size"]) + backend = os.environ["collective_backend"] + _group_mgr.create_collective_group(backend, world_size, rank, group_name) + else: + raise RuntimeError("The collective group '{}' is not " "initialized in the process.".format(group_name)) g = _group_mgr.get_group_by_name(group_name) return g diff --git a/python/ray/util/collective/examples/nccl_allreduce_example_2_1.py b/python/ray/util/collective/examples/nccl_allreduce_example_2_1.py new file mode 100644 index 000000000000..3726a6da9341 --- /dev/null +++ b/python/ray/util/collective/examples/nccl_allreduce_example_2_1.py @@ -0,0 +1,33 @@ +import os +import cupy as cp +import ray + +import ray.util.collective as collective + +@ray.remote(num_gpus=1) +class Worker: + def __init__(self): + self.send = cp.ones((4,), dtype=cp.float32) + + def compute(self): + collective.allreduce(self.send, '177') + return self.send + + def destroy(self): + collective.destroy_group('') + +if __name__ == "__main__": + ray.init(num_gpus=2) + + num_workers = 2 + workers = [] + for i in range(num_workers): + _options = {'group_name' : '177', + 'world_size' : 2, + 'rank' : i, + 'backend' : 'nccl'} + w = Worker.options(collective=_options).remote() + workers.append(w) + results = ray.get([w.compute.remote() for w in workers]) + print(results) + ray.shutdown() diff --git a/python/ray/util/collective/examples/nccl_allreduce_example_2_2.py b/python/ray/util/collective/examples/nccl_allreduce_example_2_2.py new file mode 100644 index 000000000000..06abf0c7c559 --- /dev/null +++ b/python/ray/util/collective/examples/nccl_allreduce_example_2_2.py @@ -0,0 +1,33 @@ +import cupy as cp +import ray + +import ray.util.collective as collective + +@ray.remote(num_gpus=1) +class Worker: + def __init__(self): + self.send = cp.ones((4,), dtype=cp.float32) + + def compute(self): + collective.allreduce(self.send, '177') + return self.send + + def destroy(self): + collective.destroy_group('') + +if __name__ == "__main__": + ray.init(num_gpus=2) + + num_workers = 2 + workers = [] + for i in range(num_workers): + w = Worker.remote() + workers.append(w) + options = {'group_name' : '177', + 'world_size' : 2, + 'rank' : [0, 1], + 'backend' : 'nccl'} + collective.declare_collective_group(workers, options) + results = ray.get([w.compute.remote() for w in workers]) + print(results) + ray.shutdown() diff --git a/python/ray/util/collective/util.py b/python/ray/util/collective/util.py index 9dc8d10affae..1e180a8760d4 100644 --- a/python/ray/util/collective/util.py +++ b/python/ray/util/collective/util.py @@ -23,3 +23,47 @@ def get_id(self): "The NCCL ID has not been set yet for store {}".format( self.name)) return self.nccl_id + +@ray.remote +class Info: + """Store the collective information for groups created through declare_collective_group(). + Should be used as a NamedActor.""" + + def __init__(self): + self.ids = None + self.world_size = -1 + self.rank = -1 + self.backend = None + + def set_info(self, ids, world_size, rank, backend): + """Store collective information.""" + self.ids = ids + self.world_size = world_size + self.rank = rank + self.backend = backend + + def get_info(self): + """Get previously stored collective information.""" + return self.ids, self.world_size, self.rank, self.backend + +def collective_to_envs(collective, envs): + """A helper method that get information from collective and add to envs. + Args: + collective(dict): collective information + envs(dict): os environment dict + + Returns: + envs(dict): modified os environment dict + """ + + if envs is not None: + assert all(["collective_group_name", "collective_rank", "collective_world_size", + "collective_backend"]) not in envs + else: + envs = {} + envs["collective_group_name"] = str(collective["group_name"]) + envs["collective_rank"] = str(collective["rank"]) + envs["collective_world_size"] = str(collective["world_size"]) + envs["collective_backend"] = str(collective["backend"]) + + return envs From a8f6898dea4e590072bb9a1e22a153dd22e22516 Mon Sep 17 00:00:00 2001 From: Dacheng Li Date: Sat, 5 Dec 2020 23:24:36 -0500 Subject: [PATCH 36/51] fix small bugs --- python/ray/actor.py | 2 +- python/ray/util/collective/__init__.py | 8 +++---- python/ray/util/collective/collective.py | 7 +++--- .../examples/nccl_allreduce_example.py | 6 ++--- python/ray/util/collective/util.py | 22 ----------------- python/ray/util/collective_utils.py | 24 +++++++++++++++++++ 6 files changed, 36 insertions(+), 33 deletions(-) create mode 100644 python/ray/util/collective_utils.py diff --git a/python/ray/actor.py b/python/ray/actor.py index 5cdcb3c69aa6..48c2e4ae630d 100644 --- a/python/ray/actor.py +++ b/python/ray/actor.py @@ -8,7 +8,7 @@ import ray.worker from ray.util.placement_group import ( PlacementGroup, check_placement_group_index, get_current_placement_group) -from ray.util.collective.util import collective_to_envs +from ray.util.collective_utils import collective_to_envs from ray import ActorClassID, Language from ray._raylet import PythonFunctionDescriptor diff --git a/python/ray/util/collective/__init__.py b/python/ray/util/collective/__init__.py index 68fcb78d444e..d605d5d1164a 100644 --- a/python/ray/util/collective/__init__.py +++ b/python/ray/util/collective/__init__.py @@ -1,9 +1,9 @@ from .collective import nccl_available, mpi_available, is_group_initialized, \ - init_collective_group, destroy_collective_group, get_rank, \ - get_world_size, allreduce, barrier + init_collective_group, declare_collective_group,\ + destroy_collective_group, get_rank, get_world_size, allreduce, barrier __all__ = [ "nccl_available", "mpi_available", "is_group_initialized", - "init_collective_group", "destroy_collective_group", "get_rank", - "get_world_size", "allreduce", "barrier" + "init_collective_group", "declare_collective_group,", "destroy_collective_group", + "get_rank", "get_world_size", "allreduce", "barrier" ] diff --git a/python/ray/util/collective/collective.py b/python/ray/util/collective/collective.py index e1f16adcf734..07a3612b2e9b 100644 --- a/python/ray/util/collective/collective.py +++ b/python/ray/util/collective/collective.py @@ -191,7 +191,7 @@ def declare_collective_group(actors, group_options): from ray.util.collective.util import Info # store the information into a NamedActor that can be accessed later/ - name = "info" + group_name + name = "info_" + group_name actors_id = [a._ray_actor_id for a in actors] info = Info.options(name=name, lifetime="detached").remote() ray.wait([info.set_info.remote(actors_id, world_size, rank, backend)]) @@ -278,7 +278,7 @@ def _check_and_get_group(group_name): # try loading from remote info store try: # if the information is stored in an Info object, get and create the group. - name = "info" + group_name + name = "info_" + group_name mgr = ray.get_actor(name=name) ids, world_size, rank, backend = ray.get(mgr.get_info.remote()) worker = ray.worker.global_worker @@ -287,7 +287,8 @@ def _check_and_get_group(group_name): _group_mgr.create_collective_group(backend, world_size, r, group_name) except: # check if this group is initialized using options() - if os.environ["collective_group_name"] == group_name: + if "collective_group_name" in os.environ and\ + os.environ["collective_group_name"] == group_name: rank = int(os.environ["collective_rank"]) world_size = int(os.environ["collective_world_size"]) backend = os.environ["collective_backend"] diff --git a/python/ray/util/collective/examples/nccl_allreduce_example.py b/python/ray/util/collective/examples/nccl_allreduce_example.py index 9b20f4c6a393..a0b6574229de 100644 --- a/python/ray/util/collective/examples/nccl_allreduce_example.py +++ b/python/ray/util/collective/examples/nccl_allreduce_example.py @@ -11,7 +11,7 @@ def __init__(self): self.recv = cp.zeros((4, ), dtype=cp.float32) def setup(self, world_size, rank): - collective.init_collective_group("nccl", world_size, rank, "default") + collective.init_collective_group(world_size, rank, "nccl", "default") return True def compute(self): @@ -36,7 +36,7 @@ def destroy(self): w = Worker.remote() workers.append(w) init_rets.append(w.setup.remote(num_workers, i)) - # m = ray.get(init_rets) + ray.get(init_rets) results = ray.get([w.compute.remote() for w in workers]) - # print(results) + print(results) ray.shutdown() diff --git a/python/ray/util/collective/util.py b/python/ray/util/collective/util.py index 1e180a8760d4..99cb3fb04c54 100644 --- a/python/ray/util/collective/util.py +++ b/python/ray/util/collective/util.py @@ -45,25 +45,3 @@ def set_info(self, ids, world_size, rank, backend): def get_info(self): """Get previously stored collective information.""" return self.ids, self.world_size, self.rank, self.backend - -def collective_to_envs(collective, envs): - """A helper method that get information from collective and add to envs. - Args: - collective(dict): collective information - envs(dict): os environment dict - - Returns: - envs(dict): modified os environment dict - """ - - if envs is not None: - assert all(["collective_group_name", "collective_rank", "collective_world_size", - "collective_backend"]) not in envs - else: - envs = {} - envs["collective_group_name"] = str(collective["group_name"]) - envs["collective_rank"] = str(collective["rank"]) - envs["collective_world_size"] = str(collective["world_size"]) - envs["collective_backend"] = str(collective["backend"]) - - return envs diff --git a/python/ray/util/collective_utils.py b/python/ray/util/collective_utils.py new file mode 100644 index 000000000000..e59354ab4aa6 --- /dev/null +++ b/python/ray/util/collective_utils.py @@ -0,0 +1,24 @@ +"""Collections of collective util functions""" + +def collective_to_envs(collective, envs): + """A helper method that get information from collective and add to envs. + Args: + collective(dict): collective information + envs(dict): os environment dict + + Returns: + envs(dict): modified os environment dict + """ + + if envs is not None: + assert all(["collective_group_name", "collective_rank", "collective_world_size", + "collective_backend"]) not in envs + + else: + envs = {} + envs["collective_group_name"] = str(collective["group_name"]) + envs["collective_rank"] = str(collective["rank"]) + envs["collective_world_size"] = str(collective["world_size"]) + envs["collective_backend"] = str(collective["backend"]) + + return envs From af15ca58ef6799c55890567c10248293eba9d552 Mon Sep 17 00:00:00 2001 From: Hao Zhang Date: Tue, 15 Dec 2020 01:08:55 -0500 Subject: [PATCH 37/51] minor updates --- .../collective/examples/nccl_allreduce_example_actor_options.py | 1 - 1 file changed, 1 deletion(-) diff --git a/python/ray/util/collective/examples/nccl_allreduce_example_actor_options.py b/python/ray/util/collective/examples/nccl_allreduce_example_actor_options.py index 3726a6da9341..d8da3f34e6b3 100644 --- a/python/ray/util/collective/examples/nccl_allreduce_example_actor_options.py +++ b/python/ray/util/collective/examples/nccl_allreduce_example_actor_options.py @@ -1,4 +1,3 @@ -import os import cupy as cp import ray From 9abf10f0ad85cbd7cdb212c110bb5ac48b59d9f9 Mon Sep 17 00:00:00 2001 From: Hao Zhang Date: Tue, 15 Dec 2020 01:19:43 -0500 Subject: [PATCH 38/51] linting again --- python/ray/util/collective/__init__.py | 6 +++--- python/ray/util/collective/collective.py | 11 ++++++----- .../nccl_allreduce_example_actor_options.py | 12 +++++------- ...l_allreduce_example_declare_collective_group.py | 14 ++++++-------- python/ray/util/collective/util.py | 2 +- 5 files changed, 21 insertions(+), 24 deletions(-) diff --git a/python/ray/util/collective/__init__.py b/python/ray/util/collective/__init__.py index d605d5d1164a..2f4968ee0d09 100644 --- a/python/ray/util/collective/__init__.py +++ b/python/ray/util/collective/__init__.py @@ -1,9 +1,9 @@ from .collective import nccl_available, mpi_available, is_group_initialized, \ - init_collective_group, declare_collective_group,\ - destroy_collective_group, get_rank, get_world_size, allreduce, barrier + init_collective_group, declare_collective_group,destroy_collective_group, \ + get_rank, get_world_size, allreduce, barrier __all__ = [ "nccl_available", "mpi_available", "is_group_initialized", - "init_collective_group", "declare_collective_group,", "destroy_collective_group", + "init_collective_group", "declare_collective_group", "destroy_collective_group", "get_rank", "get_world_size", "allreduce", "barrier" ] diff --git a/python/ray/util/collective/collective.py b/python/ray/util/collective/collective.py index e165f4b6560e..5bc70879bddd 100644 --- a/python/ray/util/collective/collective.py +++ b/python/ray/util/collective/collective.py @@ -159,16 +159,17 @@ def declare_collective_group(actors, group_options): Args: actors (list): a list of actors to be set in a collective group. - group_options (dict): a dictionary that contains group_name(str), world_size(int), - rank(list of int, e.g. [0,1] means the first actor is rank 0, and - the second actor is rank 1), backend(str) + group_options (dict): a dictionary that contains group_name(str), + world_size(int), rank(list of int, e.g. [0,1] + means the first actor is rank 0, and the second + actor is rank 1), backend(str). """ try: group_name = group_options["group_name"] world_size = group_options["world_size"] rank = group_options["rank"] backend = group_options["backend"] - except: + except KeyError: raise ValueError("group options incomplete.") backend = types.Backend(backend) @@ -177,7 +178,7 @@ def declare_collective_group(actors, group_options): name = "info" + group_name try: ray.get_actor(name) - raise RuntimeError('Trying to initialize a group twice.') + raise RuntimeError("Trying to initialize a group twice.") except: pass diff --git a/python/ray/util/collective/examples/nccl_allreduce_example_actor_options.py b/python/ray/util/collective/examples/nccl_allreduce_example_actor_options.py index d8da3f34e6b3..421cb9ec9f1c 100644 --- a/python/ray/util/collective/examples/nccl_allreduce_example_actor_options.py +++ b/python/ray/util/collective/examples/nccl_allreduce_example_actor_options.py @@ -9,11 +9,9 @@ def __init__(self): self.send = cp.ones((4,), dtype=cp.float32) def compute(self): - collective.allreduce(self.send, '177') + collective.allreduce(self.send, "177") return self.send - def destroy(self): - collective.destroy_group('') if __name__ == "__main__": ray.init(num_gpus=2) @@ -21,10 +19,10 @@ def destroy(self): num_workers = 2 workers = [] for i in range(num_workers): - _options = {'group_name' : '177', - 'world_size' : 2, - 'rank' : i, - 'backend' : 'nccl'} + _options = {"group_name": "177", + "world_size": 2, + "rank": i, + "backend": "nccl"} w = Worker.options(collective=_options).remote() workers.append(w) results = ray.get([w.compute.remote() for w in workers]) diff --git a/python/ray/util/collective/examples/nccl_allreduce_example_declare_collective_group.py b/python/ray/util/collective/examples/nccl_allreduce_example_declare_collective_group.py index 06abf0c7c559..de74ebbffa5f 100644 --- a/python/ray/util/collective/examples/nccl_allreduce_example_declare_collective_group.py +++ b/python/ray/util/collective/examples/nccl_allreduce_example_declare_collective_group.py @@ -9,11 +9,9 @@ def __init__(self): self.send = cp.ones((4,), dtype=cp.float32) def compute(self): - collective.allreduce(self.send, '177') + collective.allreduce(self.send, "177") return self.send - def destroy(self): - collective.destroy_group('') if __name__ == "__main__": ray.init(num_gpus=2) @@ -23,11 +21,11 @@ def destroy(self): for i in range(num_workers): w = Worker.remote() workers.append(w) - options = {'group_name' : '177', - 'world_size' : 2, - 'rank' : [0, 1], - 'backend' : 'nccl'} - collective.declare_collective_group(workers, options) + _options = {"group_name": "177", + "world_size": 2, + "rank": i, + "backend": "nccl"} + collective.declare_collective_group(workers, _options) results = ray.get([w.compute.remote() for w in workers]) print(results) ray.shutdown() diff --git a/python/ray/util/collective/util.py b/python/ray/util/collective/util.py index 30e29f75fd13..12522b3f665d 100644 --- a/python/ray/util/collective/util.py +++ b/python/ray/util/collective/util.py @@ -45,7 +45,7 @@ def get_id(self): @ray.remote class Info: """ - Store the collective information for groups created through `declare_collective_group`. + Store the collective group information created via `declare_collective_group`. Note: Should be used as a NamedActor. """ From 6aad76dc5e79114ba074c41b3f0a2cd53fd37b10 Mon Sep 17 00:00:00 2001 From: Hao Zhang Date: Tue, 15 Dec 2020 01:21:16 -0500 Subject: [PATCH 39/51] auto linting --- python/ray/actor.py | 6 +++--- python/ray/util/collective/__init__.py | 5 +++-- python/ray/util/collective/collective.py | 13 ++++++++----- .../nccl_allreduce_example_actor_options.py | 13 ++++++++----- ...cl_allreduce_example_declare_collective_group.py | 13 ++++++++----- python/ray/util/collective_utils.py | 7 +++++-- 6 files changed, 35 insertions(+), 22 deletions(-) diff --git a/python/ray/actor.py b/python/ray/actor.py index 545ff5b65711..726e0dcb5a87 100644 --- a/python/ray/actor.py +++ b/python/ray/actor.py @@ -662,10 +662,10 @@ def _remote(self, function_signature = meta.method_meta.signatures["__init__"] creation_args = signature.flatten_args(function_signature, args, kwargs) - + if collective: - override_environment_variables = collective_to_envs(collective, - override_environment_variables) + override_environment_variables = collective_to_envs( + collective, override_environment_variables) actor_id = worker.core_worker.create_actor( meta.language, diff --git a/python/ray/util/collective/__init__.py b/python/ray/util/collective/__init__.py index 2f4968ee0d09..ee19a4a5b88b 100644 --- a/python/ray/util/collective/__init__.py +++ b/python/ray/util/collective/__init__.py @@ -4,6 +4,7 @@ __all__ = [ "nccl_available", "mpi_available", "is_group_initialized", - "init_collective_group", "declare_collective_group", "destroy_collective_group", - "get_rank", "get_world_size", "allreduce", "barrier" + "init_collective_group", "declare_collective_group", + "destroy_collective_group", "get_rank", "get_world_size", "allreduce", + "barrier" ] diff --git a/python/ray/util/collective/collective.py b/python/ray/util/collective/collective.py index 5bc70879bddd..5b69fe6ea7f4 100644 --- a/python/ray/util/collective/collective.py +++ b/python/ray/util/collective/collective.py @@ -190,7 +190,7 @@ def declare_collective_group(actors, group_options): assert world_size > 0 assert all(rank) >= 0 and all(rank) < world_size - + from ray.util.collective.util import Info # store the information into a NamedActor that can be accessed later/ name = "info_" + group_name @@ -291,7 +291,8 @@ def _check_and_get_group(group_name): worker = ray.worker.global_worker id_ = worker.core_worker.get_actor_id() r = rank[ids.index(id_)] - _group_mgr.create_collective_group(backend, world_size, r, group_name) + _group_mgr.create_collective_group(backend, world_size, r, + group_name) except: # check if this group is initialized using options() if "collective_group_name" in os.environ and \ @@ -299,10 +300,12 @@ def _check_and_get_group(group_name): rank = int(os.environ["collective_rank"]) world_size = int(os.environ["collective_world_size"]) backend = os.environ["collective_backend"] - _group_mgr.create_collective_group(backend, world_size, rank, group_name) + _group_mgr.create_collective_group(backend, world_size, rank, + group_name) else: - raise RuntimeError("The collective group '{}' is not " - "initialized in the process.".format(group_name)) + raise RuntimeError( + "The collective group '{}' is not " + "initialized in the process.".format(group_name)) g = _group_mgr.get_group_by_name(group_name) return g diff --git a/python/ray/util/collective/examples/nccl_allreduce_example_actor_options.py b/python/ray/util/collective/examples/nccl_allreduce_example_actor_options.py index 421cb9ec9f1c..123c55235a04 100644 --- a/python/ray/util/collective/examples/nccl_allreduce_example_actor_options.py +++ b/python/ray/util/collective/examples/nccl_allreduce_example_actor_options.py @@ -3,10 +3,11 @@ import ray.util.collective as collective + @ray.remote(num_gpus=1) class Worker: def __init__(self): - self.send = cp.ones((4,), dtype=cp.float32) + self.send = cp.ones((4, ), dtype=cp.float32) def compute(self): collective.allreduce(self.send, "177") @@ -19,10 +20,12 @@ def compute(self): num_workers = 2 workers = [] for i in range(num_workers): - _options = {"group_name": "177", - "world_size": 2, - "rank": i, - "backend": "nccl"} + _options = { + "group_name": "177", + "world_size": 2, + "rank": i, + "backend": "nccl" + } w = Worker.options(collective=_options).remote() workers.append(w) results = ray.get([w.compute.remote() for w in workers]) diff --git a/python/ray/util/collective/examples/nccl_allreduce_example_declare_collective_group.py b/python/ray/util/collective/examples/nccl_allreduce_example_declare_collective_group.py index de74ebbffa5f..2d4a9efdbe14 100644 --- a/python/ray/util/collective/examples/nccl_allreduce_example_declare_collective_group.py +++ b/python/ray/util/collective/examples/nccl_allreduce_example_declare_collective_group.py @@ -3,10 +3,11 @@ import ray.util.collective as collective + @ray.remote(num_gpus=1) class Worker: def __init__(self): - self.send = cp.ones((4,), dtype=cp.float32) + self.send = cp.ones((4, ), dtype=cp.float32) def compute(self): collective.allreduce(self.send, "177") @@ -21,10 +22,12 @@ def compute(self): for i in range(num_workers): w = Worker.remote() workers.append(w) - _options = {"group_name": "177", - "world_size": 2, - "rank": i, - "backend": "nccl"} + _options = { + "group_name": "177", + "world_size": 2, + "rank": i, + "backend": "nccl" + } collective.declare_collective_group(workers, _options) results = ray.get([w.compute.remote() for w in workers]) print(results) diff --git a/python/ray/util/collective_utils.py b/python/ray/util/collective_utils.py index e59354ab4aa6..5a580bc821c0 100644 --- a/python/ray/util/collective_utils.py +++ b/python/ray/util/collective_utils.py @@ -1,5 +1,6 @@ """Collections of collective util functions""" + def collective_to_envs(collective, envs): """A helper method that get information from collective and add to envs. Args: @@ -11,8 +12,10 @@ def collective_to_envs(collective, envs): """ if envs is not None: - assert all(["collective_group_name", "collective_rank", "collective_world_size", - "collective_backend"]) not in envs + assert all([ + "collective_group_name", "collective_rank", + "collective_world_size", "collective_backend" + ]) not in envs else: envs = {} From 82170bf9186c5078435e31a9966f36513882a4b9 Mon Sep 17 00:00:00 2001 From: Hao Zhang Date: Tue, 15 Dec 2020 01:29:54 -0500 Subject: [PATCH 40/51] linting 2 --- python/ray/util/collective/__init__.py | 2 +- python/ray/util/collective/collective.py | 11 ++++------- python/ray/util/collective/util.py | 2 +- 3 files changed, 6 insertions(+), 9 deletions(-) diff --git a/python/ray/util/collective/__init__.py b/python/ray/util/collective/__init__.py index ee19a4a5b88b..7da20dd0708b 100644 --- a/python/ray/util/collective/__init__.py +++ b/python/ray/util/collective/__init__.py @@ -1,5 +1,5 @@ from .collective import nccl_available, mpi_available, is_group_initialized, \ - init_collective_group, declare_collective_group,destroy_collective_group, \ + init_collective_group, declare_collective_group, destroy_collective_group, \ get_rank, get_world_size, allreduce, barrier __all__ = [ diff --git a/python/ray/util/collective/collective.py b/python/ray/util/collective/collective.py index 5b69fe6ea7f4..52906f1e56a9 100644 --- a/python/ray/util/collective/collective.py +++ b/python/ray/util/collective/collective.py @@ -176,12 +176,8 @@ def declare_collective_group(actors, group_options): _check_backend_availability(backend) name = "info" + group_name - try: - ray.get_actor(name) + if ray.get_actor(name): raise RuntimeError("Trying to initialize a group twice.") - except: - pass - if len(rank) != len(actors): raise RuntimeError("Each actor should correspond to one rank.") @@ -284,7 +280,8 @@ def _check_and_get_group(group_name): if not is_group_initialized(group_name): # try loading from remote info store try: - # if the information is stored in an Info object, get and create the group. + # if the information is stored in an Info object, + # get and create the group. name = "info_" + group_name mgr = ray.get_actor(name=name) ids, world_size, rank, backend = ray.get(mgr.get_info.remote()) @@ -293,7 +290,7 @@ def _check_and_get_group(group_name): r = rank[ids.index(id_)] _group_mgr.create_collective_group(backend, world_size, r, group_name) - except: + except ValueError: # check if this group is initialized using options() if "collective_group_name" in os.environ and \ os.environ["collective_group_name"] == group_name: diff --git a/python/ray/util/collective/util.py b/python/ray/util/collective/util.py index 12522b3f665d..932c9316bddd 100644 --- a/python/ray/util/collective/util.py +++ b/python/ray/util/collective/util.py @@ -45,7 +45,7 @@ def get_id(self): @ray.remote class Info: """ - Store the collective group information created via `declare_collective_group`. + Store the group information created via `declare_collective_group`. Note: Should be used as a NamedActor. """ From 93567cf5c51b3e688220d10d1ae9f0ad3e107e00 Mon Sep 17 00:00:00 2001 From: Hao Zhang Date: Tue, 15 Dec 2020 01:31:13 -0500 Subject: [PATCH 41/51] final linting --- python/ray/util/collective/__init__.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/ray/util/collective/__init__.py b/python/ray/util/collective/__init__.py index 7da20dd0708b..a5f19842e94c 100644 --- a/python/ray/util/collective/__init__.py +++ b/python/ray/util/collective/__init__.py @@ -1,6 +1,6 @@ from .collective import nccl_available, mpi_available, is_group_initialized, \ - init_collective_group, declare_collective_group, destroy_collective_group, \ - get_rank, get_world_size, allreduce, barrier + init_collective_group, declare_collective_group, \ + destroy_collective_group, get_rank, get_world_size, allreduce, barrier __all__ = [ "nccl_available", "mpi_available", "is_group_initialized", From d521fcbb1bd05e5279123fc4968f3ce5e3e84a99 Mon Sep 17 00:00:00 2001 From: Dacheng Li Date: Fri, 18 Dec 2020 10:53:13 -0500 Subject: [PATCH 42/51] Update python/ray/util/collective_utils.py Co-authored-by: Richard Liaw --- python/ray/util/collective_utils.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/ray/util/collective_utils.py b/python/ray/util/collective_utils.py index 5a580bc821c0..c664452ca3a8 100644 --- a/python/ray/util/collective_utils.py +++ b/python/ray/util/collective_utils.py @@ -3,6 +3,7 @@ def collective_to_envs(collective, envs): """A helper method that get information from collective and add to envs. + Args: collective(dict): collective information envs(dict): os environment dict From 5a71f2c4fbce9e4040dbe95ed8440011214b7d20 Mon Sep 17 00:00:00 2001 From: Dacheng Li Date: Fri, 18 Dec 2020 10:53:23 -0500 Subject: [PATCH 43/51] Update python/ray/util/collective_utils.py Co-authored-by: Richard Liaw --- python/ray/util/collective_utils.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/python/ray/util/collective_utils.py b/python/ray/util/collective_utils.py index c664452ca3a8..49e7626288e5 100644 --- a/python/ray/util/collective_utils.py +++ b/python/ray/util/collective_utils.py @@ -5,11 +5,11 @@ def collective_to_envs(collective, envs): """A helper method that get information from collective and add to envs. Args: - collective(dict): collective information - envs(dict): os environment dict + collective (dict): collective information + envs (dict): os environment dict Returns: - envs(dict): modified os environment dict + envs (dict): modified os environment dict """ if envs is not None: From 38daf7b1a0e60eee4392de022a85a95f90bb8b3f Mon Sep 17 00:00:00 2001 From: Dacheng Li Date: Fri, 18 Dec 2020 10:53:37 -0500 Subject: [PATCH 44/51] Update python/ray/util/collective_utils.py Co-authored-by: Richard Liaw --- python/ray/util/collective_utils.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/ray/util/collective_utils.py b/python/ray/util/collective_utils.py index 49e7626288e5..f4709e170237 100644 --- a/python/ray/util/collective_utils.py +++ b/python/ray/util/collective_utils.py @@ -13,10 +13,10 @@ def collective_to_envs(collective, envs): """ if envs is not None: - assert all([ + assert not any(k in envs for k in [ "collective_group_name", "collective_rank", "collective_world_size", "collective_backend" - ]) not in envs + ]) else: envs = {} From c5c414a9d117af81eaa952d4d5e52f103404ad2a Mon Sep 17 00:00:00 2001 From: Dacheng Li Date: Fri, 18 Dec 2020 13:13:06 -0500 Subject: [PATCH 45/51] added actor test --- python/ray/tests/test_actor.py | 31 +++++++++++++++++++ python/ray/util/collective/collective.py | 8 +++-- ...reduce_example_declare_collective_group.py | 2 +- 3 files changed, 38 insertions(+), 3 deletions(-) diff --git a/python/ray/tests/test_actor.py b/python/ray/tests/test_actor.py index 05d53c1b3b11..eb43c1b01830 100644 --- a/python/ray/tests/test_actor.py +++ b/python/ray/tests/test_actor.py @@ -14,6 +14,7 @@ import ray import ray.test_utils import ray.cluster_utils +import ray.util.collective as collective # NOTE: We have to import setproctitle after ray because we bundle setproctitle # with ray. @@ -237,6 +238,36 @@ def get_val(self): assert ray.get(g.remote()) == num_remote_functions - 1 +def test_collective_envs(ray_start_10_cpus): + @ray.remote + class Actor: + def __init__(self): + pass + + def f(self): + _group_name = os.getenv("collective_group_name") + _rank = os.getenv("collective_rank") + _world_size = os.getenv("collective_world_size") + _backend = os.getenv("collective_backend") + return _group_name, _rank, _world_size, _backend + + actors = [] + for i in range(2): + _options = { + "group_name": "177", + "world_size": 2, + "rank": i, + "backend": "nccl" + } + actor = Actor.options(collective=_options).remote() + actors.append(actor) + + _group_name, _rank, _world_size, _backend =\ + ray.get(actors[0].f.remote()) + assert _group_name == "177" + assert _world_size == "2" + assert _rank == "0" + assert _backend == "nccl" def test_actor_method_metadata_cache(ray_start_regular): class Actor(object): diff --git a/python/ray/util/collective/collective.py b/python/ray/util/collective/collective.py index 52906f1e56a9..d734d9ee8ba8 100644 --- a/python/ray/util/collective/collective.py +++ b/python/ray/util/collective/collective.py @@ -175,9 +175,13 @@ def declare_collective_group(actors, group_options): backend = types.Backend(backend) _check_backend_availability(backend) - name = "info" + group_name - if ray.get_actor(name): + name = "info_" + group_name + try: + ray.get_actor(name) raise RuntimeError("Trying to initialize a group twice.") + except ValueError as v: + pass + if len(rank) != len(actors): raise RuntimeError("Each actor should correspond to one rank.") diff --git a/python/ray/util/collective/examples/nccl_allreduce_example_declare_collective_group.py b/python/ray/util/collective/examples/nccl_allreduce_example_declare_collective_group.py index 2d4a9efdbe14..1d4572d7b9ee 100644 --- a/python/ray/util/collective/examples/nccl_allreduce_example_declare_collective_group.py +++ b/python/ray/util/collective/examples/nccl_allreduce_example_declare_collective_group.py @@ -25,7 +25,7 @@ def compute(self): _options = { "group_name": "177", "world_size": 2, - "rank": i, + "rank": [0, 1], "backend": "nccl" } collective.declare_collective_group(workers, _options) From b0ab663059002cd2db3df8280e611dc20ebc0194 Mon Sep 17 00:00:00 2001 From: Dacheng Li Date: Fri, 18 Dec 2020 13:19:53 -0500 Subject: [PATCH 46/51] lint --- python/ray/tests/test_actor.py | 6 +++--- python/ray/util/collective/collective.py | 2 +- python/ray/util/collective_utils.py | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/python/ray/tests/test_actor.py b/python/ray/tests/test_actor.py index eb43c1b01830..f5f8a5cd4f54 100644 --- a/python/ray/tests/test_actor.py +++ b/python/ray/tests/test_actor.py @@ -14,8 +14,6 @@ import ray import ray.test_utils import ray.cluster_utils -import ray.util.collective as collective - # NOTE: We have to import setproctitle after ray because we bundle setproctitle # with ray. import setproctitle @@ -238,6 +236,7 @@ def get_val(self): assert ray.get(g.remote()) == num_remote_functions - 1 + def test_collective_envs(ray_start_10_cpus): @ray.remote class Actor: @@ -263,12 +262,13 @@ def f(self): actors.append(actor) _group_name, _rank, _world_size, _backend =\ - ray.get(actors[0].f.remote()) + ray.get(actors[0].f.remote()) assert _group_name == "177" assert _world_size == "2" assert _rank == "0" assert _backend == "nccl" + def test_actor_method_metadata_cache(ray_start_regular): class Actor(object): pass diff --git a/python/ray/util/collective/collective.py b/python/ray/util/collective/collective.py index d734d9ee8ba8..96e6288c65dc 100644 --- a/python/ray/util/collective/collective.py +++ b/python/ray/util/collective/collective.py @@ -179,7 +179,7 @@ def declare_collective_group(actors, group_options): try: ray.get_actor(name) raise RuntimeError("Trying to initialize a group twice.") - except ValueError as v: + except ValueError: pass if len(rank) != len(actors): diff --git a/python/ray/util/collective_utils.py b/python/ray/util/collective_utils.py index f4709e170237..0d4a857477ee 100644 --- a/python/ray/util/collective_utils.py +++ b/python/ray/util/collective_utils.py @@ -3,7 +3,7 @@ def collective_to_envs(collective, envs): """A helper method that get information from collective and add to envs. - + Args: collective (dict): collective information envs (dict): os environment dict From 322c82224e95c4a8e9aae46b84079f00531ff856 Mon Sep 17 00:00:00 2001 From: Dacheng Li Date: Fri, 18 Dec 2020 13:27:21 -0500 Subject: [PATCH 47/51] remove local sh --- .../tests/ray_start_distributed_script.sh | 15 --------------- 1 file changed, 15 deletions(-) delete mode 100644 python/ray/util/collective/tests/ray_start_distributed_script.sh diff --git a/python/ray/util/collective/tests/ray_start_distributed_script.sh b/python/ray/util/collective/tests/ray_start_distributed_script.sh deleted file mode 100644 index 21d701b82f54..000000000000 --- a/python/ray/util/collective/tests/ray_start_distributed_script.sh +++ /dev/null @@ -1,15 +0,0 @@ -#!/bin/bash -ROOT_DIR=/home/hao.zhang/project/pycharm/ray -echo $ROOT_DIR -MY_IPADDR=10.20.41.115 -source $ROOT_DIR/env/bin/activate -echo $MY_IPADDR - -ray stop -sleep 0.5 -NCCL_DEBUG=INFO NCCL_SOCKET_IFNAME=enp179s0f0 ray start --head --object-manager-port=8076 --resources='{"machine":1}' --object-store-memory=32359738368 -sleep 0.5 - -echo "=> remote node..." -ssh -o StrictHostKeyChecking=no -i /home/hao.zhang/.ssh/arion.pem hao.zhang@10.20.41.120 "source $ROOT_DIR/env/bin/activate; ray stop; NCCL_DEBUG=INFO NCCL_SOCKET_IFNAME=enp179s0f0 ray start --address='$MY_IPADDR:6379' --object-manager-port=8076 --resources='{\"machine\":1}' --object-store-memory=32359738368"; -wait From 7c5f41439a5f12edffae54cdc5c115edfa1db5b0 Mon Sep 17 00:00:00 2001 From: Hao Zhang Date: Mon, 28 Dec 2020 02:19:35 -0500 Subject: [PATCH 48/51] address most of richard's comments --- python/ray/tests/test_actor.py | 2 +- python/ray/util/collective/__init__.py | 4 +-- python/ray/util/collective/collective.py | 30 ++++++++----------- ...reduce_example_declare_collective_group.py | 2 +- python/ray/util/collective/util.py | 3 +- 5 files changed, 18 insertions(+), 23 deletions(-) diff --git a/python/ray/tests/test_actor.py b/python/ray/tests/test_actor.py index f5f8a5cd4f54..1799d3d6985b 100644 --- a/python/ray/tests/test_actor.py +++ b/python/ray/tests/test_actor.py @@ -241,7 +241,7 @@ def test_collective_envs(ray_start_10_cpus): @ray.remote class Actor: def __init__(self): - pass + print("Actor created") def f(self): _group_name = os.getenv("collective_group_name") diff --git a/python/ray/util/collective/__init__.py b/python/ray/util/collective/__init__.py index a5f19842e94c..3fd74cbf7356 100644 --- a/python/ray/util/collective/__init__.py +++ b/python/ray/util/collective/__init__.py @@ -1,5 +1,5 @@ -from .collective import nccl_available, mpi_available, is_group_initialized, \ - init_collective_group, declare_collective_group, \ +from ray.util.collective.collective import nccl_available, mpi_available, \ + is_group_initialized, init_collective_group, declare_collective_group, \ destroy_collective_group, get_rank, get_world_size, allreduce, barrier __all__ = [ diff --git a/python/ray/util/collective/collective.py b/python/ray/util/collective/collective.py index 96e6288c65dc..4f00228014c4 100644 --- a/python/ray/util/collective/collective.py +++ b/python/ray/util/collective/collective.py @@ -1,6 +1,7 @@ """APIs exposed under the namespace ray.util.collective.""" import logging import os +from typing import List import numpy as np import ray @@ -151,9 +152,12 @@ def init_collective_group(world_size: int, _group_mgr.create_collective_group(backend, world_size, rank, group_name) -def declare_collective_group(actors, group_options): - """ - Declare a list of actors in a collective group with group options. +def declare_collective_group(actors, + world_size: int, + ranks: List[int], + backend=types.Backend.NCCL, + group_name: str = "default"): + """Declare a list of actors as a collective group. Note: This function should be called in a driver process. @@ -164,14 +168,6 @@ def declare_collective_group(actors, group_options): means the first actor is rank 0, and the second actor is rank 1), backend(str). """ - try: - group_name = group_options["group_name"] - world_size = group_options["world_size"] - rank = group_options["rank"] - backend = group_options["backend"] - except KeyError: - raise ValueError("group options incomplete.") - backend = types.Backend(backend) _check_backend_availability(backend) @@ -182,21 +178,21 @@ def declare_collective_group(actors, group_options): except ValueError: pass - if len(rank) != len(actors): + if len(ranks) != len(actors): raise RuntimeError("Each actor should correspond to one rank.") - if set(rank) != set(range(len(rank))): + if set(ranks) != set(range(len(ranks))): raise RuntimeError("Rank must be a permutation from 0 to len-1.") assert world_size > 0 - assert all(rank) >= 0 and all(rank) < world_size + assert all(ranks) >= 0 and all(ranks) < world_size from ray.util.collective.util import Info # store the information into a NamedActor that can be accessed later/ name = "info_" + group_name actors_id = [a._ray_actor_id for a in actors] info = Info.options(name=name, lifetime="detached").remote() - ray.wait([info.set_info.remote(actors_id, world_size, rank, backend)]) + ray.wait([info.set_info.remote(actors_id, world_size, ranks, backend)]) def destroy_collective_group(group_name: str = "default") -> None: @@ -294,7 +290,7 @@ def _check_and_get_group(group_name): r = rank[ids.index(id_)] _group_mgr.create_collective_group(backend, world_size, r, group_name) - except ValueError: + except ValueError as exc: # check if this group is initialized using options() if "collective_group_name" in os.environ and \ os.environ["collective_group_name"] == group_name: @@ -306,7 +302,7 @@ def _check_and_get_group(group_name): else: raise RuntimeError( "The collective group '{}' is not " - "initialized in the process.".format(group_name)) + "initialized in the process.".format(group_name)) from exc g = _group_mgr.get_group_by_name(group_name) return g diff --git a/python/ray/util/collective/examples/nccl_allreduce_example_declare_collective_group.py b/python/ray/util/collective/examples/nccl_allreduce_example_declare_collective_group.py index 1d4572d7b9ee..257d725eefce 100644 --- a/python/ray/util/collective/examples/nccl_allreduce_example_declare_collective_group.py +++ b/python/ray/util/collective/examples/nccl_allreduce_example_declare_collective_group.py @@ -28,7 +28,7 @@ def compute(self): "rank": [0, 1], "backend": "nccl" } - collective.declare_collective_group(workers, _options) + collective.declare_collective_group(workers, **_options) results = ray.get([w.compute.remote() for w in workers]) print(results) ray.shutdown() diff --git a/python/ray/util/collective/util.py b/python/ray/util/collective/util.py index 932c9316bddd..56a80673a88d 100644 --- a/python/ray/util/collective/util.py +++ b/python/ray/util/collective/util.py @@ -44,8 +44,7 @@ def get_id(self): @ray.remote class Info: - """ - Store the group information created via `declare_collective_group`. + """Store the group information created via `declare_collective_group`. Note: Should be used as a NamedActor. """ From 9018ccdb7947ce86dc9f290facb11a07531cb9e9 Mon Sep 17 00:00:00 2001 From: Hao Zhang Date: Mon, 28 Dec 2020 03:06:50 -0500 Subject: [PATCH 49/51] minor update --- .../examples/nccl_allreduce_example_declare_collective_group.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/util/collective/examples/nccl_allreduce_example_declare_collective_group.py b/python/ray/util/collective/examples/nccl_allreduce_example_declare_collective_group.py index 257d725eefce..9d0335dbab11 100644 --- a/python/ray/util/collective/examples/nccl_allreduce_example_declare_collective_group.py +++ b/python/ray/util/collective/examples/nccl_allreduce_example_declare_collective_group.py @@ -25,7 +25,7 @@ def compute(self): _options = { "group_name": "177", "world_size": 2, - "rank": [0, 1], + "ranks": [0, 1], "backend": "nccl" } collective.declare_collective_group(workers, **_options) From f50758de7d592aedbb60f50b9bbe8fce3cf3cb64 Mon Sep 17 00:00:00 2001 From: Hao Zhang Date: Mon, 4 Jan 2021 02:17:31 -0500 Subject: [PATCH 50/51] remove the actor.option() interface to avoid changes in ray core --- python/ray/actor.py | 15 ++------- python/ray/tests/test_actor.py | 32 ------------------ .../nccl_allreduce_example_actor_options.py | 33 ------------------- python/ray/util/collective_utils.py | 28 ---------------- 4 files changed, 3 insertions(+), 105 deletions(-) delete mode 100644 python/ray/util/collective/examples/nccl_allreduce_example_actor_options.py delete mode 100644 python/ray/util/collective_utils.py diff --git a/python/ray/actor.py b/python/ray/actor.py index 726e0dcb5a87..a1de9f5e4544 100644 --- a/python/ray/actor.py +++ b/python/ray/actor.py @@ -9,7 +9,6 @@ import ray.worker from ray.util.placement_group import ( PlacementGroup, check_placement_group_index, get_current_placement_group) -from ray.util.collective_utils import collective_to_envs from ray import ActorClassID, Language from ray._raylet import PythonFunctionDescriptor @@ -425,8 +424,7 @@ def options(self, placement_group=None, placement_group_bundle_index=-1, placement_group_capture_child_tasks=None, - override_environment_variables=None, - collective=None): + override_environment_variables=None): """Configures and overrides the actor instantiation parameters. The arguments are the same as those that can be passed @@ -468,8 +466,7 @@ def remote(self, *args, **kwargs): placement_group_capture_child_tasks=( placement_group_capture_child_tasks), override_environment_variables=( - override_environment_variables), - collective=collective) + override_environment_variables)) return ActorOptionWrapper() @@ -490,8 +487,7 @@ def _remote(self, placement_group=None, placement_group_bundle_index=-1, placement_group_capture_child_tasks=None, - override_environment_variables=None, - collective=None): + override_environment_variables=None): """Create an actor. This method allows more flexibility than the remote method because @@ -662,11 +658,6 @@ def _remote(self, function_signature = meta.method_meta.signatures["__init__"] creation_args = signature.flatten_args(function_signature, args, kwargs) - - if collective: - override_environment_variables = collective_to_envs( - collective, override_environment_variables) - actor_id = worker.core_worker.create_actor( meta.language, meta.actor_creation_function_descriptor, diff --git a/python/ray/tests/test_actor.py b/python/ray/tests/test_actor.py index 1799d3d6985b..3cdab3d926f2 100644 --- a/python/ray/tests/test_actor.py +++ b/python/ray/tests/test_actor.py @@ -237,38 +237,6 @@ def get_val(self): assert ray.get(g.remote()) == num_remote_functions - 1 -def test_collective_envs(ray_start_10_cpus): - @ray.remote - class Actor: - def __init__(self): - print("Actor created") - - def f(self): - _group_name = os.getenv("collective_group_name") - _rank = os.getenv("collective_rank") - _world_size = os.getenv("collective_world_size") - _backend = os.getenv("collective_backend") - return _group_name, _rank, _world_size, _backend - - actors = [] - for i in range(2): - _options = { - "group_name": "177", - "world_size": 2, - "rank": i, - "backend": "nccl" - } - actor = Actor.options(collective=_options).remote() - actors.append(actor) - - _group_name, _rank, _world_size, _backend =\ - ray.get(actors[0].f.remote()) - assert _group_name == "177" - assert _world_size == "2" - assert _rank == "0" - assert _backend == "nccl" - - def test_actor_method_metadata_cache(ray_start_regular): class Actor(object): pass diff --git a/python/ray/util/collective/examples/nccl_allreduce_example_actor_options.py b/python/ray/util/collective/examples/nccl_allreduce_example_actor_options.py deleted file mode 100644 index 123c55235a04..000000000000 --- a/python/ray/util/collective/examples/nccl_allreduce_example_actor_options.py +++ /dev/null @@ -1,33 +0,0 @@ -import cupy as cp -import ray - -import ray.util.collective as collective - - -@ray.remote(num_gpus=1) -class Worker: - def __init__(self): - self.send = cp.ones((4, ), dtype=cp.float32) - - def compute(self): - collective.allreduce(self.send, "177") - return self.send - - -if __name__ == "__main__": - ray.init(num_gpus=2) - - num_workers = 2 - workers = [] - for i in range(num_workers): - _options = { - "group_name": "177", - "world_size": 2, - "rank": i, - "backend": "nccl" - } - w = Worker.options(collective=_options).remote() - workers.append(w) - results = ray.get([w.compute.remote() for w in workers]) - print(results) - ray.shutdown() diff --git a/python/ray/util/collective_utils.py b/python/ray/util/collective_utils.py deleted file mode 100644 index 0d4a857477ee..000000000000 --- a/python/ray/util/collective_utils.py +++ /dev/null @@ -1,28 +0,0 @@ -"""Collections of collective util functions""" - - -def collective_to_envs(collective, envs): - """A helper method that get information from collective and add to envs. - - Args: - collective (dict): collective information - envs (dict): os environment dict - - Returns: - envs (dict): modified os environment dict - """ - - if envs is not None: - assert not any(k in envs for k in [ - "collective_group_name", "collective_rank", - "collective_world_size", "collective_backend" - ]) - - else: - envs = {} - envs["collective_group_name"] = str(collective["group_name"]) - envs["collective_rank"] = str(collective["rank"]) - envs["collective_world_size"] = str(collective["world_size"]) - envs["collective_backend"] = str(collective["backend"]) - - return envs From 1b3ba3b3b9a69a93670858a34324e4c7dfef0242 Mon Sep 17 00:00:00 2001 From: Hao Zhang Date: Mon, 4 Jan 2021 02:31:49 -0500 Subject: [PATCH 51/51] minor updates --- python/ray/actor.py | 1 - python/ray/tests/test_actor.py | 1 + 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/actor.py b/python/ray/actor.py index a1de9f5e4544..73255856dc29 100644 --- a/python/ray/actor.py +++ b/python/ray/actor.py @@ -527,7 +527,6 @@ def _remote(self, override_environment_variables: Environment variables to override and/or introduce for this actor. This is a dictionary mapping variable names to their values. - collective: what colletive configuration to use Returns: A handle to the newly created actor. diff --git a/python/ray/tests/test_actor.py b/python/ray/tests/test_actor.py index 3cdab3d926f2..05d53c1b3b11 100644 --- a/python/ray/tests/test_actor.py +++ b/python/ray/tests/test_actor.py @@ -14,6 +14,7 @@ import ray import ray.test_utils import ray.cluster_utils + # NOTE: We have to import setproctitle after ray because we bundle setproctitle # with ray. import setproctitle