Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
47 commits
Select commit Hold shift + click to select a range
3ce2df3
Add event server for history server.
chiayi Dec 3, 2025
785df87
Update test
chiayi Dec 3, 2025
13b9187
[history server] Web Server
Future-Outlier Jan 2, 2026
ba17941
add Kun Wu's setting
Future-Outlier Jan 6, 2026
3dab9dc
Merge branch 'master' into historyserver-webserver
Future-Outlier Jan 6, 2026
f8c7214
Merge branch 'historyserver-eventserver' into historyserver-webserver
Future-Outlier Jan 6, 2026
72a9134
a worked version
Future-Outlier Jan 6, 2026
11d6eda
a worked version, will revise it
Future-Outlier Jan 6, 2026
4bd398c
Trigger CI
Future-Outlier Jan 7, 2026
44cb52e
Merge remote-tracking branch 'upstream/master' into historyserver-web…
Future-Outlier Jan 8, 2026
3912d2f
merge master
Future-Outlier Jan 8, 2026
f16a7e2
turn chinese comments to english
Future-Outlier Jan 8, 2026
1524b44
fix bugs and make dead cluster endpoint work or return not yet supported
Future-Outlier Jan 8, 2026
2b18bea
support task summarize, not yet test live cluster
Future-Outlier Jan 9, 2026
ebfff2d
support predicate
Future-Outlier Jan 9, 2026
ea1cde3
remove license
Future-Outlier Jan 10, 2026
144cf11
fix Stop signal ignored during hour-long sleep period
Future-Outlier Jan 10, 2026
14aca06
fix Main exits without waiting for graceful shutdown
Future-Outlier Jan 10, 2026
a52b19c
remove log key info
Future-Outlier Jan 10, 2026
0cae478
fix Graceful shutdown incorrectly treated as fatal error
Future-Outlier Jan 10, 2026
767d87d
fix Event processor failure causes event processing to block
Future-Outlier Jan 11, 2026
76b7f2f
Fix Task update discards all fields except attempt number, but this i…
Future-Outlier Jan 11, 2026
32532fe
fix max clusters default 0 problem, and add todo
Future-Outlier Jan 11, 2026
4e3e897
fix Missing cookie path causes repeated Kubernetes API calls
Future-Outlier Jan 11, 2026
34f5aca
fix task list problems
Future-Outlier Jan 11, 2026
a9afe32
add actor json tag
Future-Outlier Jan 11, 2026
58578b5
handle task lifecycle event, need to update to binary search
Future-Outlier Jan 11, 2026
4928b53
change upsert to merge
Future-Outlier Jan 11, 2026
b6445b8
handle task and actor endpoint better, make them complete
Future-Outlier Jan 11, 2026
cbde2ec
fix SSRF via user-controlled service name cookie
Future-Outlier Jan 11, 2026
707d319
actor and task need to solve Duplicate events appended on each hourly…
Future-Outlier Jan 11, 2026
9c13006
solve Duplicate events appended on each hourly reprocessing cycle
Future-Outlier Jan 11, 2026
ee41b2e
fix Unchecked type assertions can cause panics
Future-Outlier Jan 12, 2026
7380338
HTTP proxy requests lack timeout causing potential hangs
Future-Outlier Jan 12, 2026
c996ece
fix Nil map panic when processing null event entries
Future-Outlier Jan 12, 2026
a36304f
fix Environment variable bypasses SSRF protection for live cluster pr…
Future-Outlier Jan 12, 2026
28e11c0
support required resources and server timeout error
Future-Outlier Jan 12, 2026
9fca27d
better serviceaccount
Future-Outlier Jan 12, 2026
603da87
Add Readme
Future-Outlier Jan 12, 2026
5970b2c
Merge branch 'master' into historyserver-webserver
Future-Outlier Jan 12, 2026
a0319bc
better comments for log dir path
Future-Outlier Jan 12, 2026
e8aa0ad
fix race condition
Future-Outlier Jan 12, 2026
94135a2
better const explaination for seperator connector
Future-Outlier Jan 12, 2026
3f7e31f
1 better actor response; 2 cleanup dead code
Future-Outlier Jan 12, 2026
f195ed7
remove dead code
Future-Outlier Jan 12, 2026
a3126b4
update
Future-Outlier Jan 12, 2026
bf4576f
fix comments
Future-Outlier Jan 13, 2026
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
33 changes: 0 additions & 33 deletions historyserver/Dockerfile

This file was deleted.

29 changes: 29 additions & 0 deletions historyserver/Dockerfile.historyserver
Original file line number Diff line number Diff line change
@@ -0,0 +1,29 @@
FROM golang:1.25.1 AS builder

ENV GOPROXY=https://proxy.golang.org,direct
WORKDIR /historyserver

# Copy the go modules and manifests.
COPY go.mod go.mod
COPY go.sum go.sum
# Cache dependencies to avoid re-downloading when only sources change.
RUN go mod download

# Copy the go source.
COPY cmd/historyserver/main.go cmd/historyserver/main.go
# need collector because storage's interface is put in here, will change
# after this is merged
# https://github.com/ray-project/kuberay/pull/4302
COPY pkg/collector/ pkg/collector/
COPY pkg/historyserver/ pkg/historyserver/
COPY pkg/storage/ pkg/storage/
COPY pkg/utils/ pkg/utils/
COPY pkg/eventserver/ pkg/eventserver/

# Build the historyserver binary.
COPY Makefile Makefile
RUN make buildhistoryserver GOOS=linux GOARCH=amd64

FROM ubuntu:22.04
RUN apt-get update && apt-get upgrade -y && rm -rf /var/cache/apt/ && apt-get install -y ca-certificates
COPY --from=builder /historyserver/output/bin/historyserver /usr/local/bin/historyserver
12 changes: 8 additions & 4 deletions historyserver/Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@ BIN_DIR=$(OUT_DIR)/bin
BINARY_NAME=historyserver
BINARY_NAME_COLLECTOR=collector
COLLECTOR_IMG ?= collector:v0.1.0
HISTORYSERVER_IMG ?= historyserver:v0.1.0

# Setting SHELL to bash allows bash commands to be executed by recipes.
# Options are set to exit when a recipe line exits non-zero or a piped command fails.
Expand All @@ -18,7 +19,7 @@ COMMIT_SHORT ?= $(shell git rev-parse --short HEAD)
BRANCH ?= $(shell git branch --show-current)
VERSION ?= $(shell git describe --tags --long|awk -F '-' '{print $$1"."$$2"-"$$3""}')

PACKAGE = gitlab.alibaba-inc.com/eml/historyserver
PACKAGE = github.com/ray-project/kuberay/historyserver

GO_LDFLAGS := -extldflags "-static"
# GO_LDFLAGS += -w -s # Drop debugging symbols.
Expand Down Expand Up @@ -66,13 +67,16 @@ simplebuild:
mod:
go mod tidy

.PHONY: localimage-build
localimage-build: localimage-collector localimage-historyserver

.PHONY: localimage-collector
localimage-collector:
docker build -t $(COLLECTOR_IMG) -f Dockerfile.collector .

.PHONY: localimage
localimage: dockerbuilder_instance
docker buildx build -t historyserver:laster --platform linux/amd64 . --load
.PHONY: localimage-historyserver
localimage-historyserver: dockerbuilder_instance
docker build -t $(HISTORYSERVER_IMG) -f Dockerfile.historyserver .

.PHONY: dockerbuilder_instance
dockerbuilder_instance:
Expand Down
100 changes: 100 additions & 0 deletions historyserver/cmd/historyserver/main.go
Original file line number Diff line number Diff line change
@@ -1 +1,101 @@
package main

import (
"encoding/json"
"flag"
"os"
"os/signal"
"sync"
"syscall"

"github.com/ray-project/kuberay/historyserver/pkg/collector"
"github.com/ray-project/kuberay/historyserver/pkg/collector/types"
"github.com/ray-project/kuberay/historyserver/pkg/eventserver"
"github.com/ray-project/kuberay/historyserver/pkg/historyserver"
"github.com/sirupsen/logrus"
)

func main() {
runtimeClassName := ""
rayRootDir := ""
kubeconfigs := ""
runtimeClassConfigPath := "/var/collector-config/data"
dashboardDir := ""
flag.StringVar(&runtimeClassName, "runtime-class-name", "", "")
flag.StringVar(&rayRootDir, "ray-root-dir", "", "")
flag.StringVar(&kubeconfigs, "kubeconfigs", "", "")
flag.StringVar(&dashboardDir, "dashboard-dir", "/dashboard", "")
flag.StringVar(&runtimeClassConfigPath, "runtime-class-config-path", "", "") //"/var/collector-config/data"
flag.Parse()

cliMgr := historyserver.NewClientManager(kubeconfigs)

jsonData := make(map[string]interface{})
if runtimeClassConfigPath != "" {
data, err := os.ReadFile(runtimeClassConfigPath)
if err != nil {
panic("Failed to read runtime class config " + err.Error())
}
err = json.Unmarshal(data, &jsonData)
if err != nil {
panic("Failed to parse runtime class config: " + err.Error())
}
}

registry := collector.GetReaderRegistry()
factory, ok := registry[runtimeClassName]
if !ok {
panic("Not supported runtime class name: " + runtimeClassName + ".")
}

globalConfig := types.RayHistoryServerConfig{
RootDir: rayRootDir,
}

reader, err := factory(&globalConfig, jsonData)
if err != nil {
panic("Failed to create reader for runtime class name: " + runtimeClassName + ".")
}

// Create EventHandler with storage reader
eventHandler := eventserver.NewEventHandler(reader)

// WaitGroup to track goroutine completion
var wg sync.WaitGroup

// Start EventHandler in background goroutine
eventStop := make(chan struct{}, 1)
wg.Add(1)
go func() {
defer wg.Done()
logrus.Info("Starting EventHandler in background...")
if err := eventHandler.Run(eventStop, 2); err != nil {
logrus.Errorf("EventHandler stopped with error: %v", err)
}
logrus.Info("EventHandler shutdown complete")
}()

handler := historyserver.NewServerHandler(&globalConfig, dashboardDir, reader, cliMgr, eventHandler)

sigChan := make(chan os.Signal, 1)
stop := make(chan struct{}, 1)
signal.Notify(sigChan, syscall.SIGINT, syscall.SIGTERM)

wg.Add(1)
go func() {
defer wg.Done()
handler.Run(stop)
logrus.Info("HTTP server shutdown complete")
}()

<-sigChan
logrus.Info("Received shutdown signal, initiating graceful shutdown...")

// Stop both the server and the event handler
stop <- struct{}{}
eventStop <- struct{}{}

// Wait for both goroutines to complete
wg.Wait()
logrus.Info("Graceful shutdown complete")
}
63 changes: 63 additions & 0 deletions historyserver/config/historyserver.yaml
Original file line number Diff line number Diff line change
@@ -0,0 +1,63 @@
apiVersion: v1
kind: Service
metadata:
name: historyserver #TODO: to specify your service name
labels:
app: historyserver
spec:
selector:
app: historyserver
ports:
- protocol: TCP
name: http
port: 30080
targetPort: 8080
type: ClusterIP
---
apiVersion: apps/v1
kind: Deployment
metadata:
name: historyserver-demo
labels:
app: historyserver
spec:
replicas: 1
selector:
matchLabels:
app: historyserver
template:
metadata:
labels:
app: historyserver
spec:
serviceAccountName: historyserver
containers:
- name: historyserver
env:
- name: S3DISABLE_SSL
value: "true"
- name: AWS_S3ID
value: minioadmin
- name: AWS_S3SECRET
value: minioadmin
- name: AWS_S3TOKEN
value: ""
- name: S3_BUCKET
value: "ray-historyserver"
- name: S3_ENDPOINT
value: "minio-service.minio-dev:9000"
- name: S3_REGION
value: "test"
- name: S3FORCE_PATH_STYLE
value: "true"
image: historyserver:v0.1.0
imagePullPolicy: IfNotPresent
command:
- historyserver
- --runtime-class-name=s3
- --ray-root-dir=log
ports:
- containerPort: 8080
resources:
limits:
cpu: "500m"
4 changes: 2 additions & 2 deletions historyserver/config/rayjob.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -8,11 +8,11 @@ spec:
import ray
ray.init()

@ray.remote
@ray.remote(num_cpus=0.5)
def my_task(x):
return x * 2

@ray.remote
@ray.remote(num_cpus=0.5)
class Counter:
def __init__(self):
self.count = 0
Expand Down
27 changes: 27 additions & 0 deletions historyserver/config/service_account.yaml
Original file line number Diff line number Diff line change
@@ -0,0 +1,27 @@
apiVersion: v1
kind: ServiceAccount
metadata:
name: historyserver
automountServiceAccountToken: true
---
apiVersion: rbac.authorization.k8s.io/v1
kind: ClusterRole
metadata:
name: raycluster-reader
rules:
- apiGroups: ["ray.io"]
resources: ["rayclusters"]
verbs: ["list", "get"]
---
apiVersion: rbac.authorization.k8s.io/v1
kind: ClusterRoleBinding
metadata:
name: historyserver
namespace: default
subjects:
- kind: ServiceAccount
name: historyserver
namespace: default
roleRef:
kind: ClusterRole
name: raycluster-reader
Loading
Loading