From 805369a6018207febb99d7d52c0b34245ab566bb Mon Sep 17 00:00:00 2001 From: fscnick Date: Tue, 28 Oct 2025 15:36:14 +0800 Subject: [PATCH 01/54] [RayJob] background job info poc --- .../ray_job_submission_service_server.go | 2 +- .../config/v1alpha1/configuration_types.go | 6 +- .../controllers/ray/rayjob_controller.go | 5 + .../dashboardclient/dashboard_cache_client.go | 151 ++++++++++++++++++ .../dashboardclient/dashboard_httpclient.go | 3 +- ray-operator/controllers/ray/utils/util.go | 13 +- ray-operator/test/sampleyaml/support.go | 2 +- 7 files changed, 177 insertions(+), 5 deletions(-) create mode 100644 ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go diff --git a/apiserver/pkg/server/ray_job_submission_service_server.go b/apiserver/pkg/server/ray_job_submission_service_server.go index 4fdead1e50f..596b991821e 100644 --- a/apiserver/pkg/server/ray_job_submission_service_server.go +++ b/apiserver/pkg/server/ray_job_submission_service_server.go @@ -41,7 +41,7 @@ type RayJobSubmissionServiceServer struct { // Create RayJobSubmissionServiceServer func NewRayJobSubmissionServiceServer(clusterServer *ClusterServer, options *RayJobSubmissionServiceServerOptions) *RayJobSubmissionServiceServer { zl := zerolog.New(os.Stdout).Level(zerolog.DebugLevel) - return &RayJobSubmissionServiceServer{clusterServer: clusterServer, options: options, log: zerologr.New(&zl).WithName("jobsubmissionservice"), dashboardClientFunc: utils.GetRayDashboardClientFunc(nil, false)} + return &RayJobSubmissionServiceServer{clusterServer: clusterServer, options: options, log: zerologr.New(&zl).WithName("jobsubmissionservice"), dashboardClientFunc: utils.GetRayDashboardClientFunc(nil, false, false)} } // Submit Ray job diff --git a/ray-operator/apis/config/v1alpha1/configuration_types.go b/ray-operator/apis/config/v1alpha1/configuration_types.go index 910ec0d11ab..1abb7dedab4 100644 --- a/ray-operator/apis/config/v1alpha1/configuration_types.go +++ b/ray-operator/apis/config/v1alpha1/configuration_types.go @@ -86,10 +86,14 @@ type Configuration struct { // EnableMetrics indicates whether KubeRay operator should emit control plane metrics. EnableMetrics bool `json:"enableMetrics,omitempty"` + + // UseBackgroundGoroutine indicates that it wil use goroutine to fetch the job info from ray dashboard and + // store the job info in the cache + UseBackgroundGoroutine bool `json:"useBackgroundGoroutine,omitempty"` } func (config Configuration) GetDashboardClient(mgr manager.Manager) func(rayCluster *rayv1.RayCluster, url string) (dashboardclient.RayDashboardClientInterface, error) { - return utils.GetRayDashboardClientFunc(mgr, config.UseKubernetesProxy) + return utils.GetRayDashboardClientFunc(mgr, config.UseKubernetesProxy, config.UseBackgroundGoroutine) } func (config Configuration) GetHttpProxyClient(mgr manager.Manager) func(hostIp, podNamespace, podName string, port int) utils.RayHttpProxyClientInterface { diff --git a/ray-operator/controllers/ray/rayjob_controller.go b/ray-operator/controllers/ray/rayjob_controller.go index 5da57de6491..0cbcf45c444 100644 --- a/ray-operator/controllers/ray/rayjob_controller.go +++ b/ray-operator/controllers/ray/rayjob_controller.go @@ -2,6 +2,7 @@ package ray import ( "context" + errs "errors" "fmt" "os" "strconv" @@ -284,6 +285,10 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) jobInfo, err := rayDashboardClient.GetJobInfo(ctx, rayJobInstance.Status.JobId) if err != nil { + if errs.Is(err, dashboardclient.ErrAgain) { + logger.Info("The Ray job Info was not ready. Try again next iteration.", "JobId", rayJobInstance.Status.JobId) + return ctrl.Result{RequeueAfter: RayJobDefaultRequeueDuration}, nil + } // If the Ray job was not found, GetJobInfo returns a BadRequest error. if errors.IsBadRequest(err) { if rayJobInstance.Spec.SubmissionMode == rayv1.HTTPMode { diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go new file mode 100644 index 00000000000..40e91d49a82 --- /dev/null +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -0,0 +1,151 @@ +package dashboardclient + +import ( + "context" + "errors" + "sync" + "time" + + cmap "github.com/orcaman/concurrent-map/v2" + + rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1" + utiltypes "github.com/ray-project/kuberay/ray-operator/controllers/ray/utils/types" +) + +var ErrAgain = errors.New("EAGAIN") + +var ( + initWorkPool sync.Once + taskQueue chan Task + + // TODO: make queue size and worker size configurable. + taskQueueSize = 128 + workerSize = 8 + + cacheStorage *cmap.ConcurrentMap[string, *JobInfoCache] + + queryInterval = 3 * time.Second // TODO: make it configurable +) + +type ( + Task func() bool + JobInfoCache struct { + JobInfo *utiltypes.RayJobInfo + Err error + UpdateAt *time.Time + } +) + +var _ RayDashboardClientInterface = (*RayDashboardCacheClient)(nil) + +type RayDashboardCacheClient struct { + client RayDashboardClientInterface +} + +func (r *RayDashboardCacheClient) InitClient(client RayDashboardClientInterface) { + initWorkPool.Do(func() { + if taskQueue == nil { + taskQueue = make(chan Task, taskQueueSize) + + // TODO: should we have observability for these goroutine? + for i := 0; i < workerSize; i++ { + // TODO: should we consider the stop ? + go func() { + for task := range taskQueue { + again := task() + + if again { + time.AfterFunc(queryInterval, func() { + taskQueue <- task + }) + } + } + }() + } + } + + if cacheStorage == nil { + tmp := cmap.New[*JobInfoCache]() + cacheStorage = &tmp + } + }) + + r.client = client +} + +func (r *RayDashboardCacheClient) UpdateDeployments(ctx context.Context, configJson []byte) error { + // TODO implement me + panic("implement me") +} + +func (r *RayDashboardCacheClient) GetServeDetails(ctx context.Context) (*utiltypes.ServeDetails, error) { + // TODO implement me + panic("implement me") +} + +func (r *RayDashboardCacheClient) GetMultiApplicationStatus(ctx context.Context) (map[string]*utiltypes.ServeApplicationStatus, error) { + // TODO implement me + panic("implement me") +} + +func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) (*utiltypes.RayJobInfo, error) { + if cached, ok := cacheStorage.Get(jobId); ok { + return cached.JobInfo, cached.Err + } + cached := &JobInfoCache{Err: ErrAgain} + cacheStorage.Set(jobId, cached) + + // send to worker pool + task := func() bool { + jobInfoCache, ok := cacheStorage.Get(jobId) + if !ok { + // TODO: this should not happen + } + jobInfoCache.JobInfo, jobInfoCache.Err = r.client.GetJobInfo(ctx, jobId) + currentTime := time.Now() + jobInfoCache.UpdateAt = ¤tTime + + cacheStorage.Set(jobId, jobInfoCache) + // handle not found(ex: rayjob has deleted) + + if rayv1.IsJobTerminal(jobInfoCache.JobInfo.JobStatus) { + return false + } + + return true + } + + taskQueue <- task + + return nil, ErrAgain +} + +func (r *RayDashboardCacheClient) ListJobs(ctx context.Context) (*[]utiltypes.RayJobInfo, error) { + // TODO implement me + panic("implement me") +} + +func (r *RayDashboardCacheClient) SubmitJob(ctx context.Context, rayJob *rayv1.RayJob) (string, error) { + // TODO implement me + panic("implement me") +} + +func (r *RayDashboardCacheClient) SubmitJobReq(ctx context.Context, request *utiltypes.RayJobRequest) (string, error) { + // TODO implement me + panic("implement me") +} + +func (r *RayDashboardCacheClient) GetJobLog(ctx context.Context, jobName string) (*string, error) { + // TODO implement me + panic("implement me") +} + +func (r *RayDashboardCacheClient) StopJob(ctx context.Context, jobName string) error { + // TODO implement me + panic("implement me") +} + +func (r *RayDashboardCacheClient) DeleteJob(ctx context.Context, jobName string) error { + // TODO implement me + panic("implement me") +} diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_httpclient.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_httpclient.go index d360ebc0af9..98bd8da369c 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_httpclient.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_httpclient.go @@ -27,7 +27,8 @@ var ( ) type RayDashboardClientInterface interface { - InitClient(client *http.Client, dashboardURL string) + // Remove InitClient for adapting variable implementation + // InitClient(client *http.Client, dashboardURL string) UpdateDeployments(ctx context.Context, configJson []byte) error // V2/multi-app Rest API GetServeDetails(ctx context.Context) (*utiltypes.ServeDetails, error) diff --git a/ray-operator/controllers/ray/utils/util.go b/ray-operator/controllers/ray/utils/util.go index 540162bf9f4..9bc463de718 100644 --- a/ray-operator/controllers/ray/utils/util.go +++ b/ray-operator/controllers/ray/utils/util.go @@ -877,7 +877,7 @@ func FetchHeadServiceURL(ctx context.Context, cli client.Client, rayCluster *ray return headServiceURL, nil } -func GetRayDashboardClientFunc(mgr manager.Manager, useKubernetesProxy bool) func(rayCluster *rayv1.RayCluster, url string) (dashboardclient.RayDashboardClientInterface, error) { +func GetRayDashboardClientFunc(mgr manager.Manager, useKubernetesProxy bool, useBackgroundGoroutine bool) func(rayCluster *rayv1.RayCluster, url string) (dashboardclient.RayDashboardClientInterface, error) { return func(rayCluster *rayv1.RayCluster, url string) (dashboardclient.RayDashboardClientInterface, error) { dashboardClient := &dashboardclient.RayDashboardClient{} if useKubernetesProxy { @@ -897,12 +897,23 @@ func GetRayDashboardClientFunc(mgr manager.Manager, useKubernetesProxy bool) fun mgr.GetHTTPClient(), fmt.Sprintf("%s/api/v1/namespaces/%s/services/%s:dashboard/proxy", mgr.GetConfig().Host, rayCluster.Namespace, headSvcName), ) + if useBackgroundGoroutine { + dashboardCachedClient := &dashboardclient.RayDashboardCacheClient{} + dashboardCachedClient.InitClient(dashboardClient) + return dashboardCachedClient, nil + } return dashboardClient, nil } dashboardClient.InitClient(&http.Client{ Timeout: 2 * time.Second, }, "http://"+url) + + if useBackgroundGoroutine { + dashboardCachedClient := &dashboardclient.RayDashboardCacheClient{} + dashboardCachedClient.InitClient(dashboardClient) + return dashboardCachedClient, nil + } return dashboardClient, nil } } diff --git a/ray-operator/test/sampleyaml/support.go b/ray-operator/test/sampleyaml/support.go index 42ffd19e0a2..f470295001b 100644 --- a/ray-operator/test/sampleyaml/support.go +++ b/ray-operator/test/sampleyaml/support.go @@ -75,7 +75,7 @@ func QueryDashboardGetAppStatus(t Test, rayCluster *rayv1.RayCluster) func(Gomeg g.Expect(err).ToNot(HaveOccurred()) url := fmt.Sprintf("127.0.0.1:%d", localPort) - rayDashboardClientFunc := utils.GetRayDashboardClientFunc(nil, false) + rayDashboardClientFunc := utils.GetRayDashboardClientFunc(nil, false, false) rayDashboardClient, err := rayDashboardClientFunc(rayCluster, url) g.Expect(err).ToNot(HaveOccurred()) serveDetails, err := rayDashboardClient.GetServeDetails(t.Ctx()) From 73b14b5c0b794c873236a42b21b0fd069c44d07c Mon Sep 17 00:00:00 2001 From: fscnick Date: Tue, 28 Oct 2025 16:52:34 +0800 Subject: [PATCH 02/54] [RayJob] add implement some methods --- .../dashboardclient/dashboard_cache_client.go | 40 ++++++------------- 1 file changed, 13 insertions(+), 27 deletions(-) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index 40e91d49a82..f76c887beec 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -74,18 +74,15 @@ func (r *RayDashboardCacheClient) InitClient(client RayDashboardClientInterface) } func (r *RayDashboardCacheClient) UpdateDeployments(ctx context.Context, configJson []byte) error { - // TODO implement me - panic("implement me") + return r.client.UpdateDeployments(ctx, configJson) } func (r *RayDashboardCacheClient) GetServeDetails(ctx context.Context) (*utiltypes.ServeDetails, error) { - // TODO implement me - panic("implement me") + return r.client.GetServeDetails(ctx) } func (r *RayDashboardCacheClient) GetMultiApplicationStatus(ctx context.Context) (map[string]*utiltypes.ServeApplicationStatus, error) { - // TODO implement me - panic("implement me") + return r.client.GetMultiApplicationStatus(ctx) } func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) (*utiltypes.RayJobInfo, error) { @@ -97,10 +94,9 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) // send to worker pool task := func() bool { - jobInfoCache, ok := cacheStorage.Get(jobId) - if !ok { - // TODO: this should not happen - } + jobInfoCache, _ := cacheStorage.Get(jobId) + // TODO: should we handle cache not exist here, which it shouldn't happen + jobInfoCache.JobInfo, jobInfoCache.Err = r.client.GetJobInfo(ctx, jobId) currentTime := time.Now() jobInfoCache.UpdateAt = ¤tTime @@ -108,11 +104,7 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) cacheStorage.Set(jobId, jobInfoCache) // handle not found(ex: rayjob has deleted) - if rayv1.IsJobTerminal(jobInfoCache.JobInfo.JobStatus) { - return false - } - - return true + return !rayv1.IsJobTerminal(jobInfoCache.JobInfo.JobStatus) } taskQueue <- task @@ -121,31 +113,25 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) } func (r *RayDashboardCacheClient) ListJobs(ctx context.Context) (*[]utiltypes.RayJobInfo, error) { - // TODO implement me - panic("implement me") + return r.client.ListJobs(ctx) } func (r *RayDashboardCacheClient) SubmitJob(ctx context.Context, rayJob *rayv1.RayJob) (string, error) { - // TODO implement me - panic("implement me") + return r.client.SubmitJob(ctx, rayJob) } func (r *RayDashboardCacheClient) SubmitJobReq(ctx context.Context, request *utiltypes.RayJobRequest) (string, error) { - // TODO implement me - panic("implement me") + return r.client.SubmitJobReq(ctx, request) } func (r *RayDashboardCacheClient) GetJobLog(ctx context.Context, jobName string) (*string, error) { - // TODO implement me - panic("implement me") + return r.client.GetJobLog(ctx, jobName) } func (r *RayDashboardCacheClient) StopJob(ctx context.Context, jobName string) error { - // TODO implement me - panic("implement me") + return r.client.StopJob(ctx, jobName) } func (r *RayDashboardCacheClient) DeleteJob(ctx context.Context, jobName string) error { - // TODO implement me - panic("implement me") + return r.client.DeleteJob(ctx, jobName) } From 4ce23811cb3432898b81e84a146418eca4493785 Mon Sep 17 00:00:00 2001 From: fscnick Date: Wed, 29 Oct 2025 14:27:12 +0800 Subject: [PATCH 03/54] [RayJob] encapsulate the worker pool --- .../dashboardclient/dashboard_cache_client.go | 69 +++++++++++-------- 1 file changed, 42 insertions(+), 27 deletions(-) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index f76c887beec..6cf0f351eb7 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -14,17 +14,20 @@ import ( var ErrAgain = errors.New("EAGAIN") -var ( - initWorkPool sync.Once - taskQueue chan Task - +const ( // TODO: make queue size and worker size configurable. taskQueueSize = 128 workerSize = 8 - cacheStorage *cmap.ConcurrentMap[string, *JobInfoCache] + queryInterval = 3 * time.Second +) - queryInterval = 3 * time.Second // TODO: make it configurable +var ( + initWorkPool sync.Once + pool workerPool + + initCacheStorage sync.Once + cacheStorage *cmap.ConcurrentMap[string, *JobInfoCache] ) type ( @@ -34,8 +37,36 @@ type ( Err error UpdateAt *time.Time } + + workerPool struct { + taskQueue chan Task + } ) +func (w *workerPool) init(taskQueueSize int, workerSize int, queryInterval time.Duration) { + w.taskQueue = make(chan Task, taskQueueSize) + + // TODO: should we have observability for these goroutine? + for i := 0; i < workerSize; i++ { + // TODO: should we consider the stop ? + go func() { + for task := range w.taskQueue { + again := task() + + if again { + time.AfterFunc(queryInterval, func() { + w.taskQueue <- task + }) + } + } + }() + } +} + +func (w *workerPool) PutTask(task Task) { + w.taskQueue <- task +} + var _ RayDashboardClientInterface = (*RayDashboardCacheClient)(nil) type RayDashboardCacheClient struct { @@ -44,26 +75,10 @@ type RayDashboardCacheClient struct { func (r *RayDashboardCacheClient) InitClient(client RayDashboardClientInterface) { initWorkPool.Do(func() { - if taskQueue == nil { - taskQueue = make(chan Task, taskQueueSize) - - // TODO: should we have observability for these goroutine? - for i := 0; i < workerSize; i++ { - // TODO: should we consider the stop ? - go func() { - for task := range taskQueue { - again := task() - - if again { - time.AfterFunc(queryInterval, func() { - taskQueue <- task - }) - } - } - }() - } - } + pool.init(taskQueueSize, workerSize, queryInterval) + }) + initCacheStorage.Do(func() { if cacheStorage == nil { tmp := cmap.New[*JobInfoCache]() cacheStorage = &tmp @@ -90,7 +105,7 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) return cached.JobInfo, cached.Err } cached := &JobInfoCache{Err: ErrAgain} - cacheStorage.Set(jobId, cached) + cacheStorage.SetIfAbsent(jobId, cached) // send to worker pool task := func() bool { @@ -107,7 +122,7 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) return !rayv1.IsJobTerminal(jobInfoCache.JobInfo.JobStatus) } - taskQueue <- task + pool.PutTask(task) return nil, ErrAgain } From e184e5c6a6fc647f9ab8c769f5e5f06d47b4ad13 Mon Sep 17 00:00:00 2001 From: fscnick Date: Wed, 29 Oct 2025 15:58:32 +0800 Subject: [PATCH 04/54] [RayJob] replace concurrency map with lru cache --- go.mod | 1 + go.sum | 2 ++ .../dashboardclient/dashboard_cache_client.go | 17 +++++++++++------ ray-operator/go.mod | 1 + ray-operator/go.sum | 2 ++ 5 files changed, 17 insertions(+), 6 deletions(-) diff --git a/go.mod b/go.mod index d5dab3bca19..1ce8746cd50 100644 --- a/go.mod +++ b/go.mod @@ -67,6 +67,7 @@ require ( github.com/google/uuid v1.6.0 // indirect github.com/gorilla/websocket v1.5.4-0.20250319132907-e064f32e3674 // indirect github.com/gregjones/httpcache v0.0.0-20190611155906-901d90724c79 // indirect + github.com/hashicorp/golang-lru/v2 v2.0.7 // indirect github.com/inconshreveable/mousetrap v1.1.0 // indirect github.com/josharian/intern v1.0.0 // indirect github.com/json-iterator/go v1.1.12 // indirect diff --git a/go.sum b/go.sum index afff157cfd8..2eb06da7d9e 100644 --- a/go.sum +++ b/go.sum @@ -107,6 +107,8 @@ github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 h1:Ovs26xHkKqVztRpIrF/92Bcuy github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk= github.com/grpc-ecosystem/grpc-gateway/v2 v2.26.3 h1:5ZPtiqj0JL5oKWmcsq4VMaAW5ukBEgSGXEN89zeH1Jo= github.com/grpc-ecosystem/grpc-gateway/v2 v2.26.3/go.mod h1:ndYquD05frm2vACXE1nsccT4oJzjhw2arTS2cpUD1PI= +github.com/hashicorp/golang-lru/v2 v2.0.7 h1:a+bsQ5rvGLjzHuww6tVxozPZFVghXaHOwFs4luLUK2k= +github.com/hashicorp/golang-lru/v2 v2.0.7/go.mod h1:QeFd9opnmA6QUJc5vARoKUSoFhyfM2/ZepoAG6RGpeM= github.com/inconshreveable/mousetrap v1.1.0 h1:wN+x4NVGpMsO7ErUn/mUI3vEoE6Jt13X2s0bqwp9tc8= github.com/inconshreveable/mousetrap v1.1.0/go.mod h1:vpF70FUmC8bwa3OWnCshd2FqLfsEA9PFc4w1p2J65bw= github.com/jarcoal/httpmock v1.4.0 h1:BvhqnH0JAYbNudL2GMJKgOHe2CtKlzJ/5rWKyp+hc2k= diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index 6cf0f351eb7..c3ee6a906cd 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -6,7 +6,7 @@ import ( "sync" "time" - cmap "github.com/orcaman/concurrent-map/v2" + lru "github.com/hashicorp/golang-lru/v2" rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1" utiltypes "github.com/ray-project/kuberay/ray-operator/controllers/ray/utils/types" @@ -20,6 +20,9 @@ const ( workerSize = 8 queryInterval = 3 * time.Second + + cacheSize = 10000 + cacheExpiry = 10 * time.Minute ) var ( @@ -27,7 +30,7 @@ var ( pool workerPool initCacheStorage sync.Once - cacheStorage *cmap.ConcurrentMap[string, *JobInfoCache] + cacheStorage *lru.Cache[string, *JobInfoCache] ) type ( @@ -80,8 +83,8 @@ func (r *RayDashboardCacheClient) InitClient(client RayDashboardClientInterface) initCacheStorage.Do(func() { if cacheStorage == nil { - tmp := cmap.New[*JobInfoCache]() - cacheStorage = &tmp + // the New() returns error only if the size is less or equal than zero. + cacheStorage, _ = lru.New[string, *JobInfoCache](cacheSize) } }) @@ -105,7 +108,9 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) return cached.JobInfo, cached.Err } cached := &JobInfoCache{Err: ErrAgain} - cacheStorage.SetIfAbsent(jobId, cached) + if cached, existed, _ := cacheStorage.PeekOrAdd(jobId, cached); existed { + return cached.JobInfo, cached.Err + } // send to worker pool task := func() bool { @@ -116,7 +121,7 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) currentTime := time.Now() jobInfoCache.UpdateAt = ¤tTime - cacheStorage.Set(jobId, jobInfoCache) + cacheStorage.Add(jobId, jobInfoCache) // handle not found(ex: rayjob has deleted) return !rayv1.IsJobTerminal(jobInfoCache.JobInfo.JobStatus) diff --git a/ray-operator/go.mod b/ray-operator/go.mod index 907c9fd9e8b..cbe6d63098b 100644 --- a/ray-operator/go.mod +++ b/ray-operator/go.mod @@ -7,6 +7,7 @@ require ( github.com/coder/websocket v1.8.13 github.com/go-logr/logr v1.4.3 github.com/go-logr/zapr v1.3.0 + github.com/hashicorp/golang-lru/v2 v2.0.7 github.com/jarcoal/httpmock v1.4.0 github.com/onsi/ginkgo/v2 v2.23.4 github.com/onsi/gomega v1.37.0 diff --git a/ray-operator/go.sum b/ray-operator/go.sum index 26b72b129ae..d9a20efaf4e 100644 --- a/ray-operator/go.sum +++ b/ray-operator/go.sum @@ -53,6 +53,8 @@ github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/gorilla/websocket v1.5.4-0.20250319132907-e064f32e3674 h1:JeSE6pjso5THxAzdVpqr6/geYxZytqFMBCOtn/ujyeo= github.com/gorilla/websocket v1.5.4-0.20250319132907-e064f32e3674/go.mod h1:r4w70xmWCQKmi1ONH4KIaBptdivuRPyosB9RmPlGEwA= +github.com/hashicorp/golang-lru/v2 v2.0.7 h1:a+bsQ5rvGLjzHuww6tVxozPZFVghXaHOwFs4luLUK2k= +github.com/hashicorp/golang-lru/v2 v2.0.7/go.mod h1:QeFd9opnmA6QUJc5vARoKUSoFhyfM2/ZepoAG6RGpeM= github.com/jarcoal/httpmock v1.4.0 h1:BvhqnH0JAYbNudL2GMJKgOHe2CtKlzJ/5rWKyp+hc2k= github.com/jarcoal/httpmock v1.4.0/go.mod h1:ftW1xULwo+j0R0JJkJIIi7UKigZUXCLLanykgjwBXL0= github.com/josharian/intern v1.0.0 h1:vlS4z54oSdjm0bgjRigI+G1HpF+tI+9rE5LLzOg8HmY= From 859f6a1c9c7acf7409e2092a43f8753df22f6d7d Mon Sep 17 00:00:00 2001 From: fscnick Date: Thu, 30 Oct 2025 15:29:50 +0800 Subject: [PATCH 05/54] [RayJob] remove cache on stop and config flag --- .../config/v1alpha1/configuration_types.go | 4 +++ .../controllers/ray/rayjob_controller.go | 31 +++++++++++++------ ray-operator/controllers/ray/suite_test.go | 4 +++ .../dashboardclient/dashboard_cache_client.go | 12 +++++-- ray-operator/controllers/ray/utils/util.go | 1 + ray-operator/main.go | 3 ++ 6 files changed, 42 insertions(+), 13 deletions(-) diff --git a/ray-operator/apis/config/v1alpha1/configuration_types.go b/ray-operator/apis/config/v1alpha1/configuration_types.go index 1abb7dedab4..55e1269c6a6 100644 --- a/ray-operator/apis/config/v1alpha1/configuration_types.go +++ b/ray-operator/apis/config/v1alpha1/configuration_types.go @@ -99,3 +99,7 @@ func (config Configuration) GetDashboardClient(mgr manager.Manager) func(rayClus func (config Configuration) GetHttpProxyClient(mgr manager.Manager) func(hostIp, podNamespace, podName string, port int) utils.RayHttpProxyClientInterface { return utils.GetRayHttpProxyClientFunc(mgr, config.UseKubernetesProxy) } + +func (config Configuration) DoesUseBackgroundGoroutine() bool { + return config.UseBackgroundGoroutine +} diff --git a/ray-operator/controllers/ray/rayjob_controller.go b/ray-operator/controllers/ray/rayjob_controller.go index 0cbcf45c444..2cd24da0afa 100644 --- a/ray-operator/controllers/ray/rayjob_controller.go +++ b/ray-operator/controllers/ray/rayjob_controller.go @@ -43,11 +43,11 @@ const ( // RayJobReconciler reconciles a RayJob object type RayJobReconciler struct { client.Client - Scheme *runtime.Scheme - Recorder record.EventRecorder - - dashboardClientFunc func(rayCluster *rayv1.RayCluster, url string) (dashboardclient.RayDashboardClientInterface, error) - options RayJobReconcilerOptions + Recorder record.EventRecorder + options RayJobReconcilerOptions + Scheme *runtime.Scheme + dashboardClientFunc func(rayCluster *rayv1.RayCluster, url string) (dashboardclient.RayDashboardClientInterface, error) + useBackgroundGoroutine bool } type RayJobReconcilerOptions struct { @@ -59,11 +59,12 @@ type RayJobReconcilerOptions struct { func NewRayJobReconciler(_ context.Context, mgr manager.Manager, options RayJobReconcilerOptions, provider utils.ClientProvider) *RayJobReconciler { dashboardClientFunc := provider.GetDashboardClient(mgr) return &RayJobReconciler{ - Client: mgr.GetClient(), - Scheme: mgr.GetScheme(), - Recorder: mgr.GetEventRecorderFor("rayjob-controller"), - dashboardClientFunc: dashboardClientFunc, - options: options, + Client: mgr.GetClient(), + Scheme: mgr.GetScheme(), + Recorder: mgr.GetEventRecorderFor("rayjob-controller"), + dashboardClientFunc: dashboardClientFunc, + useBackgroundGoroutine: provider.DoesUseBackgroundGoroutine(), + options: options, } } @@ -758,6 +759,16 @@ func (r *RayJobReconciler) deleteClusterResources(ctx context.Context, rayJobIns if !cluster.DeletionTimestamp.IsZero() { logger.Info("The deletion of the associated RayCluster for RayJob is ongoing.", "RayCluster", cluster.Name) } else { + if r.useBackgroundGoroutine { + // clear cache, and it will remove this job from updating loop. + rayDashboardClient, err := r.dashboardClientFunc(&cluster, rayJobInstance.Status.DashboardURL) + if err != nil { + logger.Error(err, "Failed to get dashboard client for RayJob") + } + if err := rayDashboardClient.StopJob(ctx, rayJobInstance.Status.JobId); err != nil { + logger.Error(err, "Failed to stop job for RayJob") + } + } if err := r.Delete(ctx, &cluster); err != nil { r.Recorder.Eventf(rayJobInstance, corev1.EventTypeWarning, string(utils.FailedToDeleteRayCluster), "Failed to delete cluster %s/%s: %v", cluster.Namespace, cluster.Name, err) return false, err diff --git a/ray-operator/controllers/ray/suite_test.go b/ray-operator/controllers/ray/suite_test.go index 85c913e7bd6..e2ab5e16196 100644 --- a/ray-operator/controllers/ray/suite_test.go +++ b/ray-operator/controllers/ray/suite_test.go @@ -64,6 +64,10 @@ func (testProvider TestClientProvider) GetHttpProxyClient(_ manager.Manager) fun } } +func (testProvider TestClientProvider) DoesUseBackgroundGoroutine() bool { + return false +} + func TestAPIs(t *testing.T) { RegisterFailHandler(Fail) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index c3ee6a906cd..e1616649f45 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -21,6 +21,7 @@ const ( queryInterval = 3 * time.Second + // TODO: consider a proper size for accommodating the all live job info cacheSize = 10000 cacheExpiry = 10 * time.Minute ) @@ -114,16 +115,19 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) // send to worker pool task := func() bool { - jobInfoCache, _ := cacheStorage.Get(jobId) - // TODO: should we handle cache not exist here, which it shouldn't happen + jobInfoCache, existed := cacheStorage.Get(jobId) + if !existed { + return false + } jobInfoCache.JobInfo, jobInfoCache.Err = r.client.GetJobInfo(ctx, jobId) currentTime := time.Now() jobInfoCache.UpdateAt = ¤tTime - cacheStorage.Add(jobId, jobInfoCache) // handle not found(ex: rayjob has deleted) + cacheStorage.Add(jobId, jobInfoCache) + return !rayv1.IsJobTerminal(jobInfoCache.JobInfo.JobStatus) } @@ -149,9 +153,11 @@ func (r *RayDashboardCacheClient) GetJobLog(ctx context.Context, jobName string) } func (r *RayDashboardCacheClient) StopJob(ctx context.Context, jobName string) error { + cacheStorage.Remove(jobName) return r.client.StopJob(ctx, jobName) } func (r *RayDashboardCacheClient) DeleteJob(ctx context.Context, jobName string) error { + cacheStorage.Remove(jobName) return r.client.DeleteJob(ctx, jobName) } diff --git a/ray-operator/controllers/ray/utils/util.go b/ray-operator/controllers/ray/utils/util.go index 9bc463de718..31a93664b64 100644 --- a/ray-operator/controllers/ray/utils/util.go +++ b/ray-operator/controllers/ray/utils/util.go @@ -653,6 +653,7 @@ func EnvVarByName(envName string, envVars []corev1.EnvVar) (corev1.EnvVar, bool) type ClientProvider interface { GetDashboardClient(mgr manager.Manager) func(rayCluster *rayv1.RayCluster, url string) (dashboardclient.RayDashboardClientInterface, error) GetHttpProxyClient(mgr manager.Manager) func(hostIp, podNamespace, podName string, port int) RayHttpProxyClientInterface + DoesUseBackgroundGoroutine() bool } func ManagedByExternalController(controllerName *string) *string { diff --git a/ray-operator/main.go b/ray-operator/main.go index ceba7d4772e..feb675e717f 100644 --- a/ray-operator/main.go +++ b/ray-operator/main.go @@ -73,6 +73,7 @@ func main() { var enableMetrics bool var qps float64 var burst int + var useBackgroundGoroutine bool // TODO: remove flag-based config once Configuration API graduates to v1. flag.StringVar(&metricsAddr, "metrics-addr", configapi.DefaultMetricsAddr, "The address the metric endpoint binds to.") @@ -106,6 +107,7 @@ func main() { flag.BoolVar(&enableMetrics, "enable-metrics", false, "Enable the emission of control plane metrics.") flag.Float64Var(&qps, "qps", float64(configapi.DefaultQPS), "The QPS value for the client communicating with the Kubernetes API server.") flag.IntVar(&burst, "burst", configapi.DefaultBurst, "The maximum burst for throttling requests from this client to the Kubernetes API server.") + flag.BoolVar(&useBackgroundGoroutine, "use-background-goroutine", false, "Enable the background goroutine for fetching job info in RayJob.") opts := k8szap.Options{ TimeEncoder: zapcore.ISO8601TimeEncoder, @@ -138,6 +140,7 @@ func main() { config.EnableMetrics = enableMetrics config.QPS = &qps config.Burst = &burst + config.UseBackgroundGoroutine = useBackgroundGoroutine } stdoutEncoder, err := newLogEncoder(logStdoutEncoder) From 03ce0e96557b914fb0fc7958729f44a1a6b97398 Mon Sep 17 00:00:00 2001 From: fscnick Date: Thu, 30 Oct 2025 15:51:37 +0800 Subject: [PATCH 06/54] [RayJob] expiry cache cleanup goroutine Signed-off-by: fscnick --- .../dashboardclient/dashboard_cache_client.go | 21 ++++++++++++++++++- 1 file changed, 20 insertions(+), 1 deletion(-) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index e1616649f45..0f33589876b 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -50,8 +50,8 @@ type ( func (w *workerPool) init(taskQueueSize int, workerSize int, queryInterval time.Duration) { w.taskQueue = make(chan Task, taskQueueSize) - // TODO: should we have observability for these goroutine? for i := 0; i < workerSize; i++ { + // TODO: observability for these goroutine // TODO: should we consider the stop ? go func() { for task := range w.taskQueue { @@ -87,6 +87,25 @@ func (r *RayDashboardCacheClient) InitClient(client RayDashboardClientInterface) // the New() returns error only if the size is less or equal than zero. cacheStorage, _ = lru.New[string, *JobInfoCache](cacheSize) } + + // expiry cache cleanup + go func() { + ticker := time.NewTicker(queryInterval * 10) + defer ticker.Stop() + + // TODO: observability + // TODO: should we consider the stop? + for range ticker.C { + keys := cacheStorage.Keys() + for _, key := range keys { + if cached, ok := cacheStorage.Peek(key); ok { + if time.Now().Add(-cacheExpiry).Before(*cached.UpdateAt) { + cacheStorage.Remove(key) + } + } + } + } + }() }) r.client = client From ac275c29b0497876c3976c14ba744dd622e96c1c Mon Sep 17 00:00:00 2001 From: fscnick Date: Thu, 30 Oct 2025 22:00:48 +0800 Subject: [PATCH 07/54] [RayJob] code and comment minor fix Signed-off-by: fscnick --- .../controllers/ray/rayjob_controller.go | 2 +- .../dashboardclient/dashboard_cache_client.go | 18 +++++++++++------- .../dashboardclient/dashboard_httpclient.go | 2 -- 3 files changed, 12 insertions(+), 10 deletions(-) diff --git a/ray-operator/controllers/ray/rayjob_controller.go b/ray-operator/controllers/ray/rayjob_controller.go index 2cd24da0afa..1d05e625bdc 100644 --- a/ray-operator/controllers/ray/rayjob_controller.go +++ b/ray-operator/controllers/ray/rayjob_controller.go @@ -760,11 +760,11 @@ func (r *RayJobReconciler) deleteClusterResources(ctx context.Context, rayJobIns logger.Info("The deletion of the associated RayCluster for RayJob is ongoing.", "RayCluster", cluster.Name) } else { if r.useBackgroundGoroutine { - // clear cache, and it will remove this job from updating loop. rayDashboardClient, err := r.dashboardClientFunc(&cluster, rayJobInstance.Status.DashboardURL) if err != nil { logger.Error(err, "Failed to get dashboard client for RayJob") } + // clear cache, and it will remove this job from the cache updating loop. if err := rayDashboardClient.StopJob(ctx, rayJobInstance.Status.JobId); err != nil { logger.Error(err, "Failed to stop job for RayJob") } diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index 0f33589876b..37b7410173d 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -27,9 +27,11 @@ const ( ) var ( + // singleton initWorkPool sync.Once pool workerPool + // singleton initCacheStorage sync.Once cacheStorage *lru.Cache[string, *JobInfoCache] ) @@ -93,13 +95,14 @@ func (r *RayDashboardCacheClient) InitClient(client RayDashboardClientInterface) ticker := time.NewTicker(queryInterval * 10) defer ticker.Stop() - // TODO: observability - // TODO: should we consider the stop? + // TODO: observability ? + // TODO: should we consider the stop ? for range ticker.C { keys := cacheStorage.Keys() + expiredThreshold := time.Now().Add(-cacheExpiry) for _, key := range keys { if cached, ok := cacheStorage.Peek(key); ok { - if time.Now().Add(-cacheExpiry).Before(*cached.UpdateAt) { + if cached.UpdateAt.Before(expiredThreshold) { cacheStorage.Remove(key) } } @@ -127,12 +130,15 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) if cached, ok := cacheStorage.Get(jobId); ok { return cached.JobInfo, cached.Err } - cached := &JobInfoCache{Err: ErrAgain} + currentTime := time.Now() + cached := &JobInfoCache{Err: ErrAgain, UpdateAt: ¤tTime} + + // Put a placeholder in storage. The cache will be updated only if the placeholder exists. + // The placeholder will be removed when StopJob or DeleteJob. if cached, existed, _ := cacheStorage.PeekOrAdd(jobId, cached); existed { return cached.JobInfo, cached.Err } - // send to worker pool task := func() bool { jobInfoCache, existed := cacheStorage.Get(jobId) if !existed { @@ -143,8 +149,6 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) currentTime := time.Now() jobInfoCache.UpdateAt = ¤tTime - // handle not found(ex: rayjob has deleted) - cacheStorage.Add(jobId, jobInfoCache) return !rayv1.IsJobTerminal(jobInfoCache.JobInfo.JobStatus) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_httpclient.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_httpclient.go index 98bd8da369c..570ce082243 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_httpclient.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_httpclient.go @@ -27,8 +27,6 @@ var ( ) type RayDashboardClientInterface interface { - // Remove InitClient for adapting variable implementation - // InitClient(client *http.Client, dashboardURL string) UpdateDeployments(ctx context.Context, configJson []byte) error // V2/multi-app Rest API GetServeDetails(ctx context.Context) (*utiltypes.ServeDetails, error) From 0923ef5c1b307c166c3ae2726ee66e20d5fb6e2d Mon Sep 17 00:00:00 2001 From: fscnick Date: Sat, 1 Nov 2025 22:14:41 +0800 Subject: [PATCH 08/54] [RayJob] task check contain or not befor add Signed-off-by: fscnick --- .../ray/utils/dashboardclient/dashboard_cache_client.go | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index 37b7410173d..38deb5825b1 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -131,11 +131,11 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) return cached.JobInfo, cached.Err } currentTime := time.Now() - cached := &JobInfoCache{Err: ErrAgain, UpdateAt: ¤tTime} + placeholder := &JobInfoCache{Err: ErrAgain, UpdateAt: ¤tTime} // Put a placeholder in storage. The cache will be updated only if the placeholder exists. // The placeholder will be removed when StopJob or DeleteJob. - if cached, existed, _ := cacheStorage.PeekOrAdd(jobId, cached); existed { + if cached, existed, _ := cacheStorage.PeekOrAdd(jobId, placeholder); existed { return cached.JobInfo, cached.Err } @@ -149,7 +149,9 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) currentTime := time.Now() jobInfoCache.UpdateAt = ¤tTime - cacheStorage.Add(jobId, jobInfoCache) + if _, existed := cacheStorage.ContainsOrAdd(jobId, jobInfoCache); !existed { + return false + } return !rayv1.IsJobTerminal(jobInfoCache.JobInfo.JobStatus) } From 9f87da63f2ecb952e8d69a3b5327652dff4c8832 Mon Sep 17 00:00:00 2001 From: fscnick Date: Tue, 2 Dec 2025 20:09:25 +0800 Subject: [PATCH 09/54] [RayJob] remove delete cache from deleteClusterResources and add lock for cache Signed-off-by: fscnick --- .../controllers/ray/rayjob_controller.go | 30 ++++++------------- .../dashboardclient/dashboard_cache_client.go | 28 ++++++++++++++++- ray-operator/controllers/ray/utils/util.go | 1 - 3 files changed, 36 insertions(+), 23 deletions(-) diff --git a/ray-operator/controllers/ray/rayjob_controller.go b/ray-operator/controllers/ray/rayjob_controller.go index 1d05e625bdc..68920c27455 100644 --- a/ray-operator/controllers/ray/rayjob_controller.go +++ b/ray-operator/controllers/ray/rayjob_controller.go @@ -43,11 +43,10 @@ const ( // RayJobReconciler reconciles a RayJob object type RayJobReconciler struct { client.Client - Recorder record.EventRecorder - options RayJobReconcilerOptions - Scheme *runtime.Scheme - dashboardClientFunc func(rayCluster *rayv1.RayCluster, url string) (dashboardclient.RayDashboardClientInterface, error) - useBackgroundGoroutine bool + Recorder record.EventRecorder + options RayJobReconcilerOptions + Scheme *runtime.Scheme + dashboardClientFunc func(rayCluster *rayv1.RayCluster, url string) (dashboardclient.RayDashboardClientInterface, error) } type RayJobReconcilerOptions struct { @@ -59,12 +58,11 @@ type RayJobReconcilerOptions struct { func NewRayJobReconciler(_ context.Context, mgr manager.Manager, options RayJobReconcilerOptions, provider utils.ClientProvider) *RayJobReconciler { dashboardClientFunc := provider.GetDashboardClient(mgr) return &RayJobReconciler{ - Client: mgr.GetClient(), - Scheme: mgr.GetScheme(), - Recorder: mgr.GetEventRecorderFor("rayjob-controller"), - dashboardClientFunc: dashboardClientFunc, - useBackgroundGoroutine: provider.DoesUseBackgroundGoroutine(), - options: options, + Client: mgr.GetClient(), + Scheme: mgr.GetScheme(), + Recorder: mgr.GetEventRecorderFor("rayjob-controller"), + dashboardClientFunc: dashboardClientFunc, + options: options, } } @@ -759,16 +757,6 @@ func (r *RayJobReconciler) deleteClusterResources(ctx context.Context, rayJobIns if !cluster.DeletionTimestamp.IsZero() { logger.Info("The deletion of the associated RayCluster for RayJob is ongoing.", "RayCluster", cluster.Name) } else { - if r.useBackgroundGoroutine { - rayDashboardClient, err := r.dashboardClientFunc(&cluster, rayJobInstance.Status.DashboardURL) - if err != nil { - logger.Error(err, "Failed to get dashboard client for RayJob") - } - // clear cache, and it will remove this job from the cache updating loop. - if err := rayDashboardClient.StopJob(ctx, rayJobInstance.Status.JobId); err != nil { - logger.Error(err, "Failed to stop job for RayJob") - } - } if err := r.Delete(ctx, &cluster); err != nil { r.Recorder.Eventf(rayJobInstance, corev1.EventTypeWarning, string(utils.FailedToDeleteRayCluster), "Failed to delete cluster %s/%s: %v", cluster.Namespace, cluster.Name, err) return false, err diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index 38deb5825b1..945deb97160 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -34,6 +34,7 @@ var ( // singleton initCacheStorage sync.Once cacheStorage *lru.Cache[string, *JobInfoCache] + cacheLock sync.RWMutex ) type ( @@ -101,11 +102,13 @@ func (r *RayDashboardCacheClient) InitClient(client RayDashboardClientInterface) keys := cacheStorage.Keys() expiredThreshold := time.Now().Add(-cacheExpiry) for _, key := range keys { + cacheLock.Lock() if cached, ok := cacheStorage.Peek(key); ok { if cached.UpdateAt.Before(expiredThreshold) { cacheStorage.Remove(key) } } + cacheLock.Unlock() } } }() @@ -127,32 +130,49 @@ func (r *RayDashboardCacheClient) GetMultiApplicationStatus(ctx context.Context) } func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) (*utiltypes.RayJobInfo, error) { + cacheLock.RLock() if cached, ok := cacheStorage.Get(jobId); ok { + cacheLock.RUnlock() return cached.JobInfo, cached.Err } + cacheLock.RUnlock() + currentTime := time.Now() placeholder := &JobInfoCache{Err: ErrAgain, UpdateAt: ¤tTime} // Put a placeholder in storage. The cache will be updated only if the placeholder exists. // The placeholder will be removed when StopJob or DeleteJob. + cacheLock.Lock() if cached, existed, _ := cacheStorage.PeekOrAdd(jobId, placeholder); existed { + cacheLock.Unlock() return cached.JobInfo, cached.Err } + cacheLock.Unlock() task := func() bool { + cacheLock.RLock() jobInfoCache, existed := cacheStorage.Get(jobId) if !existed { + cacheLock.RUnlock() return false } + cacheLock.RUnlock() jobInfoCache.JobInfo, jobInfoCache.Err = r.client.GetJobInfo(ctx, jobId) currentTime := time.Now() jobInfoCache.UpdateAt = ¤tTime - if _, existed := cacheStorage.ContainsOrAdd(jobId, jobInfoCache); !existed { + cacheLock.Lock() + if existed := cacheStorage.Contains(jobId); !existed { + cacheLock.Unlock() return false } + cacheStorage.Add(jobId, jobInfoCache) + cacheLock.Unlock() + if jobInfoCache.JobInfo == nil { + return true + } return !rayv1.IsJobTerminal(jobInfoCache.JobInfo.JobStatus) } @@ -178,11 +198,17 @@ func (r *RayDashboardCacheClient) GetJobLog(ctx context.Context, jobName string) } func (r *RayDashboardCacheClient) StopJob(ctx context.Context, jobName string) error { + cacheLock.Lock() + defer cacheLock.Unlock() + cacheStorage.Remove(jobName) return r.client.StopJob(ctx, jobName) } func (r *RayDashboardCacheClient) DeleteJob(ctx context.Context, jobName string) error { + cacheLock.Lock() + defer cacheLock.Unlock() + cacheStorage.Remove(jobName) return r.client.DeleteJob(ctx, jobName) } diff --git a/ray-operator/controllers/ray/utils/util.go b/ray-operator/controllers/ray/utils/util.go index 31a93664b64..9bc463de718 100644 --- a/ray-operator/controllers/ray/utils/util.go +++ b/ray-operator/controllers/ray/utils/util.go @@ -653,7 +653,6 @@ func EnvVarByName(envName string, envVars []corev1.EnvVar) (corev1.EnvVar, bool) type ClientProvider interface { GetDashboardClient(mgr manager.Manager) func(rayCluster *rayv1.RayCluster, url string) (dashboardclient.RayDashboardClientInterface, error) GetHttpProxyClient(mgr manager.Manager) func(hostIp, podNamespace, podName string, port int) RayHttpProxyClientInterface - DoesUseBackgroundGoroutine() bool } func ManagedByExternalController(controllerName *string) *string { From 97ab4079929ca99eb580baf02e6827250e5aebf6 Mon Sep 17 00:00:00 2001 From: fscnick Date: Tue, 2 Dec 2025 20:18:52 +0800 Subject: [PATCH 10/54] [Helm] add argument for useBackgroundGoroutine Signed-off-by: fscnick --- helm-chart/kuberay-operator/README.md | 1 + helm-chart/kuberay-operator/templates/deployment.yaml | 3 +++ helm-chart/kuberay-operator/values.yaml | 4 ++++ 3 files changed, 8 insertions(+) diff --git a/helm-chart/kuberay-operator/README.md b/helm-chart/kuberay-operator/README.md index ecc0f8cf988..14e32aaec98 100644 --- a/helm-chart/kuberay-operator/README.md +++ b/helm-chart/kuberay-operator/README.md @@ -191,6 +191,7 @@ spec: | rbacEnable | bool | `true` | If rbacEnable is set to false, no RBAC resources will be created, including the Role for leader election, the Role for Pods and Services, and so on. | | crNamespacedRbacEnable | bool | `true` | When crNamespacedRbacEnable is set to true, the KubeRay operator will create a Role for RayCluster preparation (e.g., Pods, Services) and a corresponding RoleBinding for each namespace listed in the "watchNamespace" parameter. Please note that even if crNamespacedRbacEnable is set to false, the Role and RoleBinding for leader election will still be created. Note: (1) This variable is only effective when rbacEnable and singleNamespaceInstall are both set to true. (2) In most cases, it should be set to true, unless you are using a Kubernetes cluster managed by GitOps tools such as ArgoCD. | | singleNamespaceInstall | bool | `false` | When singleNamespaceInstall is true: - Install namespaced RBAC resources such as Role and RoleBinding instead of cluster-scoped ones like ClusterRole and ClusterRoleBinding so that the chart can be installed by users with permissions restricted to a single namespace. (Please note that this excludes the CRDs, which can only be installed at the cluster scope.) - If "watchNamespace" is not set, the KubeRay operator will, by default, only listen to resource events within its own namespace. | +| useBackgroundGoroutine | bool | `false` | When useBackgroundGoroutine is set to true: the KubeRay operator will use background goroutines to fetch the jobInfo from Ray clusters for RayJob status updates. | | env | string | `nil` | Environment variables. | | resources | object | `{"limits":{"cpu":"100m","memory":"512Mi"}}` | Resource requests and limits for containers. | | livenessProbe.initialDelaySeconds | int | `10` | | diff --git a/helm-chart/kuberay-operator/templates/deployment.yaml b/helm-chart/kuberay-operator/templates/deployment.yaml index 78cb0fe944d..14183095576 100644 --- a/helm-chart/kuberay-operator/templates/deployment.yaml +++ b/helm-chart/kuberay-operator/templates/deployment.yaml @@ -152,6 +152,9 @@ spec: {{- $argList = append $argList (printf "--burst=%v" .Values.kubeClient.burst) -}} {{- end -}} {{- end -}} + {{- if hasKey .Values "useBackgroundGoroutine" -}} + {{- $argList = append $argList (printf "--use-background-goroutine=%t" .Values.useBackgroundGoroutine) -}} + {{- end -}} {{- (printf "\n") -}} {{- $argList | toYaml | indent 12 }} ports: diff --git a/helm-chart/kuberay-operator/values.yaml b/helm-chart/kuberay-operator/values.yaml index 1b7b46020b0..a18c914acc5 100644 --- a/helm-chart/kuberay-operator/values.yaml +++ b/helm-chart/kuberay-operator/values.yaml @@ -185,6 +185,10 @@ crNamespacedRbacEnable: true # to resource events within its own namespace. singleNamespaceInstall: false +# -- When useBackgroundGoroutine is set to true: +# the KubeRay operator will use background goroutines to fetch the jobInfo from Ray clusters for RayJob status updates. +useBackgroundGoroutine: false + # The KubeRay operator will watch the custom resources in the namespaces listed in the "watchNamespace" parameter. # watchNamespace: # - n1 From a2a0961a5f363394cd7eefe4135ded40d64037bf Mon Sep 17 00:00:00 2001 From: fscnick Date: Wed, 3 Dec 2025 20:33:55 +0800 Subject: [PATCH 11/54] [RayJob] repeated error did not update Signed-off-by: fscnick --- .../utils/dashboardclient/dashboard_cache_client.go | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index 945deb97160..59c35db3864 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -7,6 +7,7 @@ import ( "time" lru "github.com/hashicorp/golang-lru/v2" + k8serrors "k8s.io/apimachinery/pkg/api/errors" rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1" utiltypes "github.com/ray-project/kuberay/ray-operator/controllers/ray/utils/types" @@ -158,7 +159,17 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) } cacheLock.RUnlock() - jobInfoCache.JobInfo, jobInfoCache.Err = r.client.GetJobInfo(ctx, jobId) + var statusErr *k8serrors.StatusError + jobInfo, err := r.client.GetJobInfo(ctx, jobId) + if err != nil && !errors.As(err, &statusErr) { + if jobInfoCache.Err != nil && err.Error() == jobInfoCache.Err.Error() { + // The error is the same as last time, no need to update, just put the task to execute later. + // If the error is not fixed, eventually the cache will be expired and removed. + return true + } + } + jobInfoCache.JobInfo = jobInfo + jobInfoCache.Err = err currentTime := time.Now() jobInfoCache.UpdateAt = ¤tTime From d2173bb84db7d4aca4dee972e5fcd6c9dc8339ac Mon Sep 17 00:00:00 2001 From: fscnick Date: Tue, 9 Dec 2025 23:15:50 +0800 Subject: [PATCH 12/54] [RayJob] remove unused function and background goroutine observability Signed-off-by: fscnick --- .../ray_job_submission_service_server.go | 2 +- .../config/v1alpha1/configuration_types.go | 10 +++--- .../controllers/ray/rayjob_controller.go | 4 +-- .../controllers/ray/rayservice_controller.go | 4 +-- ray-operator/controllers/ray/suite_test.go | 3 +- .../dashboardclient/dashboard_cache_client.go | 36 ++++++++++++++++--- ray-operator/controllers/ray/utils/util.go | 8 ++--- ray-operator/test/sampleyaml/support.go | 2 +- 8 files changed, 47 insertions(+), 22 deletions(-) diff --git a/apiserver/pkg/server/ray_job_submission_service_server.go b/apiserver/pkg/server/ray_job_submission_service_server.go index 596b991821e..28c07b57a04 100644 --- a/apiserver/pkg/server/ray_job_submission_service_server.go +++ b/apiserver/pkg/server/ray_job_submission_service_server.go @@ -41,7 +41,7 @@ type RayJobSubmissionServiceServer struct { // Create RayJobSubmissionServiceServer func NewRayJobSubmissionServiceServer(clusterServer *ClusterServer, options *RayJobSubmissionServiceServerOptions) *RayJobSubmissionServiceServer { zl := zerolog.New(os.Stdout).Level(zerolog.DebugLevel) - return &RayJobSubmissionServiceServer{clusterServer: clusterServer, options: options, log: zerologr.New(&zl).WithName("jobsubmissionservice"), dashboardClientFunc: utils.GetRayDashboardClientFunc(nil, false, false)} + return &RayJobSubmissionServiceServer{clusterServer: clusterServer, options: options, log: zerologr.New(&zl).WithName("jobsubmissionservice"), dashboardClientFunc: utils.GetRayDashboardClientFunc(context.Background(), nil, false, false)} } // Submit Ray job diff --git a/ray-operator/apis/config/v1alpha1/configuration_types.go b/ray-operator/apis/config/v1alpha1/configuration_types.go index 55e1269c6a6..f76285a7019 100644 --- a/ray-operator/apis/config/v1alpha1/configuration_types.go +++ b/ray-operator/apis/config/v1alpha1/configuration_types.go @@ -1,6 +1,8 @@ package v1alpha1 import ( + "context" + corev1 "k8s.io/api/core/v1" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "sigs.k8s.io/controller-runtime/pkg/manager" @@ -92,14 +94,10 @@ type Configuration struct { UseBackgroundGoroutine bool `json:"useBackgroundGoroutine,omitempty"` } -func (config Configuration) GetDashboardClient(mgr manager.Manager) func(rayCluster *rayv1.RayCluster, url string) (dashboardclient.RayDashboardClientInterface, error) { - return utils.GetRayDashboardClientFunc(mgr, config.UseKubernetesProxy, config.UseBackgroundGoroutine) +func (config Configuration) GetDashboardClient(ctx context.Context, mgr manager.Manager) func(rayCluster *rayv1.RayCluster, url string) (dashboardclient.RayDashboardClientInterface, error) { + return utils.GetRayDashboardClientFunc(ctx, mgr, config.UseKubernetesProxy, config.UseBackgroundGoroutine) } func (config Configuration) GetHttpProxyClient(mgr manager.Manager) func(hostIp, podNamespace, podName string, port int) utils.RayHttpProxyClientInterface { return utils.GetRayHttpProxyClientFunc(mgr, config.UseKubernetesProxy) } - -func (config Configuration) DoesUseBackgroundGoroutine() bool { - return config.UseBackgroundGoroutine -} diff --git a/ray-operator/controllers/ray/rayjob_controller.go b/ray-operator/controllers/ray/rayjob_controller.go index 5f413d1e4b8..9d7da493221 100644 --- a/ray-operator/controllers/ray/rayjob_controller.go +++ b/ray-operator/controllers/ray/rayjob_controller.go @@ -55,8 +55,8 @@ type RayJobReconcilerOptions struct { } // NewRayJobReconciler returns a new reconcile.Reconciler -func NewRayJobReconciler(_ context.Context, mgr manager.Manager, options RayJobReconcilerOptions, provider utils.ClientProvider) *RayJobReconciler { - dashboardClientFunc := provider.GetDashboardClient(mgr) +func NewRayJobReconciler(ctx context.Context, mgr manager.Manager, options RayJobReconcilerOptions, provider utils.ClientProvider) *RayJobReconciler { + dashboardClientFunc := provider.GetDashboardClient(ctx, mgr) return &RayJobReconciler{ Client: mgr.GetClient(), Scheme: mgr.GetScheme(), diff --git a/ray-operator/controllers/ray/rayservice_controller.go b/ray-operator/controllers/ray/rayservice_controller.go index c9fdfaa603e..60352fd35dc 100644 --- a/ray-operator/controllers/ray/rayservice_controller.go +++ b/ray-operator/controllers/ray/rayservice_controller.go @@ -62,8 +62,8 @@ type RayServiceReconciler struct { } // NewRayServiceReconciler returns a new reconcile.Reconciler -func NewRayServiceReconciler(_ context.Context, mgr manager.Manager, provider utils.ClientProvider) *RayServiceReconciler { - dashboardClientFunc := provider.GetDashboardClient(mgr) +func NewRayServiceReconciler(ctx context.Context, mgr manager.Manager, provider utils.ClientProvider) *RayServiceReconciler { + dashboardClientFunc := provider.GetDashboardClient(ctx, mgr) httpProxyClientFunc := provider.GetHttpProxyClient(mgr) return &RayServiceReconciler{ Client: mgr.GetClient(), diff --git a/ray-operator/controllers/ray/suite_test.go b/ray-operator/controllers/ray/suite_test.go index e2ab5e16196..eed4ffe5b05 100644 --- a/ray-operator/controllers/ray/suite_test.go +++ b/ray-operator/controllers/ray/suite_test.go @@ -16,6 +16,7 @@ limitations under the License. package ray import ( + "context" "os" "path/filepath" "testing" @@ -52,7 +53,7 @@ var ( type TestClientProvider struct{} -func (testProvider TestClientProvider) GetDashboardClient(_ manager.Manager) func(rayCluster *rayv1.RayCluster, url string) (dashboardclient.RayDashboardClientInterface, error) { +func (testProvider TestClientProvider) GetDashboardClient(_ context.Context, _ manager.Manager) func(rayCluster *rayv1.RayCluster, url string) (dashboardclient.RayDashboardClientInterface, error) { return func(_ *rayv1.RayCluster, _ string) (dashboardclient.RayDashboardClientInterface, error) { return fakeRayDashboardClient, nil } diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index 59c35db3864..017d473dccb 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -3,11 +3,13 @@ package dashboardclient import ( "context" "errors" + "fmt" "sync" "time" lru "github.com/hashicorp/golang-lru/v2" k8serrors "k8s.io/apimachinery/pkg/api/errors" + ctrl "sigs.k8s.io/controller-runtime" rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1" utiltypes "github.com/ray-project/kuberay/ray-operator/controllers/ray/utils/types" @@ -51,7 +53,8 @@ type ( } ) -func (w *workerPool) init(taskQueueSize int, workerSize int, queryInterval time.Duration) { +func (w *workerPool) init(ctx context.Context, taskQueueSize int, workerSize int, queryInterval time.Duration) { + logger := ctrl.LoggerFrom(ctx).WithName("WorkerPool") w.taskQueue = make(chan Task, taskQueueSize) for i := 0; i < workerSize; i++ { @@ -69,6 +72,7 @@ func (w *workerPool) init(taskQueueSize int, workerSize int, queryInterval time. } }() } + logger.Info(fmt.Sprintf("Initialize a worker pool with %d and queryInterval is %v.", workerSize, queryInterval)) } func (w *workerPool) PutTask(task Task) { @@ -81,15 +85,19 @@ type RayDashboardCacheClient struct { client RayDashboardClientInterface } -func (r *RayDashboardCacheClient) InitClient(client RayDashboardClientInterface) { +func (r *RayDashboardCacheClient) InitClient(ctx context.Context, client RayDashboardClientInterface) { + logger := ctrl.LoggerFrom(ctx).WithName("RayDashboardCacheClient") + initWorkPool.Do(func() { - pool.init(taskQueueSize, workerSize, queryInterval) + pool.init(ctx, taskQueueSize, workerSize, queryInterval) }) initCacheStorage.Do(func() { if cacheStorage == nil { // the New() returns error only if the size is less or equal than zero. - cacheStorage, _ = lru.New[string, *JobInfoCache](cacheSize) + cacheStorage, _ = lru.NewWithEvict[string, *JobInfoCache](cacheSize, func(key string, value *JobInfoCache) { + logger.Info(fmt.Sprintf("Evict cache for key %s.", key), "value", value) + }) } // expiry cache cleanup @@ -97,20 +105,28 @@ func (r *RayDashboardCacheClient) InitClient(client RayDashboardClientInterface) ticker := time.NewTicker(queryInterval * 10) defer ticker.Stop() + loggerForGC := logger.WithName("CacheCleanup") + loggerForGC.Info(fmt.Sprintf("Initialize a cache cleanup goroutine with interval %v.", queryInterval*10)) + // TODO: observability ? // TODO: should we consider the stop ? for range ticker.C { keys := cacheStorage.Keys() expiredThreshold := time.Now().Add(-cacheExpiry) + loggerForGC.Info(fmt.Sprintf("Found %d keys to verify,", len(keys)), "expiredThreshold", expiredThreshold) + + removed := keys[:0] for _, key := range keys { cacheLock.Lock() if cached, ok := cacheStorage.Peek(key); ok { if cached.UpdateAt.Before(expiredThreshold) { cacheStorage.Remove(key) + removed = append(removed, key) } } cacheLock.Unlock() } + loggerForGC.Info(fmt.Sprintf("clean up %d cache.", len(removed)), "expiredThreshold", expiredThreshold, "removed keys", keys) } }() }) @@ -131,6 +147,8 @@ func (r *RayDashboardCacheClient) GetMultiApplicationStatus(ctx context.Context) } func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) (*utiltypes.RayJobInfo, error) { + logger := ctrl.LoggerFrom(ctx).WithName("RayDashboardCacheClient") + cacheLock.RLock() if cached, ok := cacheStorage.Get(jobId); ok { cacheLock.RUnlock() @@ -155,6 +173,7 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) jobInfoCache, existed := cacheStorage.Get(jobId) if !existed { cacheLock.RUnlock() + logger.Info("The placeholder is removed for jobId", "jobId", jobId) return false } cacheLock.RUnlock() @@ -165,6 +184,7 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) if jobInfoCache.Err != nil && err.Error() == jobInfoCache.Err.Error() { // The error is the same as last time, no need to update, just put the task to execute later. // If the error is not fixed, eventually the cache will be expired and removed. + logger.Info("The error is the same as last time for jobId", "jobId", jobId, "error", err) return true } } @@ -176,6 +196,7 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) cacheLock.Lock() if existed := cacheStorage.Contains(jobId); !existed { cacheLock.Unlock() + logger.Info("The placeholder is removed before updating for jobId", "jobId", jobId) return false } cacheStorage.Add(jobId, jobInfoCache) @@ -184,10 +205,15 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) if jobInfoCache.JobInfo == nil { return true } - return !rayv1.IsJobTerminal(jobInfoCache.JobInfo.JobStatus) + if rayv1.IsJobTerminal(jobInfoCache.JobInfo.JobStatus) { + logger.Info("The job reaches terminal status for jobId", "jobId", jobId, "status", jobInfoCache.JobInfo.JobStatus) + return false + } + return true } pool.PutTask(task) + logger.Info("Put a task to fetch job info in background for jobId ", "jobId", jobId) return nil, ErrAgain } diff --git a/ray-operator/controllers/ray/utils/util.go b/ray-operator/controllers/ray/utils/util.go index 6a19f1ead30..3c5eacd8f02 100644 --- a/ray-operator/controllers/ray/utils/util.go +++ b/ray-operator/controllers/ray/utils/util.go @@ -689,7 +689,7 @@ func EnvVarByName(envName string, envVars []corev1.EnvVar) (corev1.EnvVar, bool) } type ClientProvider interface { - GetDashboardClient(mgr manager.Manager) func(rayCluster *rayv1.RayCluster, url string) (dashboardclient.RayDashboardClientInterface, error) + GetDashboardClient(ctx context.Context, mgr manager.Manager) func(rayCluster *rayv1.RayCluster, url string) (dashboardclient.RayDashboardClientInterface, error) GetHttpProxyClient(mgr manager.Manager) func(hostIp, podNamespace, podName string, port int) RayHttpProxyClientInterface } @@ -920,7 +920,7 @@ func FetchHeadServiceURL(ctx context.Context, cli client.Client, rayCluster *ray return headServiceURL, nil } -func GetRayDashboardClientFunc(mgr manager.Manager, useKubernetesProxy bool, useBackgroundGoroutine bool) func(rayCluster *rayv1.RayCluster, url string) (dashboardclient.RayDashboardClientInterface, error) { +func GetRayDashboardClientFunc(ctx context.Context, mgr manager.Manager, useKubernetesProxy bool, useBackgroundGoroutine bool) func(rayCluster *rayv1.RayCluster, url string) (dashboardclient.RayDashboardClientInterface, error) { return func(rayCluster *rayv1.RayCluster, url string) (dashboardclient.RayDashboardClientInterface, error) { dashboardClient := &dashboardclient.RayDashboardClient{} var authToken string @@ -965,7 +965,7 @@ func GetRayDashboardClientFunc(mgr manager.Manager, useKubernetesProxy bool, use ) if useBackgroundGoroutine { dashboardCachedClient := &dashboardclient.RayDashboardCacheClient{} - dashboardCachedClient.InitClient(dashboardClient) + dashboardCachedClient.InitClient(ctx, dashboardClient) return dashboardCachedClient, nil } return dashboardClient, nil @@ -977,7 +977,7 @@ func GetRayDashboardClientFunc(mgr manager.Manager, useKubernetesProxy bool, use if useBackgroundGoroutine { dashboardCachedClient := &dashboardclient.RayDashboardCacheClient{} - dashboardCachedClient.InitClient(dashboardClient) + dashboardCachedClient.InitClient(ctx, dashboardClient) return dashboardCachedClient, nil } return dashboardClient, nil diff --git a/ray-operator/test/sampleyaml/support.go b/ray-operator/test/sampleyaml/support.go index f470295001b..e330b08e63c 100644 --- a/ray-operator/test/sampleyaml/support.go +++ b/ray-operator/test/sampleyaml/support.go @@ -75,7 +75,7 @@ func QueryDashboardGetAppStatus(t Test, rayCluster *rayv1.RayCluster) func(Gomeg g.Expect(err).ToNot(HaveOccurred()) url := fmt.Sprintf("127.0.0.1:%d", localPort) - rayDashboardClientFunc := utils.GetRayDashboardClientFunc(nil, false, false) + rayDashboardClientFunc := utils.GetRayDashboardClientFunc(t.Ctx(), nil, false, false) rayDashboardClient, err := rayDashboardClientFunc(rayCluster, url) g.Expect(err).ToNot(HaveOccurred()) serveDetails, err := rayDashboardClient.GetServeDetails(t.Ctx()) From 50c9b94a788c62b5e97547251b189df5e6d2a4d0 Mon Sep 17 00:00:00 2001 From: fscnick Date: Wed, 10 Dec 2025 21:48:13 +0800 Subject: [PATCH 13/54] [RayJob] cache client support graceful shutdown Signed-off-by: fscnick --- .../dashboardclient/dashboard_cache_client.go | 73 +++++++++++-------- 1 file changed, 42 insertions(+), 31 deletions(-) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index 017d473dccb..2d5e9b24e5b 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -54,25 +54,29 @@ type ( ) func (w *workerPool) init(ctx context.Context, taskQueueSize int, workerSize int, queryInterval time.Duration) { - logger := ctrl.LoggerFrom(ctx).WithName("WorkerPool") + logger := ctrl.LoggerFrom(ctx).WithName("RayDashboardCacheClient").WithName("WorkerPool") w.taskQueue = make(chan Task, taskQueueSize) for i := 0; i < workerSize; i++ { - // TODO: observability for these goroutine - // TODO: should we consider the stop ? - go func() { - for task := range w.taskQueue { - again := task() - - if again { - time.AfterFunc(queryInterval, func() { - w.taskQueue <- task - }) + go func(workerID int) { + for { + select { + case <-ctx.Done(): + logger.Info("worker exiting...", "workerID", workerID) + return + case task := <-w.taskQueue: + again := task() + + if again { + time.AfterFunc(queryInterval, func() { + w.taskQueue <- task + }) + } } } - }() + }(i) } - logger.Info(fmt.Sprintf("Initialize a worker pool with %d and queryInterval is %v.", workerSize, queryInterval)) + logger.Info(fmt.Sprintf("Initialize a worker pool with %d goroutine and queryInterval is %v.", workerSize, queryInterval)) } func (w *workerPool) PutTask(task Task) { @@ -95,8 +99,8 @@ func (r *RayDashboardCacheClient) InitClient(ctx context.Context, client RayDash initCacheStorage.Do(func() { if cacheStorage == nil { // the New() returns error only if the size is less or equal than zero. - cacheStorage, _ = lru.NewWithEvict[string, *JobInfoCache](cacheSize, func(key string, value *JobInfoCache) { - logger.Info(fmt.Sprintf("Evict cache for key %s.", key), "value", value) + cacheStorage, _ = lru.NewWithEvict[string, *JobInfoCache](cacheSize, func(key string, _ *JobInfoCache) { + logger.WithName("cacheStorage").Info("Evict cache for key.", "key", key) }) } @@ -108,25 +112,32 @@ func (r *RayDashboardCacheClient) InitClient(ctx context.Context, client RayDash loggerForGC := logger.WithName("CacheCleanup") loggerForGC.Info(fmt.Sprintf("Initialize a cache cleanup goroutine with interval %v.", queryInterval*10)) - // TODO: observability ? - // TODO: should we consider the stop ? - for range ticker.C { - keys := cacheStorage.Keys() - expiredThreshold := time.Now().Add(-cacheExpiry) - loggerForGC.Info(fmt.Sprintf("Found %d keys to verify,", len(keys)), "expiredThreshold", expiredThreshold) - - removed := keys[:0] - for _, key := range keys { - cacheLock.Lock() - if cached, ok := cacheStorage.Peek(key); ok { - if cached.UpdateAt.Before(expiredThreshold) { - cacheStorage.Remove(key) - removed = append(removed, key) + for { + select { + case <-ctx.Done(): + loggerForGC.Info("clean up goroutine exiting...") + return + case t := <-ticker.C: + cacheLock.RLock() + keys := cacheStorage.Keys() + cacheLock.RUnlock() + + expiredThreshold := time.Now().Add(-cacheExpiry) + loggerForGC.Info(fmt.Sprintf("Found %d keys to verify,", len(keys)), "expiredThreshold", expiredThreshold, "tick at", t) + + removed := keys[:0] + for _, key := range keys { + cacheLock.Lock() + if cached, ok := cacheStorage.Peek(key); ok { + if cached.UpdateAt.Before(expiredThreshold) { + cacheStorage.Remove(key) + removed = append(removed, key) + } } + cacheLock.Unlock() } - cacheLock.Unlock() + loggerForGC.Info(fmt.Sprintf("clean up %d cache.", len(removed)), "expiredThreshold", expiredThreshold, "removed keys", removed) } - loggerForGC.Info(fmt.Sprintf("clean up %d cache.", len(removed)), "expiredThreshold", expiredThreshold, "removed keys", keys) } }() }) From 1ab70fa4bebe08ca8b7b80de9d02b9648e2d8726 Mon Sep 17 00:00:00 2001 From: fscnick Date: Sun, 14 Dec 2025 21:39:24 +0800 Subject: [PATCH 14/54] [RayJob] rename useBackgroundGoroutine to asyncJobInfoQuery Signed-off-by: fscnick --- helm-chart/kuberay-operator/README.md | 2 +- helm-chart/kuberay-operator/templates/deployment.yaml | 4 ++-- helm-chart/kuberay-operator/values.yaml | 4 ++-- ray-operator/apis/config/v1alpha1/configuration_types.go | 6 +++--- ray-operator/main.go | 6 +++--- 5 files changed, 11 insertions(+), 11 deletions(-) diff --git a/helm-chart/kuberay-operator/README.md b/helm-chart/kuberay-operator/README.md index 364bb8324e9..9a4a463954c 100644 --- a/helm-chart/kuberay-operator/README.md +++ b/helm-chart/kuberay-operator/README.md @@ -192,7 +192,7 @@ spec: | rbacEnable | bool | `true` | If rbacEnable is set to false, no RBAC resources will be created, including the Role for leader election, the Role for Pods and Services, and so on. | | crNamespacedRbacEnable | bool | `true` | When crNamespacedRbacEnable is set to true, the KubeRay operator will create a Role for RayCluster preparation (e.g., Pods, Services) and a corresponding RoleBinding for each namespace listed in the "watchNamespace" parameter. Please note that even if crNamespacedRbacEnable is set to false, the Role and RoleBinding for leader election will still be created. Note: (1) This variable is only effective when rbacEnable and singleNamespaceInstall are both set to true. (2) In most cases, it should be set to true, unless you are using a Kubernetes cluster managed by GitOps tools such as ArgoCD. | | singleNamespaceInstall | bool | `false` | When singleNamespaceInstall is true: - Install namespaced RBAC resources such as Role and RoleBinding instead of cluster-scoped ones like ClusterRole and ClusterRoleBinding so that the chart can be installed by users with permissions restricted to a single namespace. (Please note that this excludes the CRDs, which can only be installed at the cluster scope.) - If "watchNamespace" is not set, the KubeRay operator will, by default, only listen to resource events within its own namespace. | -| useBackgroundGoroutine | bool | `false` | When useBackgroundGoroutine is set to true: the KubeRay operator will use background goroutines to fetch the jobInfo from Ray clusters for RayJob status updates. | +| asyncJobInfoQuery | bool | `false` | When AsyncJobInfoQuery is set to true: the KubeRay operator will use background goroutines to fetch the jobInfo from Ray clusters for RayJob status updates. | | env | string | `nil` | Environment variables. | | resources | object | `{"limits":{"cpu":"100m","memory":"512Mi"}}` | Resource requests and limits for containers. | | livenessProbe.initialDelaySeconds | int | `10` | | diff --git a/helm-chart/kuberay-operator/templates/deployment.yaml b/helm-chart/kuberay-operator/templates/deployment.yaml index 8bd561132e2..5ca5d79f32e 100644 --- a/helm-chart/kuberay-operator/templates/deployment.yaml +++ b/helm-chart/kuberay-operator/templates/deployment.yaml @@ -152,8 +152,8 @@ spec: {{- $argList = append $argList (printf "--burst=%v" .Values.kubeClient.burst) -}} {{- end -}} {{- end -}} - {{- if hasKey .Values "useBackgroundGoroutine" -}} - {{- $argList = append $argList (printf "--use-background-goroutine=%t" .Values.useBackgroundGoroutine) -}} + {{- if hasKey .Values "asyncJobInfoQuery" -}} + {{- $argList = append $argList (printf "--async-jobinfo-query=%t" .Values.asyncJobInfoQuery) -}} {{- end -}} {{- (printf "\n") -}} {{- $argList | toYaml | indent 12 }} diff --git a/helm-chart/kuberay-operator/values.yaml b/helm-chart/kuberay-operator/values.yaml index 1af9ba8a44c..66e72047c62 100644 --- a/helm-chart/kuberay-operator/values.yaml +++ b/helm-chart/kuberay-operator/values.yaml @@ -188,9 +188,9 @@ crNamespacedRbacEnable: true # to resource events within its own namespace. singleNamespaceInstall: false -# -- When useBackgroundGoroutine is set to true: +# -- When AsyncJobInfoQuery is set to true: # the KubeRay operator will use background goroutines to fetch the jobInfo from Ray clusters for RayJob status updates. -useBackgroundGoroutine: false +asyncJobInfoQuery: false # The KubeRay operator will watch the custom resources in the namespaces listed in the "watchNamespace" parameter. # watchNamespace: diff --git a/ray-operator/apis/config/v1alpha1/configuration_types.go b/ray-operator/apis/config/v1alpha1/configuration_types.go index f76285a7019..90dcdcec856 100644 --- a/ray-operator/apis/config/v1alpha1/configuration_types.go +++ b/ray-operator/apis/config/v1alpha1/configuration_types.go @@ -89,13 +89,13 @@ type Configuration struct { // EnableMetrics indicates whether KubeRay operator should emit control plane metrics. EnableMetrics bool `json:"enableMetrics,omitempty"` - // UseBackgroundGoroutine indicates that it wil use goroutine to fetch the job info from ray dashboard and + // AsyncJobInfoQuery indicates that it wil use goroutine to fetch the job info from ray dashboard and // store the job info in the cache - UseBackgroundGoroutine bool `json:"useBackgroundGoroutine,omitempty"` + AsyncJobInfoQuery bool `json:"useBackgroundGoroutine,omitempty"` } func (config Configuration) GetDashboardClient(ctx context.Context, mgr manager.Manager) func(rayCluster *rayv1.RayCluster, url string) (dashboardclient.RayDashboardClientInterface, error) { - return utils.GetRayDashboardClientFunc(ctx, mgr, config.UseKubernetesProxy, config.UseBackgroundGoroutine) + return utils.GetRayDashboardClientFunc(ctx, mgr, config.UseKubernetesProxy, config.AsyncJobInfoQuery) } func (config Configuration) GetHttpProxyClient(mgr manager.Manager) func(hostIp, podNamespace, podName string, port int) utils.RayHttpProxyClientInterface { diff --git a/ray-operator/main.go b/ray-operator/main.go index feb675e717f..a88f816c6e2 100644 --- a/ray-operator/main.go +++ b/ray-operator/main.go @@ -73,7 +73,7 @@ func main() { var enableMetrics bool var qps float64 var burst int - var useBackgroundGoroutine bool + var asyncJobInfoQuery bool // TODO: remove flag-based config once Configuration API graduates to v1. flag.StringVar(&metricsAddr, "metrics-addr", configapi.DefaultMetricsAddr, "The address the metric endpoint binds to.") @@ -107,7 +107,7 @@ func main() { flag.BoolVar(&enableMetrics, "enable-metrics", false, "Enable the emission of control plane metrics.") flag.Float64Var(&qps, "qps", float64(configapi.DefaultQPS), "The QPS value for the client communicating with the Kubernetes API server.") flag.IntVar(&burst, "burst", configapi.DefaultBurst, "The maximum burst for throttling requests from this client to the Kubernetes API server.") - flag.BoolVar(&useBackgroundGoroutine, "use-background-goroutine", false, "Enable the background goroutine for fetching job info in RayJob.") + flag.BoolVar(&asyncJobInfoQuery, "async-jobinfo-query", false, "Enable the background goroutine for fetching job info in RayJob.") opts := k8szap.Options{ TimeEncoder: zapcore.ISO8601TimeEncoder, @@ -140,7 +140,7 @@ func main() { config.EnableMetrics = enableMetrics config.QPS = &qps config.Burst = &burst - config.UseBackgroundGoroutine = useBackgroundGoroutine + config.AsyncJobInfoQuery = asyncJobInfoQuery } stdoutEncoder, err := newLogEncoder(logStdoutEncoder) From efb7d1756ef5b40a1e3adf9c7c82bb34d0483ff4 Mon Sep 17 00:00:00 2001 From: fscnick Date: Sun, 14 Dec 2025 21:53:24 +0800 Subject: [PATCH 15/54] [RayJob] use ray job info in logger Signed-off-by: fscnick --- ray-operator/controllers/ray/rayjob_controller.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ray-operator/controllers/ray/rayjob_controller.go b/ray-operator/controllers/ray/rayjob_controller.go index 9d7da493221..ce0e8dba651 100644 --- a/ray-operator/controllers/ray/rayjob_controller.go +++ b/ray-operator/controllers/ray/rayjob_controller.go @@ -289,7 +289,7 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) jobInfo, err := rayDashboardClient.GetJobInfo(ctx, rayJobInstance.Status.JobId) if err != nil { if errs.Is(err, dashboardclient.ErrAgain) { - logger.Info("The Ray job Info was not ready. Try again next iteration.", "JobId", rayJobInstance.Status.JobId) + logger.Info("The Ray job info was not ready. Try again next iteration.", "JobId", rayJobInstance.Status.JobId) return ctrl.Result{RequeueAfter: RayJobDefaultRequeueDuration}, nil } // If the Ray job was not found, GetJobInfo returns a BadRequest error. From 9e71883b2ab7e9940d758e9f3e3b51ce7ad6c976 Mon Sep 17 00:00:00 2001 From: fscnick Date: Mon, 15 Dec 2025 21:17:43 +0800 Subject: [PATCH 16/54] [RayJob] remove cacheStorage nil check Signed-off-by: fscnick --- .../utils/dashboardclient/dashboard_cache_client.go | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index 2d5e9b24e5b..2fc87572a6b 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -97,12 +97,10 @@ func (r *RayDashboardCacheClient) InitClient(ctx context.Context, client RayDash }) initCacheStorage.Do(func() { - if cacheStorage == nil { - // the New() returns error only if the size is less or equal than zero. - cacheStorage, _ = lru.NewWithEvict[string, *JobInfoCache](cacheSize, func(key string, _ *JobInfoCache) { - logger.WithName("cacheStorage").Info("Evict cache for key.", "key", key) - }) - } + // the New() returns error only if the size is less or equal than zero. + cacheStorage, _ = lru.NewWithEvict[string, *JobInfoCache](cacheSize, func(key string, _ *JobInfoCache) { + logger.WithName("cacheStorage").Info("Evict cache for key.", "key", key) + }) // expiry cache cleanup go func() { From 82085c0a729f1a128f09974310aedcb823480435 Mon Sep 17 00:00:00 2001 From: fscnick Date: Mon, 15 Dec 2025 21:40:37 +0800 Subject: [PATCH 17/54] [RayJob] bg goroutine uses operator context instead Signed-off-by: fscnick --- .../ray/utils/dashboardclient/dashboard_cache_client.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index 2fc87572a6b..c35f042980d 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -41,7 +41,7 @@ var ( ) type ( - Task func() bool + Task func(taskCTX context.Context) bool JobInfoCache struct { JobInfo *utiltypes.RayJobInfo Err error @@ -65,7 +65,7 @@ func (w *workerPool) init(ctx context.Context, taskQueueSize int, workerSize int logger.Info("worker exiting...", "workerID", workerID) return case task := <-w.taskQueue: - again := task() + again := task(ctx) if again { time.AfterFunc(queryInterval, func() { @@ -177,7 +177,7 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) } cacheLock.Unlock() - task := func() bool { + task := func(taskCTX context.Context) bool { cacheLock.RLock() jobInfoCache, existed := cacheStorage.Get(jobId) if !existed { @@ -188,7 +188,7 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) cacheLock.RUnlock() var statusErr *k8serrors.StatusError - jobInfo, err := r.client.GetJobInfo(ctx, jobId) + jobInfo, err := r.client.GetJobInfo(taskCTX, jobId) if err != nil && !errors.As(err, &statusErr) { if jobInfoCache.Err != nil && err.Error() == jobInfoCache.Err.Error() { // The error is the same as last time, no need to update, just put the task to execute later. From bdb62b2899de7dcc84a1c4df994acd3c9b2c0dfc Mon Sep 17 00:00:00 2001 From: fscnick Date: Mon, 15 Dec 2025 22:26:49 +0800 Subject: [PATCH 18/54] [RayJob] bg goroutine handle task queue full Signed-off-by: fscnick --- .../dashboardclient/dashboard_cache_client.go | 18 +++++++++++++++--- 1 file changed, 15 insertions(+), 3 deletions(-) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index c35f042980d..ed3556655b4 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -15,7 +15,10 @@ import ( utiltypes "github.com/ray-project/kuberay/ray-operator/controllers/ray/utils/types" ) -var ErrAgain = errors.New("EAGAIN") +var ( + ErrAgain = errors.New("EAGAIN") + ErrTaskQueueFull = errors.New("task queue is full") +) const ( // TODO: make queue size and worker size configurable. @@ -79,8 +82,14 @@ func (w *workerPool) init(ctx context.Context, taskQueueSize int, workerSize int logger.Info(fmt.Sprintf("Initialize a worker pool with %d goroutine and queryInterval is %v.", workerSize, queryInterval)) } -func (w *workerPool) PutTask(task Task) { +func (w *workerPool) PutTask(task Task) error { w.taskQueue <- task + select { + case w.taskQueue <- task: + return nil + default: + return ErrTaskQueueFull + } } var _ RayDashboardClientInterface = (*RayDashboardCacheClient)(nil) @@ -221,7 +230,10 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) return true } - pool.PutTask(task) + if err := pool.PutTask(task); err != nil { + logger.Error(err, "Cannot queue more jobInfo fetching tasks.", "jobId", jobId) + return nil, err + } logger.Info("Put a task to fetch job info in background for jobId ", "jobId", jobId) return nil, ErrAgain From 75345e64810e37c24cae3f159ebba047fe037fcd Mon Sep 17 00:00:00 2001 From: fscnick Date: Tue, 16 Dec 2025 23:32:14 +0800 Subject: [PATCH 19/54] [RayJob] correct the comment Signed-off-by: fscnick --- .../ray/utils/dashboardclient/dashboard_cache_client.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index ed3556655b4..1f8bcdb77ac 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -106,7 +106,8 @@ func (r *RayDashboardCacheClient) InitClient(ctx context.Context, client RayDash }) initCacheStorage.Do(func() { - // the New() returns error only if the size is less or equal than zero. + // The NewWithEvict() returns error only if the cacheSize is less than or equal to zero. + // While we set cacheSize as constant, we can ignore the error here. cacheStorage, _ = lru.NewWithEvict[string, *JobInfoCache](cacheSize, func(key string, _ *JobInfoCache) { logger.WithName("cacheStorage").Info("Evict cache for key.", "key", key) }) From 5d471c436d593afe3d2fe1e6facff8fc3c6dde56 Mon Sep 17 00:00:00 2001 From: fscnick Date: Tue, 16 Dec 2025 23:34:51 +0800 Subject: [PATCH 20/54] [RayJob] refactor initialize dashboard client for background goroutine Signed-off-by: fscnick --- ray-operator/controllers/ray/utils/util.go | 18 ++++++------------ 1 file changed, 6 insertions(+), 12 deletions(-) diff --git a/ray-operator/controllers/ray/utils/util.go b/ray-operator/controllers/ray/utils/util.go index 3c5eacd8f02..0519c1456ca 100644 --- a/ray-operator/controllers/ray/utils/util.go +++ b/ray-operator/controllers/ray/utils/util.go @@ -920,7 +920,7 @@ func FetchHeadServiceURL(ctx context.Context, cli client.Client, rayCluster *ray return headServiceURL, nil } -func GetRayDashboardClientFunc(ctx context.Context, mgr manager.Manager, useKubernetesProxy bool, useBackgroundGoroutine bool) func(rayCluster *rayv1.RayCluster, url string) (dashboardclient.RayDashboardClientInterface, error) { +func GetRayDashboardClientFunc(ctx context.Context, mgr manager.Manager, useKubernetesProxy bool, asyncJobInfoQuery bool) func(rayCluster *rayv1.RayCluster, url string) (dashboardclient.RayDashboardClientInterface, error) { return func(rayCluster *rayv1.RayCluster, url string) (dashboardclient.RayDashboardClientInterface, error) { dashboardClient := &dashboardclient.RayDashboardClient{} var authToken string @@ -963,19 +963,13 @@ func GetRayDashboardClientFunc(ctx context.Context, mgr manager.Manager, useKube fmt.Sprintf("%s/api/v1/namespaces/%s/services/%s:dashboard/proxy", mgr.GetConfig().Host, rayCluster.Namespace, headSvcName), authToken, ) - if useBackgroundGoroutine { - dashboardCachedClient := &dashboardclient.RayDashboardCacheClient{} - dashboardCachedClient.InitClient(ctx, dashboardClient) - return dashboardCachedClient, nil - } - return dashboardClient, nil + } else { + dashboardClient.InitClient(&http.Client{ + Timeout: 2 * time.Second, + }, "http://"+url, authToken) } - dashboardClient.InitClient(&http.Client{ - Timeout: 2 * time.Second, - }, "http://"+url, authToken) - - if useBackgroundGoroutine { + if asyncJobInfoQuery { dashboardCachedClient := &dashboardclient.RayDashboardCacheClient{} dashboardCachedClient.InitClient(ctx, dashboardClient) return dashboardCachedClient, nil From 745e7a6aa6e07a2cb7baa2211cffb3e23d32add7 Mon Sep 17 00:00:00 2001 From: fscnick Date: Wed, 17 Dec 2025 20:00:04 +0800 Subject: [PATCH 21/54] [RayJob] worker handle ctx.Done correctly Signed-off-by: fscnick --- .../ray/utils/dashboardclient/dashboard_cache_client.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index 1f8bcdb77ac..b2b83fe1b9d 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -70,7 +70,7 @@ func (w *workerPool) init(ctx context.Context, taskQueueSize int, workerSize int case task := <-w.taskQueue: again := task(ctx) - if again { + if again && ctx.Err() == nil { time.AfterFunc(queryInterval, func() { w.taskQueue <- task }) From 3172fbe0ec950a99d4b9927909ea7e14ae7d7a3e Mon Sep 17 00:00:00 2001 From: fscnick Date: Wed, 17 Dec 2025 20:01:07 +0800 Subject: [PATCH 22/54] [RayJob] remove unnecessary putting task into queue --- .../ray/utils/dashboardclient/dashboard_cache_client.go | 1 - 1 file changed, 1 deletion(-) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index b2b83fe1b9d..93d18228c33 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -83,7 +83,6 @@ func (w *workerPool) init(ctx context.Context, taskQueueSize int, workerSize int } func (w *workerPool) PutTask(task Task) error { - w.taskQueue <- task select { case w.taskQueue <- task: return nil From c3363c381a13ef5a9370b09e1657724e98ad1389 Mon Sep 17 00:00:00 2001 From: fscnick Date: Wed, 17 Dec 2025 20:02:24 +0800 Subject: [PATCH 23/54] [RayJob] if queue is full, retry again --- .../ray/utils/dashboardclient/dashboard_cache_client.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index 93d18228c33..fb68fadfee5 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -232,7 +232,7 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) if err := pool.PutTask(task); err != nil { logger.Error(err, "Cannot queue more jobInfo fetching tasks.", "jobId", jobId) - return nil, err + return nil, ErrAgain } logger.Info("Put a task to fetch job info in background for jobId ", "jobId", jobId) From 1fd026863ab0bd71f7572cd8365bb0a4831f73dd Mon Sep 17 00:00:00 2001 From: fscnick Date: Fri, 19 Dec 2025 22:01:02 +0800 Subject: [PATCH 24/54] [RayJob] make cache immutable to avoid data race Signed-off-by: fscnick --- .../dashboardclient/dashboard_cache_client.go | 28 +++++++++++-------- 1 file changed, 16 insertions(+), 12 deletions(-) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index fb68fadfee5..18579636be0 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -46,9 +46,9 @@ var ( type ( Task func(taskCTX context.Context) bool JobInfoCache struct { - JobInfo *utiltypes.RayJobInfo - Err error - UpdateAt *time.Time + JobInfo *utiltypes.RayJobInfo + Err error + UpdatedAt *time.Time } workerPool struct { @@ -136,7 +136,7 @@ func (r *RayDashboardCacheClient) InitClient(ctx context.Context, client RayDash for _, key := range keys { cacheLock.Lock() if cached, ok := cacheStorage.Peek(key); ok { - if cached.UpdateAt.Before(expiredThreshold) { + if cached.UpdatedAt.Before(expiredThreshold) { cacheStorage.Remove(key) removed = append(removed, key) } @@ -175,7 +175,7 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) cacheLock.RUnlock() currentTime := time.Now() - placeholder := &JobInfoCache{Err: ErrAgain, UpdateAt: ¤tTime} + placeholder := &JobInfoCache{Err: ErrAgain, UpdatedAt: ¤tTime} // Put a placeholder in storage. The cache will be updated only if the placeholder exists. // The placeholder will be removed when StopJob or DeleteJob. @@ -206,10 +206,14 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) return true } } - jobInfoCache.JobInfo = jobInfo - jobInfoCache.Err = err currentTime := time.Now() - jobInfoCache.UpdateAt = ¤tTime + + // Make this cache immutable to avoid data race between pointer updates and read operations. + newJobInfoCache := &JobInfoCache{ + JobInfo: jobInfo, + Err: ErrAgain, + UpdatedAt: ¤tTime, + } cacheLock.Lock() if existed := cacheStorage.Contains(jobId); !existed { @@ -217,14 +221,14 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) logger.Info("The placeholder is removed before updating for jobId", "jobId", jobId) return false } - cacheStorage.Add(jobId, jobInfoCache) + cacheStorage.Add(jobId, newJobInfoCache) cacheLock.Unlock() - if jobInfoCache.JobInfo == nil { + if newJobInfoCache.JobInfo == nil { return true } - if rayv1.IsJobTerminal(jobInfoCache.JobInfo.JobStatus) { - logger.Info("The job reaches terminal status for jobId", "jobId", jobId, "status", jobInfoCache.JobInfo.JobStatus) + if rayv1.IsJobTerminal(newJobInfoCache.JobInfo.JobStatus) { + logger.Info("The job reaches terminal status for jobId", "jobId", jobId, "status", newJobInfoCache.JobInfo.JobStatus) return false } return true From dd6e750451db5e208153db93768cc44987659fb1 Mon Sep 17 00:00:00 2001 From: fscnick Date: Sat, 20 Dec 2025 09:33:13 +0800 Subject: [PATCH 25/54] [RayJob] remove unused function Signed-off-by: fscnick --- ray-operator/controllers/ray/suite_test.go | 4 ---- 1 file changed, 4 deletions(-) diff --git a/ray-operator/controllers/ray/suite_test.go b/ray-operator/controllers/ray/suite_test.go index eed4ffe5b05..1d9bfe12e83 100644 --- a/ray-operator/controllers/ray/suite_test.go +++ b/ray-operator/controllers/ray/suite_test.go @@ -65,10 +65,6 @@ func (testProvider TestClientProvider) GetHttpProxyClient(_ manager.Manager) fun } } -func (testProvider TestClientProvider) DoesUseBackgroundGoroutine() bool { - return false -} - func TestAPIs(t *testing.T) { RegisterFailHandler(Fail) From 6ec83729add5b7be24ab618125f4ac58fdb1fbd6 Mon Sep 17 00:00:00 2001 From: fscnick Date: Sat, 20 Dec 2025 09:35:27 +0800 Subject: [PATCH 26/54] [RayJob] remove cacheStorage lock Signed-off-by: fscnick --- .../dashboardclient/dashboard_cache_client.go | 23 ------------------- 1 file changed, 23 deletions(-) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index 18579636be0..a0324ce4524 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -40,7 +40,6 @@ var ( // singleton initCacheStorage sync.Once cacheStorage *lru.Cache[string, *JobInfoCache] - cacheLock sync.RWMutex ) type ( @@ -125,23 +124,19 @@ func (r *RayDashboardCacheClient) InitClient(ctx context.Context, client RayDash loggerForGC.Info("clean up goroutine exiting...") return case t := <-ticker.C: - cacheLock.RLock() keys := cacheStorage.Keys() - cacheLock.RUnlock() expiredThreshold := time.Now().Add(-cacheExpiry) loggerForGC.Info(fmt.Sprintf("Found %d keys to verify,", len(keys)), "expiredThreshold", expiredThreshold, "tick at", t) removed := keys[:0] for _, key := range keys { - cacheLock.Lock() if cached, ok := cacheStorage.Peek(key); ok { if cached.UpdatedAt.Before(expiredThreshold) { cacheStorage.Remove(key) removed = append(removed, key) } } - cacheLock.Unlock() } loggerForGC.Info(fmt.Sprintf("clean up %d cache.", len(removed)), "expiredThreshold", expiredThreshold, "removed keys", removed) } @@ -167,34 +162,25 @@ func (r *RayDashboardCacheClient) GetMultiApplicationStatus(ctx context.Context) func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) (*utiltypes.RayJobInfo, error) { logger := ctrl.LoggerFrom(ctx).WithName("RayDashboardCacheClient") - cacheLock.RLock() if cached, ok := cacheStorage.Get(jobId); ok { - cacheLock.RUnlock() return cached.JobInfo, cached.Err } - cacheLock.RUnlock() currentTime := time.Now() placeholder := &JobInfoCache{Err: ErrAgain, UpdatedAt: ¤tTime} // Put a placeholder in storage. The cache will be updated only if the placeholder exists. // The placeholder will be removed when StopJob or DeleteJob. - cacheLock.Lock() if cached, existed, _ := cacheStorage.PeekOrAdd(jobId, placeholder); existed { - cacheLock.Unlock() return cached.JobInfo, cached.Err } - cacheLock.Unlock() task := func(taskCTX context.Context) bool { - cacheLock.RLock() jobInfoCache, existed := cacheStorage.Get(jobId) if !existed { - cacheLock.RUnlock() logger.Info("The placeholder is removed for jobId", "jobId", jobId) return false } - cacheLock.RUnlock() var statusErr *k8serrors.StatusError jobInfo, err := r.client.GetJobInfo(taskCTX, jobId) @@ -215,14 +201,11 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) UpdatedAt: ¤tTime, } - cacheLock.Lock() if existed := cacheStorage.Contains(jobId); !existed { - cacheLock.Unlock() logger.Info("The placeholder is removed before updating for jobId", "jobId", jobId) return false } cacheStorage.Add(jobId, newJobInfoCache) - cacheLock.Unlock() if newJobInfoCache.JobInfo == nil { return true @@ -260,17 +243,11 @@ func (r *RayDashboardCacheClient) GetJobLog(ctx context.Context, jobName string) } func (r *RayDashboardCacheClient) StopJob(ctx context.Context, jobName string) error { - cacheLock.Lock() - defer cacheLock.Unlock() - cacheStorage.Remove(jobName) return r.client.StopJob(ctx, jobName) } func (r *RayDashboardCacheClient) DeleteJob(ctx context.Context, jobName string) error { - cacheLock.Lock() - defer cacheLock.Unlock() - cacheStorage.Remove(jobName) return r.client.DeleteJob(ctx, jobName) } From cf62d4d6cd7fe7c31d2b18fd4a681c6757bf1c2c Mon Sep 17 00:00:00 2001 From: fscnick Date: Sat, 20 Dec 2025 10:18:45 +0800 Subject: [PATCH 27/54] [RayJob] update cache error Signed-off-by: fscnick --- .../ray/utils/dashboardclient/dashboard_cache_client.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index a0324ce4524..ca97a6f7a1a 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -197,7 +197,7 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) // Make this cache immutable to avoid data race between pointer updates and read operations. newJobInfoCache := &JobInfoCache{ JobInfo: jobInfo, - Err: ErrAgain, + Err: err, UpdatedAt: ¤tTime, } From 94729a63baec76a64411df2ff43d4ecc82abd594 Mon Sep 17 00:00:00 2001 From: fscnick Date: Mon, 22 Dec 2025 21:21:04 +0800 Subject: [PATCH 28/54] [RayJob] If error on fetching job info, it removes from loop Signed-off-by: fscnick --- .../dashboardclient/dashboard_cache_client.go | 27 ++++++++++--------- 1 file changed, 14 insertions(+), 13 deletions(-) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index ca97a6f7a1a..885cf6df57d 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -8,7 +8,6 @@ import ( "time" lru "github.com/hashicorp/golang-lru/v2" - k8serrors "k8s.io/apimachinery/pkg/api/errors" ctrl "sigs.k8s.io/controller-runtime" rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1" @@ -163,6 +162,12 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) logger := ctrl.LoggerFrom(ctx).WithName("RayDashboardCacheClient") if cached, ok := cacheStorage.Get(jobId); ok { + if cached.Err != nil && errors.Is(cached.Err, ErrAgain) { + // Consume the error. + // If the RayJob is still exists, the next Reconcile iteration will put the task back for updating JobInfo + cacheStorage.Remove(jobId) + logger.Info("Consume the cached error for jobId", "jobId", jobId, "error", cached.Err) + } return cached.JobInfo, cached.Err } @@ -176,22 +181,12 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) } task := func(taskCTX context.Context) bool { - jobInfoCache, existed := cacheStorage.Get(jobId) - if !existed { + if _, existed := cacheStorage.Get(jobId); !existed { logger.Info("The placeholder is removed for jobId", "jobId", jobId) return false } - var statusErr *k8serrors.StatusError jobInfo, err := r.client.GetJobInfo(taskCTX, jobId) - if err != nil && !errors.As(err, &statusErr) { - if jobInfoCache.Err != nil && err.Error() == jobInfoCache.Err.Error() { - // The error is the same as last time, no need to update, just put the task to execute later. - // If the error is not fixed, eventually the cache will be expired and removed. - logger.Info("The error is the same as last time for jobId", "jobId", jobId, "error", err) - return true - } - } currentTime := time.Now() // Make this cache immutable to avoid data race between pointer updates and read operations. @@ -207,6 +202,12 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) } cacheStorage.Add(jobId, newJobInfoCache) + if err != nil { + // Exits the updating loop after getting an error. + // If the RayJob still exists, Reconcile will consume the error and put the JobId back to updating loop again. + logger.Error(err, "Failed to fetch job info for jobId", "jobId", jobId) + return false + } if newJobInfoCache.JobInfo == nil { return true } @@ -218,7 +219,7 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) } if err := pool.PutTask(task); err != nil { - logger.Error(err, "Cannot queue more jobInfo fetching tasks.", "jobId", jobId) + logger.Error(err, "Cannot queue more job info fetching tasks.", "jobId", jobId) return nil, ErrAgain } logger.Info("Put a task to fetch job info in background for jobId ", "jobId", jobId) From 88e6702dd7848d74c1333e492faf5bd240d548bb Mon Sep 17 00:00:00 2001 From: fscnick Date: Mon, 22 Dec 2025 23:22:18 +0800 Subject: [PATCH 29/54] [RayJob] task queue is extendable Signed-off-by: fscnick --- .../dashboardclient/dashboard_cache_client.go | 64 +++++++++++++++---- 1 file changed, 53 insertions(+), 11 deletions(-) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index 885cf6df57d..37eb7cba068 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -21,8 +21,7 @@ var ( const ( // TODO: make queue size and worker size configurable. - taskQueueSize = 128 - workerSize = 8 + workerSize = 8 queryInterval = 3 * time.Second @@ -50,13 +49,13 @@ type ( } workerPool struct { - taskQueue chan Task + taskQueue ExtendableChannel[Task] } ) -func (w *workerPool) init(ctx context.Context, taskQueueSize int, workerSize int, queryInterval time.Duration) { +func (w *workerPool) init(ctx context.Context, workerSize int, queryInterval time.Duration) { logger := ctrl.LoggerFrom(ctx).WithName("RayDashboardCacheClient").WithName("WorkerPool") - w.taskQueue = make(chan Task, taskQueueSize) + w.taskQueue = NewExtendableChannel[Task]() for i := 0; i < workerSize; i++ { go func(workerID int) { @@ -65,12 +64,12 @@ func (w *workerPool) init(ctx context.Context, taskQueueSize int, workerSize int case <-ctx.Done(): logger.Info("worker exiting...", "workerID", workerID) return - case task := <-w.taskQueue: + case task := <-w.taskQueue.Out: again := task(ctx) if again && ctx.Err() == nil { time.AfterFunc(queryInterval, func() { - w.taskQueue <- task + w.taskQueue.In <- task }) } } @@ -82,7 +81,7 @@ func (w *workerPool) init(ctx context.Context, taskQueueSize int, workerSize int func (w *workerPool) PutTask(task Task) error { select { - case w.taskQueue <- task: + case w.taskQueue.In <- task: return nil default: return ErrTaskQueueFull @@ -99,7 +98,7 @@ func (r *RayDashboardCacheClient) InitClient(ctx context.Context, client RayDash logger := ctrl.LoggerFrom(ctx).WithName("RayDashboardCacheClient") initWorkPool.Do(func() { - pool.init(ctx, taskQueueSize, workerSize, queryInterval) + pool.init(ctx, workerSize, queryInterval) }) initCacheStorage.Do(func() { @@ -162,7 +161,7 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) logger := ctrl.LoggerFrom(ctx).WithName("RayDashboardCacheClient") if cached, ok := cacheStorage.Get(jobId); ok { - if cached.Err != nil && errors.Is(cached.Err, ErrAgain) { + if cached.Err != nil && !errors.Is(cached.Err, ErrAgain) { // Consume the error. // If the RayJob is still exists, the next Reconcile iteration will put the task back for updating JobInfo cacheStorage.Remove(jobId) @@ -205,7 +204,7 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) if err != nil { // Exits the updating loop after getting an error. // If the RayJob still exists, Reconcile will consume the error and put the JobId back to updating loop again. - logger.Error(err, "Failed to fetch job info for jobId", "jobId", jobId) + logger.Info("Failed to fetch job info for jobId", "jobId", jobId, "error", err) return false } if newJobInfoCache.JobInfo == nil { @@ -252,3 +251,46 @@ func (r *RayDashboardCacheClient) DeleteJob(ctx context.Context, jobName string) cacheStorage.Remove(jobName) return r.client.DeleteJob(ctx, jobName) } + +type ExtendableChannel[T any] struct { + In chan<- T + Out <-chan T +} + +func NewExtendableChannel[T any]() ExtendableChannel[T] { + in := make(chan T) + out := make(chan T) + + go func() { + defer close(out) + var buffer []T + + for { + if len(buffer) == 0 { + v, ok := <-in + if !ok { + return + } + buffer = append(buffer, v) + } + + select { + case v, ok := <-in: + if !ok { + // Inbound closed; drain the buffer + for _, b := range buffer { + out <- b + } + return + } + + // TODO: this is not memory efficient. + buffer = append(buffer, v) + case out <- buffer[0]: + buffer = buffer[1:] + } + } + }() + + return ExtendableChannel[T]{In: in, Out: out} +} From 0d5dfe8f6941110e06d45c2994e093696ecc0748 Mon Sep 17 00:00:00 2001 From: fscnick Date: Tue, 23 Dec 2025 20:14:02 +0800 Subject: [PATCH 30/54] [RayJob] change slice to ring buffer Signed-off-by: fscnick --- .../dashboardclient/dashboard_cache_client.go | 79 ++++++++++++++++--- 1 file changed, 70 insertions(+), 9 deletions(-) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index 37eb7cba068..f68bc9eef7d 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -28,6 +28,8 @@ const ( // TODO: consider a proper size for accommodating the all live job info cacheSize = 10000 cacheExpiry = 10 * time.Minute + + initBufferSize = 128 ) var ( @@ -218,6 +220,8 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) } if err := pool.PutTask(task); err != nil { + // remove the placeholder because we cannot queue the task. + cacheStorage.Remove(jobId) logger.Error(err, "Cannot queue more job info fetching tasks.", "jobId", jobId) return nil, ErrAgain } @@ -263,34 +267,91 @@ func NewExtendableChannel[T any]() ExtendableChannel[T] { go func() { defer close(out) - var buffer []T + ringBuffer := *NewRingBuffer[T](initBufferSize) for { - if len(buffer) == 0 { + if ringBuffer.Len() == 0 { v, ok := <-in if !ok { return } - buffer = append(buffer, v) + ringBuffer.Push(v) } + // the above if-statement guarantees that ringBuffer.Len() > 0 here. + // so Pop() won't return an error. + t, _ := ringBuffer.Pop() select { case v, ok := <-in: if !ok { // Inbound closed; drain the buffer - for _, b := range buffer { - out <- b + for ringBuffer.Len() > 0 { + item, _ := ringBuffer.Pop() + out <- item } return } - // TODO: this is not memory efficient. - buffer = append(buffer, v) - case out <- buffer[0]: - buffer = buffer[1:] + ringBuffer.Push(v) + case out <- t: } } }() return ExtendableChannel[T]{In: in, Out: out} } + +type RingBuffer[T any] struct { + buffer []T + head int + tail int + size int // Current number of items + maxSize int // Maximum items in buffer +} + +func NewRingBuffer[T any](maxSize int) *RingBuffer[T] { + return &RingBuffer[T]{ + buffer: make([]T, maxSize), + maxSize: maxSize, + } +} + +func (r *RingBuffer[T]) Push(item T) { + if r.size == r.maxSize { + r.resize() + } + + r.buffer[r.head] = item + r.head = (r.head + 1) % r.maxSize + r.size++ +} + +func (r *RingBuffer[T]) resize() { + newBuffer := make([]T, r.maxSize*2) + for i := 0; i < r.size; i++ { + newBuffer[i] = r.buffer[(r.tail+i)%r.maxSize] + } + + r.buffer = newBuffer + r.tail = 0 + r.head = r.size + r.maxSize *= 2 +} + +func (r *RingBuffer[T]) Pop() (T, error) { + var zero T + if r.size == 0 { + return zero, errors.New("buffer is empty") + } + + item := r.buffer[r.tail] + r.buffer[r.tail] = zero + r.tail = (r.tail + 1) % r.maxSize + r.size-- + + return item, nil +} + +func (r *RingBuffer[T]) Len() int { + return r.size +} From 03bc5b17549a8451d9146c72ac0bc2427a2d2960 Mon Sep 17 00:00:00 2001 From: fscnick Date: Thu, 25 Dec 2025 10:53:17 +0800 Subject: [PATCH 31/54] [RayJob] rename PutTask to AddTask --- .../ray/utils/dashboardclient/dashboard_cache_client.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index f68bc9eef7d..e34026fc598 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -81,7 +81,7 @@ func (w *workerPool) init(ctx context.Context, workerSize int, queryInterval tim logger.Info(fmt.Sprintf("Initialize a worker pool with %d goroutine and queryInterval is %v.", workerSize, queryInterval)) } -func (w *workerPool) PutTask(task Task) error { +func (w *workerPool) AddTask(task Task) error { select { case w.taskQueue.In <- task: return nil @@ -129,6 +129,7 @@ func (r *RayDashboardCacheClient) InitClient(ctx context.Context, client RayDash expiredThreshold := time.Now().Add(-cacheExpiry) loggerForGC.Info(fmt.Sprintf("Found %d keys to verify,", len(keys)), "expiredThreshold", expiredThreshold, "tick at", t) + // zero allocate filtering removed := keys[:0] for _, key := range keys { if cached, ok := cacheStorage.Peek(key); ok { @@ -219,7 +220,7 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) return true } - if err := pool.PutTask(task); err != nil { + if err := pool.AddTask(task); err != nil { // remove the placeholder because we cannot queue the task. cacheStorage.Remove(jobId) logger.Error(err, "Cannot queue more job info fetching tasks.", "jobId", jobId) From 11db5a3256c4d8ac403b8b81116a6a28cd44ad80 Mon Sep 17 00:00:00 2001 From: fscnick Date: Fri, 26 Dec 2025 20:11:04 +0800 Subject: [PATCH 32/54] [RayJob] extendable channel use open source library Signed-off-by: fscnick --- go.mod | 1 + go.sum | 10 +- .../dashboardclient/dashboard_cache_client.go | 105 +----------------- ray-operator/go.mod | 1 + ray-operator/go.sum | 2 + 5 files changed, 13 insertions(+), 106 deletions(-) diff --git a/go.mod b/go.mod index 2d1ed19345c..dfbc611224d 100644 --- a/go.mod +++ b/go.mod @@ -90,6 +90,7 @@ require ( github.com/prometheus/procfs v0.17.0 // indirect github.com/robfig/cron/v3 v3.0.1 // indirect github.com/russross/blackfriday/v2 v2.1.0 // indirect + github.com/smallnest/chanx v1.2.0 // indirect github.com/x448/float16 v0.8.4 // indirect github.com/xlab/treeprint v1.2.0 // indirect go.opentelemetry.io/otel v1.37.0 // indirect diff --git a/go.sum b/go.sum index e469094b694..78993d53188 100644 --- a/go.sum +++ b/go.sum @@ -729,11 +729,11 @@ github.com/grpc-ecosystem/grpc-gateway/v2 v2.11.3/go.mod h1:o//XUCC/F+yRGJoPO/VU github.com/grpc-ecosystem/grpc-gateway/v2 v2.16.0/go.mod h1:YN5jB8ie0yfIUg6VvR9Kz84aCaG7AsGZnLjhHbUqwPg= github.com/grpc-ecosystem/grpc-gateway/v2 v2.26.3 h1:5ZPtiqj0JL5oKWmcsq4VMaAW5ukBEgSGXEN89zeH1Jo= github.com/grpc-ecosystem/grpc-gateway/v2 v2.26.3/go.mod h1:ndYquD05frm2vACXE1nsccT4oJzjhw2arTS2cpUD1PI= -github.com/hashicorp/golang-lru/v2 v2.0.7 h1:a+bsQ5rvGLjzHuww6tVxozPZFVghXaHOwFs4luLUK2k= -github.com/hashicorp/golang-lru/v2 v2.0.7/go.mod h1:QeFd9opnmA6QUJc5vARoKUSoFhyfM2/ZepoAG6RGpeM= github.com/hamba/avro/v2 v2.17.2/go.mod h1:Q9YK+qxAhtVrNqOhwlZTATLgLA8qxG2vtvkhK8fJ7Jo= github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= +github.com/hashicorp/golang-lru/v2 v2.0.7 h1:a+bsQ5rvGLjzHuww6tVxozPZFVghXaHOwFs4luLUK2k= +github.com/hashicorp/golang-lru/v2 v2.0.7/go.mod h1:QeFd9opnmA6QUJc5vARoKUSoFhyfM2/ZepoAG6RGpeM= github.com/hexops/gotextdiff v1.0.3/go.mod h1:pSWU5MAI3yDq+fZBTazCSJysOMbxWL1BSow5/V2vxeg= github.com/iancoleman/strcase v0.2.0/go.mod h1:iwCmte+B7n89clKwxIoIXy/HfoL7AsD47ZCWhYzw7ho= github.com/iancoleman/strcase v0.3.0/go.mod h1:iwCmte+B7n89clKwxIoIXy/HfoL7AsD47ZCWhYzw7ho= @@ -859,10 +859,10 @@ github.com/prometheus/common v0.65.0 h1:QDwzd+G1twt//Kwj/Ww6E9FQq1iVMmODnILtW1t2 github.com/prometheus/common v0.65.0/go.mod h1:0gZns+BLRQ3V6NdaerOhMbwwRbNh9hkGINtQAsP5GS8= github.com/prometheus/procfs v0.17.0 h1:FuLQ+05u4ZI+SS/w9+BWEM2TXiHKsUQ9TADiRH7DuK0= github.com/prometheus/procfs v0.17.0/go.mod h1:oPQLaDAMRbA+u8H5Pbfq+dl3VDAvHxMUOVhe0wYB2zw= -github.com/robfig/cron/v3 v3.0.1 h1:WdRxkvbJztn8LMz/QEvLN5sBU+xKpSqwwUO1Pjr4qDs= -github.com/robfig/cron/v3 v3.0.1/go.mod h1:eQICP3HwyT7UooqI/z+Ov+PtYAWygg1TEWWzGIFLtro= github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo= github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo= +github.com/robfig/cron/v3 v3.0.1 h1:WdRxkvbJztn8LMz/QEvLN5sBU+xKpSqwwUO1Pjr4qDs= +github.com/robfig/cron/v3 v3.0.1/go.mod h1:eQICP3HwyT7UooqI/z+Ov+PtYAWygg1TEWWzGIFLtro= github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rogpeppe/go-internal v1.9.0/go.mod h1:WtVeX8xhTBvf0smdhujwtBcq4Qrzq/fJaraNFVN+nFs= @@ -883,6 +883,8 @@ github.com/sergi/go-diff v1.2.0 h1:XU+rvMAioB0UC3q1MFrIQy4Vo5/4VsRDQQXHsEya6xQ= github.com/sergi/go-diff v1.2.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/sirupsen/logrus v1.9.3/go.mod h1:naHLuLoDiP4jHNo9R0sCBMtWGeIprob74mVsIT4qYEQ= +github.com/smallnest/chanx v1.2.0 h1:RLyldZBbQZ4O0dSvdkTMHo4+mDw20Bc1jXXTHf+ymZo= +github.com/smallnest/chanx v1.2.0/go.mod h1:+4nWMF0+CqEcU74SnX2NxaGqZ8zX4pcQ8Jcs77DbX5A= github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= github.com/spf13/afero v1.3.3/go.mod h1:5KUK8ByomD5Ti5Artl0RtHeI5pTF7MIDuXL3yY520V4= github.com/spf13/afero v1.6.0/go.mod h1:Ai8FlHk4v/PARR026UzYexafAt9roJ7LcLMAmO6Z93I= diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index e34026fc598..c1af3576a6b 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -8,6 +8,7 @@ import ( "time" lru "github.com/hashicorp/golang-lru/v2" + "github.com/smallnest/chanx" ctrl "sigs.k8s.io/controller-runtime" rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1" @@ -51,13 +52,13 @@ type ( } workerPool struct { - taskQueue ExtendableChannel[Task] + taskQueue *chanx.UnboundedChan[Task] } ) func (w *workerPool) init(ctx context.Context, workerSize int, queryInterval time.Duration) { logger := ctrl.LoggerFrom(ctx).WithName("RayDashboardCacheClient").WithName("WorkerPool") - w.taskQueue = NewExtendableChannel[Task]() + w.taskQueue = chanx.NewUnboundedChanSize[Task](ctx, 0, 0, initBufferSize) for i := 0; i < workerSize; i++ { go func(workerID int) { @@ -256,103 +257,3 @@ func (r *RayDashboardCacheClient) DeleteJob(ctx context.Context, jobName string) cacheStorage.Remove(jobName) return r.client.DeleteJob(ctx, jobName) } - -type ExtendableChannel[T any] struct { - In chan<- T - Out <-chan T -} - -func NewExtendableChannel[T any]() ExtendableChannel[T] { - in := make(chan T) - out := make(chan T) - - go func() { - defer close(out) - ringBuffer := *NewRingBuffer[T](initBufferSize) - - for { - if ringBuffer.Len() == 0 { - v, ok := <-in - if !ok { - return - } - ringBuffer.Push(v) - } - - // the above if-statement guarantees that ringBuffer.Len() > 0 here. - // so Pop() won't return an error. - t, _ := ringBuffer.Pop() - select { - case v, ok := <-in: - if !ok { - // Inbound closed; drain the buffer - for ringBuffer.Len() > 0 { - item, _ := ringBuffer.Pop() - out <- item - } - return - } - - ringBuffer.Push(v) - case out <- t: - } - } - }() - - return ExtendableChannel[T]{In: in, Out: out} -} - -type RingBuffer[T any] struct { - buffer []T - head int - tail int - size int // Current number of items - maxSize int // Maximum items in buffer -} - -func NewRingBuffer[T any](maxSize int) *RingBuffer[T] { - return &RingBuffer[T]{ - buffer: make([]T, maxSize), - maxSize: maxSize, - } -} - -func (r *RingBuffer[T]) Push(item T) { - if r.size == r.maxSize { - r.resize() - } - - r.buffer[r.head] = item - r.head = (r.head + 1) % r.maxSize - r.size++ -} - -func (r *RingBuffer[T]) resize() { - newBuffer := make([]T, r.maxSize*2) - for i := 0; i < r.size; i++ { - newBuffer[i] = r.buffer[(r.tail+i)%r.maxSize] - } - - r.buffer = newBuffer - r.tail = 0 - r.head = r.size - r.maxSize *= 2 -} - -func (r *RingBuffer[T]) Pop() (T, error) { - var zero T - if r.size == 0 { - return zero, errors.New("buffer is empty") - } - - item := r.buffer[r.tail] - r.buffer[r.tail] = zero - r.tail = (r.tail + 1) % r.maxSize - r.size-- - - return item, nil -} - -func (r *RingBuffer[T]) Len() int { - return r.size -} diff --git a/ray-operator/go.mod b/ray-operator/go.mod index 8274e5f20ce..04490363608 100644 --- a/ray-operator/go.mod +++ b/ray-operator/go.mod @@ -15,6 +15,7 @@ require ( github.com/orcaman/concurrent-map/v2 v2.0.1 github.com/prometheus/client_golang v1.23.0 github.com/robfig/cron/v3 v3.0.1 + github.com/smallnest/chanx v1.2.0 github.com/spf13/pflag v1.0.7 github.com/stretchr/testify v1.11.0 go.uber.org/mock v0.5.2 diff --git a/ray-operator/go.sum b/ray-operator/go.sum index 7883996539b..9a69a6c5f10 100644 --- a/ray-operator/go.sum +++ b/ray-operator/go.sum @@ -114,6 +114,8 @@ github.com/robfig/cron/v3 v3.0.1 h1:WdRxkvbJztn8LMz/QEvLN5sBU+xKpSqwwUO1Pjr4qDs= github.com/robfig/cron/v3 v3.0.1/go.mod h1:eQICP3HwyT7UooqI/z+Ov+PtYAWygg1TEWWzGIFLtro= github.com/rogpeppe/go-internal v1.13.1 h1:KvO1DLK/DRN07sQ1LQKScxyZJuNnedQ5/wKSR38lUII= github.com/rogpeppe/go-internal v1.13.1/go.mod h1:uMEvuHeurkdAXX61udpOXGD/AzZDWNMNyH2VO9fmH0o= +github.com/smallnest/chanx v1.2.0 h1:RLyldZBbQZ4O0dSvdkTMHo4+mDw20Bc1jXXTHf+ymZo= +github.com/smallnest/chanx v1.2.0/go.mod h1:+4nWMF0+CqEcU74SnX2NxaGqZ8zX4pcQ8Jcs77DbX5A= github.com/spf13/pflag v1.0.7 h1:vN6T9TfwStFPFM5XzjsvmzZkLuaLX+HS+0SeFLRgU6M= github.com/spf13/pflag v1.0.7/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= From d2e13db9cebc63e03914e848efa064ecc3e7c580 Mon Sep 17 00:00:00 2001 From: fscnick Date: Fri, 26 Dec 2025 20:53:33 +0800 Subject: [PATCH 33/54] [RayJob] async job info query use feature gate instead Signed-off-by: fscnick --- apiserver/pkg/server/ray_job_submission_service_server.go | 2 +- helm-chart/kuberay-operator/README.md | 1 - helm-chart/kuberay-operator/templates/deployment.yaml | 3 --- helm-chart/kuberay-operator/values.yaml | 4 ---- ray-operator/apis/config/v1alpha1/configuration_types.go | 6 +----- ray-operator/controllers/ray/utils/util.go | 4 ++-- ray-operator/main.go | 3 --- ray-operator/pkg/features/features.go | 8 ++++++++ ray-operator/test/sampleyaml/support.go | 2 +- 9 files changed, 13 insertions(+), 20 deletions(-) diff --git a/apiserver/pkg/server/ray_job_submission_service_server.go b/apiserver/pkg/server/ray_job_submission_service_server.go index 28c07b57a04..1873a94ce7e 100644 --- a/apiserver/pkg/server/ray_job_submission_service_server.go +++ b/apiserver/pkg/server/ray_job_submission_service_server.go @@ -41,7 +41,7 @@ type RayJobSubmissionServiceServer struct { // Create RayJobSubmissionServiceServer func NewRayJobSubmissionServiceServer(clusterServer *ClusterServer, options *RayJobSubmissionServiceServerOptions) *RayJobSubmissionServiceServer { zl := zerolog.New(os.Stdout).Level(zerolog.DebugLevel) - return &RayJobSubmissionServiceServer{clusterServer: clusterServer, options: options, log: zerologr.New(&zl).WithName("jobsubmissionservice"), dashboardClientFunc: utils.GetRayDashboardClientFunc(context.Background(), nil, false, false)} + return &RayJobSubmissionServiceServer{clusterServer: clusterServer, options: options, log: zerologr.New(&zl).WithName("jobsubmissionservice"), dashboardClientFunc: utils.GetRayDashboardClientFunc(context.Background(), nil, false)} } // Submit Ray job diff --git a/helm-chart/kuberay-operator/README.md b/helm-chart/kuberay-operator/README.md index 9a4a463954c..85fb7c51a39 100644 --- a/helm-chart/kuberay-operator/README.md +++ b/helm-chart/kuberay-operator/README.md @@ -192,7 +192,6 @@ spec: | rbacEnable | bool | `true` | If rbacEnable is set to false, no RBAC resources will be created, including the Role for leader election, the Role for Pods and Services, and so on. | | crNamespacedRbacEnable | bool | `true` | When crNamespacedRbacEnable is set to true, the KubeRay operator will create a Role for RayCluster preparation (e.g., Pods, Services) and a corresponding RoleBinding for each namespace listed in the "watchNamespace" parameter. Please note that even if crNamespacedRbacEnable is set to false, the Role and RoleBinding for leader election will still be created. Note: (1) This variable is only effective when rbacEnable and singleNamespaceInstall are both set to true. (2) In most cases, it should be set to true, unless you are using a Kubernetes cluster managed by GitOps tools such as ArgoCD. | | singleNamespaceInstall | bool | `false` | When singleNamespaceInstall is true: - Install namespaced RBAC resources such as Role and RoleBinding instead of cluster-scoped ones like ClusterRole and ClusterRoleBinding so that the chart can be installed by users with permissions restricted to a single namespace. (Please note that this excludes the CRDs, which can only be installed at the cluster scope.) - If "watchNamespace" is not set, the KubeRay operator will, by default, only listen to resource events within its own namespace. | -| asyncJobInfoQuery | bool | `false` | When AsyncJobInfoQuery is set to true: the KubeRay operator will use background goroutines to fetch the jobInfo from Ray clusters for RayJob status updates. | | env | string | `nil` | Environment variables. | | resources | object | `{"limits":{"cpu":"100m","memory":"512Mi"}}` | Resource requests and limits for containers. | | livenessProbe.initialDelaySeconds | int | `10` | | diff --git a/helm-chart/kuberay-operator/templates/deployment.yaml b/helm-chart/kuberay-operator/templates/deployment.yaml index 5ca5d79f32e..337dcc60bae 100644 --- a/helm-chart/kuberay-operator/templates/deployment.yaml +++ b/helm-chart/kuberay-operator/templates/deployment.yaml @@ -152,9 +152,6 @@ spec: {{- $argList = append $argList (printf "--burst=%v" .Values.kubeClient.burst) -}} {{- end -}} {{- end -}} - {{- if hasKey .Values "asyncJobInfoQuery" -}} - {{- $argList = append $argList (printf "--async-jobinfo-query=%t" .Values.asyncJobInfoQuery) -}} - {{- end -}} {{- (printf "\n") -}} {{- $argList | toYaml | indent 12 }} ports: diff --git a/helm-chart/kuberay-operator/values.yaml b/helm-chart/kuberay-operator/values.yaml index 66e72047c62..c566c6525c6 100644 --- a/helm-chart/kuberay-operator/values.yaml +++ b/helm-chart/kuberay-operator/values.yaml @@ -188,10 +188,6 @@ crNamespacedRbacEnable: true # to resource events within its own namespace. singleNamespaceInstall: false -# -- When AsyncJobInfoQuery is set to true: -# the KubeRay operator will use background goroutines to fetch the jobInfo from Ray clusters for RayJob status updates. -asyncJobInfoQuery: false - # The KubeRay operator will watch the custom resources in the namespaces listed in the "watchNamespace" parameter. # watchNamespace: # - n1 diff --git a/ray-operator/apis/config/v1alpha1/configuration_types.go b/ray-operator/apis/config/v1alpha1/configuration_types.go index 90dcdcec856..5554fbdf045 100644 --- a/ray-operator/apis/config/v1alpha1/configuration_types.go +++ b/ray-operator/apis/config/v1alpha1/configuration_types.go @@ -88,14 +88,10 @@ type Configuration struct { // EnableMetrics indicates whether KubeRay operator should emit control plane metrics. EnableMetrics bool `json:"enableMetrics,omitempty"` - - // AsyncJobInfoQuery indicates that it wil use goroutine to fetch the job info from ray dashboard and - // store the job info in the cache - AsyncJobInfoQuery bool `json:"useBackgroundGoroutine,omitempty"` } func (config Configuration) GetDashboardClient(ctx context.Context, mgr manager.Manager) func(rayCluster *rayv1.RayCluster, url string) (dashboardclient.RayDashboardClientInterface, error) { - return utils.GetRayDashboardClientFunc(ctx, mgr, config.UseKubernetesProxy, config.AsyncJobInfoQuery) + return utils.GetRayDashboardClientFunc(ctx, mgr, config.UseKubernetesProxy) } func (config Configuration) GetHttpProxyClient(mgr manager.Manager) func(hostIp, podNamespace, podName string, port int) utils.RayHttpProxyClientInterface { diff --git a/ray-operator/controllers/ray/utils/util.go b/ray-operator/controllers/ray/utils/util.go index 0519c1456ca..7549660032c 100644 --- a/ray-operator/controllers/ray/utils/util.go +++ b/ray-operator/controllers/ray/utils/util.go @@ -920,7 +920,7 @@ func FetchHeadServiceURL(ctx context.Context, cli client.Client, rayCluster *ray return headServiceURL, nil } -func GetRayDashboardClientFunc(ctx context.Context, mgr manager.Manager, useKubernetesProxy bool, asyncJobInfoQuery bool) func(rayCluster *rayv1.RayCluster, url string) (dashboardclient.RayDashboardClientInterface, error) { +func GetRayDashboardClientFunc(ctx context.Context, mgr manager.Manager, useKubernetesProxy bool) func(rayCluster *rayv1.RayCluster, url string) (dashboardclient.RayDashboardClientInterface, error) { return func(rayCluster *rayv1.RayCluster, url string) (dashboardclient.RayDashboardClientInterface, error) { dashboardClient := &dashboardclient.RayDashboardClient{} var authToken string @@ -969,7 +969,7 @@ func GetRayDashboardClientFunc(ctx context.Context, mgr manager.Manager, useKube }, "http://"+url, authToken) } - if asyncJobInfoQuery { + if features.Enabled(features.AsyncJobInfoQuery) { dashboardCachedClient := &dashboardclient.RayDashboardCacheClient{} dashboardCachedClient.InitClient(ctx, dashboardClient) return dashboardCachedClient, nil diff --git a/ray-operator/main.go b/ray-operator/main.go index 7e13d4b3d0c..c60e862a1ad 100644 --- a/ray-operator/main.go +++ b/ray-operator/main.go @@ -73,7 +73,6 @@ func main() { var enableMetrics bool var qps float64 var burst int - var asyncJobInfoQuery bool // TODO: remove flag-based config once Configuration API graduates to v1. flag.StringVar(&metricsAddr, "metrics-addr", configapi.DefaultMetricsAddr, "The address the metric endpoint binds to.") @@ -107,7 +106,6 @@ func main() { flag.BoolVar(&enableMetrics, "enable-metrics", false, "Enable the emission of control plane metrics.") flag.Float64Var(&qps, "qps", float64(configapi.DefaultQPS), "The QPS value for the client communicating with the Kubernetes API server.") flag.IntVar(&burst, "burst", configapi.DefaultBurst, "The maximum burst for throttling requests from this client to the Kubernetes API server.") - flag.BoolVar(&asyncJobInfoQuery, "async-jobinfo-query", false, "Enable the background goroutine for fetching job info in RayJob.") opts := k8szap.Options{ TimeEncoder: zapcore.ISO8601TimeEncoder, @@ -140,7 +138,6 @@ func main() { config.EnableMetrics = enableMetrics config.QPS = &qps config.Burst = &burst - config.AsyncJobInfoQuery = asyncJobInfoQuery } stdoutEncoder, err := newLogEncoder(logStdoutEncoder) diff --git a/ray-operator/pkg/features/features.go b/ray-operator/pkg/features/features.go index 33215fcd880..6a9bb622fd5 100644 --- a/ray-operator/pkg/features/features.go +++ b/ray-operator/pkg/features/features.go @@ -46,6 +46,13 @@ const ( // // Enables RayCronJob controller for scheduled RayJob execution. RayCronJob featuregate.Feature = "RayCronJob" + + // owner: @fscnick + // rep: N/A + // alpha: v1.6 + // + // Enables asynchronous job info querying. + AsyncJobInfoQuery featuregate.Feature = "AsyncJobInfoQuery" ) func init() { @@ -58,6 +65,7 @@ var defaultFeatureGates = map[featuregate.Feature]featuregate.FeatureSpec{ RayMultiHostIndexing: {Default: false, PreRelease: featuregate.Alpha}, RayServiceIncrementalUpgrade: {Default: false, PreRelease: featuregate.Alpha}, RayCronJob: {Default: false, PreRelease: featuregate.Alpha}, + AsyncJobInfoQuery: {Default: false, PreRelease: featuregate.Alpha}, } // SetFeatureGateDuringTest is a helper method to override feature gates in tests. diff --git a/ray-operator/test/sampleyaml/support.go b/ray-operator/test/sampleyaml/support.go index e330b08e63c..639eb410fad 100644 --- a/ray-operator/test/sampleyaml/support.go +++ b/ray-operator/test/sampleyaml/support.go @@ -75,7 +75,7 @@ func QueryDashboardGetAppStatus(t Test, rayCluster *rayv1.RayCluster) func(Gomeg g.Expect(err).ToNot(HaveOccurred()) url := fmt.Sprintf("127.0.0.1:%d", localPort) - rayDashboardClientFunc := utils.GetRayDashboardClientFunc(t.Ctx(), nil, false, false) + rayDashboardClientFunc := utils.GetRayDashboardClientFunc(t.Ctx(), nil, false) rayDashboardClient, err := rayDashboardClientFunc(rayCluster, url) g.Expect(err).ToNot(HaveOccurred()) serveDetails, err := rayDashboardClient.GetServeDetails(t.Ctx()) From 1f90762a7b90aab6b96ab10f1c9ed6c35d4b7556 Mon Sep 17 00:00:00 2001 From: fscnick Date: Fri, 26 Dec 2025 21:02:01 +0800 Subject: [PATCH 34/54] [RayJob] add comment for task Signed-off-by: fscnick --- .../ray/utils/dashboardclient/dashboard_cache_client.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index c1af3576a6b..87ec63421bb 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -44,6 +44,7 @@ var ( ) type ( + // Task defines a unit of work for the worker pool and the return value indicate if it should re-queue or not. Task func(taskCTX context.Context) bool JobInfoCache struct { JobInfo *utiltypes.RayJobInfo @@ -183,7 +184,7 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) return cached.JobInfo, cached.Err } - task := func(taskCTX context.Context) bool { + var task Task = func(taskCTX context.Context) bool { if _, existed := cacheStorage.Get(jobId); !existed { logger.Info("The placeholder is removed for jobId", "jobId", jobId) return false From 2765c2a4c62ace9570f182829b986305ba252809 Mon Sep 17 00:00:00 2001 From: fscnick Date: Tue, 30 Dec 2025 21:42:12 +0800 Subject: [PATCH 35/54] [RayJob] rename function signature of worker pool init function Signed-off-by: fscnick --- .../dashboardclient/dashboard_cache_client.go | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index 87ec63421bb..cc88548c5a7 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -57,11 +57,11 @@ type ( } ) -func (w *workerPool) init(ctx context.Context, workerSize int, queryInterval time.Duration) { +func (w *workerPool) start(ctx context.Context, numWorkers int, requeueDelay time.Duration) { logger := ctrl.LoggerFrom(ctx).WithName("RayDashboardCacheClient").WithName("WorkerPool") w.taskQueue = chanx.NewUnboundedChanSize[Task](ctx, 0, 0, initBufferSize) - for i := 0; i < workerSize; i++ { + for i := 0; i < numWorkers; i++ { go func(workerID int) { for { select { @@ -69,10 +69,10 @@ func (w *workerPool) init(ctx context.Context, workerSize int, queryInterval tim logger.Info("worker exiting...", "workerID", workerID) return case task := <-w.taskQueue.Out: - again := task(ctx) + shouldRequeue := task(ctx) - if again && ctx.Err() == nil { - time.AfterFunc(queryInterval, func() { + if shouldRequeue && ctx.Err() == nil { + time.AfterFunc(requeueDelay, func() { w.taskQueue.In <- task }) } @@ -80,7 +80,7 @@ func (w *workerPool) init(ctx context.Context, workerSize int, queryInterval tim } }(i) } - logger.Info(fmt.Sprintf("Initialize a worker pool with %d goroutine and queryInterval is %v.", workerSize, queryInterval)) + logger.Info(fmt.Sprintf("Initialize a worker pool with %d goroutines and requeueDelay is %v.", numWorkers, requeueDelay)) } func (w *workerPool) AddTask(task Task) error { @@ -102,7 +102,7 @@ func (r *RayDashboardCacheClient) InitClient(ctx context.Context, client RayDash logger := ctrl.LoggerFrom(ctx).WithName("RayDashboardCacheClient") initWorkPool.Do(func() { - pool.init(ctx, workerSize, queryInterval) + pool.start(ctx, workerSize, queryInterval) }) initCacheStorage.Do(func() { From c3336f28d601bfea7ac91fe07e0c3d6e0dd5ec17 Mon Sep 17 00:00:00 2001 From: fscnick Date: Tue, 30 Dec 2025 21:53:02 +0800 Subject: [PATCH 36/54] [RayJob] change ErrAgain error message Signed-off-by: fscnick --- .../ray/utils/dashboardclient/dashboard_cache_client.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index cc88548c5a7..d8c939207c5 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -16,7 +16,7 @@ import ( ) var ( - ErrAgain = errors.New("EAGAIN") + ErrAgain = errors.New("data is not ready, try again later") ErrTaskQueueFull = errors.New("task queue is full") ) From 09b501ccd278da58238a38e1395c72546e1d5a8c Mon Sep 17 00:00:00 2001 From: fscnick Date: Tue, 30 Dec 2025 22:04:41 +0800 Subject: [PATCH 37/54] [RayJob] fix lint error Signed-off-by: fscnick --- .../ray/utils/dashboardclient/dashboard_cache_client.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index d8c939207c5..993993c4c56 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -61,7 +61,7 @@ func (w *workerPool) start(ctx context.Context, numWorkers int, requeueDelay tim logger := ctrl.LoggerFrom(ctx).WithName("RayDashboardCacheClient").WithName("WorkerPool") w.taskQueue = chanx.NewUnboundedChanSize[Task](ctx, 0, 0, initBufferSize) - for i := 0; i < numWorkers; i++ { + for i := range numWorkers { go func(workerID int) { for { select { From 5d5bbde846441c6bc265d37eb54fbf245209bd3a Mon Sep 17 00:00:00 2001 From: fscnick Date: Tue, 30 Dec 2025 22:53:13 +0800 Subject: [PATCH 38/54] [RayJob] change back to EAGAIN Signed-off-by: fscnick --- .../ray/utils/dashboardclient/dashboard_cache_client.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index 993993c4c56..129112d500b 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -16,7 +16,9 @@ import ( ) var ( - ErrAgain = errors.New("data is not ready, try again later") + // ErrAgain EAGAIN means "there is no data available right now, try again later" + // https://stackoverflow.com/questions/4058368/what-does-eagain-mean + ErrAgain = errors.New("EAGAIN") ErrTaskQueueFull = errors.New("task queue is full") ) From 8475d1f6658e7c65d0f20913078b7a4b91b71faf Mon Sep 17 00:00:00 2001 From: fscnick Date: Thu, 1 Jan 2026 11:19:00 +0800 Subject: [PATCH 39/54] [RayJob] remove queue size from todo comment Signed-off-by: fscnick --- .../ray/utils/dashboardclient/dashboard_cache_client.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index 129112d500b..59c1d73e912 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -23,7 +23,7 @@ var ( ) const ( - // TODO: make queue size and worker size configurable. + // TODO: make worker size configurable. workerSize = 8 queryInterval = 3 * time.Second From a52f2a9f95793f71af08315446200c45968b44b2 Mon Sep 17 00:00:00 2001 From: fscnick Date: Thu, 1 Jan 2026 14:29:20 +0800 Subject: [PATCH 40/54] [RayJob] rename queue full error Signed-off-by: fscnick --- .../ray/utils/dashboardclient/dashboard_cache_client.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index 59c1d73e912..1b34e94b7bd 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -18,8 +18,8 @@ import ( var ( // ErrAgain EAGAIN means "there is no data available right now, try again later" // https://stackoverflow.com/questions/4058368/what-does-eagain-mean - ErrAgain = errors.New("EAGAIN") - ErrTaskQueueFull = errors.New("task queue is full") + ErrAgain = errors.New("EAGAIN") + ErrTaskQueueTemporarilyUnavailable = errors.New("task queue is temporarily unavailable") ) const ( @@ -90,7 +90,7 @@ func (w *workerPool) AddTask(task Task) error { case w.taskQueue.In <- task: return nil default: - return ErrTaskQueueFull + return ErrTaskQueueTemporarilyUnavailable } } From 4d4ed198328f32995236c72755d4b6904c85044d Mon Sep 17 00:00:00 2001 From: fscnick Date: Thu, 1 Jan 2026 15:28:37 +0800 Subject: [PATCH 41/54] [RayJob] add lock to avoid data race Signed-off-by: fscnick --- .../dashboardclient/dashboard_cache_client.go | 25 +++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index 1b34e94b7bd..e1dc260c1dd 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -43,6 +43,7 @@ var ( // singleton initCacheStorage sync.Once cacheStorage *lru.Cache[string, *JobInfoCache] + rwLock sync.RWMutex ) type ( @@ -128,7 +129,9 @@ func (r *RayDashboardCacheClient) InitClient(ctx context.Context, client RayDash loggerForGC.Info("clean up goroutine exiting...") return case t := <-ticker.C: + rwLock.RLock() keys := cacheStorage.Keys() + rwLock.RUnlock() expiredThreshold := time.Now().Add(-cacheExpiry) loggerForGC.Info(fmt.Sprintf("Found %d keys to verify,", len(keys)), "expiredThreshold", expiredThreshold, "tick at", t) @@ -136,12 +139,14 @@ func (r *RayDashboardCacheClient) InitClient(ctx context.Context, client RayDash // zero allocate filtering removed := keys[:0] for _, key := range keys { + rwLock.Lock() if cached, ok := cacheStorage.Peek(key); ok { if cached.UpdatedAt.Before(expiredThreshold) { cacheStorage.Remove(key) removed = append(removed, key) } } + rwLock.Unlock() } loggerForGC.Info(fmt.Sprintf("clean up %d cache.", len(removed)), "expiredThreshold", expiredThreshold, "removed keys", removed) } @@ -167,6 +172,7 @@ func (r *RayDashboardCacheClient) GetMultiApplicationStatus(ctx context.Context) func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) (*utiltypes.RayJobInfo, error) { logger := ctrl.LoggerFrom(ctx).WithName("RayDashboardCacheClient") + rwLock.Lock() if cached, ok := cacheStorage.Get(jobId); ok { if cached.Err != nil && !errors.Is(cached.Err, ErrAgain) { // Consume the error. @@ -174,23 +180,31 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) cacheStorage.Remove(jobId) logger.Info("Consume the cached error for jobId", "jobId", jobId, "error", cached.Err) } + rwLock.Unlock() return cached.JobInfo, cached.Err } + rwLock.Unlock() currentTime := time.Now() placeholder := &JobInfoCache{Err: ErrAgain, UpdatedAt: ¤tTime} // Put a placeholder in storage. The cache will be updated only if the placeholder exists. // The placeholder will be removed when StopJob or DeleteJob. + rwLock.Lock() if cached, existed, _ := cacheStorage.PeekOrAdd(jobId, placeholder); existed { + rwLock.Unlock() return cached.JobInfo, cached.Err } + rwLock.Unlock() var task Task = func(taskCTX context.Context) bool { + rwLock.RLock() if _, existed := cacheStorage.Get(jobId); !existed { logger.Info("The placeholder is removed for jobId", "jobId", jobId) + rwLock.RUnlock() return false } + rwLock.RUnlock() jobInfo, err := r.client.GetJobInfo(taskCTX, jobId) currentTime := time.Now() @@ -202,11 +216,14 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) UpdatedAt: ¤tTime, } + rwLock.Lock() if existed := cacheStorage.Contains(jobId); !existed { logger.Info("The placeholder is removed before updating for jobId", "jobId", jobId) + rwLock.Unlock() return false } cacheStorage.Add(jobId, newJobInfoCache) + rwLock.Unlock() if err != nil { // Exits the updating loop after getting an error. @@ -226,7 +243,9 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) if err := pool.AddTask(task); err != nil { // remove the placeholder because we cannot queue the task. + rwLock.Lock() cacheStorage.Remove(jobId) + rwLock.Unlock() logger.Error(err, "Cannot queue more job info fetching tasks.", "jobId", jobId) return nil, ErrAgain } @@ -252,11 +271,17 @@ func (r *RayDashboardCacheClient) GetJobLog(ctx context.Context, jobName string) } func (r *RayDashboardCacheClient) StopJob(ctx context.Context, jobName string) error { + rwLock.Lock() cacheStorage.Remove(jobName) + rwLock.Unlock() + return r.client.StopJob(ctx, jobName) } func (r *RayDashboardCacheClient) DeleteJob(ctx context.Context, jobName string) error { + rwLock.Lock() cacheStorage.Remove(jobName) + rwLock.Unlock() + return r.client.DeleteJob(ctx, jobName) } From 17b75d1af6d7a9007f23cfd5f25b3d84a90eb21f Mon Sep 17 00:00:00 2001 From: fscnick Date: Wed, 7 Jan 2026 22:37:51 +0800 Subject: [PATCH 42/54] [RayJob] requeue check context has canceled or not Signed-off-by: fscnick --- .../ray/utils/dashboardclient/dashboard_cache_client.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index e1dc260c1dd..616add970cb 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -76,7 +76,10 @@ func (w *workerPool) start(ctx context.Context, numWorkers int, requeueDelay tim if shouldRequeue && ctx.Err() == nil { time.AfterFunc(requeueDelay, func() { - w.taskQueue.In <- task + if ctx.Err() == nil { + // Make sure the context is still valid before re-queuing the task. + w.taskQueue.In <- task + } }) } } From 67df23237bccc28d6d9b7b35b0da68a86fa836b7 Mon Sep 17 00:00:00 2001 From: fscnick Date: Wed, 7 Jan 2026 23:22:42 +0800 Subject: [PATCH 43/54] [RayJob] add cluster name on the cache key Signed-off-by: fscnick --- .../dashboardclient/dashboard_cache_client.go | 44 ++++++++++++------- ray-operator/controllers/ray/utils/util.go | 6 ++- 2 files changed, 32 insertions(+), 18 deletions(-) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index 616add970cb..5713e1c2174 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -9,6 +9,7 @@ import ( lru "github.com/hashicorp/golang-lru/v2" "github.com/smallnest/chanx" + "k8s.io/apimachinery/pkg/types" ctrl "sigs.k8s.io/controller-runtime" rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1" @@ -101,12 +102,15 @@ func (w *workerPool) AddTask(task Task) error { var _ RayDashboardClientInterface = (*RayDashboardCacheClient)(nil) type RayDashboardCacheClient struct { - client RayDashboardClientInterface + client RayDashboardClientInterface + namespacedName types.NamespacedName } -func (r *RayDashboardCacheClient) InitClient(ctx context.Context, client RayDashboardClientInterface) { +func (r *RayDashboardCacheClient) InitClient(ctx context.Context, namespacedName types.NamespacedName, client RayDashboardClientInterface) { logger := ctrl.LoggerFrom(ctx).WithName("RayDashboardCacheClient") + r.namespacedName = namespacedName + initWorkPool.Do(func() { pool.start(ctx, workerSize, queryInterval) }) @@ -176,12 +180,12 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) logger := ctrl.LoggerFrom(ctx).WithName("RayDashboardCacheClient") rwLock.Lock() - if cached, ok := cacheStorage.Get(jobId); ok { + if cached, ok := cacheStorage.Get(cacheKey(r.namespacedName, jobId)); ok { if cached.Err != nil && !errors.Is(cached.Err, ErrAgain) { // Consume the error. // If the RayJob is still exists, the next Reconcile iteration will put the task back for updating JobInfo - cacheStorage.Remove(jobId) - logger.Info("Consume the cached error for jobId", "jobId", jobId, "error", cached.Err) + cacheStorage.Remove(cacheKey(r.namespacedName, jobId)) + logger.Info("Consume the cached error for jobId", "jobId", jobId, "error", cached.Err, "cacheKey", cacheKey(r.namespacedName, jobId)) } rwLock.Unlock() return cached.JobInfo, cached.Err @@ -194,7 +198,7 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) // Put a placeholder in storage. The cache will be updated only if the placeholder exists. // The placeholder will be removed when StopJob or DeleteJob. rwLock.Lock() - if cached, existed, _ := cacheStorage.PeekOrAdd(jobId, placeholder); existed { + if cached, existed, _ := cacheStorage.PeekOrAdd(cacheKey(r.namespacedName, jobId), placeholder); existed { rwLock.Unlock() return cached.JobInfo, cached.Err } @@ -202,8 +206,8 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) var task Task = func(taskCTX context.Context) bool { rwLock.RLock() - if _, existed := cacheStorage.Get(jobId); !existed { - logger.Info("The placeholder is removed for jobId", "jobId", jobId) + if _, existed := cacheStorage.Get(cacheKey(r.namespacedName, jobId)); !existed { + logger.Info("The placeholder is removed for jobId", "jobId", jobId, "cacheKey", cacheKey(r.namespacedName, jobId)) rwLock.RUnlock() return false } @@ -220,12 +224,12 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) } rwLock.Lock() - if existed := cacheStorage.Contains(jobId); !existed { - logger.Info("The placeholder is removed before updating for jobId", "jobId", jobId) + if existed := cacheStorage.Contains(cacheKey(r.namespacedName, jobId)); !existed { + logger.Info("The placeholder is removed before updating for jobId", "jobId", jobId, "cacheKey", cacheKey(r.namespacedName, jobId)) rwLock.Unlock() return false } - cacheStorage.Add(jobId, newJobInfoCache) + cacheStorage.Add(cacheKey(r.namespacedName, jobId), newJobInfoCache) rwLock.Unlock() if err != nil { @@ -238,7 +242,9 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) return true } if rayv1.IsJobTerminal(newJobInfoCache.JobInfo.JobStatus) { - logger.Info("The job reaches terminal status for jobId", "jobId", jobId, "status", newJobInfoCache.JobInfo.JobStatus) + logger.Info("The job reaches terminal status for jobId", "jobId", jobId, + "cacheKey", cacheKey(r.namespacedName, jobId), + "status", newJobInfoCache.JobInfo.JobStatus) return false } return true @@ -247,12 +253,12 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) if err := pool.AddTask(task); err != nil { // remove the placeholder because we cannot queue the task. rwLock.Lock() - cacheStorage.Remove(jobId) + cacheStorage.Remove(cacheKey(r.namespacedName, jobId)) rwLock.Unlock() - logger.Error(err, "Cannot queue more job info fetching tasks.", "jobId", jobId) + logger.Error(err, "Cannot queue more job info fetching tasks.", "jobId", jobId, "cacheKey", cacheKey(r.namespacedName, jobId)) return nil, ErrAgain } - logger.Info("Put a task to fetch job info in background for jobId ", "jobId", jobId) + logger.Info("Put a task to fetch job info in background for jobId ", "jobId", jobId, "cacheKey", cacheKey(r.namespacedName, jobId)) return nil, ErrAgain } @@ -275,7 +281,7 @@ func (r *RayDashboardCacheClient) GetJobLog(ctx context.Context, jobName string) func (r *RayDashboardCacheClient) StopJob(ctx context.Context, jobName string) error { rwLock.Lock() - cacheStorage.Remove(jobName) + cacheStorage.Remove(cacheKey(r.namespacedName, jobName)) rwLock.Unlock() return r.client.StopJob(ctx, jobName) @@ -283,8 +289,12 @@ func (r *RayDashboardCacheClient) StopJob(ctx context.Context, jobName string) e func (r *RayDashboardCacheClient) DeleteJob(ctx context.Context, jobName string) error { rwLock.Lock() - cacheStorage.Remove(jobName) + cacheStorage.Remove(cacheKey(r.namespacedName, jobName)) rwLock.Unlock() return r.client.DeleteJob(ctx, jobName) } + +func cacheKey(namespacedName types.NamespacedName, jobId string) string { + return namespacedName.String() + string(types.Separator) + jobId +} diff --git a/ray-operator/controllers/ray/utils/util.go b/ray-operator/controllers/ray/utils/util.go index fad6980d4ed..b9a1085ac4e 100644 --- a/ray-operator/controllers/ray/utils/util.go +++ b/ray-operator/controllers/ray/utils/util.go @@ -966,8 +966,12 @@ func GetRayDashboardClientFunc(ctx context.Context, mgr manager.Manager, useKube } if features.Enabled(features.AsyncJobInfoQuery) { + namespacedName := types.NamespacedName{ + Name: CheckName(rayCluster.Name), + Namespace: rayCluster.Namespace, + } dashboardCachedClient := &dashboardclient.RayDashboardCacheClient{} - dashboardCachedClient.InitClient(ctx, dashboardClient) + dashboardCachedClient.InitClient(ctx, namespacedName, dashboardClient) return dashboardCachedClient, nil } return dashboardClient, nil From 314df4554c35b967d74fd1430f38293378df026a Mon Sep 17 00:00:00 2001 From: fscnick Date: Wed, 7 Jan 2026 23:50:22 +0800 Subject: [PATCH 44/54] [RayJob] check raycluster is nil or not when initializing the dashboard client Signed-off-by: fscnick --- ray-operator/controllers/ray/utils/util.go | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/ray-operator/controllers/ray/utils/util.go b/ray-operator/controllers/ray/utils/util.go index b9a1085ac4e..7869017f826 100644 --- a/ray-operator/controllers/ray/utils/util.go +++ b/ray-operator/controllers/ray/utils/util.go @@ -966,9 +966,10 @@ func GetRayDashboardClientFunc(ctx context.Context, mgr manager.Manager, useKube } if features.Enabled(features.AsyncJobInfoQuery) { - namespacedName := types.NamespacedName{ - Name: CheckName(rayCluster.Name), - Namespace: rayCluster.Namespace, + var namespacedName types.NamespacedName + if rayCluster != nil { + namespacedName.Name = CheckName(rayCluster.Name) + namespacedName.Namespace = rayCluster.Namespace } dashboardCachedClient := &dashboardclient.RayDashboardCacheClient{} dashboardCachedClient.InitClient(ctx, namespacedName, dashboardClient) From e3cbe9f3bcce05890cfa5294d26a24853ced7b9f Mon Sep 17 00:00:00 2001 From: fscnick Date: Thu, 8 Jan 2026 00:01:25 +0800 Subject: [PATCH 45/54] [RayJob] avoid send to a block channel when graceful shutdown Signed-off-by: fscnick --- .../ray/utils/dashboardclient/dashboard_cache_client.go | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index 5713e1c2174..21618691238 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -77,9 +77,10 @@ func (w *workerPool) start(ctx context.Context, numWorkers int, requeueDelay tim if shouldRequeue && ctx.Err() == nil { time.AfterFunc(requeueDelay, func() { - if ctx.Err() == nil { - // Make sure the context is still valid before re-queuing the task. - w.taskQueue.In <- task + select { + case <-ctx.Done(): + return + case w.taskQueue.In <- task: } }) } From f0a3b801934a3451df845f7c35ac24512c97c4f4 Mon Sep 17 00:00:00 2001 From: fscnick Date: Thu, 8 Jan 2026 00:40:20 +0800 Subject: [PATCH 46/54] [RayJob] use contain to check the placeholder at the beginning of task Signed-off-by: fscnick --- .../ray/utils/dashboardclient/dashboard_cache_client.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index 21618691238..867b424d1de 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -207,7 +207,7 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) var task Task = func(taskCTX context.Context) bool { rwLock.RLock() - if _, existed := cacheStorage.Get(cacheKey(r.namespacedName, jobId)); !existed { + if existed := cacheStorage.Contains(cacheKey(r.namespacedName, jobId)); !existed { logger.Info("The placeholder is removed for jobId", "jobId", jobId, "cacheKey", cacheKey(r.namespacedName, jobId)) rwLock.RUnlock() return false From e55c2e05d4b48ea67ba3e9c160b95689b1f43a37 Mon Sep 17 00:00:00 2001 From: fscnick Date: Thu, 8 Jan 2026 20:39:14 +0800 Subject: [PATCH 47/54] [RayJob] graceful shutdown avoid panic from a nil task --- .../ray/utils/dashboardclient/dashboard_cache_client.go | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index 867b424d1de..0d2b3776bfe 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -72,7 +72,11 @@ func (w *workerPool) start(ctx context.Context, numWorkers int, requeueDelay tim case <-ctx.Done(): logger.Info("worker exiting...", "workerID", workerID) return - case task := <-w.taskQueue.Out: + case task, ok := <-w.taskQueue.Out: + if ok { + logger.Info("worker exiting from a closed channel", "workerID", workerID) + return + } shouldRequeue := task(ctx) if shouldRequeue && ctx.Err() == nil { From ae8bf77f2154372798e3a440aa300e82d23d73a5 Mon Sep 17 00:00:00 2001 From: fscnick Date: Thu, 8 Jan 2026 21:19:13 +0800 Subject: [PATCH 48/54] [RayJob] fix channel receive condition Signed-off-by: fscnick --- .../ray/utils/dashboardclient/dashboard_cache_client.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index 0d2b3776bfe..66f2dad1f91 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -73,7 +73,7 @@ func (w *workerPool) start(ctx context.Context, numWorkers int, requeueDelay tim logger.Info("worker exiting...", "workerID", workerID) return case task, ok := <-w.taskQueue.Out: - if ok { + if !ok { logger.Info("worker exiting from a closed channel", "workerID", workerID) return } From 5c7a5bba7d2b1730c654eea66c52b0f5401f70e2 Mon Sep 17 00:00:00 2001 From: fscnick Date: Thu, 8 Jan 2026 21:49:26 +0800 Subject: [PATCH 49/54] [RayJob] fix nil rayCluster in dashboard cache client Signed-off-by: fscnick --- ray-operator/controllers/ray/utils/util.go | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/ray-operator/controllers/ray/utils/util.go b/ray-operator/controllers/ray/utils/util.go index 7869017f826..0e1c54ee132 100644 --- a/ray-operator/controllers/ray/utils/util.go +++ b/ray-operator/controllers/ray/utils/util.go @@ -965,11 +965,10 @@ func GetRayDashboardClientFunc(ctx context.Context, mgr manager.Manager, useKube }, "http://"+url, authToken) } - if features.Enabled(features.AsyncJobInfoQuery) { - var namespacedName types.NamespacedName - if rayCluster != nil { - namespacedName.Name = CheckName(rayCluster.Name) - namespacedName.Namespace = rayCluster.Namespace + if features.Enabled(features.AsyncJobInfoQuery) && rayCluster != nil { + namespacedName := types.NamespacedName{ + Name: CheckName(rayCluster.Name), + Namespace: rayCluster.Namespace, } dashboardCachedClient := &dashboardclient.RayDashboardCacheClient{} dashboardCachedClient.InitClient(ctx, namespacedName, dashboardClient) From 8964c932ffdb196395a874906e007f629cc346c0 Mon Sep 17 00:00:00 2001 From: fscnick Date: Fri, 9 Jan 2026 22:34:20 +0800 Subject: [PATCH 50/54] [RayJob] remove with name from log for sharing purpose Signed-off-by: fscnick --- .../ray/utils/dashboardclient/dashboard_cache_client.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index 66f2dad1f91..3114c0ddd9c 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -62,7 +62,7 @@ type ( ) func (w *workerPool) start(ctx context.Context, numWorkers int, requeueDelay time.Duration) { - logger := ctrl.LoggerFrom(ctx).WithName("RayDashboardCacheClient").WithName("WorkerPool") + logger := ctrl.LoggerFrom(ctx).WithName("WorkerPool") w.taskQueue = chanx.NewUnboundedChanSize[Task](ctx, 0, 0, initBufferSize) for i := range numWorkers { From 90d2c30318db5c0698400f4aee11bab2d2397283 Mon Sep 17 00:00:00 2001 From: fscnick Date: Sat, 10 Jan 2026 10:13:27 +0800 Subject: [PATCH 51/54] [RayJob] remove checkname to avoid collision Signed-off-by: fscnick --- ray-operator/controllers/ray/utils/util.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ray-operator/controllers/ray/utils/util.go b/ray-operator/controllers/ray/utils/util.go index 0e1c54ee132..da36a7abda0 100644 --- a/ray-operator/controllers/ray/utils/util.go +++ b/ray-operator/controllers/ray/utils/util.go @@ -967,7 +967,7 @@ func GetRayDashboardClientFunc(ctx context.Context, mgr manager.Manager, useKube if features.Enabled(features.AsyncJobInfoQuery) && rayCluster != nil { namespacedName := types.NamespacedName{ - Name: CheckName(rayCluster.Name), + Name: rayCluster.Name, Namespace: rayCluster.Namespace, } dashboardCachedClient := &dashboardclient.RayDashboardCacheClient{} From 7502bc8abebdf67036a8520bf90c67ab24bec483 Mon Sep 17 00:00:00 2001 From: fscnick Date: Sat, 10 Jan 2026 11:08:52 +0800 Subject: [PATCH 52/54] [RayJob] add task with blocking send Signed-off-by: fscnick --- .../dashboardclient/dashboard_cache_client.go | 19 ++++--------------- 1 file changed, 4 insertions(+), 15 deletions(-) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index 3114c0ddd9c..6439b16938a 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -95,13 +95,8 @@ func (w *workerPool) start(ctx context.Context, numWorkers int, requeueDelay tim logger.Info(fmt.Sprintf("Initialize a worker pool with %d goroutines and requeueDelay is %v.", numWorkers, requeueDelay)) } -func (w *workerPool) AddTask(task Task) error { - select { - case w.taskQueue.In <- task: - return nil - default: - return ErrTaskQueueTemporarilyUnavailable - } +func (w *workerPool) AddTask(task Task) { + w.taskQueue.In <- task } var _ RayDashboardClientInterface = (*RayDashboardCacheClient)(nil) @@ -255,14 +250,8 @@ func (r *RayDashboardCacheClient) GetJobInfo(ctx context.Context, jobId string) return true } - if err := pool.AddTask(task); err != nil { - // remove the placeholder because we cannot queue the task. - rwLock.Lock() - cacheStorage.Remove(cacheKey(r.namespacedName, jobId)) - rwLock.Unlock() - logger.Error(err, "Cannot queue more job info fetching tasks.", "jobId", jobId, "cacheKey", cacheKey(r.namespacedName, jobId)) - return nil, ErrAgain - } + pool.AddTask(task) + logger.Info("Put a task to fetch job info in background for jobId ", "jobId", jobId, "cacheKey", cacheKey(r.namespacedName, jobId)) return nil, ErrAgain From 886c07fc0578b52d9eb630f4b3b8ea3899ec349e Mon Sep 17 00:00:00 2001 From: fscnick Date: Sun, 11 Jan 2026 09:28:53 +0800 Subject: [PATCH 53/54] [RayJob] remove unused error Signed-off-by: fscnick --- .../ray/utils/dashboardclient/dashboard_cache_client.go | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go index 6439b16938a..2ead97c52fc 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -16,12 +16,9 @@ import ( utiltypes "github.com/ray-project/kuberay/ray-operator/controllers/ray/utils/types" ) -var ( - // ErrAgain EAGAIN means "there is no data available right now, try again later" - // https://stackoverflow.com/questions/4058368/what-does-eagain-mean - ErrAgain = errors.New("EAGAIN") - ErrTaskQueueTemporarilyUnavailable = errors.New("task queue is temporarily unavailable") -) +// ErrAgain EAGAIN means "there is no data available right now, try again later" +// https://stackoverflow.com/questions/4058368/what-does-eagain-mean +var ErrAgain = errors.New("EAGAIN") const ( // TODO: make worker size configurable. From 339adf31b9bb06d2ee7414cd45e2ab11eb746e59 Mon Sep 17 00:00:00 2001 From: fscnick Date: Sun, 11 Jan 2026 20:16:49 +0800 Subject: [PATCH 54/54] [RayJob] provide raycluster name if it is absent for removing cache Signed-off-by: fscnick --- ray-operator/controllers/ray/rayjob_controller.go | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/ray-operator/controllers/ray/rayjob_controller.go b/ray-operator/controllers/ray/rayjob_controller.go index 0c1c8a946fd..6cef4cbe09c 100644 --- a/ray-operator/controllers/ray/rayjob_controller.go +++ b/ray-operator/controllers/ray/rayjob_controller.go @@ -119,6 +119,13 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) rayClusterInstance := &rayv1.RayCluster{} if err := r.Get(ctx, rayClusterNamespacedName, rayClusterInstance); err != nil { logger.Error(err, "Failed to get RayCluster") + + if features.Enabled(features.AsyncJobInfoQuery) { + // If the RayCluster is already deleted, we provide the name and namespace to the RayClusterInstance + // for the dashboard client to remove cache correctly. + rayClusterInstance.Name = rayClusterNamespacedName.Name + rayClusterInstance.Namespace = rayClusterNamespacedName.Namespace + } } rayDashboardClient, err := r.dashboardClientFunc(rayClusterInstance, rayJobInstance.Status.DashboardURL)