diff --git a/apiserver/pkg/server/ray_job_submission_service_server.go b/apiserver/pkg/server/ray_job_submission_service_server.go index 4a3d0e4fa0b..5a5440238c8 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(context.Background(), nil, false)} } // Submit Ray job diff --git a/go.mod b/go.mod index a98afa87b7c..5bf7badc546 100644 --- a/go.mod +++ b/go.mod @@ -79,6 +79,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/json-iterator/go v1.1.12 // indirect github.com/liggitt/tabwriter v0.0.0-20181228230101-89fcab3d43de // indirect @@ -99,6 +100,7 @@ require ( github.com/prometheus/procfs v0.19.2 // 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.39.0 // indirect diff --git a/go.sum b/go.sum index ea726f2b77a..dbf24293346 100644 --- a/go.sum +++ b/go.sum @@ -770,6 +770,8 @@ github.com/grpc-ecosystem/grpc-gateway/v2 v2.26.3/go.mod h1:ndYquD05frm2vACXE1ns 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= @@ -920,6 +922,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/apis/config/v1alpha1/configuration_types.go b/ray-operator/apis/config/v1alpha1/configuration_types.go index 910ec0d11ab..5554fbdf045 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" @@ -88,8 +90,8 @@ type Configuration struct { EnableMetrics bool `json:"enableMetrics,omitempty"` } -func (config Configuration) GetDashboardClient(mgr manager.Manager) func(rayCluster *rayv1.RayCluster, url string) (dashboardclient.RayDashboardClientInterface, error) { - return utils.GetRayDashboardClientFunc(mgr, config.UseKubernetesProxy) +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) } 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 81ccbde5690..6cef4cbe09c 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" "maps" "os" @@ -43,11 +44,10 @@ 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) + Recorder record.EventRecorder options RayJobReconcilerOptions + Scheme *runtime.Scheme + dashboardClientFunc func(rayCluster *rayv1.RayCluster, url string) (dashboardclient.RayDashboardClientInterface, error) } type RayJobReconcilerOptions struct { @@ -56,8 +56,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(), @@ -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) @@ -289,6 +296,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/rayservice_controller.go b/ray-operator/controllers/ray/rayservice_controller.go index 2bf23264330..31533f3f15a 100644 --- a/ray-operator/controllers/ray/rayservice_controller.go +++ b/ray-operator/controllers/ray/rayservice_controller.go @@ -65,8 +65,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 85c913e7bd6..1d9bfe12e83 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 new file mode 100644 index 00000000000..2ead97c52fc --- /dev/null +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_cache_client.go @@ -0,0 +1,291 @@ +package dashboardclient + +import ( + "context" + "errors" + "fmt" + "sync" + "time" + + 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" + utiltypes "github.com/ray-project/kuberay/ray-operator/controllers/ray/utils/types" +) + +// 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. + workerSize = 8 + + queryInterval = 3 * time.Second + + // TODO: consider a proper size for accommodating the all live job info + cacheSize = 10000 + cacheExpiry = 10 * time.Minute + + initBufferSize = 128 +) + +var ( + // singleton + initWorkPool sync.Once + pool workerPool + + // singleton + initCacheStorage sync.Once + cacheStorage *lru.Cache[string, *JobInfoCache] + rwLock sync.RWMutex +) + +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 + Err error + UpdatedAt *time.Time + } + + workerPool struct { + taskQueue *chanx.UnboundedChan[Task] + } +) + +func (w *workerPool) start(ctx context.Context, numWorkers int, requeueDelay time.Duration) { + logger := ctrl.LoggerFrom(ctx).WithName("WorkerPool") + w.taskQueue = chanx.NewUnboundedChanSize[Task](ctx, 0, 0, initBufferSize) + + for i := range numWorkers { + go func(workerID int) { + for { + select { + case <-ctx.Done(): + logger.Info("worker exiting...", "workerID", workerID) + return + 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 { + time.AfterFunc(requeueDelay, func() { + select { + case <-ctx.Done(): + return + case w.taskQueue.In <- task: + } + }) + } + } + } + }(i) + } + logger.Info(fmt.Sprintf("Initialize a worker pool with %d goroutines and requeueDelay is %v.", numWorkers, requeueDelay)) +} + +func (w *workerPool) AddTask(task Task) { + w.taskQueue.In <- task +} + +var _ RayDashboardClientInterface = (*RayDashboardCacheClient)(nil) + +type RayDashboardCacheClient struct { + client RayDashboardClientInterface + namespacedName types.NamespacedName +} + +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) + }) + + initCacheStorage.Do(func() { + // 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) + }) + + // expiry cache cleanup + go func() { + 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)) + + for { + select { + case <-ctx.Done(): + 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) + + // 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) + } + } + }() + }) + + r.client = client +} + +func (r *RayDashboardCacheClient) UpdateDeployments(ctx context.Context, configJson []byte) error { + return r.client.UpdateDeployments(ctx, configJson) +} + +func (r *RayDashboardCacheClient) GetServeDetails(ctx context.Context) (*utiltypes.ServeDetails, error) { + return r.client.GetServeDetails(ctx) +} + +func (r *RayDashboardCacheClient) GetMultiApplicationStatus(ctx context.Context) (map[string]*utiltypes.ServeApplicationStatus, error) { + return r.client.GetMultiApplicationStatus(ctx) +} + +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(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(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 + } + 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(cacheKey(r.namespacedName, 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.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 + } + rwLock.RUnlock() + + jobInfo, err := r.client.GetJobInfo(taskCTX, jobId) + currentTime := time.Now() + + // Make this cache immutable to avoid data race between pointer updates and read operations. + newJobInfoCache := &JobInfoCache{ + JobInfo: jobInfo, + Err: err, + UpdatedAt: ¤tTime, + } + + rwLock.Lock() + 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(cacheKey(r.namespacedName, jobId), newJobInfoCache) + rwLock.Unlock() + + 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.Info("Failed to fetch job info for jobId", "jobId", jobId, "error", err) + return false + } + if newJobInfoCache.JobInfo == nil { + return true + } + if rayv1.IsJobTerminal(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 + } + + 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 +} + +func (r *RayDashboardCacheClient) ListJobs(ctx context.Context) (*[]utiltypes.RayJobInfo, error) { + return r.client.ListJobs(ctx) +} + +func (r *RayDashboardCacheClient) SubmitJob(ctx context.Context, rayJob *rayv1.RayJob) (string, error) { + return r.client.SubmitJob(ctx, rayJob) +} + +func (r *RayDashboardCacheClient) SubmitJobReq(ctx context.Context, request *utiltypes.RayJobRequest) (string, error) { + return r.client.SubmitJobReq(ctx, request) +} + +func (r *RayDashboardCacheClient) GetJobLog(ctx context.Context, jobName string) (*string, error) { + return r.client.GetJobLog(ctx, jobName) +} + +func (r *RayDashboardCacheClient) StopJob(ctx context.Context, jobName string) error { + rwLock.Lock() + cacheStorage.Remove(cacheKey(r.namespacedName, jobName)) + rwLock.Unlock() + + return r.client.StopJob(ctx, jobName) +} + +func (r *RayDashboardCacheClient) DeleteJob(ctx context.Context, jobName string) error { + rwLock.Lock() + 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/dashboardclient/dashboard_httpclient.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_httpclient.go index 46b3f689a24..aecb32adc64 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_httpclient.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_httpclient.go @@ -27,7 +27,6 @@ var ( ) type RayDashboardClientInterface interface { - InitClient(client *http.Client, dashboardURL string, authToken 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 bdf87f7fa96..da36a7abda0 100644 --- a/ray-operator/controllers/ray/utils/util.go +++ b/ray-operator/controllers/ray/utils/util.go @@ -685,7 +685,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 } @@ -916,7 +916,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(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 @@ -959,17 +959,21 @@ func GetRayDashboardClientFunc(mgr manager.Manager, useKubernetesProxy bool) fun fmt.Sprintf("%s/api/v1/namespaces/%s/services/%s:dashboard/proxy", mgr.GetConfig().Host, rayCluster.Namespace, headSvcName), authToken, ) - return dashboardClient, nil - } - - dashboardClient.InitClient( - &http.Client{ + } else { + dashboardClient.InitClient(&http.Client{ Timeout: 2 * time.Second, - }, - "http://"+url, - authToken, - ) + }, "http://"+url, authToken) + } + if features.Enabled(features.AsyncJobInfoQuery) && rayCluster != nil { + namespacedName := types.NamespacedName{ + Name: rayCluster.Name, + Namespace: rayCluster.Namespace, + } + dashboardCachedClient := &dashboardclient.RayDashboardCacheClient{} + dashboardCachedClient.InitClient(ctx, namespacedName, dashboardClient) + return dashboardCachedClient, nil + } return dashboardClient, nil } } diff --git a/ray-operator/go.mod b/ray-operator/go.mod index 8e0a3e6e96f..ee57abfd78e 100644 --- a/ray-operator/go.mod +++ b/ray-operator/go.mod @@ -7,6 +7,7 @@ require ( github.com/coder/websocket v1.8.14 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.1 github.com/onsi/ginkgo/v2 v2.27.3 github.com/onsi/gomega v1.38.3 @@ -14,6 +15,7 @@ require ( github.com/orcaman/concurrent-map/v2 v2.0.1 github.com/prometheus/client_golang v1.23.2 github.com/robfig/cron/v3 v3.0.1 + github.com/smallnest/chanx v1.2.0 github.com/spf13/pflag v1.0.10 github.com/stretchr/testify v1.11.1 go.uber.org/mock v0.6.0 diff --git a/ray-operator/go.sum b/ray-operator/go.sum index f04b159a4aa..c9a03f313cd 100644 --- a/ray-operator/go.sum +++ b/ray-operator/go.sum @@ -89,6 +89,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.1 h1:0Ju+VCFuARfFlhVXFc2HxlcQkfB+Xq12/EotHko+x2A= github.com/jarcoal/httpmock v1.4.1/go.mod h1:ftW1xULwo+j0R0JJkJIIi7UKigZUXCLLanykgjwBXL0= github.com/joshdk/go-junit v1.0.0 h1:S86cUKIdwBHWwA6xCmFlf3RTLfVXYQfvanM5Uh+K6GE= @@ -148,6 +150,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.14.1 h1:UQB4HGPB6osV0SQTLymcB4TgvyWu6ZyliaW0tI/otEQ= github.com/rogpeppe/go-internal v1.14.1/go.mod h1:MaRKkUm5W0goXpeCfT7UZI6fk/L7L7so1lCWt35ZSgc= +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.10 h1:4EBh2KAYBwaONj6b2Ye1GiHfwjqyROoF4RwYO+vPwFk= github.com/spf13/pflag v1.0.10/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= 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 04781cabe93..fdb496e9b5f 100644 --- a/ray-operator/test/sampleyaml/support.go +++ b/ray-operator/test/sampleyaml/support.go @@ -74,7 +74,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(t.Ctx(), nil, false) rayDashboardClient, err := rayDashboardClientFunc(rayCluster, url) g.Expect(err).ToNot(HaveOccurred()) serveDetails, err := rayDashboardClient.GetServeDetails(t.Ctx())