Skip to content

Commit 1f98479

Browse files
authored
[feat][operator] validate Ray resource metadata in webhook (#3831)
when the Ray resource is created or updated. For users who opt into installing the webhooks, this validation makes the K8s API fail the resource creation or update request immediately in a synchronous way that's more obvious. We refactor the RayCluster webhook to use the same validation functions that the RayCluster controller uses. We also add tests for RayJob and RayService webhooks. Signed-off-by: David Xia <[email protected]>
1 parent b6bcf10 commit 1f98479

File tree

8 files changed

+449
-101
lines changed

8 files changed

+449
-101
lines changed

ray-operator/config/webhook/manifests.yaml

Lines changed: 40 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -24,3 +24,43 @@ webhooks:
2424
resources:
2525
- rayclusters
2626
sideEffects: None
27+
- admissionReviewVersions:
28+
- v1
29+
clientConfig:
30+
service:
31+
name: webhook-service
32+
namespace: system
33+
path: /validate-ray-io-v1-rayjob
34+
failurePolicy: Fail
35+
name: vrayjob.kb.io
36+
rules:
37+
- apiGroups:
38+
- ray.io
39+
apiVersions:
40+
- v1
41+
operations:
42+
- CREATE
43+
- UPDATE
44+
resources:
45+
- rayjobs
46+
sideEffects: None
47+
- admissionReviewVersions:
48+
- v1
49+
clientConfig:
50+
service:
51+
name: webhook-service
52+
namespace: system
53+
path: /validate-ray-io-v1-rayservice
54+
failurePolicy: Fail
55+
name: vrayservice.kb.io
56+
rules:
57+
- apiGroups:
58+
- ray.io
59+
apiVersions:
60+
- v1
61+
operations:
62+
- CREATE
63+
- UPDATE
64+
resources:
65+
- rayservices
66+
sideEffects: None

ray-operator/pkg/webhooks/v1/raycluster_webhook.go

Lines changed: 6 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -2,7 +2,6 @@ package v1
22

33
import (
44
"context"
5-
"regexp"
65

76
apierrors "k8s.io/apimachinery/pkg/api/errors"
87
"k8s.io/apimachinery/pkg/runtime"
@@ -14,13 +13,10 @@ import (
1413
"sigs.k8s.io/controller-runtime/pkg/webhook/admission"
1514

1615
rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1"
16+
"github.com/ray-project/kuberay/ray-operator/controllers/ray/utils"
1717
)
1818

19-
// log is for logging in this package.
20-
var (
21-
rayclusterlog = logf.Log.WithName("raycluster-resource")
22-
nameRegex, _ = regexp.Compile("^[a-z]([-a-z0-9]*[a-z0-9])?$")
23-
)
19+
var rayClusterLog = logf.Log.WithName("raycluster-resource")
2420

2521
// SetupRayClusterWebhookWithManager registers the webhook for RayCluster in the manager.
2622
func SetupRayClusterWebhookWithManager(mgr ctrl.Manager) error {
@@ -32,22 +28,21 @@ func SetupRayClusterWebhookWithManager(mgr ctrl.Manager) error {
3228

3329
type RayClusterWebhook struct{}
3430

35-
// TODO(user): change verbs to "verbs=create;update;delete" if you want to enable deletion validation.
3631
//+kubebuilder:webhook:path=/validate-ray-io-v1-raycluster,mutating=false,failurePolicy=fail,sideEffects=None,groups=ray.io,resources=rayclusters,verbs=create;update,versions=v1,name=vraycluster.kb.io,admissionReviewVersions=v1
3732

3833
var _ webhook.CustomValidator = &RayClusterWebhook{}
3934

4035
// ValidateCreate implements webhook.CustomValidator so a webhook will be registered for the type
4136
func (w *RayClusterWebhook) ValidateCreate(_ context.Context, obj runtime.Object) (admission.Warnings, error) {
4237
rayCluster := obj.(*rayv1.RayCluster)
43-
rayclusterlog.Info("validate create", "name", rayCluster.Name)
38+
rayClusterLog.Info("validate create", "name", rayCluster.Name)
4439
return nil, w.validateRayCluster(rayCluster)
4540
}
4641

4742
// ValidateUpdate implements webhook.CustomValidator so a webhook will be registered for the type
4843
func (w *RayClusterWebhook) ValidateUpdate(_ context.Context, _ runtime.Object, newObj runtime.Object) (admission.Warnings, error) {
4944
rayCluster := newObj.(*rayv1.RayCluster)
50-
rayclusterlog.Info("validate update", "name", rayCluster.Name)
45+
rayClusterLog.Info("validate update", "name", rayCluster.Name)
5146
return nil, w.validateRayCluster(rayCluster)
5247
}
5348

@@ -59,8 +54,8 @@ func (w *RayClusterWebhook) ValidateDelete(_ context.Context, _ runtime.Object)
5954
func (w *RayClusterWebhook) validateRayCluster(rayCluster *rayv1.RayCluster) error {
6055
var allErrs field.ErrorList
6156

62-
if err := w.validateName(rayCluster); err != nil {
63-
allErrs = append(allErrs, err)
57+
if err := utils.ValidateRayClusterMetadata(rayCluster.ObjectMeta); err != nil {
58+
allErrs = append(allErrs, field.Invalid(field.NewPath("metadata").Child("name"), rayCluster.Name, err.Error()))
6459
}
6560

6661
if err := w.validateWorkerGroups(rayCluster); err != nil {
@@ -76,13 +71,6 @@ func (w *RayClusterWebhook) validateRayCluster(rayCluster *rayv1.RayCluster) err
7671
rayCluster.Name, allErrs)
7772
}
7873

79-
func (w *RayClusterWebhook) validateName(rayCluster *rayv1.RayCluster) *field.Error {
80-
if !nameRegex.MatchString(rayCluster.Name) {
81-
return field.Invalid(field.NewPath("metadata").Child("name"), rayCluster.Name, "name must consist of lower case alphanumeric characters or '-', start with an alphabetic character, and end with an alphanumeric character (e.g. 'my-name', or 'abc-123', regex used for validation is '[a-z]([-a-z0-9]*[a-z0-9])?')")
82-
}
83-
return nil
84-
}
85-
8674
func (w *RayClusterWebhook) validateWorkerGroups(rayCluster *rayv1.RayCluster) *field.Error {
8775
workerGroupNames := make(map[string]bool)
8876

Lines changed: 121 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,121 @@
1+
package v1
2+
3+
import (
4+
"context"
5+
"fmt"
6+
7+
. "github.com/onsi/ginkgo/v2"
8+
. "github.com/onsi/gomega"
9+
corev1 "k8s.io/api/core/v1"
10+
//+kubebuilder:scaffold:imports
11+
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
12+
"k8s.io/apimachinery/pkg/util/rand"
13+
14+
rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1"
15+
)
16+
17+
var _ = Describe("RayCluster validating webhook", func() {
18+
Context("when name is too long", func() {
19+
It("should return error", func() {
20+
longName := "this-name-is-tooooooooooooooooooooooooooooooooooooooooooo-long-and-should-be-invalid"
21+
rayCluster := rayv1.RayCluster{
22+
ObjectMeta: metav1.ObjectMeta{
23+
Namespace: "default",
24+
Name: longName,
25+
},
26+
Spec: rayv1.RayClusterSpec{
27+
HeadGroupSpec: rayv1.HeadGroupSpec{
28+
Template: corev1.PodTemplateSpec{
29+
Spec: corev1.PodSpec{
30+
Containers: []corev1.Container{},
31+
},
32+
},
33+
},
34+
WorkerGroupSpecs: []rayv1.WorkerGroupSpec{},
35+
},
36+
}
37+
38+
err := k8sClient.Create(context.TODO(), &rayCluster)
39+
Expect(err).To(HaveOccurred())
40+
41+
Expect(err.Error()).To(ContainSubstring(fmt.Sprintf("RayCluster.ray.io \"%s\" is invalid: metadata.name", longName)))
42+
})
43+
})
44+
45+
Context("when name isn't a DNS1035 label", func() {
46+
It("should return error", func() {
47+
rayCluster := rayv1.RayCluster{
48+
ObjectMeta: metav1.ObjectMeta{
49+
Namespace: "default",
50+
Name: "invalid.name",
51+
},
52+
Spec: rayv1.RayClusterSpec{
53+
HeadGroupSpec: rayv1.HeadGroupSpec{
54+
Template: corev1.PodTemplateSpec{
55+
Spec: corev1.PodSpec{
56+
Containers: []corev1.Container{},
57+
},
58+
},
59+
},
60+
WorkerGroupSpecs: []rayv1.WorkerGroupSpec{},
61+
},
62+
}
63+
64+
err := k8sClient.Create(context.TODO(), &rayCluster)
65+
Expect(err).To(HaveOccurred())
66+
67+
Expect(err.Error()).To(ContainSubstring("RayCluster.ray.io \"invalid.name\" is invalid: metadata.name:"))
68+
})
69+
})
70+
71+
Context("when groupNames are not unique", func() {
72+
var name, namespace string
73+
var rayCluster rayv1.RayCluster
74+
75+
BeforeEach(func() {
76+
namespace = "default"
77+
name = fmt.Sprintf("test-raycluster-%d", rand.IntnRange(1000, 9000))
78+
})
79+
80+
It("should return error", func() {
81+
rayCluster = rayv1.RayCluster{
82+
ObjectMeta: metav1.ObjectMeta{
83+
Name: name,
84+
Namespace: namespace,
85+
},
86+
Spec: rayv1.RayClusterSpec{
87+
HeadGroupSpec: rayv1.HeadGroupSpec{
88+
Template: corev1.PodTemplateSpec{
89+
Spec: corev1.PodSpec{
90+
Containers: []corev1.Container{},
91+
},
92+
},
93+
},
94+
WorkerGroupSpecs: []rayv1.WorkerGroupSpec{
95+
{
96+
GroupName: "group1",
97+
Template: corev1.PodTemplateSpec{
98+
Spec: corev1.PodSpec{
99+
Containers: []corev1.Container{},
100+
},
101+
},
102+
},
103+
{
104+
GroupName: "group1",
105+
Template: corev1.PodTemplateSpec{
106+
Spec: corev1.PodSpec{
107+
Containers: []corev1.Container{},
108+
},
109+
},
110+
},
111+
},
112+
},
113+
}
114+
115+
err := k8sClient.Create(context.TODO(), &rayCluster)
116+
Expect(err).To(HaveOccurred())
117+
118+
Expect(err.Error()).To(ContainSubstring("worker group names must be unique"))
119+
})
120+
})
121+
})
Lines changed: 68 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,68 @@
1+
package v1
2+
3+
import (
4+
"context"
5+
6+
apierrors "k8s.io/apimachinery/pkg/api/errors"
7+
"k8s.io/apimachinery/pkg/runtime"
8+
"k8s.io/apimachinery/pkg/runtime/schema"
9+
"k8s.io/apimachinery/pkg/util/validation/field"
10+
ctrl "sigs.k8s.io/controller-runtime"
11+
logf "sigs.k8s.io/controller-runtime/pkg/log"
12+
"sigs.k8s.io/controller-runtime/pkg/webhook"
13+
"sigs.k8s.io/controller-runtime/pkg/webhook/admission"
14+
15+
rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1"
16+
"github.com/ray-project/kuberay/ray-operator/controllers/ray/utils"
17+
)
18+
19+
var rayJobLog = logf.Log.WithName("rayjob-resource")
20+
21+
// SetupRayJobWebhookWithManager registers the webhook for RayJob in the manager.
22+
func SetupRayJobWebhookWithManager(mgr ctrl.Manager) error {
23+
return ctrl.NewWebhookManagedBy(mgr).
24+
For(&rayv1.RayJob{}).
25+
WithValidator(&RayJobWebhook{}).
26+
Complete()
27+
}
28+
29+
type RayJobWebhook struct{}
30+
31+
//+kubebuilder:webhook:path=/validate-ray-io-v1-rayjob,mutating=false,failurePolicy=fail,sideEffects=None,groups=ray.io,resources=rayjobs,verbs=create;update,versions=v1,name=vrayjob.kb.io,admissionReviewVersions=v1
32+
33+
var _ webhook.CustomValidator = &RayJobWebhook{}
34+
35+
// ValidateCreate implements webhook.CustomValidator so a webhook will be registered for the type
36+
func (w *RayJobWebhook) ValidateCreate(_ context.Context, obj runtime.Object) (admission.Warnings, error) {
37+
rayJob := obj.(*rayv1.RayJob)
38+
rayJobLog.Info("validate create", "name", rayJob.Name)
39+
return nil, w.validateRayJob(rayJob)
40+
}
41+
42+
// ValidateUpdate implements webhook.CustomValidator so a webhook will be registered for the type
43+
func (w *RayJobWebhook) ValidateUpdate(_ context.Context, _ runtime.Object, newObj runtime.Object) (admission.Warnings, error) {
44+
rayJob := newObj.(*rayv1.RayJob)
45+
rayJobLog.Info("validate update", "name", rayJob.Name)
46+
return nil, w.validateRayJob(rayJob)
47+
}
48+
49+
// ValidateDelete implements webhook.CustomValidator so a webhook will be registered for the type
50+
func (w *RayJobWebhook) ValidateDelete(_ context.Context, _ runtime.Object) (admission.Warnings, error) {
51+
return nil, nil
52+
}
53+
54+
func (w *RayJobWebhook) validateRayJob(rayJob *rayv1.RayJob) error {
55+
var allErrs field.ErrorList
56+
57+
if err := utils.ValidateRayJobMetadata(rayJob.ObjectMeta); err != nil {
58+
allErrs = append(allErrs, field.Invalid(field.NewPath("metadata").Child("name"), rayJob.Name, err.Error()))
59+
}
60+
61+
if len(allErrs) == 0 {
62+
return nil
63+
}
64+
65+
return apierrors.NewInvalid(
66+
schema.GroupKind{Group: "ray.io", Kind: "RayJob"},
67+
rayJob.Name, allErrs)
68+
}
Lines changed: 71 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,71 @@
1+
package v1
2+
3+
import (
4+
"context"
5+
"fmt"
6+
7+
. "github.com/onsi/ginkgo/v2"
8+
. "github.com/onsi/gomega"
9+
corev1 "k8s.io/api/core/v1"
10+
//+kubebuilder:scaffold:imports
11+
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
12+
13+
rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1"
14+
)
15+
16+
var _ = Describe("RayJob validating webhook", func() {
17+
Context("when name is too long", func() {
18+
It("should return error", func() {
19+
longName := "this-name-is-tooooooooooooooooooooooooooooooooooooooooooo-long-and-should-be-invalid"
20+
rayJob := rayv1.RayJob{
21+
ObjectMeta: metav1.ObjectMeta{
22+
Namespace: "default",
23+
Name: longName,
24+
},
25+
Spec: rayv1.RayJobSpec{
26+
RayClusterSpec: &rayv1.RayClusterSpec{
27+
HeadGroupSpec: rayv1.HeadGroupSpec{
28+
Template: corev1.PodTemplateSpec{
29+
Spec: corev1.PodSpec{
30+
Containers: []corev1.Container{},
31+
},
32+
},
33+
},
34+
},
35+
},
36+
}
37+
38+
err := k8sClient.Create(context.TODO(), &rayJob)
39+
Expect(err).To(HaveOccurred())
40+
41+
Expect(err.Error()).To(ContainSubstring(fmt.Sprintf("RayJob.ray.io \"%s\" is invalid: metadata.name", longName)))
42+
})
43+
})
44+
45+
Context("when name isn't a DNS1035 label", func() {
46+
It("should return error", func() {
47+
rayJob := rayv1.RayJob{
48+
ObjectMeta: metav1.ObjectMeta{
49+
Namespace: "default",
50+
Name: "invalid.name",
51+
},
52+
Spec: rayv1.RayJobSpec{
53+
RayClusterSpec: &rayv1.RayClusterSpec{
54+
HeadGroupSpec: rayv1.HeadGroupSpec{
55+
Template: corev1.PodTemplateSpec{
56+
Spec: corev1.PodSpec{
57+
Containers: []corev1.Container{},
58+
},
59+
},
60+
},
61+
},
62+
},
63+
}
64+
65+
err := k8sClient.Create(context.TODO(), &rayJob)
66+
Expect(err).To(HaveOccurred())
67+
68+
Expect(err.Error()).To(ContainSubstring("RayJob.ray.io \"invalid.name\" is invalid: metadata.name:"))
69+
})
70+
})
71+
})

0 commit comments

Comments
 (0)