diff --git a/ray-operator/controllers/ray/raycluster_controller.go b/ray-operator/controllers/ray/raycluster_controller.go index 269e94b363a..10ec8499397 100644 --- a/ray-operator/controllers/ray/raycluster_controller.go +++ b/ray-operator/controllers/ray/raycluster_controller.go @@ -222,18 +222,34 @@ func validateRayClusterStatus(instance *rayv1.RayCluster) error { // Validation for invalid Ray Cluster configurations. func validateRayClusterSpec(instance *rayv1.RayCluster) error { - if instance.Annotations[utils.RayFTEnabledAnnotationKey] == "false" && instance.Spec.GcsFaultToleranceOptions != nil { - return fmt.Errorf("GcsFaultToleranceOptions should be nil when %s is set to false", utils.RayFTEnabledAnnotationKey) + if len(instance.Spec.HeadGroupSpec.Template.Spec.Containers) == 0 { + return fmt.Errorf("headGroupSpec should have at least one container") } - if instance.Annotations[utils.RayFTEnabledAnnotationKey] != "true" && len(instance.Spec.HeadGroupSpec.Template.Spec.Containers) > 0 { + for _, workerGroup := range instance.Spec.WorkerGroupSpecs { + if len(workerGroup.Template.Spec.Containers) == 0 { + return fmt.Errorf("workerGroupSpec should have at least one container") + } + } + + if instance.Annotations[utils.RayFTEnabledAnnotationKey] != "" && instance.Spec.GcsFaultToleranceOptions != nil { + return fmt.Errorf("%s annotation and GcsFaultToleranceOptions are both set. "+ + "Please use only GcsFaultToleranceOptions to configure GCS fault tolerance", utils.RayFTEnabledAnnotationKey) + } + + if !common.IsGCSFaultToleranceEnabled(*instance) { if utils.EnvVarExists(utils.RAY_REDIS_ADDRESS, instance.Spec.HeadGroupSpec.Template.Spec.Containers[utils.RayContainerIndex].Env) { - return fmt.Errorf( - "%s environment variable should not be set when %s annotation is not set to true", - utils.RAY_REDIS_ADDRESS, utils.RayFTEnabledAnnotationKey, - ) + return fmt.Errorf("%s is set which implicitly enables GCS fault tolerance, "+ + "but GcsFaultToleranceOptions is not set. Please set GcsFaultToleranceOptions "+ + "to enable GCS fault tolerance", utils.RAY_REDIS_ADDRESS) } } + + // TODO (kevin85421): If GcsFaultToleranceOptions is set, users should use `GcsFaultToleranceOptions.RedisAddress` instead of `RAY_REDIS_ADDRESS`. + // TODO (kevin85421): If GcsFaultToleranceOptions is set, users should use `GcsFaultToleranceOptions.RedisPassword` instead of `RAY_REDIS_PASSWORD` + // or `redis-password` in rayStartParams. + // TODO (kevin85421): If GcsFaultToleranceOptions is set, users should use `GcsFaultToleranceOptions.ExternalStorageNamespace` instead of + // the annotation `ray.io/external-storage-namespace`. return nil } diff --git a/ray-operator/controllers/ray/raycluster_controller_unit_test.go b/ray-operator/controllers/ray/raycluster_controller_unit_test.go index 2c530100f2c..0545f39e09d 100644 --- a/ray-operator/controllers/ray/raycluster_controller_unit_test.go +++ b/ray-operator/controllers/ray/raycluster_controller_unit_test.go @@ -3473,7 +3473,13 @@ func Test_ReconcileManagedBy(t *testing.T) { } } -func TestValidateRayClusterSpec(t *testing.T) { +func TestValidateRayClusterSpecGcsFaultToleranceOptions(t *testing.T) { + errorMessageBothSet := fmt.Sprintf("%s annotation and GcsFaultToleranceOptions are both set. "+ + "Please use only GcsFaultToleranceOptions to configure GCS fault tolerance", utils.RayFTEnabledAnnotationKey) + errorMessageRedisAddressSet := fmt.Sprintf("%s is set which implicitly enables GCS fault tolerance, "+ + "but GcsFaultToleranceOptions is not set. Please set GcsFaultToleranceOptions "+ + "to enable GCS fault tolerance", utils.RAY_REDIS_ADDRESS) + tests := []struct { gcsFaultToleranceOptions *rayv1.GcsFaultToleranceOptions annotations map[string]string @@ -3482,110 +3488,74 @@ func TestValidateRayClusterSpec(t *testing.T) { envVars []corev1.EnvVar expectError bool }{ + // GcsFaultToleranceOptions and ray.io/ft-enabled should not be both set. { - name: "FT disabled with GcsFaultToleranceOptions set", + name: "ray.io/ft-enabled is set to false and GcsFaultToleranceOptions is set", annotations: map[string]string{ utils.RayFTEnabledAnnotationKey: "false", }, gcsFaultToleranceOptions: &rayv1.GcsFaultToleranceOptions{}, expectError: true, - errorMessage: fmt.Sprintf("GcsFaultToleranceOptions should be nil when %s is set to false", utils.RayFTEnabledAnnotationKey), + errorMessage: errorMessageBothSet, }, { - name: "FT disabled with RAY_REDIS_ADDRESS set", + name: "ray.io/ft-enabled is set to true and GcsFaultToleranceOptions is set", annotations: map[string]string{ - utils.RayFTEnabledAnnotationKey: "false", - }, - envVars: []corev1.EnvVar{ - { - Name: utils.RAY_REDIS_ADDRESS, - Value: "redis://127.0.0.1:6379", - }, + utils.RayFTEnabledAnnotationKey: "true", }, - expectError: true, - errorMessage: fmt.Sprintf( - "%s environment variable should not be set when %s annotation is not set to true", - utils.RAY_REDIS_ADDRESS, utils.RayFTEnabledAnnotationKey, - ), + gcsFaultToleranceOptions: &rayv1.GcsFaultToleranceOptions{}, + expectError: true, + errorMessage: errorMessageBothSet, }, { - name: "FT not set with RAY_REDIS_ADDRESS set", - annotations: map[string]string{}, - envVars: []corev1.EnvVar{ - { - Name: utils.RAY_REDIS_ADDRESS, - Value: "redis://127.0.0.1:6379", - }, - }, - expectError: true, - errorMessage: fmt.Sprintf( - "%s environment variable should not be set when %s annotation is not set to true", - utils.RAY_REDIS_ADDRESS, utils.RayFTEnabledAnnotationKey, - ), + name: "ray.io/ft-enabled is not set and GcsFaultToleranceOptions is set", + gcsFaultToleranceOptions: &rayv1.GcsFaultToleranceOptions{}, + expectError: false, + }, + { + name: "ray.io/ft-enabled is not set and GcsFaultToleranceOptions is not set", + gcsFaultToleranceOptions: nil, + expectError: false, }, + // RAY_REDIS_ADDRESS should not be set if KubeRay is not aware that GCS fault tolerance is enabled. { - name: "FT disabled with other environment variables set", + name: "ray.io/ft-enabled is set to false and RAY_REDIS_ADDRESS is set", annotations: map[string]string{ utils.RayFTEnabledAnnotationKey: "false", }, envVars: []corev1.EnvVar{ { - Name: "SOME_OTHER_ENV", - Value: "some-value", + Name: utils.RAY_REDIS_ADDRESS, + Value: "redis:6379", }, }, - expectError: false, - }, - { - name: "FT enabled, GcsFaultToleranceOptions not nil", - annotations: map[string]string{ - utils.RayFTEnabledAnnotationKey: "true", - }, - gcsFaultToleranceOptions: &rayv1.GcsFaultToleranceOptions{ - RedisAddress: "redis://127.0.0.1:6379", - }, - expectError: false, - }, - { - name: "FT enabled, GcsFaultToleranceOptions is nil", - annotations: map[string]string{ - utils.RayFTEnabledAnnotationKey: "true", - }, - expectError: false, + expectError: true, + errorMessage: errorMessageRedisAddressSet, }, { - name: "FT enabled with with other environment variables set", - annotations: map[string]string{ - utils.RayFTEnabledAnnotationKey: "true", - }, + name: "FT is disabled and RAY_REDIS_ADDRESS is set", envVars: []corev1.EnvVar{ { - Name: "SOME_OTHER_ENV", - Value: "some-value", + Name: utils.RAY_REDIS_ADDRESS, + Value: "redis:6379", }, }, - expectError: false, + expectError: true, + errorMessage: errorMessageRedisAddressSet, }, { - name: "FT enabled with RAY_REDIS_ADDRESS set", + name: "ray.io/ft-enabled is set to true and RAY_REDIS_ADDRESS is set", annotations: map[string]string{ utils.RayFTEnabledAnnotationKey: "true", }, envVars: []corev1.EnvVar{ { Name: utils.RAY_REDIS_ADDRESS, - Value: "redis://127.0.0.1:6379", + Value: "redis:6379", }, }, expectError: false, }, - { - name: "FT disabled with no GcsFaultToleranceOptions and no RAY_REDIS_ADDRESS", - annotations: map[string]string{ - utils.RayFTEnabledAnnotationKey: "false", - }, - expectError: false, - }, } for _, tt := range tests { @@ -3620,6 +3590,78 @@ func TestValidateRayClusterSpec(t *testing.T) { } } +func TestValidateRayClusterSpecEmptyContainers(t *testing.T) { + headGroupSpecWithOneContainer := rayv1.HeadGroupSpec{ + Template: corev1.PodTemplateSpec{ + Spec: corev1.PodSpec{ + Containers: []corev1.Container{{Name: "ray-head"}}, + }, + }, + } + workerGroupSpecWithOneContainer := rayv1.WorkerGroupSpec{ + Template: corev1.PodTemplateSpec{ + Spec: corev1.PodSpec{ + Containers: []corev1.Container{{Name: "ray-worker"}}, + }, + }, + } + headGroupSpecWithNoContainers := *headGroupSpecWithOneContainer.DeepCopy() + headGroupSpecWithNoContainers.Template.Spec.Containers = []corev1.Container{} + workerGroupSpecWithNoContainers := *workerGroupSpecWithOneContainer.DeepCopy() + workerGroupSpecWithNoContainers.Template.Spec.Containers = []corev1.Container{} + + tests := []struct { + rayCluster *rayv1.RayCluster + name string + errorMessage string + expectError bool + }{ + { + name: "headGroupSpec has no containers", + rayCluster: &rayv1.RayCluster{ + Spec: rayv1.RayClusterSpec{ + HeadGroupSpec: headGroupSpecWithNoContainers, + }, + }, + expectError: true, + errorMessage: "headGroupSpec should have at least one container", + }, + { + name: "workerGroupSpec has no containers", + rayCluster: &rayv1.RayCluster{ + Spec: rayv1.RayClusterSpec{ + HeadGroupSpec: headGroupSpecWithOneContainer, + WorkerGroupSpecs: []rayv1.WorkerGroupSpec{workerGroupSpecWithNoContainers}, + }, + }, + expectError: true, + errorMessage: "workerGroupSpec should have at least one container", + }, + { + name: "valid cluster with containers in both head and worker groups", + rayCluster: &rayv1.RayCluster{ + Spec: rayv1.RayClusterSpec{ + HeadGroupSpec: headGroupSpecWithOneContainer, + WorkerGroupSpecs: []rayv1.WorkerGroupSpec{workerGroupSpecWithOneContainer}, + }, + }, + expectError: false, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + err := validateRayClusterSpec(tt.rayCluster) + if tt.expectError { + assert.Error(t, err) + assert.EqualError(t, err, tt.errorMessage) + } else { + assert.Nil(t, err) + } + }) + } +} + func TestValidateRayClusterStatus(t *testing.T) { tests := []struct { name string