diff --git a/ray-operator/controllers/ray/batchscheduler/scheduler-plugins/scheduler_plugins.go b/ray-operator/controllers/ray/batchscheduler/scheduler-plugins/scheduler_plugins.go index 4756e6d2281..449c9d5dbcb 100644 --- a/ray-operator/controllers/ray/batchscheduler/scheduler-plugins/scheduler_plugins.go +++ b/ray-operator/controllers/ray/batchscheduler/scheduler-plugins/scheduler_plugins.go @@ -45,7 +45,7 @@ func (k *KubeScheduler) Name() string { return schedulerInstanceName } -func createPodGroup(ctx context.Context, app *rayv1.RayCluster) *v1alpha1.PodGroup { +func createPodGroup(app *rayv1.RayCluster) *v1alpha1.PodGroup { // TODO(troychiu): Consider the case when autoscaling is enabled. podGroup := &v1alpha1.PodGroup{ @@ -62,7 +62,7 @@ func createPodGroup(ctx context.Context, app *rayv1.RayCluster) *v1alpha1.PodGro }, }, Spec: v1alpha1.PodGroupSpec{ - MinMember: utils.CalculateDesiredReplicas(ctx, app) + 1, // +1 for the head pod + MinMember: utils.CalculateDesiredReplicas(app) + 1, // +1 for the head pod MinResources: utils.CalculateDesiredResources(app), }, } @@ -82,7 +82,7 @@ func (k *KubeScheduler) DoBatchSchedulingOnSubmission(ctx context.Context, objec if !errors.IsNotFound(err) { return err } - podGroup = createPodGroup(ctx, app) + podGroup = createPodGroup(app) if err := k.cli.Create(ctx, podGroup); err != nil { if errors.IsAlreadyExists(err) { return nil diff --git a/ray-operator/controllers/ray/batchscheduler/scheduler-plugins/scheduler_plugins_test.go b/ray-operator/controllers/ray/batchscheduler/scheduler-plugins/scheduler_plugins_test.go index e2201b6cf71..3e456772e1c 100644 --- a/ray-operator/controllers/ray/batchscheduler/scheduler-plugins/scheduler_plugins_test.go +++ b/ray-operator/controllers/ray/batchscheduler/scheduler-plugins/scheduler_plugins_test.go @@ -82,7 +82,7 @@ func TestCreatePodGroup(t *testing.T) { cluster := createTestRayCluster(1) - podGroup := createPodGroup(context.TODO(), &cluster) + podGroup := createPodGroup(&cluster) // 256m * 3 (requests, not limits) a.Equal("768m", podGroup.Spec.MinResources.Cpu().String()) @@ -102,7 +102,7 @@ func TestCreatePodGroupWithMultipleHosts(t *testing.T) { cluster := createTestRayCluster(2) // 2 hosts - podGroup := createPodGroup(context.TODO(), &cluster) + podGroup := createPodGroup(&cluster) // 256m * 5 (requests, not limits) a.Equal("1280m", podGroup.Spec.MinResources.Cpu().String()) diff --git a/ray-operator/controllers/ray/batchscheduler/volcano/volcano_scheduler.go b/ray-operator/controllers/ray/batchscheduler/volcano/volcano_scheduler.go index 45d0b0d2921..8f29864c266 100644 --- a/ray-operator/controllers/ray/batchscheduler/volcano/volcano_scheduler.go +++ b/ray-operator/controllers/ray/batchscheduler/volcano/volcano_scheduler.go @@ -62,7 +62,7 @@ func (v *VolcanoBatchScheduler) handleRayCluster(ctx context.Context, raycluster return nil } - minMember, totalResource := v.calculatePodGroupParams(ctx, &raycluster.Spec) + minMember, totalResource := v.calculatePodGroupParams(&raycluster.Spec) return v.syncPodGroup(ctx, raycluster, minMember, totalResource) } @@ -74,7 +74,7 @@ func (v *VolcanoBatchScheduler) handleRayJob(ctx context.Context, rayJob *rayv1. } var totalResourceList []corev1.ResourceList - minMember, totalResource := v.calculatePodGroupParams(ctx, rayJob.Spec.RayClusterSpec) + minMember, totalResource := v.calculatePodGroupParams(rayJob.Spec.RayClusterSpec) totalResourceList = append(totalResourceList, totalResource) // MinMember intentionally excludes the submitter pod to avoid a startup deadlock @@ -186,11 +186,11 @@ func (v *VolcanoBatchScheduler) syncPodGroup(ctx context.Context, owner metav1.O return nil } -func (v *VolcanoBatchScheduler) calculatePodGroupParams(ctx context.Context, rayClusterSpec *rayv1.RayClusterSpec) (int32, corev1.ResourceList) { +func (v *VolcanoBatchScheduler) calculatePodGroupParams(rayClusterSpec *rayv1.RayClusterSpec) (int32, corev1.ResourceList) { rayCluster := &rayv1.RayCluster{Spec: *rayClusterSpec} if !utils.IsAutoscalingEnabled(rayClusterSpec) { - return utils.CalculateDesiredReplicas(ctx, rayCluster) + 1, utils.CalculateDesiredResources(rayCluster) + return utils.CalculateDesiredReplicas(rayCluster) + 1, utils.CalculateDesiredResources(rayCluster) } return utils.CalculateMinReplicas(rayCluster) + 1, utils.CalculateMinResources(rayCluster) } diff --git a/ray-operator/controllers/ray/batchscheduler/volcano/volcano_scheduler_test.go b/ray-operator/controllers/ray/batchscheduler/volcano/volcano_scheduler_test.go index 6426c205500..f7c1cadda87 100644 --- a/ray-operator/controllers/ray/batchscheduler/volcano/volcano_scheduler_test.go +++ b/ray-operator/controllers/ray/batchscheduler/volcano/volcano_scheduler_test.go @@ -160,7 +160,7 @@ func TestCreatePodGroupForRayCluster(t *testing.T) { cluster := createTestRayCluster(1) - minMember := utils.CalculateDesiredReplicas(context.Background(), &cluster) + 1 + minMember := utils.CalculateDesiredReplicas(&cluster) + 1 totalResource := utils.CalculateDesiredResources(&cluster) pg, err := createPodGroup(&cluster, getAppPodGroupName(&cluster), minMember, totalResource) require.NoError(t, err) @@ -185,7 +185,7 @@ func TestCreatePodGroupForRayCluster_NumOfHosts2(t *testing.T) { cluster := createTestRayCluster(2) - minMember := utils.CalculateDesiredReplicas(context.Background(), &cluster) + 1 + minMember := utils.CalculateDesiredReplicas(&cluster) + 1 totalResource := utils.CalculateDesiredResources(&cluster) pg, err := createPodGroup(&cluster, getAppPodGroupName(&cluster), minMember, totalResource) require.NoError(t, err) @@ -227,7 +227,7 @@ func TestCreatePodGroup_NetworkTopologyBothLabels(t *testing.T) { NetworkTopologyHighestTierAllowedLabelKey: "3", }) - minMember := utils.CalculateDesiredReplicas(context.Background(), &cluster) + 1 + minMember := utils.CalculateDesiredReplicas(&cluster) + 1 totalResource := utils.CalculateDesiredResources(&cluster) pg, err := createPodGroup(&cluster, getAppPodGroupName(&cluster), minMember, totalResource) require.NoError(t, err) @@ -246,7 +246,7 @@ func TestCreatePodGroup_NetworkTopologyOnlyModeLabel(t *testing.T) { NetworkTopologyModeLabelKey: "hard", }) - minMember := utils.CalculateDesiredReplicas(context.Background(), &cluster) + 1 + minMember := utils.CalculateDesiredReplicas(&cluster) + 1 totalResource := utils.CalculateDesiredResources(&cluster) pg, err := createPodGroup(&cluster, getAppPodGroupName(&cluster), minMember, totalResource) require.NoError(t, err) @@ -266,7 +266,7 @@ func TestCreatePodGroup_NetworkTopologyHighestTierAllowedNotInt(t *testing.T) { NetworkTopologyHighestTierAllowedLabelKey: "not-an-int", }) - minMember := utils.CalculateDesiredReplicas(context.Background(), &cluster) + 1 + minMember := utils.CalculateDesiredReplicas(&cluster) + 1 totalResource := utils.CalculateDesiredResources(&cluster) pg, err := createPodGroup(&cluster, getAppPodGroupName(&cluster), minMember, totalResource) @@ -474,7 +474,7 @@ func TestCalculatePodGroupParams(t *testing.T) { t.Run("Autoscaling disabled", func(_ *testing.T) { cluster := createTestRayCluster(1) - minMember, totalResource := scheduler.calculatePodGroupParams(context.Background(), &cluster.Spec) + minMember, totalResource := scheduler.calculatePodGroupParams(&cluster.Spec) // 1 head + 2 workers (desired replicas) a.Equal(int32(3), minMember) @@ -490,7 +490,7 @@ func TestCalculatePodGroupParams(t *testing.T) { cluster := createTestRayCluster(1) cluster.Spec.EnableInTreeAutoscaling = ptr.To(true) - minMember, totalResource := scheduler.calculatePodGroupParams(context.Background(), &cluster.Spec) + minMember, totalResource := scheduler.calculatePodGroupParams(&cluster.Spec) // 1 head + 1 worker (min replicas) a.Equal(int32(2), minMember) diff --git a/ray-operator/controllers/ray/raycluster_controller.go b/ray-operator/controllers/ray/raycluster_controller.go index dc3bb1d0b76..b3600af3485 100644 --- a/ray-operator/controllers/ray/raycluster_controller.go +++ b/ray-operator/controllers/ray/raycluster_controller.go @@ -752,7 +752,7 @@ func (r *RayClusterReconciler) reconcilePods(ctx context.Context, instance *rayv continue } // workerReplicas will store the target number of pods for this worker group. - numExpectedWorkerPods := int(utils.GetWorkerGroupDesiredReplicas(ctx, worker)) + numExpectedWorkerPods := int(utils.GetWorkerGroupDesiredReplicas(worker)) logger.Info("reconcilePods", "desired workerReplicas (always adhering to minReplicas/maxReplica)", numExpectedWorkerPods, "worker group", worker.GroupName, "maxReplicas", worker.MaxReplicas, "minReplicas", worker.MinReplicas, "replicas", worker.Replicas) workerPods := corev1.PodList{} @@ -1051,7 +1051,7 @@ func (r *RayClusterReconciler) reconcileMultiHostWorkerGroup(ctx context.Context } } numRunningReplicas := len(validReplicaGroups) - numExpectedWorkerPods := int(utils.GetWorkerGroupDesiredReplicas(ctx, *worker)) + numExpectedWorkerPods := int(utils.GetWorkerGroupDesiredReplicas(*worker)) // Ensure that if numExpectedWorkerPods is not a multiple of NumOfHosts, we log an error. if numExpectedWorkerPods%int(worker.NumOfHosts) != 0 { @@ -1580,7 +1580,7 @@ func (r *RayClusterReconciler) calculateStatus(ctx context.Context, instance *ra newInstance.Status.ReadyWorkerReplicas = utils.CalculateReadyReplicas(runtimePods) newInstance.Status.AvailableWorkerReplicas = utils.CalculateAvailableReplicas(runtimePods) - newInstance.Status.DesiredWorkerReplicas = utils.CalculateDesiredReplicas(ctx, newInstance) + newInstance.Status.DesiredWorkerReplicas = utils.CalculateDesiredReplicas(newInstance) newInstance.Status.MinWorkerReplicas = utils.CalculateMinReplicas(newInstance) newInstance.Status.MaxWorkerReplicas = utils.CalculateMaxReplicas(newInstance) diff --git a/ray-operator/controllers/ray/utils/util.go b/ray-operator/controllers/ray/utils/util.go index 4b213a50887..bb60fb4a5c4 100644 --- a/ray-operator/controllers/ray/utils/util.go +++ b/ray-operator/controllers/ray/utils/util.go @@ -384,18 +384,13 @@ func GenerateIdentifier(clusterName string, nodeType rayv1.RayNodeType) string { return fmt.Sprintf("%s-%s", clusterName, nodeType) } -func GetWorkerGroupDesiredReplicas(ctx context.Context, workerGroupSpec rayv1.WorkerGroupSpec) int32 { - log := ctrl.LoggerFrom(ctx) +func GetWorkerGroupDesiredReplicas(workerGroupSpec rayv1.WorkerGroupSpec) int32 { // Always adhere to min/max replicas constraints. var workerReplicas int32 if workerGroupSpec.Suspend != nil && *workerGroupSpec.Suspend { return 0 } - if *workerGroupSpec.MinReplicas > *workerGroupSpec.MaxReplicas { - log.Info("minReplicas is greater than maxReplicas, using maxReplicas as desired replicas. "+ - "Please fix this to avoid any unexpected behaviors.", "minReplicas", *workerGroupSpec.MinReplicas, "maxReplicas", *workerGroupSpec.MaxReplicas) - workerReplicas = *workerGroupSpec.MaxReplicas - } else if workerGroupSpec.Replicas == nil || *workerGroupSpec.Replicas < *workerGroupSpec.MinReplicas { + if workerGroupSpec.Replicas == nil || *workerGroupSpec.Replicas < *workerGroupSpec.MinReplicas { // Replicas is impossible to be nil as it has a default value assigned in the CRD. // Add this check to make testing easier. workerReplicas = *workerGroupSpec.MinReplicas @@ -408,10 +403,10 @@ func GetWorkerGroupDesiredReplicas(ctx context.Context, workerGroupSpec rayv1.Wo } // CalculateDesiredReplicas calculate desired worker replicas at the cluster level -func CalculateDesiredReplicas(ctx context.Context, cluster *rayv1.RayCluster) int32 { +func CalculateDesiredReplicas(cluster *rayv1.RayCluster) int32 { count := int32(0) for _, nodeGroup := range cluster.Spec.WorkerGroupSpecs { - count += GetWorkerGroupDesiredReplicas(ctx, nodeGroup) + count += GetWorkerGroupDesiredReplicas(nodeGroup) } return count diff --git a/ray-operator/controllers/ray/utils/util_test.go b/ray-operator/controllers/ray/utils/util_test.go index 66eb7bc078f..f8d196e7f69 100644 --- a/ray-operator/controllers/ray/utils/util_test.go +++ b/ray-operator/controllers/ray/utils/util_test.go @@ -553,7 +553,6 @@ func TestGenerateHeadServiceName(t *testing.T) { } func TestGetWorkerGroupDesiredReplicas(t *testing.T) { - ctx := context.Background() // Test 1: `WorkerGroupSpec.Replicas` is nil. // `Replicas` is impossible to be nil in a real RayCluster CR as it has a default value assigned in the CRD. numOfHosts := int32(1) @@ -565,37 +564,21 @@ func TestGetWorkerGroupDesiredReplicas(t *testing.T) { MinReplicas: &minReplicas, MaxReplicas: &maxReplicas, } - assert.Equal(t, GetWorkerGroupDesiredReplicas(ctx, workerGroupSpec), minReplicas) + assert.Equal(t, GetWorkerGroupDesiredReplicas(workerGroupSpec), minReplicas) // Test 2: `WorkerGroupSpec.Replicas` is not nil and is within the range. replicas := int32(3) workerGroupSpec.Replicas = &replicas - assert.Equal(t, GetWorkerGroupDesiredReplicas(ctx, workerGroupSpec), replicas) + assert.Equal(t, GetWorkerGroupDesiredReplicas(workerGroupSpec), replicas) - // Test 3: `WorkerGroupSpec.Replicas` is not nil but is more than maxReplicas. - replicas = int32(6) - workerGroupSpec.Replicas = &replicas - assert.Equal(t, GetWorkerGroupDesiredReplicas(ctx, workerGroupSpec), maxReplicas) - - // Test 4: `WorkerGroupSpec.Replicas` is not nil but is less than minReplicas. - replicas = int32(0) - workerGroupSpec.Replicas = &replicas - assert.Equal(t, GetWorkerGroupDesiredReplicas(ctx, workerGroupSpec), minReplicas) - - // Test 5: `WorkerGroupSpec.Replicas` is nil and minReplicas is less than maxReplicas. - workerGroupSpec.Replicas = nil - workerGroupSpec.MinReplicas = &maxReplicas - workerGroupSpec.MaxReplicas = &minReplicas - assert.Equal(t, GetWorkerGroupDesiredReplicas(ctx, workerGroupSpec), *workerGroupSpec.MaxReplicas) - - // Test 6: `WorkerGroupSpec.Suspend` is true. + // Test 3: `WorkerGroupSpec.Suspend` is true. suspend := true workerGroupSpec.MinReplicas = &maxReplicas workerGroupSpec.MaxReplicas = &minReplicas workerGroupSpec.Suspend = &suspend - assert.Zero(t, GetWorkerGroupDesiredReplicas(ctx, workerGroupSpec)) + assert.Zero(t, GetWorkerGroupDesiredReplicas(workerGroupSpec)) - // Test 7: `WorkerGroupSpec.NumOfHosts` is 4. + // Test 4: `WorkerGroupSpec.NumOfHosts` is 4. numOfHosts = int32(4) replicas = int32(5) suspend = false @@ -604,7 +587,7 @@ func TestGetWorkerGroupDesiredReplicas(t *testing.T) { workerGroupSpec.Suspend = &suspend workerGroupSpec.MinReplicas = &minReplicas workerGroupSpec.MaxReplicas = &maxReplicas - assert.Equal(t, GetWorkerGroupDesiredReplicas(ctx, workerGroupSpec), replicas*numOfHosts) + assert.Equal(t, GetWorkerGroupDesiredReplicas(workerGroupSpec), replicas*numOfHosts) } func TestCalculateMinAndMaxReplicas(t *testing.T) { @@ -801,7 +784,7 @@ func TestCalculateDesiredReplicas(t *testing.T) { }, }, } - assert.Equal(t, CalculateDesiredReplicas(context.Background(), &cluster), tc.answer) + assert.Equal(t, CalculateDesiredReplicas(&cluster), tc.answer) }) } } diff --git a/ray-operator/controllers/ray/utils/validation.go b/ray-operator/controllers/ray/utils/validation.go index 50b795d104a..f34758629e7 100644 --- a/ray-operator/controllers/ray/utils/validation.go +++ b/ray-operator/controllers/ray/utils/validation.go @@ -111,11 +111,30 @@ func ValidateRayClusterSpec(spec *rayv1.RayClusterSpec, annotations map[string]s return err } + // Check if autoscaling is enabled once to avoid repeated calls + isAutoscalingEnabled := IsAutoscalingEnabled(spec) + for _, workerGroup := range spec.WorkerGroupSpecs { if len(workerGroup.Template.Spec.Containers) == 0 { return fmt.Errorf("workerGroupSpec should have at least one container") } + // When autoscaling is enabled, MinReplicas and MaxReplicas are optional + // as users can manually update them and the autoscaler will handle the adjustment. + if !isAutoscalingEnabled && (workerGroup.MinReplicas == nil || workerGroup.MaxReplicas == nil) { + return fmt.Errorf("worker group %s must set both minReplicas and maxReplicas when autoscaling is disabled", workerGroup.GroupName) + } + if workerGroup.MinReplicas != nil && *workerGroup.MinReplicas < 0 { + return fmt.Errorf("worker group %s has negative minReplicas %d", workerGroup.GroupName, *workerGroup.MinReplicas) + } + if workerGroup.MaxReplicas != nil && *workerGroup.MaxReplicas < 0 { + return fmt.Errorf("worker group %s has negative maxReplicas %d", workerGroup.GroupName, *workerGroup.MaxReplicas) + } + if workerGroup.MinReplicas != nil && workerGroup.MaxReplicas != nil { + if *workerGroup.MinReplicas > *workerGroup.MaxReplicas { + return fmt.Errorf("worker group %s has minReplicas %d greater than maxReplicas %d", workerGroup.GroupName, *workerGroup.MinReplicas, *workerGroup.MaxReplicas) + } + } if err := validateRayGroupResources(workerGroup.GroupName, workerGroup.RayStartParams, workerGroup.Resources); err != nil { return err } @@ -175,9 +194,6 @@ func ValidateRayClusterSpec(spec *rayv1.RayClusterSpec, annotations map[string]s } } - // Check if autoscaling is enabled once to avoid repeated calls - isAutoscalingEnabled := IsAutoscalingEnabled(spec) - // Validate that RAY_enable_autoscaler_v2 environment variable is not set to "1" or "true" when autoscaler is disabled if !isAutoscalingEnabled { if envVar, exists := EnvVarByName(RAY_ENABLE_AUTOSCALER_V2, spec.HeadGroupSpec.Template.Spec.Containers[RayContainerIndex].Env); exists { diff --git a/ray-operator/controllers/ray/utils/validation_test.go b/ray-operator/controllers/ray/utils/validation_test.go index 36cadc9b260..e4e5db02acf 100644 --- a/ray-operator/controllers/ray/utils/validation_test.go +++ b/ray-operator/controllers/ray/utils/validation_test.go @@ -395,7 +395,9 @@ func TestValidateRayClusterSpecEmptyContainers(t *testing.T) { Template: podTemplateSpec(nil, nil), } workerGroupSpecWithOneContainer := rayv1.WorkerGroupSpec{ - Template: podTemplateSpec(nil, nil), + Template: podTemplateSpec(nil, nil), + MinReplicas: ptr.To(int32(0)), + MaxReplicas: ptr.To(int32(5)), } headGroupSpecWithNoContainers := *headGroupSpecWithOneContainer.DeepCopy() headGroupSpecWithNoContainers.Template.Spec.Containers = []corev1.Container{} @@ -459,8 +461,10 @@ func TestValidateRayClusterSpecSuspendingWorkerGroup(t *testing.T) { Template: podTemplateSpec(nil, nil), } workerGroupSpecSuspended := rayv1.WorkerGroupSpec{ - GroupName: "worker-group-1", - Template: podTemplateSpec(nil, nil), + GroupName: "worker-group-1", + Template: podTemplateSpec(nil, nil), + MinReplicas: ptr.To(int32(0)), + MaxReplicas: ptr.To(int32(5)), } workerGroupSpecSuspended.Suspend = ptr.To(true) @@ -692,8 +696,10 @@ func TestValidateRayClusterSpec_Resources(t *testing.T) { }, WorkerGroupSpecs: []rayv1.WorkerGroupSpec{ { - GroupName: "worker-group", - Template: podTemplateSpec(nil, nil), + GroupName: "worker-group", + Template: podTemplateSpec(nil, nil), + MinReplicas: ptr.To(int32(0)), + MaxReplicas: ptr.To(int32(5)), }, }, } @@ -773,8 +779,10 @@ func TestValidateRayClusterSpec_Labels(t *testing.T) { }, WorkerGroupSpecs: []rayv1.WorkerGroupSpec{ { - GroupName: "worker-group", - Template: podTemplateSpec(nil, nil), + GroupName: "worker-group", + Template: podTemplateSpec(nil, nil), + MinReplicas: ptr.To(int32(0)), + MaxReplicas: ptr.To(int32(5)), }, }, } @@ -2473,3 +2481,83 @@ func TestValidateRayClusterUpgradeOptions(t *testing.T) { }) } } + +func TestValidateRayClusterSpec_WorkerGroupReplicaValidation(t *testing.T) { + createSpec := func() rayv1.RayClusterSpec { + return rayv1.RayClusterSpec{ + EnableInTreeAutoscaling: ptr.To(false), + HeadGroupSpec: rayv1.HeadGroupSpec{ + Template: podTemplateSpec(nil, nil), + }, + } + } + + tests := []struct { + name string + errorMsg string + spec rayv1.RayClusterSpec + expectError bool + }{ + { + name: "minReplicas greater than maxReplicas", + spec: func() rayv1.RayClusterSpec { + s := createSpec() + s.WorkerGroupSpecs = []rayv1.WorkerGroupSpec{ + { + GroupName: "worker-group-3", + Template: podTemplateSpec(nil, nil), + MinReplicas: ptr.To(int32(5)), + MaxReplicas: ptr.To(int32(3)), + }, + } + return s + }(), + expectError: true, + errorMsg: "worker group worker-group-3 has minReplicas 5 greater than maxReplicas 3", + }, + { + name: "replicas smaller than minReplicas when autoscaling disabled", + spec: func() rayv1.RayClusterSpec { + s := createSpec() + s.WorkerGroupSpecs = []rayv1.WorkerGroupSpec{ + { + GroupName: "worker-group-3", + Template: podTemplateSpec(nil, nil), + Replicas: ptr.To(int32(1)), + MinReplicas: ptr.To(int32(2)), + MaxReplicas: ptr.To(int32(5)), + }, + } + return s + }(), + expectError: false, + }, + { + name: "valid when autoscaling enabled", + spec: func() rayv1.RayClusterSpec { + s := createSpec() + s.EnableInTreeAutoscaling = ptr.To(true) + s.WorkerGroupSpecs = []rayv1.WorkerGroupSpec{ + { + GroupName: "worker-group-3", + Template: podTemplateSpec(nil, nil), + }, + } + return s + }(), + expectError: false, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + err := ValidateRayClusterSpec(&tt.spec, nil) + if tt.expectError { + require.Error(t, err) + require.EqualError(t, err, tt.errorMsg) + } else { + require.NoError(t, err) + } + }) + } +}