From c21938dba00429c7ca422d6ee1f1ab9ad0317c74 Mon Sep 17 00:00:00 2001 From: Ping Date: Thu, 30 Oct 2025 09:51:23 +0800 Subject: [PATCH 01/32] [Bug] Sidecar mode shouldn't restart head pod when head pod is deleted (#4141) * [Bug] Sidecar mode shouldn't restart head pod when head pod is deleted Signed-off-by: 400Ping * [Fix] Fix e2e error Signed-off-by: 400Ping * [Fix] fix according to rueian's comment Signed-off-by: 400Ping * [Chore] fix ci error Signed-off-by: 400Ping * Update ray-operator/controllers/ray/raycluster_controller.go Co-authored-by: Han-Ju Chen (Future-Outlier) Signed-off-by: Ping * Update ray-operator/controllers/ray/rayjob_controller.go Co-authored-by: Han-Ju Chen (Future-Outlier) Signed-off-by: Ping * update Signed-off-by: Future-Outlier * update Signed-off-by: Future-Outlier * Trigger CI Signed-off-by: Future-Outlier --------- Signed-off-by: 400Ping Signed-off-by: Ping Signed-off-by: Future-Outlier Co-authored-by: Han-Ju Chen (Future-Outlier) --- ray-operator/controllers/ray/raycluster_controller.go | 10 ++++++++++ ray-operator/test/e2erayjob/rayjob_test.go | 4 ++-- 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/ray-operator/controllers/ray/raycluster_controller.go b/ray-operator/controllers/ray/raycluster_controller.go index 32d730db368..18b57f19922 100644 --- a/ray-operator/controllers/ray/raycluster_controller.go +++ b/ray-operator/controllers/ray/raycluster_controller.go @@ -619,6 +619,16 @@ func (r *RayClusterReconciler) reconcilePods(ctx context.Context, instance *rayv return errstd.New(reason) } } else if len(headPods.Items) == 0 { + originatedFrom := utils.GetCRDType(instance.Labels[utils.RayOriginatedFromCRDLabelKey]) + if originatedFrom == utils.RayJobCRD { + if meta.IsStatusConditionTrue(instance.Status.Conditions, string(rayv1.RayClusterProvisioned)) { + logger.Info( + "reconcilePods: Found 0 head Pods for a RayJob-managed RayCluster; skipping head creation to let RayJob controller handle the failure", + "rayCluster", instance.Name, + ) + return nil + } + } // Create head Pod if it does not exist. logger.Info("reconcilePods: Found 0 head Pods; creating a head Pod for the RayCluster.") if err := r.createHeadPod(ctx, *instance); err != nil { diff --git a/ray-operator/test/e2erayjob/rayjob_test.go b/ray-operator/test/e2erayjob/rayjob_test.go index 70de0b8718c..35a06cd16b7 100644 --- a/ray-operator/test/e2erayjob/rayjob_test.go +++ b/ray-operator/test/e2erayjob/rayjob_test.go @@ -310,10 +310,10 @@ env_vars: g.Eventually(RayJob(test, rayJob.Namespace, rayJob.Name), TestTimeoutMedium). Should(WithTransform(RayJobDeploymentStatus, Equal(rayv1.JobDeploymentStatusFailed))) g.Eventually(RayJob(test, rayJob.Namespace, rayJob.Name), TestTimeoutMedium). - Should(WithTransform(RayJobReason, Equal(rayv1.AppFailed))) + Should(WithTransform(RayJobReason, Equal(rayv1.JobDeploymentStatusTransitionGracePeriodExceeded))) g.Eventually(RayJob(test, rayJob.Namespace, rayJob.Name), TestTimeoutMedium). Should(WithTransform(func(job *rayv1.RayJob) string { return job.Status.Message }, - Equal("Submitter completed but Ray job not found in RayCluster."))) + MatchRegexp("The RayJob submitter finished at .* but the ray job did not reach terminal state within .*"))) // Cleanup err = test.Client().Ray().RayV1().RayJobs(namespace.Name).Delete(test.Ctx(), rayJob.Name, metav1.DeleteOptions{}) From 807ba964cb4383432e9ffed06ff85de7115b8808 Mon Sep 17 00:00:00 2001 From: "Han-Ju Chen (Future-Outlier)" Date: Fri, 31 Oct 2025 23:16:30 +0800 Subject: [PATCH 02/32] fix: dashboard build for kuberay 1.5.0 (#4161) Signed-off-by: Future-Outlier --- dashboard/Dockerfile | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/dashboard/Dockerfile b/dashboard/Dockerfile index 3d1b852b6e2..b5450d08478 100644 --- a/dashboard/Dockerfile +++ b/dashboard/Dockerfile @@ -9,7 +9,10 @@ WORKDIR /app # Install dependencies based on the preferred package manager COPY package.json yarn.lock* package-lock.json* pnpm-lock.yaml* ./ RUN \ - if [ -f yarn.lock ]; then yarn --frozen-lockfile; \ +if [ -f yarn.lock ]; then \ + corepack enable && corepack prepare yarn@4.9.2 --activate && \ + yarn config set nodeLinker node-modules && \ + yarn install --immutable; \ elif [ -f package-lock.json ]; then npm ci; \ elif [ -f pnpm-lock.yaml ]; then corepack enable pnpm && pnpm i --frozen-lockfile; \ else echo "Lockfile not found." && exit 1; \ @@ -28,7 +31,11 @@ COPY . . # ENV NEXT_TELEMETRY_DISABLED 1 RUN \ - if [ -f yarn.lock ]; then yarn run build; \ + if [ -f yarn.lock ]; then \ + corepack enable && corepack prepare yarn@4.9.2 --activate && \ + yarn config set nodeLinker node-modules && \ + yarn install --immutable && \ + yarn run build; \ elif [ -f package-lock.json ]; then npm run build; \ elif [ -f pnpm-lock.yaml ]; then corepack enable pnpm && pnpm run build; \ else echo "Lockfile not found." && exit 1; \ From f20cb129f6aa2fe25bcdcd011e71dd882adcb8c6 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> Date: Mon, 3 Nov 2025 17:22:19 +0000 Subject: [PATCH 03/32] [Feature Enhancement] Set ordered replica index label to support multi-slice (#4163) * [Feature Enhancement] Set ordered replica index label to support multi-slice Signed-off-by: Ryan O'Leary * rename replica-id -> replica-name Signed-off-by: Ryan O'Leary * Separate replica index feature gate logic Signed-off-by: Ryan O'Leary * remove index arg in createWorkerPod Signed-off-by: Ryan O'Leary --------- Signed-off-by: Ryan O'Leary --- ray-operator/controllers/ray/common/pod.go | 16 +- .../controllers/ray/common/pod_test.go | 27 +- .../controllers/ray/raycluster_controller.go | 125 +++++-- .../ray/raycluster_controller_test.go | 58 +++- .../controllers/ray/utils/constant.go | 14 +- .../test/e2e/raycluster_multi_host_test.go | 308 +++++++++++++----- 6 files changed, 397 insertions(+), 151 deletions(-) diff --git a/ray-operator/controllers/ray/common/pod.go b/ray-operator/controllers/ray/common/pod.go index 836272ecf37..21533cd8f49 100644 --- a/ray-operator/controllers/ray/common/pod.go +++ b/ray-operator/controllers/ray/common/pod.go @@ -251,7 +251,7 @@ func getEnableProbesInjection() bool { } // DefaultWorkerPodTemplate sets the config values -func DefaultWorkerPodTemplate(ctx context.Context, instance rayv1.RayCluster, workerSpec rayv1.WorkerGroupSpec, podName string, fqdnRayIP string, headPort string, replicaGrpName string, numHostIndex int) corev1.PodTemplateSpec { +func DefaultWorkerPodTemplate(ctx context.Context, instance rayv1.RayCluster, workerSpec rayv1.WorkerGroupSpec, podName string, fqdnRayIP string, headPort string, replicaGrpName string, replicaIndex int, numHostIndex int) corev1.PodTemplateSpec { podTemplate := workerSpec.Template podTemplate.GenerateName = podName // Pods created by RayCluster should be restricted to the namespace of the RayCluster. @@ -329,11 +329,15 @@ func DefaultWorkerPodTemplate(ctx context.Context, instance rayv1.RayCluster, wo mergedLabels := mergeLabels(workerSpec.Template.ObjectMeta.Labels, workerSpec.Labels) podTemplate.Labels = labelPod(rayv1.WorkerNode, instance.Name, workerSpec.GroupName, mergedLabels) - // Add additional labels for RayMultihostIndexing - multihostIndexingEnabled := features.Enabled(features.RayMultiHostIndexing) && workerSpec.NumOfHosts > 1 - if multihostIndexingEnabled { - podTemplate.Labels[utils.RayWorkerReplicaIndexKey] = replicaGrpName - podTemplate.Labels[utils.RayHostIndexKey] = strconv.Itoa(numHostIndex) + // Add additional labels when RayMultihostIndexing is enabled. + if features.Enabled(features.RayMultiHostIndexing) { + // The ordered replica index can be used for the single-host, multi-slice case. + podTemplate.Labels[utils.RayWorkerReplicaIndexKey] = strconv.Itoa(replicaIndex) + if workerSpec.NumOfHosts > 1 { + // These labels are specific to multi-host group setup and reconciliation. + podTemplate.Labels[utils.RayWorkerReplicaNameKey] = replicaGrpName + podTemplate.Labels[utils.RayHostIndexKey] = strconv.Itoa(numHostIndex) + } } workerSpec.RayStartParams = setMissingRayStartParams(ctx, workerSpec.RayStartParams, rayv1.WorkerNode, headPort, fqdnRayIP) diff --git a/ray-operator/controllers/ray/common/pod_test.go b/ray-operator/controllers/ray/common/pod_test.go index 27c0a3413c9..78ff88122ed 100644 --- a/ray-operator/controllers/ray/common/pod_test.go +++ b/ray-operator/controllers/ray/common/pod_test.go @@ -687,7 +687,7 @@ func TestBuildPod(t *testing.T) { worker := cluster.Spec.WorkerGroupSpecs[0] podName = cluster.Name + utils.DashSymbol + string(rayv1.WorkerNode) + utils.DashSymbol + worker.GroupName + utils.DashSymbol + utils.FormatInt32(0) fqdnRayIP := utils.GenerateFQDNServiceName(ctx, *cluster, cluster.Namespace) - podTemplateSpec = DefaultWorkerPodTemplate(ctx, *cluster, worker, podName, fqdnRayIP, "6379", "", 0) + podTemplateSpec = DefaultWorkerPodTemplate(ctx, *cluster, worker, podName, fqdnRayIP, "6379", "", 0, 0) pod = BuildPod(ctx, podTemplateSpec, rayv1.WorkerNode, worker.RayStartParams, "6379", false, utils.GetCRDType(""), fqdnRayIP, defaultContainerEnvs) // Check resources @@ -761,7 +761,7 @@ func TestBuildPod_WithNoCPULimits(t *testing.T) { worker := cluster.Spec.WorkerGroupSpecs[0] podName = cluster.Name + utils.DashSymbol + string(rayv1.WorkerNode) + utils.DashSymbol + worker.GroupName + utils.DashSymbol + utils.FormatInt32(0) fqdnRayIP := utils.GenerateFQDNServiceName(ctx, *cluster, cluster.Namespace) - podTemplateSpec = DefaultWorkerPodTemplate(ctx, *cluster, worker, podName, fqdnRayIP, "6379", "", 0) + podTemplateSpec = DefaultWorkerPodTemplate(ctx, *cluster, worker, podName, fqdnRayIP, "6379", "", 0, 0) pod = BuildPod(ctx, podTemplateSpec, rayv1.WorkerNode, worker.RayStartParams, "6379", false, utils.GetCRDType(""), fqdnRayIP, nil) expectedCommandArg = splitAndSort("ulimit -n 65536; ray start --block --dashboard-agent-listen-port=52365 --memory=1073741824 --num-cpus=2 --num-gpus=3 --address=raycluster-sample-head-svc.default.svc.cluster.local:6379 --port=6379 --metrics-export-port=8080") actualCommandArg = splitAndSort(pod.Spec.Containers[0].Args[0]) @@ -792,7 +792,7 @@ func TestBuildPod_WithOverwriteCommand(t *testing.T) { worker := cluster.Spec.WorkerGroupSpecs[0] podName = cluster.Name + utils.DashSymbol + string(rayv1.WorkerNode) + utils.DashSymbol + worker.GroupName + utils.DashSymbol + utils.FormatInt32(0) fqdnRayIP := utils.GenerateFQDNServiceName(ctx, *cluster, cluster.Namespace) - podTemplateSpec = DefaultWorkerPodTemplate(ctx, *cluster, worker, podName, fqdnRayIP, "6379", "", 0) + podTemplateSpec = DefaultWorkerPodTemplate(ctx, *cluster, worker, podName, fqdnRayIP, "6379", "", 0, 0) workerPod := BuildPod(ctx, podTemplateSpec, rayv1.WorkerNode, worker.RayStartParams, "6379", false, utils.GetCRDType(""), fqdnRayIP, nil) workerContainer := workerPod.Spec.Containers[utils.RayContainerIndex] assert.Equal(t, []string{"I am worker"}, workerContainer.Command) @@ -847,7 +847,7 @@ func TestBuildPod_WithCreatedByRayService(t *testing.T) { worker := cluster.Spec.WorkerGroupSpecs[0] podName = cluster.Name + utils.DashSymbol + string(rayv1.WorkerNode) + utils.DashSymbol + worker.GroupName + utils.DashSymbol + utils.FormatInt32(0) fqdnRayIP := utils.GenerateFQDNServiceName(ctx, *cluster, cluster.Namespace) - podTemplateSpec = DefaultWorkerPodTemplate(ctx, *cluster, worker, podName, fqdnRayIP, "6379", "", 0) + podTemplateSpec = DefaultWorkerPodTemplate(ctx, *cluster, worker, podName, fqdnRayIP, "6379", "", 0, 0) pod = BuildPod(ctx, podTemplateSpec, rayv1.WorkerNode, worker.RayStartParams, "6379", false, utils.RayServiceCRD, fqdnRayIP, nil) val, ok = pod.Labels[utils.RayClusterServingServiceLabelKey] @@ -903,7 +903,7 @@ func TestBuildPod_WithLoginBash(t *testing.T) { worker := cluster.Spec.WorkerGroupSpecs[0] podName = cluster.Name + utils.DashSymbol + string(rayv1.WorkerNode) + utils.DashSymbol + worker.GroupName + utils.DashSymbol + utils.FormatInt32(0) fqdnRayIP := utils.GenerateFQDNServiceName(ctx, *cluster, cluster.Namespace) - podTemplateSpec = DefaultWorkerPodTemplate(ctx, *cluster, worker, podName, fqdnRayIP, "6379", "", 0) + podTemplateSpec = DefaultWorkerPodTemplate(ctx, *cluster, worker, podName, fqdnRayIP, "6379", "", 0, 0) workerPod := BuildPod(ctx, podTemplateSpec, rayv1.WorkerNode, worker.RayStartParams, "6379", false, utils.RayServiceCRD, fqdnRayIP, nil) // Verify worker container command @@ -1166,7 +1166,7 @@ func TestDefaultWorkerPodTemplateWithName(t *testing.T) { expectedWorker := *worker.DeepCopy() // Pass a deep copy of worker (*worker.DeepCopy()) to prevent "worker" from updating. - podTemplateSpec := DefaultWorkerPodTemplate(ctx, *cluster, *worker.DeepCopy(), podName, fqdnRayIP, "6379", "", 0) + podTemplateSpec := DefaultWorkerPodTemplate(ctx, *cluster, *worker.DeepCopy(), podName, fqdnRayIP, "6379", "", 0, 0) assert.Empty(t, podTemplateSpec.ObjectMeta.Name) assert.Equal(t, expectedWorker, worker) } @@ -1187,9 +1187,10 @@ func TestDeafultWorkerPodTemplateWithReplicaGrpAndIndex(t *testing.T) { groupReplicaName := utils.GenerateRayWorkerReplicaGroupName(worker.GroupName) // Pass a deep copy of worker (*worker.DeepCopy()) to prevent "worker" from updating. - podTemplateSpec := DefaultWorkerPodTemplate(ctx, *cluster, *worker.DeepCopy(), podName, fqdnRayIP, "6379", groupReplicaName, 2) + podTemplateSpec := DefaultWorkerPodTemplate(ctx, *cluster, *worker.DeepCopy(), podName, fqdnRayIP, "6379", groupReplicaName, 0, 2) assert.Empty(t, podTemplateSpec.ObjectMeta.Name) - assert.Equal(t, podTemplateSpec.Labels[utils.RayWorkerReplicaIndexKey], groupReplicaName) + assert.Equal(t, podTemplateSpec.Labels[utils.RayWorkerReplicaNameKey], groupReplicaName) + assert.Equal(t, "0", podTemplateSpec.Labels[utils.RayWorkerReplicaIndexKey]) assert.Equal(t, "2", podTemplateSpec.Labels[utils.RayHostIndexKey]) } @@ -1235,7 +1236,7 @@ func TestDefaultWorkerPodTemplateWithConfigurablePorts(t *testing.T) { worker := cluster.Spec.WorkerGroupSpecs[0] podName := cluster.Name + utils.DashSymbol + string(rayv1.WorkerNode) + utils.DashSymbol + worker.GroupName + utils.DashSymbol + utils.FormatInt32(0) fqdnRayIP := utils.GenerateFQDNServiceName(ctx, *cluster, cluster.Namespace) - podTemplateSpec := DefaultWorkerPodTemplate(ctx, *cluster, worker, podName, fqdnRayIP, "6379", "", 0) + podTemplateSpec := DefaultWorkerPodTemplate(ctx, *cluster, worker, podName, fqdnRayIP, "6379", "", 0, 0) // DefaultWorkerPodTemplate will add the default metrics port if user doesn't specify it. // Verify the default metrics port exists. require.NoError(t, containerPortExists(podTemplateSpec.Spec.Containers[0].Ports, int32(utils.DefaultMetricsPort))) @@ -1245,7 +1246,7 @@ func TestDefaultWorkerPodTemplateWithConfigurablePorts(t *testing.T) { ContainerPort: customMetricsPort, } cluster.Spec.WorkerGroupSpecs[0].Template.Spec.Containers[0].Ports = []corev1.ContainerPort{metricsPort} - podTemplateSpec = DefaultWorkerPodTemplate(ctx, *cluster, worker, podName, fqdnRayIP, "6379", "", 0) + podTemplateSpec = DefaultWorkerPodTemplate(ctx, *cluster, worker, podName, fqdnRayIP, "6379", "", 0, 0) // Verify the custom metrics port exists. require.NoError(t, containerPortExists(podTemplateSpec.Spec.Containers[0].Ports, customMetricsPort)) } @@ -1284,7 +1285,7 @@ func TestDefaultWorkerPodTemplate_Autoscaling(t *testing.T) { for name, tc := range tests { t.Run(name, func(t *testing.T) { - podTemplateSpec := DefaultWorkerPodTemplate(ctx, tc.cluster, tc.cluster.Spec.WorkerGroupSpecs[0], podName, fqdnRayIP, "6379", "", 0) + podTemplateSpec := DefaultWorkerPodTemplate(ctx, tc.cluster, tc.cluster.Spec.WorkerGroupSpecs[0], podName, fqdnRayIP, "6379", "", 0, 0) assert.Equal(t, tc.expectedRestartPolicy, podTemplateSpec.Spec.RestartPolicy) }) } @@ -1300,7 +1301,7 @@ func TestDefaultInitContainer(t *testing.T) { expectedResult := len(cluster.Spec.WorkerGroupSpecs[0].Template.Spec.InitContainers) + 1 // Pass a deep copy of worker (*worker.DeepCopy()) to prevent "worker" from updating. - podTemplateSpec := DefaultWorkerPodTemplate(ctx, *cluster, *worker.DeepCopy(), podName, fqdnRayIP, "6379", "", 0) + podTemplateSpec := DefaultWorkerPodTemplate(ctx, *cluster, *worker.DeepCopy(), podName, fqdnRayIP, "6379", "", 0, 0) numInitContainers := len(podTemplateSpec.Spec.InitContainers) assert.Equal(t, expectedResult, numInitContainers, "A default init container is expected to be added.") @@ -1359,7 +1360,7 @@ func TestDefaultInitContainerImagePullPolicy(t *testing.T) { // set ray container imagePullPolicy worker.Template.Spec.Containers[utils.RayContainerIndex].ImagePullPolicy = tc.imagePullPolicy - podTemplateSpec := DefaultWorkerPodTemplate(ctx, *cluster, *worker.DeepCopy(), podName, fqdnRayIP, "6379", "", 0) + podTemplateSpec := DefaultWorkerPodTemplate(ctx, *cluster, *worker.DeepCopy(), podName, fqdnRayIP, "6379", "", 0, 0) healthCheckContainer := podTemplateSpec.Spec.InitContainers[len(podTemplateSpec.Spec.InitContainers)-1] assert.Equal(t, tc.expectedPullPolicy, healthCheckContainer.ImagePullPolicy, "The ImagePullPolicy of the init container should be the same as the Ray container.") diff --git a/ray-operator/controllers/ray/raycluster_controller.go b/ray-operator/controllers/ray/raycluster_controller.go index 18b57f19922..966d8bb2c9e 100644 --- a/ray-operator/controllers/ray/raycluster_controller.go +++ b/ray-operator/controllers/ray/raycluster_controller.go @@ -749,14 +749,42 @@ func (r *RayClusterReconciler) reconcilePods(ctx context.Context, instance *rayv logger.Info("reconcilePods", "workerReplicas", numExpectedWorkerPods, "NumOfHosts", worker.NumOfHosts, "runningPods", len(runningPods.Items), "diff", diff) + // Support replica indices for single-host, multi-slice environments. + validReplicaIndices := make(map[int]bool) + if features.Enabled(features.RayMultiHostIndexing) { + for _, pod := range runningPods.Items { + if indexStr, ok := pod.Labels[utils.RayWorkerReplicaIndexKey]; ok { + if index, err := strconv.Atoi(indexStr); err == nil { + validReplicaIndices[index] = true + } + } + } + logger.Info("reconcilePods", "found existing replica indices", "group", worker.GroupName, "indices", validReplicaIndices) + } if diff > 0 { // pods need to be added logger.Info("reconcilePods", "Number workers to add", diff, "Worker group", worker.GroupName) - // create all workers of this group - for i := 0; i < diff; i++ { - logger.Info("reconcilePods", "creating worker for group", worker.GroupName, "index", i, "total", diff) - if err := r.createWorkerPod(ctx, *instance, *worker.DeepCopy()); err != nil { - return errstd.Join(utils.ErrFailedCreateWorkerPod, err) + if features.Enabled(features.RayMultiHostIndexing) { + newReplicaIndex := 0 + // create all workers of this group + for i := 0; i < diff; i++ { + // Find the next available replica index. + for validReplicaIndices[newReplicaIndex] { + newReplicaIndex++ + } + validReplicaIndices[newReplicaIndex] = true + logger.Info("reconcilePods", "creating worker for group", worker.GroupName, "index", i, "total", diff, "replicaIndex", newReplicaIndex) + if err := r.createWorkerPodWithIndex(ctx, *instance, *worker.DeepCopy(), "", newReplicaIndex, 0); err != nil { + return errstd.Join(utils.ErrFailedCreateWorkerPod, err) + } + } + } else { + // create all workers of this group + for i := 0; i < diff; i++ { + logger.Info("reconcilePods", "creating worker for group", worker.GroupName, "index", i, "total", diff) + if err := r.createWorkerPod(ctx, *instance, *worker.DeepCopy()); err != nil { + return errstd.Join(utils.ErrFailedCreateWorkerPod, err) + } } } } else if diff == 0 { @@ -837,7 +865,7 @@ func (r *RayClusterReconciler) reconcileMultiHostWorkerGroup(ctx context.Context // 1. Group existing pods by ray.io/worker-group-replica-index. replicaMap := make(map[string][]corev1.Pod) for _, pod := range workerPods { - if replicaName, ok := pod.Labels[utils.RayWorkerReplicaIndexKey]; ok { + if replicaName, ok := pod.Labels[utils.RayWorkerReplicaNameKey]; ok { replicaMap[replicaName] = append(replicaMap[replicaName], pod) } } @@ -861,7 +889,7 @@ func (r *RayClusterReconciler) reconcileMultiHostWorkerGroup(ctx context.Context continue } if shouldDelete, reason := shouldDeletePod(pod, rayv1.WorkerNode); shouldDelete { - replicaName := pod.Labels[utils.RayWorkerReplicaIndexKey] + replicaName := pod.Labels[utils.RayWorkerReplicaNameKey] podsToDelete, ok := replicaMap[replicaName] if !ok { continue @@ -883,7 +911,7 @@ func (r *RayClusterReconciler) reconcileMultiHostWorkerGroup(ctx context.Context for _, podName := range worker.ScaleStrategy.WorkersToDelete { for _, pod := range workerPods { if pod.Name == podName { - replicaName := pod.Labels[utils.RayWorkerReplicaIndexKey] + replicaName := pod.Labels[utils.RayWorkerReplicaNameKey] for _, p := range replicaMap[replicaName] { podsToDeleteFromStrategy[p.Name] = p } @@ -909,28 +937,61 @@ func (r *RayClusterReconciler) reconcileMultiHostWorkerGroup(ctx context.Context } // 5. Calculate Pod diff for scaling up or down by NumOfHosts. - runningPodsCount := len(workerPods) - len(deletedPods) + + validReplicaGroups := make(map[string]struct{}) + for replicaName, podList := range replicaMap { + isHealthyAndComplete := true + for _, pod := range podList { + if _, isDeleted := deletedPods[pod.Name]; isDeleted { + isHealthyAndComplete = false + break + } + } + if isHealthyAndComplete { + validReplicaGroups[replicaName] = struct{}{} + } + } + numRunningReplicas := len(validReplicaGroups) numExpectedWorkerPods := int(utils.GetWorkerGroupDesiredReplicas(ctx, *worker)) - diff := numExpectedWorkerPods - runningPodsCount - logger.Info("Reconciling multi-host group", "group", worker.GroupName, "expectedPods", numExpectedWorkerPods, "runningPods", runningPodsCount, "diff", diff) - // Scale up NumOfHost workers per replica. - if diff > 0 { - logger.Info("reconcileMultiHostWorkerGroup", "Number workers to add", diff, "Worker group", worker.GroupName) - if diff%int(worker.NumOfHosts) != 0 { - return fmt.Errorf("cannot scale up multi-host group %s: required %d pods, which is not a multiple of NumOfHosts (%d)", worker.GroupName, diff, worker.NumOfHosts) + // Ensure that if numExpectedWorkerPods is not a multiple of NumOfHosts, we log an error. + if numExpectedWorkerPods%int(worker.NumOfHosts) != 0 { + return fmt.Errorf("desired worker pods (%d) is not a multiple of NumOfHosts (%d) for group %s", + numExpectedWorkerPods, worker.NumOfHosts, worker.GroupName) + } + numExpectedReplicas := numExpectedWorkerPods / int(worker.NumOfHosts) + replicasToCreate := numExpectedReplicas - numRunningReplicas + + // Track full replica groups to determine next replica index to assign to. + validReplicaIndices := make(map[int]bool) + for replicaName := range validReplicaGroups { + if len(replicaMap[replicaName]) > 0 { + pod := replicaMap[replicaName][0] + if indexStr, ok := pod.Labels[utils.RayWorkerReplicaIndexKey]; ok { + if index, err := strconv.Atoi(indexStr); err == nil { + validReplicaIndices[index] = true + } + } } - replicasToCreate := diff / int(worker.NumOfHosts) + } + logger.Info("Reconciling multi-host group", "group", worker.GroupName, "expectedReplicas", numExpectedReplicas, "runningReplicas", numRunningReplicas, "replicasToCreate", replicasToCreate, "inUseIndices", validReplicaIndices) + if replicasToCreate > 0 { logger.Info("Scaling up multi-host group", "group", worker.GroupName, "replicasToCreate", replicasToCreate) + newReplicaIndex := 0 // Find the next available index starting from 0 for i := 0; i < replicasToCreate; i++ { + for validReplicaIndices[newReplicaIndex] { + newReplicaIndex++ + } + validReplicaIndices[newReplicaIndex] = true replicaName := utils.GenerateRayWorkerReplicaGroupName(worker.GroupName) + logger.Info("Creating new replica group", "group", worker.GroupName, "replicaName", replicaName, "replicaIndex", newReplicaIndex) for j := 0; j < int(worker.NumOfHosts); j++ { - if err := r.createWorkerPodWithIndex(ctx, *instance, *worker.DeepCopy(), replicaName, j); err != nil { + if err := r.createWorkerPodWithIndex(ctx, *instance, *worker.DeepCopy(), replicaName, newReplicaIndex, j); err != nil { return errstd.Join(utils.ErrFailedCreateWorkerPod, err) } } } - } else if diff < 0 { + } else if replicasToCreate < 0 { // Scale down NumOfHost workers per replica. enableInTreeAutoscaling := utils.IsAutoscalingEnabled(&instance.Spec) enableRandomPodDelete := false @@ -940,19 +1001,19 @@ func (r *RayClusterReconciler) reconcileMultiHostWorkerGroup(ctx context.Context } } if !enableInTreeAutoscaling || enableRandomPodDelete { - workersToRemove := -diff - groupsToRemove := (workersToRemove + int(worker.NumOfHosts) - 1) / int(worker.NumOfHosts) - logger.Info("Scaling down multi-host group by randomly deleting replica groups", "group", worker.GroupName, "groupsToRemove", groupsToRemove) - - groupsDeleted := 0 - for _, podList := range replicaMap { - if groupsDeleted >= groupsToRemove { + replicasToRemove := -replicasToCreate + logger.Info("Scaling down multi-host group by randomly deleting replica groups", "group", worker.GroupName, "groupsToRemove", replicasToRemove) + replicasDeleted := 0 + // Iterate over validReplicaGroups which contains the IDs of replica groups with NumOfHosts running Pods. + for replicaID := range validReplicaGroups { + if replicasDeleted >= replicasToRemove { break } + podList := replicaMap[replicaID] if err := r.deletePods(ctx, instance, podList, worker.GroupName, "scaling down"); err != nil { return err } - groupsDeleted++ + replicasDeleted++ } } else { logger.Info("Random replica group deletion is disabled for the cluster. The only decision-maker for Pod deletions is the Ray Autoscaler.") @@ -1125,7 +1186,7 @@ func (r *RayClusterReconciler) createWorkerPod(ctx context.Context, instance ray logger := ctrl.LoggerFrom(ctx) // build the pod then create it - pod := r.buildWorkerPod(ctx, instance, worker, "", 0) + pod := r.buildWorkerPod(ctx, instance, worker, "", 0, 0) if r.options.BatchSchedulerManager != nil { if scheduler, err := r.options.BatchSchedulerManager.GetScheduler(); err == nil { scheduler.AddMetadataToChildResource(ctx, &instance, &pod, worker.GroupName) @@ -1145,11 +1206,11 @@ func (r *RayClusterReconciler) createWorkerPod(ctx context.Context, instance ray return nil } -func (r *RayClusterReconciler) createWorkerPodWithIndex(ctx context.Context, instance rayv1.RayCluster, worker rayv1.WorkerGroupSpec, replicaGrpName string, hostIndex int) error { +func (r *RayClusterReconciler) createWorkerPodWithIndex(ctx context.Context, instance rayv1.RayCluster, worker rayv1.WorkerGroupSpec, replicaGrpName string, replicaIndex int, hostIndex int) error { logger := ctrl.LoggerFrom(ctx) // build the pod then create it - pod := r.buildWorkerPod(ctx, instance, worker, replicaGrpName, hostIndex) + pod := r.buildWorkerPod(ctx, instance, worker, replicaGrpName, replicaIndex, hostIndex) if r.options.BatchSchedulerManager != nil { if scheduler, err := r.options.BatchSchedulerManager.GetScheduler(); err == nil { scheduler.AddMetadataToChildResource(ctx, &instance, &pod, worker.GroupName) @@ -1198,7 +1259,7 @@ func getCreatorCRDType(instance rayv1.RayCluster) utils.CRDType { } // Build worker instance pods. -func (r *RayClusterReconciler) buildWorkerPod(ctx context.Context, instance rayv1.RayCluster, worker rayv1.WorkerGroupSpec, replicaGrpName string, hostIndex int) corev1.Pod { +func (r *RayClusterReconciler) buildWorkerPod(ctx context.Context, instance rayv1.RayCluster, worker rayv1.WorkerGroupSpec, replicaGrpName string, replicaIndex int, hostIndex int) corev1.Pod { logger := ctrl.LoggerFrom(ctx) podName := utils.PodName(fmt.Sprintf("%s-%s", instance.Name, worker.GroupName), rayv1.WorkerNode, true) fqdnRayIP := utils.GenerateFQDNServiceName(ctx, instance, instance.Namespace) // Fully Qualified Domain Name @@ -1206,7 +1267,7 @@ func (r *RayClusterReconciler) buildWorkerPod(ctx context.Context, instance rayv // The Ray head port used by workers to connect to the cluster (GCS server port for Ray >= 1.11.0, Redis port for older Ray.) headPort := common.GetHeadPort(instance.Spec.HeadGroupSpec.RayStartParams) autoscalingEnabled := utils.IsAutoscalingEnabled(&instance.Spec) - podTemplateSpec := common.DefaultWorkerPodTemplate(ctx, instance, worker, podName, fqdnRayIP, headPort, replicaGrpName, hostIndex) + podTemplateSpec := common.DefaultWorkerPodTemplate(ctx, instance, worker, podName, fqdnRayIP, headPort, replicaGrpName, replicaIndex, hostIndex) if len(r.options.WorkerSidecarContainers) > 0 { podTemplateSpec.Spec.Containers = append(podTemplateSpec.Spec.Containers, r.options.WorkerSidecarContainers...) } diff --git a/ray-operator/controllers/ray/raycluster_controller_test.go b/ray-operator/controllers/ray/raycluster_controller_test.go index 56290797589..a11efb2de56 100644 --- a/ray-operator/controllers/ray/raycluster_controller_test.go +++ b/ray-operator/controllers/ray/raycluster_controller_test.go @@ -19,7 +19,6 @@ import ( "context" "errors" "fmt" - "slices" "strconv" "strings" "time" @@ -981,21 +980,56 @@ var _ = Context("Inside the default namespace", func() { }) It("All multi-host pods are properly labeled", func() { - workerGrpReplicaMap := make(map[string][]string) + type ReplicaInfo struct { + HostIndices map[string]bool + ReplicaIndex string + } + replicaGroups := make(map[string]ReplicaInfo) + // Track replicas to ensure unique indices are applied. + seenReplicaIndices := make(map[string]bool) + for _, pod := range workerPods.Items { + // Get all the labels hostIndex := pod.Labels[utils.RayHostIndexKey] - hostGrpId := pod.Labels[utils.RayWorkerReplicaIndexKey] - - grpReplicaIndexList, grpIdExists := workerGrpReplicaMap[hostGrpId] - if grpIdExists { - Expect(strconv.Atoi(hostIndex)).Should(BeNumerically("<", numOfHosts)) - Expect(strconv.Atoi(hostIndex)).Should(BeNumerically(">=", 0)) - Expect(slices.Contains(grpReplicaIndexList, hostIndex)).To(BeFalse()) - workerGrpReplicaMap[hostGrpId] = append(grpReplicaIndexList, hostIndex) + replicaID := pod.Labels[utils.RayWorkerReplicaNameKey] + replicaIndex := pod.Labels[utils.RayWorkerReplicaIndexKey] + + Expect(replicaIndex).NotTo(BeEmpty(), "Pod %s is missing label %s", pod.Name, utils.RayWorkerReplicaIndexKey) + seenReplicaIndices[replicaIndex] = true + + if info, ok := replicaGroups[replicaID]; ok { + // Validate replicaIndex is the same for all pods in this group. + Expect(replicaIndex).To(Equal(info.ReplicaIndex), "Pod %s in group %s has replicaIndex %s, but expected %s", pod.Name, replicaID, replicaIndex, info.ReplicaIndex) + + // Ensure hostIndex is unique within this replica group. + Expect(info.HostIndices[hostIndex]).To(BeFalse(), "Pod %s in group %s has duplicate hostIndex %s", pod.Name, replicaID, hostIndex) + info.HostIndices[hostIndex] = true } else { - workerGrpReplicaMap[hostGrpId] = []string{} - Expect(len(workerGrpReplicaMap)).Should(BeNumerically("<=", int(replicas))) + replicaGroups[replicaID] = ReplicaInfo{ + ReplicaIndex: replicaIndex, + HostIndices: map[string]bool{hostIndex: true}, + } } + + // Check hostIndex correctly set in range 0 to numOfHosts-1. + hostIndexInt, err := strconv.Atoi(hostIndex) + Expect(err).NotTo(HaveOccurred()) + Expect(hostIndexInt).To(BeNumerically("<", numOfHosts)) + Expect(hostIndexInt).To(BeNumerically(">=", 0)) + } + + // Validate we created 'replicas' number of groups. + Expect(replicaGroups).To(HaveLen(int(replicas)), "Expected %d replica groups, but found %d", replicas, len(replicaGroups)) + + // Validate replica indices are unique and indexed from 0 to replicas-1. + Expect(seenReplicaIndices).To(HaveLen(int(replicas)), "Expected %d unique replica indices, but found %d", replicas, len(seenReplicaIndices)) + Expect(seenReplicaIndices["0"]).To(BeTrue()) + Expect(seenReplicaIndices["1"]).To(BeTrue()) + Expect(seenReplicaIndices["2"]).To(BeTrue()) + + // Validate each replica group has 'numOfHosts' Pods. + for replicaID, info := range replicaGroups { + Expect(info.HostIndices).To(HaveLen(int(numOfHosts)), "Replica group %s expected %d hosts, but found %d", replicaID, numOfHosts, len(info.HostIndices)) } }) diff --git a/ray-operator/controllers/ray/utils/constant.go b/ray-operator/controllers/ray/utils/constant.go index f721464e015..e707cb71023 100644 --- a/ray-operator/controllers/ray/utils/constant.go +++ b/ray-operator/controllers/ray/utils/constant.go @@ -28,12 +28,18 @@ const ( KubeRayVersion = "ray.io/kuberay-version" // Labels for feature RayMultihostIndexing - // RayWorkerReplicaIndexKey label is the unique label for the replica in a specific worker group. It is made up of the worker group name - // and a unique identifier. e.g. multi-host-worker-group-xh3hf // - // RayHostIndexKey label represents the index of the host within the RayWorkerReplicaIndexKey. + // RayWorkerReplicaNameKey label is the unique name for the replica in a specific worker group. It is made up + // of the worker group name and a unique identifier (e.g. multi-host-worker-group-xh3hf). This label is unique + // across RayClusters. + RayWorkerReplicaNameKey = "ray.io/worker-group-replica-name" + + // RayWorkerReplicaIndexKey label is the integer index for the replica in it's worker group (0 to replicas-1). + // The value for this label is unique within its worker group, but not across worker groups or RayClusters. RayWorkerReplicaIndexKey = "ray.io/worker-group-replica-index" - RayHostIndexKey = "ray.io/replica-host-index" + + // RayHostIndexKey label represents the index of the host within the replica group. + RayHostIndexKey = "ray.io/replica-host-index" // In KubeRay, the Ray container must be the first application container in a head or worker Pod. RayContainerIndex = 0 diff --git a/ray-operator/test/e2e/raycluster_multi_host_test.go b/ray-operator/test/e2e/raycluster_multi_host_test.go index 3a234a7dba2..ed97c6fe365 100644 --- a/ray-operator/test/e2e/raycluster_multi_host_test.go +++ b/ray-operator/test/e2e/raycluster_multi_host_test.go @@ -1,13 +1,12 @@ package e2e import ( + "strconv" "testing" . "github.com/onsi/gomega" corev1 "k8s.io/api/core/v1" - "k8s.io/apimachinery/pkg/api/resource" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" - corev1ac "k8s.io/client-go/applyconfigurations/core/v1" rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1" "github.com/ray-project/kuberay/ray-operator/controllers/ray/utils" @@ -16,7 +15,198 @@ import ( . "github.com/ray-project/kuberay/ray-operator/test/support" ) -func TestRayClusterMultiHost(t *testing.T) { +// verifyWorkerGroupIndices is a helper function to check that pods in a worker group +// have the correct and unique replica/host index labels. +func verifyWorkerGroupIndices(t *testing.T, rayCluster *rayv1.RayCluster, workerGroupName string, expectedHosts int, expectedReplicas int, expectedIndices []int) { + test := With(t) + g := NewWithT(t) + + allWorkerPods, err := GetWorkerPods(test, rayCluster) + g.Expect(err).NotTo(HaveOccurred()) + groupPods := []corev1.Pod{} + for _, pod := range allWorkerPods { + if pod.Labels[utils.RayNodeGroupLabelKey] == workerGroupName { + groupPods = append(groupPods, pod) + } + } + + // Validate total number of pods for this group. + expectedPodCount := expectedReplicas * expectedHosts + g.Expect(groupPods).To(HaveLen(expectedPodCount), + "Expected %d pods for group %s (%d replicas with %d hosts each), but found %d", + expectedPodCount, workerGroupName, expectedReplicas, expectedHosts, len(groupPods)) + + // Track the indices seen when parsing the worker Pods. + seenReplicaIndices := make(map[int]bool) + expectedIndicesMap := make(map[int]bool) + for _, idx := range expectedIndices { + expectedIndicesMap[idx] = true + } + + if expectedHosts > 1 { + // For multi-host, all three labels should be set. + type ReplicaInfo struct { + HostIndices map[int]bool + ReplicaIndex int + } + replicaGroups := make(map[string]ReplicaInfo) + + for _, pod := range groupPods { + replicaID, ok := pod.Labels[utils.RayWorkerReplicaNameKey] + g.Expect(ok).To(BeTrue(), "Pod %s should have a replica ID label (%s)", pod.Name, utils.RayWorkerReplicaNameKey) + + replicaIndexStr, ok := pod.Labels[utils.RayWorkerReplicaIndexKey] + g.Expect(ok).To(BeTrue(), "Pod %s should have a replica index label (%s)", pod.Name, utils.RayWorkerReplicaIndexKey) + replicaIndex, err := strconv.Atoi(replicaIndexStr) + g.Expect(err).NotTo(HaveOccurred()) + seenReplicaIndices[replicaIndex] = true + + hostIndexStr, ok := pod.Labels[utils.RayHostIndexKey] + g.Expect(ok).To(BeTrue(), "Pod %s should have a host index label (%s)", pod.Name, utils.RayHostIndexKey) + hostIndex, err := strconv.Atoi(hostIndexStr) + g.Expect(err).NotTo(HaveOccurred()) + + // Check for duplicate host index values per replica group. + if info, exists := replicaGroups[replicaID]; exists { + g.Expect(replicaIndex).To(Equal(info.ReplicaIndex), + "Pod %s in group %s has inconsistent replicaIndex. Expected %d, got %d", pod.Name, replicaID, info.ReplicaIndex, replicaIndex) + + g.Expect(info.HostIndices[hostIndex]).To(BeFalse(), + "Pod %s in group %s has duplicate hostIndex %d", pod.Name, replicaID, hostIndex) + info.HostIndices[hostIndex] = true + } else { + replicaGroups[replicaID] = ReplicaInfo{ + ReplicaIndex: replicaIndex, + HostIndices: map[int]bool{hostIndex: true}, + } + } + } + + g.Expect(replicaGroups).To(HaveLen(expectedReplicas), "Should have %d replica groups, but found %d", expectedReplicas, len(replicaGroups)) + for replicaID, info := range replicaGroups { + g.Expect(info.HostIndices).To(HaveLen(expectedHosts), "Replica group %s should have %d hosts, but found %d", replicaID, expectedHosts, len(info.HostIndices)) + } + + } else { + // Single-host case, only replica index is set. + for _, pod := range groupPods { + g.Expect(pod.Labels).NotTo(HaveKey(utils.RayWorkerReplicaNameKey), "Pod %s should not have replica ID label for single-host group", pod.Name) + g.Expect(pod.Labels).NotTo(HaveKey(utils.RayHostIndexKey), "Pod %s should not have host index label for single-host group", pod.Name) + + // Check for unique replica index label + indexStr, ok := pod.Labels[utils.RayWorkerReplicaIndexKey] + g.Expect(ok).To(BeTrue(), "Pod %s should have a replica index label (%s)", pod.Name, utils.RayWorkerReplicaIndexKey) + + index, err := strconv.Atoi(indexStr) + g.Expect(err).NotTo(HaveOccurred(), "Failed to parse replica index '%s' for pod %s", indexStr, pod.Name) + + g.Expect(seenReplicaIndices[index]).To(BeFalse(), "Found duplicate replica index %d for pod %s", index, pod.Name) + seenReplicaIndices[index] = true + } + } + + if expectedIndices != nil { + expectedIndicesMap := make(map[int]bool) + for _, idx := range expectedIndices { + expectedIndicesMap[idx] = true + } + g.Expect(seenReplicaIndices).To(Equal(expectedIndicesMap), + "Expected replica indices %v for group %s, but found %v", expectedIndicesMap, workerGroupName, seenReplicaIndices) + } +} + +func TestRayClusterSingleHostMultiSlice(t *testing.T) { + test := With(t) + g := NewWithT(t) + + features.SetFeatureGateDuringTest(t, features.RayMultiHostIndexing, true) + + namespace := test.NewTestNamespace() + const ( + initialReplicas = 3 + clusterName = "raycluster-singlehost" + workerGroupName = "single-host-group" + ) + + // Define the RayCluster spec with a single-host worker group (NumOfHosts = 1). + rayClusterAC := rayv1ac.RayCluster(clusterName, namespace.Name). + WithSpec(rayv1ac.RayClusterSpec(). + WithRayVersion(GetRayVersion()). + WithHeadGroupSpec(rayv1ac.HeadGroupSpec(). + WithRayStartParams(map[string]string{"num-cpus": "0"}). + WithTemplate(HeadPodTemplateApplyConfiguration())). + WithWorkerGroupSpecs(rayv1ac.WorkerGroupSpec(). + WithReplicas(initialReplicas). + WithMinReplicas(0). + WithMaxReplicas(5). + WithNumOfHosts(1). + WithGroupName("single-host-group"). + WithRayStartParams(map[string]string{"num-cpus": "1"}). + WithTemplate(WorkerPodTemplateApplyConfiguration()))) + + // Create the RayCluster. + rayCluster, err := test.Client().Ray().RayV1().RayClusters(namespace.Name).Apply(test.Ctx(), rayClusterAC, TestApplyOptions) + g.Expect(err).NotTo(HaveOccurred()) + LogWithTimestamp(t, "Created RayCluster %s/%s successfully", rayCluster.Namespace, rayCluster.Name) + + // Wait for the cluster to become Ready and verify the initial Pod count. + LogWithTimestamp(t, "Waiting for RayCluster %s/%s to become ready", rayCluster.Namespace, rayCluster.Name) + g.Eventually(RayCluster(test, rayCluster.Namespace, rayCluster.Name), TestTimeoutLong). + Should(WithTransform(RayClusterState, Equal(rayv1.Ready))) + + g.Eventually(func() ([]corev1.Pod, error) { + return GetWorkerPods(test, rayCluster) + }, TestTimeoutShort).Should(HaveLen(initialReplicas)) + + // Verify that all pods are correctly labeled with indices 0 to replicas-1. + LogWithTimestamp(t, "Verifying initial labels on single-host pods for %s/%s", rayCluster.Namespace, rayCluster.Name) + verifyWorkerGroupIndices(t, rayCluster, workerGroupName, 1, initialReplicas, []int{0, 1, 2}) + + // Manually delete the pod with replica index 1. + LogWithTimestamp(t, "Deleting pod with replica index 1 for %s/%s", rayCluster.Namespace, rayCluster.Name) + workerPods, err := GetWorkerPods(test, rayCluster) + g.Expect(err).NotTo(HaveOccurred()) + + var podToDelete *corev1.Pod + for _, pod := range workerPods { + if pod.Labels[utils.RayWorkerReplicaIndexKey] == "1" { + podToDelete = &pod + break + } + } + g.Expect(podToDelete).NotTo(BeNil(), "Could not find pod with replica index 1 to delete") + + err = test.Client().Core().CoreV1().Pods(namespace.Name).Delete(test.Ctx(), podToDelete.Name, metav1.DeleteOptions{}) + g.Expect(err).NotTo(HaveOccurred()) + LogWithTimestamp(t, "Deleted pod %s", podToDelete.Name) + + // Wait for the controller to reconcile. The pod count should return to 3. + LogWithTimestamp(t, "Waiting for controller to reconcile and fill the gap") + g.Eventually(func() ([]corev1.Pod, error) { + return GetWorkerPods(test, rayCluster) + }, TestTimeoutShort).Should(HaveLen(initialReplicas), "Controller should restore pod count to %d", initialReplicas) + + // Verify that the controller replaced the missing index by creating a new pod with index 1. + LogWithTimestamp(t, "Verifying labels after pod deletion and reconciliation") + verifyWorkerGroupIndices(t, rayCluster, workerGroupName, 1, initialReplicas, []int{0, 1, 2}) + + // Scale up replicas from 3 to 4. + const scaleUpReplicas = 4 + LogWithTimestamp(t, "Scaling up RayCluster %s/%s from %d to %d replicas", rayCluster.Namespace, rayCluster.Name, initialReplicas, scaleUpReplicas) + rayClusterAC.Spec.WorkerGroupSpecs[0].WithReplicas(scaleUpReplicas) + _, err = test.Client().Ray().RayV1().RayClusters(namespace.Name).Apply(test.Ctx(), rayClusterAC, TestApplyOptions) + g.Expect(err).NotTo(HaveOccurred()) + + g.Eventually(func() ([]corev1.Pod, error) { + return GetWorkerPods(test, rayCluster) + }, TestTimeoutShort).Should(HaveLen(scaleUpReplicas), "Should scale up to %d pods", scaleUpReplicas) + + // Verify the new pod got the next available index, 3. + LogWithTimestamp(t, "Verifying labels after scale-up") + verifyWorkerGroupIndices(t, rayCluster, workerGroupName, 1, scaleUpReplicas, []int{0, 1, 2, 3}) +} + +func TestRayClusterMultiHostMultiSlice(t *testing.T) { test := With(t) g := NewWithT(t) @@ -30,12 +220,6 @@ func TestRayClusterMultiHost(t *testing.T) { initialReplicas = 2 clusterName = "raycluster-multihost" ) - sharedMemVolumeAC := corev1ac.Volume(). - WithName("shared-mem"). - WithEmptyDir(corev1ac.EmptyDirVolumeSource(). - WithMedium(corev1.StorageMediumMemory). - WithSizeLimit(resource.MustParse("1Gi")), - ) // Define the RayCluster spec with a multi-host worker group. rayClusterAC := rayv1ac.RayCluster(clusterName, namespace.Name). @@ -43,68 +227,23 @@ func TestRayClusterMultiHost(t *testing.T) { WithRayVersion(GetRayVersion()). WithHeadGroupSpec(rayv1ac.HeadGroupSpec(). WithRayStartParams(map[string]string{"dashboard-host": "0.0.0.0"}). - WithTemplate(HeadPodTemplateApplyConfiguration(). - // All PodSpec configurations go inside WithSpec. - WithSpec(corev1ac.PodSpec(). - WithVolumes(sharedMemVolumeAC). - WithRestartPolicy(corev1.RestartPolicyNever). - WithContainers(corev1ac.Container(). - WithName("ray-head"). - WithImage(GetRayImage()). - WithPorts( - corev1ac.ContainerPort().WithName(utils.GcsServerPortName).WithContainerPort(utils.DefaultGcsServerPort), - corev1ac.ContainerPort().WithName(utils.ServingPortName).WithContainerPort(utils.DefaultServingPort), - corev1ac.ContainerPort().WithName(utils.DashboardPortName).WithContainerPort(utils.DefaultDashboardPort), - corev1ac.ContainerPort().WithName(utils.ClientPortName).WithContainerPort(utils.DefaultClientPort), - ). - WithResources(corev1ac.ResourceRequirements(). - WithRequests(corev1.ResourceList{ - corev1.ResourceCPU: resource.MustParse("2"), - corev1.ResourceMemory: resource.MustParse("3Gi"), - }). - WithLimits(corev1.ResourceList{ - corev1.ResourceCPU: resource.MustParse("2"), - corev1.ResourceMemory: resource.MustParse("3Gi"), - })), - ), - ), - ), - ). + WithTemplate(HeadPodTemplateApplyConfiguration())). WithWorkerGroupSpecs(rayv1ac.WorkerGroupSpec(). - WithGroupName("multi-host-group"). WithReplicas(initialReplicas). WithMinReplicas(0). WithMaxReplicas(5). WithNumOfHosts(numOfHosts). - WithTemplate(WorkerPodTemplateApplyConfiguration(). - WithSpec(corev1ac.PodSpec(). - WithVolumes(sharedMemVolumeAC). - WithRestartPolicy(corev1.RestartPolicyNever). - WithContainers(corev1ac.Container(). - WithName("ray-worker"). - WithImage(GetRayImage()). - WithResources(corev1ac.ResourceRequirements(). - WithRequests(corev1.ResourceList{ - corev1.ResourceCPU: resource.MustParse("300m"), - corev1.ResourceMemory: resource.MustParse("1G"), - }). - WithLimits(corev1.ResourceList{ - corev1.ResourceCPU: resource.MustParse("500m"), - corev1.ResourceMemory: resource.MustParse("1G"), - })), - ), - ), - ), - ), - ) + WithGroupName("multi-host-group"). + WithRayStartParams(map[string]string{"num-cpus": "1"}). + WithTemplate(WorkerPodTemplateApplyConfiguration()))) // Create the RayCluster. rayCluster, err := test.Client().Ray().RayV1().RayClusters(namespace.Name).Apply(test.Ctx(), rayClusterAC, TestApplyOptions) g.Expect(err).NotTo(HaveOccurred()) - LogWithTimestamp(test.T(), "Created RayCluster %s/%s successfully", rayCluster.Namespace, rayCluster.Name) + LogWithTimestamp(t, "Created RayCluster %s/%s successfully", rayCluster.Namespace, rayCluster.Name) // Wait for the cluster to become Ready and verify the initial Pod count. - LogWithTimestamp(test.T(), "Waiting for RayCluster %s/%s to become ready", rayCluster.Namespace, rayCluster.Name) + LogWithTimestamp(t, "Waiting for RayCluster %s/%s to become ready", rayCluster.Namespace, rayCluster.Name) g.Eventually(RayCluster(test, rayCluster.Namespace, rayCluster.Name), TestTimeoutLong). Should(WithTransform(RayClusterState, Equal(rayv1.Ready))) @@ -113,58 +252,59 @@ func TestRayClusterMultiHost(t *testing.T) { return GetWorkerPods(test, rayCluster) }, TestTimeoutShort).Should(HaveLen(expectedPodCount)) - // Verify that all pods are correctly labeled. - LogWithTimestamp(test.T(), "Verifying labels on multi-host pods for %s/%s", rayCluster.Namespace, rayCluster.Name) - workerPods, err := GetWorkerPods(test, rayCluster) - g.Expect(err).NotTo(HaveOccurred()) - replicaMap := make(map[string][]string) - for _, pod := range workerPods { - replicaName, ok := pod.Labels[utils.RayWorkerReplicaIndexKey] - g.Expect(ok).To(BeTrue(), "Pod %s should have a replica index label", pod.Name) - hostIndex, ok := pod.Labels[utils.RayHostIndexKey] - g.Expect(ok).To(BeTrue(), "Pod %s should have a host index label", pod.Name) - replicaMap[replicaName] = append(replicaMap[replicaName], hostIndex) - } - g.Expect(replicaMap).To(HaveLen(initialReplicas), "Should have the correct number of replica groups") - for replicaName, hostIndices := range replicaMap { - g.Expect(hostIndices).To(HaveLen(numOfHosts), "Replica group %s should be complete", replicaName) - } + // Verify that all pods are correctly labeled during replica group scale up. + LogWithTimestamp(t, "Verifying labels on multi-host pods for %s/%s", rayCluster.Namespace, rayCluster.Name) + verifyWorkerGroupIndices(t, rayCluster, "multi-host-group", numOfHosts, initialReplicas, []int{0, 1}) // Scale down replicas from 2 to 1. Verify we scale by a multiple of NumOfHosts. - LogWithTimestamp(test.T(), "Scaling down RayCluster %s/%s", rayCluster.Namespace, rayCluster.Name) + const scaleDownReplicas = 1 + LogWithTimestamp(t, "Scaling down RayCluster %s/%s", rayCluster.Namespace, rayCluster.Name) rayClusterAC.Spec.WorkerGroupSpecs[0].WithReplicas(1) _, err = test.Client().Ray().RayV1().RayClusters(namespace.Name).Apply(test.Ctx(), rayClusterAC, TestApplyOptions) g.Expect(err).NotTo(HaveOccurred()) - expectedPodCount = 1 * numOfHosts + expectedPodCount = scaleDownReplicas * numOfHosts g.Eventually(func() ([]corev1.Pod, error) { return GetWorkerPods(test, rayCluster) }, TestTimeoutShort).Should(HaveLen(expectedPodCount), "Should scale down to 1 multi-host group (4 pods)") + // Verify labels again after replica group scale down. + LogWithTimestamp(t, "Verifying labels after scale-down for %s/%s", rayCluster.Namespace, rayCluster.Name) + verifyWorkerGroupIndices(t, rayCluster, "multi-host-group", numOfHosts, scaleDownReplicas, nil) + // Test scale up: Increase replicas from 1 to 3. - LogWithTimestamp(test.T(), "Scaling up RayCluster %s/%s", rayCluster.Namespace, rayCluster.Name) - rayClusterAC.Spec.WorkerGroupSpecs[0].WithReplicas(3) + const scaleUpReplicas = 3 + LogWithTimestamp(t, "Scaling up RayCluster %s/%s", rayCluster.Namespace, rayCluster.Name) + rayClusterAC.Spec.WorkerGroupSpecs[0].WithReplicas(scaleUpReplicas) _, err = test.Client().Ray().RayV1().RayClusters(namespace.Name).Apply(test.Ctx(), rayClusterAC, TestApplyOptions) g.Expect(err).NotTo(HaveOccurred()) - expectedPodCount = 3 * numOfHosts + expectedPodCount = scaleUpReplicas * numOfHosts g.Eventually(func() ([]corev1.Pod, error) { return GetWorkerPods(test, rayCluster) }, TestTimeoutShort).Should(HaveLen(expectedPodCount), "Should scale up to 3 multi-host groups (12 pods)") + // Verify labels are set with expected values after scale up again. + LogWithTimestamp(t, "Verifying labels after scale-up for %s/%s", rayCluster.Namespace, rayCluster.Name) + verifyWorkerGroupIndices(t, rayCluster, "multi-host-group", numOfHosts, scaleUpReplicas, []int{0, 1, 2}) + // Manually delete a single pod and verify the controller atomically re-creates the slice. - LogWithTimestamp(test.T(), "Testing atomic multi-host group recreation for RayCluster %s/%s", rayCluster.Namespace, rayCluster.Name) - workerPods, err = GetWorkerPods(test, rayCluster) + LogWithTimestamp(t, "Testing atomic multi-host group recreation for RayCluster %s/%s", rayCluster.Namespace, rayCluster.Name) + workerPods, err := GetWorkerPods(test, rayCluster) g.Expect(err).NotTo(HaveOccurred()) podToDelete := workerPods[0] err = test.Client().Core().CoreV1().Pods(namespace.Name).Delete(test.Ctx(), podToDelete.Name, metav1.DeleteOptions{}) g.Expect(err).NotTo(HaveOccurred()) // The controller should first clean up the broken multi-host group (-4 pods), and then re-scale it up (+4 pods). - LogWithTimestamp(test.T(), "Waiting for controller to reconcile multi-host group.") + LogWithTimestamp(t, "Waiting for controller to reconcile multi-host group.") // Reconciliation happens too quickly to catch the state where expectedPodCount-NumOfHosts, but we can test // that externally deleted Pods will be re-created to satisfy the expected number. g.Eventually(func() ([]corev1.Pod, error) { return GetWorkerPods(test, rayCluster) }, TestTimeoutShort).Should(HaveLen(expectedPodCount), "Controller restored cluster to the correct number of pods.") + + // Verify labels are still set correctly after atomic re-creation due to unhealthy Pod. + LogWithTimestamp(t, "Verifying labels after atomic recreation for %s/%s", rayCluster.Namespace, rayCluster.Name) + verifyWorkerGroupIndices(t, rayCluster, "multi-host-group", numOfHosts, scaleUpReplicas, []int{0, 1, 2}) } From 9eb8e3a23fe2f0162f9f5e737951a543038088b7 Mon Sep 17 00:00:00 2001 From: Andrew Sy Kim Date: Mon, 3 Nov 2025 18:20:35 -0500 Subject: [PATCH 04/32] update stale feature gate comments (#4174) Signed-off-by: Andrew Sy Kim --- ray-operator/pkg/features/features.go | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/ray-operator/pkg/features/features.go b/ray-operator/pkg/features/features.go index 16b23ab83ac..b6f9e37efd4 100644 --- a/ray-operator/pkg/features/features.go +++ b/ray-operator/pkg/features/features.go @@ -14,26 +14,28 @@ const ( // owner: @rueian @kevin85421 @andrewsykim // rep: https://github.com/ray-project/enhancements/pull/54 // alpha: v1.2 + // beta: v1.3 // // Enables new conditions in RayCluster status RayClusterStatusConditions featuregate.Feature = "RayClusterStatusConditions" - // owner: @andrewsykim + // owner: @andrewsykim @seanlaii // rep: N/A // alpha: v1.3 // // Enables new deletion policy API in RayJob RayJobDeletionPolicy featuregate.Feature = "RayJobDeletionPolicy" - // owner: @aaronliang + // owner: @aaronliang @ryanaoleary // rep: N/A - // alpha: v1.0 + // alpha: v1.5 + // // Enables multi-host worker indexing RayMultiHostIndexing featuregate.Feature = "RayMultiHostIndexing" // owner: @ryanaoleary - // rep: N/A - // alpha: v1.0 + // rep: https://github.com/ray-project/enhancements/pull/58 + // alpha: v1.5 // // Enabled NewClusterWithIncrementalUpgrade type for RayService zero-downtime upgrades. RayServiceIncrementalUpgrade featuregate.Feature = "RayServiceIncrementalUpgrade" From 3626d5cf4ac3f20c6417763ca8ddf0330342abc2 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Mon, 3 Nov 2025 21:16:35 -0800 Subject: [PATCH 05/32] [RayCluster] Add more context why we don't recreate head Pod for RayJob (#4175) Signed-off-by: Kai-Hsun Chen --- ray-operator/controllers/ray/raycluster_controller.go | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/ray-operator/controllers/ray/raycluster_controller.go b/ray-operator/controllers/ray/raycluster_controller.go index 966d8bb2c9e..8257f3d034a 100644 --- a/ray-operator/controllers/ray/raycluster_controller.go +++ b/ray-operator/controllers/ray/raycluster_controller.go @@ -621,6 +621,17 @@ func (r *RayClusterReconciler) reconcilePods(ctx context.Context, instance *rayv } else if len(headPods.Items) == 0 { originatedFrom := utils.GetCRDType(instance.Labels[utils.RayOriginatedFromCRDLabelKey]) if originatedFrom == utils.RayJobCRD { + // Recreating the head Pod if the RayCluster created by RayJob is provisioned doesn't help RayJob. + // + // Case 1: GCS fault tolerance is disabled + // + // In this case, the worker Pods will be killed by the new head Pod when it is created, so the new Ray job will not be running in + // a "provisioned" cluster. + // + // Case 2: GCS fault tolerance is enabled + // + // In this case, the worker Pods will not be killed by the new head Pod when it is created, but the submission ID has already been + // used by the old Ray job, so the new Ray job will fail. if meta.IsStatusConditionTrue(instance.Status.Conditions, string(rayv1.RayClusterProvisioned)) { logger.Info( "reconcilePods: Found 0 head Pods for a RayJob-managed RayCluster; skipping head creation to let RayJob controller handle the failure", From b3a00db9bfa7541a9d2c4d03d42f361d0828455c Mon Sep 17 00:00:00 2001 From: Kavish <141061817+kash2104@users.noreply.github.com> Date: Fri, 7 Nov 2025 05:32:54 +0530 Subject: [PATCH 06/32] feature: Remove empty resource list initialization. (#4168) Fixes #4142. --- ray-operator/controllers/ray/utils/util.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/ray-operator/controllers/ray/utils/util.go b/ray-operator/controllers/ray/utils/util.go index 540162bf9f4..81bf187efff 100644 --- a/ray-operator/controllers/ray/utils/util.go +++ b/ray-operator/controllers/ray/utils/util.go @@ -437,7 +437,7 @@ func CalculateAvailableReplicas(pods corev1.PodList) int32 { } func CalculateDesiredResources(cluster *rayv1.RayCluster) corev1.ResourceList { - desiredResourcesList := []corev1.ResourceList{{}} + desiredResourcesList := []corev1.ResourceList{} headPodResource := CalculatePodResource(cluster.Spec.HeadGroupSpec.Template.Spec) desiredResourcesList = append(desiredResourcesList, headPodResource) for _, nodeGroup := range cluster.Spec.WorkerGroupSpecs { @@ -454,7 +454,7 @@ func CalculateDesiredResources(cluster *rayv1.RayCluster) corev1.ResourceList { } func CalculateMinResources(cluster *rayv1.RayCluster) corev1.ResourceList { - minResourcesList := []corev1.ResourceList{{}} + minResourcesList := []corev1.ResourceList{} headPodResource := CalculatePodResource(cluster.Spec.HeadGroupSpec.Template.Spec) minResourcesList = append(minResourcesList, headPodResource) for _, nodeGroup := range cluster.Spec.WorkerGroupSpecs { From 470d651a7438c6e8f108694d7a78c65e93d52a3e Mon Sep 17 00:00:00 2001 From: Neo Chien <6762509+cchung100m@users.noreply.github.com> Date: Fri, 7 Nov 2025 08:03:24 +0800 Subject: [PATCH 07/32] [Dockerfile] [KubeRay Dashboard]: Fix Dockerfile warnings (ENV format, CMD JSON args) (#4167) * [#4166] improvement: Fix Dockerfile warnings (ENV format, CMD JSON args) * extract the hostname from CMD Signed-off-by: Neo Chien <6762509+cchung100m@users.noreply.github.com> --------- Signed-off-by: Neo Chien <6762509+cchung100m@users.noreply.github.com> Co-authored-by: cchung100m --- dashboard/Dockerfile | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/dashboard/Dockerfile b/dashboard/Dockerfile index b5450d08478..0135c3a744a 100644 --- a/dashboard/Dockerfile +++ b/dashboard/Dockerfile @@ -45,7 +45,7 @@ RUN \ FROM base AS runner WORKDIR /app -ENV NODE_ENV production +ENV NODE_ENV=production # Uncomment the following line in case you want to disable telemetry during runtime. # ENV NEXT_TELEMETRY_DISABLED 1 @@ -67,8 +67,9 @@ USER nextjs EXPOSE 3000 -ENV PORT 3000 +ENV PORT=3000 # server.js is created by next build from the standalone output # https://nextjs.org/docs/pages/api-reference/next-config-js/output -CMD HOSTNAME="0.0.0.0" node server.js +ENV HOSTNAME="0.0.0.0" +CMD ["node", "server.js"] From b51f885c1becb863166ab266a5b2daa4d6db653b Mon Sep 17 00:00:00 2001 From: JustinYeh Date: Fri, 7 Nov 2025 09:30:45 +0800 Subject: [PATCH 08/32] =?UTF-8?q?[Fix]=20Resolve=20int32=20overflow=20by?= =?UTF-8?q?=20having=20the=20calculation=20in=20int64=20and=20c=E2=80=A6?= =?UTF-8?q?=20(#4158)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * [Fix] Resolve int32 overflow by having the calculation in int64 and cap it if the count is over math.MaxInt32 Signed-off-by: justinyeh1995 * [Test] Add unit tests for CalculateReadyReplicas Signed-off-by: justinyeh1995 * [Fix] Add a nosec comment to pass the Lint (pre-commit) test Signed-off-by: justinyeh1995 * [Refactor] Add CapInt64ToInt32 to replace #nosec directives Signed-off-by: justinyeh1995 * [Refactor] Rename function to SafeInt64ToInt32 and add a underflowing prevention (it also help pass the lint test) Signed-off-by: justinyeh1995 * [Refactor] Remove the early return as SafeInt64ToInt32 handles the int32 overflow and underflow checking. Signed-off-by: justinyeh1995 --------- Signed-off-by: justinyeh1995 --- ray-operator/controllers/ray/utils/util.go | 18 +++- .../controllers/ray/utils/util_test.go | 94 +++++++++++++++++++ 2 files changed, 109 insertions(+), 3 deletions(-) diff --git a/ray-operator/controllers/ray/utils/util.go b/ray-operator/controllers/ray/utils/util.go index 81bf187efff..5235054b3f8 100644 --- a/ray-operator/controllers/ray/utils/util.go +++ b/ray-operator/controllers/ray/utils/util.go @@ -253,6 +253,18 @@ func SafeUint64ToInt64(n uint64) int64 { return int64(n) } +// SafeInt64ToInt32 converts int64 to int32, preventing overflow/underflow by +// bounding the value between [math.MinInt32, math.MaxInt32] +func SafeInt64ToInt32(n int64) int32 { + if n > math.MaxInt32 { + return math.MaxInt32 + } + if n < math.MinInt32 { + return math.MinInt32 + } + return int32(n) +} + // GetNamespace return namespace func GetNamespace(metaData metav1.ObjectMeta) string { if metaData.Namespace == "" { @@ -393,15 +405,15 @@ func CalculateMinReplicas(cluster *rayv1.RayCluster) int32 { // CalculateMaxReplicas calculates max worker replicas at the cluster level func CalculateMaxReplicas(cluster *rayv1.RayCluster) int32 { - count := int32(0) + count := int64(0) for _, nodeGroup := range cluster.Spec.WorkerGroupSpecs { if nodeGroup.Suspend != nil && *nodeGroup.Suspend { continue } - count += (*nodeGroup.MaxReplicas * nodeGroup.NumOfHosts) + count += int64(*nodeGroup.MaxReplicas) * int64(nodeGroup.NumOfHosts) } - return count + return SafeInt64ToInt32(count) } // CalculateReadyReplicas calculates ready worker replicas at the cluster level diff --git a/ray-operator/controllers/ray/utils/util_test.go b/ray-operator/controllers/ray/utils/util_test.go index 8bd37a2e7f8..9a9371acd05 100644 --- a/ray-operator/controllers/ray/utils/util_test.go +++ b/ray-operator/controllers/ray/utils/util_test.go @@ -805,6 +805,100 @@ func TestCalculateDesiredReplicas(t *testing.T) { } } +func TestCalculateMaxReplicasOverflow(t *testing.T) { + tests := []struct { + name string + specs []rayv1.WorkerGroupSpec + expected int32 + }{ + { + name: "Bug reproduction: issue report with replicas=1, minReplicas=3, numOfHosts=4", + specs: []rayv1.WorkerGroupSpec{ + { + GroupName: "workergroup", + Replicas: ptr.To[int32](1), + MinReplicas: ptr.To[int32](3), + MaxReplicas: ptr.To[int32](2147483647), // Default max int32 + NumOfHosts: 4, + }, + }, + expected: 2147483647, // Was -4 before fix, should be capped at max int32 + }, + { + name: "Single group overflow with default maxReplicas and numOfHosts=4", + specs: []rayv1.WorkerGroupSpec{ + { + NumOfHosts: 4, + MinReplicas: ptr.To[int32](3), + MaxReplicas: ptr.To[int32](2147483647), + }, + }, + expected: 2147483647, // Should be capped at max int32 + }, + { + name: "Single group overflow with large values", + specs: []rayv1.WorkerGroupSpec{ + { + NumOfHosts: 1000, + MinReplicas: ptr.To[int32](1), + MaxReplicas: ptr.To[int32](2147483647), + }, + }, + expected: 2147483647, // Should be capped + }, + { + name: "Multiple groups causing overflow when summed", + specs: []rayv1.WorkerGroupSpec{ + { + NumOfHosts: 2, + MinReplicas: ptr.To[int32](1), + MaxReplicas: ptr.To[int32](1500000000), + }, + { + NumOfHosts: 1, + MinReplicas: ptr.To[int32](1), + MaxReplicas: ptr.To[int32](1000000000), + }, + }, + expected: 2147483647, // 3B + 1B > max int32, should be capped + }, + { + name: "No overflow with reasonable values", + specs: []rayv1.WorkerGroupSpec{ + { + NumOfHosts: 4, + MinReplicas: ptr.To[int32](2), + MaxReplicas: ptr.To[int32](100), + }, + }, + expected: 400, // 100 * 4 = 400, no overflow + }, + { + name: "Edge case: exactly at max int32", + specs: []rayv1.WorkerGroupSpec{ + { + NumOfHosts: 1, + MinReplicas: ptr.To[int32](1), + MaxReplicas: ptr.To[int32](2147483647), + }, + }, + expected: 2147483647, // Exactly at limit + }, + } + + for _, tc := range tests { + t.Run(tc.name, func(t *testing.T) { + cluster := &rayv1.RayCluster{ + Spec: rayv1.RayClusterSpec{ + WorkerGroupSpecs: tc.specs, + }, + } + result := CalculateMaxReplicas(cluster) + assert.Equal(t, tc.expected, result) + }) + } +} + func TestUnmarshalRuntimeEnv(t *testing.T) { tests := []struct { name string From 044105c5b2d09d013aab0dbf23cecc117d955dd5 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> Date: Fri, 7 Nov 2025 03:41:43 +0000 Subject: [PATCH 09/32] Add RayService incremental upgrade sample for guide (#4164) Signed-off-by: Ryan O'Leary --- .../ray-service.incremental-upgrade.yaml | 96 +++++++++++++++++++ 1 file changed, 96 insertions(+) create mode 100644 ray-operator/config/samples/ray-service.incremental-upgrade.yaml diff --git a/ray-operator/config/samples/ray-service.incremental-upgrade.yaml b/ray-operator/config/samples/ray-service.incremental-upgrade.yaml new file mode 100644 index 00000000000..0352c128739 --- /dev/null +++ b/ray-operator/config/samples/ray-service.incremental-upgrade.yaml @@ -0,0 +1,96 @@ +apiVersion: ray.io/v1 +kind: RayService +metadata: + name: rayservice-incremental-upgrade +spec: + upgradeStrategy: + type: NewClusterWithIncrementalUpgrade + clusterUpgradeOptions: + gatewayClassName: istio + stepSizePercent: 10 + intervalSeconds: 30 + maxSurgePercent: 10 + serveConfigV2: | + applications: + - name: fruit_app + import_path: fruit.deployment_graph + route_prefix: /fruit + runtime_env: + working_dir: "https://github.com/ray-project/test_dag/archive/78b4a5da38796123d9f9ffff59bab2792a043e95.zip" + deployments: + - name: MangoStand + user_config: + price: 4 + ray_actor_options: + num_cpus: 0.5 + max_ongoing_requests: 100 + autoscaling_config: + min_replicas: 1 + max_replicas: 3 + - name: OrangeStand + num_replicas: 1 + user_config: + price: 2 + ray_actor_options: + num_cpus: 0 + - name: PearStand + num_replicas: 1 + user_config: + price: 1 + ray_actor_options: + num_cpus: 0 + - name: FruitMarket + ray_actor_options: + num_cpus: 0.5 + max_ongoing_requests: 100 + autoscaling_config: + min_replicas: 1 + max_replicas: 3 + rayClusterConfig: + rayVersion: "2.51.0" + enableInTreeAutoscaling: true + headGroupSpec: + rayStartParams: + num-cpus: "0" + template: + spec: + containers: + - name: ray-head + image: rayproject/ray:2.51.0 + resources: + requests: + cpu: "100m" + memory: "100Mi" + limits: + cpu: "1" + memory: "2Gi" + ports: + - containerPort: 6379 + name: gcs-server + - containerPort: 8265 + name: dashboard + - containerPort: 10001 + name: client + - containerPort: 8000 + name: serve + workerGroupSpecs: + - groupName: small-group + minReplicas: 0 + maxReplicas: 4 + rayStartParams: {} + template: + spec: + containers: + - name: ray-worker + image: rayproject/ray:2.51.0 + lifecycle: + preStop: + exec: + command: ["/bin/sh", "-c", "ray stop"] + resources: + requests: + cpu: "500m" + memory: "1Gi" + limits: + cpu: "1" + memory: "4Gi" From 3249d30ea6fd942b679421654e62152438193733 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> Date: Fri, 7 Nov 2025 05:28:06 +0000 Subject: [PATCH 10/32] Edit RayCluster example config for label selectors (#4151) Signed-off-by: Ryan O'Leary --- .../config/samples/ray-cluster-label-selector.yaml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/ray-operator/config/samples/ray-cluster-label-selector.yaml b/ray-operator/config/samples/ray-cluster-label-selector.yaml index 6b436fb1a59..fcf80a6a57e 100644 --- a/ray-operator/config/samples/ray-cluster-label-selector.yaml +++ b/ray-operator/config/samples/ray-cluster-label-selector.yaml @@ -23,7 +23,7 @@ spec: labels: ray.io/region: us-central2 resources: - cpu: "0" + CPU: "0" template: spec: containers: @@ -86,9 +86,11 @@ spec: maxReplicas: 10 groupName: accelerator-group labels: + ray.io/accelerator-type: A100 ray.io/market-type: on-demand ray.io/region: us-central2 - rayStartParams: {} + resources: + GPU: "1" template: spec: containers: @@ -97,11 +99,9 @@ spec: resources: limits: cpu: "1" - nvidia.com/gpu: "1" memory: "1G" requests: cpu: "1" - nvidia.com/gpu: "1" memory: "1G" nodeSelector: cloud.google.com/gke-spot: "true" From 87de75fe1c8f2827857f14571a5ac09c96654dde Mon Sep 17 00:00:00 2001 From: "Han-Ju Chen (Future-Outlier)" Date: Fri, 7 Nov 2025 06:05:31 -0800 Subject: [PATCH 11/32] [RayJob] update light weight submitter image from quay.io (#4181) Signed-off-by: Future-Outlier --- ray-operator/config/samples/ray-job.light-weight-submitter.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ray-operator/config/samples/ray-job.light-weight-submitter.yaml b/ray-operator/config/samples/ray-job.light-weight-submitter.yaml index 39351fe288b..a5e2539048f 100644 --- a/ray-operator/config/samples/ray-job.light-weight-submitter.yaml +++ b/ray-operator/config/samples/ray-job.light-weight-submitter.yaml @@ -70,7 +70,7 @@ spec: restartPolicy: Never containers: - name: my-custom-rayjob-submitter - image: kuberay/submitter:nightly + image: quay.io/kuberay/submitter:v1.5.0 command: ["/submitter"] args: ["--runtime-env-json", '{"pip":["requests==2.26.0","pendulum==2.1.2"],"env_vars":{"counter_name":"test_counter"}}', "--", "python", "/home/ray/samples/sample_code.py"] From 4a623fd5b4e7c8f7563319aab3f8a043eed7cf76 Mon Sep 17 00:00:00 2001 From: "Han-Ju Chen (Future-Outlier)" Date: Fri, 7 Nov 2025 12:53:51 -0800 Subject: [PATCH 12/32] [flaky] RayJob fails when head Pod is deleted when job is running (#4182) Signed-off-by: Future-Outlier --- ray-operator/test/e2erayjob/rayjob_test.go | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/ray-operator/test/e2erayjob/rayjob_test.go b/ray-operator/test/e2erayjob/rayjob_test.go index 35a06cd16b7..c9c8e87791f 100644 --- a/ray-operator/test/e2erayjob/rayjob_test.go +++ b/ray-operator/test/e2erayjob/rayjob_test.go @@ -310,11 +310,10 @@ env_vars: g.Eventually(RayJob(test, rayJob.Namespace, rayJob.Name), TestTimeoutMedium). Should(WithTransform(RayJobDeploymentStatus, Equal(rayv1.JobDeploymentStatusFailed))) g.Eventually(RayJob(test, rayJob.Namespace, rayJob.Name), TestTimeoutMedium). - Should(WithTransform(RayJobReason, Equal(rayv1.JobDeploymentStatusTransitionGracePeriodExceeded))) - g.Eventually(RayJob(test, rayJob.Namespace, rayJob.Name), TestTimeoutMedium). - Should(WithTransform(func(job *rayv1.RayJob) string { return job.Status.Message }, - MatchRegexp("The RayJob submitter finished at .* but the ray job did not reach terminal state within .*"))) - + Should(WithTransform(RayJobReason, Or( + Equal(rayv1.JobDeploymentStatusTransitionGracePeriodExceeded), + Equal(rayv1.SubmissionFailed), + ))) // Cleanup err = test.Client().Ray().RayV1().RayJobs(namespace.Name).Delete(test.Ctx(), rayJob.Name, metav1.DeleteOptions{}) g.Expect(err).NotTo(HaveOccurred()) From 79bd74974bd68ba716d5267f0bd2f69ca4561a6c Mon Sep 17 00:00:00 2001 From: Jun-Hao Wan Date: Thu, 13 Nov 2025 07:31:04 +0800 Subject: [PATCH 13/32] [CI] Pin Docker api version to avoid API version mismatch (#4188) Signed-off-by: win5923 --- .buildkite/setup-env.sh | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.buildkite/setup-env.sh b/.buildkite/setup-env.sh index 497c8dd5d48..68b6ef84cc6 100755 --- a/.buildkite/setup-env.sh +++ b/.buildkite/setup-env.sh @@ -3,6 +3,9 @@ # Install Go export PATH=$PATH:/usr/local/go/bin +# Pin Docker API version +export DOCKER_API_VERSION=1.43 + # Install kind curl -Lo ./kind https://kind.sigs.k8s.io/dl/v0.22.0/kind-linux-amd64 chmod +x ./kind From 0a9a3e32e928dd3c0a637cb9483d1c062903b7f5 Mon Sep 17 00:00:00 2001 From: Divyam Raj <41264059+divyamraj18@users.noreply.github.com> Date: Mon, 17 Nov 2025 08:18:58 +0530 Subject: [PATCH 14/32] Make replicas configurable for kuberay-operator #4180 (#4195) * Make replicas configurable for kuberay-operator #4180 * Make replicas configurable for kuberay-operator #4180 --- helm-chart/kuberay-operator/README.md | 1 + helm-chart/kuberay-operator/templates/deployment.yaml | 2 +- helm-chart/kuberay-operator/tests/deployment_test.yaml | 8 ++++++++ helm-chart/kuberay-operator/values.yaml | 3 +++ 4 files changed, 13 insertions(+), 1 deletion(-) diff --git a/helm-chart/kuberay-operator/README.md b/helm-chart/kuberay-operator/README.md index ecc0f8cf988..85fb7c51a39 100644 --- a/helm-chart/kuberay-operator/README.md +++ b/helm-chart/kuberay-operator/README.md @@ -147,6 +147,7 @@ spec: | nameOverride | string | `"kuberay-operator"` | String to partially override release name. | | fullnameOverride | string | `"kuberay-operator"` | String to fully override release name. | | componentOverride | string | `"kuberay-operator"` | String to override component name. | +| replicas | int | `1` | Number of replicas for the KubeRay operator Deployment. | | image.repository | string | `"quay.io/kuberay/operator"` | Image repository. | | image.tag | string | `"nightly"` | Image tag. | | image.pullPolicy | string | `"IfNotPresent"` | Image pull policy. | diff --git a/helm-chart/kuberay-operator/templates/deployment.yaml b/helm-chart/kuberay-operator/templates/deployment.yaml index 78cb0fe944d..337dcc60bae 100644 --- a/helm-chart/kuberay-operator/templates/deployment.yaml +++ b/helm-chart/kuberay-operator/templates/deployment.yaml @@ -9,7 +9,7 @@ metadata: {{- toYaml . | nindent 4 }} {{- end }} spec: - replicas: 1 + replicas: {{ .Values.replicas | default 1 }} strategy: type: Recreate selector: diff --git a/helm-chart/kuberay-operator/tests/deployment_test.yaml b/helm-chart/kuberay-operator/tests/deployment_test.yaml index 7af12aa9b67..7a420217596 100644 --- a/helm-chart/kuberay-operator/tests/deployment_test.yaml +++ b/helm-chart/kuberay-operator/tests/deployment_test.yaml @@ -276,6 +276,14 @@ tests: path: spec.template.spec.priorityClassName value: high-priority + - it: Should set replicas when `replicas` is set + set: + replicas: 3 + asserts: + - equal: + path: spec.replicas + value: 3 + - it: Should use custom reconcile concurrency when set set: reconcileConcurrency: 5 diff --git a/helm-chart/kuberay-operator/values.yaml b/helm-chart/kuberay-operator/values.yaml index 1b7b46020b0..c566c6525c6 100644 --- a/helm-chart/kuberay-operator/values.yaml +++ b/helm-chart/kuberay-operator/values.yaml @@ -11,6 +11,9 @@ fullnameOverride: kuberay-operator # -- String to override component name. componentOverride: kuberay-operator +# -- Number of replicas for the KubeRay operator Deployment. +replicas: 1 + image: # -- Image repository. repository: quay.io/kuberay/operator From 2a6057473732351bf140c2da034d42f8bf7316b9 Mon Sep 17 00:00:00 2001 From: Nary Yeh <60069744+machichima@users.noreply.github.com> Date: Sun, 16 Nov 2025 21:37:54 -0800 Subject: [PATCH 15/32] [Fix] rayjob update raycluster status (#4192) * feat: check if raycluster status update in rayjob * test: e2e test to check the rayjob raycluster status update --- .../controllers/ray/rayjob_controller.go | 6 ++- ray-operator/test/e2erayjob/rayjob_test.go | 41 +++++++++++++++++++ 2 files changed, 46 insertions(+), 1 deletion(-) diff --git a/ray-operator/controllers/ray/rayjob_controller.go b/ray-operator/controllers/ray/rayjob_controller.go index 5da57de6491..c6832920357 100644 --- a/ray-operator/controllers/ray/rayjob_controller.go +++ b/ray-operator/controllers/ray/rayjob_controller.go @@ -854,10 +854,14 @@ func (r *RayJobReconciler) updateRayJobStatus(ctx context.Context, oldRayJob *ra oldRayJobStatus := oldRayJob.Status newRayJobStatus := newRayJob.Status logger.Info("updateRayJobStatus", "oldRayJobStatus", oldRayJobStatus, "newRayJobStatus", newRayJobStatus) + + rayClusterStatusChanged := utils.InconsistentRayClusterStatus(oldRayJobStatus.RayClusterStatus, newRayJobStatus.RayClusterStatus) + // If a status field is crucial for the RayJob state machine, it MUST be // updated with a distinct JobStatus or JobDeploymentStatus value. if oldRayJobStatus.JobStatus != newRayJobStatus.JobStatus || - oldRayJobStatus.JobDeploymentStatus != newRayJobStatus.JobDeploymentStatus { + oldRayJobStatus.JobDeploymentStatus != newRayJobStatus.JobDeploymentStatus || + rayClusterStatusChanged { if newRayJobStatus.JobDeploymentStatus == rayv1.JobDeploymentStatusComplete || newRayJobStatus.JobDeploymentStatus == rayv1.JobDeploymentStatusFailed { newRayJob.Status.EndTime = &metav1.Time{Time: time.Now()} diff --git a/ray-operator/test/e2erayjob/rayjob_test.go b/ray-operator/test/e2erayjob/rayjob_test.go index c9c8e87791f..716315b5a11 100644 --- a/ray-operator/test/e2erayjob/rayjob_test.go +++ b/ray-operator/test/e2erayjob/rayjob_test.go @@ -433,4 +433,45 @@ env_vars: g.Expect(reason).To(Equal(rayv1.JobDeploymentStatusTransitionGracePeriodExceeded)) g.Expect(message).To(MatchRegexp(`The RayJob submitter finished at .* but the ray job did not reach terminal state within .*`)) }) + + test.T().Run("RayCluster status update propagates to RayJob", func(_ *testing.T) { + rayJobAC := rayv1ac.RayJob("cluster-status-update", namespace.Name). + WithSpec(rayv1ac.RayJobSpec(). + WithRayClusterSpec(NewRayClusterSpec(MountConfigMap[rayv1ac.RayClusterSpecApplyConfiguration](jobs, "/home/ray/jobs"))). + WithEntrypoint("python /home/ray/jobs/long_running.py"). + WithShutdownAfterJobFinishes(false). + WithSubmitterPodTemplate(JobSubmitterPodTemplateApplyConfiguration())) + + rayJob, err := test.Client().Ray().RayV1().RayJobs(namespace.Name).Apply(test.Ctx(), rayJobAC, TestApplyOptions) + g.Expect(err).NotTo(HaveOccurred()) + LogWithTimestamp(test.T(), "Created RayJob %s/%s successfully", rayJob.Namespace, rayJob.Name) + + g.Eventually(RayJob(test, rayJob.Namespace, rayJob.Name), TestTimeoutMedium). + Should(WithTransform(RayJobStatus, Equal(rayv1.JobStatusRunning))) + + rayJob, err = GetRayJob(test, rayJob.Namespace, rayJob.Name) + g.Expect(err).NotTo(HaveOccurred()) + rayCluster, err := GetRayCluster(test, namespace.Name, rayJob.Status.RayClusterName) + g.Expect(err).NotTo(HaveOccurred()) + + originalMaxWorkerReplica := rayCluster.Status.MaxWorkerReplicas + g.Expect(rayJob.Status.RayClusterStatus.MaxWorkerReplicas).To(Equal(originalMaxWorkerReplica)) + + newMaxWorkerReplica := originalMaxWorkerReplica + 2 + rayCluster.Status.MaxWorkerReplicas = newMaxWorkerReplica + _, err = test.Client().Ray().RayV1().RayClusters(namespace.Name).UpdateStatus(test.Ctx(), rayCluster, metav1.UpdateOptions{}) + g.Expect(err).NotTo(HaveOccurred()) + + g.Eventually(func() int32 { + job, err := GetRayJob(test, rayJob.Namespace, rayJob.Name) + if err != nil { + return originalMaxWorkerReplica + } + return job.Status.RayClusterStatus.MaxWorkerReplicas + }, TestTimeoutShort).Should(Equal(newMaxWorkerReplica)) + + err = test.Client().Ray().RayV1().RayJobs(namespace.Name).Delete(test.Ctx(), rayJob.Name, metav1.DeleteOptions{}) + g.Expect(err).NotTo(HaveOccurred()) + LogWithTimestamp(test.T(), "Deleted RayJob %s/%s successfully", rayJob.Namespace, rayJob.Name) + }) } From 7a945934553d7ffee224b5e7eb422c6df07838a2 Mon Sep 17 00:00:00 2001 From: Alima Azamat <92766804+alimaazamat@users.noreply.github.com> Date: Mon, 17 Nov 2025 08:41:41 -0800 Subject: [PATCH 16/32] fix: dashboard http client tests discovered and passing (#4173) Signed-off-by: alimaazamat --- .../ray/utils/dashboardclient/dashboard_httpclient_test.go | 3 ++- .../{utils_suite_test.go => dashboardclient/suite_test.go} | 6 +++--- 2 files changed, 5 insertions(+), 4 deletions(-) rename ray-operator/controllers/ray/utils/{utils_suite_test.go => dashboardclient/suite_test.go} (52%) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_httpclient_test.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_httpclient_test.go index 1bdb3a1fb75..50088a91c5e 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_httpclient_test.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_httpclient_test.go @@ -3,6 +3,7 @@ package dashboardclient import ( "context" "encoding/json" + "errors" "net/http" "github.com/jarcoal/httpmock" @@ -295,6 +296,6 @@ var _ = Describe("RayFrameworkGenerator", func() { _, err := rayDashboardClient.GetServeDetails(context.TODO()) Expect(err).To(HaveOccurred()) - Expect(err).To(Equal(context.DeadlineExceeded)) + Expect(errors.Is(err, context.DeadlineExceeded)).To(BeTrue()) }) }) diff --git a/ray-operator/controllers/ray/utils/utils_suite_test.go b/ray-operator/controllers/ray/utils/dashboardclient/suite_test.go similarity index 52% rename from ray-operator/controllers/ray/utils/utils_suite_test.go rename to ray-operator/controllers/ray/utils/dashboardclient/suite_test.go index 9ca82ff0d5b..707bbcddf82 100644 --- a/ray-operator/controllers/ray/utils/utils_suite_test.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/suite_test.go @@ -1,4 +1,4 @@ -package utils_test +package dashboardclient import ( "testing" @@ -7,7 +7,7 @@ import ( . "github.com/onsi/gomega" ) -func TestUtils(t *testing.T) { +func TestDashboardClient(t *testing.T) { RegisterFailHandler(Fail) - RunSpecs(t, "Utils Suite") + RunSpecs(t, "Dashboard Client Suite") } From cae2f514d1cc010a3de0dcbc50620e952cc964d9 Mon Sep 17 00:00:00 2001 From: Spencer Peterson Date: Mon, 17 Nov 2025 15:18:06 -0800 Subject: [PATCH 17/32] [RayJob] Lift cluster status while initializing (#4191) Signed-off-by: Spencer Peterson --- .../controllers/ray/rayjob_controller.go | 14 ++++-- .../controllers/ray/rayjob_controller_test.go | 28 ++++++++++++ .../controllers/ray/suite_helpers_test.go | 45 ++++++++++++++----- 3 files changed, 71 insertions(+), 16 deletions(-) diff --git a/ray-operator/controllers/ray/rayjob_controller.go b/ray-operator/controllers/ray/rayjob_controller.go index c6832920357..048d5b33758 100644 --- a/ray-operator/controllers/ray/rayjob_controller.go +++ b/ray-operator/controllers/ray/rayjob_controller.go @@ -152,8 +152,8 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) rayJobInstance.Status.Reason = rayv1.ValidationFailed rayJobInstance.Status.Message = err.Error() - // This is the only 2 places where we update the RayJob status. This will directly - // update the JobDeploymentStatus to ValidationFailed if there's validation error + // This is one of the only 3 places where we update the RayJob status. This will directly + // update the JobDeploymentStatus to ValidationFailed if there's validation error. if err = r.updateRayJobStatus(ctx, originalRayJobInstance, rayJobInstance); err != nil { logger.Info("Failed to update RayJob status", "error", err) return ctrl.Result{RequeueAfter: RayJobDefaultRequeueDuration}, err @@ -204,6 +204,12 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) if clientURL := rayJobInstance.Status.DashboardURL; clientURL == "" { if rayClusterInstance.Status.State != rayv1.Ready { logger.Info("Wait for the RayCluster.Status.State to be ready before submitting the job.", "RayCluster", rayClusterInstance.Name, "State", rayClusterInstance.Status.State) + // This is one of only 3 places where we update the RayJob status. For observability + // while waiting for the RayCluster to become ready, we lift the cluster status. + rayJobInstance.Status.RayClusterStatus = rayClusterInstance.Status + if err = r.updateRayJobStatus(ctx, originalRayJobInstance, rayJobInstance); err != nil { + logger.Info("Failed to update RayJob status", "error", err) + } return ctrl.Result{RequeueAfter: RayJobDefaultRequeueDuration}, err } @@ -419,8 +425,8 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) } checkBackoffLimitAndUpdateStatusIfNeeded(ctx, rayJobInstance) - // This is the only 2 places where we update the RayJob status. Please do NOT add any code - // between `checkBackoffLimitAndUpdateStatusIfNeeded` and the following code. + // This is one of the only 3 places where we update the RayJob status. Please do NOT add any + // code between `checkBackoffLimitAndUpdateStatusIfNeeded` and the following code. if err = r.updateRayJobStatus(ctx, originalRayJobInstance, rayJobInstance); err != nil { logger.Info("Failed to update RayJob status", "error", err) return ctrl.Result{RequeueAfter: RayJobDefaultRequeueDuration}, err diff --git a/ray-operator/controllers/ray/rayjob_controller_test.go b/ray-operator/controllers/ray/rayjob_controller_test.go index b9c168400fe..e8ac2be20e3 100644 --- a/ray-operator/controllers/ray/rayjob_controller_test.go +++ b/ray-operator/controllers/ray/rayjob_controller_test.go @@ -241,6 +241,34 @@ var _ = Context("RayJob with different submission modes", func() { Expect(rayCluster.Annotations).Should(Equal(rayJob.Annotations)) }) + It("In Initializing state, the JobStatus should show the RayCluster status", func() { + // The RayCluster is not 'Ready' yet because Pods are not running and ready. + Expect(rayCluster.Status.State).NotTo(Equal(rayv1.Ready)) + + updateHeadPodToRunningNotReady(ctx, rayJob.Status.RayClusterName, namespace) + + // Now the cluster should have nonzero conditions. + Eventually( + func() int { + status := getClusterStatus(ctx, namespace, rayCluster.Name)() + return len(status.Conditions) + }, + time.Second*3, time.Millisecond*500).ShouldNot(Equal(0)) + + // We expect the RayJob's RayClusterStatus to eventually mirror the cluster's status. + Eventually( + func() (int, error) { + currentRayJob := &rayv1.RayJob{} + err := k8sClient.Get(ctx, client.ObjectKey{Name: rayJob.Name, Namespace: namespace}, currentRayJob) + if err != nil { + return 0, err + } + return len(currentRayJob.Status.RayClusterStatus.Conditions), nil + }, + time.Second*3, time.Millisecond*500, + ).ShouldNot(Equal(0)) + }) + It("Make RayCluster.Status.State to be rayv1.Ready", func() { // The RayCluster is not 'Ready' yet because Pods are not running and ready. Expect(rayCluster.Status.State).NotTo(Equal(rayv1.Ready)) diff --git a/ray-operator/controllers/ray/suite_helpers_test.go b/ray-operator/controllers/ray/suite_helpers_test.go index 4ecd9f8aef8..7935291bdfa 100644 --- a/ray-operator/controllers/ray/suite_helpers_test.go +++ b/ray-operator/controllers/ray/suite_helpers_test.go @@ -249,6 +249,36 @@ func checkServeApplicationExists(ctx context.Context, rayService *rayv1.RayServi // So Pods are created, but no controller updates them from Pending to Running. // See https://book.kubebuilder.io/reference/envtest.html for more details. func updateHeadPodToRunningAndReady(ctx context.Context, rayClusterName string, namespace string) { + updateHeadPodToPhaseAndConditions(ctx, rayClusterName, namespace, corev1.PodRunning, []corev1.PodCondition{ + { + Type: corev1.PodReady, + Status: corev1.ConditionTrue, + }, + }) +} + +func updateHeadPodToRunningNotReady(ctx context.Context, rayClusterName string, namespace string) { + updateHeadPodToPhaseAndConditions(ctx, rayClusterName, namespace, corev1.PodRunning, []corev1.PodCondition{ + { + Type: corev1.PodScheduled, + Status: corev1.ConditionTrue, + }, + { + Type: corev1.PodInitialized, + Status: corev1.ConditionTrue, + }, + { + Type: corev1.PodReady, + Status: corev1.ConditionFalse, + }, + { + Type: corev1.ContainersReady, + Status: corev1.ConditionFalse, + }, + }) +} + +func updateHeadPodToPhaseAndConditions(ctx context.Context, rayClusterName string, namespace string, phase corev1.PodPhase, conditions []corev1.PodCondition) { var instance rayv1.RayCluster gomega.Eventually( getResourceFunc(ctx, client.ObjectKey{Name: rayClusterName, Namespace: namespace}, &instance), @@ -262,19 +292,10 @@ func updateHeadPodToRunningAndReady(ctx context.Context, rayClusterName string, time.Second*3, time.Millisecond*500).Should(gomega.Equal(1), "Head pod list should have only 1 Pod = %v", headPods.Items) headPod := headPods.Items[0] - headPod.Status.Phase = corev1.PodRunning - headPod.Status.Conditions = []corev1.PodCondition{ - { - Type: corev1.PodReady, - Status: corev1.ConditionTrue, - }, - } + headPod.Status.Phase = phase + headPod.Status.Conditions = conditions err := k8sClient.Status().Update(ctx, &headPod) - gomega.Expect(err).NotTo(gomega.HaveOccurred(), "Failed to update head Pod status to PodRunning") - - // Make sure the head Pod is updated. - gomega.Eventually( - isAllPodsRunningByFilters).WithContext(ctx).WithArguments(headPods, headLabels).WithTimeout(time.Second*15).WithPolling(time.Millisecond*500).Should(gomega.BeTrue(), "Head Pod should be running: %v", headPods.Items) + gomega.Expect(err).NotTo(gomega.HaveOccurred(), "Failed to update head Pod status to not ready") } // Update the status of the worker Pods to Running and Ready. Similar to updateHeadPodToRunningAndReady. From ac4c3c840b236ee862725a0bc36d367c6a3ca40a Mon Sep 17 00:00:00 2001 From: Spencer Peterson Date: Mon, 17 Nov 2025 17:30:04 -0800 Subject: [PATCH 18/32] [RayJob] Remove updateJobStatus call (#4198) Fast follow to #4191 Signed-off-by: Spencer Peterson --- ray-operator/controllers/ray/rayjob_controller.go | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/ray-operator/controllers/ray/rayjob_controller.go b/ray-operator/controllers/ray/rayjob_controller.go index 048d5b33758..1f522dd8ca2 100644 --- a/ray-operator/controllers/ray/rayjob_controller.go +++ b/ray-operator/controllers/ray/rayjob_controller.go @@ -152,7 +152,7 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) rayJobInstance.Status.Reason = rayv1.ValidationFailed rayJobInstance.Status.Message = err.Error() - // This is one of the only 3 places where we update the RayJob status. This will directly + // This is one of the only 2 places where we update the RayJob status. This will directly // update the JobDeploymentStatus to ValidationFailed if there's validation error. if err = r.updateRayJobStatus(ctx, originalRayJobInstance, rayJobInstance); err != nil { logger.Info("Failed to update RayJob status", "error", err) @@ -204,13 +204,11 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) if clientURL := rayJobInstance.Status.DashboardURL; clientURL == "" { if rayClusterInstance.Status.State != rayv1.Ready { logger.Info("Wait for the RayCluster.Status.State to be ready before submitting the job.", "RayCluster", rayClusterInstance.Name, "State", rayClusterInstance.Status.State) - // This is one of only 3 places where we update the RayJob status. For observability - // while waiting for the RayCluster to become ready, we lift the cluster status. + // The nonready RayCluster status should be reflected in the RayJob's status. + // Breaking from the switch statement will drop directly to the status update code + // and return a default requeue duration and no error. rayJobInstance.Status.RayClusterStatus = rayClusterInstance.Status - if err = r.updateRayJobStatus(ctx, originalRayJobInstance, rayJobInstance); err != nil { - logger.Info("Failed to update RayJob status", "error", err) - } - return ctrl.Result{RequeueAfter: RayJobDefaultRequeueDuration}, err + break } if clientURL, err = utils.FetchHeadServiceURL(ctx, r.Client, rayClusterInstance, utils.DashboardPortName); err != nil || clientURL == "" { @@ -425,7 +423,7 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) } checkBackoffLimitAndUpdateStatusIfNeeded(ctx, rayJobInstance) - // This is one of the only 3 places where we update the RayJob status. Please do NOT add any + // This is one of the only 2 places where we update the RayJob status. Please do NOT add any // code between `checkBackoffLimitAndUpdateStatusIfNeeded` and the following code. if err = r.updateRayJobStatus(ctx, originalRayJobInstance, rayJobInstance); err != nil { logger.Info("Failed to update RayJob status", "error", err) From 58c2aad8d3b339064e76718487f8bea6ed04086e Mon Sep 17 00:00:00 2001 From: Andrew Sy Kim Date: Mon, 17 Nov 2025 22:35:09 -0500 Subject: [PATCH 19/32] Add support for Ray token auth (#4179) * Add support for Ray token auth Signed-off-by: Andrew Sy Kim * add e2e test for Ray cluster auth Signed-off-by: Andrew Sy Kim * address nits from Ruiean Signed-off-by: Andrew Sy Kim * update RAY_auth_mode -> RAY_AUTH_MODE Signed-off-by: Andrew Sy Kim * configure auth for Ray autoscaler Signed-off-by: Andrew Sy Kim --------- Signed-off-by: Andrew Sy Kim --- docs/reference/api.md | 30 ++++++ .../crds/ray.io_rayclusters.yaml | 8 ++ .../kuberay-operator/crds/ray.io_rayjobs.yaml | 8 ++ .../crds/ray.io_rayservices.yaml | 8 ++ .../kuberay-operator/templates/_helpers.tpl | 11 +++ ray-operator/apis/ray/v1/raycluster_types.go | 23 +++++ .../apis/ray/v1/zz_generated.deepcopy.go | 20 ++++ .../config/crd/bases/ray.io_rayclusters.yaml | 8 ++ .../config/crd/bases/ray.io_rayjobs.yaml | 8 ++ .../config/crd/bases/ray.io_rayservices.yaml | 8 ++ ray-operator/config/rbac/role.yaml | 11 +++ ray-operator/controllers/ray/common/pod.go | 47 ++++++++++ .../controllers/ray/raycluster_controller.go | 60 ++++++++++++ .../ray/raycluster_controller_unit_test.go | 93 +++++++++++++++++++ .../controllers/ray/utils/constant.go | 7 ++ ray-operator/controllers/ray/utils/util.go | 5 + .../controllers/ray/utils/validation.go | 20 ++++ .../controllers/ray/utils/validation_test.go | 64 +++++++++++++ .../applyconfiguration/ray/v1/authoptions.go | 27 ++++++ .../ray/v1/rayclusterspec.go | 9 ++ .../pkg/client/applyconfiguration/utils.go | 2 + ray-operator/test/e2e/raycluster_auth_test.go | 82 ++++++++++++++++ 22 files changed, 559 insertions(+) create mode 100644 ray-operator/pkg/client/applyconfiguration/ray/v1/authoptions.go create mode 100644 ray-operator/test/e2e/raycluster_auth_test.go diff --git a/docs/reference/api.md b/docs/reference/api.md index dc621718f0a..fb6a624645e 100644 --- a/docs/reference/api.md +++ b/docs/reference/api.md @@ -16,6 +16,35 @@ Package v1 contains API Schema definitions for the ray v1 API group +#### AuthMode + +_Underlying type:_ _string_ + +AuthMode describes the authentication mode for the Ray cluster. + + + +_Appears in:_ +- [AuthOptions](#authoptions) + + + +#### AuthOptions + + + +AuthOptions defines the authentication options for a RayCluster. + + + +_Appears in:_ +- [RayClusterSpec](#rayclusterspec) + +| Field | Description | Default | Validation | +| --- | --- | --- | --- | +| `mode` _[AuthMode](#authmode)_ | Mode specifies the authentication mode.
Supported values are "disabled" and "token".
Defaults to "token". | | Enum: [disabled token]
| + + #### AutoscalerOptions @@ -268,6 +297,7 @@ _Appears in:_ | Field | Description | Default | Validation | | --- | --- | --- | --- | +| `authOptions` _[AuthOptions](#authoptions)_ | AuthOptions specifies the authentication options for the RayCluster. | | | | `suspend` _boolean_ | Suspend indicates whether a RayCluster should be suspended.
A suspended RayCluster will have head pods and worker pods deleted. | | | | `managedBy` _string_ | ManagedBy is an optional configuration for the controller or entity that manages a RayCluster.
The value must be either 'ray.io/kuberay-operator' or 'kueue.x-k8s.io/multikueue'.
The kuberay-operator reconciles a RayCluster which doesn't have this field at all or
the field value is the reserved string 'ray.io/kuberay-operator',
but delegates reconciling the RayCluster with 'kueue.x-k8s.io/multikueue' to the Kueue.
The field is immutable. | | | | `autoscalerOptions` _[AutoscalerOptions](#autoscaleroptions)_ | AutoscalerOptions specifies optional configuration for the Ray autoscaler. | | | diff --git a/helm-chart/kuberay-operator/crds/ray.io_rayclusters.yaml b/helm-chart/kuberay-operator/crds/ray.io_rayclusters.yaml index d73302f625e..45f5406c411 100644 --- a/helm-chart/kuberay-operator/crds/ray.io_rayclusters.yaml +++ b/helm-chart/kuberay-operator/crds/ray.io_rayclusters.yaml @@ -62,6 +62,14 @@ spec: type: object spec: properties: + authOptions: + properties: + mode: + enum: + - disabled + - token + type: string + type: object autoscalerOptions: properties: env: diff --git a/helm-chart/kuberay-operator/crds/ray.io_rayjobs.yaml b/helm-chart/kuberay-operator/crds/ray.io_rayjobs.yaml index 4ac9eb961a3..1f4c8432168 100644 --- a/helm-chart/kuberay-operator/crds/ray.io_rayjobs.yaml +++ b/helm-chart/kuberay-operator/crds/ray.io_rayjobs.yaml @@ -144,6 +144,14 @@ spec: type: object rayClusterSpec: properties: + authOptions: + properties: + mode: + enum: + - disabled + - token + type: string + type: object autoscalerOptions: properties: env: diff --git a/helm-chart/kuberay-operator/crds/ray.io_rayservices.yaml b/helm-chart/kuberay-operator/crds/ray.io_rayservices.yaml index 2595ecec7ba..baec5cfe1f3 100644 --- a/helm-chart/kuberay-operator/crds/ray.io_rayservices.yaml +++ b/helm-chart/kuberay-operator/crds/ray.io_rayservices.yaml @@ -42,6 +42,14 @@ spec: type: boolean rayClusterConfig: properties: + authOptions: + properties: + mode: + enum: + - disabled + - token + type: string + type: object autoscalerOptions: properties: env: diff --git a/helm-chart/kuberay-operator/templates/_helpers.tpl b/helm-chart/kuberay-operator/templates/_helpers.tpl index d5e0e7352d0..a495b816bd9 100644 --- a/helm-chart/kuberay-operator/templates/_helpers.tpl +++ b/helm-chart/kuberay-operator/templates/_helpers.tpl @@ -169,6 +169,17 @@ rules: - pods/resize verbs: - patch +- apiGroups: + - "" + resources: + - secrets + verbs: + - create + - get + - list + - patch + - update + - watch - apiGroups: - "" resources: diff --git a/ray-operator/apis/ray/v1/raycluster_types.go b/ray-operator/apis/ray/v1/raycluster_types.go index 6a6d40b8278..f67a80ba854 100644 --- a/ray-operator/apis/ray/v1/raycluster_types.go +++ b/ray-operator/apis/ray/v1/raycluster_types.go @@ -11,6 +11,9 @@ import ( // RayClusterSpec defines the desired state of RayCluster type RayClusterSpec struct { + // AuthOptions specifies the authentication options for the RayCluster. + // +optional + AuthOptions *AuthOptions `json:"authOptions,omitempty"` // Suspend indicates whether a RayCluster should be suspended. // A suspended RayCluster will have head pods and worker pods deleted. // +optional @@ -46,6 +49,26 @@ type RayClusterSpec struct { WorkerGroupSpecs []WorkerGroupSpec `json:"workerGroupSpecs,omitempty"` } +// AuthMode describes the authentication mode for the Ray cluster. +type AuthMode string + +const ( + // AuthModeDisabled disables authentication. + AuthModeDisabled AuthMode = "disabled" + // AuthModeToken enables token-based authentication. + AuthModeToken AuthMode = "token" +) + +// AuthOptions defines the authentication options for a RayCluster. +type AuthOptions struct { + // Mode specifies the authentication mode. + // Supported values are "disabled" and "token". + // Defaults to "token". + // +kubebuilder:validation:Enum=disabled;token + // +optional + Mode AuthMode `json:"mode,omitempty"` +} + // GcsFaultToleranceOptions contains configs for GCS FT type GcsFaultToleranceOptions struct { // +optional diff --git a/ray-operator/apis/ray/v1/zz_generated.deepcopy.go b/ray-operator/apis/ray/v1/zz_generated.deepcopy.go index 8deb750000c..cd710592d98 100644 --- a/ray-operator/apis/ray/v1/zz_generated.deepcopy.go +++ b/ray-operator/apis/ray/v1/zz_generated.deepcopy.go @@ -32,6 +32,21 @@ func (in *AppStatus) DeepCopy() *AppStatus { return out } +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *AuthOptions) DeepCopyInto(out *AuthOptions) { + *out = *in +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new AuthOptions. +func (in *AuthOptions) DeepCopy() *AuthOptions { + if in == nil { + return nil + } + out := new(AuthOptions) + in.DeepCopyInto(out) + return out +} + // DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. func (in *AutoscalerOptions) DeepCopyInto(out *AutoscalerOptions) { *out = *in @@ -363,6 +378,11 @@ func (in *RayClusterList) DeepCopyObject() runtime.Object { // DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. func (in *RayClusterSpec) DeepCopyInto(out *RayClusterSpec) { *out = *in + if in.AuthOptions != nil { + in, out := &in.AuthOptions, &out.AuthOptions + *out = new(AuthOptions) + **out = **in + } if in.Suspend != nil { in, out := &in.Suspend, &out.Suspend *out = new(bool) diff --git a/ray-operator/config/crd/bases/ray.io_rayclusters.yaml b/ray-operator/config/crd/bases/ray.io_rayclusters.yaml index d73302f625e..45f5406c411 100644 --- a/ray-operator/config/crd/bases/ray.io_rayclusters.yaml +++ b/ray-operator/config/crd/bases/ray.io_rayclusters.yaml @@ -62,6 +62,14 @@ spec: type: object spec: properties: + authOptions: + properties: + mode: + enum: + - disabled + - token + type: string + type: object autoscalerOptions: properties: env: diff --git a/ray-operator/config/crd/bases/ray.io_rayjobs.yaml b/ray-operator/config/crd/bases/ray.io_rayjobs.yaml index 4ac9eb961a3..1f4c8432168 100644 --- a/ray-operator/config/crd/bases/ray.io_rayjobs.yaml +++ b/ray-operator/config/crd/bases/ray.io_rayjobs.yaml @@ -144,6 +144,14 @@ spec: type: object rayClusterSpec: properties: + authOptions: + properties: + mode: + enum: + - disabled + - token + type: string + type: object autoscalerOptions: properties: env: diff --git a/ray-operator/config/crd/bases/ray.io_rayservices.yaml b/ray-operator/config/crd/bases/ray.io_rayservices.yaml index 2595ecec7ba..baec5cfe1f3 100644 --- a/ray-operator/config/crd/bases/ray.io_rayservices.yaml +++ b/ray-operator/config/crd/bases/ray.io_rayservices.yaml @@ -42,6 +42,14 @@ spec: type: boolean rayClusterConfig: properties: + authOptions: + properties: + mode: + enum: + - disabled + - token + type: string + type: object autoscalerOptions: properties: env: diff --git a/ray-operator/config/rbac/role.yaml b/ray-operator/config/rbac/role.yaml index 9ea1db93190..edd446f97d2 100644 --- a/ray-operator/config/rbac/role.yaml +++ b/ray-operator/config/rbac/role.yaml @@ -54,6 +54,17 @@ rules: - pods/resize verbs: - patch +- apiGroups: + - "" + resources: + - secrets + verbs: + - create + - get + - list + - patch + - update + - watch - apiGroups: - "" resources: diff --git a/ray-operator/controllers/ray/common/pod.go b/ray-operator/controllers/ray/common/pod.go index 21533cd8f49..a42f9184912 100644 --- a/ray-operator/controllers/ray/common/pod.go +++ b/ray-operator/controllers/ray/common/pod.go @@ -199,6 +199,12 @@ func DefaultHeadPodTemplate(ctx context.Context, instance rayv1.RayCluster, head autoscalerImage := podTemplate.Spec.Containers[utils.RayContainerIndex].Image // inject autoscaler container into head pod autoscalerContainer := BuildAutoscalerContainer(autoscalerImage) + + // Configure RAY_AUTH_TOKEN and RAY_AUTH_MODE if auth is enabled. + if utils.IsAuthEnabled(&instance.Spec) { + setContainerTokenAuthEnvVars(instance.Name, &autoscalerContainer) + } + // Merge the user overrides from autoscalerOptions into the autoscaler container config. mergeAutoscalerOverrides(&autoscalerContainer, instance.Spec.AutoscalerOptions) podTemplate.Spec.Containers = append(podTemplate.Spec.Containers, autoscalerContainer) @@ -221,6 +227,10 @@ func DefaultHeadPodTemplate(ctx context.Context, instance rayv1.RayCluster, head podTemplate.Spec.Containers[utils.RayContainerIndex].Ports = append(podTemplate.Spec.Containers[utils.RayContainerIndex].Ports, metricsPort) } + if utils.IsAuthEnabled(&instance.Spec) { + configureTokenAuth(instance.Name, &podTemplate) + } + return podTemplate } @@ -236,6 +246,39 @@ func setAutoscalerV2EnvVars(podTemplate *corev1.PodTemplateSpec) { }) } +// configureTokenAuth sets environment variables required for Ray token authentication +func configureTokenAuth(clusterName string, podTemplate *corev1.PodTemplateSpec) { + setContainerTokenAuthEnvVars(clusterName, &podTemplate.Spec.Containers[utils.RayContainerIndex]) + + // Configure auth token for wait-gcs-ready init container if it exists + for i, initContainer := range podTemplate.Spec.InitContainers { + if initContainer.Name != "wait-gcs-ready" { + continue + } + + setContainerTokenAuthEnvVars(clusterName, &podTemplate.Spec.InitContainers[i]) + } +} + +// setContainerTokenAuthEnvVars sets Ray authentication env vars for a container. +func setContainerTokenAuthEnvVars(clusterName string, container *corev1.Container) { + container.Env = append(container.Env, corev1.EnvVar{ + Name: utils.RAY_AUTH_MODE_ENV_VAR, + Value: string(rayv1.AuthModeToken), + }) + + secretName := utils.CheckName(clusterName) + container.Env = append(container.Env, corev1.EnvVar{ + Name: utils.RAY_AUTH_TOKEN_ENV_VAR, + ValueFrom: &corev1.EnvVarSource{ + SecretKeyRef: &corev1.SecretKeySelector{ + LocalObjectReference: corev1.LocalObjectReference{Name: secretName}, + Key: utils.RAY_AUTH_TOKEN_SECRET_KEY, + }, + }, + }) +} + func getEnableInitContainerInjection() bool { if s := os.Getenv(EnableInitContainerInjectionEnvKey); strings.ToLower(s) == "false" { return false @@ -358,6 +401,10 @@ func DefaultWorkerPodTemplate(ctx context.Context, instance rayv1.RayCluster, wo podTemplate.Spec.RestartPolicy = corev1.RestartPolicyNever } + if utils.IsAuthEnabled(&instance.Spec) { + configureTokenAuth(instance.Name, &podTemplate) + } + return podTemplate } diff --git a/ray-operator/controllers/ray/raycluster_controller.go b/ray-operator/controllers/ray/raycluster_controller.go index 8257f3d034a..7d4d8fab8f6 100644 --- a/ray-operator/controllers/ray/raycluster_controller.go +++ b/ray-operator/controllers/ray/raycluster_controller.go @@ -2,6 +2,8 @@ package ray import ( "context" + "crypto/rand" + "encoding/base64" errstd "errors" "fmt" "os" @@ -98,6 +100,7 @@ type RayClusterReconcilerOptions struct { // +kubebuilder:rbac:groups=core,resources=pods,verbs=get;list;watch;create;update;patch;delete;deletecollection // +kubebuilder:rbac:groups=core,resources=pods/status,verbs=get;list;watch;create;update;patch;delete // +kubebuilder:rbac:groups=core,resources=pods/resize,verbs=patch +// +kubebuilder:rbac:groups=core,resources=secrets,verbs=get;list;watch;create;update;patch // +kubebuilder:rbac:groups=core,resources=services,verbs=get;list;watch;create;update;patch;delete // +kubebuilder:rbac:groups=core,resources=services/status,verbs=get;update;patch // +kubebuilder:rbac:groups=coordination.k8s.io,resources=leases,verbs=get;list;create;update @@ -298,6 +301,7 @@ func (r *RayClusterReconciler) rayClusterReconcile(ctx context.Context, instance r.reconcileAutoscalerRole, r.reconcileAutoscalerRoleBinding, r.reconcileIngress, + r.reconcileAuthSecret, r.reconcileHeadService, r.reconcileHeadlessService, r.reconcileServeService, @@ -354,6 +358,62 @@ func (r *RayClusterReconciler) rayClusterReconcile(ctx context.Context, instance return ctrl.Result{RequeueAfter: time.Duration(requeueAfterSeconds) * time.Second}, nil } +func (r *RayClusterReconciler) reconcileAuthSecret(ctx context.Context, instance *rayv1.RayCluster) error { + logger := ctrl.LoggerFrom(ctx) + logger.Info("Reconciling Auth") + + if instance.Spec.AuthOptions == nil || instance.Spec.AuthOptions.Mode == rayv1.AuthModeDisabled { + return nil + } + + secret := &corev1.Secret{} + secretName := utils.CheckName(instance.Name) + err := r.Get(ctx, types.NamespacedName{Name: secretName, Namespace: instance.Namespace}, secret) + if err != nil { + if errors.IsNotFound(err) { + return r.createAuthSecret(ctx, instance, secretName) + } + return err + } + + return nil +} + +// createAuthSecret generates a new secret with a random token. +func (r *RayClusterReconciler) createAuthSecret(ctx context.Context, rayCluster *rayv1.RayCluster, secretName string) error { + token, err := generateRandomToken(32) + if err != nil { + return err + } + + secret := &corev1.Secret{ + ObjectMeta: metav1.ObjectMeta{ + Name: secretName, + Namespace: rayCluster.Namespace, + Labels: map[string]string{ + utils.RayClusterLabelKey: rayCluster.Name, + }, + OwnerReferences: []metav1.OwnerReference{ + *metav1.NewControllerRef(rayCluster, rayv1.SchemeGroupVersion.WithKind("RayCluster")), + }, + }, + StringData: map[string]string{ + "auth_token": token, + }, + } + + return r.Create(ctx, secret) +} + +// generateRandomToken creates a random base64 encoded string. +func generateRandomToken(length int) (string, error) { + bytes := make([]byte, length) + if _, err := rand.Read(bytes); err != nil { + return "", err + } + return base64.StdEncoding.EncodeToString(bytes), nil +} + func (r *RayClusterReconciler) reconcileIngress(ctx context.Context, instance *rayv1.RayCluster) error { logger := ctrl.LoggerFrom(ctx) logger.Info("Reconciling Ingress") diff --git a/ray-operator/controllers/ray/raycluster_controller_unit_test.go b/ray-operator/controllers/ray/raycluster_controller_unit_test.go index 6749b80dea3..9c1ac017023 100644 --- a/ray-operator/controllers/ray/raycluster_controller_unit_test.go +++ b/ray-operator/controllers/ray/raycluster_controller_unit_test.go @@ -17,9 +17,11 @@ package ray import ( "context" + "encoding/base64" "errors" "math" "os" + "reflect" "strconv" "strings" "testing" @@ -3550,3 +3552,94 @@ func TestSetDefaults(t *testing.T) { assert.Equal(t, map[string]string{}, cluster.Spec.WorkerGroupSpecs[i].RayStartParams) } } + +func TestReconcile_AuthSecret(t *testing.T) { + setupTest(t) + + testRayCluster.Spec.AuthOptions = &rayv1.AuthOptions{Mode: rayv1.AuthModeToken} + + fakeClient := clientFake.NewClientBuilder().WithRuntimeObjects(testPods...).Build() + ctx := context.Background() + + secretNamespacedName := types.NamespacedName{ + Name: instanceName, + Namespace: namespaceStr, + } + + secret := corev1.Secret{} + err := fakeClient.Get(ctx, secretNamespacedName, &secret) + assert.True(t, k8serrors.IsNotFound(err), "Secret should not exist yet") + + testRayClusterReconciler := &RayClusterReconciler{ + Client: fakeClient, + Recorder: &record.FakeRecorder{}, + Scheme: scheme.Scheme, + rayClusterScaleExpectation: expectations.NewRayClusterScaleExpectation(fakeClient), + } + + err = testRayClusterReconciler.reconcileAuthSecret(ctx, testRayCluster) + require.NoError(t, err, "Fail to reconcile auth token secret") + + err = fakeClient.Get(ctx, secretNamespacedName, &secret) + require.NoError(t, err, "Fail to get auth Secret after reconciliation") + + decodedBytes, err := base64.StdEncoding.DecodeString(secret.StringData["auth_token"]) + require.NoError(t, err) + + assert.Len(t, decodedBytes, 32) +} + +func TestReconcile_PodsWithAuthToken(t *testing.T) { + setupTest(t) + + testRayCluster.Spec.AuthOptions = &rayv1.AuthOptions{Mode: rayv1.AuthModeToken} + + fakeClient := clientFake.NewClientBuilder().WithRuntimeObjects().Build() + ctx := context.Background() + + testRayClusterReconciler := &RayClusterReconciler{ + Client: fakeClient, + Recorder: &record.FakeRecorder{}, + Scheme: scheme.Scheme, + rayClusterScaleExpectation: expectations.NewRayClusterScaleExpectation(fakeClient), + } + + err := testRayClusterReconciler.reconcilePods(ctx, testRayCluster) + require.NoError(t, err, "Fail to reconcile Pods") + + podList := corev1.PodList{} + err = fakeClient.List(ctx, &podList, client.InNamespace(namespaceStr)) + require.NoError(t, err, "Fail to get pod list") + numAllPods := len(podList.Items) + expectedNumPods := int(*testRayCluster.Spec.WorkerGroupSpecs[0].Replicas) + 1 + assert.Equal(t, expectedNumPods, numAllPods, "unexpected number of pods") + + // Assert that all Pods have RAY_AUTH_MODE and RAY_AUTH_TOKEN environment variables + for _, pod := range podList.Items { + authTokenEnvFound := false + authModeEnvFound := false + for _, env := range pod.Spec.Containers[utils.RayContainerIndex].Env { + if reflect.DeepEqual(corev1.EnvVar{Name: utils.RAY_AUTH_MODE_ENV_VAR, Value: string(rayv1.AuthModeToken)}, env) { + authModeEnvFound = true + continue + } + + expectedSecretValue := corev1.EnvVar{ + Name: utils.RAY_AUTH_TOKEN_ENV_VAR, + ValueFrom: &corev1.EnvVarSource{ + SecretKeyRef: &corev1.SecretKeySelector{ + LocalObjectReference: corev1.LocalObjectReference{Name: testRayCluster.Name}, + Key: "auth_token", + }, + }, + } + if reflect.DeepEqual(expectedSecretValue, env) { + authTokenEnvFound = true + continue + } + } + + assert.True(t, authTokenEnvFound, "Auth token env vars not found") + assert.True(t, authModeEnvFound, "Auth mode env vars not found") + } +} diff --git a/ray-operator/controllers/ray/utils/constant.go b/ray-operator/controllers/ray/utils/constant.go index e707cb71023..3e0e86d1096 100644 --- a/ray-operator/controllers/ray/utils/constant.go +++ b/ray-operator/controllers/ray/utils/constant.go @@ -156,6 +156,13 @@ const ( RAY_NODE_TYPE_NAME = "RAY_NODE_TYPE_NAME" RAY_ENABLE_AUTOSCALER_V2 = "RAY_enable_autoscaler_v2" + // RAY_AUTH_MODE_ENV_VAR is the Ray environment variable for configuring the authentication mode + RAY_AUTH_MODE_ENV_VAR = "RAY_AUTH_MODE" + // RAY_AUTH_TOKEN_ENV_VAR is the Ray environment variable containing the authentication token. + RAY_AUTH_TOKEN_ENV_VAR = "RAY_AUTH_TOKEN" // #nosec G101 + // RAY_AUTH_TOKEN_SECRET_KEY is the key used in the Secret containing Ray auth token + RAY_AUTH_TOKEN_SECRET_KEY = "auth_token" + // This KubeRay operator environment variable is used to determine if random Pod // deletion should be enabled. Note that this only takes effect when autoscaling // is enabled for the RayCluster. This is a feature flag for v0.6.0, and will be diff --git a/ray-operator/controllers/ray/utils/util.go b/ray-operator/controllers/ray/utils/util.go index 5235054b3f8..cc4ed92fec7 100644 --- a/ray-operator/controllers/ray/utils/util.go +++ b/ray-operator/controllers/ray/utils/util.go @@ -689,6 +689,11 @@ func IsGCSFaultToleranceEnabled(spec *rayv1.RayClusterSpec, annotations map[stri return (ok && strings.ToLower(v) == "true") || spec.GcsFaultToleranceOptions != nil } +// IsAuthEnabled returns whether Ray auth is enabled. +func IsAuthEnabled(spec *rayv1.RayClusterSpec) bool { + return spec.AuthOptions != nil && spec.AuthOptions.Mode == rayv1.AuthModeToken +} + // GetRayClusterNameFromService returns the name of the RayCluster that the service points to func GetRayClusterNameFromService(svc *corev1.Service) string { if svc == nil || svc.Spec.Selector == nil { diff --git a/ray-operator/controllers/ray/utils/validation.go b/ray-operator/controllers/ray/utils/validation.go index c7ff5ec782a..a3e6da66e80 100644 --- a/ray-operator/controllers/ray/utils/validation.go +++ b/ray-operator/controllers/ray/utils/validation.go @@ -11,6 +11,7 @@ import ( "k8s.io/apimachinery/pkg/api/meta" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "k8s.io/apimachinery/pkg/util/validation" + "k8s.io/apimachinery/pkg/util/version" rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1" "github.com/ray-project/kuberay/ray-operator/controllers/ray/utils/dashboardclient" @@ -189,6 +190,25 @@ func ValidateRayClusterSpec(spec *rayv1.RayClusterSpec, annotations map[string]s } } } + + if IsAuthEnabled(spec) { + if spec.RayVersion == "" { + return fmt.Errorf("authOptions.mode is 'token' but RayVersion was not specified. Ray version 2.51.0 or later is required") + } + + rayVersion, err := version.ParseGeneric(spec.RayVersion) + if err != nil { + return fmt.Errorf("authOptions.mode is 'token' but RayVersion format is invalid: %s, %w", spec.RayVersion, err) + } + + // Require minimum Ray version 2.51.0 + minVersion := version.MustParseGeneric("2.51.0") + if rayVersion.LessThan(minVersion) { + return fmt.Errorf("authOptions.mode is 'token' but minimum Ray version is 2.51.0, got %s", spec.RayVersion) + } + + } + return nil } diff --git a/ray-operator/controllers/ray/utils/validation_test.go b/ray-operator/controllers/ray/utils/validation_test.go index 88aee6c9fc4..494fe39c29c 100644 --- a/ray-operator/controllers/ray/utils/validation_test.go +++ b/ray-operator/controllers/ray/utils/validation_test.go @@ -851,6 +851,70 @@ func TestValidateRayClusterSpec_Labels(t *testing.T) { } } +func TestValidateRayClusterSpecRayVersionForAuth(t *testing.T) { + tests := []struct { + name string + rayVersion string + errorMessage string + expectError bool + }{ + { + name: "Valid Ray version 2.51.0", + rayVersion: "2.51.0", + expectError: false, + }, + { + name: "Valid Ray version 2.52.0", + rayVersion: "2.52.0", + expectError: false, + }, + { + name: "Valid Ray version 3.0.0", + rayVersion: "3.0.0", + expectError: false, + }, + { + name: "Invalid Ray version 2.50.0", + rayVersion: "2.50.0", + expectError: true, + errorMessage: "authOptions.mode is 'token' but minimum Ray version is 2.51.0, got 2.50.0", + }, + { + name: "Invalid Ray version format", + rayVersion: "invalid-version", + expectError: true, + errorMessage: "authOptions.mode is 'token' but RayVersion format is invalid: invalid-version, could not parse \"invalid-version\" as version", + }, + { + name: "Empty Ray version", + rayVersion: "", + expectError: true, + errorMessage: "authOptions.mode is 'token' but RayVersion was not specified. Ray version 2.51.0 or later is required", + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + spec := &rayv1.RayClusterSpec{ + RayVersion: tt.rayVersion, + HeadGroupSpec: rayv1.HeadGroupSpec{ + Template: podTemplateSpec(nil, nil), + }, + AuthOptions: &rayv1.AuthOptions{ + Mode: rayv1.AuthModeToken, + }, + } + err := ValidateRayClusterSpec(spec, nil) + if tt.expectError { + require.Error(t, err) + assert.Contains(t, err.Error(), tt.errorMessage) + } else { + require.NoError(t, err) + } + }) + } +} + func TestValidateRayJobStatus(t *testing.T) { tests := []struct { name string diff --git a/ray-operator/pkg/client/applyconfiguration/ray/v1/authoptions.go b/ray-operator/pkg/client/applyconfiguration/ray/v1/authoptions.go new file mode 100644 index 00000000000..6f2cdaf680b --- /dev/null +++ b/ray-operator/pkg/client/applyconfiguration/ray/v1/authoptions.go @@ -0,0 +1,27 @@ +// Code generated by applyconfiguration-gen. DO NOT EDIT. + +package v1 + +import ( + rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1" +) + +// AuthOptionsApplyConfiguration represents a declarative configuration of the AuthOptions type for use +// with apply. +type AuthOptionsApplyConfiguration struct { + Mode *rayv1.AuthMode `json:"mode,omitempty"` +} + +// AuthOptionsApplyConfiguration constructs a declarative configuration of the AuthOptions type for use with +// apply. +func AuthOptions() *AuthOptionsApplyConfiguration { + return &AuthOptionsApplyConfiguration{} +} + +// WithMode sets the Mode field in the declarative configuration to the given value +// and returns the receiver, so that objects can be built by chaining "With" function invocations. +// If called multiple times, the Mode field is set to the value of the last call. +func (b *AuthOptionsApplyConfiguration) WithMode(value rayv1.AuthMode) *AuthOptionsApplyConfiguration { + b.Mode = &value + return b +} diff --git a/ray-operator/pkg/client/applyconfiguration/ray/v1/rayclusterspec.go b/ray-operator/pkg/client/applyconfiguration/ray/v1/rayclusterspec.go index 7508aa5bb31..14cb592037f 100644 --- a/ray-operator/pkg/client/applyconfiguration/ray/v1/rayclusterspec.go +++ b/ray-operator/pkg/client/applyconfiguration/ray/v1/rayclusterspec.go @@ -5,6 +5,7 @@ package v1 // RayClusterSpecApplyConfiguration represents a declarative configuration of the RayClusterSpec type for use // with apply. type RayClusterSpecApplyConfiguration struct { + AuthOptions *AuthOptionsApplyConfiguration `json:"authOptions,omitempty"` Suspend *bool `json:"suspend,omitempty"` ManagedBy *string `json:"managedBy,omitempty"` AutoscalerOptions *AutoscalerOptionsApplyConfiguration `json:"autoscalerOptions,omitempty"` @@ -22,6 +23,14 @@ func RayClusterSpec() *RayClusterSpecApplyConfiguration { return &RayClusterSpecApplyConfiguration{} } +// WithAuthOptions sets the AuthOptions field in the declarative configuration to the given value +// and returns the receiver, so that objects can be built by chaining "With" function invocations. +// If called multiple times, the AuthOptions field is set to the value of the last call. +func (b *RayClusterSpecApplyConfiguration) WithAuthOptions(value *AuthOptionsApplyConfiguration) *RayClusterSpecApplyConfiguration { + b.AuthOptions = value + return b +} + // WithSuspend sets the Suspend field in the declarative configuration to the given value // and returns the receiver, so that objects can be built by chaining "With" function invocations. // If called multiple times, the Suspend field is set to the value of the last call. diff --git a/ray-operator/pkg/client/applyconfiguration/utils.go b/ray-operator/pkg/client/applyconfiguration/utils.go index 6173cf5c943..9ad6513e0c8 100644 --- a/ray-operator/pkg/client/applyconfiguration/utils.go +++ b/ray-operator/pkg/client/applyconfiguration/utils.go @@ -18,6 +18,8 @@ func ForKind(kind schema.GroupVersionKind) interface{} { // Group=ray.io, Version=v1 case v1.SchemeGroupVersion.WithKind("AppStatus"): return &rayv1.AppStatusApplyConfiguration{} + case v1.SchemeGroupVersion.WithKind("AuthOptions"): + return &rayv1.AuthOptionsApplyConfiguration{} case v1.SchemeGroupVersion.WithKind("AutoscalerOptions"): return &rayv1.AutoscalerOptionsApplyConfiguration{} case v1.SchemeGroupVersion.WithKind("ClusterUpgradeOptions"): diff --git a/ray-operator/test/e2e/raycluster_auth_test.go b/ray-operator/test/e2e/raycluster_auth_test.go new file mode 100644 index 00000000000..67f2a7e4113 --- /dev/null +++ b/ray-operator/test/e2e/raycluster_auth_test.go @@ -0,0 +1,82 @@ +package e2e + +import ( + "testing" + + . "github.com/onsi/gomega" + corev1 "k8s.io/api/core/v1" + + rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1" + "github.com/ray-project/kuberay/ray-operator/controllers/ray/utils" + rayv1ac "github.com/ray-project/kuberay/ray-operator/pkg/client/applyconfiguration/ray/v1" + . "github.com/ray-project/kuberay/ray-operator/test/support" +) + +// NewRayClusterSpecWithAuth creates a new RayClusterSpec with the specified AuthMode. +func NewRayClusterSpecWithAuth(authMode rayv1.AuthMode) *rayv1ac.RayClusterSpecApplyConfiguration { + return NewRayClusterSpec(). + WithAuthOptions(rayv1ac.AuthOptions().WithMode(authMode)) +} + +func TestRayClusterAuthOptions(t *testing.T) { + test := With(t) + g := NewWithT(t) + + namespace := test.NewTestNamespace() + + test.T().Run("RayCluster with token authentication enabled", func(t *testing.T) { + t.Parallel() + + rayClusterAC := rayv1ac.RayCluster("raycluster-auth-token", namespace.Name). + WithSpec(NewRayClusterSpecWithAuth(rayv1.AuthModeToken).WithRayVersion("2.51")) + + rayCluster, err := test.Client().Ray().RayV1().RayClusters(namespace.Name).Apply(test.Ctx(), rayClusterAC, TestApplyOptions) + g.Expect(err).NotTo(HaveOccurred()) + LogWithTimestamp(test.T(), "Created RayCluster %s/%s successfully with AuthModeToken", rayCluster.Namespace, rayCluster.Name) + + LogWithTimestamp(test.T(), "Waiting for RayCluster %s/%s to become ready", rayCluster.Namespace, rayCluster.Name) + g.Eventually(RayCluster(test, rayCluster.Namespace, rayCluster.Name), TestTimeoutMedium). + Should(WithTransform(RayClusterState, Equal(rayv1.Ready))) + + headPod, err := GetHeadPod(test, rayCluster) + g.Expect(err).NotTo(HaveOccurred()) + g.Expect(headPod).NotTo(BeNil()) + verifyAuthTokenEnvVars(t, rayCluster, *headPod) + + workerPods, err := GetWorkerPods(test, rayCluster) + g.Expect(err).NotTo(HaveOccurred()) + g.Expect(workerPods).ToNot(BeEmpty()) + for _, workerPod := range workerPods { + verifyAuthTokenEnvVars(t, rayCluster, workerPod) + } + + // TODO(andrewsykim): add job submission test with and without token once a Ray version with token support is released. + }) +} + +func verifyAuthTokenEnvVars(t *testing.T, rayCluster *rayv1.RayCluster, pod corev1.Pod) { + g := NewWithT(t) + + var rayAuthModeEnvVar *corev1.EnvVar + for _, envVar := range pod.Spec.Containers[0].Env { + if envVar.Name == utils.RAY_AUTH_MODE_ENV_VAR { + rayAuthModeEnvVar = &envVar + break + } + } + g.Expect(rayAuthModeEnvVar).NotTo(BeNil(), "RAY_AUTH_MODE environment variable should be set") + g.Expect(rayAuthModeEnvVar.Value).To(Equal(string(rayv1.AuthModeToken)), "RAY_AUTH_MODE should be %s", rayv1.AuthModeToken) + + var rayAuthTokenEnvVar *corev1.EnvVar + for _, envVar := range pod.Spec.Containers[0].Env { + if envVar.Name == utils.RAY_AUTH_TOKEN_ENV_VAR { + rayAuthTokenEnvVar = &envVar + break + } + } + g.Expect(rayAuthTokenEnvVar).NotTo(BeNil(), "RAY_AUTH_TOKEN environment variable should be set for AuthModeToken") + g.Expect(rayAuthTokenEnvVar.ValueFrom).NotTo(BeNil(), "RAY_AUTH_TOKEN should be populated from a secret") + g.Expect(rayAuthTokenEnvVar.ValueFrom.SecretKeyRef).NotTo(BeNil(), "RAY_AUTH_TOKEN should be populated from a secret key ref") + g.Expect(rayAuthTokenEnvVar.ValueFrom.SecretKeyRef.Name).To(ContainSubstring(rayCluster.Name), "Secret name should contain RayCluster name") + g.Expect(rayAuthTokenEnvVar.ValueFrom.SecretKeyRef.Key).To(Equal(utils.RAY_AUTH_TOKEN_SECRET_KEY), "Secret key should be %s", utils.RAY_AUTH_TOKEN_SECRET_KEY) +} From 6a809e0ab742f4f8a0be6ac62658d192e7b8ee59 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 18 Nov 2025 10:32:28 -0800 Subject: [PATCH 20/32] Bump js-yaml from 4.1.0 to 4.1.1 in /dashboard (#4194) Bumps [js-yaml](https://github.com/nodeca/js-yaml) from 4.1.0 to 4.1.1. - [Changelog](https://github.com/nodeca/js-yaml/blob/master/CHANGELOG.md) - [Commits](https://github.com/nodeca/js-yaml/compare/4.1.0...4.1.1) --- updated-dependencies: - dependency-name: js-yaml dependency-version: 4.1.1 dependency-type: indirect ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- dashboard/yarn.lock | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dashboard/yarn.lock b/dashboard/yarn.lock index c432647f2da..808ceef6d00 100644 --- a/dashboard/yarn.lock +++ b/dashboard/yarn.lock @@ -3699,13 +3699,13 @@ __metadata: linkType: hard "js-yaml@npm:^4.1.0": - version: 4.1.0 - resolution: "js-yaml@npm:4.1.0" + version: 4.1.1 + resolution: "js-yaml@npm:4.1.1" dependencies: argparse: "npm:^2.0.1" bin: js-yaml: bin/js-yaml.js - checksum: 10c0/184a24b4eaacfce40ad9074c64fd42ac83cf74d8c8cd137718d456ced75051229e5061b8633c3366b8aada17945a7a356b337828c19da92b51ae62126575018f + checksum: 10c0/561c7d7088c40a9bb53cc75becbfb1df6ae49b34b5e6e5a81744b14ae8667ec564ad2527709d1a6e7d5e5fa6d483aa0f373a50ad98d42fde368ec4a190d4fae7 languageName: node linkType: hard From dd5c76e3df067ea17061cb9c19080209753b9679 Mon Sep 17 00:00:00 2001 From: Andrew Sy Kim Date: Tue, 18 Nov 2025 20:42:32 -0500 Subject: [PATCH 21/32] update minimum Ray version required for token authentication to 2.52.0 (#4201) * update minimum Ray version required for token authentication to 2.52.0 Signed-off-by: Andrew Sy Kim * update RayCluster auth e2e test to use Ray v2.52 Signed-off-by: Andrew Sy Kim --------- Signed-off-by: Andrew Sy Kim --- ray-operator/controllers/ray/utils/validation.go | 8 ++++---- ray-operator/controllers/ray/utils/validation_test.go | 9 ++------- ray-operator/test/e2e/raycluster_auth_test.go | 2 +- 3 files changed, 7 insertions(+), 12 deletions(-) diff --git a/ray-operator/controllers/ray/utils/validation.go b/ray-operator/controllers/ray/utils/validation.go index a3e6da66e80..652ff0e14b6 100644 --- a/ray-operator/controllers/ray/utils/validation.go +++ b/ray-operator/controllers/ray/utils/validation.go @@ -193,7 +193,7 @@ func ValidateRayClusterSpec(spec *rayv1.RayClusterSpec, annotations map[string]s if IsAuthEnabled(spec) { if spec.RayVersion == "" { - return fmt.Errorf("authOptions.mode is 'token' but RayVersion was not specified. Ray version 2.51.0 or later is required") + return fmt.Errorf("authOptions.mode is 'token' but RayVersion was not specified. Ray version 2.52.0 or later is required") } rayVersion, err := version.ParseGeneric(spec.RayVersion) @@ -201,10 +201,10 @@ func ValidateRayClusterSpec(spec *rayv1.RayClusterSpec, annotations map[string]s return fmt.Errorf("authOptions.mode is 'token' but RayVersion format is invalid: %s, %w", spec.RayVersion, err) } - // Require minimum Ray version 2.51.0 - minVersion := version.MustParseGeneric("2.51.0") + // Require minimum Ray version 2.52.0 + minVersion := version.MustParseGeneric("2.52.0") if rayVersion.LessThan(minVersion) { - return fmt.Errorf("authOptions.mode is 'token' but minimum Ray version is 2.51.0, got %s", spec.RayVersion) + return fmt.Errorf("authOptions.mode is 'token' but minimum Ray version is 2.52.0, got %s", spec.RayVersion) } } diff --git a/ray-operator/controllers/ray/utils/validation_test.go b/ray-operator/controllers/ray/utils/validation_test.go index 494fe39c29c..014d0917d68 100644 --- a/ray-operator/controllers/ray/utils/validation_test.go +++ b/ray-operator/controllers/ray/utils/validation_test.go @@ -858,11 +858,6 @@ func TestValidateRayClusterSpecRayVersionForAuth(t *testing.T) { errorMessage string expectError bool }{ - { - name: "Valid Ray version 2.51.0", - rayVersion: "2.51.0", - expectError: false, - }, { name: "Valid Ray version 2.52.0", rayVersion: "2.52.0", @@ -877,7 +872,7 @@ func TestValidateRayClusterSpecRayVersionForAuth(t *testing.T) { name: "Invalid Ray version 2.50.0", rayVersion: "2.50.0", expectError: true, - errorMessage: "authOptions.mode is 'token' but minimum Ray version is 2.51.0, got 2.50.0", + errorMessage: "authOptions.mode is 'token' but minimum Ray version is 2.52.0, got 2.50.0", }, { name: "Invalid Ray version format", @@ -889,7 +884,7 @@ func TestValidateRayClusterSpecRayVersionForAuth(t *testing.T) { name: "Empty Ray version", rayVersion: "", expectError: true, - errorMessage: "authOptions.mode is 'token' but RayVersion was not specified. Ray version 2.51.0 or later is required", + errorMessage: "authOptions.mode is 'token' but RayVersion was not specified. Ray version 2.52.0 or later is required", }, } diff --git a/ray-operator/test/e2e/raycluster_auth_test.go b/ray-operator/test/e2e/raycluster_auth_test.go index 67f2a7e4113..de37e401209 100644 --- a/ray-operator/test/e2e/raycluster_auth_test.go +++ b/ray-operator/test/e2e/raycluster_auth_test.go @@ -28,7 +28,7 @@ func TestRayClusterAuthOptions(t *testing.T) { t.Parallel() rayClusterAC := rayv1ac.RayCluster("raycluster-auth-token", namespace.Name). - WithSpec(NewRayClusterSpecWithAuth(rayv1.AuthModeToken).WithRayVersion("2.51")) + WithSpec(NewRayClusterSpecWithAuth(rayv1.AuthModeToken).WithRayVersion("2.52")) rayCluster, err := test.Client().Ray().RayV1().RayClusters(namespace.Name).Apply(test.Ctx(), rayClusterAC, TestApplyOptions) g.Expect(err).NotTo(HaveOccurred()) From 6077c276af4ba34ec9793c2775fa25f45dd89b54 Mon Sep 17 00:00:00 2001 From: Andrew Sy Kim Date: Tue, 18 Nov 2025 21:00:36 -0500 Subject: [PATCH 22/32] add samples for RayCluster token auth (#4200) Signed-off-by: Andrew Sy Kim --- .../samples/ray-cluster.auth-manual.yaml | 61 +++++++++ .../config/samples/ray-cluster.auth.yaml | 122 +++--------------- 2 files changed, 80 insertions(+), 103 deletions(-) create mode 100644 ray-operator/config/samples/ray-cluster.auth-manual.yaml diff --git a/ray-operator/config/samples/ray-cluster.auth-manual.yaml b/ray-operator/config/samples/ray-cluster.auth-manual.yaml new file mode 100644 index 00000000000..a213919ced0 --- /dev/null +++ b/ray-operator/config/samples/ray-cluster.auth-manual.yaml @@ -0,0 +1,61 @@ +apiVersion: ray.io/v1 +kind: RayCluster +metadata: + name: ray-cluster-with-auth +spec: + headGroupSpec: + rayStartParams: {} + template: + spec: + containers: + - name: ray-head + image: rayproject/ray:nightly-py311-cpu + env: + - name: RAY_AUTH_MODE + value: token + # You can create the secret manually with the following command: + # kubectl create secret generic ray-cluster-with-manual-auth --from-literal=auth_token='raycluster_secret' -n default + # And then use the following valueFrom to reference the secret: + - name: RAY_AUTH_TOKEN + valueFrom: + secretKeyRef: + key: auth_token + name: ray-cluster-with-auth # change this to match your Secret name + resources: + limits: + memory: 8G + requests: + cpu: 4 + memory: 8G + ports: + - containerPort: 6379 + name: gcs-server + - containerPort: 8265 + name: dashboard + - containerPort: 10001 + name: client + workerGroupSpecs: + - replicas: 1 + minReplicas: 1 + maxReplicas: 5 + groupName: workergroup + rayStartParams: {} + template: + spec: + containers: + - name: ray-worker + image: rayproject/ray:nightly-py311-cpu + env: + - name: RAY_AUTH_MODE + value: token + - name: RAY_AUTH_TOKEN + valueFrom: + secretKeyRef: + key: auth_token + name: ray-cluster-with-auth # change this to match your Secret name + resources: + limits: + memory: 8G + requests: + cpu: 4 + memory: 8G diff --git a/ray-operator/config/samples/ray-cluster.auth.yaml b/ray-operator/config/samples/ray-cluster.auth.yaml index bd6f97fb2f1..af6638263c6 100644 --- a/ray-operator/config/samples/ray-cluster.auth.yaml +++ b/ray-operator/config/samples/ray-cluster.auth.yaml @@ -1,130 +1,46 @@ -apiVersion: v1 -kind: ConfigMap -metadata: - name: kube-rbac-proxy -data: - config-file.yaml: | - authorization: - resourceAttributes: - namespace: default - apiVersion: v1 - apiGroup: ray.io - resource: rayclusters - name: ray-cluster-with-auth ---- -apiVersion: v1 -kind: ServiceAccount -metadata: - name: kube-rbac-proxy ---- -apiVersion: rbac.authorization.k8s.io/v1 -kind: ClusterRoleBinding -metadata: - name: kube-rbac-proxy -roleRef: - apiGroup: rbac.authorization.k8s.io - kind: ClusterRole - name: kube-rbac-proxy -subjects: -- kind: ServiceAccount - name: kube-rbac-proxy - namespace: default ---- -apiVersion: rbac.authorization.k8s.io/v1 -kind: ClusterRole -metadata: - name: kube-rbac-proxy -rules: -- apiGroups: ["authentication.k8s.io"] - resources: - - tokenreviews - verbs: ["create"] -- apiGroups: ["authorization.k8s.io"] - resources: - - subjectaccessreviews - verbs: ["create"] ---- apiVersion: ray.io/v1 kind: RayCluster metadata: name: ray-cluster-with-auth spec: + enableInTreeAutoscaling: true + authOptions: + mode: token + rayVersion: '2.52.0' headGroupSpec: - rayStartParams: - dashboard-host: '127.0.0.1' - dashboard-port: '8443' + rayStartParams: {} template: - metadata: spec: - serviceAccountName: kube-rbac-proxy containers: - name: ray-head - image: rayproject/ray:2.46.0 + image: rayproject/ray:nightly-py311-cpu + resources: + limits: + memory: 8G + requests: + cpu: 4 + memory: 8G ports: - containerPort: 6379 - name: gcs + name: gcs-server - containerPort: 8265 name: dashboard - containerPort: 10001 name: client - resources: - limits: - cpu: "2" - memory: "4Gi" - requests: - cpu: "2" - memory: "4Gi" - readinessProbe: - exec: - command: - - bash - - -c - - wget -T 2 -q -O- http://localhost:52365/api/local_raylet_healthz | grep success && wget -T 10 -q -O- http://localhost:8443/api/gcs_healthz | grep success - failureThreshold: 10 - initialDelaySeconds: 10 - periodSeconds: 5 - successThreshold: 1 - timeoutSeconds: 2 - livenessProbe: - exec: - command: - - bash - - -c - - wget -T 2 -q -O- http://localhost:52365/api/local_raylet_healthz | grep success && wget -T 10 -q -O- http://localhost:8443/api/gcs_healthz | grep success - failureThreshold: 120 - initialDelaySeconds: 30 - periodSeconds: 5 - successThreshold: 1 - timeoutSeconds: 2 - - name: kube-rbac-proxy - image: quay.io/brancz/kube-rbac-proxy:v0.18.1 - args: - - "--insecure-listen-address=0.0.0.0:8265" - - "--upstream=http://127.0.0.1:8443/" - - "--config-file=/etc/kube-rbac-proxy/config-file.yaml" - - "--logtostderr=true" - volumeMounts: - - name: config - mountPath: /etc/kube-rbac-proxy - volumes: - - name: config - configMap: - name: kube-rbac-proxy workerGroupSpecs: - - replicas: 2 + - replicas: 1 minReplicas: 1 maxReplicas: 5 - groupName: worker-group + groupName: workergroup rayStartParams: {} template: spec: containers: - name: ray-worker - image: rayproject/ray:2.46.0 + image: rayproject/ray:nightly-py311-cpu resources: limits: - cpu: 1 - memory: "4Gi" + memory: 8G requests: - cpu: 1 - memory: "4Gi" + cpu: 4 + memory: 8G From 82033117d67b1b29df2104784fa7f514a975cb24 Mon Sep 17 00:00:00 2001 From: "Han-Ju Chen (Future-Outlier)" Date: Wed, 19 Nov 2025 23:17:00 +0800 Subject: [PATCH 23/32] update (#4208) Signed-off-by: Future-Outlier --- ray-operator/test/e2erayjob/rayjob_sidecar_mode_test.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/ray-operator/test/e2erayjob/rayjob_sidecar_mode_test.go b/ray-operator/test/e2erayjob/rayjob_sidecar_mode_test.go index da9ef0d2ca8..ec6fcb9bb2d 100644 --- a/ray-operator/test/e2erayjob/rayjob_sidecar_mode_test.go +++ b/ray-operator/test/e2erayjob/rayjob_sidecar_mode_test.go @@ -188,10 +188,10 @@ env_vars: g.Eventually(RayJob(test, rayJob.Namespace, rayJob.Name), TestTimeoutMedium). Should(WithTransform(RayJobDeploymentStatus, Equal(rayv1.JobDeploymentStatusFailed))) g.Eventually(RayJob(test, rayJob.Namespace, rayJob.Name), TestTimeoutMedium). - Should(WithTransform(RayJobReason, Equal(rayv1.AppFailed))) - g.Eventually(RayJob(test, rayJob.Namespace, rayJob.Name), TestTimeoutMedium). - Should(WithTransform(func(job *rayv1.RayJob) string { return job.Status.Message }, - Equal("Ray head pod not found."))) + Should(WithTransform(RayJobReason, Or( + Equal(rayv1.AppFailed), + Equal(rayv1.SubmissionFailed), + ))) // Cleanup err = test.Client().Ray().RayV1().RayJobs(namespace.Name).Delete(test.Ctx(), rayJob.Name, metav1.DeleteOptions{}) From a739b78be26b11f512c6e6309a104c27b2ec8d6a Mon Sep 17 00:00:00 2001 From: "Han-Ju Chen (Future-Outlier)" Date: Thu, 20 Nov 2025 07:16:20 +0800 Subject: [PATCH 24/32] [RayJob] Add token authentication support for All mode (#4210) * dashboard client authentication support Signed-off-by: Future-Outlier * support rayjob Signed-off-by: Future-Outlier * update to fix api serverr err Signed-off-by: Future-Outlier * update Signed-off-by: Future-Outlier * updarte Signed-off-by: Future-Outlier * Rayjob sidecar mode auth token mode support Signed-off-by: Future-Outlier * RayJob support k8s job mode Signed-off-by: Future-Outlier * update Signed-off-by: Future-Outlier * update Signed-off-by: Future-Outlier * update Signed-off-by: Future-Outlier * Address Andrew's advice Signed-off-by: Future-Outlier * add todo x-ray-authorization comments Signed-off-by: Future-Outlier --------- Signed-off-by: Future-Outlier --- ray-operator/controllers/ray/common/pod.go | 11 +++--- .../controllers/ray/rayjob_controller.go | 10 ++++-- .../dashboardclient/dashboard_httpclient.go | 29 +++++++++++++-- .../ray/utils/fake_serve_httpclient.go | 2 +- ray-operator/controllers/ray/utils/util.go | 35 +++++++++++++++++-- ray-operator/rayjob-submitter/cmd/main.go | 2 +- 6 files changed, 74 insertions(+), 15 deletions(-) diff --git a/ray-operator/controllers/ray/common/pod.go b/ray-operator/controllers/ray/common/pod.go index a42f9184912..be4a64cdfaf 100644 --- a/ray-operator/controllers/ray/common/pod.go +++ b/ray-operator/controllers/ray/common/pod.go @@ -202,7 +202,7 @@ func DefaultHeadPodTemplate(ctx context.Context, instance rayv1.RayCluster, head // Configure RAY_AUTH_TOKEN and RAY_AUTH_MODE if auth is enabled. if utils.IsAuthEnabled(&instance.Spec) { - setContainerTokenAuthEnvVars(instance.Name, &autoscalerContainer) + SetContainerTokenAuthEnvVars(instance.Name, &autoscalerContainer) } // Merge the user overrides from autoscalerOptions into the autoscaler container config. @@ -248,7 +248,8 @@ func setAutoscalerV2EnvVars(podTemplate *corev1.PodTemplateSpec) { // configureTokenAuth sets environment variables required for Ray token authentication func configureTokenAuth(clusterName string, podTemplate *corev1.PodTemplateSpec) { - setContainerTokenAuthEnvVars(clusterName, &podTemplate.Spec.Containers[utils.RayContainerIndex]) + SetContainerTokenAuthEnvVars(clusterName, &podTemplate.Spec.Containers[utils.RayContainerIndex]) + // For RayJob Sidecar mode, we need to set the auth token for the submitter container. // Configure auth token for wait-gcs-ready init container if it exists for i, initContainer := range podTemplate.Spec.InitContainers { @@ -256,12 +257,12 @@ func configureTokenAuth(clusterName string, podTemplate *corev1.PodTemplateSpec) continue } - setContainerTokenAuthEnvVars(clusterName, &podTemplate.Spec.InitContainers[i]) + SetContainerTokenAuthEnvVars(clusterName, &podTemplate.Spec.InitContainers[i]) } } -// setContainerTokenAuthEnvVars sets Ray authentication env vars for a container. -func setContainerTokenAuthEnvVars(clusterName string, container *corev1.Container) { +// SetContainerTokenAuthEnvVars sets Ray authentication env vars for a container. +func SetContainerTokenAuthEnvVars(clusterName string, container *corev1.Container) { container.Env = append(container.Env, corev1.EnvVar{ Name: utils.RAY_AUTH_MODE_ENV_VAR, Value: string(rayv1.AuthModeToken), diff --git a/ray-operator/controllers/ray/rayjob_controller.go b/ray-operator/controllers/ray/rayjob_controller.go index 1f522dd8ca2..e09810022f0 100644 --- a/ray-operator/controllers/ray/rayjob_controller.go +++ b/ray-operator/controllers/ray/rayjob_controller.go @@ -559,7 +559,7 @@ func getSubmitterTemplate(rayJobInstance *rayv1.RayJob, rayClusterInstance *rayv // Set the default value for the optional field SubmitterPodTemplate if not provided. submitterTemplate := common.GetSubmitterTemplate(&rayJobInstance.Spec, &rayClusterInstance.Spec) - if err := configureSubmitterContainer(&submitterTemplate.Spec.Containers[utils.RayContainerIndex], rayJobInstance, rayv1.K8sJobMode); err != nil { + if err := configureSubmitterContainer(&submitterTemplate.Spec.Containers[utils.RayContainerIndex], rayJobInstance, rayClusterInstance, rayv1.K8sJobMode); err != nil { return corev1.PodTemplateSpec{}, err } @@ -570,14 +570,15 @@ func getSubmitterTemplate(rayJobInstance *rayv1.RayJob, rayClusterInstance *rayv func getSubmitterContainer(rayJobInstance *rayv1.RayJob, rayClusterInstance *rayv1.RayCluster) (corev1.Container, error) { var submitterContainer corev1.Container = common.GetDefaultSubmitterContainer(&rayClusterInstance.Spec) - if err := configureSubmitterContainer(&submitterContainer, rayJobInstance, rayv1.SidecarMode); err != nil { + if err := configureSubmitterContainer(&submitterContainer, rayJobInstance, rayClusterInstance, rayv1.SidecarMode); err != nil { return corev1.Container{}, err } return submitterContainer, nil } -func configureSubmitterContainer(container *corev1.Container, rayJobInstance *rayv1.RayJob, submissionMode rayv1.JobSubmissionMode) error { +// pass the RayCluster instance for cluster selector case +func configureSubmitterContainer(container *corev1.Container, rayJobInstance *rayv1.RayJob, rayClusterInstance *rayv1.RayCluster, submissionMode rayv1.JobSubmissionMode) error { // If the command in the submitter container manifest isn't set, use the default command. jobCmd, err := common.BuildJobSubmitCommand(rayJobInstance, submissionMode) if err != nil { @@ -600,6 +601,9 @@ func configureSubmitterContainer(container *corev1.Container, rayJobInstance *ra // ray job submit --address=http://$RAY_DASHBOARD_ADDRESS --submission-id=$RAY_JOB_SUBMISSION_ID ... container.Env = append(container.Env, corev1.EnvVar{Name: utils.RAY_DASHBOARD_ADDRESS, Value: rayJobInstance.Status.DashboardURL}) container.Env = append(container.Env, corev1.EnvVar{Name: utils.RAY_JOB_SUBMISSION_ID, Value: rayJobInstance.Status.JobId}) + if rayClusterInstance != nil && utils.IsAuthEnabled(&rayClusterInstance.Spec) { + common.SetContainerTokenAuthEnvVars(rayClusterInstance.Name, container) + } return nil } diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_httpclient.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_httpclient.go index d360ebc0af9..2198cf0aef8 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_httpclient.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_httpclient.go @@ -27,7 +27,7 @@ var ( ) type RayDashboardClientInterface interface { - InitClient(client *http.Client, dashboardURL string) + 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) @@ -44,11 +44,20 @@ type RayDashboardClientInterface interface { type RayDashboardClient struct { client *http.Client dashboardURL string + authToken string } -func (r *RayDashboardClient) InitClient(client *http.Client, dashboardURL string) { +func (r *RayDashboardClient) InitClient(client *http.Client, dashboardURL string, authToken string) { r.client = client r.dashboardURL = dashboardURL + r.authToken = authToken +} + +// TODO: change authorization to x-ray-authorization after this PR is merged: https://github.com/ray-project/ray/pull/58819 +func (r *RayDashboardClient) setAuthHeader(req *http.Request) { + if r.authToken != "" { + req.Header.Set("authorization", fmt.Sprintf("Bearer %s", r.authToken)) + } } // UpdateDeployments update the deployments in the Ray cluster. @@ -60,6 +69,7 @@ func (r *RayDashboardClient) UpdateDeployments(ctx context.Context, configJson [ } req.Header.Set("Content-Type", "application/json") + r.setAuthHeader(req) resp, err := r.client.Do(req) if err != nil { @@ -102,6 +112,8 @@ func (r *RayDashboardClient) GetServeDetails(ctx context.Context) (*utiltypes.Se return nil, err } + r.setAuthHeader(req) + resp, err := r.client.Do(req) if err != nil { return nil, err @@ -147,6 +159,8 @@ func (r *RayDashboardClient) GetJobInfo(ctx context.Context, jobId string) (*uti return nil, err } + r.setAuthHeader(req) + resp, err := r.client.Do(req) if err != nil { return nil, err @@ -177,6 +191,8 @@ func (r *RayDashboardClient) ListJobs(ctx context.Context) (*[]utiltypes.RayJobI return nil, err } + r.setAuthHeader(req) + resp, err := r.client.Do(req) if err != nil { return nil, err @@ -221,6 +237,8 @@ func (r *RayDashboardClient) SubmitJobReq(ctx context.Context, request *utiltype } req.Header.Set("Content-Type", "application/json") + r.setAuthHeader(req) + resp, err := r.client.Do(req) if err != nil { return @@ -255,6 +273,9 @@ func (r *RayDashboardClient) GetJobLog(ctx context.Context, jobName string) (*st if err != nil { return nil, err } + + r.setAuthHeader(req) + resp, err := r.client.Do(req) if err != nil { return nil, err @@ -288,6 +309,8 @@ func (r *RayDashboardClient) StopJob(ctx context.Context, jobName string) (err e } req.Header.Set("Content-Type", "application/json") + r.setAuthHeader(req) + resp, err := r.client.Do(req) if err != nil { return err @@ -324,6 +347,8 @@ func (r *RayDashboardClient) DeleteJob(ctx context.Context, jobName string) erro } req.Header.Set("Content-Type", "application/json") + r.setAuthHeader(req) + resp, err := r.client.Do(req) if err != nil { return err diff --git a/ray-operator/controllers/ray/utils/fake_serve_httpclient.go b/ray-operator/controllers/ray/utils/fake_serve_httpclient.go index 1bf0588c403..35d15bdf07c 100644 --- a/ray-operator/controllers/ray/utils/fake_serve_httpclient.go +++ b/ray-operator/controllers/ray/utils/fake_serve_httpclient.go @@ -20,7 +20,7 @@ type FakeRayDashboardClient struct { var _ dashboardclient.RayDashboardClientInterface = (*FakeRayDashboardClient)(nil) -func (r *FakeRayDashboardClient) InitClient(_ *http.Client, _ string) { +func (r *FakeRayDashboardClient) InitClient(_ *http.Client, _ string, _ string) { } func (r *FakeRayDashboardClient) UpdateDeployments(_ context.Context, configJson []byte) error { diff --git a/ray-operator/controllers/ray/utils/util.go b/ray-operator/controllers/ray/utils/util.go index cc4ed92fec7..00f0242cfcb 100644 --- a/ray-operator/controllers/ray/utils/util.go +++ b/ray-operator/controllers/ray/utils/util.go @@ -19,6 +19,7 @@ import ( meta "k8s.io/apimachinery/pkg/api/meta" "k8s.io/apimachinery/pkg/api/resource" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/types" "k8s.io/apimachinery/pkg/util/json" "k8s.io/apimachinery/pkg/util/rand" "k8s.io/client-go/discovery" @@ -897,6 +898,28 @@ func FetchHeadServiceURL(ctx context.Context, cli client.Client, rayCluster *ray func GetRayDashboardClientFunc(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 + + if rayCluster != nil && rayCluster.Spec.AuthOptions != nil && rayCluster.Spec.AuthOptions.Mode == rayv1.AuthModeToken { + secretName := CheckName(rayCluster.Name) + secret := &corev1.Secret{} + secretKey := types.NamespacedName{ + Name: secretName, + Namespace: rayCluster.Namespace, + } + + if err := mgr.GetClient().Get(context.Background(), secretKey, secret); err != nil { + return nil, fmt.Errorf("failed to get auth secret %s/%s: %w", rayCluster.Namespace, secretName, err) + } + + tokenBytes, exists := secret.Data[RAY_AUTH_TOKEN_SECRET_KEY] + if !exists { + return nil, fmt.Errorf("auth token key '%q' not found in secret %s/%s", RAY_AUTH_TOKEN_SECRET_KEY, rayCluster.Namespace, secretName) + } + + authToken = string(tokenBytes) + } + if useKubernetesProxy { var err error headSvcName := rayCluster.Status.Head.ServiceName @@ -913,13 +936,19 @@ func GetRayDashboardClientFunc(mgr manager.Manager, useKubernetesProxy bool) fun // configured to communicate with the Kubernetes API server. mgr.GetHTTPClient(), 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{ - Timeout: 2 * time.Second, - }, "http://"+url) + dashboardClient.InitClient( + &http.Client{ + Timeout: 2 * time.Second, + }, + "http://"+url, + authToken, + ) + return dashboardClient, nil } } diff --git a/ray-operator/rayjob-submitter/cmd/main.go b/ray-operator/rayjob-submitter/cmd/main.go index dd4f68eb420..cbc52cc3968 100644 --- a/ray-operator/rayjob-submitter/cmd/main.go +++ b/ray-operator/rayjob-submitter/cmd/main.go @@ -64,7 +64,7 @@ func main() { } rayDashboardClient := &dashboardclient.RayDashboardClient{} address = rayjobsubmitter.JobSubmissionURL(address) - rayDashboardClient.InitClient(&http.Client{Timeout: time.Second * 10}, address) + rayDashboardClient.InitClient(&http.Client{Timeout: time.Second * 10}, address, "") submissionId, err := rayDashboardClient.SubmitJobReq(context.Background(), &req) if err != nil { if strings.Contains(err.Error(), "Please use a different submission_id") { From 1c846b3bce2628483715996e3d0b7bc94831666c Mon Sep 17 00:00:00 2001 From: "Han-Ju Chen (Future-Outlier)" Date: Thu, 20 Nov 2025 07:26:39 +0800 Subject: [PATCH 25/32] [RayCluster] Enable Secret informer watch/list and remove unused RBAC verbs (#4202) * Add authentication secret reconciliation support Signed-off-by: Future-Outlier * update Signed-off-by: Future-Outlier * update Signed-off-by: Future-Outlier * fix flaky test Signed-off-by: Future-Outlier * remove test fix Signed-off-by: Rueian --------- Signed-off-by: Future-Outlier Signed-off-by: Rueian Co-authored-by: Rueian --- helm-chart/kuberay-operator/templates/_helpers.tpl | 2 -- ray-operator/config/rbac/role.yaml | 2 -- ray-operator/controllers/ray/raycluster_controller.go | 5 +++-- 3 files changed, 3 insertions(+), 6 deletions(-) diff --git a/helm-chart/kuberay-operator/templates/_helpers.tpl b/helm-chart/kuberay-operator/templates/_helpers.tpl index a495b816bd9..e9053c339dc 100644 --- a/helm-chart/kuberay-operator/templates/_helpers.tpl +++ b/helm-chart/kuberay-operator/templates/_helpers.tpl @@ -177,8 +177,6 @@ rules: - create - get - list - - patch - - update - watch - apiGroups: - "" diff --git a/ray-operator/config/rbac/role.yaml b/ray-operator/config/rbac/role.yaml index edd446f97d2..374c0adae2d 100644 --- a/ray-operator/config/rbac/role.yaml +++ b/ray-operator/config/rbac/role.yaml @@ -62,8 +62,6 @@ rules: - create - get - list - - patch - - update - watch - apiGroups: - "" diff --git a/ray-operator/controllers/ray/raycluster_controller.go b/ray-operator/controllers/ray/raycluster_controller.go index 7d4d8fab8f6..76b21e47527 100644 --- a/ray-operator/controllers/ray/raycluster_controller.go +++ b/ray-operator/controllers/ray/raycluster_controller.go @@ -100,7 +100,7 @@ type RayClusterReconcilerOptions struct { // +kubebuilder:rbac:groups=core,resources=pods,verbs=get;list;watch;create;update;patch;delete;deletecollection // +kubebuilder:rbac:groups=core,resources=pods/status,verbs=get;list;watch;create;update;patch;delete // +kubebuilder:rbac:groups=core,resources=pods/resize,verbs=patch -// +kubebuilder:rbac:groups=core,resources=secrets,verbs=get;list;watch;create;update;patch +// +kubebuilder:rbac:groups=core,resources=secrets,verbs=get;list;watch;create; // +kubebuilder:rbac:groups=core,resources=services,verbs=get;list;watch;create;update;patch;delete // +kubebuilder:rbac:groups=core,resources=services/status,verbs=get;update;patch // +kubebuilder:rbac:groups=coordination.k8s.io,resources=leases,verbs=get;list;create;update @@ -1446,7 +1446,8 @@ func (r *RayClusterReconciler) SetupWithManager(mgr ctrl.Manager, reconcileConcu predicate.AnnotationChangedPredicate{}, ))). Owns(&corev1.Pod{}). - Owns(&corev1.Service{}) + Owns(&corev1.Service{}). + Owns(&corev1.Secret{}) if r.options.BatchSchedulerManager != nil { r.options.BatchSchedulerManager.ConfigureReconciler(b) } From c7669d0968caf61e01bf787815e7dc8f25707f1f Mon Sep 17 00:00:00 2001 From: JustinYeh Date: Thu, 20 Nov 2025 12:50:54 +0800 Subject: [PATCH 26/32] [APIServer][Docs] Add user guide for retry behavior & configuration (#4144) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * [Docs] Add the draft description about feature intro, configurations, and usecases Signed-off-by: justinyeh1995 * [Fix] Update the retry walk-through Signed-off-by: justinyeh1995 * [Doc] rewrite the first 2 sections Signed-off-by: justinyeh1995 * [Doc] Revise documentation wording and add Observing Retry Behavior section Signed-off-by: justinyeh1995 * [Fix] fix linting issue by running pre-commit run berfore commiting Signed-off-by: justinyeh1995 * [Fix] fix linting errors in the Markdown linting Signed-off-by: justinyeh1995 * [Fix] Clean up the math equation Signed-off-by: justinyeh1995 * Update the math formula of Backoff calculation. Co-authored-by: Nary Yeh <60069744+machichima@users.noreply.github.com> Signed-off-by: JustinYeh * [Fix] Explicitly mentioned exponential backoff and removed the customization parts Signed-off-by: justinyeh1995 * [Docs] Clarify naming by replacing “APIServer” with “KubeRay APIServer” Co-authored-by: Cheng-Yeh Chung Signed-off-by: JustinYeh * [Docs] Rename retry-configuration.md to retry-behavior.md for accuracy Signed-off-by: justinyeh1995 * Update Title to KubeRay APIServer Retry Behavior Co-authored-by: Cheng-Yeh Chung Signed-off-by: JustinYeh * [Docs] Add a note about the limitation of retry configuration Signed-off-by: justinyeh1995 --------- Signed-off-by: justinyeh1995 Signed-off-by: JustinYeh Co-authored-by: Nary Yeh <60069744+machichima@users.noreply.github.com> Co-authored-by: Cheng-Yeh Chung --- apiserversdk/docs/retry-behavior.md | 32 +++++++++++++++++++++++++++++ 1 file changed, 32 insertions(+) create mode 100644 apiserversdk/docs/retry-behavior.md diff --git a/apiserversdk/docs/retry-behavior.md b/apiserversdk/docs/retry-behavior.md new file mode 100644 index 00000000000..0faa03bb9cc --- /dev/null +++ b/apiserversdk/docs/retry-behavior.md @@ -0,0 +1,32 @@ +# KubeRay APIServer Retry Behavior + +The KubeRay APIServer automatically retries failed requests to the Kubernetes API when transient errors occur. +This built-in mechanism uses exponential backoff to improve reliability without requiring manual intervention. +As of `v1.5.0`, the retry configuration is hard-coded and cannot be customized. +This guide describes the default retry behavior. + +## Default Retry Behavior + +The KubeRay APIServer automatically retries with exponential backoff for these HTTP status codes: + +- 408 (Request Timeout) +- 429 (Too Many Requests) +- 500 (Internal Server Error) +- 502 (Bad Gateway) +- 503 (Service Unavailable) +- 504 (Gateway Timeout) + +Note that non-retryable errors (4xx except 408/429) fail immediately without retries. + +The following default configuration explains how retry works: + +- **MaxRetry**: 3 retries (4 total attempts including the initial one) +- **InitBackoff**: 500ms (initial wait time) +- **BackoffFactor**: 2.0 (exponential multiplier) +- **MaxBackoff**: 10s (maximum wait time between retries) +- **OverallTimeout**: 30s (total timeout for all attempts) + +which means $$\text{Backoff}_i = \min(\text{InitBackoff} \times \text{BackoffFactor}^i, \text{MaxBackoff})$$ + +where $i$ is the attempt number (starting from 0). +The retries will stop if the total time exceeds the `OverallTimeout`. From 594ff850eda79b6beaa13c77c5884530e30fb255 Mon Sep 17 00:00:00 2001 From: "Han-Ju Chen (Future-Outlier)" Date: Fri, 21 Nov 2025 00:02:07 +0800 Subject: [PATCH 27/32] Support X-Ray-Authorization fallback header for accepting auth token via proxy (#4213) * Support X-Ray-Authorization fallback header for accepting auth token in dashboard Signed-off-by: Future-Outlier * remove todo comment Signed-off-by: Future-Outlier --------- Signed-off-by: Future-Outlier --- .../ray/utils/dashboardclient/dashboard_httpclient.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_httpclient.go b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_httpclient.go index 2198cf0aef8..46b3f689a24 100644 --- a/ray-operator/controllers/ray/utils/dashboardclient/dashboard_httpclient.go +++ b/ray-operator/controllers/ray/utils/dashboardclient/dashboard_httpclient.go @@ -53,10 +53,9 @@ func (r *RayDashboardClient) InitClient(client *http.Client, dashboardURL string r.authToken = authToken } -// TODO: change authorization to x-ray-authorization after this PR is merged: https://github.com/ray-project/ray/pull/58819 func (r *RayDashboardClient) setAuthHeader(req *http.Request) { if r.authToken != "" { - req.Header.Set("authorization", fmt.Sprintf("Bearer %s", r.authToken)) + req.Header.Set("x-ray-authorization", fmt.Sprintf("Bearer %s", r.authToken)) } } From 13467c893df2cc602b83436cbdc43bca996a7a77 Mon Sep 17 00:00:00 2001 From: fscnick <6858627+fscnick@users.noreply.github.com> Date: Fri, 21 Nov 2025 00:02:30 +0800 Subject: [PATCH 28/32] [RayCluster] make auth token secret name consistency (#4216) Signed-off-by: fscnick --- ray-operator/config/samples/ray-cluster.auth-manual.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ray-operator/config/samples/ray-cluster.auth-manual.yaml b/ray-operator/config/samples/ray-cluster.auth-manual.yaml index a213919ced0..33a0c596cd6 100644 --- a/ray-operator/config/samples/ray-cluster.auth-manual.yaml +++ b/ray-operator/config/samples/ray-cluster.auth-manual.yaml @@ -14,7 +14,7 @@ spec: - name: RAY_AUTH_MODE value: token # You can create the secret manually with the following command: - # kubectl create secret generic ray-cluster-with-manual-auth --from-literal=auth_token='raycluster_secret' -n default + # kubectl create secret generic ray-cluster-with-auth --from-literal=auth_token='raycluster_secret' -n default # And then use the following valueFrom to reference the secret: - name: RAY_AUTH_TOKEN valueFrom: From 3bd363d8cc5be40390592cfce534fe0b90de8aeb Mon Sep 17 00:00:00 2001 From: Spencer Peterson Date: Thu, 20 Nov 2025 11:02:56 -0800 Subject: [PATCH 29/32] [RayCluster] Status includes head containter status message (#4196) * [RayCluster] Status includes head containter status message Signed-off-by: Spencer Peterson * lint Signed-off-by: Spencer Peterson * [RayCluster] Containers not ready status reflects structured reason Signed-off-by: Spencer Peterson * nit Signed-off-by: Spencer Peterson --------- Signed-off-by: Spencer Peterson --- ray-operator/controllers/ray/utils/util.go | 25 +++++ .../controllers/ray/utils/util_test.go | 93 ++++++++++++++++++- 2 files changed, 113 insertions(+), 5 deletions(-) diff --git a/ray-operator/controllers/ray/utils/util.go b/ray-operator/controllers/ray/utils/util.go index 00f0242cfcb..8c806d6d413 100644 --- a/ray-operator/controllers/ray/utils/util.go +++ b/ray-operator/controllers/ray/utils/util.go @@ -38,6 +38,7 @@ const ( ServeName = "serve" ClusterDomainEnvKey = "CLUSTER_DOMAIN" DefaultDomainName = "cluster.local" + ContainersNotReady = "ContainersNotReady" ) // TODO (kevin85421): Define CRDType here rather than constant.go to avoid circular dependency. @@ -103,12 +104,36 @@ func FindHeadPodReadyCondition(headPod *corev1.Pod) metav1.Condition { headPodReadyCondition.Reason = reason } + // If reason is ContainersNotReady, then replace it with an available + // container status that may illuminate why the container is not ready. + if reason == ContainersNotReady { + reason, message, ok := firstNotReadyContainerStatus(headPod) + if ok { + if headPodReadyCondition.Message != "" { + headPodReadyCondition.Message += "; " + } + headPodReadyCondition.Message += message + headPodReadyCondition.Reason = reason + } + } + // Since we're only interested in the PodReady condition, break after processing it break } return headPodReadyCondition } +func firstNotReadyContainerStatus(pod *corev1.Pod) (reason string, message string, ok bool) { + for _, status := range pod.Status.ContainerStatuses { + if status.State.Waiting != nil { + return status.State.Waiting.Reason, fmt.Sprintf("%s: %s", status.Name, status.State.Waiting.Message), true + } else if status.State.Terminated != nil { + return status.State.Terminated.Reason, fmt.Sprintf("%s: %s", status.Name, status.State.Terminated.Message), true + } + } + return "", "", false +} + // FindRayClusterSuspendStatus returns the current suspend status from two conditions: // 1. rayv1.RayClusterSuspending // 2. rayv1.RayClusterSuspended diff --git a/ray-operator/controllers/ray/utils/util_test.go b/ray-operator/controllers/ray/utils/util_test.go index 9a9371acd05..47afa55c85a 100644 --- a/ray-operator/controllers/ray/utils/util_test.go +++ b/ray-operator/controllers/ray/utils/util_test.go @@ -328,7 +328,7 @@ func createSomePodWithCondition(typ corev1.PodConditionType, status corev1.Condi } } -func createRayHeadPodWithPhaseAndCondition(phase corev1.PodPhase, typ corev1.PodConditionType, status corev1.ConditionStatus) (pod *corev1.Pod) { +func createRayHeadPodWithPhaseAndCondition(phase corev1.PodPhase, status corev1.ConditionStatus) (pod *corev1.Pod) { return &corev1.Pod{ TypeMeta: metav1.TypeMeta{ APIVersion: "v1", @@ -345,8 +345,9 @@ func createRayHeadPodWithPhaseAndCondition(phase corev1.PodPhase, typ corev1.Pod Phase: phase, Conditions: []corev1.PodCondition{ { - Type: typ, + Type: corev1.PodReady, Status: status, + Reason: ContainersNotReady, }, }, }, @@ -945,7 +946,7 @@ func TestFindHeadPodReadyCondition(t *testing.T) { }{ { name: "condition true if Ray head pod is running and ready", - pod: createRayHeadPodWithPhaseAndCondition(corev1.PodRunning, corev1.PodReady, corev1.ConditionTrue), + pod: createRayHeadPodWithPhaseAndCondition(corev1.PodRunning, corev1.ConditionTrue), expected: metav1.Condition{ Type: string(rayv1.HeadPodReady), Status: metav1.ConditionTrue, @@ -953,7 +954,7 @@ func TestFindHeadPodReadyCondition(t *testing.T) { }, { name: "condition false if Ray head pod is not running", - pod: createRayHeadPodWithPhaseAndCondition(corev1.PodPending, corev1.PodReady, corev1.ConditionFalse), + pod: createRayHeadPodWithPhaseAndCondition(corev1.PodPending, corev1.ConditionFalse), expected: metav1.Condition{ Type: string(rayv1.HeadPodReady), Status: metav1.ConditionFalse, @@ -961,7 +962,7 @@ func TestFindHeadPodReadyCondition(t *testing.T) { }, { name: "condition false if Ray head pod is not ready", - pod: createRayHeadPodWithPhaseAndCondition(corev1.PodRunning, corev1.PodReady, corev1.ConditionFalse), + pod: createRayHeadPodWithPhaseAndCondition(corev1.PodRunning, corev1.ConditionFalse), expected: metav1.Condition{ Type: string(rayv1.HeadPodReady), Status: metav1.ConditionFalse, @@ -977,6 +978,88 @@ func TestFindHeadPodReadyCondition(t *testing.T) { } } +func TestFindHeadPodReadyMessage(t *testing.T) { + tests := []struct { + name string + message string + wantMessage string + wantReason string + status []corev1.ContainerStatus + }{{ + name: "no message no status want original reason", + wantReason: ContainersNotReady, + }, { + name: "no container status want original reason", + message: "TooEarlyInTheMorning", + wantMessage: "TooEarlyInTheMorning", + wantReason: ContainersNotReady, + }, { + name: "one reason one status", + message: "containers not ready", + status: []corev1.ContainerStatus{{ + Name: "ray", + State: corev1.ContainerState{ + Waiting: &corev1.ContainerStateWaiting{ + Reason: "ImagePullBackOff", + Message: `Back-off pulling image royproject/roy:latest: ErrImagePull: rpc error: code = NotFound`, + }, + }, + }}, + wantReason: "ImagePullBackOff", + wantMessage: `containers not ready; ray: Back-off pulling image royproject/roy:latest: ErrImagePull: rpc error: code = NotFound`, + }, { + name: "one reason two statuses only copy first", + message: "aesthetic problems", + status: []corev1.ContainerStatus{{ + Name: "indigo", + State: corev1.ContainerState{ + Waiting: &corev1.ContainerStateWaiting{ + Reason: "BadColor", + Message: "too blue", + }, + }, + }, { + Name: "circle", + State: corev1.ContainerState{ + Terminated: &corev1.ContainerStateTerminated{ + Reason: "BadGeometry", + Message: "too round", + }, + }, + }}, + wantReason: "BadColor", + wantMessage: "aesthetic problems; indigo: too blue", + }, { + name: "no reason one status", + status: []corev1.ContainerStatus{{ + Name: "my-image", + State: corev1.ContainerState{ + Terminated: &corev1.ContainerStateTerminated{ + Reason: "Crashed", + Message: "bash not found", + }, + }, + }}, + wantReason: "Crashed", + wantMessage: "my-image: bash not found", + }} + + for _, tc := range tests { + t.Run(tc.name, func(t *testing.T) { + pod := createRayHeadPodWithPhaseAndCondition(corev1.PodPending, corev1.ConditionFalse) + pod.Status.Conditions[0].Message = tc.message + pod.Status.ContainerStatuses = tc.status + cond := FindHeadPodReadyCondition(pod) + if cond.Message != tc.wantMessage { + t.Errorf("FindHeadPodReadyCondition(...) returned condition with message %q, but wanted %q", cond.Message, tc.wantMessage) + } + if cond.Reason != tc.wantReason { + t.Errorf("FindHeadPodReadyCondition(...) returned condition with reason %q, but wanted %q", cond.Reason, tc.wantReason) + } + }) + } +} + func TestErrRayClusterReplicaFailureReason(t *testing.T) { assert.Equal(t, "FailedDeleteAllPods", RayClusterReplicaFailureReason(ErrFailedDeleteAllPods)) assert.Equal(t, "FailedDeleteHeadPod", RayClusterReplicaFailureReason(ErrFailedDeleteHeadPod)) From 4df3f2edfb6b02303f7652170193ea304b0cb572 Mon Sep 17 00:00:00 2001 From: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com> Date: Thu, 20 Nov 2025 20:41:54 +0000 Subject: [PATCH 30/32] Remove erroneous call in applyServeTargetCapacity (#4212) Signed-off-by: Ryan O'Leary --- ray-operator/controllers/ray/rayservice_controller.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/ray-operator/controllers/ray/rayservice_controller.go b/ray-operator/controllers/ray/rayservice_controller.go index a774cd3d127..53405652c96 100644 --- a/ray-operator/controllers/ray/rayservice_controller.go +++ b/ray-operator/controllers/ray/rayservice_controller.go @@ -1361,13 +1361,12 @@ func (r *RayServiceReconciler) applyServeTargetCapacity(ctx context.Context, ray return err } - // Update the status fields and cache new Serve config. + // Update the TargetCapacity status fields. if rayClusterInstance.Name == rayServiceInstance.Status.ActiveServiceStatus.RayClusterName { rayServiceInstance.Status.ActiveServiceStatus.TargetCapacity = ptr.To(goalTargetCapacity) } else if rayClusterInstance.Name == rayServiceInstance.Status.PendingServiceStatus.RayClusterName { rayServiceInstance.Status.PendingServiceStatus.TargetCapacity = ptr.To(goalTargetCapacity) } - r.cacheServeConfig(rayServiceInstance, rayClusterInstance.Name) return nil } From 1daadda8b0bcad24d12a680722eaf06818522efc Mon Sep 17 00:00:00 2001 From: "Han-Ju Chen (Future-Outlier)" Date: Fri, 21 Nov 2025 05:45:19 +0800 Subject: [PATCH 31/32] [RayJob] Add token authentication support for light weight job submitter (#4215) * [RayJob] light weight job submitter auth token support Signed-off-by: Future-Outlier * X-Ray-Authorization Signed-off-by: Rueian --------- Signed-off-by: Future-Outlier Signed-off-by: Rueian Co-authored-by: Rueian --- ray-operator/rayjob-submitter/cmd/main.go | 5 +++-- ray-operator/rayjob-submitter/rayjob-submitter.go | 14 ++++++++++++-- 2 files changed, 15 insertions(+), 4 deletions(-) diff --git a/ray-operator/rayjob-submitter/cmd/main.go b/ray-operator/rayjob-submitter/cmd/main.go index cbc52cc3968..d827d4d9aeb 100644 --- a/ray-operator/rayjob-submitter/cmd/main.go +++ b/ray-operator/rayjob-submitter/cmd/main.go @@ -64,7 +64,8 @@ func main() { } rayDashboardClient := &dashboardclient.RayDashboardClient{} address = rayjobsubmitter.JobSubmissionURL(address) - rayDashboardClient.InitClient(&http.Client{Timeout: time.Second * 10}, address, "") + authToken := os.Getenv("RAY_AUTH_TOKEN") + rayDashboardClient.InitClient(&http.Client{Timeout: time.Second * 10}, address, authToken) submissionId, err := rayDashboardClient.SubmitJobReq(context.Background(), &req) if err != nil { if strings.Contains(err.Error(), "Please use a different submission_id") { @@ -76,7 +77,7 @@ func main() { fmt.Fprintf(os.Stdout, "SUCC -- Job '%s' submitted successfully\n", submissionId) } fmt.Fprintf(os.Stdout, "INFO -- Tailing logs until the job finishes:\n") - err = rayjobsubmitter.TailJobLogs(address, submissionId, os.Stdout) + err = rayjobsubmitter.TailJobLogs(address, submissionId, authToken, os.Stdout) exitOnError(err) } diff --git a/ray-operator/rayjob-submitter/rayjob-submitter.go b/ray-operator/rayjob-submitter/rayjob-submitter.go index 0c2e9fa88e0..bc8a4b7baa1 100644 --- a/ray-operator/rayjob-submitter/rayjob-submitter.go +++ b/ray-operator/rayjob-submitter/rayjob-submitter.go @@ -31,12 +31,22 @@ func logTailingURL(address, submissionId string) (string, error) { return address, nil } -func TailJobLogs(address, submissionId string, out io.Writer) error { +func TailJobLogs(address, submissionId string, authToken string, out io.Writer) error { wsAddr, err := logTailingURL(address, submissionId) if err != nil { return err } - c, _, err := websocket.Dial(context.Background(), wsAddr, nil) + + var dialOptions *websocket.DialOptions + if authToken != "" { + dialOptions = &websocket.DialOptions{ + HTTPHeader: map[string][]string{ + "X-Ray-Authorization": {fmt.Sprintf("Bearer %s", authToken)}, + }, + } + } + + c, _, err := websocket.Dial(context.Background(), wsAddr, dialOptions) if err != nil { return err } From e23189e2c7b672042ffaa3b94625d084240ff3dc Mon Sep 17 00:00:00 2001 From: Rueian Date: Thu, 20 Nov 2025 19:44:44 -0800 Subject: [PATCH 32/32] feat: kubectl ray get token command (#4218) * feat: kubectl ray get token command Signed-off-by: Rueian * Update kubectl-plugin/pkg/cmd/get/get_token_test.go Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com> Signed-off-by: Rueian * Update kubectl-plugin/pkg/cmd/get/get_token.go Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com> Signed-off-by: Rueian * make sure the raycluster exists before getting the secret Signed-off-by: Rueian * better ux Signed-off-by: Rueian * Update kubectl-plugin/pkg/cmd/get/get_token.go Co-authored-by: Han-Ju Chen (Future-Outlier) Signed-off-by: Rueian --------- Signed-off-by: Rueian Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com> Co-authored-by: Han-Ju Chen (Future-Outlier) --- kubectl-plugin/pkg/cmd/get/get.go | 1 + kubectl-plugin/pkg/cmd/get/get_token.go | 90 ++++++++++++++++++++ kubectl-plugin/pkg/cmd/get/get_token_test.go | 61 +++++++++++++ 3 files changed, 152 insertions(+) create mode 100644 kubectl-plugin/pkg/cmd/get/get_token.go create mode 100644 kubectl-plugin/pkg/cmd/get/get_token_test.go diff --git a/kubectl-plugin/pkg/cmd/get/get.go b/kubectl-plugin/pkg/cmd/get/get.go index 530f5f78bc1..c634d00410a 100644 --- a/kubectl-plugin/pkg/cmd/get/get.go +++ b/kubectl-plugin/pkg/cmd/get/get.go @@ -27,6 +27,7 @@ func NewGetCommand(cmdFactory cmdutil.Factory, streams genericclioptions.IOStrea cmd.AddCommand(NewGetClusterCommand(cmdFactory, streams)) cmd.AddCommand(NewGetWorkerGroupCommand(cmdFactory, streams)) cmd.AddCommand(NewGetNodesCommand(cmdFactory, streams)) + cmd.AddCommand(NewGetTokenCommand(cmdFactory, streams)) return cmd } diff --git a/kubectl-plugin/pkg/cmd/get/get_token.go b/kubectl-plugin/pkg/cmd/get/get_token.go new file mode 100644 index 00000000000..7c99a3ab4a9 --- /dev/null +++ b/kubectl-plugin/pkg/cmd/get/get_token.go @@ -0,0 +1,90 @@ +package get + +import ( + "context" + "fmt" + + "github.com/spf13/cobra" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/cli-runtime/pkg/genericclioptions" + cmdutil "k8s.io/kubectl/pkg/cmd/util" + + "github.com/ray-project/kuberay/kubectl-plugin/pkg/util/client" + "github.com/ray-project/kuberay/kubectl-plugin/pkg/util/completion" + rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1" + "github.com/ray-project/kuberay/ray-operator/controllers/ray/utils" +) + +type GetTokenOptions struct { + cmdFactory cmdutil.Factory + ioStreams *genericclioptions.IOStreams + namespace string + cluster string +} + +func NewGetTokenOptions(cmdFactory cmdutil.Factory, streams genericclioptions.IOStreams) *GetTokenOptions { + return &GetTokenOptions{ + cmdFactory: cmdFactory, + ioStreams: &streams, + } +} + +func NewGetTokenCommand(cmdFactory cmdutil.Factory, streams genericclioptions.IOStreams) *cobra.Command { + options := NewGetTokenOptions(cmdFactory, streams) + + cmd := &cobra.Command{ + Use: "token [CLUSTER NAME]", + Aliases: []string{"token"}, + Short: "Get the auth token from the ray cluster.", + SilenceUsage: true, + ValidArgsFunction: completion.RayClusterCompletionFunc(cmdFactory), + Args: cobra.ExactArgs(1), + RunE: func(cmd *cobra.Command, args []string) error { + if err := options.Complete(args, cmd); err != nil { + return err + } + // running cmd.Execute or cmd.ExecuteE sets the context, which will be done by root + k8sClient, err := client.NewClient(cmdFactory) + if err != nil { + return fmt.Errorf("failed to create client: %w", err) + } + return options.Run(cmd.Context(), k8sClient) + }, + } + return cmd +} + +func (options *GetTokenOptions) Complete(args []string, cmd *cobra.Command) error { + namespace, err := cmd.Flags().GetString("namespace") + if err != nil { + return fmt.Errorf("failed to get namespace: %w", err) + } + options.namespace = namespace + if options.namespace == "" { + options.namespace = "default" + } + // guarded by cobra.ExactArgs(1) + options.cluster = args[0] + return nil +} + +func (options *GetTokenOptions) Run(ctx context.Context, k8sClient client.Client) error { + cluster, err := k8sClient.RayClient().RayV1().RayClusters(options.namespace).Get(ctx, options.cluster, v1.GetOptions{}) + if err != nil { + return fmt.Errorf("failed to get RayCluster %s/%s: %w", options.namespace, options.cluster, err) + } + if cluster.Spec.AuthOptions == nil || cluster.Spec.AuthOptions.Mode != rayv1.AuthModeToken { + return fmt.Errorf("RayCluster %s/%s was not configured to use authentication tokens", options.namespace, options.cluster) + } + // TODO: support custom token secret? + secret, err := k8sClient.KubernetesClient().CoreV1().Secrets(options.namespace).Get(ctx, options.cluster, v1.GetOptions{}) + if err != nil { + return fmt.Errorf("failed to get secret %s/%s: %w", options.namespace, options.cluster, err) + } + if token, ok := secret.Data[utils.RAY_AUTH_TOKEN_SECRET_KEY]; ok { + _, err = fmt.Fprint(options.ioStreams.Out, string(token)) + } else { + err = fmt.Errorf("secret %s/%s does not have an auth_token", options.namespace, options.cluster) + } + return err +} diff --git a/kubectl-plugin/pkg/cmd/get/get_token_test.go b/kubectl-plugin/pkg/cmd/get/get_token_test.go new file mode 100644 index 00000000000..e09d16942a0 --- /dev/null +++ b/kubectl-plugin/pkg/cmd/get/get_token_test.go @@ -0,0 +1,61 @@ +package get + +import ( + "testing" + + "github.com/spf13/cobra" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + corev1 "k8s.io/api/core/v1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/cli-runtime/pkg/genericclioptions" + kubefake "k8s.io/client-go/kubernetes/fake" + cmdutil "k8s.io/kubectl/pkg/cmd/util" + + "github.com/ray-project/kuberay/kubectl-plugin/pkg/util/client" + rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1" + rayClientFake "github.com/ray-project/kuberay/ray-operator/pkg/client/clientset/versioned/fake" +) + +// Tests the Run() step of the command and ensure that the output is as expected. +func TestTokenGetRun(t *testing.T) { + cmdFactory := cmdutil.NewFactory(genericclioptions.NewConfigFlags(true)) + + testStreams, _, resBuf, _ := genericclioptions.NewTestIOStreams() + fakeTokenGetOptions := NewGetTokenOptions(cmdFactory, testStreams) + + rayCluster := &rayv1.RayCluster{ + ObjectMeta: v1.ObjectMeta{ + Name: "raycluster-kuberay", + Namespace: "test", + }, + Spec: rayv1.RayClusterSpec{ + AuthOptions: &rayv1.AuthOptions{ + Mode: rayv1.AuthModeToken, + }, + }, + } + + secret := &corev1.Secret{ + ObjectMeta: v1.ObjectMeta{ + Name: "raycluster-kuberay", + Namespace: "test", + }, + Data: map[string][]byte{ + "auth_token": []byte("token"), + }, + } + + kubeClientSet := kubefake.NewClientset(secret) + rayClient := rayClientFake.NewSimpleClientset(rayCluster) + k8sClients := client.NewClientForTesting(kubeClientSet, rayClient) + + cmd := &cobra.Command{} + cmd.Flags().StringVarP(&fakeTokenGetOptions.namespace, "namespace", "n", secret.Namespace, "") + err := fakeTokenGetOptions.Complete([]string{rayCluster.Name}, cmd) + require.NoError(t, err) + err = fakeTokenGetOptions.Run(t.Context(), k8sClients) + require.NoError(t, err) + + assert.Equal(t, secret.Data["auth_token"], resBuf.Bytes()) +}