diff --git a/PROJECT b/PROJECT index cb6903086..0fc5562e2 100644 --- a/PROJECT +++ b/PROJECT @@ -12,8 +12,12 @@ projectName: codeflare-operator repo: github.com/project-codeflare/codeflare-operator resources: - controller: true - domain: codeflare.dev + domain: ray.io group: ray kind: RayCluster + path: github.com/project-codeflare/codeflare-operator/pkg/controllers version: v1 + webhooks: + defaulting: true + webhookVersion: v1 version: "3" diff --git a/config/default/kustomization.yaml b/config/default/kustomization.yaml index 6e926aed2..42fa71796 100644 --- a/config/default/kustomization.yaml +++ b/config/default/kustomization.yaml @@ -14,10 +14,10 @@ commonLabels: app.kubernetes.io/part-of: codeflare bases: -- ../rbac -- ../manager + - ../rbac + - ../manager # [PROMETHEUS] To enable prometheus monitor, uncomment all sections with 'PROMETHEUS'. # - ../prometheus resources: -- metrics_service.yaml + - metrics_service.yaml diff --git a/config/openshift/kustomization.yaml b/config/openshift/kustomization.yaml new file mode 100644 index 000000000..22e168162 --- /dev/null +++ b/config/openshift/kustomization.yaml @@ -0,0 +1,22 @@ +# Adds namespace to all resources. +namespace: openshift-operators + +# Value of this field is prepended to the +# names of all resources, e.g. a deployment named +# "wordpress" becomes "alices-wordpress". +# Note that it should also match with the prefix (text before '-') of the namespace +# field above. +namePrefix: codeflare-operator- + +# Labels to add to all resources and selectors. +commonLabels: + app.kubernetes.io/name: codeflare-operator + app.kubernetes.io/part-of: codeflare + +bases: + - ../default + - ../webhook + +patches: +- path: manager_webhook_patch.yaml +- path: webhookcainjection_patch.yaml diff --git a/config/openshift/manager_webhook_patch.yaml b/config/openshift/manager_webhook_patch.yaml new file mode 100644 index 000000000..5d6b541dc --- /dev/null +++ b/config/openshift/manager_webhook_patch.yaml @@ -0,0 +1,23 @@ +apiVersion: apps/v1 +kind: Deployment +metadata: + name: manager + namespace: system +spec: + template: + spec: + containers: + - name: manager + ports: + - containerPort: 9443 + name: webhook-server + protocol: TCP + volumeMounts: + - mountPath: /tmp/k8s-webhook-server/serving-certs + name: cert + readOnly: true + volumes: + - name: cert + secret: + defaultMode: 420 + secretName: codeflare-operator-raycluster-webhook-cert diff --git a/config/openshift/webhookcainjection_patch.yaml b/config/openshift/webhookcainjection_patch.yaml new file mode 100644 index 000000000..39db86347 --- /dev/null +++ b/config/openshift/webhookcainjection_patch.yaml @@ -0,0 +1,7 @@ +# This patch add annotation to admission webhook config +apiVersion: admissionregistration.k8s.io/v1 +kind: MutatingWebhookConfiguration +metadata: + name: mutating-webhook-configuration + annotations: + service.beta.openshift.io/inject-cabundle: "true" diff --git a/config/webhook/kustomization.yaml b/config/webhook/kustomization.yaml new file mode 100644 index 000000000..9cf26134e --- /dev/null +++ b/config/webhook/kustomization.yaml @@ -0,0 +1,6 @@ +resources: +- manifests.yaml +- service.yaml + +configurations: +- kustomizeconfig.yaml diff --git a/config/webhook/kustomizeconfig.yaml b/config/webhook/kustomizeconfig.yaml new file mode 100644 index 000000000..25e21e3c9 --- /dev/null +++ b/config/webhook/kustomizeconfig.yaml @@ -0,0 +1,25 @@ +# the following config is for teaching kustomize where to look at when substituting vars. +# It requires kustomize v2.1.0 or newer to work properly. +nameReference: +- kind: Service + version: v1 + fieldSpecs: + - kind: MutatingWebhookConfiguration + group: admissionregistration.k8s.io + path: webhooks/clientConfig/service/name + - kind: ValidatingWebhookConfiguration + group: admissionregistration.k8s.io + path: webhooks/clientConfig/service/name + +namespace: +- kind: MutatingWebhookConfiguration + group: admissionregistration.k8s.io + path: webhooks/clientConfig/service/namespace + create: true +- kind: ValidatingWebhookConfiguration + group: admissionregistration.k8s.io + path: webhooks/clientConfig/service/namespace + create: true + +varReference: +- path: metadata/annotations diff --git a/config/webhook/manifests.yaml b/config/webhook/manifests.yaml new file mode 100644 index 000000000..faf91696b --- /dev/null +++ b/config/webhook/manifests.yaml @@ -0,0 +1,27 @@ +--- +apiVersion: admissionregistration.k8s.io/v1 +kind: MutatingWebhookConfiguration +metadata: + creationTimestamp: null + name: mutating-webhook-configuration +webhooks: +- admissionReviewVersions: + - v1 + clientConfig: + service: + name: webhook-service + namespace: system + path: /mutate-ray-io-v1-raycluster + failurePolicy: Fail + name: mraycluster.kb.io + rules: + - apiGroups: + - ray.io + apiVersions: + - v1 + operations: + - CREATE + - UPDATE + resources: + - rayclusters + sideEffects: None diff --git a/config/webhook/service.yaml b/config/webhook/service.yaml new file mode 100644 index 000000000..ada0e3281 --- /dev/null +++ b/config/webhook/service.yaml @@ -0,0 +1,15 @@ +apiVersion: v1 +kind: Service +metadata: + name: webhook-service + namespace: openshift-operators + annotations: + service.beta.openshift.io/serving-cert-secret-name: codeflare-operator-raycluster-webhook-cert +spec: + ports: + - port: 443 + protocol: TCP + targetPort: 9443 + selector: + app.kubernetes.io/part-of: codeflare + app.kubernetes.io/name: codeflare-operator diff --git a/main.go b/main.go index 09ebe2599..7eff5fc40 100644 --- a/main.go +++ b/main.go @@ -147,9 +147,21 @@ func main() { }) exitOnError(err, "unable to start manager") - ok, err := HasAPIResourceForGVK(kubeClient.DiscoveryClient, rayv1.GroupVersion.WithKind("RayCluster")) + OpenShift := isOpenShift(ctx, kubeClient.DiscoveryClient) + + if OpenShift { + // TODO: setup the RayCluster webhook on vanilla Kubernetes + exitOnError(controllers.SetupRayClusterWebhookWithManager(mgr, cfg.KubeRay), "error setting up RayCluster webhook") + } + + ok, err := hasAPIResourceForGVK(kubeClient.DiscoveryClient, rayv1.GroupVersion.WithKind("RayCluster")) if ok { - rayClusterController := controllers.RayClusterReconciler{Client: mgr.GetClient(), Scheme: mgr.GetScheme(), Config: cfg.KubeRay} + rayClusterController := controllers.RayClusterReconciler{ + Client: mgr.GetClient(), + Scheme: mgr.GetScheme(), + Config: cfg.KubeRay, + IsOpenShift: OpenShift, + } exitOnError(rayClusterController.SetupWithManager(mgr), "Error setting up RayCluster controller") } else if err != nil { exitOnError(err, "Could not determine if RayCluster CR present on cluster.") @@ -205,7 +217,7 @@ func createConfigMap(ctx context.Context, client kubernetes.Interface, ns, name return err } -func HasAPIResourceForGVK(dc discovery.DiscoveryInterface, gvk schema.GroupVersionKind) (bool, error) { +func hasAPIResourceForGVK(dc discovery.DiscoveryInterface, gvk schema.GroupVersionKind) (bool, error) { gv, kind := gvk.ToAPIVersionAndKind() if resources, err := dc.ServerResourcesForGroupVersion(gv); err != nil { if apierrors.IsNotFound(err) { @@ -245,3 +257,20 @@ func exitOnError(err error, msg string) { os.Exit(1) } } + +func isOpenShift(ctx context.Context, dc discovery.DiscoveryInterface) bool { + logger := ctrl.LoggerFrom(ctx) + apiGroupList, err := dc.ServerGroups() + if err != nil { + logger.Info("Error while querying ServerGroups, assuming we're on Vanilla Kubernetes") + return false + } + for i := 0; i < len(apiGroupList.Groups); i++ { + if strings.HasSuffix(apiGroupList.Groups[i].Name, ".openshift.io") { + logger.Info("We detected being on OpenShift!") + return true + } + } + logger.Info("We detected being on Vanilla Kubernetes!") + return false +} diff --git a/pkg/controllers/raycluster_controller.go b/pkg/controllers/raycluster_controller.go index 551367e49..ba6ca84fe 100644 --- a/pkg/controllers/raycluster_controller.go +++ b/pkg/controllers/raycluster_controller.go @@ -21,6 +21,7 @@ import ( "crypto/rand" "crypto/sha1" "encoding/base64" + "fmt" rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1" @@ -48,13 +49,12 @@ import ( // RayClusterReconciler reconciles a RayCluster object type RayClusterReconciler struct { client.Client - kubeClient *kubernetes.Clientset - routeClient *routev1client.RouteV1Client - Scheme *runtime.Scheme - CookieSalt string - Config *config.KubeRayConfiguration - IsOpenShift bool - IsOpenShiftInitialized bool + kubeClient *kubernetes.Clientset + routeClient *routev1client.RouteV1Client + Scheme *runtime.Scheme + CookieSalt string + Config *config.KubeRayConfiguration + IsOpenShift bool } const ( @@ -97,36 +97,31 @@ var ( func (r *RayClusterReconciler) Reconcile(ctx context.Context, req ctrl.Request) (ctrl.Result, error) { logger := ctrl.LoggerFrom(ctx) - var cluster rayv1.RayCluster + cluster := &rayv1.RayCluster{} - if err := r.Get(ctx, req.NamespacedName, &cluster); err != nil { + if err := r.Get(ctx, req.NamespacedName, cluster); err != nil { if !errors.IsNotFound(err) { logger.Error(err, "Error getting RayCluster resource") } return ctrl.Result{}, client.IgnoreNotFound(err) } - if !r.IsOpenShiftInitialized { - r.IsOpenShift = isOpenShift(ctx, r.kubeClient, &cluster) - r.IsOpenShiftInitialized = true - } - if cluster.ObjectMeta.DeletionTimestamp.IsZero() { - if !controllerutil.ContainsFinalizer(&cluster, oAuthFinalizer) { + if !controllerutil.ContainsFinalizer(cluster, oAuthFinalizer) { logger.Info("Add a finalizer", "finalizer", oAuthFinalizer) - controllerutil.AddFinalizer(&cluster, oAuthFinalizer) - if err := r.Update(ctx, &cluster); err != nil { + controllerutil.AddFinalizer(cluster, oAuthFinalizer) + if err := r.Update(ctx, cluster); err != nil { // this log is info level since errors are not fatal and are expected logger.Info("WARN: Failed to update RayCluster with finalizer", "error", err.Error(), logRequeueing, true) return ctrl.Result{RequeueAfter: requeueTime}, err } } - } else if controllerutil.ContainsFinalizer(&cluster, oAuthFinalizer) { + } else if controllerutil.ContainsFinalizer(cluster, oAuthFinalizer) { err := client.IgnoreNotFound(r.Client.Delete( ctx, &rbacv1.ClusterRoleBinding{ ObjectMeta: metav1.ObjectMeta{ - Name: crbNameFromCluster(&cluster), + Name: crbNameFromCluster(cluster), }, }, &deleteOptions, @@ -135,8 +130,8 @@ func (r *RayClusterReconciler) Reconcile(ctx context.Context, req ctrl.Request) logger.Error(err, "Failed to remove OAuth ClusterRoleBinding.", logRequeueing, true) return ctrl.Result{RequeueAfter: requeueTime}, err } - controllerutil.RemoveFinalizer(&cluster, oAuthFinalizer) - if err := r.Update(ctx, &cluster); err != nil { + controllerutil.RemoveFinalizer(cluster, oAuthFinalizer) + if err := r.Update(ctx, cluster); err != nil { logger.Error(err, "Failed to remove finalizer from RayCluster", logRequeueing, true) return ctrl.Result{RequeueAfter: requeueTime}, err } @@ -144,66 +139,66 @@ func (r *RayClusterReconciler) Reconcile(ctx context.Context, req ctrl.Request) return ctrl.Result{}, nil } - if cluster.Status.State != "suspended" && r.isRayDashboardOAuthEnabled() && r.IsOpenShift { + if cluster.Status.State != "suspended" && isRayDashboardOAuthEnabled(r.Config) && r.IsOpenShift { logger.Info("Creating OAuth Objects") - _, err := r.routeClient.Routes(cluster.Namespace).Apply(ctx, desiredClusterRoute(&cluster), metav1.ApplyOptions{FieldManager: controllerName, Force: true}) + _, err := r.routeClient.Routes(cluster.Namespace).Apply(ctx, desiredClusterRoute(cluster), metav1.ApplyOptions{FieldManager: controllerName, Force: true}) if err != nil { logger.Error(err, "Failed to update OAuth Route") return ctrl.Result{RequeueAfter: requeueTime}, err } - _, err = r.kubeClient.CoreV1().Secrets(cluster.Namespace).Apply(ctx, desiredOAuthSecret(&cluster, r), metav1.ApplyOptions{FieldManager: controllerName, Force: true}) + _, err = r.kubeClient.CoreV1().Secrets(cluster.Namespace).Apply(ctx, desiredOAuthSecret(cluster, r), metav1.ApplyOptions{FieldManager: controllerName, Force: true}) if err != nil { logger.Error(err, "Failed to create OAuth Secret") return ctrl.Result{RequeueAfter: requeueTime}, err } - _, err = r.kubeClient.CoreV1().Services(cluster.Namespace).Apply(ctx, desiredOAuthService(&cluster), metav1.ApplyOptions{FieldManager: controllerName, Force: true}) + _, err = r.kubeClient.CoreV1().Services(cluster.Namespace).Apply(ctx, desiredOAuthService(cluster), metav1.ApplyOptions{FieldManager: controllerName, Force: true}) if err != nil { logger.Error(err, "Failed to update OAuth Service") return ctrl.Result{RequeueAfter: requeueTime}, err } - _, err = r.kubeClient.CoreV1().ServiceAccounts(cluster.Namespace).Apply(ctx, desiredServiceAccount(&cluster), metav1.ApplyOptions{FieldManager: controllerName, Force: true}) + _, err = r.kubeClient.CoreV1().ServiceAccounts(cluster.Namespace).Apply(ctx, desiredServiceAccount(cluster), metav1.ApplyOptions{FieldManager: controllerName, Force: true}) if err != nil { logger.Error(err, "Failed to update OAuth ServiceAccount") return ctrl.Result{RequeueAfter: requeueTime}, err } - _, err = r.kubeClient.RbacV1().ClusterRoleBindings().Apply(ctx, desiredOAuthClusterRoleBinding(&cluster), metav1.ApplyOptions{FieldManager: controllerName, Force: true}) + _, err = r.kubeClient.RbacV1().ClusterRoleBindings().Apply(ctx, desiredOAuthClusterRoleBinding(cluster), metav1.ApplyOptions{FieldManager: controllerName, Force: true}) if err != nil { logger.Error(err, "Failed to update OAuth ClusterRoleBinding") return ctrl.Result{RequeueAfter: requeueTime}, err } logger.Info("Creating RayClient Route") - _, err = r.routeClient.Routes(cluster.Namespace).Apply(ctx, desiredRayClientRoute(&cluster), metav1.ApplyOptions{FieldManager: controllerName, Force: true}) + _, err = r.routeClient.Routes(cluster.Namespace).Apply(ctx, desiredRayClientRoute(cluster), metav1.ApplyOptions{FieldManager: controllerName, Force: true}) if err != nil { logger.Error(err, "Failed to update RayClient Route") return ctrl.Result{RequeueAfter: requeueTime}, err } - } else if cluster.Status.State != "suspended" && !r.isRayDashboardOAuthEnabled() && !r.IsOpenShift { + } else if cluster.Status.State != "suspended" && !isRayDashboardOAuthEnabled(r.Config) && !r.IsOpenShift { logger.Info("We detected being on Vanilla Kubernetes!") logger.Info("Creating Dashboard Ingress") - dashboardName := dashboardNameFromCluster(&cluster) - dashboardIngressHost, err := r.getIngressHost(ctx, r.kubeClient, &cluster, dashboardName) + dashboardName := dashboardNameFromCluster(cluster) + dashboardIngressHost, err := getIngressHost(r.Config, cluster, dashboardName) if err != nil { return ctrl.Result{RequeueAfter: requeueTime}, err } - _, err = r.kubeClient.NetworkingV1().Ingresses(cluster.Namespace).Apply(ctx, desiredClusterIngress(&cluster, dashboardIngressHost), metav1.ApplyOptions{FieldManager: controllerName, Force: true}) + _, err = r.kubeClient.NetworkingV1().Ingresses(cluster.Namespace).Apply(ctx, desiredClusterIngress(cluster, dashboardIngressHost), metav1.ApplyOptions{FieldManager: controllerName, Force: true}) if err != nil { // This log is info level since errors are not fatal and are expected logger.Info("WARN: Failed to update Dashboard Ingress", "error", err.Error(), logRequeueing, true) return ctrl.Result{RequeueAfter: requeueTime}, err } logger.Info("Creating RayClient Ingress") - rayClientName := rayClientNameFromCluster(&cluster) - rayClientIngressHost, err := r.getIngressHost(ctx, r.kubeClient, &cluster, rayClientName) + rayClientName := rayClientNameFromCluster(cluster) + rayClientIngressHost, err := getIngressHost(r.Config, cluster, rayClientName) if err != nil { return ctrl.Result{RequeueAfter: requeueTime}, err } - _, err = r.kubeClient.NetworkingV1().Ingresses(cluster.Namespace).Apply(ctx, desiredRayClientIngress(&cluster, rayClientIngressHost), metav1.ApplyOptions{FieldManager: controllerName, Force: true}) + _, err = r.kubeClient.NetworkingV1().Ingresses(cluster.Namespace).Apply(ctx, desiredRayClientIngress(cluster, rayClientIngressHost), metav1.ApplyOptions{FieldManager: controllerName, Force: true}) if err != nil { logger.Error(err, "Failed to update RayClient Ingress") return ctrl.Result{RequeueAfter: requeueTime}, err @@ -213,6 +208,24 @@ func (r *RayClusterReconciler) Reconcile(ctx context.Context, req ctrl.Request) return ctrl.Result{}, nil } +// getIngressHost generates the cluster URL string based on the cluster type, RayCluster, and ingress domain. +func getIngressHost(cfg *config.KubeRayConfiguration, cluster *rayv1.RayCluster, ingressNameFromCluster string) (string, error) { + ingressDomain := "" + if cfg != nil && cfg.IngressDomain != "" { + ingressDomain = cfg.IngressDomain + } else { + return "", fmt.Errorf("missing IngressDomain configuration in ConfigMap 'codeflare-operator-config'") + } + return fmt.Sprintf("%s-%s.%s", ingressNameFromCluster, cluster.Namespace, ingressDomain), nil +} + +func isRayDashboardOAuthEnabled(cfg *config.KubeRayConfiguration) bool { + if cfg != nil && cfg.RayDashboardOAuthEnabled != nil { + return *cfg.RayDashboardOAuthEnabled + } + return true +} + func crbNameFromCluster(cluster *rayv1.RayCluster) string { return cluster.Name + "-" + cluster.Namespace + "-auth" // NOTE: potential naming conflicts ie {name: foo, ns: bar-baz} and {name: foo-bar, ns: baz} } diff --git a/pkg/controllers/raycluster_webhook.go b/pkg/controllers/raycluster_webhook.go new file mode 100644 index 000000000..0cd635040 --- /dev/null +++ b/pkg/controllers/raycluster_webhook.go @@ -0,0 +1,134 @@ +/* +Copyright 2023. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package controllers + +import ( + "context" + + rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1" + + corev1 "k8s.io/api/core/v1" + "k8s.io/apimachinery/pkg/runtime" + "k8s.io/utils/pointer" + ctrl "sigs.k8s.io/controller-runtime" + logf "sigs.k8s.io/controller-runtime/pkg/log" + "sigs.k8s.io/controller-runtime/pkg/webhook" + + "github.com/project-codeflare/codeflare-operator/pkg/config" +) + +// log is for logging in this package. +var rayclusterlog = logf.Log.WithName("raycluster-resource") + +func SetupRayClusterWebhookWithManager(mgr ctrl.Manager, cfg *config.KubeRayConfiguration) error { + return ctrl.NewWebhookManagedBy(mgr). + For(&rayv1.RayCluster{}). + WithDefaulter(&rayClusterDefaulter{ + Config: cfg, + }). + Complete() +} + +// +kubebuilder:webhook:path=/mutate-ray-io-v1-raycluster,mutating=true,failurePolicy=fail,sideEffects=None,groups=ray.io,resources=rayclusters,verbs=create;update,versions=v1,name=mraycluster.kb.io,admissionReviewVersions=v1 + +type rayClusterDefaulter struct { + Config *config.KubeRayConfiguration +} + +var _ webhook.CustomDefaulter = &rayClusterDefaulter{} + +// Default implements webhook.Defaulter so a webhook will be registered for the type +func (r *rayClusterDefaulter) Default(ctx context.Context, obj runtime.Object) error { + raycluster := obj.(*rayv1.RayCluster) + + if !pointer.BoolDeref(r.Config.RayDashboardOAuthEnabled, true) { + return nil + } + + // Check and add OAuth proxy if it does not exist + for _, container := range raycluster.Spec.HeadGroupSpec.Template.Spec.Containers { + if container.Name == "oauth-proxy" { + rayclusterlog.V(2).Info("OAuth sidecar already exists, no patch needed") + return nil + } + } + + rayclusterlog.V(2).Info("Adding OAuth sidecar container") + // definition of the new container + newOAuthSidecar := corev1.Container{ + Name: "oauth-proxy", + Image: "registry.redhat.io/openshift4/ose-oauth-proxy@sha256:1ea6a01bf3e63cdcf125c6064cbd4a4a270deaf0f157b3eabb78f60556840366", + Ports: []corev1.ContainerPort{ + {ContainerPort: 8443, Name: "oauth-proxy"}, + }, + Args: []string{ + "--https-address=:8443", + "--provider=openshift", + "--openshift-service-account=" + raycluster.Name + "-oauth-proxy", + "--upstream=http://localhost:8265", + "--tls-cert=/etc/tls/private/tls.crt", + "--tls-key=/etc/tls/private/tls.key", + "--cookie-secret=$(COOKIE_SECRET)", + "--openshift-delegate-urls={\"/\":{\"resource\":\"pods\",\"namespace\":\"default\",\"verb\":\"get\"}}", + }, + VolumeMounts: []corev1.VolumeMount{ + { + Name: "proxy-tls-secret", + MountPath: "/etc/tls/private", + ReadOnly: true, + }, + }, + } + + // Adding the new OAuth sidecar container + raycluster.Spec.HeadGroupSpec.Template.Spec.Containers = append(raycluster.Spec.HeadGroupSpec.Template.Spec.Containers, newOAuthSidecar) + + cookieSecret := corev1.EnvVar{ + Name: "COOKIE_SECRET", + ValueFrom: &corev1.EnvVarSource{ + SecretKeyRef: &corev1.SecretKeySelector{ + LocalObjectReference: corev1.LocalObjectReference{ + Name: raycluster.Name + "-oauth-config", + }, + Key: "cookie_secret", + }, + }, + } + + raycluster.Spec.HeadGroupSpec.Template.Spec.Containers[0].Env = append( + raycluster.Spec.HeadGroupSpec.Template.Spec.Containers[0].Env, + cookieSecret, + ) + + tlsSecretVolume := corev1.Volume{ + Name: "proxy-tls-secret", + VolumeSource: corev1.VolumeSource{ + Secret: &corev1.SecretVolumeSource{ + SecretName: raycluster.Name + "-proxy-tls-secret", + }, + }, + } + + raycluster.Spec.HeadGroupSpec.Template.Spec.Volumes = append(raycluster.Spec.HeadGroupSpec.Template.Spec.Volumes, tlsSecretVolume) + + // Ensure the service account is set + if raycluster.Spec.HeadGroupSpec.Template.Spec.ServiceAccountName == "" { + raycluster.Spec.HeadGroupSpec.Template.Spec.ServiceAccountName = raycluster.Name + "-oauth-proxy" + } + + return nil +} diff --git a/pkg/controllers/support.go b/pkg/controllers/support.go index 344e6215f..673d59419 100644 --- a/pkg/controllers/support.go +++ b/pkg/controllers/support.go @@ -1,10 +1,6 @@ package controllers import ( - "context" - "fmt" - "strings" - rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1" networkingv1 "k8s.io/api/networking/v1" @@ -12,10 +8,6 @@ import ( "k8s.io/apimachinery/pkg/util/intstr" v1 "k8s.io/client-go/applyconfigurations/meta/v1" networkingv1ac "k8s.io/client-go/applyconfigurations/networking/v1" - "k8s.io/client-go/discovery" - "k8s.io/client-go/kubernetes" - "k8s.io/client-go/rest" - ctrl "sigs.k8s.io/controller-runtime" routeapply "github.com/openshift/client-go/route/applyconfigurations/route/v1" ) @@ -102,56 +94,3 @@ func desiredClusterIngress(cluster *rayv1.RayCluster, ingressHost string) *netwo ), ) } - -// getDiscoveryClient returns a discovery client for the current reconciler -func getDiscoveryClient(config *rest.Config) (*discovery.DiscoveryClient, error) { - return discovery.NewDiscoveryClientForConfig(config) -} - -// Check where we are running. We are trying to distinguish here whether -// this is vanilla kubernetes cluster or Openshift -func isOpenShift(ctx context.Context, clientset *kubernetes.Clientset, cluster *rayv1.RayCluster) bool { - // The discovery package is used to discover APIs supported by a Kubernetes API server. - logger := ctrl.LoggerFrom(ctx) - config, err := ctrl.GetConfig() - if err != nil && config == nil { - logger.Info("Cannot retrieve config, assuming we're on Vanilla Kubernetes") - return false - } - dclient, err := getDiscoveryClient(config) - if err != nil && dclient == nil { - logger.Info("Cannot retrieve a DiscoveryClient, assuming we're on Vanilla Kubernetes") - return false - } - apiGroupList, err := dclient.ServerGroups() - if err != nil { - logger.Info("Error while querying ServerGroups, assuming we're on Vanilla Kubernetes") - return false - } - for i := 0; i < len(apiGroupList.Groups); i++ { - if strings.HasSuffix(apiGroupList.Groups[i].Name, ".openshift.io") { - logger.Info("We detected being on OpenShift!") - return true - } - } - logger.Info("We detected being on Vanilla Kubernetes!") - return false -} - -// getIngressHost generates the cluster URL string based on the cluster type, RayCluster, and ingress domain. -func (r *RayClusterReconciler) getIngressHost(ctx context.Context, clientset *kubernetes.Clientset, cluster *rayv1.RayCluster, ingressNameFromCluster string) (string, error) { - ingressDomain := "" - if r.Config != nil && r.Config.IngressDomain != "" { - ingressDomain = r.Config.IngressDomain - } else { - return "", fmt.Errorf("missing IngressDomain configuration in ConfigMap 'codeflare-operator-config'") - } - return fmt.Sprintf("%s-%s.%s", ingressNameFromCluster, cluster.Namespace, ingressDomain), nil -} - -func (r *RayClusterReconciler) isRayDashboardOAuthEnabled() bool { - if r.Config != nil && r.Config.RayDashboardOAuthEnabled != nil { - return *r.Config.RayDashboardOAuthEnabled - } - return true -}