Skip to content
Open
Show file tree
Hide file tree
Changes from 12 commits
Commits
Show all changes
26 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
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
10 changes: 7 additions & 3 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 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 . --load

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

import (
"encoding/json"
"flag"
"os"
"os/signal"
"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)

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

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, syscall.SIGKILL)
go handler.Run(stop)
<-sigChan
// Stop both the server and the event handler
stop <- struct{}{}
eventStop <- struct{}{}
}
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:
imagePullSecrets:
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"
30 changes: 30 additions & 0 deletions historyserver/config/service_account.yaml
Original file line number Diff line number Diff line change
@@ -0,0 +1,30 @@
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: default
namespace: default
- kind: ServiceAccount
name: historyserver
namespace: default
roleRef:
kind: ClusterRole
name: raycluster-reader
52 changes: 27 additions & 25 deletions historyserver/go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -9,15 +9,16 @@ require (
github.com/aws/aws-sdk-go v1.55.8
github.com/emicklei/go-restful/v3 v3.13.0
github.com/fsnotify/fsnotify v1.9.0
github.com/onsi/gomega v1.37.0
github.com/onsi/gomega v1.38.2
github.com/ray-project/kuberay/ray-operator v1.5.1
github.com/sirupsen/logrus v1.9.3
k8s.io/api v0.34.3
k8s.io/apimachinery v0.34.3
k8s.io/apimachinery v0.35.0
k8s.io/client-go v0.35.0
sigs.k8s.io/controller-runtime v0.22.4
)

require (
github.com/Masterminds/semver/v3 v3.3.1 // indirect
github.com/Masterminds/semver/v3 v3.4.0 // indirect
github.com/alibabacloud-go/debug v1.0.1 // indirect
github.com/beorn7/perks v1.0.1 // indirect
github.com/blang/semver/v4 v4.0.0 // indirect
Expand All @@ -33,60 +34,61 @@ require (
github.com/gogo/protobuf v1.3.2 // indirect
github.com/google/btree v1.1.3 // indirect
github.com/google/gnostic-models v0.7.0 // indirect
github.com/google/go-cmp v0.7.0 // indirect
github.com/google/go-cmp v0.7.0
github.com/google/pprof v0.0.0-20250403155104-27863c87afa6 // indirect
github.com/google/uuid v1.6.0 // indirect
github.com/gorilla/websocket v1.5.4-0.20250319132907-e064f32e3674 // indirect
github.com/jmespath/go-jmespath v0.4.0 // indirect
github.com/josharian/intern v1.0.0 // indirect
github.com/json-iterator/go v1.1.12 // indirect
github.com/mailru/easyjson v0.9.0 // indirect
github.com/moby/spdystream v0.5.0 // indirect
github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect
github.com/modern-go/reflect2 v1.0.3-0.20250322232337-35a7c28c31ee // indirect
github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 // indirect
github.com/mxk/go-flowrate v0.0.0-20140419014527-cca7078d478f // indirect
github.com/onsi/ginkgo/v2 v2.23.4 // indirect
github.com/onsi/ginkgo/v2 v2.27.2 // indirect
github.com/openshift/api v0.0.0-20250602203052-b29811a290c7 // indirect
github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect
github.com/prometheus/client_golang v1.23.0 // indirect
github.com/prometheus/client_model v0.6.2 // indirect
github.com/prometheus/common v0.65.0 // indirect
github.com/prometheus/procfs v0.17.0 // indirect
github.com/rogpeppe/go-internal v1.14.1 // indirect
github.com/spf13/pflag v1.0.7 // indirect
github.com/stretchr/testify v1.11.1 // indirect
github.com/spf13/pflag v1.0.9 // indirect
github.com/x448/float16 v0.8.4 // indirect
go.opentelemetry.io/otel v1.35.0 // indirect
go.opentelemetry.io/otel/trace v1.35.0 // indirect
go.uber.org/automaxprocs v1.6.0 // indirect
go.yaml.in/yaml/v2 v2.4.2 // indirect
go.yaml.in/yaml/v2 v2.4.3 // indirect
go.yaml.in/yaml/v3 v3.0.4 // indirect
golang.org/x/net v0.46.0 // indirect
golang.org/x/net v0.47.0 // indirect
golang.org/x/oauth2 v0.30.0 // indirect
golang.org/x/sync v0.17.0 // indirect
golang.org/x/sys v0.37.0 // indirect
golang.org/x/term v0.36.0 // indirect
golang.org/x/text v0.30.0 // indirect
golang.org/x/sync v0.18.0 // indirect
golang.org/x/sys v0.38.0 // indirect
golang.org/x/term v0.37.0 // indirect
golang.org/x/text v0.31.0 // indirect
golang.org/x/time v0.14.0 // indirect
golang.org/x/tools v0.37.0 // indirect
gomodules.xyz/jsonpatch/v2 v2.4.0 // indirect
google.golang.org/protobuf v1.36.8 // indirect
gopkg.in/evanphx/json-patch.v4 v4.13.0 // indirect
gopkg.in/inf.v0 v0.9.1 // indirect
gopkg.in/ini.v1 v1.67.0 // indirect
gopkg.in/yaml.v3 v3.0.1 // indirect
k8s.io/api v0.35.0
k8s.io/apiextensions-apiserver v0.34.1 // indirect
k8s.io/apiserver v0.34.1 // indirect
k8s.io/client-go v0.34.3 // indirect
k8s.io/component-base v0.34.1 // indirect
k8s.io/klog/v2 v2.130.1 // indirect
k8s.io/kube-openapi v0.0.0-20250814151709-d7b6acb124c3 // indirect
k8s.io/utils v0.0.0-20250820121507-0af2bda4dd1d // indirect
sigs.k8s.io/controller-runtime v0.22.1 // indirect
k8s.io/kube-openapi v0.0.0-20250910181357-589584f1c912 // indirect
k8s.io/utils v0.0.0-20251002143259-bc988d571ff4 // indirect
sigs.k8s.io/gateway-api v1.4.0 // indirect
sigs.k8s.io/json v0.0.0-20250730193827-2d320260d730 // indirect
sigs.k8s.io/randfill v1.0.0 // indirect
sigs.k8s.io/structured-merge-diff/v6 v6.3.0 // indirect
sigs.k8s.io/yaml v1.6.0 // indirect
)

require (
github.com/moby/spdystream v0.5.0 // indirect
github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect
github.com/stretchr/testify v1.11.1 // indirect
golang.org/x/mod v0.29.0 // indirect
golang.org/x/tools v0.38.0 // indirect
gomodules.xyz/jsonpatch/v2 v2.4.0 // indirect
gopkg.in/evanphx/json-patch.v4 v4.13.0 // indirect
)
Loading
Loading