-
Notifications
You must be signed in to change notification settings - Fork 7k
[core] allow reporter agent to get pid via rpc to raylet #57004
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
eb8924e
d28c905
5927027
f76f633
c3dca66
1ea7081
af61506
204caef
2b32615
c8f0fa4
ac20283
b4e9a1d
53e693c
62b48cf
be70b12
f09822a
6a6b742
02656c0
3ed4295
3521a91
a93cb28
9940d37
e3c79a2
5edc6f0
e281cf4
3b97fb6
a6e7342
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -41,7 +41,7 @@ | |
| OpenTelemetryMetricRecorder, | ||
| ) | ||
| from ray._private.utils import get_system_memory | ||
| from ray._raylet import GCS_PID_KEY, WorkerID | ||
| from ray._raylet import GCS_PID_KEY, RayletClient, WorkerID | ||
| from ray.core.generated import reporter_pb2, reporter_pb2_grpc | ||
| from ray.dashboard import k8s_utils | ||
| from ray.dashboard.consts import ( | ||
|
|
@@ -66,6 +66,10 @@ | |
| from ray.dashboard.modules.reporter.reporter_models import ( | ||
| StatsPayload, | ||
| ) | ||
| from ray.exceptions import ( | ||
| GetTimeoutError, | ||
| RpcError, | ||
| ) | ||
|
|
||
| import psutil | ||
|
|
||
|
|
@@ -395,9 +399,10 @@ class ReporterAgent( | |
|
|
||
| Attributes: | ||
| dashboard_agent: The DashboardAgent object contains global config | ||
| raylet_client: The RayletClient object to access raylet server | ||
| """ | ||
|
|
||
| def __init__(self, dashboard_agent): | ||
| def __init__(self, dashboard_agent, raylet_client=None): | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Are the
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. yes |
||
| """Initialize the reporter object.""" | ||
| super().__init__(dashboard_agent) | ||
|
|
||
|
|
@@ -486,6 +491,13 @@ def __init__(self, dashboard_agent): | |
| # Create GPU metric provider instance | ||
| self._gpu_metric_provider = GpuMetricProvider() | ||
|
|
||
| if raylet_client: | ||
| self._raylet_client = raylet_client | ||
| else: | ||
| self._raylet_client = RayletClient( | ||
| ip_address=self._ip, port=self._dashboard_agent.node_manager_port | ||
| ) | ||
|
|
||
| async def GetTraceback(self, request, context): | ||
| pid = request.pid | ||
| native = request.native | ||
|
|
@@ -888,6 +900,14 @@ def _get_disk_io_stats(): | |
| stats.write_count, | ||
| ) | ||
|
|
||
| async def _async_get_worker_pids_from_raylet(self) -> List[int]: | ||
| try: | ||
| # Get worker pids from raylet via gRPC. | ||
| return await self._raylet_client.async_get_worker_pids() | ||
| except (GetTimeoutError, RpcError): | ||
| logger.exception("Failed to get worker pids from raylet") | ||
| return [] | ||
|
|
||
| def _get_agent_proc(self) -> psutil.Process: | ||
| # Agent is the current process. | ||
| # This method is not necessary, but we have it for mock testing. | ||
|
|
@@ -896,27 +916,23 @@ def _get_agent_proc(self) -> psutil.Process: | |
| def _generate_worker_key(self, proc: psutil.Process) -> Tuple[int, float]: | ||
| return (proc.pid, proc.create_time()) | ||
|
|
||
| def _get_worker_processes(self): | ||
| raylet_proc = self._get_raylet_proc() | ||
| if raylet_proc is None: | ||
| async def _async_get_worker_processes(self): | ||
| pids = await self._async_get_worker_pids_from_raylet() | ||
| logger.debug(f"Worker PIDs from raylet: {pids}") | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. |
||
| if not pids: | ||
| return [] | ||
| else: | ||
| workers = {} | ||
| if sys.platform == "win32": | ||
| # windows, get the child process not the runner | ||
| for child in raylet_proc.children(): | ||
| if child.children(): | ||
| child = child.children()[0] | ||
| workers[self._generate_worker_key(child)] = child | ||
| else: | ||
| workers = { | ||
| self._generate_worker_key(proc): proc | ||
| for proc in raylet_proc.children() | ||
| } | ||
| return workers | ||
|
|
||
| def _get_workers(self, gpus: Optional[List[GpuUtilizationInfo]] = None): | ||
| workers = self._get_worker_processes() | ||
| workers = {} | ||
| for pid in pids: | ||
| try: | ||
| proc = psutil.Process(pid) | ||
| workers[self._generate_worker_key(proc)] = proc | ||
| except (psutil.NoSuchProcess, psutil.AccessDenied): | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. logger.error("...") |
||
| logger.error(f"Failed to access worker process {pid}") | ||
| continue | ||
| return workers | ||
|
|
||
| async def _async_get_workers(self, gpus: Optional[List[GpuUtilizationInfo]] = None): | ||
| workers = await self._async_get_worker_processes() | ||
| if not workers: | ||
| return [] | ||
| else: | ||
|
|
@@ -936,9 +952,6 @@ def _get_workers(self, gpus: Optional[List[GpuUtilizationInfo]] = None): | |
| for k in keys_to_pop: | ||
| self._workers.pop(k) | ||
|
|
||
| # Remove the current process (reporter agent), which is also a child of | ||
| # the Raylet. | ||
| self._workers.pop(self._generate_worker_key(self._get_agent_proc())) | ||
| # Build process ID -> GPU info mapping for faster lookups | ||
| gpu_pid_mapping = defaultdict(list) | ||
| if gpus is not None: | ||
|
|
@@ -1058,7 +1071,7 @@ def _get_shm_usage(self): | |
| return None | ||
| return mem.shared | ||
|
|
||
| def _collect_stats(self): | ||
| async def _async_collect_stats(self): | ||
| now = dashboard_utils.to_posix_time(datetime.datetime.utcnow()) | ||
| network_stats = self._get_network_stats() | ||
| self._network_stats_hist.append((now, network_stats)) | ||
|
|
@@ -1079,7 +1092,7 @@ def _collect_stats(self): | |
| "mem": self._get_mem_usage(), | ||
| # Unit is in bytes. None if | ||
| "shm": self._get_shm_usage(), | ||
| "workers": self._get_workers(gpus), | ||
| "workers": await self._async_get_workers(gpus), | ||
| "raylet": raylet, | ||
| "agent": self._get_agent(), | ||
| "bootTime": self._get_boot_time(), | ||
|
|
@@ -1726,7 +1739,7 @@ async def _run_loop(self): | |
| # executor (TPE) to avoid blocking the Agent's event-loop | ||
| json_payload = await loop.run_in_executor( | ||
| self._executor, | ||
| self._compose_stats_payload, | ||
| self._run_in_executor, | ||
| autoscaler_status_json_bytes, | ||
| ) | ||
|
|
||
|
|
@@ -1739,10 +1752,15 @@ async def _run_loop(self): | |
|
|
||
| await asyncio.sleep(reporter_consts.REPORTER_UPDATE_INTERVAL_MS / 1000) | ||
|
|
||
| def _compose_stats_payload( | ||
| def _run_in_executor(self, cluster_autoscaling_stats_json: Optional[bytes]) -> str: | ||
| return asyncio.run( | ||
| self._async_compose_stats_payload(cluster_autoscaling_stats_json) | ||
| ) | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. |
||
|
|
||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Bug: Async Method Causes Event Loop ConflictsThe |
||
| async def _async_compose_stats_payload( | ||
| self, cluster_autoscaling_stats_json: Optional[bytes] | ||
| ) -> str: | ||
| stats = self._collect_stats() | ||
| stats = await self._async_collect_stats() | ||
|
|
||
| # Report stats only when metrics collection is enabled. | ||
| if not self._metrics_collection_disabled: | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
what are these refactoring for?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
#57004 (comment)
@jjyao suggested to expose this c++ constant to python, avoiding duplicate definition and potential inconsistency