Skip to content

Commit bbdce36

Browse files
authored
Fix GPU metrics (#56009)
<!-- Thank you for your contribution! Please review https://github.com/ray-project/ray/blob/master/CONTRIBUTING.rst before opening a pull request. --> <!-- Please add a reviewer to the assignee section when you create a PR. If you don't have the access to it, we will shortly find a reviewer and assign them to your PR. --> ## Why are these changes needed? Bugs introduced in #52102 Two bugs: - proc is a TypedDict so it needs to be fetched via `proc["pid"]` instead of `proc.pid`. - Changing `processes_pid` is backwards-incompatible change that ends up changing the dashboard APIs that power the ray dashboard. Maintain backwards-compatibility <!-- Please give a short summary of the change and the problem this solves. --> Verified fix: Metrics work again: <img width="947" height="441" alt="Screenshot 2025-08-27 at 12 22 40 PM" src="https://github.com/user-attachments/assets/0a9a83e7-b720-4ad0-b90e-1baa394edde5" /> Ray Dashboard works again: <img width="1824" height="1029" alt="Screenshot 2025-08-27 at 12 21 51 PM" src="https://github.com/user-attachments/assets/6b0e08e4-69c9-4223-b736-ff69b8d306db" /> ## Related issue number <!-- For example: "Closes #1234" --> ## Checks - [x] I've signed off every commit(by using the -s flag, i.e., `git commit -s`) in this PR. - [x] I've run `scripts/format.sh` to lint the changes in this PR. - [ ] I've included any doc changes needed for https://docs.ray.io/en/master/. - [ ] I've added any new APIs to the API Reference. For example, if I added a method in Tune, I've added it in `doc/source/tune/api/` under the corresponding `.rst` file. - [ ] I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/ - Testing Strategy - [ ] Unit tests - [ ] Release tests - [ ] This PR is not tested :( --------- Signed-off-by: Alan Guo <[email protected]>
1 parent 9241930 commit bbdce36

File tree

4 files changed

+299
-54
lines changed

4 files changed

+299
-54
lines changed

python/ray/dashboard/modules/node/datacenter.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -221,7 +221,7 @@ async def _get_actor_info(actor: Optional[dict]) -> Optional[dict]:
221221
break
222222

223223
for gpu_stats in node_physical_stats.get("gpus", []):
224-
# gpu_stats.get("processes") can be None, an empty list or a
224+
# gpu_stats.get("processesPids") can be None, an empty list or a
225225
# list of dictionaries.
226226
for process in gpu_stats.get("processesPids") or []:
227227
if process["pid"] == pid:

python/ray/dashboard/modules/reporter/gpu_providers.py

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -230,9 +230,9 @@ def _parse_nvsmi_pmon_output(
230230
1 7175 C 86 26 - - - - ray::TorchGPUWo
231231
2 - - - - - - - - -
232232
233-
Returns a dict mapping GPU index to list of ProcessGPUInfo.
233+
Returns a dict mapping GPU index to dict of pid to ProcessGPUInfo.
234234
"""
235-
process_utilizations = defaultdict(list)
235+
process_utilizations = defaultdict(dict)
236236
lines = nvsmi_stdout.splitlines()
237237
# Get the first line that is started with #
238238
table_header = None
@@ -275,7 +275,7 @@ def _parse_nvsmi_pmon_output(
275275
), # Convert percentage to MB
276276
gpu_utilization=sm,
277277
)
278-
process_utilizations[gpu_id].append(process_info)
278+
process_utilizations[gpu_id][pid] = process_info
279279
return process_utilizations
280280

281281
def _get_pynvml_gpu_usage(self) -> List[GpuUtilizationInfo]:

python/ray/dashboard/modules/reporter/reporter_agent.py

Lines changed: 17 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -893,7 +893,7 @@ def _get_agent_proc(self) -> psutil.Process:
893893
def _generate_worker_key(self, proc: psutil.Process) -> Tuple[int, float]:
894894
return (proc.pid, proc.create_time())
895895

896-
def _get_workers(self, gpus: Optional[List[GpuUtilizationInfo]] = None):
896+
def _get_worker_processes(self):
897897
raylet_proc = self._get_raylet_proc()
898898
if raylet_proc is None:
899899
return []
@@ -910,7 +910,13 @@ def _get_workers(self, gpus: Optional[List[GpuUtilizationInfo]] = None):
910910
self._generate_worker_key(proc): proc
911911
for proc in raylet_proc.children()
912912
}
913+
return workers
913914

915+
def _get_workers(self, gpus: Optional[List[GpuUtilizationInfo]] = None):
916+
workers = self._get_worker_processes()
917+
if not workers:
918+
return []
919+
else:
914920
# We should keep `raylet_proc.children()` in `self` because
915921
# when `cpu_percent` is first called, it returns the meaningless 0.
916922
# See more: https://github.com/ray-project/ray/issues/29848
@@ -937,7 +943,7 @@ def _get_workers(self, gpus: Optional[List[GpuUtilizationInfo]] = None):
937943
processes = gpu.get("processes_pids")
938944
if processes:
939945
for proc in processes.values():
940-
gpu_pid_mapping[proc.pid].append(proc)
946+
gpu_pid_mapping[proc["pid"]].append(proc)
941947

942948
result = []
943949
for w in self._workers.values():
@@ -1763,6 +1769,15 @@ def _compose_stats_payload(
17631769

17641770
self._metrics_agent.clean_all_dead_worker_metrics()
17651771

1772+
# Convert processes_pids back to a list of dictionaries to maintain backwards-compatibility
1773+
for gpu in stats["gpus"]:
1774+
if isinstance(gpu.get("processes_pids"), dict):
1775+
gpu["processes_pids"] = list(gpu["processes_pids"].values())
1776+
1777+
# TODO(aguo): Add a pydantic model for this dict to maintain compatibility
1778+
# with the Ray Dashboard API and UI code.
1779+
1780+
# NOTE: This converts keys to "Google style", (e.g: "processes_pids" -> "processesPids")
17661781
return jsonify_asdict(stats)
17671782

17681783
async def run(self, server):

0 commit comments

Comments
 (0)