Skip to content

Commit

Permalink
[Core][Dashboard] Remove gpustats dependencies from Ray[default] (#41399
Browse files Browse the repository at this point in the history
)

Signed-off-by: Jonathan Nitisastro <jonathancn@anyscale.com>
  • Loading branch information
jonathan-anyscale authored Dec 15, 2023
1 parent dfa5519 commit 9f30c6b
Show file tree
Hide file tree
Showing 11 changed files with 195 additions and 1,380 deletions.
1 change: 0 additions & 1 deletion ci/env/check_minimal_install.py
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@
"aiohttp_cors",
"colorful",
"py-spy",
"gpustat",
"opencensus",
"prometheus_client",
"smart_open",
Expand Down
15 changes: 8 additions & 7 deletions dashboard/client/src/pages/node/index.tsx
Original file line number Diff line number Diff line change
Expand Up @@ -85,13 +85,14 @@ const columns = [
helpInfo: (
<Typography>
Usage of each GPU device. If no GPU usage is detected, here are the
potential root causes: <br />
1. library gpustsat is not installed. Install gpustat and try again.
<br /> 2. non-GPU Ray image is used on this node. Switch to a GPU Ray
image and try again. <br />
3. AMD GPUs are being used. AMD GPUs are not currently supported by
gpustat module. <br />
4. gpustat module raises an exception.
potential root causes:
<br />
1. non-GPU Ray image is used on this node. Switch to a GPU Ray image and
try again. <br />
2. Non Nvidia GPUs are being used. Non Nvidia GPUs' utilizations are not
currently supported.
<br />
3. pynvml module raises an exception.
</Typography>
),
},
Expand Down
15 changes: 4 additions & 11 deletions dashboard/client/src/type/node.d.ts
Original file line number Diff line number Diff line change
Expand Up @@ -53,28 +53,21 @@ export type NodeListRsp = {
msg: string;
};

export type GPUProcessStats = {
// Sub stat of GPU stats, this type represents the GPU
// utilization of a single process of a single GPU.
username: string;
command: string;
gpuMemoryUsage: number;
export type ProcessGPUUsage = {
// This gpu usage stats from a process
pid: number;
gpuMemoryUsage: number;
};

export type GPUStats = {
// This represents stats fetched from a node about a single GPU
uuid: string;
index: number;
name: string;
temperatureGpu: number;
fanSpeed: number;
utilizationGpu?: number;
powerDraw: number;
enforcedPowerLimit: number;
memoryUsed: number;
memoryTotal: number;
processes?: GPUProcessStats[];
processes?: ProcessGPUUsage[];
};

export type NodeDetailExtend = {
Expand Down
127 changes: 84 additions & 43 deletions dashboard/modules/reporter/reporter_agent.py
Original file line number Diff line number Diff line change
Expand Up @@ -6,11 +6,10 @@
import socket
import sys
import traceback
import warnings

import psutil

from typing import List, Optional, Tuple
from typing import List, Optional, Tuple, TypedDict, Union
from collections import defaultdict

import ray
Expand All @@ -29,7 +28,6 @@
from ray._private.metrics_agent import Gauge, MetricsAgent, Record
from ray._private.ray_constants import DEBUG_AUTOSCALING_STATUS
from ray.core.generated import reporter_pb2, reporter_pb2_grpc
from ray.util.debug import log_once
from ray.dashboard import k8s_utils
from ray._raylet import WorkerID

Expand All @@ -48,25 +46,6 @@
# Using existence of /sys/fs/cgroup as the criterion is consistent with
# Ray's existing resource logic, see e.g. ray._private.utils.get_num_cpus().

try:
import gpustat.core as gpustat
except ModuleNotFoundError:
gpustat = None
if log_once("gpustat_import_warning"):
warnings.warn(
"`gpustat` package is not installed. GPU monitoring is "
"not available. To have full functionality of the "
"dashboard please install `pip install ray["
"default]`.)"
)
except ImportError as e:
gpustat = None
if log_once("gpustat_import_warning"):
warnings.warn(
"Importing gpustat failed, fix this to have full "
"functionality of the dashboard. The original error was:\n\n" + e.msg
)


def recursive_asdict(o):
if isinstance(o, tuple) and hasattr(o, "_asdict"):
Expand Down Expand Up @@ -268,6 +247,29 @@ def jsonify_asdict(o) -> str:
),
}

MB = 1024 * 1024

# Types
Percentage = int
Megabytes = int


# gpu utilization for nvidia gpu from a single process
class ProcessGPUInfo(TypedDict):
pid: int
gpu_memory_usage: Megabytes


# gpu utilization for nvidia gpu
class GpuUtilizationInfo(TypedDict):
index: int
name: str
uuid: str
utilization_gpu: Optional[Percentage]
memory_used: Megabytes
memory_total: Megabytes
processes_pids: Optional[List[ProcessGPUInfo]]


class ReporterAgent(
dashboard_utils.DashboardAgentModule, reporter_pb2_grpc.ReporterServiceServicer
Expand Down Expand Up @@ -393,31 +395,77 @@ def _get_cpu_percent(in_k8s: bool):

@staticmethod
def _get_gpu_usage():
import ray._private.thirdparty.pynvml as pynvml

global enable_gpu_usage_check
if gpustat is None or not enable_gpu_usage_check:
if not enable_gpu_usage_check:
return []
gpu_utilizations = []
gpus = []

def decode(b: Union[str, bytes]) -> str:
if isinstance(b, bytes):
return b.decode("utf-8") # for python3, to unicode
return b

try:
gpus = gpustat.new_query().gpus
pynvml.nvmlInit()
except Exception as e:
logger.debug(f"gpustat failed to retrieve GPU information: {e}")
logger.debug(f"pynvml failed to retrieve GPU information: {e}")

# gpustat calls pynvml.nvmlInit()
# On machines without GPUs, this can run subprocesses that spew to
# stderr. Then with log_to_driver=True, we get log spew from every
# On machines without GPUs, pynvml.nvmlInit() can run subprocesses that
# spew to stderr. Then with log_to_driver=True, we get log spew from every
# single raylet. To avoid this, disable the GPU usage check on
# certain errors.
# https://github.com/ray-project/ray/issues/14305
# https://github.com/ray-project/ray/pull/21686
if type(e).__name__ == "NVMLError_DriverNotLoaded":
enable_gpu_usage_check = False
return gpu_utilizations

num_gpus = pynvml.nvmlDeviceGetCount()
for i in range(num_gpus):
gpu_handle = pynvml.nvmlDeviceGetHandleByIndex(i)
memory_info = pynvml.nvmlDeviceGetMemoryInfo(gpu_handle)
utilization = None
try:
utilization_info = pynvml.nvmlDeviceGetUtilizationRates(gpu_handle)
utilization = int(utilization_info.gpu)
except pynvml.NVMLError as e:
logger.debug(f"pynvml failed to retrieve GPU utilization: {e}")

# processes pids
processes_pids = None
try:
nv_comp_processes = pynvml.nvmlDeviceGetComputeRunningProcesses(
gpu_handle
)
nv_graphics_processes = pynvml.nvmlDeviceGetGraphicsRunningProcesses(
gpu_handle
)
processes_pids = [
ProcessGPUInfo(
pid=int(nv_process.pid),
gpu_memory_usage=int(nv_process.usedGpuMemory) // MB
if nv_process.usedGpuMemory
else 0,
)
for nv_process in (nv_comp_processes + nv_graphics_processes)
]
except pynvml.NVMLError as e:
logger.debug(f"pynvml failed to retrieve GPU processes: {e}")

info = GpuUtilizationInfo(
index=i,
name=decode(pynvml.nvmlDeviceGetName(gpu_handle)),
uuid=decode(pynvml.nvmlDeviceGetUUID(gpu_handle)),
utilization_gpu=utilization,
memory_used=int(memory_info.used) // MB,
memory_total=int(memory_info.total) // MB,
processes_pids=processes_pids,
)
gpu_utilizations.append(info)
pynvml.nvmlShutdown()

for gpu in gpus:
# Note the keys in this dict have periods which throws
# off javascript so we change .s to _s
gpu_data = {"_".join(key.split(".")): val for key, val in gpu.entry.items()}
gpu_utilizations.append(gpu_data)
return gpu_utilizations

@staticmethod
Expand Down Expand Up @@ -911,21 +959,14 @@ def _record_stats(self, stats, cluster_stats):
)
records_reported.append(node_mem_shared)

# The output example of gpustats.
# The output example of GpuUtilizationInfo.
"""
{'index': 0,
'uuid': 'GPU-36e1567d-37ed-051e-f8ff-df807517b396',
'name': 'NVIDIA A10G',
'temperature_gpu': 20,
'fan_speed': 0,
'utilization_gpu': 1,
'utilization_enc': 0,
'utilization_dec': 0,
'power_draw': 51,
'enforced_power_limit': 300,
'memory_used': 0,
'memory_total': 22731,
'processes': []}
'memory_total': 22731}
"""
# -- GPU per node --
gpus = stats["gpus"]
Expand Down
36 changes: 0 additions & 36 deletions dashboard/modules/reporter/tests/test_reporter.py
Original file line number Diff line number Diff line change
Expand Up @@ -359,13 +359,7 @@ def test_report_stats_gpu():
{'index': 0,
'uuid': 'GPU-36e1567d-37ed-051e-f8ff-df807517b396',
'name': 'NVIDIA A10G',
'temperature_gpu': 20,
'fan_speed': 0,
'utilization_gpu': 1,
'utilization_enc': 0,
'utilization_dec': 0,
'power_draw': 51,
'enforced_power_limit': 300,
'memory_used': 0,
'memory_total': 22731,
'processes': []}
Expand All @@ -376,13 +370,7 @@ def test_report_stats_gpu():
"index": 0,
"uuid": "GPU-36e1567d-37ed-051e-f8ff-df807517b396",
"name": "NVIDIA A10G",
"temperature_gpu": 20,
"fan_speed": 0,
"utilization_gpu": 0,
"utilization_enc": 0,
"utilization_dec": 0,
"power_draw": 51,
"enforced_power_limit": 300,
"memory_used": 0,
"memory_total": GPU_MEMORY,
"processes": [],
Expand All @@ -391,13 +379,7 @@ def test_report_stats_gpu():
"index": 1,
"uuid": "GPU-36e1567d-37ed-051e-f8ff-df807517b397",
"name": "NVIDIA A10G",
"temperature_gpu": 20,
"fan_speed": 0,
"utilization_gpu": 1,
"utilization_enc": 0,
"utilization_dec": 0,
"power_draw": 51,
"enforced_power_limit": 300,
"memory_used": 1,
"memory_total": GPU_MEMORY,
"processes": [],
Expand All @@ -406,13 +388,7 @@ def test_report_stats_gpu():
"index": 2,
"uuid": "GPU-36e1567d-37ed-051e-f8ff-df807517b398",
"name": "NVIDIA A10G",
"temperature_gpu": 20,
"fan_speed": 0,
"utilization_gpu": 2,
"utilization_enc": 0,
"utilization_dec": 0,
"power_draw": 51,
"enforced_power_limit": 300,
"memory_used": 2,
"memory_total": GPU_MEMORY,
"processes": [],
Expand All @@ -421,13 +397,7 @@ def test_report_stats_gpu():
{
"index": 3,
"uuid": "GPU-36e1567d-37ed-051e-f8ff-df807517b398",
"temperature_gpu": 20,
"fan_speed": 0,
"utilization_gpu": 3,
"utilization_enc": 0,
"utilization_dec": 0,
"power_draw": 51,
"enforced_power_limit": 300,
"memory_used": 3,
"memory_total": GPU_MEMORY,
"processes": [],
Expand All @@ -436,13 +406,7 @@ def test_report_stats_gpu():
{
"uuid": "GPU-36e1567d-37ed-051e-f8ff-df807517b398",
"name": "NVIDIA A10G",
"temperature_gpu": 20,
"fan_speed": 0,
"utilization_gpu": 3,
"utilization_enc": 0,
"utilization_dec": 0,
"power_draw": 51,
"enforced_power_limit": 300,
"memory_used": 3,
"memory_total": 22731,
"processes": [],
Expand Down
12 changes: 8 additions & 4 deletions python/ray/_private/accelerators/nvidia_gpu.py
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,6 @@
import os
import logging
from typing import Optional, List, Tuple
import ray._private.thirdparty.pynvml as pynvml

from ray._private.accelerators.accelerator import AcceleratorManager

Expand Down Expand Up @@ -45,6 +44,8 @@ def get_current_process_visible_accelerator_ids() -> Optional[List[str]]:

@staticmethod
def get_current_node_num_accelerators() -> int:
import ray._private.thirdparty.pynvml as pynvml

try:
pynvml.nvmlInit()
except pynvml.NVMLError:
Expand All @@ -55,6 +56,8 @@ def get_current_node_num_accelerators() -> int:

@staticmethod
def get_current_node_accelerator_type() -> Optional[str]:
import ray._private.thirdparty.pynvml as pynvml

try:
pynvml.nvmlInit()
except pynvml.NVMLError:
Expand All @@ -63,10 +66,11 @@ def get_current_node_accelerator_type() -> Optional[str]:
cuda_device_type = None
if device_count > 0:
handle = pynvml.nvmlDeviceGetHandleByIndex(0)
device_name = pynvml.nvmlDeviceGetName(handle)
if isinstance(device_name, bytes):
device_name = device_name.decode("utf-8")
cuda_device_type = (
NvidiaGPUAcceleratorManager._gpu_name_to_accelerator_type(
pynvml.nvmlDeviceGetName(handle)
)
NvidiaGPUAcceleratorManager._gpu_name_to_accelerator_type(device_name)
)
pynvml.nvmlShutdown()
return cuda_device_type
Expand Down
5 changes: 3 additions & 2 deletions python/ray/_private/thirdparty/pynvml/__init__.py
Original file line number Diff line number Diff line change
@@ -1,3 +1,4 @@
from ray._private.thirdparty.pynvml.pynvml import *
# current version
__version__ = "12.535.133"
# nvdia-ml-py version
# Note: we pick this version to use the V2 API which is supported by older drivers
__version__ = "11.495.46"
Loading

0 comments on commit 9f30c6b

Please sign in to comment.