Skip to content

Commit 7a8b212

Browse files
committed
Use webhook.CustomValidator instead of deprecated webhook.Validator.
1 parent aafe2e0 commit 7a8b212

File tree

5 files changed

+113
-102
lines changed

5 files changed

+113
-102
lines changed

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

Lines changed: 0 additions & 89 deletions
This file was deleted.

ray-operator/apis/ray/v1/zz_generated.deepcopy.go

Lines changed: 1 addition & 1 deletion
Some generated files are not rendered by default. Learn more about customizing how changed files appear on GitHub.

ray-operator/main.go

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -35,6 +35,7 @@ import (
3535
"github.com/ray-project/kuberay/ray-operator/controllers/ray"
3636
"github.com/ray-project/kuberay/ray-operator/controllers/ray/utils"
3737
"github.com/ray-project/kuberay/ray-operator/pkg/features"
38+
"github.com/ray-project/kuberay/ray-operator/pkg/webhooks/v1"
3839
// +kubebuilder:scaffold:imports
3940
)
4041

@@ -241,7 +242,7 @@ func main() {
241242
"unable to create controller", "controller", "RayJob")
242243

243244
if os.Getenv("ENABLE_WEBHOOKS") == "true" {
244-
exitOnError((&rayv1.RayCluster{}).SetupWebhookWithManager(mgr),
245+
exitOnError(v1.SetupRayClusterWebhookWithManager(mgr),
245246
"unable to create webhook", "webhook", "RayCluster")
246247
}
247248
// +kubebuilder:scaffold:builder
Lines changed: 97 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,97 @@
1+
package v1
2+
3+
import (
4+
"context"
5+
"regexp"
6+
7+
apierrors "k8s.io/apimachinery/pkg/api/errors"
8+
"k8s.io/apimachinery/pkg/runtime"
9+
"k8s.io/apimachinery/pkg/runtime/schema"
10+
"k8s.io/apimachinery/pkg/util/validation/field"
11+
ctrl "sigs.k8s.io/controller-runtime"
12+
logf "sigs.k8s.io/controller-runtime/pkg/log"
13+
"sigs.k8s.io/controller-runtime/pkg/webhook"
14+
"sigs.k8s.io/controller-runtime/pkg/webhook/admission"
15+
16+
rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1"
17+
)
18+
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+
)
24+
25+
// SetupRayClusterWebhookWithManager registers the webhook for RayCluster in the manager.
26+
func SetupRayClusterWebhookWithManager(mgr ctrl.Manager) error {
27+
return ctrl.NewWebhookManagedBy(mgr).
28+
For(&rayv1.RayCluster{}).
29+
WithValidator(&RayClusterWebhook{}).
30+
Complete()
31+
}
32+
33+
type RayClusterWebhook struct{}
34+
35+
// TODO(user): change verbs to "verbs=create;update;delete" if you want to enable deletion validation.
36+
//+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
37+
38+
var _ webhook.CustomValidator = &RayClusterWebhook{}
39+
40+
// ValidateCreate implements webhook.CustomValidator so a webhook will be registered for the type
41+
func (w *RayClusterWebhook) ValidateCreate(_ context.Context, obj runtime.Object) (admission.Warnings, error) {
42+
rayCluster := obj.(*rayv1.RayCluster)
43+
rayclusterlog.Info("validate create", "name", rayCluster.Name)
44+
return nil, w.validateRayCluster(rayCluster)
45+
}
46+
47+
// ValidateUpdate implements webhook.CustomValidator so a webhook will be registered for the type
48+
func (w *RayClusterWebhook) ValidateUpdate(_ context.Context, _ runtime.Object, newObj runtime.Object) (admission.Warnings, error) {
49+
rayCluster := newObj.(*rayv1.RayCluster)
50+
rayclusterlog.Info("validate update", "name", rayCluster.Name)
51+
return nil, w.validateRayCluster(rayCluster)
52+
}
53+
54+
// ValidateDelete implements webhook.CustomValidator so a webhook will be registered for the type
55+
func (w *RayClusterWebhook) ValidateDelete(_ context.Context, _ runtime.Object) (admission.Warnings, error) {
56+
return nil, nil
57+
}
58+
59+
func (w *RayClusterWebhook) validateRayCluster(rayCluster *rayv1.RayCluster) error {
60+
var allErrs field.ErrorList
61+
62+
if err := w.validateName(rayCluster); err != nil {
63+
allErrs = append(allErrs, err)
64+
}
65+
66+
if err := w.validateWorkerGroups(rayCluster); err != nil {
67+
allErrs = append(allErrs, err)
68+
}
69+
70+
if len(allErrs) == 0 {
71+
return nil
72+
}
73+
74+
return apierrors.NewInvalid(
75+
schema.GroupKind{Group: "ray.io", Kind: "RayCluster"},
76+
rayCluster.Name, allErrs)
77+
}
78+
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+
86+
func (w *RayClusterWebhook) validateWorkerGroups(rayCluster *rayv1.RayCluster) *field.Error {
87+
workerGroupNames := make(map[string]bool)
88+
89+
for i, workerGroup := range rayCluster.Spec.WorkerGroupSpecs {
90+
if _, ok := workerGroupNames[workerGroup.GroupName]; ok {
91+
return field.Invalid(field.NewPath("spec").Child("workerGroupSpecs").Index(i), workerGroup, "worker group names must be unique")
92+
}
93+
workerGroupNames[workerGroup.GroupName] = true
94+
}
95+
96+
return nil
97+
}

ray-operator/apis/ray/v1/webhook_suite_test.go renamed to ray-operator/pkg/webhooks/v1/webhook_suite_test.go

Lines changed: 13 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,8 @@ import (
2727
logf "sigs.k8s.io/controller-runtime/pkg/log"
2828
"sigs.k8s.io/controller-runtime/pkg/log/zap"
2929
"sigs.k8s.io/controller-runtime/pkg/webhook"
30+
31+
rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1"
3032
)
3133

3234
// These tests use Ginkgo (BDD-style Go testing framework). Refer to
@@ -67,7 +69,7 @@ var _ = BeforeSuite(func() {
6769
Expect(cfg).NotTo(BeNil())
6870

6971
scheme := runtime.NewScheme()
70-
err = AddToScheme(scheme)
72+
err = rayv1.AddToScheme(scheme)
7173
Expect(err).NotTo(HaveOccurred())
7274

7375
err = admissionv1beta1.AddToScheme(scheme)
@@ -95,7 +97,7 @@ var _ = BeforeSuite(func() {
9597
})
9698
Expect(err).NotTo(HaveOccurred())
9799

98-
err = (&RayCluster{}).SetupWebhookWithManager(mgr)
100+
err = SetupRayClusterWebhookWithManager(mgr)
99101
Expect(err).NotTo(HaveOccurred())
100102

101103
//+kubebuilder:scaffold:webhook
@@ -126,21 +128,21 @@ var _ = BeforeSuite(func() {
126128
var _ = Describe("RayCluster validating webhook", func() {
127129
Context("when name is invalid", func() {
128130
It("should return error", func() {
129-
rayCluster := RayCluster{
131+
rayCluster := rayv1.RayCluster{
130132
ObjectMeta: metav1.ObjectMeta{
131133
Namespace: "default",
132134
Name: "invalid.name",
133135
},
134-
Spec: RayClusterSpec{
135-
HeadGroupSpec: HeadGroupSpec{
136+
Spec: rayv1.RayClusterSpec{
137+
HeadGroupSpec: rayv1.HeadGroupSpec{
136138
RayStartParams: map[string]string{"DEADBEEF": "DEADBEEF"},
137139
Template: corev1.PodTemplateSpec{
138140
Spec: corev1.PodSpec{
139141
Containers: []corev1.Container{},
140142
},
141143
},
142144
},
143-
WorkerGroupSpecs: []WorkerGroupSpec{},
145+
WorkerGroupSpecs: []rayv1.WorkerGroupSpec{},
144146
},
145147
}
146148

@@ -153,29 +155,29 @@ var _ = Describe("RayCluster validating webhook", func() {
153155

154156
Context("when groupNames are not unique", func() {
155157
var name, namespace string
156-
var rayCluster RayCluster
158+
var rayCluster rayv1.RayCluster
157159

158160
BeforeEach(func() {
159161
namespace = "default"
160162
name = fmt.Sprintf("test-raycluster-%d", rand.IntnRange(1000, 9000))
161163
})
162164

163165
It("should return error", func() {
164-
rayCluster = RayCluster{
166+
rayCluster = rayv1.RayCluster{
165167
ObjectMeta: metav1.ObjectMeta{
166168
Name: name,
167169
Namespace: namespace,
168170
},
169-
Spec: RayClusterSpec{
170-
HeadGroupSpec: HeadGroupSpec{
171+
Spec: rayv1.RayClusterSpec{
172+
HeadGroupSpec: rayv1.HeadGroupSpec{
171173
RayStartParams: map[string]string{"DEADBEEF": "DEADBEEF"},
172174
Template: corev1.PodTemplateSpec{
173175
Spec: corev1.PodSpec{
174176
Containers: []corev1.Container{},
175177
},
176178
},
177179
},
178-
WorkerGroupSpecs: []WorkerGroupSpec{
180+
WorkerGroupSpecs: []rayv1.WorkerGroupSpec{
179181
{
180182
GroupName: "group1",
181183
RayStartParams: map[string]string{"DEADBEEF": "DEADBEEF"},

0 commit comments

Comments
 (0)