Skip to content

Commit 8aa4d10

Browse files
authored
[RayService] Directly fail CR if is invalid (#4228)
* [RayService] Directly fail CR if is invalid Signed-off-by: win5923 <[email protected]> * nit: set the name with strings.Repeat(a, 48) Signed-off-by: win5923 <[email protected]> --------- Signed-off-by: win5923 <[email protected]>
1 parent 04530dd commit 8aa4d10

File tree

4 files changed

+84
-17
lines changed

4 files changed

+84
-17
lines changed

ray-operator/apis/ray/v1/rayservice_types.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -206,6 +206,7 @@ const (
206206
BothActivePendingClustersExist RayServiceConditionReason = "BothActivePendingClustersExist"
207207
NoPendingCluster RayServiceConditionReason = "NoPendingCluster"
208208
NoActiveCluster RayServiceConditionReason = "NoActiveCluster"
209+
RayServiceValidationFailed RayServiceConditionReason = "ValidationFailed"
209210
)
210211

211212
// +kubebuilder:object:root=true

ray-operator/controllers/ray/rayservice_controller.go

Lines changed: 34 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -115,15 +115,21 @@ func (r *RayServiceReconciler) Reconcile(ctx context.Context, request ctrl.Reque
115115
}
116116
originalRayServiceInstance := rayServiceInstance.DeepCopy()
117117

118-
if err := utils.ValidateRayServiceMetadata(rayServiceInstance.ObjectMeta); err != nil {
119-
r.Recorder.Eventf(rayServiceInstance, corev1.EventTypeWarning, string(utils.InvalidRayServiceMetadata),
120-
"The RayService metadata is invalid %s/%s: %v", rayServiceInstance.Namespace, rayServiceInstance.Name, err)
121-
return ctrl.Result{RequeueAfter: ServiceDefaultRequeueDuration}, err
122-
}
123-
if err := utils.ValidateRayServiceSpec(rayServiceInstance); err != nil {
124-
r.Recorder.Eventf(rayServiceInstance, corev1.EventTypeWarning, string(utils.InvalidRayServiceSpec),
125-
"The RayService spec is invalid %s/%s: %v", rayServiceInstance.Namespace, rayServiceInstance.Name, err)
126-
return ctrl.Result{RequeueAfter: ServiceDefaultRequeueDuration}, err
118+
// Perform all validations and directly fail the RayService if any of the validation fails
119+
errType, err := validateRayService(ctx, rayServiceInstance)
120+
// Immediately update the status after validation
121+
if err != nil {
122+
r.Recorder.Eventf(rayServiceInstance, corev1.EventTypeWarning, string(errType),
123+
"%s/%s: %v", rayServiceInstance.Namespace, rayServiceInstance.Name, err)
124+
125+
setCondition(rayServiceInstance, rayv1.RayServiceReady, metav1.ConditionFalse, rayv1.RayServiceValidationFailed, err.Error())
126+
rayServiceInstance.Status.LastUpdateTime = &metav1.Time{Time: time.Now()}
127+
128+
if updateErr := r.Status().Update(ctx, rayServiceInstance); updateErr != nil {
129+
logger.Info("Failed to update RayService status", "error", updateErr)
130+
return ctrl.Result{RequeueAfter: ServiceDefaultRequeueDuration}, updateErr
131+
}
132+
return ctrl.Result{}, nil
127133
}
128134

129135
r.cleanUpServeConfigCache(ctx, rayServiceInstance)
@@ -265,6 +271,25 @@ func (r *RayServiceReconciler) Reconcile(ctx context.Context, request ctrl.Reque
265271
return ctrl.Result{RequeueAfter: ServiceDefaultRequeueDuration}, nil
266272
}
267273

274+
func validateRayService(ctx context.Context, rayServiceInstance *rayv1.RayService) (utils.K8sEventType, error) {
275+
logger := ctrl.LoggerFrom(ctx)
276+
validationRules := []struct {
277+
validate func() error
278+
errType utils.K8sEventType
279+
}{
280+
{func() error { return utils.ValidateRayServiceMetadata(rayServiceInstance.ObjectMeta) }, utils.InvalidRayServiceMetadata},
281+
{func() error { return utils.ValidateRayServiceSpec(rayServiceInstance) }, utils.InvalidRayServiceSpec},
282+
}
283+
284+
for _, validation := range validationRules {
285+
if err := validation.validate(); err != nil {
286+
logger.Error(err, err.Error())
287+
return validation.errType, err
288+
}
289+
}
290+
return "", nil
291+
}
292+
268293
func (r *RayServiceReconciler) reconcileServicesToReadyCluster(ctx context.Context, rayServiceInstance *rayv1.RayService, rayClusterInstance *rayv1.RayCluster) (*corev1.Service, *corev1.Service, error) {
269294
// Create K8s services if they don't exist. If they do exist, update the services to point to the RayCluster passed in.
270295
headSvc, err := r.reconcileServices(ctx, rayServiceInstance, rayClusterInstance, utils.HeadService)

ray-operator/controllers/ray/utils/validation.go

Lines changed: 10 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -311,15 +311,15 @@ func ValidateRayJobSpec(rayJob *rayv1.RayJob) error {
311311

312312
func ValidateRayServiceMetadata(metadata metav1.ObjectMeta) error {
313313
if len(metadata.Name) > MaxRayServiceNameLength {
314-
return fmt.Errorf("RayService name should be no more than %d characters", MaxRayServiceNameLength)
314+
return fmt.Errorf("The RayService metadata is invalid: RayService name should be no more than %d characters", MaxRayServiceNameLength)
315315
}
316316
if errs := validation.IsDNS1035Label(metadata.Name); len(errs) > 0 {
317-
return fmt.Errorf("RayService name should be a valid DNS1035 label: %v", errs)
317+
return fmt.Errorf("The RayService metadata is invalid: RayService name should be a valid DNS1035 label: %v", errs)
318318
}
319319

320320
// Validate initializing timeout annotation if present
321321
if err := validateInitializingTimeout(metadata.Annotations); err != nil {
322-
return fmt.Errorf("RayService annotations is invalid: %w", err)
322+
return fmt.Errorf("The RayService metadata is invalid: RayService annotations is invalid: %w", err)
323323
}
324324

325325
return nil
@@ -359,11 +359,11 @@ func validateInitializingTimeout(annotations map[string]string) error {
359359

360360
func ValidateRayServiceSpec(rayService *rayv1.RayService) error {
361361
if err := ValidateRayClusterSpec(&rayService.Spec.RayClusterSpec, rayService.Annotations); err != nil {
362-
return err
362+
return fmt.Errorf("The RayService spec is invalid: %w", err)
363363
}
364364

365365
if headSvc := rayService.Spec.RayClusterSpec.HeadGroupSpec.HeadService; headSvc != nil && headSvc.Name != "" {
366-
return fmt.Errorf("spec.rayClusterConfig.headGroupSpec.headService.metadata.name should not be set")
366+
return fmt.Errorf("The RayService spec is invalid: spec.rayClusterConfig.headGroupSpec.headService.metadata.name should not be set")
367367
}
368368

369369
// only NewClusterWithIncrementalUpgrade, NewCluster, and None are valid upgradeType
@@ -372,17 +372,19 @@ func ValidateRayServiceSpec(rayService *rayv1.RayService) error {
372372
*rayService.Spec.UpgradeStrategy.Type != rayv1.None &&
373373
*rayService.Spec.UpgradeStrategy.Type != rayv1.NewCluster &&
374374
*rayService.Spec.UpgradeStrategy.Type != rayv1.NewClusterWithIncrementalUpgrade {
375-
return fmt.Errorf("Spec.UpgradeStrategy.Type value %s is invalid, valid options are %s, %s, or %s", *rayService.Spec.UpgradeStrategy.Type, rayv1.NewClusterWithIncrementalUpgrade, rayv1.NewCluster, rayv1.None)
375+
return fmt.Errorf("The RayService spec is invalid: Spec.UpgradeStrategy.Type value %s is invalid, valid options are %s, %s, or %s", *rayService.Spec.UpgradeStrategy.Type, rayv1.NewClusterWithIncrementalUpgrade, rayv1.NewCluster, rayv1.None)
376376
}
377377

378378
if rayService.Spec.RayClusterDeletionDelaySeconds != nil &&
379379
*rayService.Spec.RayClusterDeletionDelaySeconds < 0 {
380-
return fmt.Errorf("Spec.RayClusterDeletionDelaySeconds should be a non-negative integer, got %d", *rayService.Spec.RayClusterDeletionDelaySeconds)
380+
return fmt.Errorf("The RayService spec is invalid: Spec.RayClusterDeletionDelaySeconds should be a non-negative integer, got %d", *rayService.Spec.RayClusterDeletionDelaySeconds)
381381
}
382382

383383
// If type is NewClusterWithIncrementalUpgrade, validate the ClusterUpgradeOptions
384384
if IsIncrementalUpgradeEnabled(&rayService.Spec) {
385-
return ValidateClusterUpgradeOptions(rayService)
385+
if err := ValidateClusterUpgradeOptions(rayService); err != nil {
386+
return fmt.Errorf("The RayService spec is invalid: %w", err)
387+
}
386388
}
387389

388390
return nil
Lines changed: 39 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,39 @@
1+
package e2erayservice
2+
3+
import (
4+
"strings"
5+
"testing"
6+
7+
. "github.com/onsi/gomega"
8+
"k8s.io/apimachinery/pkg/api/meta"
9+
10+
rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1"
11+
rayv1ac "github.com/ray-project/kuberay/ray-operator/pkg/client/applyconfiguration/ray/v1"
12+
. "github.com/ray-project/kuberay/ray-operator/test/support"
13+
)
14+
15+
func TestRayServiceValidation(t *testing.T) {
16+
test := With(t)
17+
g := NewWithT(t)
18+
19+
// Create a namespace
20+
namespace := test.NewTestNamespace()
21+
22+
test.T().Run("RayService name too long with 48 characters", func(_ *testing.T) {
23+
rayServiceAC := rayv1ac.RayService(strings.Repeat("a", 48), namespace.Name).
24+
WithSpec(RayServiceSampleYamlApplyConfiguration())
25+
26+
rayService, err := test.Client().Ray().RayV1().RayServices(namespace.Name).Apply(test.Ctx(), rayServiceAC, TestApplyOptions)
27+
g.Expect(err).NotTo(HaveOccurred())
28+
LogWithTimestamp(test.T(), "Created RayService %s/%s successfully", rayService.Namespace, rayService.Name)
29+
30+
g.Eventually(RayService(test, rayService.Namespace, rayService.Name), TestTimeoutShort).
31+
Should(WithTransform(func(rs *rayv1.RayService) bool {
32+
condition := meta.FindStatusCondition(rs.Status.Conditions, string(rayv1.RayServiceReady))
33+
if condition == nil {
34+
return false
35+
}
36+
return condition.Status == "False" && condition.Reason == string(rayv1.RayServiceValidationFailed)
37+
}, BeTrue()))
38+
})
39+
}

0 commit comments

Comments
 (0)