Skip to content

Commit

Permalink
Use webhook.CustomValidator instead of deprecated webhook.Validator.
Browse files Browse the repository at this point in the history
  • Loading branch information
mbobrovskyi committed Jan 27, 2025
1 parent aafe2e0 commit 7a8b212
Show file tree
Hide file tree
Showing 5 changed files with 113 additions and 102 deletions.
89 changes: 0 additions & 89 deletions ray-operator/apis/ray/v1/raycluster_webhook.go

This file was deleted.

2 changes: 1 addition & 1 deletion ray-operator/apis/ray/v1/zz_generated.deepcopy.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

3 changes: 2 additions & 1 deletion ray-operator/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ import (
"github.com/ray-project/kuberay/ray-operator/controllers/ray"
"github.com/ray-project/kuberay/ray-operator/controllers/ray/utils"
"github.com/ray-project/kuberay/ray-operator/pkg/features"
"github.com/ray-project/kuberay/ray-operator/pkg/webhooks/v1"
// +kubebuilder:scaffold:imports
)

Expand Down Expand Up @@ -241,7 +242,7 @@ func main() {
"unable to create controller", "controller", "RayJob")

if os.Getenv("ENABLE_WEBHOOKS") == "true" {
exitOnError((&rayv1.RayCluster{}).SetupWebhookWithManager(mgr),
exitOnError(v1.SetupRayClusterWebhookWithManager(mgr),
"unable to create webhook", "webhook", "RayCluster")
}
// +kubebuilder:scaffold:builder
Expand Down
97 changes: 97 additions & 0 deletions ray-operator/pkg/webhooks/v1/raycluster_webhook.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,97 @@
package v1

import (
"context"
"regexp"

apierrors "k8s.io/apimachinery/pkg/api/errors"
"k8s.io/apimachinery/pkg/runtime"
"k8s.io/apimachinery/pkg/runtime/schema"
"k8s.io/apimachinery/pkg/util/validation/field"
ctrl "sigs.k8s.io/controller-runtime"
logf "sigs.k8s.io/controller-runtime/pkg/log"
"sigs.k8s.io/controller-runtime/pkg/webhook"
"sigs.k8s.io/controller-runtime/pkg/webhook/admission"

rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1"
)

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

// SetupRayClusterWebhookWithManager registers the webhook for RayCluster in the manager.
func SetupRayClusterWebhookWithManager(mgr ctrl.Manager) error {
return ctrl.NewWebhookManagedBy(mgr).
For(&rayv1.RayCluster{}).
WithValidator(&RayClusterWebhook{}).
Complete()
}

type RayClusterWebhook struct{}

// TODO(user): change verbs to "verbs=create;update;delete" if you want to enable deletion validation.
//+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

var _ webhook.CustomValidator = &RayClusterWebhook{}

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

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

// ValidateDelete implements webhook.CustomValidator so a webhook will be registered for the type
func (w *RayClusterWebhook) ValidateDelete(_ context.Context, _ runtime.Object) (admission.Warnings, error) {
return nil, nil
}

func (w *RayClusterWebhook) validateRayCluster(rayCluster *rayv1.RayCluster) error {
var allErrs field.ErrorList

if err := w.validateName(rayCluster); err != nil {
allErrs = append(allErrs, err)
}

if err := w.validateWorkerGroups(rayCluster); err != nil {
allErrs = append(allErrs, err)
}

if len(allErrs) == 0 {
return nil
}

return apierrors.NewInvalid(
schema.GroupKind{Group: "ray.io", Kind: "RayCluster"},
rayCluster.Name, allErrs)
}

func (w *RayClusterWebhook) validateName(rayCluster *rayv1.RayCluster) *field.Error {
if !nameRegex.MatchString(rayCluster.Name) {
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])?')")
}
return nil
}

func (w *RayClusterWebhook) validateWorkerGroups(rayCluster *rayv1.RayCluster) *field.Error {
workerGroupNames := make(map[string]bool)

for i, workerGroup := range rayCluster.Spec.WorkerGroupSpecs {
if _, ok := workerGroupNames[workerGroup.GroupName]; ok {
return field.Invalid(field.NewPath("spec").Child("workerGroupSpecs").Index(i), workerGroup, "worker group names must be unique")
}
workerGroupNames[workerGroup.GroupName] = true
}

return nil
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,8 @@ import (
logf "sigs.k8s.io/controller-runtime/pkg/log"
"sigs.k8s.io/controller-runtime/pkg/log/zap"
"sigs.k8s.io/controller-runtime/pkg/webhook"

rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1"
)

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

scheme := runtime.NewScheme()
err = AddToScheme(scheme)
err = rayv1.AddToScheme(scheme)
Expect(err).NotTo(HaveOccurred())

err = admissionv1beta1.AddToScheme(scheme)
Expand Down Expand Up @@ -95,7 +97,7 @@ var _ = BeforeSuite(func() {
})
Expect(err).NotTo(HaveOccurred())

err = (&RayCluster{}).SetupWebhookWithManager(mgr)
err = SetupRayClusterWebhookWithManager(mgr)
Expect(err).NotTo(HaveOccurred())

//+kubebuilder:scaffold:webhook
Expand Down Expand Up @@ -126,21 +128,21 @@ var _ = BeforeSuite(func() {
var _ = Describe("RayCluster validating webhook", func() {
Context("when name is invalid", func() {
It("should return error", func() {
rayCluster := RayCluster{
rayCluster := rayv1.RayCluster{
ObjectMeta: metav1.ObjectMeta{
Namespace: "default",
Name: "invalid.name",
},
Spec: RayClusterSpec{
HeadGroupSpec: HeadGroupSpec{
Spec: rayv1.RayClusterSpec{
HeadGroupSpec: rayv1.HeadGroupSpec{
RayStartParams: map[string]string{"DEADBEEF": "DEADBEEF"},
Template: corev1.PodTemplateSpec{
Spec: corev1.PodSpec{
Containers: []corev1.Container{},
},
},
},
WorkerGroupSpecs: []WorkerGroupSpec{},
WorkerGroupSpecs: []rayv1.WorkerGroupSpec{},
},
}

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

Context("when groupNames are not unique", func() {
var name, namespace string
var rayCluster RayCluster
var rayCluster rayv1.RayCluster

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

It("should return error", func() {
rayCluster = RayCluster{
rayCluster = rayv1.RayCluster{
ObjectMeta: metav1.ObjectMeta{
Name: name,
Namespace: namespace,
},
Spec: RayClusterSpec{
HeadGroupSpec: HeadGroupSpec{
Spec: rayv1.RayClusterSpec{
HeadGroupSpec: rayv1.HeadGroupSpec{
RayStartParams: map[string]string{"DEADBEEF": "DEADBEEF"},
Template: corev1.PodTemplateSpec{
Spec: corev1.PodSpec{
Containers: []corev1.Container{},
},
},
},
WorkerGroupSpecs: []WorkerGroupSpec{
WorkerGroupSpecs: []rayv1.WorkerGroupSpec{
{
GroupName: "group1",
RayStartParams: map[string]string{"DEADBEEF": "DEADBEEF"},
Expand Down

0 comments on commit 7a8b212

Please sign in to comment.