From fac40c83588a68bb78b3a83c4fa240f19492bab4 Mon Sep 17 00:00:00 2001 From: Andrew Sy Kim Date: Fri, 21 Nov 2025 16:39:39 +0000 Subject: [PATCH] Add initial enhancement proposal for Ray History Server Signed-off-by: Andrew Sy Kim --- .../2025-11-21-ray-history-server.md | 174 ++++++++++++++++++ .../events_file_structure.png | Bin 0 -> 33344 bytes .../history_server_architecture.png | Bin 0 -> 481126 bytes 3 files changed, 174 insertions(+) create mode 100644 reps/2025-11-21-ray-history-server/2025-11-21-ray-history-server.md create mode 100644 reps/2025-11-21-ray-history-server/events_file_structure.png create mode 100644 reps/2025-11-21-ray-history-server/history_server_architecture.png diff --git a/reps/2025-11-21-ray-history-server/2025-11-21-ray-history-server.md b/reps/2025-11-21-ray-history-server/2025-11-21-ray-history-server.md new file mode 100644 index 0000000..41aeb67 --- /dev/null +++ b/reps/2025-11-21-ray-history-server/2025-11-21-ray-history-server.md @@ -0,0 +1,174 @@ +## Ray History Server + +### General Motivation + +It is becoming increasingly common for Ray users to treat Ray clusters as ephemeral units of compute +that only run for the duration of a single (or multiple) Ray jobs. This pattern results in significant improvements in +cost efficiency and resource sharing, especially in capacity-constrained environments where hardware accelerators are scarce. + +However, a fundamental trade-off of this approach, compared to long-lived interactive Ray clusters, is that users +lose access to the Ray Dashboard, which is often treated as the entry point for most observability signals +in a Ray cluster. While it’s possible to export the relevant signals to external sources, users prefer the experience +of using the Ray Dashboard as a single source of truth to debug job failures. + +This enhancement proposal introduces the concept of a Ray History Server, which can orchestrate the reconstruction of the +Ray Dashboard even for terminated Ray clusters. This will be accomplished by leveraging Ray’s Event Export API to persist +task/actor/job state, and native components in KubeRay for pushing logs and events to a blob store (GCS, S3, etc). + +### Should this change be within `ray` or outside? + +Some components/libraries, such as the event exporter, will be in Ray. Everything else will be hosted in the KubeRay project. + +## Stewardship + +### Owners + +- @KunWuLuan (Alibaba) +- @MengjinYan, @Future-Outlier, @rueian (Anyscale) +- @andrewsykim, @Chia-ya (Google) + +### Shepherd of the Proposal (should be a senior committer) + +@edoakes + +## Design and Architecture + +### Components and Libraries + +The Ray History Server project will introduce the following components and libraries across Ray and KubeRay: + +* Ray: + * Updated Ray Dashboard frontend that can dynamically adjust request paths to fetch task/actor/job state from a history server. + * Ray Event Export API, available starting in Ray 2.49, which supports task/actor/job/node events. + The events can be used to generate the state of the Ray cluster at any given timestamp. +* KubeRay: + * An events collector sidecar that receives push events from Ray (via RAY_enable_core_worker_ray_event_to_aggregator) + and persists events to the blob store. + * A logging collector sidecar that uploads Ray logs to the blob store. + * A history server (standalone Deployment) that can process events for historical Ray clusters and + serve Ray API endpoints requested by Ray Dashboards. + * A storage reader/writer library providing a pluggable interface for different storage implementations. + +![Ray History Server Architecture](history_server_architecture.png) + +#### Events Collector + +The Events Collector is a sidecar container deployed alongside every Ray node. It operates as an +HTTP server ingesting events from Ray’s Event Export API (enabled via `RAY_enable_core_worker_ray_event_to_aggregator`). +The server exposes a single POST /events endpoint which receives event data as JSON objects. +Ray is configured to push these events to a localhost endpoint (configured with `RAY_DASHBOARD_AGGREGATOR_AGENT_EVENTS_EXPORT_ADDR`). +The Events Collector is strictly responsible for persisting raw events to blob storage; it does not perform any pre-processing or deduplication. + +#### Logging Collector + +The Logging Collector is responsible for persisting logs in `/tmp/ray/session_latest/logs` to blob store. +While the Ray cluster is active, the Logging Collector will periodically upload snapshot of logs to storage. +Upon receiving a termination signal, it will attempt to upload a final snapshot of logs before exiting. + +#### History Server + +The History Server component is a stateless Kubernetes Deployment that serves API endpoints that are compatible with the Ray Dashboard. +To serve endpoints like `/api/v0/tasks`, the History Server will be responsible for server-side processing of events +in blob store that were uploaded by the Events Collector. In alpha / beta milestones, the history server will store +final task / actor / job states in-memory. For GA, we may reconsider this approoach if we identify scalability limitations. +More details on event processing below. + +#### Event Processor + +The Event Processor runs as a process within the History Server container. It is responsible for downloading the +complete event history of terminated clusters and aggregating that data into final states. These processed states +are then used to serve API requests from Ray Dashboard clients. + +### File structure for persisted events & logs + +Users rarely filter events by node name; instead, they typically filter by job ID and time range. +Therefore, building an index based on job ID and timestamps is critical. Unlike the Spark History Server, +Ray events are emitted by an aggregation agent residing on each node; therefore, the collector on each specific node +is responsible for grouping the events. + +All events will initially be partitioned by Job ID. Specifically, task events associated with the same Job ID will be stored in the same directory. +* Node-level events will be stored in: cluster_name_cluster_uid/session_id/node_events/-