diff --git a/doc/source/api.rst b/doc/source/api.rst index 799fd21ce..7d5f96d17 100644 --- a/doc/source/api.rst +++ b/doc/source/api.rst @@ -32,8 +32,8 @@ If there are GPUs available on the machine, you should specify this with the ray.init(num_cpus=20, num_gpus=2) -By default, Ray will use ``psutil.cpu_count()`` to determine the number of CPUs, -and by default the number of GPUs will be zero. +By default, Ray will use ``psutil.cpu_count()`` to determine the number of CPUs. +Ray will also attempt to automatically determine the number of GPUs. Instead of thinking about the number of "worker" processes on each node, we prefer to think in terms of the quantities of CPU and GPU resources on each diff --git a/doc/source/resources.rst b/doc/source/resources.rst index 784d32ed7..6b70254a5 100644 --- a/doc/source/resources.rst +++ b/doc/source/resources.rst @@ -33,7 +33,7 @@ through ``ray.init``, do the following. If the number of CPUs is unspecified, Ray will automatically determine the number by running ``psutil.cpu_count()``. If the number of GPUs is unspecified, -Ray will default to 0 GPUs. +Ray will attempt to automatically detect the number of GPUs. Specifying a task's CPU and GPU requirements ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ diff --git a/python/ray/services.py b/python/ray/services.py index b0f8fad96..3e28c41e7 100644 --- a/python/ray/services.py +++ b/python/ray/services.py @@ -279,6 +279,20 @@ def wait_for_redis_to_start(redis_ip_address, redis_port, num_retries=5): "configured properly.") +def _autodetect_num_gpus(): + """Attempt to detect the number of GPUs on this machine. + + TODO(rkn): This currently assumes Nvidia GPUs and Linux. + + Returns: + The number of GPUs if any were detected, otherwise 0. + """ + proc_gpus_path = "/proc/driver/nvidia/gpus" + if os.path.isdir(proc_gpus_path): + return len(os.listdir(proc_gpus_path)) + return 0 + + def _compute_version_info(): """Compute the versions of Python, cloudpickle, pyarrow, and Ray. @@ -679,6 +693,9 @@ def start_local_scheduler(redis_address, # By default, use the number of hardware execution threads for the # number of cores. resources["CPU"] = psutil.cpu_count() + if "GPU" not in resources: + # Try to automatically detect the number of GPUs. + resources["GPU"] = _autodetect_num_gpus() print("Starting local scheduler with the following resources: {}." .format(resources)) local_scheduler_name, p = ray.local_scheduler.start_local_scheduler( diff --git a/python/ray/worker.py b/python/ray/worker.py index 52d2fce9c..807a832e2 100644 --- a/python/ray/worker.py +++ b/python/ray/worker.py @@ -1224,7 +1224,9 @@ def _init(address_info=None, num_cpus (int): Number of cpus the user wishes all local schedulers to be configured with. num_gpus (int): Number of gpus the user wishes all local schedulers to - be configured with. + be configured with. If unspecified, Ray will attempt to autodetect + the number of GPUs available on the node (note that autodetection + currently only works for Nvidia GPUs). resources: A dictionary mapping resource names to the quantity of that resource available. num_redis_shards: The number of Redis shards to start in addition to