From 981349eaadc1942c57d12b843a1d221bf110c4e2 Mon Sep 17 00:00:00 2001 From: Jonathan Nitisastro Date: Sat, 18 Nov 2023 18:11:03 -0800 Subject: [PATCH 1/7] init rep doc Signed-off-by: Jonathan Nitisastro --- reps/2023-10-30-gpu-memory-support.md | 176 ++++++++++++++++++++++++++ 1 file changed, 176 insertions(+) create mode 100644 reps/2023-10-30-gpu-memory-support.md diff --git a/reps/2023-10-30-gpu-memory-support.md b/reps/2023-10-30-gpu-memory-support.md new file mode 100644 index 0000000..8794251 --- /dev/null +++ b/reps/2023-10-30-gpu-memory-support.md @@ -0,0 +1,176 @@ +# Ray GPU Memory Support + +## Summary + +Design docummentation for supporting gpu-memory on ray core resource scheduler. + +### General Motivation + +Currently, `ray` support `num_gpus` to scheduler resource to tasks/actors, which then assign either `num_gpus` number of gpus or a fraction of a single gpu if `num_gpus < 1`. However, there are often use cases where users want to specify gpu by amount of memory. The current workaround is ray users require to specify the gpu they want to use and convert the gpu memory into fractional value w.r.t the specified gpu. This new scheduler design will allows users to directly scheduler gpu resources by amount of memory. + +**Ray community users’s demand**: +https://github.com/ray-project/ray/issues/37574 + +https://github.com/ray-project/ray/issues/26929 + +https://discuss.ray.io/t/how-to-specify-gpu-resources-in-terms-of-gpu-ram-and-not-fraction-of-gpu/4128 + +https://discuss.ray.io/t/gpu-memory-aware-scheduling/2922/5 + +### Should this change be within `ray` or outside? + +Inside `ray` project since we want to add new parameter `_gpu_memory` to the `ray` remote function. + +## Stewardship + +### Required Reviewers +The proposal will be open to the public, but please suggest a few experienced Ray contributors in this technical domain whose comments will help this proposal. Ideally, the list should include Ray committers. + +@pcmoritz, @jjyao, @scv119 + +### Shepherd of the Proposal (should be a senior committer) +To make the review process more productive, the owner of each proposal should identify a **shepherd** (should be a senior Ray committer). The shepherd is responsible for working with the owner and making sure the proposal is in good shape (with necessary information) before marking it as ready for broader review. + +@jjyao + +## Design + +### API +Users will be able to specify amount of gpu memory to their ray tasks/actors using `gpu_memory` on `ray.remote`. The specified `gpu_memory` will be the amount of gpu resources from a single gpu that will be allocated to users ray tasks/actors. + +```python +# Request a fractional GPU with specified gpu_memory in megabytes. +# Mutually exclusive with num_gpus. +@ray.remote(_gpu_memory=1024) +def task: + … +``` + +When a Ray node is started, Ray will auto detect the number of GPUs and GPU memory of each GPU and set the GPU and _gpu_memory resources accordingly. Users also have the option to manually specify `_gpu_memory` resources as the sum of total gpu memory across all gpus in the node. The default value is `num_gpus` * total memory of the gpu type in the node. + +```bash +ray start # auto detection + +ray start –num_gpus=3 –gpu_memory=3000 # manual override, each gpu has 1000mb total memory +``` + +Note that GPU memory and compute unit is 1-1 conversion, means 20GB of gpu memory is equivalent to 0.5 fractional value of an `A100_40GB` gpu. So, for simplicity and consistency, ray doesn't allow users to specify both `num_gpus` and `_gpu_memory` in a single ray task/actor. + +```python +# Request a fractional GPU both num_gpus and _gpu_memory is not allowed +@ray.remote(_gpu_memory=1024, num_gpus=0.5) # raise ValueError exception +def not_allowed_task: + … + +# Request a fractional GPU with specified num_gpus. +@ray.remote(num_gpus=0.5) +def num_gpus_task: + … + +# Request a fractional GPU with specified _gpu_memory. +@ray.remote(_gpu_memory=1024) +def gpu_memory_task: + … +``` + +Additionally, users can still specify which gpu type they want to use by specifying `accelerator_type`. + +```python +# Request a fractional of A100 GPU with specified _gpu_memory +@ray.remote(_gpu_memory=1024, accelerator_type="NVIDIA_A100") +def nvidia_a100_gpu_task: + … + +# Requesting 30GB of gpu memory from a A10 GPU with 24GB of memory. +# Task won't be able to be scheduled. +@ray.remote(_gpu_memory=30 * 1024, accelerator_type="NVIDIA_TESLA_A10G") +def nvidia_a10_gpu_task: + … +``` + +### Implementation + +In the implementation phase of the project, we introduce a new `ResourceID` named `GPU_Memory` to specify the amount of GPU memory resources. `GPU_Memory` is treated as a GPU resource with a distinct representation, where the relationship is defined as `GPU` equals to `GPU_Memory` divided by the total memory of a single GPU in the node as GPU resources in the node is homogeneous. Despite their distinct representations, both `GPU` and `GPU_Memory` signify the same underlying resource, with the caveat that Ray currently supports only a single GPU. + +The primary implementation entails the automatic detection of GPU memory during the initialization of a Ray cluster. Subsequently, when users specify resources using `ray.remote`, particularly the `gpu_memory` resource, Ray will calculate the `num_gpus` from the specified `gpu_memory` in the `ResourceRequest`. This calculation is carried out by a function called `ConvertRelativeResource`, which updates `ResourceSet` from the `ResourceRequest` relative to the resources in the assigned node. Additionally, we incorporate a round-up mechanism during the conversion to `num_gpus`, ensuring precision up to $10^{-4}$. + +There are two pivotal options in the major implementation of this project, each with distinct characteristics: + +#### Option 1: +In Option 1, we choose to store `GPU_Memory` as part of the NodeResource. This implementation ensures synchronization between GPU and `GPU_Memory`. During node feasibility checks and resource allocation, the `ConvertRelativeResource` function performs two conversions: calculating `gpu_memory` if `num_gpus` is specified and vice versa. + +```c++ +const ResourceSet NodeResources::ConvertRelativeResource( + const ResourceSet &resource) const { + ResourceSet adjusted_resource = resource; + double total_gpu_memory = this->total.Get(ResourceID::GPU_Memory()).Double() / + this->total.Get(ResourceID::GPU()).Double(); // get single GPU total memory + // convert gpu_memory to GPU + if (resource.Has(ResourceID::GPU_Memory())) { + double num_gpus_request = 0; + double gpu_memory_request = resource.Get(ResourceID::GPU()).Double(); + if (total_gpu_memory > 0) { + // round up to closes kResourceUnitScaling + num_gpus_request = + (resource.Get(ResourceID::GPU_Memory()).Double() / total_gpu_memory) + + 1 / static_cast(2 * kResourceUnitScaling); + // update gpu_memory after roundup + gpu_memory_request = num_gpus_request * total_gpu_memory; + } + adjusted_resource.Set(ResourceID::GPU(), num_gpus_request); + adjusted_resource.Set(ResourceID::GPU_Memory(), gpu_memory_request); + } else if (resource.Has(ResourceID::GPU())) { + double gpu_memory_request = 0; + if (total_gpu_memory > 0) { + gpu_memory_request = + (resource.Get(ResourceID::GPU()).Double() * total_gpu_memory) + + 0.5; + } + adjusted_resource.Set(ResourceID::GPU_Memory(), gpu_memory_request); + } + return adjusted_resource; +} +``` + +Pros: +- Enhanced Observability: Users can see the remaining GPU memory resources after roundup allocation, providing detailed insights. + +Cons: +- Synchronization Overhead: Requires synchronization for both `GPU` and `GPU_Memory`, introducing an additional layer of complexity by updating both `GPU` and `GPU_Memory` for rounding up. +- Resource Duality: Users need to grasp that both resources, `GPU` and `GPU_Memory`, essentially denote the same underlying resource. + +#### Option 2: +In Option 2, we opt to store only GPU and convert `gpu_memory` to GPU. This implementation involves saving metadata of the single node's `total_gpu_memory`. `ConvertRelativeResource` converts `gpu_memory` in `ResourceRequest` to `num_gpus` based on the single node's `total_gpu_memory` during node feasibility checks and resource allocation. + +```c++ +const ResourceSet NodeResources::ConvertRelativeResource( + const ResourceSet &resource) const { + ResourceSet adjusted_resource = resource; + // convert gpu_memory to GPU + if (resource.Has(ResourceID::GPU_Memory())) { + double total_gpu_memory = 0; + if (this->labels.find("gpu_memory") != this->labels.end()) { + // TODO: raise exception if this is not true + total_gpu_memory = std::stod(this->labels.at("gpu_memory")); + } + double num_gpus_request = 0; + if (total_gpu_memory > 0) { + // round up to closes kResourceUnitScaling + num_gpus_request = + (resource.Get(ResourceID::GPU_Memory()).Double() / total_gpu_memory) + + 1 / static_cast(2 * kResourceUnitScaling); + } + adjusted_resource.Set(ResourceID::GPU(), num_gpus_request); + adjusted_resource.Set(ResourceID::GPU_Memory(), 0); + } + return adjusted_resource; +} +``` + +Pros: +- Simplified Resource Model: Better emphasizes to new users that Ray has a single GPU resource, simplifying the resource model. +- Straightforward Conversion: `GPU_Memory` is converted to GPU based on the single node's total_gpu_memory during node feasibility checks and resource allocation with the roundup logic applied underhood. + +Cons: +- Limited Observability: `ray.available_resources()` only displays remaining GPU resources in terms of percentage, without specific amounts for `GPU_Memory`. +- Incompatibility with Heterogeneous GPUs: Doesn't work for heterogeneous GPUs in a single node, a limitation existing in Ray's current support. From 0f73faa511214c74c4258c08213049fe6c05d394 Mon Sep 17 00:00:00 2001 From: Jonathan Nitisastro Date: Mon, 20 Nov 2023 10:22:01 -0800 Subject: [PATCH 2/7] add conversion example Signed-off-by: Jonathan Nitisastro --- reps/2023-10-30-gpu-memory-support.md | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/reps/2023-10-30-gpu-memory-support.md b/reps/2023-10-30-gpu-memory-support.md index 8794251..fba4713 100644 --- a/reps/2023-10-30-gpu-memory-support.md +++ b/reps/2023-10-30-gpu-memory-support.md @@ -100,6 +100,16 @@ There are two pivotal options in the major implementation of this project, each In Option 1, we choose to store `GPU_Memory` as part of the NodeResource. This implementation ensures synchronization between GPU and `GPU_Memory`. During node feasibility checks and resource allocation, the `ConvertRelativeResource` function performs two conversions: calculating `gpu_memory` if `num_gpus` is specified and vice versa. ```c++ +// Example: Given a Node with total_single_gpu_memory = 110000 (in mb) with ResourceRequest.ResourceSet +// { +// "gpu_memory": 1010, +// } +// which is then converted to equivalent GPU rounded up and recompute gpu_memory after roundup +// { +// "GPU": 0.0092, (1010/100000 = 0.00918181818 the rounded up) +// "gpu_memory": 1012, (0.0092 x 110000) +// } + const ResourceSet NodeResources::ConvertRelativeResource( const ResourceSet &resource) const { ResourceSet adjusted_resource = resource; @@ -143,6 +153,15 @@ Cons: In Option 2, we opt to store only GPU and convert `gpu_memory` to GPU. This implementation involves saving metadata of the single node's `total_gpu_memory`. `ConvertRelativeResource` converts `gpu_memory` in `ResourceRequest` to `num_gpus` based on the single node's `total_gpu_memory` during node feasibility checks and resource allocation. ```c++ +// Example: Given a Node with total_single_gpu_memory = 110000 (in mb) with ResourceRequest.ResourceSet +// { +// "gpu_memory": 1010 +// } +// which is then converted to equivalent GPU resource +// { +// "GPU": 0.0092 (1010/110000 = 0.00918181818 the rounded up) +// } + const ResourceSet NodeResources::ConvertRelativeResource( const ResourceSet &resource) const { ResourceSet adjusted_resource = resource; From cc21d1338ec50d75b6f64b7043821ac75d87c6ce Mon Sep 17 00:00:00 2001 From: Jonathan Nitisastro Date: Mon, 20 Nov 2023 17:11:12 -0800 Subject: [PATCH 3/7] add usage example Signed-off-by: Jonathan Nitisastro --- .ipynb_checkpoints/Untitled-checkpoint.ipynb | 6 + Untitled.ipynb | 6 + reps/2023-10-30-gpu-memory-support.md | 245 ++++++++++++------- 3 files changed, 162 insertions(+), 95 deletions(-) create mode 100644 .ipynb_checkpoints/Untitled-checkpoint.ipynb create mode 100644 Untitled.ipynb diff --git a/.ipynb_checkpoints/Untitled-checkpoint.ipynb b/.ipynb_checkpoints/Untitled-checkpoint.ipynb new file mode 100644 index 0000000..363fcab --- /dev/null +++ b/.ipynb_checkpoints/Untitled-checkpoint.ipynb @@ -0,0 +1,6 @@ +{ + "cells": [], + "metadata": {}, + "nbformat": 4, + "nbformat_minor": 5 +} diff --git a/Untitled.ipynb b/Untitled.ipynb new file mode 100644 index 0000000..363fcab --- /dev/null +++ b/Untitled.ipynb @@ -0,0 +1,6 @@ +{ + "cells": [], + "metadata": {}, + "nbformat": 4, + "nbformat_minor": 5 +} diff --git a/reps/2023-10-30-gpu-memory-support.md b/reps/2023-10-30-gpu-memory-support.md index fba4713..a742d97 100644 --- a/reps/2023-10-30-gpu-memory-support.md +++ b/reps/2023-10-30-gpu-memory-support.md @@ -9,6 +9,7 @@ Design docummentation for supporting gpu-memory on ray core resource scheduler. Currently, `ray` support `num_gpus` to scheduler resource to tasks/actors, which then assign either `num_gpus` number of gpus or a fraction of a single gpu if `num_gpus < 1`. However, there are often use cases where users want to specify gpu by amount of memory. The current workaround is ray users require to specify the gpu they want to use and convert the gpu memory into fractional value w.r.t the specified gpu. This new scheduler design will allows users to directly scheduler gpu resources by amount of memory. **Ray community users’s demand**: + https://github.com/ray-project/ray/issues/37574 https://github.com/ray-project/ray/issues/26929 @@ -39,9 +40,9 @@ To make the review process more productive, the owner of each proposal should id Users will be able to specify amount of gpu memory to their ray tasks/actors using `gpu_memory` on `ray.remote`. The specified `gpu_memory` will be the amount of gpu resources from a single gpu that will be allocated to users ray tasks/actors. ```python -# Request a fractional GPU with specified gpu_memory in megabytes. +# Request a fractional GPU with specified gpu_memory in bytes. # Mutually exclusive with num_gpus. -@ray.remote(_gpu_memory=1024) +@ray.remote(_gpu_memory=1024 * 1024 * 1024) # 1 mb request def task: … ``` @@ -51,14 +52,14 @@ When a Ray node is started, Ray will auto detect the number of GPUs and GPU memo ```bash ray start # auto detection -ray start –num_gpus=3 –gpu_memory=3000 # manual override, each gpu has 1000mb total memory +ray start –num_gpus=3 –gpu_memory=3000 * 1024 * 1024 * 1024 # manual override, each gpu has 1000mb total memory ``` Note that GPU memory and compute unit is 1-1 conversion, means 20GB of gpu memory is equivalent to 0.5 fractional value of an `A100_40GB` gpu. So, for simplicity and consistency, ray doesn't allow users to specify both `num_gpus` and `_gpu_memory` in a single ray task/actor. ```python # Request a fractional GPU both num_gpus and _gpu_memory is not allowed -@ray.remote(_gpu_memory=1024, num_gpus=0.5) # raise ValueError exception +@ray.remote(_gpu_memory=1024 * 1024 * 1024, num_gpus=0.5) # raise ValueError exception def not_allowed_task: … @@ -68,7 +69,7 @@ def num_gpus_task: … # Request a fractional GPU with specified _gpu_memory. -@ray.remote(_gpu_memory=1024) +@ray.remote(_gpu_memory=1024 * 1024 * 1024) def gpu_memory_task: … ``` @@ -77,69 +78,60 @@ Additionally, users can still specify which gpu type they want to use by specify ```python # Request a fractional of A100 GPU with specified _gpu_memory -@ray.remote(_gpu_memory=1024, accelerator_type="NVIDIA_A100") +@ray.remote(_gpu_memory=1024 * 1024 * 1024 * 1024, accelerator_type="NVIDIA_A100") def nvidia_a100_gpu_task: … # Requesting 30GB of gpu memory from a A10 GPU with 24GB of memory. # Task won't be able to be scheduled. -@ray.remote(_gpu_memory=30 * 1024, accelerator_type="NVIDIA_TESLA_A10G") +@ray.remote(_gpu_memory=30 * 1024 * 1024 * 1024 * 1024, accelerator_type="NVIDIA_TESLA_A10G") def nvidia_a10_gpu_task: … ``` +### Resource API +We introduce a new `ResourceID` named `GPU_Memory` (`gpu_memory` in string) to specify the amount of GPU memory resources. `GPU_Memory` is treated as a GPU resource with a distinct representation, where the relationship is defined as `GPU` equals to `GPU_Memory` divided by the total memory of a single GPU in the node as GPU resources in the node is homogeneous. Despite their distinct representations, both `GPU` and `GPU_Memory` signify the same underlying resource, with the caveat that Ray currently only supports homogeneous GPU type for each node. -### Implementation +There are two pivotal options in how we store `GPU_Memory` in `NodeResources`: + +#### Option 1: Only store GPU Resources +We opt to store only `GPU` in `NodeResource` and convert `gpu_memory` to GPU means `gpu_memory` is `ResourceRequest` only resource. This implementation involves saving metadata of the single node's `total_single_gpu_memory`. `ConvertRelativeResource` converts `gpu_memory` in `ResourceRequest` to `num_gpus` based on the single node's `total_single_gpu_memory` during node feasibility checks and resource allocation. + +```python +# Suppose we have two nodes with GPU type A100 40GB and A100 80gb respectively +NodeResource(available={"GPU": [1,1,1]}, label={"gpu_memory_per_gpu": "40GB"}) +NodeResource(available={"GPU": [1,1,1]}, label={"gpu_memory_per_gpu": "80GB"}) + + +# gpu_memory request +task.options(_gpu_memory="10GB") + +# equivalent resource request when scheduled in Node 1 +ResourceRequest({"GPU": 0.25}) +# remaining resources in Node 1, check using ray.available_resources() +NodeResource(available={"GPU": [0.75,1,1]}) -In the implementation phase of the project, we introduce a new `ResourceID` named `GPU_Memory` to specify the amount of GPU memory resources. `GPU_Memory` is treated as a GPU resource with a distinct representation, where the relationship is defined as `GPU` equals to `GPU_Memory` divided by the total memory of a single GPU in the node as GPU resources in the node is homogeneous. Despite their distinct representations, both `GPU` and `GPU_Memory` signify the same underlying resource, with the caveat that Ray currently supports only a single GPU. - -The primary implementation entails the automatic detection of GPU memory during the initialization of a Ray cluster. Subsequently, when users specify resources using `ray.remote`, particularly the `gpu_memory` resource, Ray will calculate the `num_gpus` from the specified `gpu_memory` in the `ResourceRequest`. This calculation is carried out by a function called `ConvertRelativeResource`, which updates `ResourceSet` from the `ResourceRequest` relative to the resources in the assigned node. Additionally, we incorporate a round-up mechanism during the conversion to `num_gpus`, ensuring precision up to $10^{-4}$. - -There are two pivotal options in the major implementation of this project, each with distinct characteristics: - -#### Option 1: -In Option 1, we choose to store `GPU_Memory` as part of the NodeResource. This implementation ensures synchronization between GPU and `GPU_Memory`. During node feasibility checks and resource allocation, the `ConvertRelativeResource` function performs two conversions: calculating `gpu_memory` if `num_gpus` is specified and vice versa. - -```c++ -// Example: Given a Node with total_single_gpu_memory = 110000 (in mb) with ResourceRequest.ResourceSet -// { -// "gpu_memory": 1010, -// } -// which is then converted to equivalent GPU rounded up and recompute gpu_memory after roundup -// { -// "GPU": 0.0092, (1010/100000 = 0.00918181818 the rounded up) -// "gpu_memory": 1012, (0.0092 x 110000) -// } - -const ResourceSet NodeResources::ConvertRelativeResource( - const ResourceSet &resource) const { - ResourceSet adjusted_resource = resource; - double total_gpu_memory = this->total.Get(ResourceID::GPU_Memory()).Double() / - this->total.Get(ResourceID::GPU()).Double(); // get single GPU total memory - // convert gpu_memory to GPU - if (resource.Has(ResourceID::GPU_Memory())) { - double num_gpus_request = 0; - double gpu_memory_request = resource.Get(ResourceID::GPU()).Double(); - if (total_gpu_memory > 0) { - // round up to closes kResourceUnitScaling - num_gpus_request = - (resource.Get(ResourceID::GPU_Memory()).Double() / total_gpu_memory) + - 1 / static_cast(2 * kResourceUnitScaling); - // update gpu_memory after roundup - gpu_memory_request = num_gpus_request * total_gpu_memory; - } - adjusted_resource.Set(ResourceID::GPU(), num_gpus_request); - adjusted_resource.Set(ResourceID::GPU_Memory(), gpu_memory_request); - } else if (resource.Has(ResourceID::GPU())) { - double gpu_memory_request = 0; - if (total_gpu_memory > 0) { - gpu_memory_request = - (resource.Get(ResourceID::GPU()).Double() * total_gpu_memory) - + 0.5; - } - adjusted_resource.Set(ResourceID::GPU_Memory(), gpu_memory_request); - } - return adjusted_resource; -} +# equivalent resource request when scheduled in Node 2 +ResourceRequest({"GPU": 0.125}) +# remaining resources in Node 2, check using ray.available_resources() +NodeResource(available={"GPU": [0.875,1,1]}) + + +# Roundup gpu_memory request +task.options(_gpu_memory="10MB") + +# equivalent resource request when scheduled in Node 1 +ResourceRequest({"GPU": 0.00025}) +# round up to nearest 10^{-4} +ResourceRequest({"GPU": 0.0003}) +# remaining resources in Node 1, check using ray.available_resources() +NodeResource(available={"GPU": [0.9997,1,1]}) + +# equivalent resource request when scheduled in Node 2 +ResourceRequest({"GPU": 0.000125}) +# round up to nearest 10^{-4} +ResourceRequest({"GPU": 0.0002}) +# remaining resources in Node 2, check using ray.available_resources() +NodeResource(available={"GPU": [0.9998,1,1]}) ``` Pros: @@ -149,47 +141,110 @@ Cons: - Synchronization Overhead: Requires synchronization for both `GPU` and `GPU_Memory`, introducing an additional layer of complexity by updating both `GPU` and `GPU_Memory` for rounding up. - Resource Duality: Users need to grasp that both resources, `GPU` and `GPU_Memory`, essentially denote the same underlying resource. -#### Option 2: -In Option 2, we opt to store only GPU and convert `gpu_memory` to GPU. This implementation involves saving metadata of the single node's `total_gpu_memory`. `ConvertRelativeResource` converts `gpu_memory` in `ResourceRequest` to `num_gpus` based on the single node's `total_gpu_memory` during node feasibility checks and resource allocation. - -```c++ -// Example: Given a Node with total_single_gpu_memory = 110000 (in mb) with ResourceRequest.ResourceSet -// { -// "gpu_memory": 1010 -// } -// which is then converted to equivalent GPU resource -// { -// "GPU": 0.0092 (1010/110000 = 0.00918181818 the rounded up) -// } - -const ResourceSet NodeResources::ConvertRelativeResource( - const ResourceSet &resource) const { - ResourceSet adjusted_resource = resource; - // convert gpu_memory to GPU - if (resource.Has(ResourceID::GPU_Memory())) { - double total_gpu_memory = 0; - if (this->labels.find("gpu_memory") != this->labels.end()) { - // TODO: raise exception if this is not true - total_gpu_memory = std::stod(this->labels.at("gpu_memory")); - } - double num_gpus_request = 0; - if (total_gpu_memory > 0) { - // round up to closes kResourceUnitScaling - num_gpus_request = - (resource.Get(ResourceID::GPU_Memory()).Double() / total_gpu_memory) + - 1 / static_cast(2 * kResourceUnitScaling); - } - adjusted_resource.Set(ResourceID::GPU(), num_gpus_request); - adjusted_resource.Set(ResourceID::GPU_Memory(), 0); - } - return adjusted_resource; -} +#### Option 2: Store both GPU and GPU_Memory Resources +We store `GPU_Memory` as part of the `NodeResource`. This implementation ensures synchronization between GPU and `GPU_Memory`. During node feasibility checks and resource allocation, the `ConvertRelativeResource` function performs two conversions: calculating `gpu_memory` if `num_gpus` is specified and vice versa. + +```python +# Suppose we have two nodes with GPU type A100 40GB and A100 80gb respectively +NodeResource(available={"GPU": [1,1,1], "gpu_memory": ["40GB", "40GB", "40GB"]}) +NodeResource(available={"GPU": [1,1,1], "gpu_memory": ["80GB", "80GB", "80GB"]}) + + +# gpu_memory request +task.options(_gpu_memory="10GB") + +# equivalent resource request when scheduled in Node 1 +ResourceRequest({"GPU": 0.25, "gpu_memory": "10GB"}) +# remaining resources in Node 1, check using ray.available_resources() +NodeResource(available={"GPU": [0.75,1,1], "gpu_memory": ["30GB", "80GB", "80GB"]}) + +# equivalent resource request when scheduled in Node 2 +ResourceRequest({"GPU": 0.125, "gpu_memory": "10GB"}) +# remaining resources in Node 2, check using ray.available_resources() +NodeResource(available={"GPU": [0.875,1,1], "gpu_memory": ["70GB", "80GB", "80GB"]}) + + +# Roundup gpu_memory request +task.options(_gpu_memory="10MB") + +# equivalent resource request when scheduled in Node 1 +ResourceRequest({"GPU": 0.00025, "gpu_memory": "10MB"}) +# round up to nearest 10^{-4} +ResourceRequest({"GPU": 0.0003, "gpu_memory": "12MB"}) +# remaining resources in Node 1, check using ray.available_resources() +NodeResource(available={"GPU": [0.9997,1,1], "gpu_memory": ["39.988GB", "40GB", "40GB"]}) + +# equivalent resource request when scheduled in Node 2 +ResourceRequest({"GPU": 0.000125, "gpu_memory": "10MB"}) +# round up to nearest 10^{-4} +ResourceRequest({"GPU": 0.0002, "gpu_memory": "16MB"}) +# remaining resources in Node 2, check using ray.available_resources() +NodeResource(available={"GPU": [0.9998,1,1], "gpu_memory": ["79.984GB", "80GB", "80GB"]}) ``` Pros: -- Simplified Resource Model: Better emphasizes to new users that Ray has a single GPU resource, simplifying the resource model. -- Straightforward Conversion: `GPU_Memory` is converted to GPU based on the single node's total_gpu_memory during node feasibility checks and resource allocation with the roundup logic applied underhood. +- Simplified Resource Model: Better emphasizes to new users that Ray only have `GPU` to represent GPU resource, simplifying the resource model. +- Straightforward Conversion: `GPU_Memory` is converted to GPU based on the single node's total_single_gpu_memory during node feasibility checks and resource allocation with the roundup logic applied underhood. Cons: - Limited Observability: `ray.available_resources()` only displays remaining GPU resources in terms of percentage, without specific amounts for `GPU_Memory`. - Incompatibility with Heterogeneous GPUs: Doesn't work for heterogeneous GPUs in a single node, a limitation existing in Ray's current support. + +### Implementation +The primary implementation entails the automatic detection of GPU memory during the initialization of a Ray cluster. + +```python +class AcceleratorManager: + # return 0 if accelerator is not GPU, + # else return total GPU memory of a single GPU + def get_current_node_gpu_memory(self): + ... +``` + +Subsequently, we added another change within the scheduler to convert `gpu_memory` to `num_gpus` depending on which node the request got assigned to check if resource request is feasible in the node and allocate the resource. + +```pseudocode +for node in node_list: + +++ def convert_relative_resources(resource_request, node): # option 1 implementation +++ if gpu_memory in resource_request: +++ resource_request.num_gpus = roundup(gpu_memory / node.label["gpu_memory_per_gpu"] , 0.0001) +++ resource_request.gpu_memory = 0 +++ return resource_request +++ convert_relative_resources(resource_request, node) + + if check_is_feasible(resource_request): + allocation = TryAllocate(resource_request) +``` + +#### Autoscaler +For the first option, we require to node label to store `gpu_memory_per_gpu`. However, currently label is not included in autoscaler as label-based node scheduling is yet to be supported for autoscaler. Therefore, for current solution, we introduce a new private resources `node:gpu_memory_per_gpu__` as a constant label value representing `gpu_memory_per_gpu` node label. Next, we also add `convert_relative_resources` function before in `_fits` and `_inplace_subtract` in `resource_demand_scheduler.py` + +```python +def _convert_relative_resources(node, resources): + adjusted_resources = resources.copy() + if "gpu_memory" in resources: + if "node:gpu_memory_per_gpu" not in node or node["node:gpu_memory_per_gpu"] == 0: + return None + adjusted_resources["GPU"] = ( + resources["gpu_memory"] / node["node:gpu_memory_per_gpu"] + ) + del adjusted_resources["gpu_memory"] + return adjusted_resources + +def _fits(node: ResourceDict, resources: ResourceDict) -> bool: + adjusted_resources = _convert_relative_resources(node, resources) + if adjusted_resources is None: + return False + for k, v in adjusted_resources.items(): + ... + +def _inplace_subtract(node: ResourceDict, resources: ResourceDict) -> None: + adjusted_resources = _convert_relative_resources(node, resources) + if adjusted_resources is None: + return + for k, v in adjusted_resources.items(): + ... +``` + +For the second option, there's no required addition of `node:gpu_memory_per_gpu_` since `GPU_Memory` is part of resources, but the `_convert_relative_resources` still required. From 35839fdc6c9a1d3863adeadd072053900f803dc5 Mon Sep 17 00:00:00 2001 From: Jonathan Nitisastro Date: Mon, 27 Nov 2023 12:15:30 -0800 Subject: [PATCH 4/7] clean rep Signed-off-by: Jonathan Nitisastro --- .ipynb_checkpoints/Untitled-checkpoint.ipynb | 6 ---- Untitled.ipynb | 6 ---- reps/2023-10-30-gpu-memory-support.md | 32 +++++++++++++++----- 3 files changed, 24 insertions(+), 20 deletions(-) delete mode 100644 .ipynb_checkpoints/Untitled-checkpoint.ipynb delete mode 100644 Untitled.ipynb diff --git a/.ipynb_checkpoints/Untitled-checkpoint.ipynb b/.ipynb_checkpoints/Untitled-checkpoint.ipynb deleted file mode 100644 index 363fcab..0000000 --- a/.ipynb_checkpoints/Untitled-checkpoint.ipynb +++ /dev/null @@ -1,6 +0,0 @@ -{ - "cells": [], - "metadata": {}, - "nbformat": 4, - "nbformat_minor": 5 -} diff --git a/Untitled.ipynb b/Untitled.ipynb deleted file mode 100644 index 363fcab..0000000 --- a/Untitled.ipynb +++ /dev/null @@ -1,6 +0,0 @@ -{ - "cells": [], - "metadata": {}, - "nbformat": 4, - "nbformat_minor": 5 -} diff --git a/reps/2023-10-30-gpu-memory-support.md b/reps/2023-10-30-gpu-memory-support.md index a742d97..6afb041 100644 --- a/reps/2023-10-30-gpu-memory-support.md +++ b/reps/2023-10-30-gpu-memory-support.md @@ -6,7 +6,11 @@ Design docummentation for supporting gpu-memory on ray core resource scheduler. ### General Motivation -Currently, `ray` support `num_gpus` to scheduler resource to tasks/actors, which then assign either `num_gpus` number of gpus or a fraction of a single gpu if `num_gpus < 1`. However, there are often use cases where users want to specify gpu by amount of memory. The current workaround is ray users require to specify the gpu they want to use and convert the gpu memory into fractional value w.r.t the specified gpu. This new scheduler design will allows users to directly scheduler gpu resources by amount of memory. +Currently, `ray` supports `num_gpus` to scheduler resource to tasks/actors, which then assign either `num_gpus` number of gpu ids to be used by the tasks/actors. Additionally, `ray` provides support fractional gpus allocation by specifying `num_gpus < 1` so a single GPU can be used to run multiple tasks. This works well if the cluster only has a single type of GPUs. However, imagining a cluster has both A100 40GB and A100 80GB GPUs, setting num_gpus to a fixed number doesn’t work that well: if we set to 0.1 then we will get 4GB if the scheduler picks A100 40GB but 8GB if the scheduler picks A100 80GB which is a waste of resource. We can also set accelerator_type to A100_40GB and num_gpus to 0.1 to make sure we get the exact amount of GPU memory we need but then the task cannot run on A100 80GB even if it’s free. + +This new scheduler design will allows users to directly schedule fractional gpu resources by amount of memory. In our example, if user specify `_gpu_memory = 20GB`, then `ray` automatically convert the value to `num_gpus` depending on which nodes the request is assigned too. As example, if it's scheduled on A100 40GB node, then `num_gpus = 0.5`, otherwise if it's scheduled on A100 80GB node, then `num_gpus = 0.25`. As a result, user can schedule a fixed amount of GPU resources without depending on which types of GPUs the tasks/actos are scheduled to. + +... issue with num gpus **Ray community users’s demand**: @@ -18,6 +22,10 @@ https://discuss.ray.io/t/how-to-specify-gpu-resources-in-terms-of-gpu-ram-and-no https://discuss.ray.io/t/gpu-memory-aware-scheduling/2922/5 +https://discuss.ray.io/t/automatic-calculation-of-a-value-for-the-num-gpu-param/7844/4 + +https://discuss.ray.io/t/ray-train-ray-tune-ray-clusters-handling-different-gpus-with-different-gpu-memory-sizes-in-a-ray-cluster/9220 + ### Should this change be within `ray` or outside? Inside `ray` project since we want to add new parameter `_gpu_memory` to the `ray` remote function. @@ -88,6 +96,10 @@ def nvidia_a100_gpu_task: def nvidia_a10_gpu_task: … ``` + +#### Placement Group +TBD + ### Resource API We introduce a new `ResourceID` named `GPU_Memory` (`gpu_memory` in string) to specify the amount of GPU memory resources. `GPU_Memory` is treated as a GPU resource with a distinct representation, where the relationship is defined as `GPU` equals to `GPU_Memory` divided by the total memory of a single GPU in the node as GPU resources in the node is homogeneous. Despite their distinct representations, both `GPU` and `GPU_Memory` signify the same underlying resource, with the caveat that Ray currently only supports homogeneous GPU type for each node. @@ -135,11 +147,12 @@ NodeResource(available={"GPU": [0.9998,1,1]}) ``` Pros: -- Enhanced Observability: Users can see the remaining GPU memory resources after roundup allocation, providing detailed insights. +- Simplified Resource Model: Better emphasizes to new users that Ray only have `GPU` to represent GPU resource, simplifying the resource model. +- Straightforward Conversion: `GPU_Memory` is converted to GPU based on the single node's total_single_gpu_memory during node feasibility checks and resource allocation with the roundup logic applied underhood. Cons: -- Synchronization Overhead: Requires synchronization for both `GPU` and `GPU_Memory`, introducing an additional layer of complexity by updating both `GPU` and `GPU_Memory` for rounding up. -- Resource Duality: Users need to grasp that both resources, `GPU` and `GPU_Memory`, essentially denote the same underlying resource. +- Limited Observability: `ray.available_resources()` only displays remaining GPU resources in terms of percentage, without specific amounts for `GPU_Memory`. +- Incompatibility with Heterogeneous GPUs: Doesn't work for heterogeneous GPUs in a single node, a limitation existing in Ray's current support. #### Option 2: Store both GPU and GPU_Memory Resources We store `GPU_Memory` as part of the `NodeResource`. This implementation ensures synchronization between GPU and `GPU_Memory`. During node feasibility checks and resource allocation, the `ConvertRelativeResource` function performs two conversions: calculating `gpu_memory` if `num_gpus` is specified and vice versa. @@ -183,12 +196,12 @@ NodeResource(available={"GPU": [0.9998,1,1], "gpu_memory": ["79.984GB", "80GB", ``` Pros: -- Simplified Resource Model: Better emphasizes to new users that Ray only have `GPU` to represent GPU resource, simplifying the resource model. -- Straightforward Conversion: `GPU_Memory` is converted to GPU based on the single node's total_single_gpu_memory during node feasibility checks and resource allocation with the roundup logic applied underhood. +- Enhanced Observability: Users can see the remaining GPU memory resources after roundup allocation, providing detailed insights. Cons: -- Limited Observability: `ray.available_resources()` only displays remaining GPU resources in terms of percentage, without specific amounts for `GPU_Memory`. -- Incompatibility with Heterogeneous GPUs: Doesn't work for heterogeneous GPUs in a single node, a limitation existing in Ray's current support. +- Synchronization Overhead: Requires synchronization for both `GPU` and `GPU_Memory`, introducing an additional layer of complexity by updating both `GPU` and `GPU_Memory` for rounding up. +- Resource Duality: Users need to grasp that both resources, `GPU` and `GPU_Memory`, essentially denote the same underlying resource. + ### Implementation The primary implementation entails the automatic detection of GPU memory during the initialization of a Ray cluster. @@ -248,3 +261,6 @@ def _inplace_subtract(node: ResourceDict, resources: ResourceDict) -> None: ``` For the second option, there's no required addition of `node:gpu_memory_per_gpu_` since `GPU_Memory` is part of resources, but the `_convert_relative_resources` still required. + +#### Placement Group +TBD \ No newline at end of file From 38ee994dbe99967ea276351ac958e95b9ae2e212 Mon Sep 17 00:00:00 2001 From: Jonathan Nitisastro Date: Tue, 28 Nov 2023 17:34:44 -0800 Subject: [PATCH 5/7] more fix Signed-off-by: Jonathan Nitisastro --- reps/2023-10-30-gpu-memory-support.md | 112 ++++++++++++++------------ 1 file changed, 60 insertions(+), 52 deletions(-) diff --git a/reps/2023-10-30-gpu-memory-support.md b/reps/2023-10-30-gpu-memory-support.md index 6afb041..3431b1e 100644 --- a/reps/2023-10-30-gpu-memory-support.md +++ b/reps/2023-10-30-gpu-memory-support.md @@ -2,13 +2,25 @@ ## Summary -Design docummentation for supporting gpu-memory on ray core resource scheduler. +Enhance Ray fractional gpu support with gpu memory based scheduling. ### General Motivation -Currently, `ray` supports `num_gpus` to scheduler resource to tasks/actors, which then assign either `num_gpus` number of gpu ids to be used by the tasks/actors. Additionally, `ray` provides support fractional gpus allocation by specifying `num_gpus < 1` so a single GPU can be used to run multiple tasks. This works well if the cluster only has a single type of GPUs. However, imagining a cluster has both A100 40GB and A100 80GB GPUs, setting num_gpus to a fixed number doesn’t work that well: if we set to 0.1 then we will get 4GB if the scheduler picks A100 40GB but 8GB if the scheduler picks A100 80GB which is a waste of resource. We can also set accelerator_type to A100_40GB and num_gpus to 0.1 to make sure we get the exact amount of GPU memory we need but then the task cannot run on A100 80GB even if it’s free. +Currently, `ray` supports `num_gpus` to scheduler resource to tasks/actors, which then assign either `num_gpus` number of gpu ids to be used by the tasks/actors. Additionally, `ray` provides support fractional gpus allocation by specifying `num_gpus < 1` so a single GPU can be used to run multiple tasks. This works well if the cluster only has a single type of GPUs. However, imagining a cluster has both A100 40GB and A100 80GB GPUs, setting num_gpus to a fixed number doesn’t work that well: if we set to 0.1 then we will get 4GB if the scheduler picks A100 40GB but 8GB if the scheduler picks A100 80GB which is a waste of resource if the task only needs 4GB. We can also set accelerator_type to A100_40GB and num_gpus to 0.1 to make sure we get the exact amount of GPU memory we need but then the task cannot run on A100 80GB even if it’s free. Many users also have encountered this issue: -This new scheduler design will allows users to directly schedule fractional gpu resources by amount of memory. In our example, if user specify `_gpu_memory = 20GB`, then `ray` automatically convert the value to `num_gpus` depending on which nodes the request is assigned too. As example, if it's scheduled on A100 40GB node, then `num_gpus = 0.5`, otherwise if it's scheduled on A100 80GB node, then `num_gpus = 0.25`. As a result, user can schedule a fixed amount of GPU resources without depending on which types of GPUs the tasks/actos are scheduled to. +- https://github.com/ray-project/ray/issues/37574 + +- https://github.com/ray-project/ray/issues/26929 + +- https://discuss.ray.io/t/how-to-specify-gpu-resources-in-terms-of-gpu-ram-and-not-fraction-of-gpu/4128 + +- https://discuss.ray.io/t/gpu-memory-aware-scheduling/2922/5 + +- https://discuss.ray.io/t/automatic-calculation-of-a-value-for-the-num-gpu-param/7844/4 + +- https://discuss.ray.io/t/ray-train-ray-tune-ray-clusters-handling-different-gpus-with-different-gpu-memory-sizes-in-a-ray-cluster/9220 + +This REP allows users to directly schedule fractional gpu resources by amount of memory. In our example, if user specify `gpu_memory = 20GB`, then `ray` automatically convert the value to `num_gpus` depending on which nodes the request is assigned too. As example, if it's scheduled on A100 40GB node, then `num_gpus = 0.5`, otherwise if it's scheduled on A100 80GB node, then `num_gpus = 0.25`. As a result, user can schedule a fixed amount of GPU resources without depending on which types of GPUs the tasks/actos are scheduled to. ... issue with num gpus @@ -28,7 +40,7 @@ https://discuss.ray.io/t/ray-train-ray-tune-ray-clusters-handling-different-gpus ### Should this change be within `ray` or outside? -Inside `ray` project since we want to add new parameter `_gpu_memory` to the `ray` remote function. +Inside `ray` project since we want to add new parameter `gpu_memory` to the `ray` remote function. ## Stewardship @@ -50,12 +62,12 @@ Users will be able to specify amount of gpu memory to their ray tasks/actors usi ```python # Request a fractional GPU with specified gpu_memory in bytes. # Mutually exclusive with num_gpus. -@ray.remote(_gpu_memory=1024 * 1024 * 1024) # 1 mb request +@ray.remote(gpu_memory=1024 * 1024 * 1024) # 1 mb request def task: … ``` -When a Ray node is started, Ray will auto detect the number of GPUs and GPU memory of each GPU and set the GPU and _gpu_memory resources accordingly. Users also have the option to manually specify `_gpu_memory` resources as the sum of total gpu memory across all gpus in the node. The default value is `num_gpus` * total memory of the gpu type in the node. +When a Ray node is started, Ray will auto detect the number of GPUs and GPU memory of each GPU and set the GPU and gpu_memory resources accordingly. Users also have the option to manually specify `gpu_memory` resources as the sum of total gpu memory across all gpus in the node. The default value is `num_gpus` * total memory of the gpu type in the node. ```bash ray start # auto detection @@ -63,11 +75,11 @@ ray start # auto detection ray start –num_gpus=3 –gpu_memory=3000 * 1024 * 1024 * 1024 # manual override, each gpu has 1000mb total memory ``` -Note that GPU memory and compute unit is 1-1 conversion, means 20GB of gpu memory is equivalent to 0.5 fractional value of an `A100_40GB` gpu. So, for simplicity and consistency, ray doesn't allow users to specify both `num_gpus` and `_gpu_memory` in a single ray task/actor. +Note that GPU memory and compute unit is 1-1 conversion, means 20GB of gpu memory is equivalent to 0.5 fractional value of an `A100_40GB` gpu. So, for simplicity and consistency, ray doesn't allow users to specify both `num_gpus` and `gpu_memory` in a single ray task/actor. ```python -# Request a fractional GPU both num_gpus and _gpu_memory is not allowed -@ray.remote(_gpu_memory=1024 * 1024 * 1024, num_gpus=0.5) # raise ValueError exception +# Request a fractional GPU both num_gpus and gpu_memory is not allowed +@ray.remote(gpu_memory=1024 * 1024 * 1024, num_gpus=0.5) # raise ValueError exception def not_allowed_task: … @@ -76,8 +88,8 @@ def not_allowed_task: def num_gpus_task: … -# Request a fractional GPU with specified _gpu_memory. -@ray.remote(_gpu_memory=1024 * 1024 * 1024) +# Request a fractional GPU with specified gpu_memory. +@ray.remote(gpu_memory=1024 * 1024 * 1024) def gpu_memory_task: … ``` @@ -85,14 +97,14 @@ def gpu_memory_task: Additionally, users can still specify which gpu type they want to use by specifying `accelerator_type`. ```python -# Request a fractional of A100 GPU with specified _gpu_memory -@ray.remote(_gpu_memory=1024 * 1024 * 1024 * 1024, accelerator_type="NVIDIA_A100") +# Request a fractional of A100 GPU with specified gpu_memory +@ray.remote(gpu_memory=1024 * 1024 * 1024 * 1024, accelerator_type="NVIDIA_A100") def nvidia_a100_gpu_task: … # Requesting 30GB of gpu memory from a A10 GPU with 24GB of memory. # Task won't be able to be scheduled. -@ray.remote(_gpu_memory=30 * 1024 * 1024 * 1024 * 1024, accelerator_type="NVIDIA_TESLA_A10G") +@ray.remote(gpu_memory=30 * 1024 * 1024 * 1024 * 1024, accelerator_type="NVIDIA_TESLA_A10G") def nvidia_a10_gpu_task: … ``` @@ -100,12 +112,36 @@ def nvidia_a10_gpu_task: #### Placement Group TBD -### Resource API -We introduce a new `ResourceID` named `GPU_Memory` (`gpu_memory` in string) to specify the amount of GPU memory resources. `GPU_Memory` is treated as a GPU resource with a distinct representation, where the relationship is defined as `GPU` equals to `GPU_Memory` divided by the total memory of a single GPU in the node as GPU resources in the node is homogeneous. Despite their distinct representations, both `GPU` and `GPU_Memory` signify the same underlying resource, with the caveat that Ray currently only supports homogeneous GPU type for each node. +### Implementation +The primary implementation entails the automatic detection of GPU memory during the initialization of a Ray cluster. + +```python +class AcceleratorManager: + # return 0 if accelerator is not GPU, + # else return total GPU memory of a single GPU + def get_current_node_gpu_memory(self): + ... +``` + +Subsequently, we added another change within the scheduler to convert `gpu_memory` to `num_gpus` depending on which node the request got assigned to check if resource request is feasible in the node and allocate the resource. + +```pseudocode +for node in node_list: + +++ def convert_relative_resources(resource_request, node): # option 1 implementation +++ if gpu_memory in resource_request: +++ resource_request.num_gpus = roundup(gpu_memory / node.label["gpu_memory_per_gpu"] , 0.0001) +++ resource_request.gpu_memory = 0 +++ return resource_request +++ convert_relative_resources(resource_request, node) + + if check_is_feasible(resource_request): + allocation = TryAllocate(resource_request) +``` -There are two pivotal options in how we store `GPU_Memory` in `NodeResources`: +#### Resources API +We introduce a new `ResourceID` named `GPU_Memory` (`gpu_memory` in string) to specify the amount of GPU memory resources. `GPU_Memory` is treated as a GPU resource with a distinct representation, where the relationship is defined as `GPU` equals to `GPU_Memory` divided by the total memory of a single GPU in the node as GPU resources in the node is homogeneous. Despite their distinct representations, both `GPU` and `GPU_Memory` signify the same underlying resource, with the caveat that Ray currently only supports homogeneous GPU type for each node. -#### Option 1: Only store GPU Resources We opt to store only `GPU` in `NodeResource` and convert `gpu_memory` to GPU means `gpu_memory` is `ResourceRequest` only resource. This implementation involves saving metadata of the single node's `total_single_gpu_memory`. `ConvertRelativeResource` converts `gpu_memory` in `ResourceRequest` to `num_gpus` based on the single node's `total_single_gpu_memory` during node feasibility checks and resource allocation. ```python @@ -115,7 +151,7 @@ NodeResource(available={"GPU": [1,1,1]}, label={"gpu_memory_per_gpu": "80GB"}) # gpu_memory request -task.options(_gpu_memory="10GB") +task.options(gpu_memory="10GB") # equivalent resource request when scheduled in Node 1 ResourceRequest({"GPU": 0.25}) @@ -129,7 +165,7 @@ NodeResource(available={"GPU": [0.875,1,1]}) # Roundup gpu_memory request -task.options(_gpu_memory="10MB") +task.options(gpu_memory="10MB") # equivalent resource request when scheduled in Node 1 ResourceRequest({"GPU": 0.00025}) @@ -154,8 +190,8 @@ Cons: - Limited Observability: `ray.available_resources()` only displays remaining GPU resources in terms of percentage, without specific amounts for `GPU_Memory`. - Incompatibility with Heterogeneous GPUs: Doesn't work for heterogeneous GPUs in a single node, a limitation existing in Ray's current support. -#### Option 2: Store both GPU and GPU_Memory Resources -We store `GPU_Memory` as part of the `NodeResource`. This implementation ensures synchronization between GPU and `GPU_Memory`. During node feasibility checks and resource allocation, the `ConvertRelativeResource` function performs two conversions: calculating `gpu_memory` if `num_gpus` is specified and vice versa. +##### Alternatives NodeResources API +For the alternative resource implementtion, we store `GPU_Memory` as part of the `NodeResource`. This implementation ensures synchronization between GPU and `GPU_Memory`. During node feasibility checks and resource allocation, the `ConvertRelativeResource` function performs two conversions: calculating `gpu_memory` if `num_gpus` is specified and vice versa. ```python # Suppose we have two nodes with GPU type A100 40GB and A100 80gb respectively @@ -164,7 +200,7 @@ NodeResource(available={"GPU": [1,1,1], "gpu_memory": ["80GB", "80GB", "80GB"]}) # gpu_memory request -task.options(_gpu_memory="10GB") +task.options(gpu_memory="10GB") # equivalent resource request when scheduled in Node 1 ResourceRequest({"GPU": 0.25, "gpu_memory": "10GB"}) @@ -178,7 +214,7 @@ NodeResource(available={"GPU": [0.875,1,1], "gpu_memory": ["70GB", "80GB", "80GB # Roundup gpu_memory request -task.options(_gpu_memory="10MB") +task.options(gpu_memory="10MB") # equivalent resource request when scheduled in Node 1 ResourceRequest({"GPU": 0.00025, "gpu_memory": "10MB"}) @@ -202,34 +238,6 @@ Cons: - Synchronization Overhead: Requires synchronization for both `GPU` and `GPU_Memory`, introducing an additional layer of complexity by updating both `GPU` and `GPU_Memory` for rounding up. - Resource Duality: Users need to grasp that both resources, `GPU` and `GPU_Memory`, essentially denote the same underlying resource. - -### Implementation -The primary implementation entails the automatic detection of GPU memory during the initialization of a Ray cluster. - -```python -class AcceleratorManager: - # return 0 if accelerator is not GPU, - # else return total GPU memory of a single GPU - def get_current_node_gpu_memory(self): - ... -``` - -Subsequently, we added another change within the scheduler to convert `gpu_memory` to `num_gpus` depending on which node the request got assigned to check if resource request is feasible in the node and allocate the resource. - -```pseudocode -for node in node_list: - -++ def convert_relative_resources(resource_request, node): # option 1 implementation -++ if gpu_memory in resource_request: -++ resource_request.num_gpus = roundup(gpu_memory / node.label["gpu_memory_per_gpu"] , 0.0001) -++ resource_request.gpu_memory = 0 -++ return resource_request -++ convert_relative_resources(resource_request, node) - - if check_is_feasible(resource_request): - allocation = TryAllocate(resource_request) -``` - #### Autoscaler For the first option, we require to node label to store `gpu_memory_per_gpu`. However, currently label is not included in autoscaler as label-based node scheduling is yet to be supported for autoscaler. Therefore, for current solution, we introduce a new private resources `node:gpu_memory_per_gpu__` as a constant label value representing `gpu_memory_per_gpu` node label. Next, we also add `convert_relative_resources` function before in `_fits` and `_inplace_subtract` in `resource_demand_scheduler.py` From af0926693c02e56dfdf63c2349c25e75ede56068 Mon Sep 17 00:00:00 2001 From: Jonathan Nitisastro Date: Sat, 2 Dec 2023 20:20:03 -0800 Subject: [PATCH 6/7] placement group example Signed-off-by: Jonathan Nitisastro --- reps/2023-10-30-gpu-memory-support.md | 72 +++++++++++++-------------- 1 file changed, 34 insertions(+), 38 deletions(-) diff --git a/reps/2023-10-30-gpu-memory-support.md b/reps/2023-10-30-gpu-memory-support.md index 3431b1e..8943e09 100644 --- a/reps/2023-10-30-gpu-memory-support.md +++ b/reps/2023-10-30-gpu-memory-support.md @@ -6,7 +6,7 @@ Enhance Ray fractional gpu support with gpu memory based scheduling. ### General Motivation -Currently, `ray` supports `num_gpus` to scheduler resource to tasks/actors, which then assign either `num_gpus` number of gpu ids to be used by the tasks/actors. Additionally, `ray` provides support fractional gpus allocation by specifying `num_gpus < 1` so a single GPU can be used to run multiple tasks. This works well if the cluster only has a single type of GPUs. However, imagining a cluster has both A100 40GB and A100 80GB GPUs, setting num_gpus to a fixed number doesn’t work that well: if we set to 0.1 then we will get 4GB if the scheduler picks A100 40GB but 8GB if the scheduler picks A100 80GB which is a waste of resource if the task only needs 4GB. We can also set accelerator_type to A100_40GB and num_gpus to 0.1 to make sure we get the exact amount of GPU memory we need but then the task cannot run on A100 80GB even if it’s free. Many users also have encountered this issue: +Currently, `ray` supports `num_gpus` to scheduler resource to tasks/actors, which then assign `num_gpus` number of gpu ids to be used by the tasks/actors. Additionally, `ray` provides fractional gpus allocation by specifying `num_gpus < 1` so a single GPU can be used to run multiple tasks. This works well if the cluster only has a single type of GPUs. However, imagining a cluster has both A100 40GB and A100 80GB GPUs, setting num_gpus to a fixed number doesn’t work that well: if we set to 0.1 then we will get 4GB if the scheduler picks A100 40GB but 8GB if the scheduler picks A100 80GB which is a waste of resource if the task only needs 4GB. We can also set accelerator_type to A100_40GB and num_gpus to 0.1 to make sure we get the exact amount of GPU memory we need but then the task cannot run on A100 80GB even if it’s free. Many users also have encountered this issue: - https://github.com/ray-project/ray/issues/37574 @@ -20,23 +20,7 @@ Currently, `ray` supports `num_gpus` to scheduler resource to tasks/actors, whic - https://discuss.ray.io/t/ray-train-ray-tune-ray-clusters-handling-different-gpus-with-different-gpu-memory-sizes-in-a-ray-cluster/9220 -This REP allows users to directly schedule fractional gpu resources by amount of memory. In our example, if user specify `gpu_memory = 20GB`, then `ray` automatically convert the value to `num_gpus` depending on which nodes the request is assigned too. As example, if it's scheduled on A100 40GB node, then `num_gpus = 0.5`, otherwise if it's scheduled on A100 80GB node, then `num_gpus = 0.25`. As a result, user can schedule a fixed amount of GPU resources without depending on which types of GPUs the tasks/actos are scheduled to. - -... issue with num gpus - -**Ray community users’s demand**: - -https://github.com/ray-project/ray/issues/37574 - -https://github.com/ray-project/ray/issues/26929 - -https://discuss.ray.io/t/how-to-specify-gpu-resources-in-terms-of-gpu-ram-and-not-fraction-of-gpu/4128 - -https://discuss.ray.io/t/gpu-memory-aware-scheduling/2922/5 - -https://discuss.ray.io/t/automatic-calculation-of-a-value-for-the-num-gpu-param/7844/4 - -https://discuss.ray.io/t/ray-train-ray-tune-ray-clusters-handling-different-gpus-with-different-gpu-memory-sizes-in-a-ray-cluster/9220 +This REP allows users to directly schedule fractional gpu resources by amount of memory. In our example, if user specify `gpu_memory = 20GB`, then `ray` automatically converts the value to `num_gpus` depending on which nodes the request is assigned to. As example, if it's scheduled on A100 40GB node, then `num_gpus = 0.5`, otherwise if it's scheduled on A100 80GB node, then `num_gpus = 0.25`. As a result, user can schedule a fixed amount of GPU resources without depending on which types of GPUs the tasks/actos are scheduled to. ### Should this change be within `ray` or outside? @@ -76,6 +60,8 @@ ray start –num_gpus=3 –gpu_memory=3000 * 1024 * 1024 * 1024 # manual overrid ``` Note that GPU memory and compute unit is 1-1 conversion, means 20GB of gpu memory is equivalent to 0.5 fractional value of an `A100_40GB` gpu. So, for simplicity and consistency, ray doesn't allow users to specify both `num_gpus` and `gpu_memory` in a single ray task/actor. +Note that GPU memory and GPU is 1-1 conversion, means 20GB of gpu memory is equivalent to 0.5 fractional value of an `A100_40GB` gpu. So, for simplicity and consistency, ray doesn't allow users to specify both `num_gpus` and `gpu_memory` in a single ray task/actor. + ```python # Request a fractional GPU both num_gpus and gpu_memory is not allowed @@ -110,7 +96,17 @@ def nvidia_a10_gpu_task: ``` #### Placement Group -TBD +User also able to request `gpu_memory` as a placement group as follows: + +```python +pg = placement_group([{"gpu_memory": 1024 * 1024, "CPU": 1}, {"GPU": 1}]) + +# Wait until placement group is created. +ray.get(pg.ready(), timeout=10) + +# You can also use ray.wait. +ready, unready = ray.wait([pg.ready()], timeout=10) +``` ### Implementation The primary implementation entails the automatic detection of GPU memory during the initialization of a Ray cluster. @@ -121,26 +117,29 @@ class AcceleratorManager: # else return total GPU memory of a single GPU def get_current_node_gpu_memory(self): ... -``` +``` +The detected memory is added as a node label. -Subsequently, we added another change within the scheduler to convert `gpu_memory` to `num_gpus` depending on which node the request got assigned to check if resource request is feasible in the node and allocate the resource. +During scheduling, the resource request that contains `gpu_memory` will be converted to the corresponding `num_gpus` resource request depending on which node the scheduler is considering. ```pseudocode -for node in node_list: - -++ def convert_relative_resources(resource_request, node): # option 1 implementation -++ if gpu_memory in resource_request: -++ resource_request.num_gpus = roundup(gpu_memory / node.label["gpu_memory_per_gpu"] , 0.0001) -++ resource_request.gpu_memory = 0 -++ return resource_request -++ convert_relative_resources(resource_request, node) - - if check_is_feasible(resource_request): - allocation = TryAllocate(resource_request) +for node in nodes: + def convert_relative_resources(resource_request, node): + if gpu_memory in resource_request: + resource_request.num_gpus = roundup(gpu_memory / node.label["gpu_memory_per_gpu"] , 0.0001) + resource_request.gpu_memory = 0 + return resource_request + + convert_relative_resources(resource_request, node) + + # After converting from gpu_memory to num_gpus, the remaining is the same + if check_is_available(resource_request, node): + allocation = allocate(resource_request, node) + break ``` #### Resources API -We introduce a new `ResourceID` named `GPU_Memory` (`gpu_memory` in string) to specify the amount of GPU memory resources. `GPU_Memory` is treated as a GPU resource with a distinct representation, where the relationship is defined as `GPU` equals to `GPU_Memory` divided by the total memory of a single GPU in the node as GPU resources in the node is homogeneous. Despite their distinct representations, both `GPU` and `GPU_Memory` signify the same underlying resource, with the caveat that Ray currently only supports homogeneous GPU type for each node. +We introduce a new `ResourceID` named `GPU_Memory` (`gpu_memory` in string) to specify the amount of GPU memory resources. `GPU_Memory` is treated as a GPU resource with a distinct representation, where the relationship is defined as `GPU` equals to `GPU_Memory` divided by the total memory of a single GPU in the node as GPU resources in the node is homogeneous. Despite their distinct representations, both `GPU` and `GPU_Memory` signify the same underlying node resource, with the caveat that Ray currently only supports homogeneous GPU type for each node. We opt to store only `GPU` in `NodeResource` and convert `gpu_memory` to GPU means `gpu_memory` is `ResourceRequest` only resource. This implementation involves saving metadata of the single node's `total_single_gpu_memory`. `ConvertRelativeResource` converts `gpu_memory` in `ResourceRequest` to `num_gpus` based on the single node's `total_single_gpu_memory` during node feasibility checks and resource allocation. @@ -190,8 +189,8 @@ Cons: - Limited Observability: `ray.available_resources()` only displays remaining GPU resources in terms of percentage, without specific amounts for `GPU_Memory`. - Incompatibility with Heterogeneous GPUs: Doesn't work for heterogeneous GPUs in a single node, a limitation existing in Ray's current support. -##### Alternatives NodeResources API -For the alternative resource implementtion, we store `GPU_Memory` as part of the `NodeResource`. This implementation ensures synchronization between GPU and `GPU_Memory`. During node feasibility checks and resource allocation, the `ConvertRelativeResource` function performs two conversions: calculating `gpu_memory` if `num_gpus` is specified and vice versa. +##### Alternative +For the alternative resource implementtion, we store `GPU_Memory` as part of the `NodeResource`. This implementation ensures synchronization between GPU and `GPU_Memory` Node resources. During node availability checks and resource allocation, the `ConvertRelativeResource` function performs two conversions: calculating `gpu_memory` if `num_gpus` is specified and vice versa. ```python # Suppose we have two nodes with GPU type A100 40GB and A100 80gb respectively @@ -269,6 +268,3 @@ def _inplace_subtract(node: ResourceDict, resources: ResourceDict) -> None: ``` For the second option, there's no required addition of `node:gpu_memory_per_gpu_` since `GPU_Memory` is part of resources, but the `_convert_relative_resources` still required. - -#### Placement Group -TBD \ No newline at end of file From 20e553b420723cb00703c03880188eb5566f6d53 Mon Sep 17 00:00:00 2001 From: Jiajun Yao Date: Wed, 6 Dec 2023 11:53:55 -0800 Subject: [PATCH 7/7] final update Signed-off-by: Jiajun Yao --- reps/2023-10-30-gpu-memory-support.md | 122 +++++++------------------- 1 file changed, 30 insertions(+), 92 deletions(-) diff --git a/reps/2023-10-30-gpu-memory-support.md b/reps/2023-10-30-gpu-memory-support.md index 8943e09..4b160c7 100644 --- a/reps/2023-10-30-gpu-memory-support.md +++ b/reps/2023-10-30-gpu-memory-support.md @@ -2,11 +2,11 @@ ## Summary -Enhance Ray fractional gpu support with gpu memory based scheduling. +Enhance Ray fractional GPU support with GPU memory based scheduling. ### General Motivation -Currently, `ray` supports `num_gpus` to scheduler resource to tasks/actors, which then assign `num_gpus` number of gpu ids to be used by the tasks/actors. Additionally, `ray` provides fractional gpus allocation by specifying `num_gpus < 1` so a single GPU can be used to run multiple tasks. This works well if the cluster only has a single type of GPUs. However, imagining a cluster has both A100 40GB and A100 80GB GPUs, setting num_gpus to a fixed number doesn’t work that well: if we set to 0.1 then we will get 4GB if the scheduler picks A100 40GB but 8GB if the scheduler picks A100 80GB which is a waste of resource if the task only needs 4GB. We can also set accelerator_type to A100_40GB and num_gpus to 0.1 to make sure we get the exact amount of GPU memory we need but then the task cannot run on A100 80GB even if it’s free. Many users also have encountered this issue: +Currently, `ray` supports `num_gpus` for scheduling, which assigns `num_gpus` number of GPUs to tasks/actors. Additionally, `ray` provides fractional GPU allocation by specifying `num_gpus < 1` so a single GPU can be used to run multiple tasks. This works well if the cluster only has a single type of GPUs. However, imagining a cluster has both A100 40GB and A100 80GB GPUs, setting num_gpus to a fixed number doesn’t work that well: if we set to 0.1 then we will get 4GB if the scheduler picks A100 40GB but 8GB if the scheduler picks A100 80GB which is a waste of resource if the task only needs 4GB. We can also set accelerator_type to A100_40GB and num_gpus to 0.1 to make sure we get the exact amount of GPU memory we need but then the task cannot run on A100 80GB even if it’s free. Many users also have encountered this issue: - https://github.com/ray-project/ray/issues/37574 @@ -20,7 +20,7 @@ Currently, `ray` supports `num_gpus` to scheduler resource to tasks/actors, whic - https://discuss.ray.io/t/ray-train-ray-tune-ray-clusters-handling-different-gpus-with-different-gpu-memory-sizes-in-a-ray-cluster/9220 -This REP allows users to directly schedule fractional gpu resources by amount of memory. In our example, if user specify `gpu_memory = 20GB`, then `ray` automatically converts the value to `num_gpus` depending on which nodes the request is assigned to. As example, if it's scheduled on A100 40GB node, then `num_gpus = 0.5`, otherwise if it's scheduled on A100 80GB node, then `num_gpus = 0.25`. As a result, user can schedule a fixed amount of GPU resources without depending on which types of GPUs the tasks/actos are scheduled to. +This REP allows users to directly schedule fractional GPU resources by amount of GPU memory. In our design, if a user specifies `gpu_memory = 20GB`, then `ray` automatically converts the value to `num_gpus` depending on which node the request is assigned to. As example, if it's scheduled on A100 40GB node, then `num_gpus = 0.5`, otherwise if it's scheduled on A100 80GB node, then `num_gpus = 0.25`. As a result, user can schedule a fixed amount of GPU resources without depending on which types of GPUs the tasks/actos are scheduled to. ### Should this change be within `ray` or outside? @@ -41,7 +41,7 @@ To make the review process more productive, the owner of each proposal should id ## Design ### API -Users will be able to specify amount of gpu memory to their ray tasks/actors using `gpu_memory` on `ray.remote`. The specified `gpu_memory` will be the amount of gpu resources from a single gpu that will be allocated to users ray tasks/actors. +Users will be able to specify amount of GPU memory to their ray tasks/actors using `gpu_memory` on `ray.remote`. The specified `gpu_memory` will be the amount of GPU resources from a single GPU that will be allocated to users ray tasks/actors. ```python # Request a fractional GPU with specified gpu_memory in bytes. @@ -51,16 +51,15 @@ def task: … ``` -When a Ray node is started, Ray will auto detect the number of GPUs and GPU memory of each GPU and set the GPU and gpu_memory resources accordingly. Users also have the option to manually specify `gpu_memory` resources as the sum of total gpu memory across all gpus in the node. The default value is `num_gpus` * total memory of the gpu type in the node. +When a Ray node is started, Ray will auto detect the number of GPUs and GPU memory of each GPU. Users also have the option to manually specify them: ```bash ray start # auto detection -ray start –num_gpus=3 –gpu_memory=3000 * 1024 * 1024 * 1024 # manual override, each gpu has 1000mb total memory +ray start –num_gpus=3 –gpu_memory=1000 * 1024 * 1024 * 1024 # manual override, each GPU has 1000mb memory ``` -Note that GPU memory and compute unit is 1-1 conversion, means 20GB of gpu memory is equivalent to 0.5 fractional value of an `A100_40GB` gpu. So, for simplicity and consistency, ray doesn't allow users to specify both `num_gpus` and `gpu_memory` in a single ray task/actor. -Note that GPU memory and GPU is 1-1 conversion, means 20GB of gpu memory is equivalent to 0.5 fractional value of an `A100_40GB` gpu. So, for simplicity and consistency, ray doesn't allow users to specify both `num_gpus` and `gpu_memory` in a single ray task/actor. +Note that GPU memory and GPU is 1-1 conversion, means 20GB of GPU memory is equivalent to 0.5 fractional value of an `A100_40GB` GPU. So, for simplicity and consistency, ray doesn't allow users to specify both `num_gpus` and `gpu_memory` in a single ray task/actor. ```python @@ -80,7 +79,7 @@ def gpu_memory_task: … ``` -Additionally, users can still specify which gpu type they want to use by specifying `accelerator_type`. +Additionally, users can still specify which GPU type they want to use by specifying `accelerator_type`. ```python # Request a fractional of A100 GPU with specified gpu_memory @@ -88,7 +87,7 @@ Additionally, users can still specify which gpu type they want to use by specify def nvidia_a100_gpu_task: … -# Requesting 30GB of gpu memory from a A10 GPU with 24GB of memory. +# Requesting 30GB of GPU memory from a A10 GPU with 24GB of memory. # Task won't be able to be scheduled. @ray.remote(gpu_memory=30 * 1024 * 1024 * 1024 * 1024, accelerator_type="NVIDIA_TESLA_A10G") def nvidia_a10_gpu_task: @@ -96,16 +95,10 @@ def nvidia_a10_gpu_task: ``` #### Placement Group -User also able to request `gpu_memory` as a placement group as follows: +User is also able to request `gpu_memory` in placement group bundles as follows: ```python pg = placement_group([{"gpu_memory": 1024 * 1024, "CPU": 1}, {"GPU": 1}]) - -# Wait until placement group is created. -ray.get(pg.ready(), timeout=10) - -# You can also use ray.wait. -ready, unready = ray.wait([pg.ready()], timeout=10) ``` ### Implementation @@ -118,7 +111,7 @@ class AcceleratorManager: def get_current_node_gpu_memory(self): ... ``` -The detected memory is added as a node label. +The detected GPU memory is added as a node label. During scheduling, the resource request that contains `gpu_memory` will be converted to the corresponding `num_gpus` resource request depending on which node the scheduler is considering. @@ -130,36 +123,31 @@ for node in nodes: resource_request.gpu_memory = 0 return resource_request - convert_relative_resources(resource_request, node) + resource_request = convert_relative_resources(resource_request, node) # After converting from gpu_memory to num_gpus, the remaining is the same if check_is_available(resource_request, node): - allocation = allocate(resource_request, node) - break + allocation = allocate(resource_request, node) + break ``` -#### Resources API -We introduce a new `ResourceID` named `GPU_Memory` (`gpu_memory` in string) to specify the amount of GPU memory resources. `GPU_Memory` is treated as a GPU resource with a distinct representation, where the relationship is defined as `GPU` equals to `GPU_Memory` divided by the total memory of a single GPU in the node as GPU resources in the node is homogeneous. Despite their distinct representations, both `GPU` and `GPU_Memory` signify the same underlying node resource, with the caveat that Ray currently only supports homogeneous GPU type for each node. - -We opt to store only `GPU` in `NodeResource` and convert `gpu_memory` to GPU means `gpu_memory` is `ResourceRequest` only resource. This implementation involves saving metadata of the single node's `total_single_gpu_memory`. `ConvertRelativeResource` converts `gpu_memory` in `ResourceRequest` to `num_gpus` based on the single node's `total_single_gpu_memory` during node feasibility checks and resource allocation. - ```python # Suppose we have two nodes with GPU type A100 40GB and A100 80gb respectively -NodeResource(available={"GPU": [1,1,1]}, label={"gpu_memory_per_gpu": "40GB"}) -NodeResource(available={"GPU": [1,1,1]}, label={"gpu_memory_per_gpu": "80GB"}) +NodeResource(available={"GPU": [1,1,1]}, label={"gpu_memory_per_gpu": 40GB}) +NodeResource(available={"GPU": [1,1,1]}, label={"gpu_memory_per_gpu": 80GB}) # gpu_memory request -task.options(gpu_memory="10GB") +task.options(gpu_memory=10GB) # equivalent resource request when scheduled in Node 1 ResourceRequest({"GPU": 0.25}) -# remaining resources in Node 1, check using ray.available_resources() +# remaining resources in Node 1 NodeResource(available={"GPU": [0.75,1,1]}) # equivalent resource request when scheduled in Node 2 ResourceRequest({"GPU": 0.125}) -# remaining resources in Node 2, check using ray.available_resources() +# remaining resources in Node 2 NodeResource(available={"GPU": [0.875,1,1]}) @@ -168,103 +156,53 @@ task.options(gpu_memory="10MB") # equivalent resource request when scheduled in Node 1 ResourceRequest({"GPU": 0.00025}) -# round up to nearest 10^{-4} +# round up to nearest 10^{-4} due to the precision limitation of FixedPoint ResourceRequest({"GPU": 0.0003}) -# remaining resources in Node 1, check using ray.available_resources() +# remaining resources in Node 1 NodeResource(available={"GPU": [0.9997,1,1]}) # equivalent resource request when scheduled in Node 2 ResourceRequest({"GPU": 0.000125}) # round up to nearest 10^{-4} ResourceRequest({"GPU": 0.0002}) -# remaining resources in Node 2, check using ray.available_resources() +# remaining resources in Node 2 NodeResource(available={"GPU": [0.9998,1,1]}) ``` Pros: -- Simplified Resource Model: Better emphasizes to new users that Ray only have `GPU` to represent GPU resource, simplifying the resource model. -- Straightforward Conversion: `GPU_Memory` is converted to GPU based on the single node's total_single_gpu_memory during node feasibility checks and resource allocation with the roundup logic applied underhood. +- Simplified Resource Model: There is still only GPU node resource, gpu_memory is just another way to request the same GPU node resource. +- Straightforward Conversion: `gpu_memory` is converted to `num_gpus` based on the target node's GPU memory during scheduling. Cons: -- Limited Observability: `ray.available_resources()` only displays remaining GPU resources in terms of percentage, without specific amounts for `GPU_Memory`. - Incompatibility with Heterogeneous GPUs: Doesn't work for heterogeneous GPUs in a single node, a limitation existing in Ray's current support. -##### Alternative -For the alternative resource implementtion, we store `GPU_Memory` as part of the `NodeResource`. This implementation ensures synchronization between GPU and `GPU_Memory` Node resources. During node availability checks and resource allocation, the `ConvertRelativeResource` function performs two conversions: calculating `gpu_memory` if `num_gpus` is specified and vice versa. - -```python -# Suppose we have two nodes with GPU type A100 40GB and A100 80gb respectively -NodeResource(available={"GPU": [1,1,1], "gpu_memory": ["40GB", "40GB", "40GB"]}) -NodeResource(available={"GPU": [1,1,1], "gpu_memory": ["80GB", "80GB", "80GB"]}) - - -# gpu_memory request -task.options(gpu_memory="10GB") - -# equivalent resource request when scheduled in Node 1 -ResourceRequest({"GPU": 0.25, "gpu_memory": "10GB"}) -# remaining resources in Node 1, check using ray.available_resources() -NodeResource(available={"GPU": [0.75,1,1], "gpu_memory": ["30GB", "80GB", "80GB"]}) - -# equivalent resource request when scheduled in Node 2 -ResourceRequest({"GPU": 0.125, "gpu_memory": "10GB"}) -# remaining resources in Node 2, check using ray.available_resources() -NodeResource(available={"GPU": [0.875,1,1], "gpu_memory": ["70GB", "80GB", "80GB"]}) - - -# Roundup gpu_memory request -task.options(gpu_memory="10MB") - -# equivalent resource request when scheduled in Node 1 -ResourceRequest({"GPU": 0.00025, "gpu_memory": "10MB"}) -# round up to nearest 10^{-4} -ResourceRequest({"GPU": 0.0003, "gpu_memory": "12MB"}) -# remaining resources in Node 1, check using ray.available_resources() -NodeResource(available={"GPU": [0.9997,1,1], "gpu_memory": ["39.988GB", "40GB", "40GB"]}) +#### Alternatives -# equivalent resource request when scheduled in Node 2 -ResourceRequest({"GPU": 0.000125, "gpu_memory": "10MB"}) -# round up to nearest 10^{-4} -ResourceRequest({"GPU": 0.0002, "gpu_memory": "16MB"}) -# remaining resources in Node 2, check using ray.available_resources() -NodeResource(available={"GPU": [0.9998,1,1], "gpu_memory": ["79.984GB", "80GB", "80GB"]}) -``` +For one alternative, we can store `GPU_Memory` as node resource and `num_gpus` resource request will be converted to `gpu_memory` resource request during scheduling but this is a bigger and breaking change compared to the proposed option. -Pros: -- Enhanced Observability: Users can see the remaining GPU memory resources after roundup allocation, providing detailed insights. - -Cons: -- Synchronization Overhead: Requires synchronization for both `GPU` and `GPU_Memory`, introducing an additional layer of complexity by updating both `GPU` and `GPU_Memory` for rounding up. -- Resource Duality: Users need to grasp that both resources, `GPU` and `GPU_Memory`, essentially denote the same underlying resource. +Another alternative is having both `GPU_Memory` and `GPU` as node resources. But since they denote the same underlying resource, this modeling adds more confusion and the implementation needs to make sure these two node resources are synchronized (i.e. requesting one will also subtract the other). #### Autoscaler -For the first option, we require to node label to store `gpu_memory_per_gpu`. However, currently label is not included in autoscaler as label-based node scheduling is yet to be supported for autoscaler. Therefore, for current solution, we introduce a new private resources `node:gpu_memory_per_gpu__` as a constant label value representing `gpu_memory_per_gpu` node label. Next, we also add `convert_relative_resources` function before in `_fits` and `_inplace_subtract` in `resource_demand_scheduler.py` +Autoscaler change is similar. We need to convert `gpu_memory` resource demand to `num_gpus` resource demand when considering each node type. Concretely, we also add `convert_relative_resources` function before in `_fits` and `_inplace_subtract` in `resource_demand_scheduler.py`: ```python def _convert_relative_resources(node, resources): - adjusted_resources = resources.copy() + adjusted_resources = resources.copy() if "gpu_memory" in resources: - if "node:gpu_memory_per_gpu" not in node or node["node:gpu_memory_per_gpu"] == 0: - return None adjusted_resources["GPU"] = ( - resources["gpu_memory"] / node["node:gpu_memory_per_gpu"] + resources["gpu_memory"] / node.labels["gpu_memory_per_gpu"] ) del adjusted_resources["gpu_memory"] return adjusted_resources def _fits(node: ResourceDict, resources: ResourceDict) -> bool: adjusted_resources = _convert_relative_resources(node, resources) - if adjusted_resources is None: - return False for k, v in adjusted_resources.items(): ... def _inplace_subtract(node: ResourceDict, resources: ResourceDict) -> None: adjusted_resources = _convert_relative_resources(node, resources) - if adjusted_resources is None: - return for k, v in adjusted_resources.items(): ... ``` -For the second option, there's no required addition of `node:gpu_memory_per_gpu_` since `GPU_Memory` is part of resources, but the `_convert_relative_resources` still required.