From 55106cab3b3565d4ebc74cecdbfcd7acbd2757cd Mon Sep 17 00:00:00 2001 From: brucez-anyscale Date: Wed, 1 Jun 2022 14:59:37 -0700 Subject: [PATCH 01/19] Implement RayService CRD and controller --- .../apis/ray/v1alpha1/rayservice_types.go | 67 +++- .../controllers/ray/rayservice_controller.go | 311 +++++++++++++++++- .../controllers/ray/utils/serve_httpclient.go | 111 +++++++ 3 files changed, 476 insertions(+), 13 deletions(-) create mode 100644 ray-operator/controllers/ray/utils/serve_httpclient.go diff --git a/ray-operator/apis/ray/v1alpha1/rayservice_types.go b/ray-operator/apis/ray/v1alpha1/rayservice_types.go index 587fe5ea4c..04285925d0 100644 --- a/ray-operator/apis/ray/v1alpha1/rayservice_types.go +++ b/ray-operator/apis/ray/v1alpha1/rayservice_types.go @@ -1,22 +1,85 @@ package v1alpha1 import ( + v1 "k8s.io/api/core/v1" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" ) // EDIT THIS FILE! THIS IS SCAFFOLDING FOR YOU TO OWN! // NOTE: json tags are required. Any new fields you add must have json tags for the fields to be serialized. +type ServiceStatus string + +const ( + FAIL_TO_GET_RAYSERVICE ServiceStatus = "fail_to_get_rayservice" + FAIL_TO_GET_OR_CREATE_RAYCLUSTER ServiceStatus = "fail_to_get_or_create_RAYCLUSTER" + WAIT_FOR_DASHBOARD ServiceStatus = "wait_for_dashboard" + FAIL_SERVE_DEPLOY ServiceStatus = "fail_serve_deploy" + FAIL_GET_SERVE_DEPLOYMENT_STATUS ServiceStatus = "fail_get_serve_deployment_status" + RUNNING ServiceStatus = "running" + RESTARTING ServiceStatus = "restarting" + FAIL_DELETE_RAYCLUSTER ServiceStatus = "fail_delete_raycluster" +) + +// ServeConfigSpec defines the desired state of RayService +type ServeConfigSpec struct { + Name string `json:"name"` + ImportPath string `json:"import_path"` + InitArgs []string `json:"init_args,omitempty"` + InitKwargs map[string]string `json:"init_kwargs,omitempty"` + NumReplicas *int32 `json:"num_replicas,omitempty"` + RoutePrefix string `json:"route_prefix,omitempty"` + MaxConcurrentQueries *int32 `json:"max_concurrent_queries,omitempty"` + UserConfig map[string]string `json:"user_config,omitempty"` + AutoscalingConfig map[string]string `json:"autoscaling_config,omitempty"` + GracefulShutdownWaitLoopS *float64 `json:"graceful_shutdown_wait_loop_s,omitempty"` + GracefulShutdownTimeoutS *float64 `json:"graceful_shutdown_timeout_s,omitempty"` + HealthCheckPeriodS *float64 `json:"health_check_period_s,omitempty"` + HealthCheckTimeoutS *float64 `json:"health_check_timeout_s,omitempty"` + RayActorOptions RayActorOptionSpec `json:"ray_actor_options,omitempty"` +} + +// RayActorOptionSpec defines the desired state of RayActor +type RayActorOptionSpec struct { + RuntimeEnv map[string][]string `json:"runtime_env,omitempty"` + NumCpus *float64 `json:"num_cpus,omitempty"` + NumGpus *float64 `json:"num_gpus,omitempty"` + Memory *int32 `json:"memory,omitempty"` + ObjectStoreMemory *int32 `json:"object_store_memory,omitempty"` + Resources map[string]string `json:"resources,omitempty"` + AcceleratorType string `json:"accelerator_type,omitempty"` +} + // RayServiceSpec defines the desired state of RayService type RayServiceSpec struct { - // INSERT ADDITIONAL SPEC FIELDS - desired state of cluster // Important: Run "make" to regenerate code after modifying this file + + HealthCheckProbe *v1.Probe `json:"healthCheckConfig,omitempty"` + ServeConfigSpecs []ServeConfigSpec `json:"serveConfigs,omitempty"` + RayClusterSpec RayClusterSpec `json:"rayClusterConfig,omitempty"` +} + +// ServeStatus defines the desired state of Serve Deployment +type ServeStatus struct { + Name string `json:"name,omitempty"` + Status string `json:"status,omitempty"` + Message string `json:"message,omitempty"` + // Keep track of how long the service is healthy. + // Update when Serve Deployment is healthy or first time convert to unhealthy from healthy. + HealthLastUpdateTime metav1.Time `json:"healthLastUpdateTime,omitempty"` +} + +// ServeStatuses defines the desired states of all Serve Deployments +type ServeStatuses struct { + Statuses []ServeStatus `json:"statuses,omitempty"` } // RayServiceStatus defines the observed state of RayService type RayServiceStatus struct { - // INSERT ADDITIONAL STATUS FIELD - define observed state of cluster // Important: Run "make" to regenerate code after modifying this file + ServiceStatus ServiceStatus `json:"serviceStatus,omitempty"` + ServeStatuses ServeStatuses `json:"serveStatuses,omitempty"` + RayClusterStatus RayClusterStatus `json:"rayClusterStatus,omitempty"` } //+kubebuilder:object:root=true diff --git a/ray-operator/controllers/ray/rayservice_controller.go b/ray-operator/controllers/ray/rayservice_controller.go index 4d11e07535..e22a2b3cf0 100644 --- a/ray-operator/controllers/ray/rayservice_controller.go +++ b/ray-operator/controllers/ray/rayservice_controller.go @@ -2,8 +2,19 @@ package ray import ( "context" + "fmt" + "github.com/ray-project/kuberay/ray-operator/controllers/ray/common" + "github.com/ray-project/kuberay/ray-operator/controllers/ray/utils" + corev1 "k8s.io/api/core/v1" + "k8s.io/apimachinery/pkg/api/errors" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/types" + "math/rand" + "reflect" + "time" "github.com/go-logr/logr" + fmtErrors "github.com/pkg/errors" "k8s.io/client-go/tools/record" "sigs.k8s.io/controller-runtime/pkg/manager" @@ -15,21 +26,30 @@ import ( rayv1alpha1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1alpha1" ) +var ( + rayServiceLog = logf.Log.WithName("rayservice-controller") + RayServiceDefaultRequeueDuration = 2 * time.Second + RayServiceRestartRequeueDuration = 10 * time.Second + rayClusterSuffix = "-raycluster" +) + // RayServiceReconciler reconciles a RayService object type RayServiceReconciler struct { client.Client - Scheme *runtime.Scheme - Log logr.Logger - Recorder record.EventRecorder + Scheme *runtime.Scheme + Log logr.Logger + Recorder record.EventRecorder + ServeDeploymentConfigMap map[types.NamespacedName]rayv1alpha1.RayServiceSpec } // NewRayServiceReconciler returns a new reconcile.Reconciler func NewRayServiceReconciler(mgr manager.Manager) *RayServiceReconciler { return &RayServiceReconciler{ - Client: mgr.GetClient(), - Scheme: mgr.GetScheme(), - Log: ctrl.Log.WithName("controllers").WithName("RayService"), - Recorder: mgr.GetEventRecorderFor("rayservice-controller"), + Client: mgr.GetClient(), + Scheme: mgr.GetScheme(), + Log: ctrl.Log.WithName("controllers").WithName("RayService"), + Recorder: mgr.GetEventRecorderFor("rayservice-controller"), + ServeDeploymentConfigMap: make(map[types.NamespacedName]rayv1alpha1.RayServiceSpec), } } @@ -58,12 +78,76 @@ func NewRayServiceReconciler(mgr manager.Manager) *RayServiceReconciler { // // For more details, check Reconcile and its Result here: // - https://pkg.go.dev/sigs.k8s.io/controller-runtime@v0.11.2/pkg/reconcile -func (r *RayServiceReconciler) Reconcile(ctx context.Context, req ctrl.Request) (ctrl.Result, error) { - _ = logf.FromContext(ctx) +func (r *RayServiceReconciler) Reconcile(ctx context.Context, request ctrl.Request) (ctrl.Result, error) { + _ = r.Log.WithValues("rayservice", request.NamespacedName) + log.Info("reconciling RayService", "service NamespacedName", request.NamespacedName) + + // Get serving cluster instance + var err error + var rayServiceInstance *rayv1alpha1.RayService + + if rayServiceInstance, err = r.getRayServiceInstance(request); err != nil { + err = r.updateState(rayServiceInstance, rayv1alpha1.FAIL_TO_GET_RAYSERVICE, err) + return ctrl.Result{}, client.IgnoreNotFound(err) + } + + var rayClusterInstance *rayv1alpha1.RayCluster + if rayClusterInstance, err = r.getOrCreateRayClusterInstance(rayServiceInstance); err != nil { + err = r.updateState(rayServiceInstance, rayv1alpha1.FAIL_TO_GET_OR_CREATE_RAYCLUSTER, err) + return ctrl.Result{}, client.IgnoreNotFound(err) + } + + rayServiceInstance.Status.RayClusterStatus = rayClusterInstance.Status + + var clientURL string + if clientURL, err = r.fetchDashboardURL(rayClusterInstance); err != nil || clientURL == "" { + err = r.updateState(rayServiceInstance, rayv1alpha1.WAIT_FOR_DASHBOARD, err) + return ctrl.Result{}, err + } - // TODO(user): your logic here + rayDashboardClient := utils.RayDashboardClient{} + rayDashboardClient.InitClient(clientURL) + + shouldUpdate := r.checkIfNeedSubmitServeDeployment(rayServiceInstance, request) + + if shouldUpdate { + if err = r.updateServeDeployment(rayServiceInstance, rayDashboardClient, request); err != nil { + err = r.updateState(rayServiceInstance, rayv1alpha1.FAIL_SERVE_DEPLOY, err) + return ctrl.Result{}, err + } + } + + var isHealthy bool + if isHealthy, err = r.getAndCheckServeStatus(rayServiceInstance, rayDashboardClient); err != nil { + err = r.updateState(rayServiceInstance, rayv1alpha1.FAIL_GET_SERVE_DEPLOYMENT_STATUS, err) + return ctrl.Result{}, err + } + + if isHealthy { + rayServiceInstance.Status.ServiceStatus = rayv1alpha1.RUNNING + if err := r.Status().Update(context.Background(), rayServiceInstance); err != nil { + return ctrl.Result{}, err + } + } else { + rayServiceInstance.Status.ServiceStatus = rayv1alpha1.RESTARTING + if err := r.Status().Update(context.Background(), rayServiceInstance); err != nil { + return ctrl.Result{}, err + } + + delete(r.ServeDeploymentConfigMap, request.NamespacedName) + + // restart raycluster + if err := r.Delete(ctx, rayClusterInstance); err != nil { + err = r.updateState(rayServiceInstance, rayv1alpha1.FAIL_DELETE_RAYCLUSTER, err) + return ctrl.Result{}, err + } + + log.V(1).Info("Deleted rayCluster for rayService run", "rayCluster", rayClusterInstance) + // Wait a while for the cluster delete + return ctrl.Result{RequeueAfter: RayServiceRestartRequeueDuration}, nil + } - return ctrl.Result{}, nil + return ctrl.Result{RequeueAfter: RayServiceDefaultRequeueDuration}, nil } // SetupWithManager sets up the controller with the Manager. @@ -72,3 +156,208 @@ func (r *RayServiceReconciler) SetupWithManager(mgr ctrl.Manager) error { For(&rayv1alpha1.RayService{}). Complete(r) } + +func (r *RayServiceReconciler) getRayServiceInstance(request ctrl.Request) (*rayv1alpha1.RayService, error) { + rayServiceInstance := &rayv1alpha1.RayService{} + if err := r.Get(context.TODO(), request.NamespacedName, rayServiceInstance); err != nil { + if errors.IsNotFound(err) { + rayServiceLog.Info("Read request instance not found error!") + } else { + rayServiceLog.Error(err, "Read request instance error!") + } + // Error reading the object - requeue the request. + return nil, err + } + return rayServiceInstance, nil +} + +func (r *RayServiceReconciler) updateState(rayServiceInstance *rayv1alpha1.RayService, status rayv1alpha1.ServiceStatus, err error) error { + rayServiceInstance.Status.ServiceStatus = status + if errStatus := r.Status().Update(context.Background(), rayServiceInstance); errStatus != nil { + return fmt.Errorf("combined error: %v %v", err, errStatus) + } + return err +} + +func (r *RayServiceReconciler) getOrCreateRayClusterInstance(rayServiceInstance *rayv1alpha1.RayService) (*rayv1alpha1.RayCluster, error) { + // Update ray cluster + rayClusterNamespacedName := types.NamespacedName{ + Namespace: rayServiceInstance.Namespace, + Name: rayServiceInstance.Name + rayClusterSuffix, + } + + rayClusterInstance := &rayv1alpha1.RayCluster{} + err := r.Get(context.TODO(), rayClusterNamespacedName, rayClusterInstance) + + if err == nil { + rayClusterInstance.Spec = rayServiceInstance.Spec.RayClusterSpec + + r.Log.Info("Update ray cluster spec") + if err := r.Update(context.TODO(), rayClusterInstance); err != nil { + r.Log.Error(err, "Fail to update ray cluster instance!") + // Error updating the RayCluster object. + return nil, client.IgnoreNotFound(err) + } + } else if errors.IsNotFound(err) { + r.Log.Info("Not found rayCluster, creating rayCluster!") + rayClusterInstance, err = r.constructRayClusterForRayService(rayServiceInstance) + if err != nil { + r.Log.Error(err, "unable to construct rayCluster from spec") + // Error construct the RayCluster object - requeue the request. + return nil, err + } + if err := r.Create(context.TODO(), rayClusterInstance); err != nil { + r.Log.Error(err, "unable to create rayCluster for rayService", "rayCluster", rayClusterInstance) + // Error creating the RayCluster object - requeue the request. + return nil, err + } + r.Log.V(1).Info("created rayCluster for rayService run", "rayCluster", rayClusterInstance) + } else { + r.Log.Error(err, "Get request rayCluster instance error!") + // Error reading the RayCluster object - requeue the request. + return nil, client.IgnoreNotFound(err) + } + + return rayClusterInstance, nil +} + +func (r *RayServiceReconciler) constructRayClusterForRayService(rayService *rayv1alpha1.RayService) (*rayv1alpha1.RayCluster, error) { + name := fmt.Sprintf("%s%s", rayService.Name, rayClusterSuffix) + + rayCluster := &rayv1alpha1.RayCluster{ + ObjectMeta: metav1.ObjectMeta{ + Labels: rayService.Labels, + Annotations: rayService.Annotations, + Name: name, + Namespace: rayService.Namespace, + }, + Spec: *rayService.Spec.RayClusterSpec.DeepCopy(), + } + + // Set the ownership in order to do the garbage collection by k8s. + if err := ctrl.SetControllerReference(rayService, rayCluster, r.Scheme); err != nil { + return nil, err + } + + return rayCluster, nil +} + +func (r *RayServiceReconciler) fetchDashboardURL(rayCluster *rayv1alpha1.RayCluster) (string, error) { + headServices := corev1.ServiceList{} + filterLabels := client.MatchingLabels{common.RayClusterLabelKey: rayCluster.Name} + if err := r.List(context.TODO(), &headServices, client.InNamespace(rayCluster.Namespace), filterLabels); err != nil { + return "", err + } + + dashboardURL := "" + + if headServices.Items != nil { + if len(headServices.Items) == 1 { + r.Log.Info("reconcileServices ", "head service found", headServices.Items[0].Name) + // TODO: compare diff and reconcile the object. For example. ServiceType might be changed or port might be modified + servicePorts := headServices.Items[0].Spec.Ports + + dashboardPort := int32(-1) + + for _, servicePort := range servicePorts { + if servicePort.Name == "dashboard" { + dashboardPort = servicePort.Port + break + } + } + + if dashboardPort == int32(-1) { + return "", fmtErrors.Errorf("dashboard port not found") + } + + dashboardURL = fmt.Sprintf("%s.%s.svc.cluster.local:%v", + headServices.Items[0].Name, + headServices.Items[0].Namespace, + dashboardPort) + return dashboardURL, nil + } + + // This should never happen. + // We add the protection here just in case controller has race issue or user manually create service with same label. + if len(headServices.Items) > 1 { + r.Log.Info("reconcileServices ", "Duplicates head service found", len(headServices.Items)) + return "", fmtErrors.Errorf("Duplicates head service found %v", len(headServices.Items)) + } + } + + // Create head service if there's no existing one in the cluster. + if headServices.Items == nil || len(headServices.Items) == 0 { + return "", fmtErrors.Errorf("No head service found") + } + + return "", nil +} + +func (r *RayServiceReconciler) checkIfNeedSubmitServeDeployment(rayServiceInstance *rayv1alpha1.RayService, request ctrl.Request) bool { + existConfig, exist := r.ServeDeploymentConfigMap[request.NamespacedName] + + shouldUpdate := true + + if !exist || reflect.DeepEqual(existConfig, rayServiceInstance.Spec) || len(rayServiceInstance.Status.ServeStatuses.Statuses) != len(existConfig.ServeConfigSpecs) { + shouldUpdate = false + } + + log.V(1).Info("status check", "len(rayServiceInstance.Status.ServeStatuses.Statuses) ", len(rayServiceInstance.Status.ServeStatuses.Statuses), "len(existConfig.ServeConfigSpecs)", len(existConfig.ServeConfigSpecs)) + + return shouldUpdate +} + +func (r *RayServiceReconciler) updateServeDeployment(rayServiceInstance *rayv1alpha1.RayService, rayDashboardClient utils.RayDashboardClient, request ctrl.Request) error { + r.Log.Info("shouldUpdate") + if err := rayDashboardClient.UpdateDeployments(rayServiceInstance.Spec.ServeConfigSpecs); err != nil { + r.Log.Error(err, "fail to update deployment") + return err + } + + r.ServeDeploymentConfigMap[request.NamespacedName] = rayServiceInstance.Spec + return nil +} + +func (r *RayServiceReconciler) getAndCheckServeStatus(rayServiceInstance *rayv1alpha1.RayService, dashboardClient utils.RayDashboardClient) (bool, error) { + var serveStatuses *rayv1alpha1.ServeStatuses + var err error + if serveStatuses, err = dashboardClient.GetDeploymentsStatus(); err != nil { + r.Log.Error(err, "fail to get deployment status") + return false, err + } + + statusMap := make(map[string]rayv1alpha1.ServeStatus) + + for _, status := range rayServiceInstance.Status.ServeStatuses.Statuses { + statusMap[status.Name] = status + } + + isHealthy := true + for i := 0; i < len(serveStatuses.Statuses); i++ { + serveStatuses.Statuses[i].HealthLastUpdateTime = metav1.Now() + if serveStatuses.Statuses[i].Status != "HEALTHY" { + prevStatus, exist := statusMap[serveStatuses.Statuses[i].Name] + if exist { + if prevStatus.Status != "HEALTHY" { + serveStatuses.Statuses[i].HealthLastUpdateTime = prevStatus.HealthLastUpdateTime + + if time.Now().Sub(prevStatus.HealthLastUpdateTime.Time).Seconds() > 60 { + isHealthy = false + } + } + } + } + } + + rayServiceInstance.Status.ServeStatuses = *serveStatuses + + r.Log.Info("getAndCheckServeStatus ", "statusMap", statusMap, "serveStatuses", serveStatuses) + + rNum := rand.Intn(100) + if rNum < 10 { + r.Log.Info("getAndCheckServeStatus ", "rand", rNum) + isHealthy = false + } + + return isHealthy, nil +} diff --git a/ray-operator/controllers/ray/utils/serve_httpclient.go b/ray-operator/controllers/ray/utils/serve_httpclient.go new file mode 100644 index 0000000000..39b80727d9 --- /dev/null +++ b/ray-operator/controllers/ray/utils/serve_httpclient.go @@ -0,0 +1,111 @@ +package utils + +import ( + "bytes" + "fmt" + rayv1alpha1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1alpha1" + "io/ioutil" + "k8s.io/apimachinery/pkg/util/json" + "net/http" + "reflect" +) + +var ( + DEPLOY_PATH = "/api/serve/deployments/" + STATUS_PATH = "/api/serve/deployments/status" +) + +type ServingClusterDeployments struct { + Deployments []rayv1alpha1.ServeConfigSpec `json:"deployments,omitempty"` +} + +type RayDashboardClient struct { + client http.Client + dashboardURL string +} + +func (r *RayDashboardClient) InitClient(url string) { + r.client = http.Client{} + r.dashboardURL = "http://" + url +} + +func (r *RayDashboardClient) GetDeployments() (string, error) { + req, err := http.NewRequest("GET", r.dashboardURL+DEPLOY_PATH, nil) + if err != nil { + return "", err + } + + resp, err := r.client.Do(req) + + if err != nil { + return "", err + } + defer resp.Body.Close() + + body, _ := ioutil.ReadAll(resp.Body) + + return string(body), nil +} + +func (r *RayDashboardClient) UpdateDeployments(specs []rayv1alpha1.ServeConfigSpec) error { + + servingClusterDeployments := ServingClusterDeployments{ + Deployments: specs, + } + + deploymentJson, err := json.Marshal(servingClusterDeployments) + + var existDeploymentConfigJson string + if existDeploymentConfigJson, err = r.GetDeployments(); err != nil { + return err + } + existDeploymentConfig := ServingClusterDeployments{} + _ = json.Unmarshal([]byte(existDeploymentConfigJson), &existDeploymentConfig) + + if reflect.DeepEqual(existDeploymentConfig, servingClusterDeployments) { + return nil + } + + if err != nil { + return err + } + + req, err := http.NewRequest("PUT", r.dashboardURL+DEPLOY_PATH, bytes.NewBuffer(deploymentJson)) + if err != nil { + return err + } + req.Header.Set("Content-Type", "application/json") + + resp, err := r.client.Do(req) + + if err != nil { + return err + } + defer resp.Body.Close() + + return nil +} + +func (r *RayDashboardClient) GetDeploymentsStatus() (*rayv1alpha1.ServeStatuses, error) { + req, err := http.NewRequest("GET", r.dashboardURL+STATUS_PATH, nil) + if err != nil { + return nil, err + } + + resp, err := r.client.Do(req) + + if err != nil { + return nil, err + } + defer resp.Body.Close() + + fmt.Println("response Status:", resp.Status) + fmt.Println("response Headers:", resp.Header) + body, _ := ioutil.ReadAll(resp.Body) + fmt.Println("response Body:", string(body)) + + var serveStatuses rayv1alpha1.ServeStatuses + _ = json.Unmarshal(body, &serveStatuses) + + return &serveStatuses, nil +} From 680c33e2614f6cffb462425530932e8ee669be62 Mon Sep 17 00:00:00 2001 From: brucez-anyscale Date: Fri, 3 Jun 2022 18:23:32 -0700 Subject: [PATCH 02/19] Update config and controller --- ray-operator/Makefile | 4 +- .../config/crd/bases/ray.io_rayservices.yaml | 11686 ++++++++++++++++ .../config/manager/kustomization.yaml | 4 + ray-operator/config/rbac/role.yaml | 2 +- .../samples/ray_v1alpha1_rayservice.yaml | 91 + .../controllers/ray/raycluster_controller.go | 2 +- .../controllers/ray/rayservice_controller.go | 11 +- 7 files changed, 11792 insertions(+), 8 deletions(-) diff --git a/ray-operator/Makefile b/ray-operator/Makefile index 7ff551cb82..dc5ba07448 100644 --- a/ray-operator/Makefile +++ b/ray-operator/Makefile @@ -4,7 +4,7 @@ COMMIT_SHA1 := $(shell git rev-parse HEAD ) # Image URL to use all building/pushing image targets IMG ?= controller:latest # Produce CRDs that work back to Kubernetes 1.11 (no version conversion) -CRD_OPTIONS ?= "crd:maxDescLen=100,trivialVersions=true,preserveUnknownFields=false,generateEmbeddedObjectMeta=true" +CRD_OPTIONS ?= "crd:maxDescLen=100,trivialVersions=true,preserveUnknownFields=false,generateEmbeddedObjectMeta=true,allowDangerousTypes=true" # Get the currently used golang install path (in GOPATH/bin, unless GOBIN is set) ifeq (,$(shell go env GOBIN)) @@ -88,7 +88,7 @@ uninstall: manifests kustomize ## Uninstall CRDs from the K8s cluster specified $(KUSTOMIZE) build config/crd | kubectl delete -f - deploy: manifests kustomize ## Deploy controller to the K8s cluster specified in ~/.kube/config. - cd config/manager && $(KUSTOMIZE) edit set image controller=${IMG} + cd config/manager && $(KUSTOMIZE) edit set image kuberay/operator=${IMG} ($(KUSTOMIZE) build config/default | kubectl create -f -) || ($(KUSTOMIZE) build config/default | kubectl replace -f -) undeploy: ## Undeploy controller from the K8s cluster specified in ~/.kube/config. diff --git a/ray-operator/config/crd/bases/ray.io_rayservices.yaml b/ray-operator/config/crd/bases/ray.io_rayservices.yaml index 2a80b54f6b..1f417b3d55 100644 --- a/ray-operator/config/crd/bases/ray.io_rayservices.yaml +++ b/ray-operator/config/crd/bases/ray.io_rayservices.yaml @@ -33,9 +33,11695 @@ spec: type: object spec: description: RayServiceSpec defines the desired state of RayService + properties: + healthCheckConfig: + description: 'Probe describes a health check to be performed against + a container to determine whether it is alive ' + properties: + exec: + description: Exec specifies the action to take. + properties: + command: + description: 'Command is the command line to execute inside + the container, the working directory for the command ' + items: + type: string + type: array + type: object + failureThreshold: + description: Minimum consecutive failures for the probe to be + considered failed after having succeeded. + format: int32 + type: integer + grpc: + description: GRPC specifies an action involving a GRPC port. + properties: + port: + description: Port number of the gRPC service. Number must + be in the range 1 to 65535. + format: int32 + type: integer + service: + description: Service is the name of the service to place in + the gRPC HealthCheckRequest (see https://github. + type: string + required: + - port + type: object + httpGet: + description: HTTPGet specifies the http request to perform. + properties: + host: + description: Host name to connect to, defaults to the pod + IP. + type: string + httpHeaders: + description: Custom headers to set in the request. HTTP allows + repeated headers. + items: + description: HTTPHeader describes a custom header to be + used in HTTP probes + properties: + name: + description: The header field name + type: string + value: + description: The header field value + type: string + required: + - name + - value + type: object + type: array + path: + description: Path to access on the HTTP server. + type: string + port: + anyOf: + - type: integer + - type: string + description: Name or number of the port to access on the container. + Number must be in the range 1 to 65535. + x-kubernetes-int-or-string: true + scheme: + description: Scheme to use for connecting to the host. Defaults + to HTTP. + type: string + required: + - port + type: object + initialDelaySeconds: + description: Number of seconds after the container has started + before liveness probes are initiated. + format: int32 + type: integer + periodSeconds: + description: How often (in seconds) to perform the probe. Default + to 10 seconds. Minimum value is 1. + format: int32 + type: integer + successThreshold: + description: Minimum consecutive successes for the probe to be + considered successful after having failed. + format: int32 + type: integer + tcpSocket: + description: TCPSocket specifies an action involving a TCP port. + properties: + host: + description: 'Optional: Host name to connect to, defaults + to the pod IP.' + type: string + port: + anyOf: + - type: integer + - type: string + description: Number or name of the port to access on the container. + Number must be in the range 1 to 65535. + x-kubernetes-int-or-string: true + required: + - port + type: object + terminationGracePeriodSeconds: + description: Optional duration in seconds the pod needs to terminate + gracefully upon probe failure. + format: int64 + type: integer + timeoutSeconds: + description: Number of seconds after which the probe times out. + Defaults to 1 second. Minimum value is 1. + format: int32 + type: integer + type: object + rayClusterConfig: + description: 'EDIT THIS FILE! THIS IS SCAFFOLDING FOR YOU TO OWN! + NOTE: json tags are required.' + properties: + autoscalerOptions: + description: AutoscalerOptions specifies optional configuration + for the Ray autoscaler. + properties: + idleTimeoutSeconds: + description: IdleTimeoutSeconds is the number of seconds to + wait before scaling down a worker pod which is not us + format: int32 + type: integer + image: + description: Image optionally overrides the autoscaler's container + image. + type: string + resources: + description: Resources specifies resource requests and limits + for the autoscaler container. + properties: + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + description: 'Limits describes the maximum amount of compute + resources allowed. More info: https://kubernetes.' + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + description: Requests describes the minimum amount of + compute resources required. + type: object + type: object + upscalingMode: + description: UpscalineMode is "Default" or "Aggressive. + enum: + - Default + - Aggressive + type: string + type: object + enableInTreeAutoscaling: + description: EnableInTreeAutoscaling indicates whether operator + should create in tree autoscaling configs + type: boolean + headGroupSpec: + description: 'INSERT ADDITIONAL SPEC FIELDS - desired state of + cluster Important: Run "make" to regenerate code af' + properties: + enableIngress: + description: EnableIngress indicates whether operator should + create ingress object for head service or not. + type: boolean + rayStartParams: + additionalProperties: + type: string + description: 'RayStartParams are the params of the start command: + node-manager-port, object-store-memory, ...' + type: object + replicas: + description: Number of desired pods in this pod group. + format: int32 + type: integer + serviceType: + description: ServiceType is Kubernetes service type of the + head service. + type: string + template: + description: Template is the eaxct pod template used in K8s + depoyments, statefulsets, etc. + properties: + metadata: + description: 'Standard object''s metadata. More info: + https://git.k8s.' + properties: + annotations: + additionalProperties: + type: string + type: object + finalizers: + items: + type: string + type: array + labels: + additionalProperties: + type: string + type: object + name: + type: string + namespace: + type: string + type: object + spec: + description: 'Specification of the desired behavior of + the pod. More info: https://git.k8s.' + properties: + activeDeadlineSeconds: + description: Optional duration in seconds the pod + may be active on the node relative to StartTime + before the syst + format: int64 + type: integer + affinity: + description: If specified, the pod's scheduling constraints + properties: + nodeAffinity: + description: Describes node affinity scheduling + rules for the pod. + properties: + preferredDuringSchedulingIgnoredDuringExecution: + description: 'The scheduler will prefer to + schedule pods to nodes that satisfy the + affinity expressions specified ' + items: + description: An empty preferred scheduling + term matches all objects with implicit + weight 0 (i.e. it's a no-op). + properties: + preference: + description: A node selector term, associated + with the corresponding weight. + properties: + matchExpressions: + description: A list of node selector + requirements by node's labels. + items: + description: 'A node selector + requirement is a selector that + contains values, a key, and + an operator that relates ' + properties: + key: + description: The label key + that the selector applies + to. + type: string + operator: + description: Represents a + key's relationship to a + set of values. + type: string + values: + description: An array of string + values. If the operator + is In or NotIn, the values + array must be non-empty. + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchFields: + description: A list of node selector + requirements by node's fields. + items: + description: 'A node selector + requirement is a selector that + contains values, a key, and + an operator that relates ' + properties: + key: + description: The label key + that the selector applies + to. + type: string + operator: + description: Represents a + key's relationship to a + set of values. + type: string + values: + description: An array of string + values. If the operator + is In or NotIn, the values + array must be non-empty. + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + type: object + weight: + description: Weight associated with + matching the corresponding nodeSelectorTerm, + in the range 1-100. + format: int32 + type: integer + required: + - preference + - weight + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + description: If the affinity requirements + specified by this field are not met at scheduling + time, the pod will no + properties: + nodeSelectorTerms: + description: Required. A list of node + selector terms. The terms are ORed. + items: + description: A null or empty node selector + term matches no objects. The requirements + of them are ANDed. + properties: + matchExpressions: + description: A list of node selector + requirements by node's labels. + items: + description: 'A node selector + requirement is a selector that + contains values, a key, and + an operator that relates ' + properties: + key: + description: The label key + that the selector applies + to. + type: string + operator: + description: Represents a + key's relationship to a + set of values. + type: string + values: + description: An array of string + values. If the operator + is In or NotIn, the values + array must be non-empty. + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchFields: + description: A list of node selector + requirements by node's fields. + items: + description: 'A node selector + requirement is a selector that + contains values, a key, and + an operator that relates ' + properties: + key: + description: The label key + that the selector applies + to. + type: string + operator: + description: Represents a + key's relationship to a + set of values. + type: string + values: + description: An array of string + values. If the operator + is In or NotIn, the values + array must be non-empty. + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + type: object + type: array + required: + - nodeSelectorTerms + type: object + type: object + podAffinity: + description: Describes pod affinity scheduling + rules (e.g. co-locate this pod in the same node, + zone, etc. + properties: + preferredDuringSchedulingIgnoredDuringExecution: + description: 'The scheduler will prefer to + schedule pods to nodes that satisfy the + affinity expressions specified ' + items: + description: The weights of all of the matched + WeightedPodAffinityTerm fields are added + per-node to find the most + properties: + podAffinityTerm: + description: Required. A pod affinity + term, associated with the corresponding + weight. + properties: + labelSelector: + description: A label query over + a set of resources, in this case + pods. + properties: + matchExpressions: + description: matchExpressions + is a list of label selector + requirements. The requirements + are ANDed. + items: + description: A label selector + requirement is a selector + that contains values, a + key, and an operator that + relates + properties: + key: + description: key is the + label key that the selector + applies to. + type: string + operator: + description: operator + represents a key's relationship + to a set of values. + type: string + values: + description: values is + an array of string values. + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + description: matchLabels is + a map of {key,value} pairs. + type: object + type: object + namespaceSelector: + description: A label query over + the set of namespaces that the + term applies to. + properties: + matchExpressions: + description: matchExpressions + is a list of label selector + requirements. The requirements + are ANDed. + items: + description: A label selector + requirement is a selector + that contains values, a + key, and an operator that + relates + properties: + key: + description: key is the + label key that the selector + applies to. + type: string + operator: + description: operator + represents a key's relationship + to a set of values. + type: string + values: + description: values is + an array of string values. + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + description: matchLabels is + a map of {key,value} pairs. + type: object + type: object + namespaces: + description: namespaces specifies + a static list of namespace names + that the term applies to. + items: + type: string + type: array + topologyKey: + description: This pod should be + co-located (affinity) or not co-located + (anti-affinity) with the pods + matching th + type: string + required: + - topologyKey + type: object + weight: + description: weight associated with + matching the corresponding podAffinityTerm, + in the range 1-100. + format: int32 + type: integer + required: + - podAffinityTerm + - weight + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + description: If the affinity requirements + specified by this field are not met at scheduling + time, the pod will no + items: + description: Defines a set of pods (namely + those matching the labelSelector relative + to the given namespace(s)) t + properties: + labelSelector: + description: A label query over a set + of resources, in this case pods. + properties: + matchExpressions: + description: matchExpressions is + a list of label selector requirements. + The requirements are ANDed. + items: + description: A label selector + requirement is a selector that + contains values, a key, and + an operator that relates + properties: + key: + description: key is the label + key that the selector applies + to. + type: string + operator: + description: operator represents + a key's relationship to + a set of values. + type: string + values: + description: values is an + array of string values. + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + description: matchLabels is a map + of {key,value} pairs. + type: object + type: object + namespaceSelector: + description: A label query over the + set of namespaces that the term applies + to. + properties: + matchExpressions: + description: matchExpressions is + a list of label selector requirements. + The requirements are ANDed. + items: + description: A label selector + requirement is a selector that + contains values, a key, and + an operator that relates + properties: + key: + description: key is the label + key that the selector applies + to. + type: string + operator: + description: operator represents + a key's relationship to + a set of values. + type: string + values: + description: values is an + array of string values. + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + description: matchLabels is a map + of {key,value} pairs. + type: object + type: object + namespaces: + description: namespaces specifies a + static list of namespace names that + the term applies to. + items: + type: string + type: array + topologyKey: + description: This pod should be co-located + (affinity) or not co-located (anti-affinity) + with the pods matching th + type: string + required: + - topologyKey + type: object + type: array + type: object + podAntiAffinity: + description: Describes pod anti-affinity scheduling + rules (e.g. + properties: + preferredDuringSchedulingIgnoredDuringExecution: + description: The scheduler will prefer to + schedule pods to nodes that satisfy the + anti-affinity expressions speci + items: + description: The weights of all of the matched + WeightedPodAffinityTerm fields are added + per-node to find the most + properties: + podAffinityTerm: + description: Required. A pod affinity + term, associated with the corresponding + weight. + properties: + labelSelector: + description: A label query over + a set of resources, in this case + pods. + properties: + matchExpressions: + description: matchExpressions + is a list of label selector + requirements. The requirements + are ANDed. + items: + description: A label selector + requirement is a selector + that contains values, a + key, and an operator that + relates + properties: + key: + description: key is the + label key that the selector + applies to. + type: string + operator: + description: operator + represents a key's relationship + to a set of values. + type: string + values: + description: values is + an array of string values. + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + description: matchLabels is + a map of {key,value} pairs. + type: object + type: object + namespaceSelector: + description: A label query over + the set of namespaces that the + term applies to. + properties: + matchExpressions: + description: matchExpressions + is a list of label selector + requirements. The requirements + are ANDed. + items: + description: A label selector + requirement is a selector + that contains values, a + key, and an operator that + relates + properties: + key: + description: key is the + label key that the selector + applies to. + type: string + operator: + description: operator + represents a key's relationship + to a set of values. + type: string + values: + description: values is + an array of string values. + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + description: matchLabels is + a map of {key,value} pairs. + type: object + type: object + namespaces: + description: namespaces specifies + a static list of namespace names + that the term applies to. + items: + type: string + type: array + topologyKey: + description: This pod should be + co-located (affinity) or not co-located + (anti-affinity) with the pods + matching th + type: string + required: + - topologyKey + type: object + weight: + description: weight associated with + matching the corresponding podAffinityTerm, + in the range 1-100. + format: int32 + type: integer + required: + - podAffinityTerm + - weight + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + description: If the anti-affinity requirements + specified by this field are not met at scheduling + time, the pod wi + items: + description: Defines a set of pods (namely + those matching the labelSelector relative + to the given namespace(s)) t + properties: + labelSelector: + description: A label query over a set + of resources, in this case pods. + properties: + matchExpressions: + description: matchExpressions is + a list of label selector requirements. + The requirements are ANDed. + items: + description: A label selector + requirement is a selector that + contains values, a key, and + an operator that relates + properties: + key: + description: key is the label + key that the selector applies + to. + type: string + operator: + description: operator represents + a key's relationship to + a set of values. + type: string + values: + description: values is an + array of string values. + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + description: matchLabels is a map + of {key,value} pairs. + type: object + type: object + namespaceSelector: + description: A label query over the + set of namespaces that the term applies + to. + properties: + matchExpressions: + description: matchExpressions is + a list of label selector requirements. + The requirements are ANDed. + items: + description: A label selector + requirement is a selector that + contains values, a key, and + an operator that relates + properties: + key: + description: key is the label + key that the selector applies + to. + type: string + operator: + description: operator represents + a key's relationship to + a set of values. + type: string + values: + description: values is an + array of string values. + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + description: matchLabels is a map + of {key,value} pairs. + type: object + type: object + namespaces: + description: namespaces specifies a + static list of namespace names that + the term applies to. + items: + type: string + type: array + topologyKey: + description: This pod should be co-located + (affinity) or not co-located (anti-affinity) + with the pods matching th + type: string + required: + - topologyKey + type: object + type: array + type: object + type: object + automountServiceAccountToken: + description: AutomountServiceAccountToken indicates + whether a service account token should be automatically + mount + type: boolean + containers: + description: List of containers belonging to the pod. + Containers cannot currently be added or removed. + items: + description: A single application container that + you want to run within a pod. + properties: + args: + description: Arguments to the entrypoint. The + docker image's CMD is used if this is not + provided. + items: + type: string + type: array + command: + description: Entrypoint array. Not executed + within a shell. + items: + type: string + type: array + env: + description: List of environment variables to + set in the container. Cannot be updated. + items: + description: EnvVar represents an environment + variable present in a Container. + properties: + name: + description: Name of the environment variable. + Must be a C_IDENTIFIER. + type: string + value: + description: Variable references $(VAR_NAME) + are expanded using the previously defined + environment variables in t + type: string + valueFrom: + description: Source for the environment + variable's value. Cannot be used if + value is not empty. + properties: + configMapKeyRef: + description: Selects a key of a ConfigMap. + properties: + key: + description: The key to select. + type: string + name: + description: 'Name of the referent. + More info: https://kubernetes.' + type: string + optional: + description: Specify whether the + ConfigMap or its key must be + defined + type: boolean + required: + - key + type: object + fieldRef: + description: 'Selects a field of the + pod: supports metadata.name, metadata.namespace, + `metadata.' + properties: + apiVersion: + description: Version of the schema + the FieldPath is written in + terms of, defaults to "v1". + type: string + fieldPath: + description: Path of the field + to select in the specified API + version. + type: string + required: + - fieldPath + type: object + resourceFieldRef: + description: 'Selects a resource of + the container: only resources limits + and requests (limits.cpu, limits.' + properties: + containerName: + description: 'Container name: + required for volumes, optional + for env vars' + type: string + divisor: + anyOf: + - type: integer + - type: string + description: Specifies the output + format of the exposed resources, + defaults to "1" + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: + description: 'Required: resource + to select' + type: string + required: + - resource + type: object + secretKeyRef: + description: Selects a key of a secret + in the pod's namespace + properties: + key: + description: The key of the secret + to select from. Must be a valid + secret key. + type: string + name: + description: 'Name of the referent. + More info: https://kubernetes.' + type: string + optional: + description: Specify whether the + Secret or its key must be defined + type: boolean + required: + - key + type: object + type: object + required: + - name + type: object + type: array + envFrom: + description: List of sources to populate environment + variables in the container. + items: + description: EnvFromSource represents the + source of a set of ConfigMaps + properties: + configMapRef: + description: The ConfigMap to select from + properties: + name: + description: 'Name of the referent. + More info: https://kubernetes.' + type: string + optional: + description: Specify whether the ConfigMap + must be defined + type: boolean + type: object + prefix: + description: An optional identifier to + prepend to each key in the ConfigMap. + Must be a C_IDENTIFIER. + type: string + secretRef: + description: The Secret to select from + properties: + name: + description: 'Name of the referent. + More info: https://kubernetes.' + type: string + optional: + description: Specify whether the Secret + must be defined + type: boolean + type: object + type: object + type: array + image: + description: 'Docker image name. More info: + https://kubernetes.' + type: string + imagePullPolicy: + description: Image pull policy. One of Always, + Never, IfNotPresent. + type: string + lifecycle: + description: Actions that the management system + should take in response to container lifecycle + events. + properties: + postStart: + description: PostStart is called immediately + after a container is created. + properties: + exec: + description: Exec specifies the action + to take. + properties: + command: + description: 'Command is the command + line to execute inside the container, + the working directory for the + command ' + items: + type: string + type: array + type: object + httpGet: + description: HTTPGet specifies the http + request to perform. + properties: + host: + description: Host name to connect + to, defaults to the pod IP. + type: string + httpHeaders: + description: Custom headers to set + in the request. HTTP allows repeated + headers. + items: + description: HTTPHeader describes + a custom header to be used in + HTTP probes + properties: + name: + description: The header field + name + type: string + value: + description: The header field + value + type: string + required: + - name + - value + type: object + type: array + path: + description: Path to access on the + HTTP server. + type: string + port: + anyOf: + - type: integer + - type: string + description: Name or number of the + port to access on the container. + Number must be in the range 1 + to 65535. + x-kubernetes-int-or-string: true + scheme: + description: Scheme to use for connecting + to the host. Defaults to HTTP. + type: string + required: + - port + type: object + tcpSocket: + description: Deprecated. + properties: + host: + description: 'Optional: Host name + to connect to, defaults to the + pod IP.' + type: string + port: + anyOf: + - type: integer + - type: string + description: Number or name of the + port to access on the container. + Number must be in the range 1 + to 65535. + x-kubernetes-int-or-string: true + required: + - port + type: object + type: object + preStop: + description: PreStop is called immediately + before a container is terminated due to + an API request or management e + properties: + exec: + description: Exec specifies the action + to take. + properties: + command: + description: 'Command is the command + line to execute inside the container, + the working directory for the + command ' + items: + type: string + type: array + type: object + httpGet: + description: HTTPGet specifies the http + request to perform. + properties: + host: + description: Host name to connect + to, defaults to the pod IP. + type: string + httpHeaders: + description: Custom headers to set + in the request. HTTP allows repeated + headers. + items: + description: HTTPHeader describes + a custom header to be used in + HTTP probes + properties: + name: + description: The header field + name + type: string + value: + description: The header field + value + type: string + required: + - name + - value + type: object + type: array + path: + description: Path to access on the + HTTP server. + type: string + port: + anyOf: + - type: integer + - type: string + description: Name or number of the + port to access on the container. + Number must be in the range 1 + to 65535. + x-kubernetes-int-or-string: true + scheme: + description: Scheme to use for connecting + to the host. Defaults to HTTP. + type: string + required: + - port + type: object + tcpSocket: + description: Deprecated. + properties: + host: + description: 'Optional: Host name + to connect to, defaults to the + pod IP.' + type: string + port: + anyOf: + - type: integer + - type: string + description: Number or name of the + port to access on the container. + Number must be in the range 1 + to 65535. + x-kubernetes-int-or-string: true + required: + - port + type: object + type: object + type: object + livenessProbe: + description: Periodic probe of container liveness. + Container will be restarted if the probe fails. + properties: + exec: + description: Exec specifies the action to + take. + properties: + command: + description: 'Command is the command + line to execute inside the container, + the working directory for the command ' + items: + type: string + type: array + type: object + failureThreshold: + description: Minimum consecutive failures + for the probe to be considered failed + after having succeeded. + format: int32 + type: integer + grpc: + description: GRPC specifies an action involving + a GRPC port. + properties: + port: + description: Port number of the gRPC + service. Number must be in the range + 1 to 65535. + format: int32 + type: integer + service: + description: Service is the name of + the service to place in the gRPC HealthCheckRequest + (see https://github. + type: string + required: + - port + type: object + httpGet: + description: HTTPGet specifies the http + request to perform. + properties: + host: + description: Host name to connect to, + defaults to the pod IP. + type: string + httpHeaders: + description: Custom headers to set in + the request. HTTP allows repeated + headers. + items: + description: HTTPHeader describes + a custom header to be used in HTTP + probes + properties: + name: + description: The header field + name + type: string + value: + description: The header field + value + type: string + required: + - name + - value + type: object + type: array + path: + description: Path to access on the HTTP + server. + type: string + port: + anyOf: + - type: integer + - type: string + description: Name or number of the port + to access on the container. Number + must be in the range 1 to 65535. + x-kubernetes-int-or-string: true + scheme: + description: Scheme to use for connecting + to the host. Defaults to HTTP. + type: string + required: + - port + type: object + initialDelaySeconds: + description: Number of seconds after the + container has started before liveness + probes are initiated. + format: int32 + type: integer + periodSeconds: + description: How often (in seconds) to perform + the probe. Default to 10 seconds. Minimum + value is 1. + format: int32 + type: integer + successThreshold: + description: Minimum consecutive successes + for the probe to be considered successful + after having failed. + format: int32 + type: integer + tcpSocket: + description: TCPSocket specifies an action + involving a TCP port. + properties: + host: + description: 'Optional: Host name to + connect to, defaults to the pod IP.' + type: string + port: + anyOf: + - type: integer + - type: string + description: Number or name of the port + to access on the container. Number + must be in the range 1 to 65535. + x-kubernetes-int-or-string: true + required: + - port + type: object + terminationGracePeriodSeconds: + description: Optional duration in seconds + the pod needs to terminate gracefully + upon probe failure. + format: int64 + type: integer + timeoutSeconds: + description: Number of seconds after which + the probe times out. Defaults to 1 second. + Minimum value is 1. + format: int32 + type: integer + type: object + name: + description: Name of the container specified + as a DNS_LABEL. + type: string + ports: + description: List of ports to expose from the + container. + items: + description: ContainerPort represents a network + port in a single container. + properties: + containerPort: + description: Number of port to expose + on the pod's IP address. This must be + a valid port number, 0 < x < 65536. + format: int32 + type: integer + hostIP: + description: What host IP to bind the + external port to. + type: string + hostPort: + description: Number of port to expose + on the host. If specified, this must + be a valid port number, 0 < x < 65536. + format: int32 + type: integer + name: + description: If specified, this must be + an IANA_SVC_NAME and unique within the + pod. + type: string + protocol: + default: TCP + description: Protocol for port. Must be + UDP, TCP, or SCTP. Defaults to "TCP". + type: string + required: + - containerPort + type: object + type: array + x-kubernetes-list-map-keys: + - containerPort + - protocol + x-kubernetes-list-type: map + readinessProbe: + description: Periodic probe of container service + readiness. + properties: + exec: + description: Exec specifies the action to + take. + properties: + command: + description: 'Command is the command + line to execute inside the container, + the working directory for the command ' + items: + type: string + type: array + type: object + failureThreshold: + description: Minimum consecutive failures + for the probe to be considered failed + after having succeeded. + format: int32 + type: integer + grpc: + description: GRPC specifies an action involving + a GRPC port. + properties: + port: + description: Port number of the gRPC + service. Number must be in the range + 1 to 65535. + format: int32 + type: integer + service: + description: Service is the name of + the service to place in the gRPC HealthCheckRequest + (see https://github. + type: string + required: + - port + type: object + httpGet: + description: HTTPGet specifies the http + request to perform. + properties: + host: + description: Host name to connect to, + defaults to the pod IP. + type: string + httpHeaders: + description: Custom headers to set in + the request. HTTP allows repeated + headers. + items: + description: HTTPHeader describes + a custom header to be used in HTTP + probes + properties: + name: + description: The header field + name + type: string + value: + description: The header field + value + type: string + required: + - name + - value + type: object + type: array + path: + description: Path to access on the HTTP + server. + type: string + port: + anyOf: + - type: integer + - type: string + description: Name or number of the port + to access on the container. Number + must be in the range 1 to 65535. + x-kubernetes-int-or-string: true + scheme: + description: Scheme to use for connecting + to the host. Defaults to HTTP. + type: string + required: + - port + type: object + initialDelaySeconds: + description: Number of seconds after the + container has started before liveness + probes are initiated. + format: int32 + type: integer + periodSeconds: + description: How often (in seconds) to perform + the probe. Default to 10 seconds. Minimum + value is 1. + format: int32 + type: integer + successThreshold: + description: Minimum consecutive successes + for the probe to be considered successful + after having failed. + format: int32 + type: integer + tcpSocket: + description: TCPSocket specifies an action + involving a TCP port. + properties: + host: + description: 'Optional: Host name to + connect to, defaults to the pod IP.' + type: string + port: + anyOf: + - type: integer + - type: string + description: Number or name of the port + to access on the container. Number + must be in the range 1 to 65535. + x-kubernetes-int-or-string: true + required: + - port + type: object + terminationGracePeriodSeconds: + description: Optional duration in seconds + the pod needs to terminate gracefully + upon probe failure. + format: int64 + type: integer + timeoutSeconds: + description: Number of seconds after which + the probe times out. Defaults to 1 second. + Minimum value is 1. + format: int32 + type: integer + type: object + resources: + description: 'Compute Resources required by + this container. Cannot be updated. More info: + https://kubernetes.' + properties: + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + description: 'Limits describes the maximum + amount of compute resources allowed. More + info: https://kubernetes.' + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + description: Requests describes the minimum + amount of compute resources required. + type: object + type: object + securityContext: + description: SecurityContext defines the security + options the container should be run with. + properties: + allowPrivilegeEscalation: + description: AllowPrivilegeEscalation controls + whether a process can gain more privileges + than its parent process + type: boolean + capabilities: + description: The capabilities to add/drop + when running containers. + properties: + add: + description: Added capabilities + items: + description: Capability represent + POSIX capabilities type + type: string + type: array + drop: + description: Removed capabilities + items: + description: Capability represent + POSIX capabilities type + type: string + type: array + type: object + privileged: + description: Run container in privileged + mode. + type: boolean + procMount: + description: procMount denotes the type + of proc mount to use for the containers. + type: string + readOnlyRootFilesystem: + description: Whether this container has + a read-only root filesystem. Default is + false. + type: boolean + runAsGroup: + description: The GID to run the entrypoint + of the container process. Uses runtime + default if unset. + format: int64 + type: integer + runAsNonRoot: + description: Indicates that the container + must run as a non-root user. + type: boolean + runAsUser: + description: The UID to run the entrypoint + of the container process. + format: int64 + type: integer + seLinuxOptions: + description: The SELinux context to be applied + to the container. + properties: + level: + description: Level is SELinux level + label that applies to the container. + type: string + role: + description: Role is a SELinux role + label that applies to the container. + type: string + type: + description: Type is a SELinux type + label that applies to the container. + type: string + user: + description: User is a SELinux user + label that applies to the container. + type: string + type: object + seccompProfile: + description: The seccomp options to use + by this container. + properties: + localhostProfile: + description: localhostProfile indicates + a profile defined in a file on the + node should be used. + type: string + type: + description: type indicates which kind + of seccomp profile will be applied. + type: string + required: + - type + type: object + windowsOptions: + description: The Windows specific settings + applied to all containers. + properties: + gmsaCredentialSpec: + description: GMSACredentialSpec is where + the GMSA admission webhook (https://github. + type: string + gmsaCredentialSpecName: + description: GMSACredentialSpecName + is the name of the GMSA credential + spec to use. + type: string + hostProcess: + description: HostProcess determines + if a container should be run as a + 'Host Process' container. + type: boolean + runAsUserName: + description: The UserName in Windows + to run the entrypoint of the container + process. + type: string + type: object + type: object + startupProbe: + description: StartupProbe indicates that the + Pod has successfully initialized. + properties: + exec: + description: Exec specifies the action to + take. + properties: + command: + description: 'Command is the command + line to execute inside the container, + the working directory for the command ' + items: + type: string + type: array + type: object + failureThreshold: + description: Minimum consecutive failures + for the probe to be considered failed + after having succeeded. + format: int32 + type: integer + grpc: + description: GRPC specifies an action involving + a GRPC port. + properties: + port: + description: Port number of the gRPC + service. Number must be in the range + 1 to 65535. + format: int32 + type: integer + service: + description: Service is the name of + the service to place in the gRPC HealthCheckRequest + (see https://github. + type: string + required: + - port + type: object + httpGet: + description: HTTPGet specifies the http + request to perform. + properties: + host: + description: Host name to connect to, + defaults to the pod IP. + type: string + httpHeaders: + description: Custom headers to set in + the request. HTTP allows repeated + headers. + items: + description: HTTPHeader describes + a custom header to be used in HTTP + probes + properties: + name: + description: The header field + name + type: string + value: + description: The header field + value + type: string + required: + - name + - value + type: object + type: array + path: + description: Path to access on the HTTP + server. + type: string + port: + anyOf: + - type: integer + - type: string + description: Name or number of the port + to access on the container. Number + must be in the range 1 to 65535. + x-kubernetes-int-or-string: true + scheme: + description: Scheme to use for connecting + to the host. Defaults to HTTP. + type: string + required: + - port + type: object + initialDelaySeconds: + description: Number of seconds after the + container has started before liveness + probes are initiated. + format: int32 + type: integer + periodSeconds: + description: How often (in seconds) to perform + the probe. Default to 10 seconds. Minimum + value is 1. + format: int32 + type: integer + successThreshold: + description: Minimum consecutive successes + for the probe to be considered successful + after having failed. + format: int32 + type: integer + tcpSocket: + description: TCPSocket specifies an action + involving a TCP port. + properties: + host: + description: 'Optional: Host name to + connect to, defaults to the pod IP.' + type: string + port: + anyOf: + - type: integer + - type: string + description: Number or name of the port + to access on the container. Number + must be in the range 1 to 65535. + x-kubernetes-int-or-string: true + required: + - port + type: object + terminationGracePeriodSeconds: + description: Optional duration in seconds + the pod needs to terminate gracefully + upon probe failure. + format: int64 + type: integer + timeoutSeconds: + description: Number of seconds after which + the probe times out. Defaults to 1 second. + Minimum value is 1. + format: int32 + type: integer + type: object + stdin: + description: Whether this container should allocate + a buffer for stdin in the container runtime. + type: boolean + stdinOnce: + description: Whether the container runtime should + close the stdin channel after it has been + opened by a single at + type: boolean + terminationMessagePath: + description: 'Optional: Path at which the file + to which the container''s termination message + will be written is mou' + type: string + terminationMessagePolicy: + description: Indicate how the termination message + should be populated. + type: string + tty: + description: Whether this container should allocate + a TTY for itself, also requires 'stdin' to + be true. + type: boolean + volumeDevices: + description: volumeDevices is the list of block + devices to be used by the container. + items: + description: volumeDevice describes a mapping + of a raw block device within a container. + properties: + devicePath: + description: devicePath is the path inside + of the container that the device will + be mapped to. + type: string + name: + description: name must match the name + of a persistentVolumeClaim in the pod + type: string + required: + - devicePath + - name + type: object + type: array + volumeMounts: + description: Pod volumes to mount into the container's + filesystem. Cannot be updated. + items: + description: VolumeMount describes a mounting + of a Volume within a container. + properties: + mountPath: + description: Path within the container + at which the volume should be mounted. Must + not contain ':'. + type: string + mountPropagation: + description: mountPropagation determines + how mounts are propagated from the host + to container and the other way a + type: string + name: + description: This must match the Name + of a Volume. + type: string + readOnly: + description: Mounted read-only if true, + read-write otherwise (false or unspecified). + Defaults to false. + type: boolean + subPath: + description: Path within the volume from + which the container's volume should + be mounted. + type: string + subPathExpr: + description: Expanded path within the + volume from which the container's volume + should be mounted. + type: string + required: + - mountPath + - name + type: object + type: array + workingDir: + description: Container's working directory. + type: string + required: + - name + type: object + type: array + dnsConfig: + description: Specifies the DNS parameters of a pod. + properties: + nameservers: + description: A list of DNS name server IP addresses. + items: + type: string + type: array + options: + description: A list of DNS resolver options. This + will be merged with the base options generated + from DNSPolicy. + items: + description: PodDNSConfigOption defines DNS + resolver options of a pod. + properties: + name: + description: Required. + type: string + value: + type: string + type: object + type: array + searches: + description: A list of DNS search domains for + host-name lookup. + items: + type: string + type: array + type: object + dnsPolicy: + description: Set DNS policy for the pod. Defaults + to "ClusterFirst". + type: string + enableServiceLinks: + description: EnableServiceLinks indicates whether + information about services should be injected into + pod's enviro + type: boolean + ephemeralContainers: + description: List of ephemeral containers run in this + pod. + items: + description: An EphemeralContainer is a temporary + container that you may add to an existing Pod + for user-initiate + properties: + args: + description: Arguments to the entrypoint. The + docker image's CMD is used if this is not + provided. + items: + type: string + type: array + command: + description: Entrypoint array. Not executed + within a shell. + items: + type: string + type: array + env: + description: List of environment variables to + set in the container. Cannot be updated. + items: + description: EnvVar represents an environment + variable present in a Container. + properties: + name: + description: Name of the environment variable. + Must be a C_IDENTIFIER. + type: string + value: + description: Variable references $(VAR_NAME) + are expanded using the previously defined + environment variables in t + type: string + valueFrom: + description: Source for the environment + variable's value. Cannot be used if + value is not empty. + properties: + configMapKeyRef: + description: Selects a key of a ConfigMap. + properties: + key: + description: The key to select. + type: string + name: + description: 'Name of the referent. + More info: https://kubernetes.' + type: string + optional: + description: Specify whether the + ConfigMap or its key must be + defined + type: boolean + required: + - key + type: object + fieldRef: + description: 'Selects a field of the + pod: supports metadata.name, metadata.namespace, + `metadata.' + properties: + apiVersion: + description: Version of the schema + the FieldPath is written in + terms of, defaults to "v1". + type: string + fieldPath: + description: Path of the field + to select in the specified API + version. + type: string + required: + - fieldPath + type: object + resourceFieldRef: + description: 'Selects a resource of + the container: only resources limits + and requests (limits.cpu, limits.' + properties: + containerName: + description: 'Container name: + required for volumes, optional + for env vars' + type: string + divisor: + anyOf: + - type: integer + - type: string + description: Specifies the output + format of the exposed resources, + defaults to "1" + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: + description: 'Required: resource + to select' + type: string + required: + - resource + type: object + secretKeyRef: + description: Selects a key of a secret + in the pod's namespace + properties: + key: + description: The key of the secret + to select from. Must be a valid + secret key. + type: string + name: + description: 'Name of the referent. + More info: https://kubernetes.' + type: string + optional: + description: Specify whether the + Secret or its key must be defined + type: boolean + required: + - key + type: object + type: object + required: + - name + type: object + type: array + envFrom: + description: List of sources to populate environment + variables in the container. + items: + description: EnvFromSource represents the + source of a set of ConfigMaps + properties: + configMapRef: + description: The ConfigMap to select from + properties: + name: + description: 'Name of the referent. + More info: https://kubernetes.' + type: string + optional: + description: Specify whether the ConfigMap + must be defined + type: boolean + type: object + prefix: + description: An optional identifier to + prepend to each key in the ConfigMap. + Must be a C_IDENTIFIER. + type: string + secretRef: + description: The Secret to select from + properties: + name: + description: 'Name of the referent. + More info: https://kubernetes.' + type: string + optional: + description: Specify whether the Secret + must be defined + type: boolean + type: object + type: object + type: array + image: + description: 'Docker image name. More info: + https://kubernetes.io/docs/concepts/containers/images' + type: string + imagePullPolicy: + description: Image pull policy. One of Always, + Never, IfNotPresent. + type: string + lifecycle: + description: Lifecycle is not allowed for ephemeral + containers. + properties: + postStart: + description: PostStart is called immediately + after a container is created. + properties: + exec: + description: Exec specifies the action + to take. + properties: + command: + description: 'Command is the command + line to execute inside the container, + the working directory for the + command ' + items: + type: string + type: array + type: object + httpGet: + description: HTTPGet specifies the http + request to perform. + properties: + host: + description: Host name to connect + to, defaults to the pod IP. + type: string + httpHeaders: + description: Custom headers to set + in the request. HTTP allows repeated + headers. + items: + description: HTTPHeader describes + a custom header to be used in + HTTP probes + properties: + name: + description: The header field + name + type: string + value: + description: The header field + value + type: string + required: + - name + - value + type: object + type: array + path: + description: Path to access on the + HTTP server. + type: string + port: + anyOf: + - type: integer + - type: string + description: Name or number of the + port to access on the container. + Number must be in the range 1 + to 65535. + x-kubernetes-int-or-string: true + scheme: + description: Scheme to use for connecting + to the host. Defaults to HTTP. + type: string + required: + - port + type: object + tcpSocket: + description: Deprecated. + properties: + host: + description: 'Optional: Host name + to connect to, defaults to the + pod IP.' + type: string + port: + anyOf: + - type: integer + - type: string + description: Number or name of the + port to access on the container. + Number must be in the range 1 + to 65535. + x-kubernetes-int-or-string: true + required: + - port + type: object + type: object + preStop: + description: PreStop is called immediately + before a container is terminated due to + an API request or management e + properties: + exec: + description: Exec specifies the action + to take. + properties: + command: + description: 'Command is the command + line to execute inside the container, + the working directory for the + command ' + items: + type: string + type: array + type: object + httpGet: + description: HTTPGet specifies the http + request to perform. + properties: + host: + description: Host name to connect + to, defaults to the pod IP. + type: string + httpHeaders: + description: Custom headers to set + in the request. HTTP allows repeated + headers. + items: + description: HTTPHeader describes + a custom header to be used in + HTTP probes + properties: + name: + description: The header field + name + type: string + value: + description: The header field + value + type: string + required: + - name + - value + type: object + type: array + path: + description: Path to access on the + HTTP server. + type: string + port: + anyOf: + - type: integer + - type: string + description: Name or number of the + port to access on the container. + Number must be in the range 1 + to 65535. + x-kubernetes-int-or-string: true + scheme: + description: Scheme to use for connecting + to the host. Defaults to HTTP. + type: string + required: + - port + type: object + tcpSocket: + description: Deprecated. + properties: + host: + description: 'Optional: Host name + to connect to, defaults to the + pod IP.' + type: string + port: + anyOf: + - type: integer + - type: string + description: Number or name of the + port to access on the container. + Number must be in the range 1 + to 65535. + x-kubernetes-int-or-string: true + required: + - port + type: object + type: object + type: object + livenessProbe: + description: Probes are not allowed for ephemeral + containers. + properties: + exec: + description: Exec specifies the action to + take. + properties: + command: + description: 'Command is the command + line to execute inside the container, + the working directory for the command ' + items: + type: string + type: array + type: object + failureThreshold: + description: Minimum consecutive failures + for the probe to be considered failed + after having succeeded. + format: int32 + type: integer + grpc: + description: GRPC specifies an action involving + a GRPC port. + properties: + port: + description: Port number of the gRPC + service. Number must be in the range + 1 to 65535. + format: int32 + type: integer + service: + description: Service is the name of + the service to place in the gRPC HealthCheckRequest + (see https://github. + type: string + required: + - port + type: object + httpGet: + description: HTTPGet specifies the http + request to perform. + properties: + host: + description: Host name to connect to, + defaults to the pod IP. + type: string + httpHeaders: + description: Custom headers to set in + the request. HTTP allows repeated + headers. + items: + description: HTTPHeader describes + a custom header to be used in HTTP + probes + properties: + name: + description: The header field + name + type: string + value: + description: The header field + value + type: string + required: + - name + - value + type: object + type: array + path: + description: Path to access on the HTTP + server. + type: string + port: + anyOf: + - type: integer + - type: string + description: Name or number of the port + to access on the container. Number + must be in the range 1 to 65535. + x-kubernetes-int-or-string: true + scheme: + description: Scheme to use for connecting + to the host. Defaults to HTTP. + type: string + required: + - port + type: object + initialDelaySeconds: + description: Number of seconds after the + container has started before liveness + probes are initiated. + format: int32 + type: integer + periodSeconds: + description: How often (in seconds) to perform + the probe. Default to 10 seconds. Minimum + value is 1. + format: int32 + type: integer + successThreshold: + description: Minimum consecutive successes + for the probe to be considered successful + after having failed. + format: int32 + type: integer + tcpSocket: + description: TCPSocket specifies an action + involving a TCP port. + properties: + host: + description: 'Optional: Host name to + connect to, defaults to the pod IP.' + type: string + port: + anyOf: + - type: integer + - type: string + description: Number or name of the port + to access on the container. Number + must be in the range 1 to 65535. + x-kubernetes-int-or-string: true + required: + - port + type: object + terminationGracePeriodSeconds: + description: Optional duration in seconds + the pod needs to terminate gracefully + upon probe failure. + format: int64 + type: integer + timeoutSeconds: + description: Number of seconds after which + the probe times out. Defaults to 1 second. + Minimum value is 1. + format: int32 + type: integer + type: object + name: + description: Name of the ephemeral container + specified as a DNS_LABEL. + type: string + ports: + description: Ports are not allowed for ephemeral + containers. + items: + description: ContainerPort represents a network + port in a single container. + properties: + containerPort: + description: Number of port to expose + on the pod's IP address. This must be + a valid port number, 0 < x < 65536. + format: int32 + type: integer + hostIP: + description: What host IP to bind the + external port to. + type: string + hostPort: + description: Number of port to expose + on the host. If specified, this must + be a valid port number, 0 < x < 65536. + format: int32 + type: integer + name: + description: If specified, this must be + an IANA_SVC_NAME and unique within the + pod. + type: string + protocol: + default: TCP + description: Protocol for port. Must be + UDP, TCP, or SCTP. Defaults to "TCP". + type: string + required: + - containerPort + type: object + type: array + x-kubernetes-list-map-keys: + - containerPort + - protocol + x-kubernetes-list-type: map + readinessProbe: + description: Probes are not allowed for ephemeral + containers. + properties: + exec: + description: Exec specifies the action to + take. + properties: + command: + description: 'Command is the command + line to execute inside the container, + the working directory for the command ' + items: + type: string + type: array + type: object + failureThreshold: + description: Minimum consecutive failures + for the probe to be considered failed + after having succeeded. + format: int32 + type: integer + grpc: + description: GRPC specifies an action involving + a GRPC port. + properties: + port: + description: Port number of the gRPC + service. Number must be in the range + 1 to 65535. + format: int32 + type: integer + service: + description: Service is the name of + the service to place in the gRPC HealthCheckRequest + (see https://github. + type: string + required: + - port + type: object + httpGet: + description: HTTPGet specifies the http + request to perform. + properties: + host: + description: Host name to connect to, + defaults to the pod IP. + type: string + httpHeaders: + description: Custom headers to set in + the request. HTTP allows repeated + headers. + items: + description: HTTPHeader describes + a custom header to be used in HTTP + probes + properties: + name: + description: The header field + name + type: string + value: + description: The header field + value + type: string + required: + - name + - value + type: object + type: array + path: + description: Path to access on the HTTP + server. + type: string + port: + anyOf: + - type: integer + - type: string + description: Name or number of the port + to access on the container. Number + must be in the range 1 to 65535. + x-kubernetes-int-or-string: true + scheme: + description: Scheme to use for connecting + to the host. Defaults to HTTP. + type: string + required: + - port + type: object + initialDelaySeconds: + description: Number of seconds after the + container has started before liveness + probes are initiated. + format: int32 + type: integer + periodSeconds: + description: How often (in seconds) to perform + the probe. Default to 10 seconds. Minimum + value is 1. + format: int32 + type: integer + successThreshold: + description: Minimum consecutive successes + for the probe to be considered successful + after having failed. + format: int32 + type: integer + tcpSocket: + description: TCPSocket specifies an action + involving a TCP port. + properties: + host: + description: 'Optional: Host name to + connect to, defaults to the pod IP.' + type: string + port: + anyOf: + - type: integer + - type: string + description: Number or name of the port + to access on the container. Number + must be in the range 1 to 65535. + x-kubernetes-int-or-string: true + required: + - port + type: object + terminationGracePeriodSeconds: + description: Optional duration in seconds + the pod needs to terminate gracefully + upon probe failure. + format: int64 + type: integer + timeoutSeconds: + description: Number of seconds after which + the probe times out. Defaults to 1 second. + Minimum value is 1. + format: int32 + type: integer + type: object + resources: + description: Resources are not allowed for ephemeral + containers. + properties: + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + description: 'Limits describes the maximum + amount of compute resources allowed. More + info: https://kubernetes.' + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + description: Requests describes the minimum + amount of compute resources required. + type: object + type: object + securityContext: + description: 'Optional: SecurityContext defines + the security options the ephemeral container + should be run with.' + properties: + allowPrivilegeEscalation: + description: AllowPrivilegeEscalation controls + whether a process can gain more privileges + than its parent process + type: boolean + capabilities: + description: The capabilities to add/drop + when running containers. + properties: + add: + description: Added capabilities + items: + description: Capability represent + POSIX capabilities type + type: string + type: array + drop: + description: Removed capabilities + items: + description: Capability represent + POSIX capabilities type + type: string + type: array + type: object + privileged: + description: Run container in privileged + mode. + type: boolean + procMount: + description: procMount denotes the type + of proc mount to use for the containers. + type: string + readOnlyRootFilesystem: + description: Whether this container has + a read-only root filesystem. Default is + false. + type: boolean + runAsGroup: + description: The GID to run the entrypoint + of the container process. Uses runtime + default if unset. + format: int64 + type: integer + runAsNonRoot: + description: Indicates that the container + must run as a non-root user. + type: boolean + runAsUser: + description: The UID to run the entrypoint + of the container process. + format: int64 + type: integer + seLinuxOptions: + description: The SELinux context to be applied + to the container. + properties: + level: + description: Level is SELinux level + label that applies to the container. + type: string + role: + description: Role is a SELinux role + label that applies to the container. + type: string + type: + description: Type is a SELinux type + label that applies to the container. + type: string + user: + description: User is a SELinux user + label that applies to the container. + type: string + type: object + seccompProfile: + description: The seccomp options to use + by this container. + properties: + localhostProfile: + description: localhostProfile indicates + a profile defined in a file on the + node should be used. + type: string + type: + description: type indicates which kind + of seccomp profile will be applied. + type: string + required: + - type + type: object + windowsOptions: + description: The Windows specific settings + applied to all containers. + properties: + gmsaCredentialSpec: + description: GMSACredentialSpec is where + the GMSA admission webhook (https://github. + type: string + gmsaCredentialSpecName: + description: GMSACredentialSpecName + is the name of the GMSA credential + spec to use. + type: string + hostProcess: + description: HostProcess determines + if a container should be run as a + 'Host Process' container. + type: boolean + runAsUserName: + description: The UserName in Windows + to run the entrypoint of the container + process. + type: string + type: object + type: object + startupProbe: + description: Probes are not allowed for ephemeral + containers. + properties: + exec: + description: Exec specifies the action to + take. + properties: + command: + description: 'Command is the command + line to execute inside the container, + the working directory for the command ' + items: + type: string + type: array + type: object + failureThreshold: + description: Minimum consecutive failures + for the probe to be considered failed + after having succeeded. + format: int32 + type: integer + grpc: + description: GRPC specifies an action involving + a GRPC port. + properties: + port: + description: Port number of the gRPC + service. Number must be in the range + 1 to 65535. + format: int32 + type: integer + service: + description: Service is the name of + the service to place in the gRPC HealthCheckRequest + (see https://github. + type: string + required: + - port + type: object + httpGet: + description: HTTPGet specifies the http + request to perform. + properties: + host: + description: Host name to connect to, + defaults to the pod IP. + type: string + httpHeaders: + description: Custom headers to set in + the request. HTTP allows repeated + headers. + items: + description: HTTPHeader describes + a custom header to be used in HTTP + probes + properties: + name: + description: The header field + name + type: string + value: + description: The header field + value + type: string + required: + - name + - value + type: object + type: array + path: + description: Path to access on the HTTP + server. + type: string + port: + anyOf: + - type: integer + - type: string + description: Name or number of the port + to access on the container. Number + must be in the range 1 to 65535. + x-kubernetes-int-or-string: true + scheme: + description: Scheme to use for connecting + to the host. Defaults to HTTP. + type: string + required: + - port + type: object + initialDelaySeconds: + description: Number of seconds after the + container has started before liveness + probes are initiated. + format: int32 + type: integer + periodSeconds: + description: How often (in seconds) to perform + the probe. Default to 10 seconds. Minimum + value is 1. + format: int32 + type: integer + successThreshold: + description: Minimum consecutive successes + for the probe to be considered successful + after having failed. + format: int32 + type: integer + tcpSocket: + description: TCPSocket specifies an action + involving a TCP port. + properties: + host: + description: 'Optional: Host name to + connect to, defaults to the pod IP.' + type: string + port: + anyOf: + - type: integer + - type: string + description: Number or name of the port + to access on the container. Number + must be in the range 1 to 65535. + x-kubernetes-int-or-string: true + required: + - port + type: object + terminationGracePeriodSeconds: + description: Optional duration in seconds + the pod needs to terminate gracefully + upon probe failure. + format: int64 + type: integer + timeoutSeconds: + description: Number of seconds after which + the probe times out. Defaults to 1 second. + Minimum value is 1. + format: int32 + type: integer + type: object + stdin: + description: Whether this container should allocate + a buffer for stdin in the container runtime. + type: boolean + stdinOnce: + description: Whether the container runtime should + close the stdin channel after it has been + opened by a single at + type: boolean + targetContainerName: + description: If set, the name of the container + from PodSpec that this ephemeral container + targets. + type: string + terminationMessagePath: + description: 'Optional: Path at which the file + to which the container''s termination message + will be written is mou' + type: string + terminationMessagePolicy: + description: Indicate how the termination message + should be populated. + type: string + tty: + description: Whether this container should allocate + a TTY for itself, also requires 'stdin' to + be true. + type: boolean + volumeDevices: + description: volumeDevices is the list of block + devices to be used by the container. + items: + description: volumeDevice describes a mapping + of a raw block device within a container. + properties: + devicePath: + description: devicePath is the path inside + of the container that the device will + be mapped to. + type: string + name: + description: name must match the name + of a persistentVolumeClaim in the pod + type: string + required: + - devicePath + - name + type: object + type: array + volumeMounts: + description: Pod volumes to mount into the container's + filesystem. + items: + description: VolumeMount describes a mounting + of a Volume within a container. + properties: + mountPath: + description: Path within the container + at which the volume should be mounted. Must + not contain ':'. + type: string + mountPropagation: + description: mountPropagation determines + how mounts are propagated from the host + to container and the other way a + type: string + name: + description: This must match the Name + of a Volume. + type: string + readOnly: + description: Mounted read-only if true, + read-write otherwise (false or unspecified). + Defaults to false. + type: boolean + subPath: + description: Path within the volume from + which the container's volume should + be mounted. + type: string + subPathExpr: + description: Expanded path within the + volume from which the container's volume + should be mounted. + type: string + required: + - mountPath + - name + type: object + type: array + workingDir: + description: Container's working directory. + type: string + required: + - name + type: object + type: array + hostAliases: + description: 'HostAliases is an optional list of hosts + and IPs that will be injected into the pod''s hosts + file if ' + items: + description: 'HostAlias holds the mapping between + IP and hostnames that will be injected as an entry + in the pod''s ' + properties: + hostnames: + description: Hostnames for the above IP address. + items: + type: string + type: array + ip: + description: IP address of the host file entry. + type: string + type: object + type: array + hostIPC: + description: 'Use the host''s ipc namespace. Optional: + Default to false.' + type: boolean + hostNetwork: + description: Host networking requested for this pod. + Use the host's network namespace. + type: boolean + hostPID: + description: 'Use the host''s pid namespace. Optional: + Default to false.' + type: boolean + hostname: + description: Specifies the hostname of the Pod If + not specified, the pod's hostname will be set to + a system-defin + type: string + imagePullSecrets: + description: ImagePullSecrets is an optional list + of references to secrets in the same namespace to + use for pulli + items: + description: 'LocalObjectReference contains enough + information to let you locate the referenced object + inside the ' + properties: + name: + description: 'Name of the referent. More info: + https://kubernetes.' + type: string + type: object + type: array + initContainers: + description: List of initialization containers belonging + to the pod. + items: + description: A single application container that + you want to run within a pod. + properties: + args: + description: Arguments to the entrypoint. The + docker image's CMD is used if this is not + provided. + items: + type: string + type: array + command: + description: Entrypoint array. Not executed + within a shell. + items: + type: string + type: array + env: + description: List of environment variables to + set in the container. Cannot be updated. + items: + description: EnvVar represents an environment + variable present in a Container. + properties: + name: + description: Name of the environment variable. + Must be a C_IDENTIFIER. + type: string + value: + description: Variable references $(VAR_NAME) + are expanded using the previously defined + environment variables in t + type: string + valueFrom: + description: Source for the environment + variable's value. Cannot be used if + value is not empty. + properties: + configMapKeyRef: + description: Selects a key of a ConfigMap. + properties: + key: + description: The key to select. + type: string + name: + description: 'Name of the referent. + More info: https://kubernetes.' + type: string + optional: + description: Specify whether the + ConfigMap or its key must be + defined + type: boolean + required: + - key + type: object + fieldRef: + description: 'Selects a field of the + pod: supports metadata.name, metadata.namespace, + `metadata.' + properties: + apiVersion: + description: Version of the schema + the FieldPath is written in + terms of, defaults to "v1". + type: string + fieldPath: + description: Path of the field + to select in the specified API + version. + type: string + required: + - fieldPath + type: object + resourceFieldRef: + description: 'Selects a resource of + the container: only resources limits + and requests (limits.cpu, limits.' + properties: + containerName: + description: 'Container name: + required for volumes, optional + for env vars' + type: string + divisor: + anyOf: + - type: integer + - type: string + description: Specifies the output + format of the exposed resources, + defaults to "1" + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: + description: 'Required: resource + to select' + type: string + required: + - resource + type: object + secretKeyRef: + description: Selects a key of a secret + in the pod's namespace + properties: + key: + description: The key of the secret + to select from. Must be a valid + secret key. + type: string + name: + description: 'Name of the referent. + More info: https://kubernetes.' + type: string + optional: + description: Specify whether the + Secret or its key must be defined + type: boolean + required: + - key + type: object + type: object + required: + - name + type: object + type: array + envFrom: + description: List of sources to populate environment + variables in the container. + items: + description: EnvFromSource represents the + source of a set of ConfigMaps + properties: + configMapRef: + description: The ConfigMap to select from + properties: + name: + description: 'Name of the referent. + More info: https://kubernetes.' + type: string + optional: + description: Specify whether the ConfigMap + must be defined + type: boolean + type: object + prefix: + description: An optional identifier to + prepend to each key in the ConfigMap. + Must be a C_IDENTIFIER. + type: string + secretRef: + description: The Secret to select from + properties: + name: + description: 'Name of the referent. + More info: https://kubernetes.' + type: string + optional: + description: Specify whether the Secret + must be defined + type: boolean + type: object + type: object + type: array + image: + description: 'Docker image name. More info: + https://kubernetes.' + type: string + imagePullPolicy: + description: Image pull policy. One of Always, + Never, IfNotPresent. + type: string + lifecycle: + description: Actions that the management system + should take in response to container lifecycle + events. + properties: + postStart: + description: PostStart is called immediately + after a container is created. + properties: + exec: + description: Exec specifies the action + to take. + properties: + command: + description: 'Command is the command + line to execute inside the container, + the working directory for the + command ' + items: + type: string + type: array + type: object + httpGet: + description: HTTPGet specifies the http + request to perform. + properties: + host: + description: Host name to connect + to, defaults to the pod IP. + type: string + httpHeaders: + description: Custom headers to set + in the request. HTTP allows repeated + headers. + items: + description: HTTPHeader describes + a custom header to be used in + HTTP probes + properties: + name: + description: The header field + name + type: string + value: + description: The header field + value + type: string + required: + - name + - value + type: object + type: array + path: + description: Path to access on the + HTTP server. + type: string + port: + anyOf: + - type: integer + - type: string + description: Name or number of the + port to access on the container. + Number must be in the range 1 + to 65535. + x-kubernetes-int-or-string: true + scheme: + description: Scheme to use for connecting + to the host. Defaults to HTTP. + type: string + required: + - port + type: object + tcpSocket: + description: Deprecated. + properties: + host: + description: 'Optional: Host name + to connect to, defaults to the + pod IP.' + type: string + port: + anyOf: + - type: integer + - type: string + description: Number or name of the + port to access on the container. + Number must be in the range 1 + to 65535. + x-kubernetes-int-or-string: true + required: + - port + type: object + type: object + preStop: + description: PreStop is called immediately + before a container is terminated due to + an API request or management e + properties: + exec: + description: Exec specifies the action + to take. + properties: + command: + description: 'Command is the command + line to execute inside the container, + the working directory for the + command ' + items: + type: string + type: array + type: object + httpGet: + description: HTTPGet specifies the http + request to perform. + properties: + host: + description: Host name to connect + to, defaults to the pod IP. + type: string + httpHeaders: + description: Custom headers to set + in the request. HTTP allows repeated + headers. + items: + description: HTTPHeader describes + a custom header to be used in + HTTP probes + properties: + name: + description: The header field + name + type: string + value: + description: The header field + value + type: string + required: + - name + - value + type: object + type: array + path: + description: Path to access on the + HTTP server. + type: string + port: + anyOf: + - type: integer + - type: string + description: Name or number of the + port to access on the container. + Number must be in the range 1 + to 65535. + x-kubernetes-int-or-string: true + scheme: + description: Scheme to use for connecting + to the host. Defaults to HTTP. + type: string + required: + - port + type: object + tcpSocket: + description: Deprecated. + properties: + host: + description: 'Optional: Host name + to connect to, defaults to the + pod IP.' + type: string + port: + anyOf: + - type: integer + - type: string + description: Number or name of the + port to access on the container. + Number must be in the range 1 + to 65535. + x-kubernetes-int-or-string: true + required: + - port + type: object + type: object + type: object + livenessProbe: + description: Periodic probe of container liveness. + Container will be restarted if the probe fails. + properties: + exec: + description: Exec specifies the action to + take. + properties: + command: + description: 'Command is the command + line to execute inside the container, + the working directory for the command ' + items: + type: string + type: array + type: object + failureThreshold: + description: Minimum consecutive failures + for the probe to be considered failed + after having succeeded. + format: int32 + type: integer + grpc: + description: GRPC specifies an action involving + a GRPC port. + properties: + port: + description: Port number of the gRPC + service. Number must be in the range + 1 to 65535. + format: int32 + type: integer + service: + description: Service is the name of + the service to place in the gRPC HealthCheckRequest + (see https://github. + type: string + required: + - port + type: object + httpGet: + description: HTTPGet specifies the http + request to perform. + properties: + host: + description: Host name to connect to, + defaults to the pod IP. + type: string + httpHeaders: + description: Custom headers to set in + the request. HTTP allows repeated + headers. + items: + description: HTTPHeader describes + a custom header to be used in HTTP + probes + properties: + name: + description: The header field + name + type: string + value: + description: The header field + value + type: string + required: + - name + - value + type: object + type: array + path: + description: Path to access on the HTTP + server. + type: string + port: + anyOf: + - type: integer + - type: string + description: Name or number of the port + to access on the container. Number + must be in the range 1 to 65535. + x-kubernetes-int-or-string: true + scheme: + description: Scheme to use for connecting + to the host. Defaults to HTTP. + type: string + required: + - port + type: object + initialDelaySeconds: + description: Number of seconds after the + container has started before liveness + probes are initiated. + format: int32 + type: integer + periodSeconds: + description: How often (in seconds) to perform + the probe. Default to 10 seconds. Minimum + value is 1. + format: int32 + type: integer + successThreshold: + description: Minimum consecutive successes + for the probe to be considered successful + after having failed. + format: int32 + type: integer + tcpSocket: + description: TCPSocket specifies an action + involving a TCP port. + properties: + host: + description: 'Optional: Host name to + connect to, defaults to the pod IP.' + type: string + port: + anyOf: + - type: integer + - type: string + description: Number or name of the port + to access on the container. Number + must be in the range 1 to 65535. + x-kubernetes-int-or-string: true + required: + - port + type: object + terminationGracePeriodSeconds: + description: Optional duration in seconds + the pod needs to terminate gracefully + upon probe failure. + format: int64 + type: integer + timeoutSeconds: + description: Number of seconds after which + the probe times out. Defaults to 1 second. + Minimum value is 1. + format: int32 + type: integer + type: object + name: + description: Name of the container specified + as a DNS_LABEL. + type: string + ports: + description: List of ports to expose from the + container. + items: + description: ContainerPort represents a network + port in a single container. + properties: + containerPort: + description: Number of port to expose + on the pod's IP address. This must be + a valid port number, 0 < x < 65536. + format: int32 + type: integer + hostIP: + description: What host IP to bind the + external port to. + type: string + hostPort: + description: Number of port to expose + on the host. If specified, this must + be a valid port number, 0 < x < 65536. + format: int32 + type: integer + name: + description: If specified, this must be + an IANA_SVC_NAME and unique within the + pod. + type: string + protocol: + default: TCP + description: Protocol for port. Must be + UDP, TCP, or SCTP. Defaults to "TCP". + type: string + required: + - containerPort + type: object + type: array + x-kubernetes-list-map-keys: + - containerPort + - protocol + x-kubernetes-list-type: map + readinessProbe: + description: Periodic probe of container service + readiness. + properties: + exec: + description: Exec specifies the action to + take. + properties: + command: + description: 'Command is the command + line to execute inside the container, + the working directory for the command ' + items: + type: string + type: array + type: object + failureThreshold: + description: Minimum consecutive failures + for the probe to be considered failed + after having succeeded. + format: int32 + type: integer + grpc: + description: GRPC specifies an action involving + a GRPC port. + properties: + port: + description: Port number of the gRPC + service. Number must be in the range + 1 to 65535. + format: int32 + type: integer + service: + description: Service is the name of + the service to place in the gRPC HealthCheckRequest + (see https://github. + type: string + required: + - port + type: object + httpGet: + description: HTTPGet specifies the http + request to perform. + properties: + host: + description: Host name to connect to, + defaults to the pod IP. + type: string + httpHeaders: + description: Custom headers to set in + the request. HTTP allows repeated + headers. + items: + description: HTTPHeader describes + a custom header to be used in HTTP + probes + properties: + name: + description: The header field + name + type: string + value: + description: The header field + value + type: string + required: + - name + - value + type: object + type: array + path: + description: Path to access on the HTTP + server. + type: string + port: + anyOf: + - type: integer + - type: string + description: Name or number of the port + to access on the container. Number + must be in the range 1 to 65535. + x-kubernetes-int-or-string: true + scheme: + description: Scheme to use for connecting + to the host. Defaults to HTTP. + type: string + required: + - port + type: object + initialDelaySeconds: + description: Number of seconds after the + container has started before liveness + probes are initiated. + format: int32 + type: integer + periodSeconds: + description: How often (in seconds) to perform + the probe. Default to 10 seconds. Minimum + value is 1. + format: int32 + type: integer + successThreshold: + description: Minimum consecutive successes + for the probe to be considered successful + after having failed. + format: int32 + type: integer + tcpSocket: + description: TCPSocket specifies an action + involving a TCP port. + properties: + host: + description: 'Optional: Host name to + connect to, defaults to the pod IP.' + type: string + port: + anyOf: + - type: integer + - type: string + description: Number or name of the port + to access on the container. Number + must be in the range 1 to 65535. + x-kubernetes-int-or-string: true + required: + - port + type: object + terminationGracePeriodSeconds: + description: Optional duration in seconds + the pod needs to terminate gracefully + upon probe failure. + format: int64 + type: integer + timeoutSeconds: + description: Number of seconds after which + the probe times out. Defaults to 1 second. + Minimum value is 1. + format: int32 + type: integer + type: object + resources: + description: 'Compute Resources required by + this container. Cannot be updated. More info: + https://kubernetes.' + properties: + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + description: 'Limits describes the maximum + amount of compute resources allowed. More + info: https://kubernetes.' + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + description: Requests describes the minimum + amount of compute resources required. + type: object + type: object + securityContext: + description: SecurityContext defines the security + options the container should be run with. + properties: + allowPrivilegeEscalation: + description: AllowPrivilegeEscalation controls + whether a process can gain more privileges + than its parent process + type: boolean + capabilities: + description: The capabilities to add/drop + when running containers. + properties: + add: + description: Added capabilities + items: + description: Capability represent + POSIX capabilities type + type: string + type: array + drop: + description: Removed capabilities + items: + description: Capability represent + POSIX capabilities type + type: string + type: array + type: object + privileged: + description: Run container in privileged + mode. + type: boolean + procMount: + description: procMount denotes the type + of proc mount to use for the containers. + type: string + readOnlyRootFilesystem: + description: Whether this container has + a read-only root filesystem. Default is + false. + type: boolean + runAsGroup: + description: The GID to run the entrypoint + of the container process. Uses runtime + default if unset. + format: int64 + type: integer + runAsNonRoot: + description: Indicates that the container + must run as a non-root user. + type: boolean + runAsUser: + description: The UID to run the entrypoint + of the container process. + format: int64 + type: integer + seLinuxOptions: + description: The SELinux context to be applied + to the container. + properties: + level: + description: Level is SELinux level + label that applies to the container. + type: string + role: + description: Role is a SELinux role + label that applies to the container. + type: string + type: + description: Type is a SELinux type + label that applies to the container. + type: string + user: + description: User is a SELinux user + label that applies to the container. + type: string + type: object + seccompProfile: + description: The seccomp options to use + by this container. + properties: + localhostProfile: + description: localhostProfile indicates + a profile defined in a file on the + node should be used. + type: string + type: + description: type indicates which kind + of seccomp profile will be applied. + type: string + required: + - type + type: object + windowsOptions: + description: The Windows specific settings + applied to all containers. + properties: + gmsaCredentialSpec: + description: GMSACredentialSpec is where + the GMSA admission webhook (https://github. + type: string + gmsaCredentialSpecName: + description: GMSACredentialSpecName + is the name of the GMSA credential + spec to use. + type: string + hostProcess: + description: HostProcess determines + if a container should be run as a + 'Host Process' container. + type: boolean + runAsUserName: + description: The UserName in Windows + to run the entrypoint of the container + process. + type: string + type: object + type: object + startupProbe: + description: StartupProbe indicates that the + Pod has successfully initialized. + properties: + exec: + description: Exec specifies the action to + take. + properties: + command: + description: 'Command is the command + line to execute inside the container, + the working directory for the command ' + items: + type: string + type: array + type: object + failureThreshold: + description: Minimum consecutive failures + for the probe to be considered failed + after having succeeded. + format: int32 + type: integer + grpc: + description: GRPC specifies an action involving + a GRPC port. + properties: + port: + description: Port number of the gRPC + service. Number must be in the range + 1 to 65535. + format: int32 + type: integer + service: + description: Service is the name of + the service to place in the gRPC HealthCheckRequest + (see https://github. + type: string + required: + - port + type: object + httpGet: + description: HTTPGet specifies the http + request to perform. + properties: + host: + description: Host name to connect to, + defaults to the pod IP. + type: string + httpHeaders: + description: Custom headers to set in + the request. HTTP allows repeated + headers. + items: + description: HTTPHeader describes + a custom header to be used in HTTP + probes + properties: + name: + description: The header field + name + type: string + value: + description: The header field + value + type: string + required: + - name + - value + type: object + type: array + path: + description: Path to access on the HTTP + server. + type: string + port: + anyOf: + - type: integer + - type: string + description: Name or number of the port + to access on the container. Number + must be in the range 1 to 65535. + x-kubernetes-int-or-string: true + scheme: + description: Scheme to use for connecting + to the host. Defaults to HTTP. + type: string + required: + - port + type: object + initialDelaySeconds: + description: Number of seconds after the + container has started before liveness + probes are initiated. + format: int32 + type: integer + periodSeconds: + description: How often (in seconds) to perform + the probe. Default to 10 seconds. Minimum + value is 1. + format: int32 + type: integer + successThreshold: + description: Minimum consecutive successes + for the probe to be considered successful + after having failed. + format: int32 + type: integer + tcpSocket: + description: TCPSocket specifies an action + involving a TCP port. + properties: + host: + description: 'Optional: Host name to + connect to, defaults to the pod IP.' + type: string + port: + anyOf: + - type: integer + - type: string + description: Number or name of the port + to access on the container. Number + must be in the range 1 to 65535. + x-kubernetes-int-or-string: true + required: + - port + type: object + terminationGracePeriodSeconds: + description: Optional duration in seconds + the pod needs to terminate gracefully + upon probe failure. + format: int64 + type: integer + timeoutSeconds: + description: Number of seconds after which + the probe times out. Defaults to 1 second. + Minimum value is 1. + format: int32 + type: integer + type: object + stdin: + description: Whether this container should allocate + a buffer for stdin in the container runtime. + type: boolean + stdinOnce: + description: Whether the container runtime should + close the stdin channel after it has been + opened by a single at + type: boolean + terminationMessagePath: + description: 'Optional: Path at which the file + to which the container''s termination message + will be written is mou' + type: string + terminationMessagePolicy: + description: Indicate how the termination message + should be populated. + type: string + tty: + description: Whether this container should allocate + a TTY for itself, also requires 'stdin' to + be true. + type: boolean + volumeDevices: + description: volumeDevices is the list of block + devices to be used by the container. + items: + description: volumeDevice describes a mapping + of a raw block device within a container. + properties: + devicePath: + description: devicePath is the path inside + of the container that the device will + be mapped to. + type: string + name: + description: name must match the name + of a persistentVolumeClaim in the pod + type: string + required: + - devicePath + - name + type: object + type: array + volumeMounts: + description: Pod volumes to mount into the container's + filesystem. Cannot be updated. + items: + description: VolumeMount describes a mounting + of a Volume within a container. + properties: + mountPath: + description: Path within the container + at which the volume should be mounted. Must + not contain ':'. + type: string + mountPropagation: + description: mountPropagation determines + how mounts are propagated from the host + to container and the other way a + type: string + name: + description: This must match the Name + of a Volume. + type: string + readOnly: + description: Mounted read-only if true, + read-write otherwise (false or unspecified). + Defaults to false. + type: boolean + subPath: + description: Path within the volume from + which the container's volume should + be mounted. + type: string + subPathExpr: + description: Expanded path within the + volume from which the container's volume + should be mounted. + type: string + required: + - mountPath + - name + type: object + type: array + workingDir: + description: Container's working directory. + type: string + required: + - name + type: object + type: array + nodeName: + description: NodeName is a request to schedule this + pod onto a specific node. + type: string + nodeSelector: + additionalProperties: + type: string + description: NodeSelector is a selector which must + be true for the pod to fit on a node. + type: object + x-kubernetes-map-type: atomic + os: + description: Specifies the OS of the containers in + the pod. + properties: + name: + description: Name is the name of the operating + system. The currently supported values are linux + and windows. + type: string + required: + - name + type: object + overhead: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + description: Overhead represents the resource overhead + associated with running a pod for a given RuntimeClass. + type: object + preemptionPolicy: + description: PreemptionPolicy is the Policy for preempting + pods with lower priority. + type: string + priority: + description: The priority value. Various system components + use this field to find the priority of the pod. + format: int32 + type: integer + priorityClassName: + description: If specified, indicates the pod's priority. + type: string + readinessGates: + description: If specified, all readiness gates will + be evaluated for pod readiness. + items: + description: PodReadinessGate contains the reference + to a pod condition + properties: + conditionType: + description: ConditionType refers to a condition + in the pod's condition list with matching + type. + type: string + required: + - conditionType + type: object + type: array + restartPolicy: + description: Restart policy for all containers within + the pod. One of Always, OnFailure, Never. + type: string + runtimeClassName: + description: RuntimeClassName refers to a RuntimeClass + object in the node.k8s. + type: string + schedulerName: + description: If specified, the pod will be dispatched + by specified scheduler. + type: string + securityContext: + description: SecurityContext holds pod-level security + attributes and common container settings. + properties: + fsGroup: + description: A special supplemental group that + applies to all containers in a pod. + format: int64 + type: integer + fsGroupChangePolicy: + description: fsGroupChangePolicy defines behavior + of changing ownership and permission of the + volume before being + type: string + runAsGroup: + description: The GID to run the entrypoint of + the container process. Uses runtime default + if unset. + format: int64 + type: integer + runAsNonRoot: + description: Indicates that the container must + run as a non-root user. + type: boolean + runAsUser: + description: The UID to run the entrypoint of + the container process. + format: int64 + type: integer + seLinuxOptions: + description: The SELinux context to be applied + to all containers. + properties: + level: + description: Level is SELinux level label + that applies to the container. + type: string + role: + description: Role is a SELinux role label + that applies to the container. + type: string + type: + description: Type is a SELinux type label + that applies to the container. + type: string + user: + description: User is a SELinux user label + that applies to the container. + type: string + type: object + seccompProfile: + description: The seccomp options to use by the + containers in this pod. + properties: + localhostProfile: + description: localhostProfile indicates a + profile defined in a file on the node should + be used. + type: string + type: + description: type indicates which kind of + seccomp profile will be applied. + type: string + required: + - type + type: object + supplementalGroups: + description: 'A list of groups applied to the + first process run in each container, in addition + to the container''s ' + items: + format: int64 + type: integer + type: array + sysctls: + description: Sysctls hold a list of namespaced + sysctls used for the pod. + items: + description: Sysctl defines a kernel parameter + to be set + properties: + name: + description: Name of a property to set + type: string + value: + description: Value of a property to set + type: string + required: + - name + - value + type: object + type: array + windowsOptions: + description: The Windows specific settings applied + to all containers. + properties: + gmsaCredentialSpec: + description: GMSACredentialSpec is where the + GMSA admission webhook (https://github. + type: string + gmsaCredentialSpecName: + description: GMSACredentialSpecName is the + name of the GMSA credential spec to use. + type: string + hostProcess: + description: HostProcess determines if a container + should be run as a 'Host Process' container. + type: boolean + runAsUserName: + description: The UserName in Windows to run + the entrypoint of the container process. + type: string + type: object + type: object + serviceAccount: + description: DeprecatedServiceAccount is a depreciated + alias for ServiceAccountName. + type: string + serviceAccountName: + description: ServiceAccountName is the name of the + ServiceAccount to use to run this pod. + type: string + setHostnameAsFQDN: + description: If true the pod's hostname will be configured + as the pod's FQDN, rather than the leaf name (the + defa + type: boolean + shareProcessNamespace: + description: Share a single process namespace between + all of the containers in a pod. + type: boolean + subdomain: + description: If specified, the fully qualified Pod + hostname will be "...svc. + type: string + terminationGracePeriodSeconds: + description: Optional duration in seconds the pod + needs to terminate gracefully. + format: int64 + type: integer + tolerations: + description: If specified, the pod's tolerations. + items: + description: The pod this Toleration is attached + to tolerates any taint that matches the triple + name on the dataset for + Flocker should be considered as de + type: string + datasetUUID: + description: UUID of the dataset. This is + unique identifier of a Flocker dataset + type: string + type: object + gcePersistentDisk: + description: GCEPersistentDisk represents a + GCE Disk resource that is attached to a kubelet's + host machine and th + properties: + fsType: + description: Filesystem type of the volume + that you want to mount. + type: string + partition: + description: The partition in the volume + that you want to mount. + format: int32 + type: integer + pdName: + description: Unique name of the PD resource + in GCE. Used to identify the disk in GCE. + type: string + readOnly: + description: ReadOnly here will force the + ReadOnly setting in VolumeMounts. Defaults + to false. + type: boolean + required: + - pdName + type: object + gitRepo: + description: 'GitRepo represents a git repository + at a particular revision. DEPRECATED: GitRepo + is deprecated.' + properties: + directory: + description: Target directory name. Must + not contain or start with '..'. If '. + type: string + repository: + description: Repository URL + type: string + revision: + description: Commit hash for the specified + revision. + type: string + required: + - repository + type: object + glusterfs: + description: Glusterfs represents a Glusterfs + mount on the host that shares a pod's lifetime. + properties: + endpoints: + description: 'EndpointsName is the endpoint + name that details Glusterfs topology. + More info: https://examples.k8s.' + type: string + path: + description: 'Path is the Glusterfs volume + path. More info: https://examples.k8s.io/volumes/glusterfs/README.' + type: string + readOnly: + description: ReadOnly here will force the + Glusterfs volume to be mounted with read-only + permissions. + type: boolean + required: + - endpoints + - path + type: object + hostPath: + description: HostPath represents a pre-existing + file or directory on the host machine that + is directly exposed to + properties: + path: + description: Path of the directory on the + host. + type: string + type: + description: 'Type for HostPath Volume Defaults + to "" More info: https://kubernetes.' + type: string + required: + - path + type: object + iscsi: + description: ISCSI represents an ISCSI Disk + resource that is attached to a kubelet's host + machine and then expose + properties: + chapAuthDiscovery: + description: whether support iSCSI Discovery + CHAP authentication + type: boolean + chapAuthSession: + description: whether support iSCSI Session + CHAP authentication + type: boolean + fsType: + description: Filesystem type of the volume + that you want to mount. + type: string + initiatorName: + description: Custom iSCSI Initiator Name. + type: string + iqn: + description: Target iSCSI Qualified Name. + type: string + iscsiInterface: + description: iSCSI Interface Name that uses + an iSCSI transport. Defaults to 'default' + (tcp). + type: string + lun: + description: iSCSI Target Lun number. + format: int32 + type: integer + portals: + description: iSCSI Target Portal List. + items: + type: string + type: array + readOnly: + description: ReadOnly here will force the + ReadOnly setting in VolumeMounts. Defaults + to false. + type: boolean + secretRef: + description: CHAP Secret for iSCSI target + and initiator authentication + properties: + name: + description: 'Name of the referent. + More info: https://kubernetes.' + type: string + type: object + targetPortal: + description: iSCSI Target Portal. + type: string + required: + - iqn + - lun + - targetPortal + type: object + name: + description: 'Volume''s name. Must be a DNS_LABEL + and unique within the pod. More info: https://kubernetes.' + type: string + nfs: + description: 'NFS represents an NFS mount on + the host that shares a pod''s lifetime More + info: https://kubernetes.' + properties: + path: + description: 'Path that is exported by the + NFS server. More info: https://kubernetes.' + type: string + readOnly: + description: ReadOnly here will force the + NFS export to be mounted with read-only + permissions. Defaults to false. + type: boolean + server: + description: 'Server is the hostname or + IP address of the NFS server. More info: + https://kubernetes.' + type: string + required: + - path + - server + type: object + persistentVolumeClaim: + description: PersistentVolumeClaimVolumeSource + represents a reference to a PersistentVolumeClaim + in the same name + properties: + claimName: + description: ClaimName is the name of a + PersistentVolumeClaim in the same namespace + as the pod using this volume. + type: string + readOnly: + description: Will force the ReadOnly setting + in VolumeMounts. Default false. + type: boolean + required: + - claimName + type: object + photonPersistentDisk: + description: 'PhotonPersistentDisk represents + a PhotonController persistent disk attached + and mounted on kubelets ' + properties: + fsType: + description: Filesystem type to mount. Must + be a filesystem type supported by the + host operating system. Ex. + type: string + pdID: + description: ID that identifies Photon Controller + persistent disk + type: string + required: + - pdID + type: object + portworxVolume: + description: PortworxVolume represents a portworx + volume attached and mounted on kubelets host + machine + properties: + fsType: + description: FSType represents the filesystem + type to mount Must be a filesystem type + supported by the host opera + type: string + readOnly: + description: Defaults to false (read/write). + ReadOnly here will force the ReadOnly + setting in VolumeMounts. + type: boolean + volumeID: + description: VolumeID uniquely identifies + a Portworx volume + type: string + required: + - volumeID + type: object + projected: + description: Items for all in one resources + secrets, configmaps, and downward API + properties: + defaultMode: + description: Mode bits used to set permissions + on created files by default. + format: int32 + type: integer + sources: + description: list of volume projections + items: + description: Projection that may be projected + along with other supported volume types + properties: + configMap: + description: information about the + configMap data to project + properties: + items: + description: 'If unspecified, + each key-value pair in the Data + field of the referenced ConfigMap + will be projected ' + items: + description: Maps a string key + to a path within a volume. + properties: + key: + description: The key to + project. + type: string + mode: + description: 'Optional: + mode bits used to set + permissions on this file.' + format: int32 + type: integer + path: + description: The relative + path of the file to map + the key to. May not be + an absolute path. + type: string + required: + - key + - path + type: object + type: array + name: + description: 'Name of the referent. + More info: https://kubernetes.' + type: string + optional: + description: Specify whether the + ConfigMap or its keys must be + defined + type: boolean + type: object + downwardAPI: + description: information about the + downwardAPI data to project + properties: + items: + description: Items is a list of + DownwardAPIVolume file + items: + description: DownwardAPIVolumeFile + represents information to + create the file containing + the pod field + properties: + fieldRef: + description: 'Required: + Selects a field of the + pod: only annotations, + labels, name and namespace + are supported.' + properties: + apiVersion: + description: Version + of the schema the + FieldPath is written + in terms of, defaults + to "v1". + type: string + fieldPath: + description: Path of + the field to select + in the specified API + version. + type: string + required: + - fieldPath + type: object + mode: + description: 'Optional: + mode bits used to set + permissions on this file, + must be an octal value + between 0000 and 07' + format: int32 + type: integer + path: + description: 'Required: + Path is the relative + path name of the file + to be created.' + type: string + resourceFieldRef: + description: 'Selects a + resource of the container: + only resources limits + and requests (limits.cpu, + limits.' + properties: + containerName: + description: 'Container + name: required for + volumes, optional + for env vars' + type: string + divisor: + anyOf: + - type: integer + - type: string + description: Specifies + the output format + of the exposed resources, + defaults to "1" + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: + description: 'Required: + resource to select' + type: string + required: + - resource + type: object + required: + - path + type: object + type: array + type: object + secret: + description: information about the + secret data to project + properties: + items: + description: If unspecified, each + key-value pair in the Data field + of the referenced Secret will + be projected int + items: + description: Maps a string key + to a path within a volume. + properties: + key: + description: The key to + project. + type: string + mode: + description: 'Optional: + mode bits used to set + permissions on this file.' + format: int32 + type: integer + path: + description: The relative + path of the file to map + the key to. May not be + an absolute path. + type: string + required: + - key + - path + type: object + type: array + name: + description: 'Name of the referent. + More info: https://kubernetes.' + type: string + optional: + description: Specify whether the + Secret or its key must be defined + type: boolean + type: object + serviceAccountToken: + description: information about the + serviceAccountToken data to project + properties: + audience: + description: Audience is the intended + audience of the token. + type: string + expirationSeconds: + description: ExpirationSeconds + is the requested duration of + validity of the service account + token. + format: int64 + type: integer + path: + description: Path is the path + relative to the mount point + of the file to project the token + into. + type: string + required: + - path + type: object + type: object + type: array + type: object + quobyte: + description: Quobyte represents a Quobyte mount + on the host that shares a pod's lifetime + properties: + group: + description: Group to map volume access + to Default is no group + type: string + readOnly: + description: ReadOnly here will force the + Quobyte volume to be mounted with read-only + permissions. + type: boolean + registry: + description: Registry represents a single + or multiple Quobyte Registry services + specified as a string as host:por + type: string + tenant: + description: Tenant owning the given Quobyte + volume in the Backend Used with dynamically + provisioned Quobyte volu + type: string + user: + description: User to map volume access to + Defaults to serivceaccount user + type: string + volume: + description: Volume is a string that references + an already created Quobyte volume by name. + type: string + required: + - registry + - volume + type: object + rbd: + description: RBD represents a Rados Block Device + mount on the host that shares a pod's lifetime. + properties: + fsType: + description: Filesystem type of the volume + that you want to mount. + type: string + image: + description: 'The rados image name. More + info: https://examples.k8s.io/volumes/rbd/README.md#how-to-use-it' + type: string + keyring: + description: Keyring is the path to key + ring for RBDUser. Default is /etc/ceph/keyring. + type: string + monitors: + description: 'A collection of Ceph monitors. + More info: https://examples.k8s.io/volumes/rbd/README.' + items: + type: string + type: array + pool: + description: 'The rados pool name. Default + is rbd. More info: https://examples.k8s.io/volumes/rbd/README.' + type: string + readOnly: + description: ReadOnly here will force the + ReadOnly setting in VolumeMounts. Defaults + to false. + type: boolean + secretRef: + description: SecretRef is name of the authentication + secret for RBDUser. If provided overrides + keyring. + properties: + name: + description: 'Name of the referent. + More info: https://kubernetes.' + type: string + type: object + user: + description: 'The rados user name. Default + is admin. More info: https://examples.k8s.io/volumes/rbd/README.' + type: string + required: + - image + - monitors + type: object + scaleIO: + description: ScaleIO represents a ScaleIO persistent + volume attached and mounted on Kubernetes + nodes. + properties: + fsType: + description: Filesystem type to mount. Must + be a filesystem type supported by the + host operating system. Ex. + type: string + gateway: + description: The host address of the ScaleIO + API Gateway. + type: string + protectionDomain: + description: The name of the ScaleIO Protection + Domain for the configured storage. + type: string + readOnly: + description: Defaults to false (read/write). + ReadOnly here will force the ReadOnly + setting in VolumeMounts. + type: boolean + secretRef: + description: SecretRef references to the + secret for ScaleIO user and other sensitive + information. + properties: + name: + description: 'Name of the referent. + More info: https://kubernetes.' + type: string + type: object + sslEnabled: + description: Flag to enable/disable SSL + communication with Gateway, default false + type: boolean + storageMode: + description: Indicates whether the storage + for a volume should be ThickProvisioned + or ThinProvisioned. + type: string + storagePool: + description: The ScaleIO Storage Pool associated + with the protection domain. + type: string + system: + description: The name of the storage system + as configured in ScaleIO. + type: string + volumeName: + description: The name of a volume already + created in the ScaleIO system that is + associated with this volume sourc + type: string + required: + - gateway + - secretRef + - system + type: object + secret: + description: 'Secret represents a secret that + should populate this volume. More info: https://kubernetes.' + properties: + defaultMode: + description: 'Optional: mode bits used to + set permissions on created files by default.' + format: int32 + type: integer + items: + description: If unspecified, each key-value + pair in the Data field of the referenced + Secret will be projected int + items: + description: Maps a string key to a path + within a volume. + properties: + key: + description: The key to project. + type: string + mode: + description: 'Optional: mode bits + used to set permissions on this + file.' + format: int32 + type: integer + path: + description: The relative path of + the file to map the key to. May + not be an absolute path. + type: string + required: + - key + - path + type: object + type: array + optional: + description: Specify whether the Secret + or its keys must be defined + type: boolean + secretName: + description: 'Name of the secret in the + pod''s namespace to use. More info: https://kubernetes.' + type: string + type: object + storageos: + description: StorageOS represents a StorageOS + volume attached and mounted on Kubernetes + nodes. + properties: + fsType: + description: Filesystem type to mount. Must + be a filesystem type supported by the + host operating system. Ex. + type: string + readOnly: + description: Defaults to false (read/write). + ReadOnly here will force the ReadOnly + setting in VolumeMounts. + type: boolean + secretRef: + description: SecretRef specifies the secret + to use for obtaining the StorageOS API + credentials. + properties: + name: + description: 'Name of the referent. + More info: https://kubernetes.' + type: string + type: object + volumeName: + description: VolumeName is the human-readable + name of the StorageOS volume. + type: string + volumeNamespace: + description: VolumeNamespace specifies the + scope of the volume within StorageOS. + type: string + type: object + vsphereVolume: + description: VsphereVolume represents a vSphere + volume attached and mounted on kubelets host + machine + properties: + fsType: + description: Filesystem type to mount. Must + be a filesystem type supported by the + host operating system. Ex. + type: string + storagePolicyID: + description: Storage Policy Based Management + (SPBM) profile ID associated with the + StoragePolicyName. + type: string + storagePolicyName: + description: Storage Policy Based Management + (SPBM) profile name. + type: string + volumePath: + description: Path that identifies vSphere + volume vmdk + type: string + required: + - volumePath + type: object + required: + - name + type: object + type: array + required: + - containers + type: object + type: object + required: + - rayStartParams + - replicas + - serviceType + - template + type: object + rayVersion: + description: RayVersion is the version of ray being used. this + affects the command used to start ray + type: string + workerGroupSpecs: + description: WorkerGroupSpecs are the specs for the worker pods + items: + description: WorkerGroupSpec are the specs for the worker pods + properties: + groupName: + description: we can have multiple worker groups, we distinguish + them by name + type: string + maxReplicas: + description: MaxReplicas defaults to maxInt32 + format: int32 + type: integer + minReplicas: + description: MinReplicas defaults to 1 + format: int32 + type: integer + rayStartParams: + additionalProperties: + type: string + description: 'RayStartParams are the params of the start + command: address, object-store-memory, ...' + type: object + replicas: + description: Replicas Number of desired pods in this pod + group. + format: int32 + type: integer + scaleStrategy: + description: ScaleStrategy defines which pods to remove + properties: + workersToDelete: + description: WorkersToDelete workers to be deleted + items: + type: string + type: array + type: object + template: + description: Template a pod template for the worker + properties: + metadata: + description: 'Standard object''s metadata. More info: + https://git.k8s.' + properties: + annotations: + additionalProperties: + type: string + type: object + finalizers: + items: + type: string + type: array + labels: + additionalProperties: + type: string + type: object + name: + type: string + namespace: + type: string + type: object + spec: + description: 'Specification of the desired behavior + of the pod. More info: https://git.k8s.' + properties: + activeDeadlineSeconds: + description: Optional duration in seconds the pod + may be active on the node relative to StartTime + before the syst + format: int64 + type: integer + affinity: + description: If specified, the pod's scheduling + constraints + properties: + nodeAffinity: + description: Describes node affinity scheduling + rules for the pod. + properties: + preferredDuringSchedulingIgnoredDuringExecution: + description: 'The scheduler will prefer + to schedule pods to nodes that satisfy + the affinity expressions specified ' + items: + description: An empty preferred scheduling + term matches all objects with implicit + weight 0 (i.e. it's a no-op). + properties: + preference: + description: A node selector term, + associated with the corresponding + weight. + properties: + matchExpressions: + description: A list of node selector + requirements by node's labels. + items: + description: 'A node selector + requirement is a selector + that contains values, a key, + and an operator that relates ' + properties: + key: + description: The label key + that the selector applies + to. + type: string + operator: + description: Represents + a key's relationship to + a set of values. + type: string + values: + description: An array of + string values. If the + operator is In or NotIn, + the values array must + be non-empty. + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchFields: + description: A list of node selector + requirements by node's fields. + items: + description: 'A node selector + requirement is a selector + that contains values, a key, + and an operator that relates ' + properties: + key: + description: The label key + that the selector applies + to. + type: string + operator: + description: Represents + a key's relationship to + a set of values. + type: string + values: + description: An array of + string values. If the + operator is In or NotIn, + the values array must + be non-empty. + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + type: object + weight: + description: Weight associated with + matching the corresponding nodeSelectorTerm, + in the range 1-100. + format: int32 + type: integer + required: + - preference + - weight + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + description: If the affinity requirements + specified by this field are not met at + scheduling time, the pod will no + properties: + nodeSelectorTerms: + description: Required. A list of node + selector terms. The terms are ORed. + items: + description: A null or empty node + selector term matches no objects. + The requirements of them are ANDed. + properties: + matchExpressions: + description: A list of node selector + requirements by node's labels. + items: + description: 'A node selector + requirement is a selector + that contains values, a key, + and an operator that relates ' + properties: + key: + description: The label key + that the selector applies + to. + type: string + operator: + description: Represents + a key's relationship to + a set of values. + type: string + values: + description: An array of + string values. If the + operator is In or NotIn, + the values array must + be non-empty. + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchFields: + description: A list of node selector + requirements by node's fields. + items: + description: 'A node selector + requirement is a selector + that contains values, a key, + and an operator that relates ' + properties: + key: + description: The label key + that the selector applies + to. + type: string + operator: + description: Represents + a key's relationship to + a set of values. + type: string + values: + description: An array of + string values. If the + operator is In or NotIn, + the values array must + be non-empty. + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + type: object + type: array + required: + - nodeSelectorTerms + type: object + type: object + podAffinity: + description: Describes pod affinity scheduling + rules (e.g. co-locate this pod in the same + node, zone, etc. + properties: + preferredDuringSchedulingIgnoredDuringExecution: + description: 'The scheduler will prefer + to schedule pods to nodes that satisfy + the affinity expressions specified ' + items: + description: The weights of all of the + matched WeightedPodAffinityTerm fields + are added per-node to find the most + properties: + podAffinityTerm: + description: Required. A pod affinity + term, associated with the corresponding + weight. + properties: + labelSelector: + description: A label query over + a set of resources, in this + case pods. + properties: + matchExpressions: + description: matchExpressions + is a list of label selector + requirements. The requirements + are ANDed. + items: + description: A label selector + requirement is a selector + that contains values, + a key, and an operator + that relates + properties: + key: + description: key is + the label key that + the selector applies + to. + type: string + operator: + description: operator + represents a key's + relationship to a + set of values. + type: string + values: + description: values + is an array of string + values. + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + description: matchLabels is + a map of {key,value} pairs. + type: object + type: object + namespaceSelector: + description: A label query over + the set of namespaces that the + term applies to. + properties: + matchExpressions: + description: matchExpressions + is a list of label selector + requirements. The requirements + are ANDed. + items: + description: A label selector + requirement is a selector + that contains values, + a key, and an operator + that relates + properties: + key: + description: key is + the label key that + the selector applies + to. + type: string + operator: + description: operator + represents a key's + relationship to a + set of values. + type: string + values: + description: values + is an array of string + values. + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + description: matchLabels is + a map of {key,value} pairs. + type: object + type: object + namespaces: + description: namespaces specifies + a static list of namespace names + that the term applies to. + items: + type: string + type: array + topologyKey: + description: This pod should be + co-located (affinity) or not + co-located (anti-affinity) with + the pods matching th + type: string + required: + - topologyKey + type: object + weight: + description: weight associated with + matching the corresponding podAffinityTerm, + in the range 1-100. + format: int32 + type: integer + required: + - podAffinityTerm + - weight + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + description: If the affinity requirements + specified by this field are not met at + scheduling time, the pod will no + items: + description: Defines a set of pods (namely + those matching the labelSelector relative + to the given namespace(s)) t + properties: + labelSelector: + description: A label query over a + set of resources, in this case pods. + properties: + matchExpressions: + description: matchExpressions + is a list of label selector + requirements. The requirements + are ANDed. + items: + description: A label selector + requirement is a selector + that contains values, a key, + and an operator that relates + properties: + key: + description: key is the + label key that the selector + applies to. + type: string + operator: + description: operator represents + a key's relationship to + a set of values. + type: string + values: + description: values is an + array of string values. + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + description: matchLabels is a + map of {key,value} pairs. + type: object + type: object + namespaceSelector: + description: A label query over the + set of namespaces that the term + applies to. + properties: + matchExpressions: + description: matchExpressions + is a list of label selector + requirements. The requirements + are ANDed. + items: + description: A label selector + requirement is a selector + that contains values, a key, + and an operator that relates + properties: + key: + description: key is the + label key that the selector + applies to. + type: string + operator: + description: operator represents + a key's relationship to + a set of values. + type: string + values: + description: values is an + array of string values. + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + description: matchLabels is a + map of {key,value} pairs. + type: object + type: object + namespaces: + description: namespaces specifies + a static list of namespace names + that the term applies to. + items: + type: string + type: array + topologyKey: + description: This pod should be co-located + (affinity) or not co-located (anti-affinity) + with the pods matching th + type: string + required: + - topologyKey + type: object + type: array + type: object + podAntiAffinity: + description: Describes pod anti-affinity scheduling + rules (e.g. + properties: + preferredDuringSchedulingIgnoredDuringExecution: + description: The scheduler will prefer to + schedule pods to nodes that satisfy the + anti-affinity expressions speci + items: + description: The weights of all of the + matched WeightedPodAffinityTerm fields + are added per-node to find the most + properties: + podAffinityTerm: + description: Required. A pod affinity + term, associated with the corresponding + weight. + properties: + labelSelector: + description: A label query over + a set of resources, in this + case pods. + properties: + matchExpressions: + description: matchExpressions + is a list of label selector + requirements. The requirements + are ANDed. + items: + description: A label selector + requirement is a selector + that contains values, + a key, and an operator + that relates + properties: + key: + description: key is + the label key that + the selector applies + to. + type: string + operator: + description: operator + represents a key's + relationship to a + set of values. + type: string + values: + description: values + is an array of string + values. + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + description: matchLabels is + a map of {key,value} pairs. + type: object + type: object + namespaceSelector: + description: A label query over + the set of namespaces that the + term applies to. + properties: + matchExpressions: + description: matchExpressions + is a list of label selector + requirements. The requirements + are ANDed. + items: + description: A label selector + requirement is a selector + that contains values, + a key, and an operator + that relates + properties: + key: + description: key is + the label key that + the selector applies + to. + type: string + operator: + description: operator + represents a key's + relationship to a + set of values. + type: string + values: + description: values + is an array of string + values. + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + description: matchLabels is + a map of {key,value} pairs. + type: object + type: object + namespaces: + description: namespaces specifies + a static list of namespace names + that the term applies to. + items: + type: string + type: array + topologyKey: + description: This pod should be + co-located (affinity) or not + co-located (anti-affinity) with + the pods matching th + type: string + required: + - topologyKey + type: object + weight: + description: weight associated with + matching the corresponding podAffinityTerm, + in the range 1-100. + format: int32 + type: integer + required: + - podAffinityTerm + - weight + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + description: If the anti-affinity requirements + specified by this field are not met at + scheduling time, the pod wi + items: + description: Defines a set of pods (namely + those matching the labelSelector relative + to the given namespace(s)) t + properties: + labelSelector: + description: A label query over a + set of resources, in this case pods. + properties: + matchExpressions: + description: matchExpressions + is a list of label selector + requirements. The requirements + are ANDed. + items: + description: A label selector + requirement is a selector + that contains values, a key, + and an operator that relates + properties: + key: + description: key is the + label key that the selector + applies to. + type: string + operator: + description: operator represents + a key's relationship to + a set of values. + type: string + values: + description: values is an + array of string values. + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + description: matchLabels is a + map of {key,value} pairs. + type: object + type: object + namespaceSelector: + description: A label query over the + set of namespaces that the term + applies to. + properties: + matchExpressions: + description: matchExpressions + is a list of label selector + requirements. The requirements + are ANDed. + items: + description: A label selector + requirement is a selector + that contains values, a key, + and an operator that relates + properties: + key: + description: key is the + label key that the selector + applies to. + type: string + operator: + description: operator represents + a key's relationship to + a set of values. + type: string + values: + description: values is an + array of string values. + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + description: matchLabels is a + map of {key,value} pairs. + type: object + type: object + namespaces: + description: namespaces specifies + a static list of namespace names + that the term applies to. + items: + type: string + type: array + topologyKey: + description: This pod should be co-located + (affinity) or not co-located (anti-affinity) + with the pods matching th + type: string + required: + - topologyKey + type: object + type: array + type: object + type: object + automountServiceAccountToken: + description: AutomountServiceAccountToken indicates + whether a service account token should be automatically + mount + type: boolean + containers: + description: List of containers belonging to the + pod. Containers cannot currently be added or removed. + items: + description: A single application container that + you want to run within a pod. + properties: + args: + description: Arguments to the entrypoint. + The docker image's CMD is used if this is + not provided. + items: + type: string + type: array + command: + description: Entrypoint array. Not executed + within a shell. + items: + type: string + type: array + env: + description: List of environment variables + to set in the container. Cannot be updated. + items: + description: EnvVar represents an environment + variable present in a Container. + properties: + name: + description: Name of the environment + variable. Must be a C_IDENTIFIER. + type: string + value: + description: Variable references $(VAR_NAME) + are expanded using the previously + defined environment variables in t + type: string + valueFrom: + description: Source for the environment + variable's value. Cannot be used if + value is not empty. + properties: + configMapKeyRef: + description: Selects a key of a + ConfigMap. + properties: + key: + description: The key to select. + type: string + name: + description: 'Name of the referent. + More info: https://kubernetes.' + type: string + optional: + description: Specify whether + the ConfigMap or its key must + be defined + type: boolean + required: + - key + type: object + fieldRef: + description: 'Selects a field of + the pod: supports metadata.name, + metadata.namespace, `metadata.' + properties: + apiVersion: + description: Version of the + schema the FieldPath is written + in terms of, defaults to "v1". + type: string + fieldPath: + description: Path of the field + to select in the specified + API version. + type: string + required: + - fieldPath + type: object + resourceFieldRef: + description: 'Selects a resource + of the container: only resources + limits and requests (limits.cpu, + limits.' + properties: + containerName: + description: 'Container name: + required for volumes, optional + for env vars' + type: string + divisor: + anyOf: + - type: integer + - type: string + description: Specifies the output + format of the exposed resources, + defaults to "1" + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: + description: 'Required: resource + to select' + type: string + required: + - resource + type: object + secretKeyRef: + description: Selects a key of a + secret in the pod's namespace + properties: + key: + description: The key of the + secret to select from. Must + be a valid secret key. + type: string + name: + description: 'Name of the referent. + More info: https://kubernetes.' + type: string + optional: + description: Specify whether + the Secret or its key must + be defined + type: boolean + required: + - key + type: object + type: object + required: + - name + type: object + type: array + envFrom: + description: List of sources to populate environment + variables in the container. + items: + description: EnvFromSource represents the + source of a set of ConfigMaps + properties: + configMapRef: + description: The ConfigMap to select + from + properties: + name: + description: 'Name of the referent. + More info: https://kubernetes.' + type: string + optional: + description: Specify whether the + ConfigMap must be defined + type: boolean + type: object + prefix: + description: An optional identifier + to prepend to each key in the ConfigMap. + Must be a C_IDENTIFIER. + type: string + secretRef: + description: The Secret to select from + properties: + name: + description: 'Name of the referent. + More info: https://kubernetes.' + type: string + optional: + description: Specify whether the + Secret must be defined + type: boolean + type: object + type: object + type: array + image: + description: 'Docker image name. More info: + https://kubernetes.' + type: string + imagePullPolicy: + description: Image pull policy. One of Always, + Never, IfNotPresent. + type: string + lifecycle: + description: Actions that the management system + should take in response to container lifecycle + events. + properties: + postStart: + description: PostStart is called immediately + after a container is created. + properties: + exec: + description: Exec specifies the action + to take. + properties: + command: + description: 'Command is the command + line to execute inside the container, + the working directory for the + command ' + items: + type: string + type: array + type: object + httpGet: + description: HTTPGet specifies the + http request to perform. + properties: + host: + description: Host name to connect + to, defaults to the pod IP. + type: string + httpHeaders: + description: Custom headers to + set in the request. HTTP allows + repeated headers. + items: + description: HTTPHeader describes + a custom header to be used + in HTTP probes + properties: + name: + description: The header + field name + type: string + value: + description: The header + field value + type: string + required: + - name + - value + type: object + type: array + path: + description: Path to access on + the HTTP server. + type: string + port: + anyOf: + - type: integer + - type: string + description: Name or number of + the port to access on the container. + Number must be in the range + 1 to 65535. + x-kubernetes-int-or-string: true + scheme: + description: Scheme to use for + connecting to the host. Defaults + to HTTP. + type: string + required: + - port + type: object + tcpSocket: + description: Deprecated. + properties: + host: + description: 'Optional: Host name + to connect to, defaults to the + pod IP.' + type: string + port: + anyOf: + - type: integer + - type: string + description: Number or name of + the port to access on the container. + Number must be in the range + 1 to 65535. + x-kubernetes-int-or-string: true + required: + - port + type: object + type: object + preStop: + description: PreStop is called immediately + before a container is terminated due + to an API request or management e + properties: + exec: + description: Exec specifies the action + to take. + properties: + command: + description: 'Command is the command + line to execute inside the container, + the working directory for the + command ' + items: + type: string + type: array + type: object + httpGet: + description: HTTPGet specifies the + http request to perform. + properties: + host: + description: Host name to connect + to, defaults to the pod IP. + type: string + httpHeaders: + description: Custom headers to + set in the request. HTTP allows + repeated headers. + items: + description: HTTPHeader describes + a custom header to be used + in HTTP probes + properties: + name: + description: The header + field name + type: string + value: + description: The header + field value + type: string + required: + - name + - value + type: object + type: array + path: + description: Path to access on + the HTTP server. + type: string + port: + anyOf: + - type: integer + - type: string + description: Name or number of + the port to access on the container. + Number must be in the range + 1 to 65535. + x-kubernetes-int-or-string: true + scheme: + description: Scheme to use for + connecting to the host. Defaults + to HTTP. + type: string + required: + - port + type: object + tcpSocket: + description: Deprecated. + properties: + host: + description: 'Optional: Host name + to connect to, defaults to the + pod IP.' + type: string + port: + anyOf: + - type: integer + - type: string + description: Number or name of + the port to access on the container. + Number must be in the range + 1 to 65535. + x-kubernetes-int-or-string: true + required: + - port + type: object + type: object + type: object + livenessProbe: + description: Periodic probe of container liveness. + Container will be restarted if the probe + fails. + properties: + exec: + description: Exec specifies the action + to take. + properties: + command: + description: 'Command is the command + line to execute inside the container, + the working directory for the command ' + items: + type: string + type: array + type: object + failureThreshold: + description: Minimum consecutive failures + for the probe to be considered failed + after having succeeded. + format: int32 + type: integer + grpc: + description: GRPC specifies an action + involving a GRPC port. + properties: + port: + description: Port number of the gRPC + service. Number must be in the range + 1 to 65535. + format: int32 + type: integer + service: + description: Service is the name of + the service to place in the gRPC + HealthCheckRequest (see https://github. + type: string + required: + - port + type: object + httpGet: + description: HTTPGet specifies the http + request to perform. + properties: + host: + description: Host name to connect + to, defaults to the pod IP. + type: string + httpHeaders: + description: Custom headers to set + in the request. HTTP allows repeated + headers. + items: + description: HTTPHeader describes + a custom header to be used in + HTTP probes + properties: + name: + description: The header field + name + type: string + value: + description: The header field + value + type: string + required: + - name + - value + type: object + type: array + path: + description: Path to access on the + HTTP server. + type: string + port: + anyOf: + - type: integer + - type: string + description: Name or number of the + port to access on the container. + Number must be in the range 1 to + 65535. + x-kubernetes-int-or-string: true + scheme: + description: Scheme to use for connecting + to the host. Defaults to HTTP. + type: string + required: + - port + type: object + initialDelaySeconds: + description: Number of seconds after the + container has started before liveness + probes are initiated. + format: int32 + type: integer + periodSeconds: + description: How often (in seconds) to + perform the probe. Default to 10 seconds. + Minimum value is 1. + format: int32 + type: integer + successThreshold: + description: Minimum consecutive successes + for the probe to be considered successful + after having failed. + format: int32 + type: integer + tcpSocket: + description: TCPSocket specifies an action + involving a TCP port. + properties: + host: + description: 'Optional: Host name + to connect to, defaults to the pod + IP.' + type: string + port: + anyOf: + - type: integer + - type: string + description: Number or name of the + port to access on the container. + Number must be in the range 1 to + 65535. + x-kubernetes-int-or-string: true + required: + - port + type: object + terminationGracePeriodSeconds: + description: Optional duration in seconds + the pod needs to terminate gracefully + upon probe failure. + format: int64 + type: integer + timeoutSeconds: + description: Number of seconds after which + the probe times out. Defaults to 1 second. + Minimum value is 1. + format: int32 + type: integer + type: object + name: + description: Name of the container specified + as a DNS_LABEL. + type: string + ports: + description: List of ports to expose from + the container. + items: + description: ContainerPort represents a + network port in a single container. + properties: + containerPort: + description: Number of port to expose + on the pod's IP address. This must + be a valid port number, 0 < x < 65536. + format: int32 + type: integer + hostIP: + description: What host IP to bind the + external port to. + type: string + hostPort: + description: Number of port to expose + on the host. If specified, this must + be a valid port number, 0 < x < 65536. + format: int32 + type: integer + name: + description: If specified, this must + be an IANA_SVC_NAME and unique within + the pod. + type: string + protocol: + default: TCP + description: Protocol for port. Must + be UDP, TCP, or SCTP. Defaults to + "TCP". + type: string + required: + - containerPort + type: object + type: array + x-kubernetes-list-map-keys: + - containerPort + - protocol + x-kubernetes-list-type: map + readinessProbe: + description: Periodic probe of container service + readiness. + properties: + exec: + description: Exec specifies the action + to take. + properties: + command: + description: 'Command is the command + line to execute inside the container, + the working directory for the command ' + items: + type: string + type: array + type: object + failureThreshold: + description: Minimum consecutive failures + for the probe to be considered failed + after having succeeded. + format: int32 + type: integer + grpc: + description: GRPC specifies an action + involving a GRPC port. + properties: + port: + description: Port number of the gRPC + service. Number must be in the range + 1 to 65535. + format: int32 + type: integer + service: + description: Service is the name of + the service to place in the gRPC + HealthCheckRequest (see https://github. + type: string + required: + - port + type: object + httpGet: + description: HTTPGet specifies the http + request to perform. + properties: + host: + description: Host name to connect + to, defaults to the pod IP. + type: string + httpHeaders: + description: Custom headers to set + in the request. HTTP allows repeated + headers. + items: + description: HTTPHeader describes + a custom header to be used in + HTTP probes + properties: + name: + description: The header field + name + type: string + value: + description: The header field + value + type: string + required: + - name + - value + type: object + type: array + path: + description: Path to access on the + HTTP server. + type: string + port: + anyOf: + - type: integer + - type: string + description: Name or number of the + port to access on the container. + Number must be in the range 1 to + 65535. + x-kubernetes-int-or-string: true + scheme: + description: Scheme to use for connecting + to the host. Defaults to HTTP. + type: string + required: + - port + type: object + initialDelaySeconds: + description: Number of seconds after the + container has started before liveness + probes are initiated. + format: int32 + type: integer + periodSeconds: + description: How often (in seconds) to + perform the probe. Default to 10 seconds. + Minimum value is 1. + format: int32 + type: integer + successThreshold: + description: Minimum consecutive successes + for the probe to be considered successful + after having failed. + format: int32 + type: integer + tcpSocket: + description: TCPSocket specifies an action + involving a TCP port. + properties: + host: + description: 'Optional: Host name + to connect to, defaults to the pod + IP.' + type: string + port: + anyOf: + - type: integer + - type: string + description: Number or name of the + port to access on the container. + Number must be in the range 1 to + 65535. + x-kubernetes-int-or-string: true + required: + - port + type: object + terminationGracePeriodSeconds: + description: Optional duration in seconds + the pod needs to terminate gracefully + upon probe failure. + format: int64 + type: integer + timeoutSeconds: + description: Number of seconds after which + the probe times out. Defaults to 1 second. + Minimum value is 1. + format: int32 + type: integer + type: object + resources: + description: 'Compute Resources required by + this container. Cannot be updated. More + info: https://kubernetes.' + properties: + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + description: 'Limits describes the maximum + amount of compute resources allowed. + More info: https://kubernetes.' + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + description: Requests describes the minimum + amount of compute resources required. + type: object + type: object + securityContext: + description: SecurityContext defines the security + options the container should be run with. + properties: + allowPrivilegeEscalation: + description: AllowPrivilegeEscalation + controls whether a process can gain + more privileges than its parent process + type: boolean + capabilities: + description: The capabilities to add/drop + when running containers. + properties: + add: + description: Added capabilities + items: + description: Capability represent + POSIX capabilities type + type: string + type: array + drop: + description: Removed capabilities + items: + description: Capability represent + POSIX capabilities type + type: string + type: array + type: object + privileged: + description: Run container in privileged + mode. + type: boolean + procMount: + description: procMount denotes the type + of proc mount to use for the containers. + type: string + readOnlyRootFilesystem: + description: Whether this container has + a read-only root filesystem. Default + is false. + type: boolean + runAsGroup: + description: The GID to run the entrypoint + of the container process. Uses runtime + default if unset. + format: int64 + type: integer + runAsNonRoot: + description: Indicates that the container + must run as a non-root user. + type: boolean + runAsUser: + description: The UID to run the entrypoint + of the container process. + format: int64 + type: integer + seLinuxOptions: + description: The SELinux context to be + applied to the container. + properties: + level: + description: Level is SELinux level + label that applies to the container. + type: string + role: + description: Role is a SELinux role + label that applies to the container. + type: string + type: + description: Type is a SELinux type + label that applies to the container. + type: string + user: + description: User is a SELinux user + label that applies to the container. + type: string + type: object + seccompProfile: + description: The seccomp options to use + by this container. + properties: + localhostProfile: + description: localhostProfile indicates + a profile defined in a file on the + node should be used. + type: string + type: + description: type indicates which + kind of seccomp profile will be + applied. + type: string + required: + - type + type: object + windowsOptions: + description: The Windows specific settings + applied to all containers. + properties: + gmsaCredentialSpec: + description: GMSACredentialSpec is + where the GMSA admission webhook + (https://github. + type: string + gmsaCredentialSpecName: + description: GMSACredentialSpecName + is the name of the GMSA credential + spec to use. + type: string + hostProcess: + description: HostProcess determines + if a container should be run as + a 'Host Process' container. + type: boolean + runAsUserName: + description: The UserName in Windows + to run the entrypoint of the container + process. + type: string + type: object + type: object + startupProbe: + description: StartupProbe indicates that the + Pod has successfully initialized. + properties: + exec: + description: Exec specifies the action + to take. + properties: + command: + description: 'Command is the command + line to execute inside the container, + the working directory for the command ' + items: + type: string + type: array + type: object + failureThreshold: + description: Minimum consecutive failures + for the probe to be considered failed + after having succeeded. + format: int32 + type: integer + grpc: + description: GRPC specifies an action + involving a GRPC port. + properties: + port: + description: Port number of the gRPC + service. Number must be in the range + 1 to 65535. + format: int32 + type: integer + service: + description: Service is the name of + the service to place in the gRPC + HealthCheckRequest (see https://github. + type: string + required: + - port + type: object + httpGet: + description: HTTPGet specifies the http + request to perform. + properties: + host: + description: Host name to connect + to, defaults to the pod IP. + type: string + httpHeaders: + description: Custom headers to set + in the request. HTTP allows repeated + headers. + items: + description: HTTPHeader describes + a custom header to be used in + HTTP probes + properties: + name: + description: The header field + name + type: string + value: + description: The header field + value + type: string + required: + - name + - value + type: object + type: array + path: + description: Path to access on the + HTTP server. + type: string + port: + anyOf: + - type: integer + - type: string + description: Name or number of the + port to access on the container. + Number must be in the range 1 to + 65535. + x-kubernetes-int-or-string: true + scheme: + description: Scheme to use for connecting + to the host. Defaults to HTTP. + type: string + required: + - port + type: object + initialDelaySeconds: + description: Number of seconds after the + container has started before liveness + probes are initiated. + format: int32 + type: integer + periodSeconds: + description: How often (in seconds) to + perform the probe. Default to 10 seconds. + Minimum value is 1. + format: int32 + type: integer + successThreshold: + description: Minimum consecutive successes + for the probe to be considered successful + after having failed. + format: int32 + type: integer + tcpSocket: + description: TCPSocket specifies an action + involving a TCP port. + properties: + host: + description: 'Optional: Host name + to connect to, defaults to the pod + IP.' + type: string + port: + anyOf: + - type: integer + - type: string + description: Number or name of the + port to access on the container. + Number must be in the range 1 to + 65535. + x-kubernetes-int-or-string: true + required: + - port + type: object + terminationGracePeriodSeconds: + description: Optional duration in seconds + the pod needs to terminate gracefully + upon probe failure. + format: int64 + type: integer + timeoutSeconds: + description: Number of seconds after which + the probe times out. Defaults to 1 second. + Minimum value is 1. + format: int32 + type: integer + type: object + stdin: + description: Whether this container should + allocate a buffer for stdin in the container + runtime. + type: boolean + stdinOnce: + description: Whether the container runtime + should close the stdin channel after it + has been opened by a single at + type: boolean + terminationMessagePath: + description: 'Optional: Path at which the + file to which the container''s termination + message will be written is mou' + type: string + terminationMessagePolicy: + description: Indicate how the termination + message should be populated. + type: string + tty: + description: Whether this container should + allocate a TTY for itself, also requires + 'stdin' to be true. + type: boolean + volumeDevices: + description: volumeDevices is the list of + block devices to be used by the container. + items: + description: volumeDevice describes a mapping + of a raw block device within a container. + properties: + devicePath: + description: devicePath is the path + inside of the container that the device + will be mapped to. + type: string + name: + description: name must match the name + of a persistentVolumeClaim in the + pod + type: string + required: + - devicePath + - name + type: object + type: array + volumeMounts: + description: Pod volumes to mount into the + container's filesystem. Cannot be updated. + items: + description: VolumeMount describes a mounting + of a Volume within a container. + properties: + mountPath: + description: Path within the container + at which the volume should be mounted. Must + not contain ':'. + type: string + mountPropagation: + description: mountPropagation determines + how mounts are propagated from the + host to container and the other way + a + type: string + name: + description: This must match the Name + of a Volume. + type: string + readOnly: + description: Mounted read-only if true, + read-write otherwise (false or unspecified). + Defaults to false. + type: boolean + subPath: + description: Path within the volume + from which the container's volume + should be mounted. + type: string + subPathExpr: + description: Expanded path within the + volume from which the container's + volume should be mounted. + type: string + required: + - mountPath + - name + type: object + type: array + workingDir: + description: Container's working directory. + type: string + required: + - name + type: object + type: array + dnsConfig: + description: Specifies the DNS parameters of a pod. + properties: + nameservers: + description: A list of DNS name server IP addresses. + items: + type: string + type: array + options: + description: A list of DNS resolver options. + This will be merged with the base options + generated from DNSPolicy. + items: + description: PodDNSConfigOption defines DNS + resolver options of a pod. + properties: + name: + description: Required. + type: string + value: + type: string + type: object + type: array + searches: + description: A list of DNS search domains for + host-name lookup. + items: + type: string + type: array + type: object + dnsPolicy: + description: Set DNS policy for the pod. Defaults + to "ClusterFirst". + type: string + enableServiceLinks: + description: EnableServiceLinks indicates whether + information about services should be injected + into pod's enviro + type: boolean + ephemeralContainers: + description: List of ephemeral containers run in + this pod. + items: + description: An EphemeralContainer is a temporary + container that you may add to an existing Pod + for user-initiate + properties: + args: + description: Arguments to the entrypoint. + The docker image's CMD is used if this is + not provided. + items: + type: string + type: array + command: + description: Entrypoint array. Not executed + within a shell. + items: + type: string + type: array + env: + description: List of environment variables + to set in the container. Cannot be updated. + items: + description: EnvVar represents an environment + variable present in a Container. + properties: + name: + description: Name of the environment + variable. Must be a C_IDENTIFIER. + type: string + value: + description: Variable references $(VAR_NAME) + are expanded using the previously + defined environment variables in t + type: string + valueFrom: + description: Source for the environment + variable's value. Cannot be used if + value is not empty. + properties: + configMapKeyRef: + description: Selects a key of a + ConfigMap. + properties: + key: + description: The key to select. + type: string + name: + description: 'Name of the referent. + More info: https://kubernetes.' + type: string + optional: + description: Specify whether + the ConfigMap or its key must + be defined + type: boolean + required: + - key + type: object + fieldRef: + description: 'Selects a field of + the pod: supports metadata.name, + metadata.namespace, `metadata.' + properties: + apiVersion: + description: Version of the + schema the FieldPath is written + in terms of, defaults to "v1". + type: string + fieldPath: + description: Path of the field + to select in the specified + API version. + type: string + required: + - fieldPath + type: object + resourceFieldRef: + description: 'Selects a resource + of the container: only resources + limits and requests (limits.cpu, + limits.' + properties: + containerName: + description: 'Container name: + required for volumes, optional + for env vars' + type: string + divisor: + anyOf: + - type: integer + - type: string + description: Specifies the output + format of the exposed resources, + defaults to "1" + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: + description: 'Required: resource + to select' + type: string + required: + - resource + type: object + secretKeyRef: + description: Selects a key of a + secret in the pod's namespace + properties: + key: + description: The key of the + secret to select from. Must + be a valid secret key. + type: string + name: + description: 'Name of the referent. + More info: https://kubernetes.' + type: string + optional: + description: Specify whether + the Secret or its key must + be defined + type: boolean + required: + - key + type: object + type: object + required: + - name + type: object + type: array + envFrom: + description: List of sources to populate environment + variables in the container. + items: + description: EnvFromSource represents the + source of a set of ConfigMaps + properties: + configMapRef: + description: The ConfigMap to select + from + properties: + name: + description: 'Name of the referent. + More info: https://kubernetes.' + type: string + optional: + description: Specify whether the + ConfigMap must be defined + type: boolean + type: object + prefix: + description: An optional identifier + to prepend to each key in the ConfigMap. + Must be a C_IDENTIFIER. + type: string + secretRef: + description: The Secret to select from + properties: + name: + description: 'Name of the referent. + More info: https://kubernetes.' + type: string + optional: + description: Specify whether the + Secret must be defined + type: boolean + type: object + type: object + type: array + image: + description: 'Docker image name. More info: + https://kubernetes.io/docs/concepts/containers/images' + type: string + imagePullPolicy: + description: Image pull policy. One of Always, + Never, IfNotPresent. + type: string + lifecycle: + description: Lifecycle is not allowed for + ephemeral containers. + properties: + postStart: + description: PostStart is called immediately + after a container is created. + properties: + exec: + description: Exec specifies the action + to take. + properties: + command: + description: 'Command is the command + line to execute inside the container, + the working directory for the + command ' + items: + type: string + type: array + type: object + httpGet: + description: HTTPGet specifies the + http request to perform. + properties: + host: + description: Host name to connect + to, defaults to the pod IP. + type: string + httpHeaders: + description: Custom headers to + set in the request. HTTP allows + repeated headers. + items: + description: HTTPHeader describes + a custom header to be used + in HTTP probes + properties: + name: + description: The header + field name + type: string + value: + description: The header + field value + type: string + required: + - name + - value + type: object + type: array + path: + description: Path to access on + the HTTP server. + type: string + port: + anyOf: + - type: integer + - type: string + description: Name or number of + the port to access on the container. + Number must be in the range + 1 to 65535. + x-kubernetes-int-or-string: true + scheme: + description: Scheme to use for + connecting to the host. Defaults + to HTTP. + type: string + required: + - port + type: object + tcpSocket: + description: Deprecated. + properties: + host: + description: 'Optional: Host name + to connect to, defaults to the + pod IP.' + type: string + port: + anyOf: + - type: integer + - type: string + description: Number or name of + the port to access on the container. + Number must be in the range + 1 to 65535. + x-kubernetes-int-or-string: true + required: + - port + type: object + type: object + preStop: + description: PreStop is called immediately + before a container is terminated due + to an API request or management e + properties: + exec: + description: Exec specifies the action + to take. + properties: + command: + description: 'Command is the command + line to execute inside the container, + the working directory for the + command ' + items: + type: string + type: array + type: object + httpGet: + description: HTTPGet specifies the + http request to perform. + properties: + host: + description: Host name to connect + to, defaults to the pod IP. + type: string + httpHeaders: + description: Custom headers to + set in the request. HTTP allows + repeated headers. + items: + description: HTTPHeader describes + a custom header to be used + in HTTP probes + properties: + name: + description: The header + field name + type: string + value: + description: The header + field value + type: string + required: + - name + - value + type: object + type: array + path: + description: Path to access on + the HTTP server. + type: string + port: + anyOf: + - type: integer + - type: string + description: Name or number of + the port to access on the container. + Number must be in the range + 1 to 65535. + x-kubernetes-int-or-string: true + scheme: + description: Scheme to use for + connecting to the host. Defaults + to HTTP. + type: string + required: + - port + type: object + tcpSocket: + description: Deprecated. + properties: + host: + description: 'Optional: Host name + to connect to, defaults to the + pod IP.' + type: string + port: + anyOf: + - type: integer + - type: string + description: Number or name of + the port to access on the container. + Number must be in the range + 1 to 65535. + x-kubernetes-int-or-string: true + required: + - port + type: object + type: object + type: object + livenessProbe: + description: Probes are not allowed for ephemeral + containers. + properties: + exec: + description: Exec specifies the action + to take. + properties: + command: + description: 'Command is the command + line to execute inside the container, + the working directory for the command ' + items: + type: string + type: array + type: object + failureThreshold: + description: Minimum consecutive failures + for the probe to be considered failed + after having succeeded. + format: int32 + type: integer + grpc: + description: GRPC specifies an action + involving a GRPC port. + properties: + port: + description: Port number of the gRPC + service. Number must be in the range + 1 to 65535. + format: int32 + type: integer + service: + description: Service is the name of + the service to place in the gRPC + HealthCheckRequest (see https://github. + type: string + required: + - port + type: object + httpGet: + description: HTTPGet specifies the http + request to perform. + properties: + host: + description: Host name to connect + to, defaults to the pod IP. + type: string + httpHeaders: + description: Custom headers to set + in the request. HTTP allows repeated + headers. + items: + description: HTTPHeader describes + a custom header to be used in + HTTP probes + properties: + name: + description: The header field + name + type: string + value: + description: The header field + value + type: string + required: + - name + - value + type: object + type: array + path: + description: Path to access on the + HTTP server. + type: string + port: + anyOf: + - type: integer + - type: string + description: Name or number of the + port to access on the container. + Number must be in the range 1 to + 65535. + x-kubernetes-int-or-string: true + scheme: + description: Scheme to use for connecting + to the host. Defaults to HTTP. + type: string + required: + - port + type: object + initialDelaySeconds: + description: Number of seconds after the + container has started before liveness + probes are initiated. + format: int32 + type: integer + periodSeconds: + description: How often (in seconds) to + perform the probe. Default to 10 seconds. + Minimum value is 1. + format: int32 + type: integer + successThreshold: + description: Minimum consecutive successes + for the probe to be considered successful + after having failed. + format: int32 + type: integer + tcpSocket: + description: TCPSocket specifies an action + involving a TCP port. + properties: + host: + description: 'Optional: Host name + to connect to, defaults to the pod + IP.' + type: string + port: + anyOf: + - type: integer + - type: string + description: Number or name of the + port to access on the container. + Number must be in the range 1 to + 65535. + x-kubernetes-int-or-string: true + required: + - port + type: object + terminationGracePeriodSeconds: + description: Optional duration in seconds + the pod needs to terminate gracefully + upon probe failure. + format: int64 + type: integer + timeoutSeconds: + description: Number of seconds after which + the probe times out. Defaults to 1 second. + Minimum value is 1. + format: int32 + type: integer + type: object + name: + description: Name of the ephemeral container + specified as a DNS_LABEL. + type: string + ports: + description: Ports are not allowed for ephemeral + containers. + items: + description: ContainerPort represents a + network port in a single container. + properties: + containerPort: + description: Number of port to expose + on the pod's IP address. This must + be a valid port number, 0 < x < 65536. + format: int32 + type: integer + hostIP: + description: What host IP to bind the + external port to. + type: string + hostPort: + description: Number of port to expose + on the host. If specified, this must + be a valid port number, 0 < x < 65536. + format: int32 + type: integer + name: + description: If specified, this must + be an IANA_SVC_NAME and unique within + the pod. + type: string + protocol: + default: TCP + description: Protocol for port. Must + be UDP, TCP, or SCTP. Defaults to + "TCP". + type: string + required: + - containerPort + type: object + type: array + x-kubernetes-list-map-keys: + - containerPort + - protocol + x-kubernetes-list-type: map + readinessProbe: + description: Probes are not allowed for ephemeral + containers. + properties: + exec: + description: Exec specifies the action + to take. + properties: + command: + description: 'Command is the command + line to execute inside the container, + the working directory for the command ' + items: + type: string + type: array + type: object + failureThreshold: + description: Minimum consecutive failures + for the probe to be considered failed + after having succeeded. + format: int32 + type: integer + grpc: + description: GRPC specifies an action + involving a GRPC port. + properties: + port: + description: Port number of the gRPC + service. Number must be in the range + 1 to 65535. + format: int32 + type: integer + service: + description: Service is the name of + the service to place in the gRPC + HealthCheckRequest (see https://github. + type: string + required: + - port + type: object + httpGet: + description: HTTPGet specifies the http + request to perform. + properties: + host: + description: Host name to connect + to, defaults to the pod IP. + type: string + httpHeaders: + description: Custom headers to set + in the request. HTTP allows repeated + headers. + items: + description: HTTPHeader describes + a custom header to be used in + HTTP probes + properties: + name: + description: The header field + name + type: string + value: + description: The header field + value + type: string + required: + - name + - value + type: object + type: array + path: + description: Path to access on the + HTTP server. + type: string + port: + anyOf: + - type: integer + - type: string + description: Name or number of the + port to access on the container. + Number must be in the range 1 to + 65535. + x-kubernetes-int-or-string: true + scheme: + description: Scheme to use for connecting + to the host. Defaults to HTTP. + type: string + required: + - port + type: object + initialDelaySeconds: + description: Number of seconds after the + container has started before liveness + probes are initiated. + format: int32 + type: integer + periodSeconds: + description: How often (in seconds) to + perform the probe. Default to 10 seconds. + Minimum value is 1. + format: int32 + type: integer + successThreshold: + description: Minimum consecutive successes + for the probe to be considered successful + after having failed. + format: int32 + type: integer + tcpSocket: + description: TCPSocket specifies an action + involving a TCP port. + properties: + host: + description: 'Optional: Host name + to connect to, defaults to the pod + IP.' + type: string + port: + anyOf: + - type: integer + - type: string + description: Number or name of the + port to access on the container. + Number must be in the range 1 to + 65535. + x-kubernetes-int-or-string: true + required: + - port + type: object + terminationGracePeriodSeconds: + description: Optional duration in seconds + the pod needs to terminate gracefully + upon probe failure. + format: int64 + type: integer + timeoutSeconds: + description: Number of seconds after which + the probe times out. Defaults to 1 second. + Minimum value is 1. + format: int32 + type: integer + type: object + resources: + description: Resources are not allowed for + ephemeral containers. + properties: + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + description: 'Limits describes the maximum + amount of compute resources allowed. + More info: https://kubernetes.' + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + description: Requests describes the minimum + amount of compute resources required. + type: object + type: object + securityContext: + description: 'Optional: SecurityContext defines + the security options the ephemeral container + should be run with.' + properties: + allowPrivilegeEscalation: + description: AllowPrivilegeEscalation + controls whether a process can gain + more privileges than its parent process + type: boolean + capabilities: + description: The capabilities to add/drop + when running containers. + properties: + add: + description: Added capabilities + items: + description: Capability represent + POSIX capabilities type + type: string + type: array + drop: + description: Removed capabilities + items: + description: Capability represent + POSIX capabilities type + type: string + type: array + type: object + privileged: + description: Run container in privileged + mode. + type: boolean + procMount: + description: procMount denotes the type + of proc mount to use for the containers. + type: string + readOnlyRootFilesystem: + description: Whether this container has + a read-only root filesystem. Default + is false. + type: boolean + runAsGroup: + description: The GID to run the entrypoint + of the container process. Uses runtime + default if unset. + format: int64 + type: integer + runAsNonRoot: + description: Indicates that the container + must run as a non-root user. + type: boolean + runAsUser: + description: The UID to run the entrypoint + of the container process. + format: int64 + type: integer + seLinuxOptions: + description: The SELinux context to be + applied to the container. + properties: + level: + description: Level is SELinux level + label that applies to the container. + type: string + role: + description: Role is a SELinux role + label that applies to the container. + type: string + type: + description: Type is a SELinux type + label that applies to the container. + type: string + user: + description: User is a SELinux user + label that applies to the container. + type: string + type: object + seccompProfile: + description: The seccomp options to use + by this container. + properties: + localhostProfile: + description: localhostProfile indicates + a profile defined in a file on the + node should be used. + type: string + type: + description: type indicates which + kind of seccomp profile will be + applied. + type: string + required: + - type + type: object + windowsOptions: + description: The Windows specific settings + applied to all containers. + properties: + gmsaCredentialSpec: + description: GMSACredentialSpec is + where the GMSA admission webhook + (https://github. + type: string + gmsaCredentialSpecName: + description: GMSACredentialSpecName + is the name of the GMSA credential + spec to use. + type: string + hostProcess: + description: HostProcess determines + if a container should be run as + a 'Host Process' container. + type: boolean + runAsUserName: + description: The UserName in Windows + to run the entrypoint of the container + process. + type: string + type: object + type: object + startupProbe: + description: Probes are not allowed for ephemeral + containers. + properties: + exec: + description: Exec specifies the action + to take. + properties: + command: + description: 'Command is the command + line to execute inside the container, + the working directory for the command ' + items: + type: string + type: array + type: object + failureThreshold: + description: Minimum consecutive failures + for the probe to be considered failed + after having succeeded. + format: int32 + type: integer + grpc: + description: GRPC specifies an action + involving a GRPC port. + properties: + port: + description: Port number of the gRPC + service. Number must be in the range + 1 to 65535. + format: int32 + type: integer + service: + description: Service is the name of + the service to place in the gRPC + HealthCheckRequest (see https://github. + type: string + required: + - port + type: object + httpGet: + description: HTTPGet specifies the http + request to perform. + properties: + host: + description: Host name to connect + to, defaults to the pod IP. + type: string + httpHeaders: + description: Custom headers to set + in the request. HTTP allows repeated + headers. + items: + description: HTTPHeader describes + a custom header to be used in + HTTP probes + properties: + name: + description: The header field + name + type: string + value: + description: The header field + value + type: string + required: + - name + - value + type: object + type: array + path: + description: Path to access on the + HTTP server. + type: string + port: + anyOf: + - type: integer + - type: string + description: Name or number of the + port to access on the container. + Number must be in the range 1 to + 65535. + x-kubernetes-int-or-string: true + scheme: + description: Scheme to use for connecting + to the host. Defaults to HTTP. + type: string + required: + - port + type: object + initialDelaySeconds: + description: Number of seconds after the + container has started before liveness + probes are initiated. + format: int32 + type: integer + periodSeconds: + description: How often (in seconds) to + perform the probe. Default to 10 seconds. + Minimum value is 1. + format: int32 + type: integer + successThreshold: + description: Minimum consecutive successes + for the probe to be considered successful + after having failed. + format: int32 + type: integer + tcpSocket: + description: TCPSocket specifies an action + involving a TCP port. + properties: + host: + description: 'Optional: Host name + to connect to, defaults to the pod + IP.' + type: string + port: + anyOf: + - type: integer + - type: string + description: Number or name of the + port to access on the container. + Number must be in the range 1 to + 65535. + x-kubernetes-int-or-string: true + required: + - port + type: object + terminationGracePeriodSeconds: + description: Optional duration in seconds + the pod needs to terminate gracefully + upon probe failure. + format: int64 + type: integer + timeoutSeconds: + description: Number of seconds after which + the probe times out. Defaults to 1 second. + Minimum value is 1. + format: int32 + type: integer + type: object + stdin: + description: Whether this container should + allocate a buffer for stdin in the container + runtime. + type: boolean + stdinOnce: + description: Whether the container runtime + should close the stdin channel after it + has been opened by a single at + type: boolean + targetContainerName: + description: If set, the name of the container + from PodSpec that this ephemeral container + targets. + type: string + terminationMessagePath: + description: 'Optional: Path at which the + file to which the container''s termination + message will be written is mou' + type: string + terminationMessagePolicy: + description: Indicate how the termination + message should be populated. + type: string + tty: + description: Whether this container should + allocate a TTY for itself, also requires + 'stdin' to be true. + type: boolean + volumeDevices: + description: volumeDevices is the list of + block devices to be used by the container. + items: + description: volumeDevice describes a mapping + of a raw block device within a container. + properties: + devicePath: + description: devicePath is the path + inside of the container that the device + will be mapped to. + type: string + name: + description: name must match the name + of a persistentVolumeClaim in the + pod + type: string + required: + - devicePath + - name + type: object + type: array + volumeMounts: + description: Pod volumes to mount into the + container's filesystem. + items: + description: VolumeMount describes a mounting + of a Volume within a container. + properties: + mountPath: + description: Path within the container + at which the volume should be mounted. Must + not contain ':'. + type: string + mountPropagation: + description: mountPropagation determines + how mounts are propagated from the + host to container and the other way + a + type: string + name: + description: This must match the Name + of a Volume. + type: string + readOnly: + description: Mounted read-only if true, + read-write otherwise (false or unspecified). + Defaults to false. + type: boolean + subPath: + description: Path within the volume + from which the container's volume + should be mounted. + type: string + subPathExpr: + description: Expanded path within the + volume from which the container's + volume should be mounted. + type: string + required: + - mountPath + - name + type: object + type: array + workingDir: + description: Container's working directory. + type: string + required: + - name + type: object + type: array + hostAliases: + description: 'HostAliases is an optional list of + hosts and IPs that will be injected into the pod''s + hosts file if ' + items: + description: 'HostAlias holds the mapping between + IP and hostnames that will be injected as an + entry in the pod''s ' + properties: + hostnames: + description: Hostnames for the above IP address. + items: + type: string + type: array + ip: + description: IP address of the host file entry. + type: string + type: object + type: array + hostIPC: + description: 'Use the host''s ipc namespace. Optional: + Default to false.' + type: boolean + hostNetwork: + description: Host networking requested for this + pod. Use the host's network namespace. + type: boolean + hostPID: + description: 'Use the host''s pid namespace. Optional: + Default to false.' + type: boolean + hostname: + description: Specifies the hostname of the Pod If + not specified, the pod's hostname will be set + to a system-defin + type: string + imagePullSecrets: + description: ImagePullSecrets is an optional list + of references to secrets in the same namespace + to use for pulli + items: + description: 'LocalObjectReference contains enough + information to let you locate the referenced + object inside the ' + properties: + name: + description: 'Name of the referent. More info: + https://kubernetes.' + type: string + type: object + type: array + initContainers: + description: List of initialization containers belonging + to the pod. + items: + description: A single application container that + you want to run within a pod. + properties: + args: + description: Arguments to the entrypoint. + The docker image's CMD is used if this is + not provided. + items: + type: string + type: array + command: + description: Entrypoint array. Not executed + within a shell. + items: + type: string + type: array + env: + description: List of environment variables + to set in the container. Cannot be updated. + items: + description: EnvVar represents an environment + variable present in a Container. + properties: + name: + description: Name of the environment + variable. Must be a C_IDENTIFIER. + type: string + value: + description: Variable references $(VAR_NAME) + are expanded using the previously + defined environment variables in t + type: string + valueFrom: + description: Source for the environment + variable's value. Cannot be used if + value is not empty. + properties: + configMapKeyRef: + description: Selects a key of a + ConfigMap. + properties: + key: + description: The key to select. + type: string + name: + description: 'Name of the referent. + More info: https://kubernetes.' + type: string + optional: + description: Specify whether + the ConfigMap or its key must + be defined + type: boolean + required: + - key + type: object + fieldRef: + description: 'Selects a field of + the pod: supports metadata.name, + metadata.namespace, `metadata.' + properties: + apiVersion: + description: Version of the + schema the FieldPath is written + in terms of, defaults to "v1". + type: string + fieldPath: + description: Path of the field + to select in the specified + API version. + type: string + required: + - fieldPath + type: object + resourceFieldRef: + description: 'Selects a resource + of the container: only resources + limits and requests (limits.cpu, + limits.' + properties: + containerName: + description: 'Container name: + required for volumes, optional + for env vars' + type: string + divisor: + anyOf: + - type: integer + - type: string + description: Specifies the output + format of the exposed resources, + defaults to "1" + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: + description: 'Required: resource + to select' + type: string + required: + - resource + type: object + secretKeyRef: + description: Selects a key of a + secret in the pod's namespace + properties: + key: + description: The key of the + secret to select from. Must + be a valid secret key. + type: string + name: + description: 'Name of the referent. + More info: https://kubernetes.' + type: string + optional: + description: Specify whether + the Secret or its key must + be defined + type: boolean + required: + - key + type: object + type: object + required: + - name + type: object + type: array + envFrom: + description: List of sources to populate environment + variables in the container. + items: + description: EnvFromSource represents the + source of a set of ConfigMaps + properties: + configMapRef: + description: The ConfigMap to select + from + properties: + name: + description: 'Name of the referent. + More info: https://kubernetes.' + type: string + optional: + description: Specify whether the + ConfigMap must be defined + type: boolean + type: object + prefix: + description: An optional identifier + to prepend to each key in the ConfigMap. + Must be a C_IDENTIFIER. + type: string + secretRef: + description: The Secret to select from + properties: + name: + description: 'Name of the referent. + More info: https://kubernetes.' + type: string + optional: + description: Specify whether the + Secret must be defined + type: boolean + type: object + type: object + type: array + image: + description: 'Docker image name. More info: + https://kubernetes.' + type: string + imagePullPolicy: + description: Image pull policy. One of Always, + Never, IfNotPresent. + type: string + lifecycle: + description: Actions that the management system + should take in response to container lifecycle + events. + properties: + postStart: + description: PostStart is called immediately + after a container is created. + properties: + exec: + description: Exec specifies the action + to take. + properties: + command: + description: 'Command is the command + line to execute inside the container, + the working directory for the + command ' + items: + type: string + type: array + type: object + httpGet: + description: HTTPGet specifies the + http request to perform. + properties: + host: + description: Host name to connect + to, defaults to the pod IP. + type: string + httpHeaders: + description: Custom headers to + set in the request. HTTP allows + repeated headers. + items: + description: HTTPHeader describes + a custom header to be used + in HTTP probes + properties: + name: + description: The header + field name + type: string + value: + description: The header + field value + type: string + required: + - name + - value + type: object + type: array + path: + description: Path to access on + the HTTP server. + type: string + port: + anyOf: + - type: integer + - type: string + description: Name or number of + the port to access on the container. + Number must be in the range + 1 to 65535. + x-kubernetes-int-or-string: true + scheme: + description: Scheme to use for + connecting to the host. Defaults + to HTTP. + type: string + required: + - port + type: object + tcpSocket: + description: Deprecated. + properties: + host: + description: 'Optional: Host name + to connect to, defaults to the + pod IP.' + type: string + port: + anyOf: + - type: integer + - type: string + description: Number or name of + the port to access on the container. + Number must be in the range + 1 to 65535. + x-kubernetes-int-or-string: true + required: + - port + type: object + type: object + preStop: + description: PreStop is called immediately + before a container is terminated due + to an API request or management e + properties: + exec: + description: Exec specifies the action + to take. + properties: + command: + description: 'Command is the command + line to execute inside the container, + the working directory for the + command ' + items: + type: string + type: array + type: object + httpGet: + description: HTTPGet specifies the + http request to perform. + properties: + host: + description: Host name to connect + to, defaults to the pod IP. + type: string + httpHeaders: + description: Custom headers to + set in the request. HTTP allows + repeated headers. + items: + description: HTTPHeader describes + a custom header to be used + in HTTP probes + properties: + name: + description: The header + field name + type: string + value: + description: The header + field value + type: string + required: + - name + - value + type: object + type: array + path: + description: Path to access on + the HTTP server. + type: string + port: + anyOf: + - type: integer + - type: string + description: Name or number of + the port to access on the container. + Number must be in the range + 1 to 65535. + x-kubernetes-int-or-string: true + scheme: + description: Scheme to use for + connecting to the host. Defaults + to HTTP. + type: string + required: + - port + type: object + tcpSocket: + description: Deprecated. + properties: + host: + description: 'Optional: Host name + to connect to, defaults to the + pod IP.' + type: string + port: + anyOf: + - type: integer + - type: string + description: Number or name of + the port to access on the container. + Number must be in the range + 1 to 65535. + x-kubernetes-int-or-string: true + required: + - port + type: object + type: object + type: object + livenessProbe: + description: Periodic probe of container liveness. + Container will be restarted if the probe + fails. + properties: + exec: + description: Exec specifies the action + to take. + properties: + command: + description: 'Command is the command + line to execute inside the container, + the working directory for the command ' + items: + type: string + type: array + type: object + failureThreshold: + description: Minimum consecutive failures + for the probe to be considered failed + after having succeeded. + format: int32 + type: integer + grpc: + description: GRPC specifies an action + involving a GRPC port. + properties: + port: + description: Port number of the gRPC + service. Number must be in the range + 1 to 65535. + format: int32 + type: integer + service: + description: Service is the name of + the service to place in the gRPC + HealthCheckRequest (see https://github. + type: string + required: + - port + type: object + httpGet: + description: HTTPGet specifies the http + request to perform. + properties: + host: + description: Host name to connect + to, defaults to the pod IP. + type: string + httpHeaders: + description: Custom headers to set + in the request. HTTP allows repeated + headers. + items: + description: HTTPHeader describes + a custom header to be used in + HTTP probes + properties: + name: + description: The header field + name + type: string + value: + description: The header field + value + type: string + required: + - name + - value + type: object + type: array + path: + description: Path to access on the + HTTP server. + type: string + port: + anyOf: + - type: integer + - type: string + description: Name or number of the + port to access on the container. + Number must be in the range 1 to + 65535. + x-kubernetes-int-or-string: true + scheme: + description: Scheme to use for connecting + to the host. Defaults to HTTP. + type: string + required: + - port + type: object + initialDelaySeconds: + description: Number of seconds after the + container has started before liveness + probes are initiated. + format: int32 + type: integer + periodSeconds: + description: How often (in seconds) to + perform the probe. Default to 10 seconds. + Minimum value is 1. + format: int32 + type: integer + successThreshold: + description: Minimum consecutive successes + for the probe to be considered successful + after having failed. + format: int32 + type: integer + tcpSocket: + description: TCPSocket specifies an action + involving a TCP port. + properties: + host: + description: 'Optional: Host name + to connect to, defaults to the pod + IP.' + type: string + port: + anyOf: + - type: integer + - type: string + description: Number or name of the + port to access on the container. + Number must be in the range 1 to + 65535. + x-kubernetes-int-or-string: true + required: + - port + type: object + terminationGracePeriodSeconds: + description: Optional duration in seconds + the pod needs to terminate gracefully + upon probe failure. + format: int64 + type: integer + timeoutSeconds: + description: Number of seconds after which + the probe times out. Defaults to 1 second. + Minimum value is 1. + format: int32 + type: integer + type: object + name: + description: Name of the container specified + as a DNS_LABEL. + type: string + ports: + description: List of ports to expose from + the container. + items: + description: ContainerPort represents a + network port in a single container. + properties: + containerPort: + description: Number of port to expose + on the pod's IP address. This must + be a valid port number, 0 < x < 65536. + format: int32 + type: integer + hostIP: + description: What host IP to bind the + external port to. + type: string + hostPort: + description: Number of port to expose + on the host. If specified, this must + be a valid port number, 0 < x < 65536. + format: int32 + type: integer + name: + description: If specified, this must + be an IANA_SVC_NAME and unique within + the pod. + type: string + protocol: + default: TCP + description: Protocol for port. Must + be UDP, TCP, or SCTP. Defaults to + "TCP". + type: string + required: + - containerPort + type: object + type: array + x-kubernetes-list-map-keys: + - containerPort + - protocol + x-kubernetes-list-type: map + readinessProbe: + description: Periodic probe of container service + readiness. + properties: + exec: + description: Exec specifies the action + to take. + properties: + command: + description: 'Command is the command + line to execute inside the container, + the working directory for the command ' + items: + type: string + type: array + type: object + failureThreshold: + description: Minimum consecutive failures + for the probe to be considered failed + after having succeeded. + format: int32 + type: integer + grpc: + description: GRPC specifies an action + involving a GRPC port. + properties: + port: + description: Port number of the gRPC + service. Number must be in the range + 1 to 65535. + format: int32 + type: integer + service: + description: Service is the name of + the service to place in the gRPC + HealthCheckRequest (see https://github. + type: string + required: + - port + type: object + httpGet: + description: HTTPGet specifies the http + request to perform. + properties: + host: + description: Host name to connect + to, defaults to the pod IP. + type: string + httpHeaders: + description: Custom headers to set + in the request. HTTP allows repeated + headers. + items: + description: HTTPHeader describes + a custom header to be used in + HTTP probes + properties: + name: + description: The header field + name + type: string + value: + description: The header field + value + type: string + required: + - name + - value + type: object + type: array + path: + description: Path to access on the + HTTP server. + type: string + port: + anyOf: + - type: integer + - type: string + description: Name or number of the + port to access on the container. + Number must be in the range 1 to + 65535. + x-kubernetes-int-or-string: true + scheme: + description: Scheme to use for connecting + to the host. Defaults to HTTP. + type: string + required: + - port + type: object + initialDelaySeconds: + description: Number of seconds after the + container has started before liveness + probes are initiated. + format: int32 + type: integer + periodSeconds: + description: How often (in seconds) to + perform the probe. Default to 10 seconds. + Minimum value is 1. + format: int32 + type: integer + successThreshold: + description: Minimum consecutive successes + for the probe to be considered successful + after having failed. + format: int32 + type: integer + tcpSocket: + description: TCPSocket specifies an action + involving a TCP port. + properties: + host: + description: 'Optional: Host name + to connect to, defaults to the pod + IP.' + type: string + port: + anyOf: + - type: integer + - type: string + description: Number or name of the + port to access on the container. + Number must be in the range 1 to + 65535. + x-kubernetes-int-or-string: true + required: + - port + type: object + terminationGracePeriodSeconds: + description: Optional duration in seconds + the pod needs to terminate gracefully + upon probe failure. + format: int64 + type: integer + timeoutSeconds: + description: Number of seconds after which + the probe times out. Defaults to 1 second. + Minimum value is 1. + format: int32 + type: integer + type: object + resources: + description: 'Compute Resources required by + this container. Cannot be updated. More + info: https://kubernetes.' + properties: + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + description: 'Limits describes the maximum + amount of compute resources allowed. + More info: https://kubernetes.' + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + description: Requests describes the minimum + amount of compute resources required. + type: object + type: object + securityContext: + description: SecurityContext defines the security + options the container should be run with. + properties: + allowPrivilegeEscalation: + description: AllowPrivilegeEscalation + controls whether a process can gain + more privileges than its parent process + type: boolean + capabilities: + description: The capabilities to add/drop + when running containers. + properties: + add: + description: Added capabilities + items: + description: Capability represent + POSIX capabilities type + type: string + type: array + drop: + description: Removed capabilities + items: + description: Capability represent + POSIX capabilities type + type: string + type: array + type: object + privileged: + description: Run container in privileged + mode. + type: boolean + procMount: + description: procMount denotes the type + of proc mount to use for the containers. + type: string + readOnlyRootFilesystem: + description: Whether this container has + a read-only root filesystem. Default + is false. + type: boolean + runAsGroup: + description: The GID to run the entrypoint + of the container process. Uses runtime + default if unset. + format: int64 + type: integer + runAsNonRoot: + description: Indicates that the container + must run as a non-root user. + type: boolean + runAsUser: + description: The UID to run the entrypoint + of the container process. + format: int64 + type: integer + seLinuxOptions: + description: The SELinux context to be + applied to the container. + properties: + level: + description: Level is SELinux level + label that applies to the container. + type: string + role: + description: Role is a SELinux role + label that applies to the container. + type: string + type: + description: Type is a SELinux type + label that applies to the container. + type: string + user: + description: User is a SELinux user + label that applies to the container. + type: string + type: object + seccompProfile: + description: The seccomp options to use + by this container. + properties: + localhostProfile: + description: localhostProfile indicates + a profile defined in a file on the + node should be used. + type: string + type: + description: type indicates which + kind of seccomp profile will be + applied. + type: string + required: + - type + type: object + windowsOptions: + description: The Windows specific settings + applied to all containers. + properties: + gmsaCredentialSpec: + description: GMSACredentialSpec is + where the GMSA admission webhook + (https://github. + type: string + gmsaCredentialSpecName: + description: GMSACredentialSpecName + is the name of the GMSA credential + spec to use. + type: string + hostProcess: + description: HostProcess determines + if a container should be run as + a 'Host Process' container. + type: boolean + runAsUserName: + description: The UserName in Windows + to run the entrypoint of the container + process. + type: string + type: object + type: object + startupProbe: + description: StartupProbe indicates that the + Pod has successfully initialized. + properties: + exec: + description: Exec specifies the action + to take. + properties: + command: + description: 'Command is the command + line to execute inside the container, + the working directory for the command ' + items: + type: string + type: array + type: object + failureThreshold: + description: Minimum consecutive failures + for the probe to be considered failed + after having succeeded. + format: int32 + type: integer + grpc: + description: GRPC specifies an action + involving a GRPC port. + properties: + port: + description: Port number of the gRPC + service. Number must be in the range + 1 to 65535. + format: int32 + type: integer + service: + description: Service is the name of + the service to place in the gRPC + HealthCheckRequest (see https://github. + type: string + required: + - port + type: object + httpGet: + description: HTTPGet specifies the http + request to perform. + properties: + host: + description: Host name to connect + to, defaults to the pod IP. + type: string + httpHeaders: + description: Custom headers to set + in the request. HTTP allows repeated + headers. + items: + description: HTTPHeader describes + a custom header to be used in + HTTP probes + properties: + name: + description: The header field + name + type: string + value: + description: The header field + value + type: string + required: + - name + - value + type: object + type: array + path: + description: Path to access on the + HTTP server. + type: string + port: + anyOf: + - type: integer + - type: string + description: Name or number of the + port to access on the container. + Number must be in the range 1 to + 65535. + x-kubernetes-int-or-string: true + scheme: + description: Scheme to use for connecting + to the host. Defaults to HTTP. + type: string + required: + - port + type: object + initialDelaySeconds: + description: Number of seconds after the + container has started before liveness + probes are initiated. + format: int32 + type: integer + periodSeconds: + description: How often (in seconds) to + perform the probe. Default to 10 seconds. + Minimum value is 1. + format: int32 + type: integer + successThreshold: + description: Minimum consecutive successes + for the probe to be considered successful + after having failed. + format: int32 + type: integer + tcpSocket: + description: TCPSocket specifies an action + involving a TCP port. + properties: + host: + description: 'Optional: Host name + to connect to, defaults to the pod + IP.' + type: string + port: + anyOf: + - type: integer + - type: string + description: Number or name of the + port to access on the container. + Number must be in the range 1 to + 65535. + x-kubernetes-int-or-string: true + required: + - port + type: object + terminationGracePeriodSeconds: + description: Optional duration in seconds + the pod needs to terminate gracefully + upon probe failure. + format: int64 + type: integer + timeoutSeconds: + description: Number of seconds after which + the probe times out. Defaults to 1 second. + Minimum value is 1. + format: int32 + type: integer + type: object + stdin: + description: Whether this container should + allocate a buffer for stdin in the container + runtime. + type: boolean + stdinOnce: + description: Whether the container runtime + should close the stdin channel after it + has been opened by a single at + type: boolean + terminationMessagePath: + description: 'Optional: Path at which the + file to which the container''s termination + message will be written is mou' + type: string + terminationMessagePolicy: + description: Indicate how the termination + message should be populated. + type: string + tty: + description: Whether this container should + allocate a TTY for itself, also requires + 'stdin' to be true. + type: boolean + volumeDevices: + description: volumeDevices is the list of + block devices to be used by the container. + items: + description: volumeDevice describes a mapping + of a raw block device within a container. + properties: + devicePath: + description: devicePath is the path + inside of the container that the device + will be mapped to. + type: string + name: + description: name must match the name + of a persistentVolumeClaim in the + pod + type: string + required: + - devicePath + - name + type: object + type: array + volumeMounts: + description: Pod volumes to mount into the + container's filesystem. Cannot be updated. + items: + description: VolumeMount describes a mounting + of a Volume within a container. + properties: + mountPath: + description: Path within the container + at which the volume should be mounted. Must + not contain ':'. + type: string + mountPropagation: + description: mountPropagation determines + how mounts are propagated from the + host to container and the other way + a + type: string + name: + description: This must match the Name + of a Volume. + type: string + readOnly: + description: Mounted read-only if true, + read-write otherwise (false or unspecified). + Defaults to false. + type: boolean + subPath: + description: Path within the volume + from which the container's volume + should be mounted. + type: string + subPathExpr: + description: Expanded path within the + volume from which the container's + volume should be mounted. + type: string + required: + - mountPath + - name + type: object + type: array + workingDir: + description: Container's working directory. + type: string + required: + - name + type: object + type: array + nodeName: + description: NodeName is a request to schedule this + pod onto a specific node. + type: string + nodeSelector: + additionalProperties: + type: string + description: NodeSelector is a selector which must + be true for the pod to fit on a node. + type: object + x-kubernetes-map-type: atomic + os: + description: Specifies the OS of the containers + in the pod. + properties: + name: + description: Name is the name of the operating + system. The currently supported values are + linux and windows. + type: string + required: + - name + type: object + overhead: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + description: Overhead represents the resource overhead + associated with running a pod for a given RuntimeClass. + type: object + preemptionPolicy: + description: PreemptionPolicy is the Policy for + preempting pods with lower priority. + type: string + priority: + description: The priority value. Various system + components use this field to find the priority + of the pod. + format: int32 + type: integer + priorityClassName: + description: If specified, indicates the pod's priority. + type: string + readinessGates: + description: If specified, all readiness gates will + be evaluated for pod readiness. + items: + description: PodReadinessGate contains the reference + to a pod condition + properties: + conditionType: + description: ConditionType refers to a condition + in the pod's condition list with matching + type. + type: string + required: + - conditionType + type: object + type: array + restartPolicy: + description: Restart policy for all containers within + the pod. One of Always, OnFailure, Never. + type: string + runtimeClassName: + description: RuntimeClassName refers to a RuntimeClass + object in the node.k8s. + type: string + schedulerName: + description: If specified, the pod will be dispatched + by specified scheduler. + type: string + securityContext: + description: SecurityContext holds pod-level security + attributes and common container settings. + properties: + fsGroup: + description: A special supplemental group that + applies to all containers in a pod. + format: int64 + type: integer + fsGroupChangePolicy: + description: fsGroupChangePolicy defines behavior + of changing ownership and permission of the + volume before being + type: string + runAsGroup: + description: The GID to run the entrypoint of + the container process. Uses runtime default + if unset. + format: int64 + type: integer + runAsNonRoot: + description: Indicates that the container must + run as a non-root user. + type: boolean + runAsUser: + description: The UID to run the entrypoint of + the container process. + format: int64 + type: integer + seLinuxOptions: + description: The SELinux context to be applied + to all containers. + properties: + level: + description: Level is SELinux level label + that applies to the container. + type: string + role: + description: Role is a SELinux role label + that applies to the container. + type: string + type: + description: Type is a SELinux type label + that applies to the container. + type: string + user: + description: User is a SELinux user label + that applies to the container. + type: string + type: object + seccompProfile: + description: The seccomp options to use by the + containers in this pod. + properties: + localhostProfile: + description: localhostProfile indicates + a profile defined in a file on the node + should be used. + type: string + type: + description: type indicates which kind of + seccomp profile will be applied. + type: string + required: + - type + type: object + supplementalGroups: + description: 'A list of groups applied to the + first process run in each container, in addition + to the container''s ' + items: + format: int64 + type: integer + type: array + sysctls: + description: Sysctls hold a list of namespaced + sysctls used for the pod. + items: + description: Sysctl defines a kernel parameter + to be set + properties: + name: + description: Name of a property to set + type: string + value: + description: Value of a property to set + type: string + required: + - name + - value + type: object + type: array + windowsOptions: + description: The Windows specific settings applied + to all containers. + properties: + gmsaCredentialSpec: + description: GMSACredentialSpec is where + the GMSA admission webhook (https://github. + type: string + gmsaCredentialSpecName: + description: GMSACredentialSpecName is the + name of the GMSA credential spec to use. + type: string + hostProcess: + description: HostProcess determines if a + container should be run as a 'Host Process' + container. + type: boolean + runAsUserName: + description: The UserName in Windows to + run the entrypoint of the container process. + type: string + type: object + type: object + serviceAccount: + description: DeprecatedServiceAccount is a depreciated + alias for ServiceAccountName. + type: string + serviceAccountName: + description: ServiceAccountName is the name of the + ServiceAccount to use to run this pod. + type: string + setHostnameAsFQDN: + description: If true the pod's hostname will be + configured as the pod's FQDN, rather than the + leaf name (the defa + type: boolean + shareProcessNamespace: + description: Share a single process namespace between + all of the containers in a pod. + type: boolean + subdomain: + description: If specified, the fully qualified Pod + hostname will be "...svc. + type: string + terminationGracePeriodSeconds: + description: Optional duration in seconds the pod + needs to terminate gracefully. + format: int64 + type: integer + tolerations: + description: If specified, the pod's tolerations. + items: + description: The pod this Toleration is attached + to tolerates any taint that matches the triple + name on the dataset for + Flocker should be considered as de + type: string + datasetUUID: + description: UUID of the dataset. This + is unique identifier of a Flocker dataset + type: string + type: object + gcePersistentDisk: + description: GCEPersistentDisk represents + a GCE Disk resource that is attached to + a kubelet's host machine and th + properties: + fsType: + description: Filesystem type of the volume + that you want to mount. + type: string + partition: + description: The partition in the volume + that you want to mount. + format: int32 + type: integer + pdName: + description: Unique name of the PD resource + in GCE. Used to identify the disk in + GCE. + type: string + readOnly: + description: ReadOnly here will force + the ReadOnly setting in VolumeMounts. + Defaults to false. + type: boolean + required: + - pdName + type: object + gitRepo: + description: 'GitRepo represents a git repository + at a particular revision. DEPRECATED: GitRepo + is deprecated.' + properties: + directory: + description: Target directory name. Must + not contain or start with '..'. If + '. + type: string + repository: + description: Repository URL + type: string + revision: + description: Commit hash for the specified + revision. + type: string + required: + - repository + type: object + glusterfs: + description: Glusterfs represents a Glusterfs + mount on the host that shares a pod's lifetime. + properties: + endpoints: + description: 'EndpointsName is the endpoint + name that details Glusterfs topology. + More info: https://examples.k8s.' + type: string + path: + description: 'Path is the Glusterfs volume + path. More info: https://examples.k8s.io/volumes/glusterfs/README.' + type: string + readOnly: + description: ReadOnly here will force + the Glusterfs volume to be mounted with + read-only permissions. + type: boolean + required: + - endpoints + - path + type: object + hostPath: + description: HostPath represents a pre-existing + file or directory on the host machine that + is directly exposed to + properties: + path: + description: Path of the directory on + the host. + type: string + type: + description: 'Type for HostPath Volume + Defaults to "" More info: https://kubernetes.' + type: string + required: + - path + type: object + iscsi: + description: ISCSI represents an ISCSI Disk + resource that is attached to a kubelet's + host machine and then expose + properties: + chapAuthDiscovery: + description: whether support iSCSI Discovery + CHAP authentication + type: boolean + chapAuthSession: + description: whether support iSCSI Session + CHAP authentication + type: boolean + fsType: + description: Filesystem type of the volume + that you want to mount. + type: string + initiatorName: + description: Custom iSCSI Initiator Name. + type: string + iqn: + description: Target iSCSI Qualified Name. + type: string + iscsiInterface: + description: iSCSI Interface Name that + uses an iSCSI transport. Defaults to + 'default' (tcp). + type: string + lun: + description: iSCSI Target Lun number. + format: int32 + type: integer + portals: + description: iSCSI Target Portal List. + items: + type: string + type: array + readOnly: + description: ReadOnly here will force + the ReadOnly setting in VolumeMounts. + Defaults to false. + type: boolean + secretRef: + description: CHAP Secret for iSCSI target + and initiator authentication + properties: + name: + description: 'Name of the referent. + More info: https://kubernetes.' + type: string + type: object + targetPortal: + description: iSCSI Target Portal. + type: string + required: + - iqn + - lun + - targetPortal + type: object + name: + description: 'Volume''s name. Must be a DNS_LABEL + and unique within the pod. More info: https://kubernetes.' + type: string + nfs: + description: 'NFS represents an NFS mount + on the host that shares a pod''s lifetime + More info: https://kubernetes.' + properties: + path: + description: 'Path that is exported by + the NFS server. More info: https://kubernetes.' + type: string + readOnly: + description: ReadOnly here will force + the NFS export to be mounted with read-only + permissions. Defaults to false. + type: boolean + server: + description: 'Server is the hostname or + IP address of the NFS server. More info: + https://kubernetes.' + type: string + required: + - path + - server + type: object + persistentVolumeClaim: + description: PersistentVolumeClaimVolumeSource + represents a reference to a PersistentVolumeClaim + in the same name + properties: + claimName: + description: ClaimName is the name of + a PersistentVolumeClaim in the same + namespace as the pod using this volume. + type: string + readOnly: + description: Will force the ReadOnly setting + in VolumeMounts. Default false. + type: boolean + required: + - claimName + type: object + photonPersistentDisk: + description: 'PhotonPersistentDisk represents + a PhotonController persistent disk attached + and mounted on kubelets ' + properties: + fsType: + description: Filesystem type to mount. + Must be a filesystem type supported + by the host operating system. Ex. + type: string + pdID: + description: ID that identifies Photon + Controller persistent disk + type: string + required: + - pdID + type: object + portworxVolume: + description: PortworxVolume represents a portworx + volume attached and mounted on kubelets + host machine + properties: + fsType: + description: FSType represents the filesystem + type to mount Must be a filesystem type + supported by the host opera + type: string + readOnly: + description: Defaults to false (read/write). + ReadOnly here will force the ReadOnly + setting in VolumeMounts. + type: boolean + volumeID: + description: VolumeID uniquely identifies + a Portworx volume + type: string + required: + - volumeID + type: object + projected: + description: Items for all in one resources + secrets, configmaps, and downward API + properties: + defaultMode: + description: Mode bits used to set permissions + on created files by default. + format: int32 + type: integer + sources: + description: list of volume projections + items: + description: Projection that may be + projected along with other supported + volume types + properties: + configMap: + description: information about the + configMap data to project + properties: + items: + description: 'If unspecified, + each key-value pair in the + Data field of the referenced + ConfigMap will be projected ' + items: + description: Maps a string + key to a path within a volume. + properties: + key: + description: The key to + project. + type: string + mode: + description: 'Optional: + mode bits used to set + permissions on this + file.' + format: int32 + type: integer + path: + description: The relative + path of the file to + map the key to. May + not be an absolute path. + type: string + required: + - key + - path + type: object + type: array + name: + description: 'Name of the referent. + More info: https://kubernetes.' + type: string + optional: + description: Specify whether + the ConfigMap or its keys + must be defined + type: boolean + type: object + downwardAPI: + description: information about the + downwardAPI data to project + properties: + items: + description: Items is a list + of DownwardAPIVolume file + items: + description: DownwardAPIVolumeFile + represents information to + create the file containing + the pod field + properties: + fieldRef: + description: 'Required: + Selects a field of the + pod: only annotations, + labels, name and namespace + are supported.' + properties: + apiVersion: + description: Version + of the schema the + FieldPath is written + in terms of, defaults + to "v1". + type: string + fieldPath: + description: Path + of the field to + select in the specified + API version. + type: string + required: + - fieldPath + type: object + mode: + description: 'Optional: + mode bits used to set + permissions on this + file, must be an octal + value between 0000 and + 07' + format: int32 + type: integer + path: + description: 'Required: + Path is the relative + path name of the file + to be created.' + type: string + resourceFieldRef: + description: 'Selects + a resource of the container: + only resources limits + and requests (limits.cpu, + limits.' + properties: + containerName: + description: 'Container + name: required for + volumes, optional + for env vars' + type: string + divisor: + anyOf: + - type: integer + - type: string + description: Specifies + the output format + of the exposed resources, + defaults to "1" + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + resource: + description: 'Required: + resource to select' + type: string + required: + - resource + type: object + required: + - path + type: object + type: array + type: object + secret: + description: information about the + secret data to project + properties: + items: + description: If unspecified, + each key-value pair in the + Data field of the referenced + Secret will be projected int + items: + description: Maps a string + key to a path within a volume. + properties: + key: + description: The key to + project. + type: string + mode: + description: 'Optional: + mode bits used to set + permissions on this + file.' + format: int32 + type: integer + path: + description: The relative + path of the file to + map the key to. May + not be an absolute path. + type: string + required: + - key + - path + type: object + type: array + name: + description: 'Name of the referent. + More info: https://kubernetes.' + type: string + optional: + description: Specify whether + the Secret or its key must + be defined + type: boolean + type: object + serviceAccountToken: + description: information about the + serviceAccountToken data to project + properties: + audience: + description: Audience is the + intended audience of the token. + type: string + expirationSeconds: + description: ExpirationSeconds + is the requested duration + of validity of the service + account token. + format: int64 + type: integer + path: + description: Path is the path + relative to the mount point + of the file to project the + token into. + type: string + required: + - path + type: object + type: object + type: array + type: object + quobyte: + description: Quobyte represents a Quobyte + mount on the host that shares a pod's lifetime + properties: + group: + description: Group to map volume access + to Default is no group + type: string + readOnly: + description: ReadOnly here will force + the Quobyte volume to be mounted with + read-only permissions. + type: boolean + registry: + description: Registry represents a single + or multiple Quobyte Registry services + specified as a string as host:por + type: string + tenant: + description: Tenant owning the given Quobyte + volume in the Backend Used with dynamically + provisioned Quobyte volu + type: string + user: + description: User to map volume access + to Defaults to serivceaccount user + type: string + volume: + description: Volume is a string that references + an already created Quobyte volume by + name. + type: string + required: + - registry + - volume + type: object + rbd: + description: RBD represents a Rados Block + Device mount on the host that shares a pod's + lifetime. + properties: + fsType: + description: Filesystem type of the volume + that you want to mount. + type: string + image: + description: 'The rados image name. More + info: https://examples.k8s.io/volumes/rbd/README.md#how-to-use-it' + type: string + keyring: + description: Keyring is the path to key + ring for RBDUser. Default is /etc/ceph/keyring. + type: string + monitors: + description: 'A collection of Ceph monitors. + More info: https://examples.k8s.io/volumes/rbd/README.' + items: + type: string + type: array + pool: + description: 'The rados pool name. Default + is rbd. More info: https://examples.k8s.io/volumes/rbd/README.' + type: string + readOnly: + description: ReadOnly here will force + the ReadOnly setting in VolumeMounts. + Defaults to false. + type: boolean + secretRef: + description: SecretRef is name of the + authentication secret for RBDUser. If + provided overrides keyring. + properties: + name: + description: 'Name of the referent. + More info: https://kubernetes.' + type: string + type: object + user: + description: 'The rados user name. Default + is admin. More info: https://examples.k8s.io/volumes/rbd/README.' + type: string + required: + - image + - monitors + type: object + scaleIO: + description: ScaleIO represents a ScaleIO + persistent volume attached and mounted on + Kubernetes nodes. + properties: + fsType: + description: Filesystem type to mount. + Must be a filesystem type supported + by the host operating system. Ex. + type: string + gateway: + description: The host address of the ScaleIO + API Gateway. + type: string + protectionDomain: + description: The name of the ScaleIO Protection + Domain for the configured storage. + type: string + readOnly: + description: Defaults to false (read/write). + ReadOnly here will force the ReadOnly + setting in VolumeMounts. + type: boolean + secretRef: + description: SecretRef references to the + secret for ScaleIO user and other sensitive + information. + properties: + name: + description: 'Name of the referent. + More info: https://kubernetes.' + type: string + type: object + sslEnabled: + description: Flag to enable/disable SSL + communication with Gateway, default + false + type: boolean + storageMode: + description: Indicates whether the storage + for a volume should be ThickProvisioned + or ThinProvisioned. + type: string + storagePool: + description: The ScaleIO Storage Pool + associated with the protection domain. + type: string + system: + description: The name of the storage system + as configured in ScaleIO. + type: string + volumeName: + description: The name of a volume already + created in the ScaleIO system that is + associated with this volume sourc + type: string + required: + - gateway + - secretRef + - system + type: object + secret: + description: 'Secret represents a secret that + should populate this volume. More info: + https://kubernetes.' + properties: + defaultMode: + description: 'Optional: mode bits used + to set permissions on created files + by default.' + format: int32 + type: integer + items: + description: If unspecified, each key-value + pair in the Data field of the referenced + Secret will be projected int + items: + description: Maps a string key to a + path within a volume. + properties: + key: + description: The key to project. + type: string + mode: + description: 'Optional: mode bits + used to set permissions on this + file.' + format: int32 + type: integer + path: + description: The relative path of + the file to map the key to. May + not be an absolute path. + type: string + required: + - key + - path + type: object + type: array + optional: + description: Specify whether the Secret + or its keys must be defined + type: boolean + secretName: + description: 'Name of the secret in the + pod''s namespace to use. More info: + https://kubernetes.' + type: string + type: object + storageos: + description: StorageOS represents a StorageOS + volume attached and mounted on Kubernetes + nodes. + properties: + fsType: + description: Filesystem type to mount. + Must be a filesystem type supported + by the host operating system. Ex. + type: string + readOnly: + description: Defaults to false (read/write). + ReadOnly here will force the ReadOnly + setting in VolumeMounts. + type: boolean + secretRef: + description: SecretRef specifies the secret + to use for obtaining the StorageOS API + credentials. + properties: + name: + description: 'Name of the referent. + More info: https://kubernetes.' + type: string + type: object + volumeName: + description: VolumeName is the human-readable + name of the StorageOS volume. + type: string + volumeNamespace: + description: VolumeNamespace specifies + the scope of the volume within StorageOS. + type: string + type: object + vsphereVolume: + description: VsphereVolume represents a vSphere + volume attached and mounted on kubelets + host machine + properties: + fsType: + description: Filesystem type to mount. + Must be a filesystem type supported + by the host operating system. Ex. + type: string + storagePolicyID: + description: Storage Policy Based Management + (SPBM) profile ID associated with the + StoragePolicyName. + type: string + storagePolicyName: + description: Storage Policy Based Management + (SPBM) profile name. + type: string + volumePath: + description: Path that identifies vSphere + volume vmdk + type: string + required: + - volumePath + type: object + required: + - name + type: object + type: array + required: + - containers + type: object + type: object + required: + - groupName + - maxReplicas + - minReplicas + - rayStartParams + - replicas + - template + type: object + type: array + required: + - headGroupSpec + type: object + serveConfigs: + items: + description: ServeConfigSpec defines the desired state of RayService + properties: + autoscaling_config: + additionalProperties: + type: string + type: object + graceful_shutdown_timeout_s: + type: number + graceful_shutdown_wait_loop_s: + type: number + health_check_period_s: + type: number + health_check_timeout_s: + type: number + import_path: + type: string + init_args: + items: + type: string + type: array + init_kwargs: + additionalProperties: + type: string + type: object + max_concurrent_queries: + format: int32 + type: integer + name: + type: string + num_replicas: + format: int32 + type: integer + ray_actor_options: + description: RayActorOptionSpec defines the desired state of + RayActor + properties: + accelerator_type: + type: string + memory: + format: int32 + type: integer + num_cpus: + type: number + num_gpus: + type: number + object_store_memory: + format: int32 + type: integer + resources: + additionalProperties: + type: string + type: object + runtime_env: + additionalProperties: + items: + type: string + type: array + type: object + type: object + route_prefix: + type: string + user_config: + additionalProperties: + type: string + type: object + required: + - import_path + - name + type: object + type: array type: object status: description: RayServiceStatus defines the observed state of RayService + properties: + rayClusterStatus: + description: RayClusterStatus defines the observed state of RayCluster + properties: + availableWorkerReplicas: + description: AvailableWorkerReplicas indicates how many replicas + are available in the cluster + format: int32 + type: integer + desiredWorkerReplicas: + description: DesiredWorkerReplicas indicates overall desired replicas + claimed by the user at the cluster level. + format: int32 + type: integer + lastUpdateTime: + description: LastUpdateTime indicates last update timestamp for + this cluster status. + format: date-time + nullable: true + type: string + maxWorkerReplicas: + description: MaxWorkerReplicas indicates sum of maximum replicas + of each node group. + format: int32 + type: integer + minWorkerReplicas: + description: MinWorkerReplicas indicates sum of minimum replicas + of each node group. + format: int32 + type: integer + state: + description: 'INSERT ADDITIONAL STATUS FIELD - define observed + state of cluster Important: Run "make" to regenerat' + type: string + type: object + serveStatuses: + description: ServeStatuses defines the desired states of all Serve + Deployments + properties: + statuses: + items: + description: ServeStatus defines the desired state of Serve + Deployment + properties: + healthLastUpdateTime: + description: Keep track of how long the service is healthy. + format: date-time + type: string + message: + type: string + name: + type: string + status: + type: string + type: object + type: array + type: object + serviceStatus: + description: 'Important: Run "make" to regenerate code after modifying + this file' + type: string type: object type: object served: true diff --git a/ray-operator/config/manager/kustomization.yaml b/ray-operator/config/manager/kustomization.yaml index 7e3a0f40b3..3006e30f68 100644 --- a/ray-operator/config/manager/kustomization.yaml +++ b/ray-operator/config/manager/kustomization.yaml @@ -3,3 +3,7 @@ resources: - service.yaml apiVersion: kustomize.config.k8s.io/v1beta1 kind: Kustomization +images: +- name: kuberay/operator + newName: brucezhang90/bruce-docker-repo + newTag: health_kuberay diff --git a/ray-operator/config/rbac/role.yaml b/ray-operator/config/rbac/role.yaml index 5ce947c609..eb6a6eb3be 100644 --- a/ray-operator/config/rbac/role.yaml +++ b/ray-operator/config/rbac/role.yaml @@ -54,7 +54,7 @@ rules: - apiGroups: - "" resources: - - serviceaccount + - serviceaccounts verbs: - create - delete diff --git a/ray-operator/config/samples/ray_v1alpha1_rayservice.yaml b/ray-operator/config/samples/ray_v1alpha1_rayservice.yaml index df42b6f305..a9d84c1f6d 100644 --- a/ray-operator/config/samples/ray_v1alpha1_rayservice.yaml +++ b/ray-operator/config/samples/ray_v1alpha1_rayservice.yaml @@ -3,4 +3,95 @@ kind: RayService metadata: name: rayservice-sample spec: + serveConfigs: + - name: shallow + import_path: "test_env.shallow_import.ShallowClass" + num_replicas: 1 + route_prefix: "/shallow" + ray_actor_options: + num_cpus: 0.1 + runtime_env: + py_modules: + - "https://github.com/shrekris-anyscale/test_deploy_group/archive/HEAD.zip" + - "https://github.com/shrekris-anyscale/test_module/archive/HEAD.zip" + - name: deep + import_path: "test_env.subdir1.subdir2.deep_import.DeepClass" + num_replicas: 1 + route_prefix: "/deep" + ray_actor_options: + num_cpus: 0.1 + runtime_env: + py_modules: + - "https://github.com/shrekris-anyscale/test_deploy_group/archive/HEAD.zip" + - "https://github.com/shrekris-anyscale/test_module/archive/HEAD.zip" + - name: one + import_path: "test_module.test.one" + num_replicas: 1 + ray_actor_options: + num_cpus: 0.1 + runtime_env: + py_modules: + - "https://github.com/shrekris-anyscale/test_deploy_group/archive/HEAD.zip" + - "https://github.com/shrekris-anyscale/test_module/archive/HEAD.zip" # TODO(user): Add fields here + rayClusterConfig: + rayVersion: '1.8.0' # should match the Ray version in the image of the containers + ######################headGroupSpecs################################# + # head group template and specs, (perhaps 'group' is not needed in the name) + headGroupSpec: + # Kubernetes Service Type, valid values are 'ClusterIP', 'NodePort' and 'LoadBalancer' + serviceType: ClusterIP + # the pod replicas in this group typed head (assuming there could be more than 1 in the future) + replicas: 4 + # logical group name, for this called head-group, also can be functional + # pod type head or worker + # rayNodeType: head # Not needed since it is under the headgroup + # the following params are used to complete the ray start: ray start --head --block --redis-port=6379 ... + rayStartParams: + port: '6379' # should match headService targetPort + #include_webui: 'true' + object-store-memory: '100000000' + redis-password: 'LetMeInRay' # Deprecated since Ray 1.11 due to GCS bootstrapping enabled + # webui_host: "10.1.2.60" + dashboard-host: '0.0.0.0' + num-cpus: '1' # can be auto-completed from the limits + node-ip-address: $MY_POD_IP # auto-completed as the head pod IP + block: 'true' + #pod template + template: + metadata: + labels: + # custom labels. NOTE: do not define custom labels start with `raycluster.`, they may be used in controller. + # Refer to https://kubernetes.io/docs/concepts/overview/working-with-objects/labels/ + rayCluster: raycluster-sample # will be injected if missing + rayNodeType: head # will be injected if missing, must be head or wroker + groupName: headgroup # will be injected if missing + # annotations for pod + annotations: + key: value + spec: + containers: + - name: ray-head + image: rayproject/ray:1.12.0 + #image: rayproject/ray:nightly + #image: bonsaidev.azurecr.io/bonsai/lazer-0-9-0-cpu:dev + env: + - name: MY_POD_IP + valueFrom: + fieldRef: + fieldPath: status.podIP + resources: + limits: + cpu: 1 + memory: 2Gi + requests: + cpu: 1 + memory: 2Gi + ports: + - containerPort: 6379 + name: redis + - containerPort: 8265 # Ray dashboard + name: dashboard + - containerPort: 10001 + name: client + diff --git a/ray-operator/controllers/ray/raycluster_controller.go b/ray-operator/controllers/ray/raycluster_controller.go index 47edeb0832..1018d84578 100644 --- a/ray-operator/controllers/ray/raycluster_controller.go +++ b/ray-operator/controllers/ray/raycluster_controller.go @@ -74,7 +74,7 @@ type RayClusterReconciler struct { // +kubebuilder:rbac:groups=core,resources=services,verbs=get;list;watch;create;update;patch;delete // +kubebuilder:rbac:groups=core,resources=services/status,verbs=get;update;patch // +kubebuilder:rbac:groups=coordination.k8s.io,resources=leases,verbs=get;list;create;update -// +kubebuilder:rbac:groups=core,resources=serviceaccount,verbs=get;list;watch;create;delete +// +kubebuilder:rbac:groups=core,resources=serviceaccounts,verbs=get;list;watch;create;delete // +kubebuilder:rbac:groups="rbac.authorization.k8s.io",resources=roles,verbs=get;list;watch;create;delete;update // +kubebuilder:rbac:groups="rbac.authorization.k8s.io",resources=rolebindings,verbs=get;list;watch;create;delete // Reconcile used to bridge the desired state with the current state diff --git a/ray-operator/controllers/ray/rayservice_controller.go b/ray-operator/controllers/ray/rayservice_controller.go index e22a2b3cf0..f291acaf21 100644 --- a/ray-operator/controllers/ray/rayservice_controller.go +++ b/ray-operator/controllers/ray/rayservice_controller.go @@ -65,7 +65,7 @@ func NewRayServiceReconciler(mgr manager.Manager) *RayServiceReconciler { // +kubebuilder:rbac:groups=core,resources=services,verbs=get;list;watch;create;update;patch;delete // +kubebuilder:rbac:groups=core,resources=services/status,verbs=get;update;patch // +kubebuilder:rbac:groups=coordination.k8s.io,resources=leases,verbs=get;list;create;update -// +kubebuilder:rbac:groups=core,resources=serviceaccount,verbs=get;list;watch;create;delete +// +kubebuilder:rbac:groups=core,resources=serviceaccounts,verbs=get;list;watch;create;delete // +kubebuilder:rbac:groups="rbac.authorization.k8s.io",resources=roles,verbs=get;list;watch;create;delete;update // +kubebuilder:rbac:groups="rbac.authorization.k8s.io",resources=rolebindings,verbs=get;list;watch;create;delete @@ -123,6 +123,8 @@ func (r *RayServiceReconciler) Reconcile(ctx context.Context, request ctrl.Reque return ctrl.Result{}, err } + log.Info("Check serve health", "isHealthy", isHealthy) + if isHealthy { rayServiceInstance.Status.ServiceStatus = rayv1alpha1.RUNNING if err := r.Status().Update(context.Background(), rayServiceInstance); err != nil { @@ -296,11 +298,12 @@ func (r *RayServiceReconciler) fetchDashboardURL(rayCluster *rayv1alpha1.RayClus func (r *RayServiceReconciler) checkIfNeedSubmitServeDeployment(rayServiceInstance *rayv1alpha1.RayService, request ctrl.Request) bool { existConfig, exist := r.ServeDeploymentConfigMap[request.NamespacedName] - shouldUpdate := true + shouldUpdate := false - if !exist || reflect.DeepEqual(existConfig, rayServiceInstance.Spec) || len(rayServiceInstance.Status.ServeStatuses.Statuses) != len(existConfig.ServeConfigSpecs) { - shouldUpdate = false + if !exist || !reflect.DeepEqual(existConfig, rayServiceInstance.Spec) || len(rayServiceInstance.Status.ServeStatuses.Statuses) != len(existConfig.ServeConfigSpecs) { + shouldUpdate = true } + r.Log.Info("shouldUpdate value", "shouldUpdate", shouldUpdate) log.V(1).Info("status check", "len(rayServiceInstance.Status.ServeStatuses.Statuses) ", len(rayServiceInstance.Status.ServeStatuses.Statuses), "len(existConfig.ServeConfigSpecs)", len(existConfig.ServeConfigSpecs)) From fc7e6a2f013e8457fc9a902191109ca858ce73f1 Mon Sep 17 00:00:00 2001 From: brucez-anyscale Date: Fri, 3 Jun 2022 18:24:59 -0700 Subject: [PATCH 03/19] update --- ray-operator/config/samples/ray_v1alpha1_rayservice.yaml | 1 - 1 file changed, 1 deletion(-) diff --git a/ray-operator/config/samples/ray_v1alpha1_rayservice.yaml b/ray-operator/config/samples/ray_v1alpha1_rayservice.yaml index a9d84c1f6d..9604148398 100644 --- a/ray-operator/config/samples/ray_v1alpha1_rayservice.yaml +++ b/ray-operator/config/samples/ray_v1alpha1_rayservice.yaml @@ -33,7 +33,6 @@ spec: py_modules: - "https://github.com/shrekris-anyscale/test_deploy_group/archive/HEAD.zip" - "https://github.com/shrekris-anyscale/test_module/archive/HEAD.zip" - # TODO(user): Add fields here rayClusterConfig: rayVersion: '1.8.0' # should match the Ray version in the image of the containers ######################headGroupSpecs################################# From b4a8737d9e72f61d270263e3e828df93a7db41e9 Mon Sep 17 00:00:00 2001 From: brucez-anyscale Date: Fri, 3 Jun 2022 18:31:26 -0700 Subject: [PATCH 04/19] update --- ray-operator/controllers/ray/rayservice_controller.go | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/ray-operator/controllers/ray/rayservice_controller.go b/ray-operator/controllers/ray/rayservice_controller.go index f291acaf21..08c13b9bf8 100644 --- a/ray-operator/controllers/ray/rayservice_controller.go +++ b/ray-operator/controllers/ray/rayservice_controller.go @@ -9,7 +9,6 @@ import ( "k8s.io/apimachinery/pkg/api/errors" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "k8s.io/apimachinery/pkg/types" - "math/rand" "reflect" "time" @@ -344,7 +343,7 @@ func (r *RayServiceReconciler) getAndCheckServeStatus(rayServiceInstance *rayv1a if prevStatus.Status != "HEALTHY" { serveStatuses.Statuses[i].HealthLastUpdateTime = prevStatus.HealthLastUpdateTime - if time.Now().Sub(prevStatus.HealthLastUpdateTime.Time).Seconds() > 60 { + if time.Since(prevStatus.HealthLastUpdateTime.Time).Seconds() > 60 { isHealthy = false } } @@ -356,11 +355,5 @@ func (r *RayServiceReconciler) getAndCheckServeStatus(rayServiceInstance *rayv1a r.Log.Info("getAndCheckServeStatus ", "statusMap", statusMap, "serveStatuses", serveStatuses) - rNum := rand.Intn(100) - if rNum < 10 { - r.Log.Info("getAndCheckServeStatus ", "rand", rNum) - isHealthy = false - } - return isHealthy, nil } From 9baf70520614b7034d3cdd806a165551d5cb448e Mon Sep 17 00:00:00 2001 From: brucez-anyscale Date: Fri, 3 Jun 2022 18:34:16 -0700 Subject: [PATCH 05/19] refactor --- .../controllers/ray/utils/serve_httpclient.go | 20 +++---------------- 1 file changed, 3 insertions(+), 17 deletions(-) diff --git a/ray-operator/controllers/ray/utils/serve_httpclient.go b/ray-operator/controllers/ray/utils/serve_httpclient.go index 39b80727d9..2b3a7cffcb 100644 --- a/ray-operator/controllers/ray/utils/serve_httpclient.go +++ b/ray-operator/controllers/ray/utils/serve_httpclient.go @@ -2,12 +2,10 @@ package utils import ( "bytes" - "fmt" rayv1alpha1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1alpha1" "io/ioutil" "k8s.io/apimachinery/pkg/util/json" "net/http" - "reflect" ) var ( @@ -55,17 +53,6 @@ func (r *RayDashboardClient) UpdateDeployments(specs []rayv1alpha1.ServeConfigSp deploymentJson, err := json.Marshal(servingClusterDeployments) - var existDeploymentConfigJson string - if existDeploymentConfigJson, err = r.GetDeployments(); err != nil { - return err - } - existDeploymentConfig := ServingClusterDeployments{} - _ = json.Unmarshal([]byte(existDeploymentConfigJson), &existDeploymentConfig) - - if reflect.DeepEqual(existDeploymentConfig, servingClusterDeployments) { - return nil - } - if err != nil { return err } @@ -99,13 +86,12 @@ func (r *RayDashboardClient) GetDeploymentsStatus() (*rayv1alpha1.ServeStatuses, } defer resp.Body.Close() - fmt.Println("response Status:", resp.Status) - fmt.Println("response Headers:", resp.Header) body, _ := ioutil.ReadAll(resp.Body) - fmt.Println("response Body:", string(body)) var serveStatuses rayv1alpha1.ServeStatuses - _ = json.Unmarshal(body, &serveStatuses) + if err = json.Unmarshal(body, &serveStatuses); err != nil { + return nil, err + } return &serveStatuses, nil } From 9bd49909294bfd1f68ff83b6c045b2947a68178a Mon Sep 17 00:00:00 2001 From: brucez-anyscale Date: Fri, 3 Jun 2022 18:35:08 -0700 Subject: [PATCH 06/19] goimports --- ray-operator/controllers/ray/rayservice_controller.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/ray-operator/controllers/ray/rayservice_controller.go b/ray-operator/controllers/ray/rayservice_controller.go index 08c13b9bf8..ac774ec970 100644 --- a/ray-operator/controllers/ray/rayservice_controller.go +++ b/ray-operator/controllers/ray/rayservice_controller.go @@ -3,14 +3,15 @@ package ray import ( "context" "fmt" + "reflect" + "time" + "github.com/ray-project/kuberay/ray-operator/controllers/ray/common" "github.com/ray-project/kuberay/ray-operator/controllers/ray/utils" corev1 "k8s.io/api/core/v1" "k8s.io/apimachinery/pkg/api/errors" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "k8s.io/apimachinery/pkg/types" - "reflect" - "time" "github.com/go-logr/logr" fmtErrors "github.com/pkg/errors" From 1812252e7cc7c24e02d74cd82cb4942ea085e750 Mon Sep 17 00:00:00 2001 From: brucez-anyscale Date: Fri, 3 Jun 2022 18:39:10 -0700 Subject: [PATCH 07/19] goimports --- ray-operator/controllers/ray/utils/serve_httpclient.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/ray-operator/controllers/ray/utils/serve_httpclient.go b/ray-operator/controllers/ray/utils/serve_httpclient.go index 2b3a7cffcb..a590dd82b8 100644 --- a/ray-operator/controllers/ray/utils/serve_httpclient.go +++ b/ray-operator/controllers/ray/utils/serve_httpclient.go @@ -2,10 +2,11 @@ package utils import ( "bytes" - rayv1alpha1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1alpha1" "io/ioutil" - "k8s.io/apimachinery/pkg/util/json" "net/http" + + rayv1alpha1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1alpha1" + "k8s.io/apimachinery/pkg/util/json" ) var ( From 072d745e5e111e28be308fdcce7d845294b87833 Mon Sep 17 00:00:00 2001 From: brucez-anyscale Date: Tue, 7 Jun 2022 11:16:52 -0700 Subject: [PATCH 08/19] address comments --- ray-operator/Makefile | 1 + 1 file changed, 1 insertion(+) diff --git a/ray-operator/Makefile b/ray-operator/Makefile index fd33a91b86..9d9cfe71ce 100644 --- a/ray-operator/Makefile +++ b/ray-operator/Makefile @@ -4,6 +4,7 @@ COMMIT_SHA1 := $(shell git rev-parse HEAD ) # Image URL to use all building/pushing image targets IMG ?= controller:latest # Produce CRDs that work back to Kubernetes 1.11 (no version conversion) +# Golang forbids converting float from Json to struct. Need allowDangerousTypes=true to unblock build. CRD_OPTIONS ?= "crd:maxDescLen=100,trivialVersions=true,preserveUnknownFields=false,generateEmbeddedObjectMeta=true,allowDangerousTypes=true" # Get the currently used golang install path (in GOPATH/bin, unless GOBIN is set) From 45920eb81f6b90cd4dbe1d3d946795fdf85c7fb5 Mon Sep 17 00:00:00 2001 From: brucez-anyscale Date: Tue, 7 Jun 2022 17:37:46 -0700 Subject: [PATCH 09/19] address comments --- .../apis/ray/v1alpha1/rayservice_types.go | 91 +++++++++---------- .../config/samples/ray-cluster.ingress.yaml | 7 +- .../config/samples/ray-cluster.mini.yaml | 9 +- .../samples/ray_v1alpha1_rayservice.yaml | 45 +++++---- .../controllers/ray/rayservice_controller.go | 74 +++++++-------- .../controllers/ray/utils/serve_httpclient.go | 80 ++++++++++++++-- 6 files changed, 181 insertions(+), 125 deletions(-) diff --git a/ray-operator/apis/ray/v1alpha1/rayservice_types.go b/ray-operator/apis/ray/v1alpha1/rayservice_types.go index 04285925d0..1271a8ce5a 100644 --- a/ray-operator/apis/ray/v1alpha1/rayservice_types.go +++ b/ray-operator/apis/ray/v1alpha1/rayservice_types.go @@ -1,7 +1,6 @@ package v1alpha1 import ( - v1 "k8s.io/api/core/v1" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" ) @@ -11,56 +10,67 @@ import ( type ServiceStatus string const ( - FAIL_TO_GET_RAYSERVICE ServiceStatus = "fail_to_get_rayservice" - FAIL_TO_GET_OR_CREATE_RAYCLUSTER ServiceStatus = "fail_to_get_or_create_RAYCLUSTER" - WAIT_FOR_DASHBOARD ServiceStatus = "wait_for_dashboard" - FAIL_SERVE_DEPLOY ServiceStatus = "fail_serve_deploy" - FAIL_GET_SERVE_DEPLOYMENT_STATUS ServiceStatus = "fail_get_serve_deployment_status" - RUNNING ServiceStatus = "running" - RESTARTING ServiceStatus = "restarting" - FAIL_DELETE_RAYCLUSTER ServiceStatus = "fail_delete_raycluster" + FailToGetRayService ServiceStatus = "FailToGetRayService" + FailToGetOrCreateRayCluster ServiceStatus = "FailToGetOrCreateRayCluster" + WaitForDashboard ServiceStatus = "WaitForDashboard" + FailServeDeploy ServiceStatus = "FailServeDeploy" + FailGetServeDeploymentStatus ServiceStatus = "FailGetServeDeploymentStatus" + Running ServiceStatus = "Running" + Restarting ServiceStatus = "Restarting" + FailDeleteRayCluster ServiceStatus = "FailDeleteRayCluster" ) +// RayServiceSpec defines the desired state of RayService +type RayServiceSpec struct { + // Important: Run "make" to regenerate code after modifying this file + ServeConfigSpecs []ServeConfigSpec `json:"serveConfigs,omitempty"` + RayClusterSpec RayClusterSpec `json:"rayClusterConfig,omitempty"` +} + // ServeConfigSpec defines the desired state of RayService type ServeConfigSpec struct { Name string `json:"name"` - ImportPath string `json:"import_path"` - InitArgs []string `json:"init_args,omitempty"` - InitKwargs map[string]string `json:"init_kwargs,omitempty"` - NumReplicas *int32 `json:"num_replicas,omitempty"` - RoutePrefix string `json:"route_prefix,omitempty"` - MaxConcurrentQueries *int32 `json:"max_concurrent_queries,omitempty"` - UserConfig map[string]string `json:"user_config,omitempty"` - AutoscalingConfig map[string]string `json:"autoscaling_config,omitempty"` - GracefulShutdownWaitLoopS *float64 `json:"graceful_shutdown_wait_loop_s,omitempty"` - GracefulShutdownTimeoutS *float64 `json:"graceful_shutdown_timeout_s,omitempty"` - HealthCheckPeriodS *float64 `json:"health_check_period_s,omitempty"` - HealthCheckTimeoutS *float64 `json:"health_check_timeout_s,omitempty"` - RayActorOptions RayActorOptionSpec `json:"ray_actor_options,omitempty"` + ImportPath string `json:"importPath"` + InitArgs []string `json:"initArgs,omitempty"` + InitKwargs map[string]string `json:"initKwargs,omitempty"` + NumReplicas *int32 `json:"numReplicas,omitempty"` + RoutePrefix string `json:"routePrefix,omitempty"` + MaxConcurrentQueries *int32 `json:"maxConcurrentQueries,omitempty"` + UserConfig map[string]string `json:"userConfig,omitempty"` + AutoscalingConfig map[string]string `json:"autoscalingConfig,omitempty"` + GracefulShutdownWaitLoopS *float64 `json:"gracefulShutdownWaitLoopS,omitempty"` + GracefulShutdownTimeoutS *float64 `json:"gracefulShutdownTimeoutS,omitempty"` + HealthCheckPeriodS *float64 `json:"healthCheckPeriodS,omitempty"` + HealthCheckTimeoutS *float64 `json:"healthCheckTimeoutS,omitempty"` + RayActorOptions RayActorOptionSpec `json:"rayActorOptions,omitempty"` } // RayActorOptionSpec defines the desired state of RayActor type RayActorOptionSpec struct { - RuntimeEnv map[string][]string `json:"runtime_env,omitempty"` - NumCpus *float64 `json:"num_cpus,omitempty"` - NumGpus *float64 `json:"num_gpus,omitempty"` + RuntimeEnv map[string][]string `json:"runtimeEnv,omitempty"` + NumCpus *float64 `json:"numCpus,omitempty"` + NumGpus *float64 `json:"numGpus,omitempty"` Memory *int32 `json:"memory,omitempty"` - ObjectStoreMemory *int32 `json:"object_store_memory,omitempty"` + ObjectStoreMemory *int32 `json:"objectStoreMemory,omitempty"` Resources map[string]string `json:"resources,omitempty"` - AcceleratorType string `json:"accelerator_type,omitempty"` + AcceleratorType string `json:"acceleratorType,omitempty"` } -// RayServiceSpec defines the desired state of RayService -type RayServiceSpec struct { +// RayServiceStatus defines the observed state of RayService +type RayServiceStatus struct { // Important: Run "make" to regenerate code after modifying this file + ServiceStatus ServiceStatus `json:"serviceStatus,omitempty"` + ServeStatuses ServeDeploymentStatuses `json:"serveDeploymentStatuses,omitempty"` + RayClusterStatus RayClusterStatus `json:"rayClusterStatus,omitempty"` +} - HealthCheckProbe *v1.Probe `json:"healthCheckConfig,omitempty"` - ServeConfigSpecs []ServeConfigSpec `json:"serveConfigs,omitempty"` - RayClusterSpec RayClusterSpec `json:"rayClusterConfig,omitempty"` +// ServeDeploymentStatuses defines the desired states of all Serve Deployments +type ServeDeploymentStatuses struct { + Statuses []ServeDeploymentStatus `json:"statuses,omitempty"` } -// ServeStatus defines the desired state of Serve Deployment -type ServeStatus struct { +// ServeDeploymentStatus defines the desired state of Serve Deployment +type ServeDeploymentStatus struct { Name string `json:"name,omitempty"` Status string `json:"status,omitempty"` Message string `json:"message,omitempty"` @@ -69,19 +79,6 @@ type ServeStatus struct { HealthLastUpdateTime metav1.Time `json:"healthLastUpdateTime,omitempty"` } -// ServeStatuses defines the desired states of all Serve Deployments -type ServeStatuses struct { - Statuses []ServeStatus `json:"statuses,omitempty"` -} - -// RayServiceStatus defines the observed state of RayService -type RayServiceStatus struct { - // Important: Run "make" to regenerate code after modifying this file - ServiceStatus ServiceStatus `json:"serviceStatus,omitempty"` - ServeStatuses ServeStatuses `json:"serveStatuses,omitempty"` - RayClusterStatus RayClusterStatus `json:"rayClusterStatus,omitempty"` -} - //+kubebuilder:object:root=true //+kubebuilder:subresource:status diff --git a/ray-operator/config/samples/ray-cluster.ingress.yaml b/ray-operator/config/samples/ray-cluster.ingress.yaml index c1cadbd80c..8356de094a 100644 --- a/ray-operator/config/samples/ray-cluster.ingress.yaml +++ b/ray-operator/config/samples/ray-cluster.ingress.yaml @@ -5,14 +5,13 @@ metadata: kubernetes.io/ingress.class: nginx name: raycluster-ingress spec: - rayVersion: '1.6.0' # should match the Ray version in the image of the containers + rayVersion: '1.12.1' # should match the Ray version in the image of the containers headGroupSpec: serviceType: NodePort enableIngress: true replicas: 1 rayStartParams: port: '6379' - redis-password: 'LetMeInRay' # Deprecated since Ray 1.11 due to GCS bootstrapping enabled dashboard-host: '0.0.0.0' num-cpus: '1' # can be auto-completed from the limits node-ip-address: $MY_POD_IP # auto-completed as the head pod IP @@ -21,7 +20,7 @@ spec: spec: containers: - name: ray-head - image: rayproject/ray:1.9.2 + image: rayproject/ray:1.12.1 env: - name: MY_POD_IP valueFrom: @@ -29,7 +28,7 @@ spec: fieldPath: status.podIP ports: - containerPort: 6379 - name: redis + name: gcs-server - containerPort: 8265 # Ray dashboard name: dashboard - containerPort: 10001 diff --git a/ray-operator/config/samples/ray-cluster.mini.yaml b/ray-operator/config/samples/ray-cluster.mini.yaml index 34d8e0b441..babfb753c7 100644 --- a/ray-operator/config/samples/ray-cluster.mini.yaml +++ b/ray-operator/config/samples/ray-cluster.mini.yaml @@ -6,7 +6,7 @@ metadata: # An unique identifier for the head node and workers of this cluster. name: raycluster-mini spec: - rayVersion: '1.8.0' # should match the Ray version in the image of the containers + rayVersion: '1.12.1' # should match the Ray version in the image of the containers ######################headGroupSpecs################################# # head group template and specs, (perhaps 'group' is not needed in the name) headGroupSpec: @@ -19,10 +19,9 @@ spec: # rayNodeType: head # Not needed since it is under the headgroup # the following params are used to complete the ray start: ray start --head --block --redis-port=6379 ... rayStartParams: - port: '6379' # should match headService targetPort + port: '6379' # should match container port named gcs-server #include_webui: 'true' object-store-memory: '100000000' - redis-password: 'LetMeInRay' # Deprecated since Ray 1.11 due to GCS bootstrapping enabled # webui_host: "10.1.2.60" dashboard-host: '0.0.0.0' num-cpus: '1' # can be auto-completed from the limits @@ -43,7 +42,7 @@ spec: spec: containers: - name: ray-head - image: rayproject/ray:1.8.0 + image: rayproject/ray:1.12.1 #image: rayproject/ray:nightly #image: bonsaidev.azurecr.io/bonsai/lazer-0-9-0-cpu:dev env: @@ -60,7 +59,7 @@ spec: memory: 2Gi ports: - containerPort: 6379 - name: redis + name: gcs-server - containerPort: 8265 # Ray dashboard name: dashboard - containerPort: 10001 diff --git a/ray-operator/config/samples/ray_v1alpha1_rayservice.yaml b/ray-operator/config/samples/ray_v1alpha1_rayservice.yaml index 9604148398..e25f295d4e 100644 --- a/ray-operator/config/samples/ray_v1alpha1_rayservice.yaml +++ b/ray-operator/config/samples/ray_v1alpha1_rayservice.yaml @@ -5,52 +5,49 @@ metadata: spec: serveConfigs: - name: shallow - import_path: "test_env.shallow_import.ShallowClass" - num_replicas: 1 - route_prefix: "/shallow" - ray_actor_options: - num_cpus: 0.1 - runtime_env: + importPath: "test_env.shallow_import.ShallowClass" + numReplicas: 1 + routePrefix: "/shallow" + rayActorOptions: + numCpus: 0.1 + runtimeEnv: py_modules: - "https://github.com/shrekris-anyscale/test_deploy_group/archive/HEAD.zip" - "https://github.com/shrekris-anyscale/test_module/archive/HEAD.zip" - name: deep - import_path: "test_env.subdir1.subdir2.deep_import.DeepClass" - num_replicas: 1 - route_prefix: "/deep" - ray_actor_options: - num_cpus: 0.1 - runtime_env: + importPath: "test_env.subdir1.subdir2.deep_import.DeepClass" + numReplicas: 1 + routePrefix: "/deep" + rayActorOptions: + numCpus: 0.1 + runtimeEnv: py_modules: - "https://github.com/shrekris-anyscale/test_deploy_group/archive/HEAD.zip" - "https://github.com/shrekris-anyscale/test_module/archive/HEAD.zip" - name: one - import_path: "test_module.test.one" - num_replicas: 1 - ray_actor_options: - num_cpus: 0.1 - runtime_env: + importPath: "test_module.test.one" + numReplicas: 1 + rayActorOptions: + numCpus: 0.1 + runtimeEnv: py_modules: - "https://github.com/shrekris-anyscale/test_deploy_group/archive/HEAD.zip" - "https://github.com/shrekris-anyscale/test_module/archive/HEAD.zip" rayClusterConfig: - rayVersion: '1.8.0' # should match the Ray version in the image of the containers + rayVersion: '1.12.1' # should match the Ray version in the image of the containers ######################headGroupSpecs################################# # head group template and specs, (perhaps 'group' is not needed in the name) headGroupSpec: # Kubernetes Service Type, valid values are 'ClusterIP', 'NodePort' and 'LoadBalancer' serviceType: ClusterIP - # the pod replicas in this group typed head (assuming there could be more than 1 in the future) - replicas: 4 # logical group name, for this called head-group, also can be functional # pod type head or worker # rayNodeType: head # Not needed since it is under the headgroup # the following params are used to complete the ray start: ray start --head --block --redis-port=6379 ... rayStartParams: - port: '6379' # should match headService targetPort + port: '6379' # should match container port named gcs-server #include_webui: 'true' object-store-memory: '100000000' - redis-password: 'LetMeInRay' # Deprecated since Ray 1.11 due to GCS bootstrapping enabled # webui_host: "10.1.2.60" dashboard-host: '0.0.0.0' num-cpus: '1' # can be auto-completed from the limits @@ -71,7 +68,7 @@ spec: spec: containers: - name: ray-head - image: rayproject/ray:1.12.0 + image: rayproject/ray:1.12.1 #image: rayproject/ray:nightly #image: bonsaidev.azurecr.io/bonsai/lazer-0-9-0-cpu:dev env: @@ -88,7 +85,7 @@ spec: memory: 2Gi ports: - containerPort: 6379 - name: redis + name: gcs-server - containerPort: 8265 # Ray dashboard name: dashboard - containerPort: 10001 diff --git a/ray-operator/controllers/ray/rayservice_controller.go b/ray-operator/controllers/ray/rayservice_controller.go index ac774ec970..559415eb33 100644 --- a/ray-operator/controllers/ray/rayservice_controller.go +++ b/ray-operator/controllers/ray/rayservice_controller.go @@ -27,18 +27,20 @@ import ( ) var ( - rayServiceLog = logf.Log.WithName("rayservice-controller") - RayServiceDefaultRequeueDuration = 2 * time.Second - RayServiceRestartRequeueDuration = 10 * time.Second - rayClusterSuffix = "-raycluster" + rayServiceLog = logf.Log.WithName("rayservice-controller") + RayServiceDefaultRequeueDuration = 2 * time.Second + RayServiceRestartRequeueDuration = 10 * time.Second + RayServeDeploymentUnhealthSecondThreshold = 60.0 + rayClusterSuffix = "-raycluster" ) // RayServiceReconciler reconciles a RayService object type RayServiceReconciler struct { client.Client - Scheme *runtime.Scheme - Log logr.Logger - Recorder record.EventRecorder + Scheme *runtime.Scheme + Log logr.Logger + Recorder record.EventRecorder + // Now Ray dashboard does not cache serve deployment config. To avoid updating the same config repeatedly, cache the Serve Deployment config in this map. ServeDeploymentConfigMap map[types.NamespacedName]rayv1alpha1.RayServiceSpec } @@ -86,22 +88,22 @@ func (r *RayServiceReconciler) Reconcile(ctx context.Context, request ctrl.Reque var err error var rayServiceInstance *rayv1alpha1.RayService - if rayServiceInstance, err = r.getRayServiceInstance(request); err != nil { - err = r.updateState(rayServiceInstance, rayv1alpha1.FAIL_TO_GET_RAYSERVICE, err) + if rayServiceInstance, err = r.getRayServiceInstance(ctx, request); err != nil { + err = r.updateState(ctx, rayServiceInstance, rayv1alpha1.FailToGetRayService, err) return ctrl.Result{}, client.IgnoreNotFound(err) } var rayClusterInstance *rayv1alpha1.RayCluster - if rayClusterInstance, err = r.getOrCreateRayClusterInstance(rayServiceInstance); err != nil { - err = r.updateState(rayServiceInstance, rayv1alpha1.FAIL_TO_GET_OR_CREATE_RAYCLUSTER, err) + if rayClusterInstance, err = r.getOrCreateRayClusterInstance(ctx, rayServiceInstance); err != nil { + err = r.updateState(ctx, rayServiceInstance, rayv1alpha1.FailToGetOrCreateRayCluster, err) return ctrl.Result{}, client.IgnoreNotFound(err) } rayServiceInstance.Status.RayClusterStatus = rayClusterInstance.Status var clientURL string - if clientURL, err = r.fetchDashboardURL(rayClusterInstance); err != nil || clientURL == "" { - err = r.updateState(rayServiceInstance, rayv1alpha1.WAIT_FOR_DASHBOARD, err) + if clientURL, err = r.fetchDashboardURL(ctx, rayClusterInstance); err != nil || clientURL == "" { + err = r.updateState(ctx, rayServiceInstance, rayv1alpha1.WaitForDashboard, err) return ctrl.Result{}, err } @@ -112,27 +114,27 @@ func (r *RayServiceReconciler) Reconcile(ctx context.Context, request ctrl.Reque if shouldUpdate { if err = r.updateServeDeployment(rayServiceInstance, rayDashboardClient, request); err != nil { - err = r.updateState(rayServiceInstance, rayv1alpha1.FAIL_SERVE_DEPLOY, err) + err = r.updateState(ctx, rayServiceInstance, rayv1alpha1.FailServeDeploy, err) return ctrl.Result{}, err } } var isHealthy bool if isHealthy, err = r.getAndCheckServeStatus(rayServiceInstance, rayDashboardClient); err != nil { - err = r.updateState(rayServiceInstance, rayv1alpha1.FAIL_GET_SERVE_DEPLOYMENT_STATUS, err) + err = r.updateState(ctx, rayServiceInstance, rayv1alpha1.FailGetServeDeploymentStatus, err) return ctrl.Result{}, err } log.Info("Check serve health", "isHealthy", isHealthy) if isHealthy { - rayServiceInstance.Status.ServiceStatus = rayv1alpha1.RUNNING - if err := r.Status().Update(context.Background(), rayServiceInstance); err != nil { + rayServiceInstance.Status.ServiceStatus = rayv1alpha1.Running + if err := r.Status().Update(ctx, rayServiceInstance); err != nil { return ctrl.Result{}, err } } else { - rayServiceInstance.Status.ServiceStatus = rayv1alpha1.RESTARTING - if err := r.Status().Update(context.Background(), rayServiceInstance); err != nil { + rayServiceInstance.Status.ServiceStatus = rayv1alpha1.Restarting + if err := r.Status().Update(ctx, rayServiceInstance); err != nil { return ctrl.Result{}, err } @@ -140,7 +142,7 @@ func (r *RayServiceReconciler) Reconcile(ctx context.Context, request ctrl.Reque // restart raycluster if err := r.Delete(ctx, rayClusterInstance); err != nil { - err = r.updateState(rayServiceInstance, rayv1alpha1.FAIL_DELETE_RAYCLUSTER, err) + err = r.updateState(ctx, rayServiceInstance, rayv1alpha1.FailDeleteRayCluster, err) return ctrl.Result{}, err } @@ -159,9 +161,9 @@ func (r *RayServiceReconciler) SetupWithManager(mgr ctrl.Manager) error { Complete(r) } -func (r *RayServiceReconciler) getRayServiceInstance(request ctrl.Request) (*rayv1alpha1.RayService, error) { +func (r *RayServiceReconciler) getRayServiceInstance(ctx context.Context, request ctrl.Request) (*rayv1alpha1.RayService, error) { rayServiceInstance := &rayv1alpha1.RayService{} - if err := r.Get(context.TODO(), request.NamespacedName, rayServiceInstance); err != nil { + if err := r.Get(ctx, request.NamespacedName, rayServiceInstance); err != nil { if errors.IsNotFound(err) { rayServiceLog.Info("Read request instance not found error!") } else { @@ -173,15 +175,15 @@ func (r *RayServiceReconciler) getRayServiceInstance(request ctrl.Request) (*ray return rayServiceInstance, nil } -func (r *RayServiceReconciler) updateState(rayServiceInstance *rayv1alpha1.RayService, status rayv1alpha1.ServiceStatus, err error) error { +func (r *RayServiceReconciler) updateState(ctx context.Context, rayServiceInstance *rayv1alpha1.RayService, status rayv1alpha1.ServiceStatus, err error) error { rayServiceInstance.Status.ServiceStatus = status - if errStatus := r.Status().Update(context.Background(), rayServiceInstance); errStatus != nil { - return fmt.Errorf("combined error: %v %v", err, errStatus) + if errStatus := r.Status().Update(ctx, rayServiceInstance); errStatus != nil { + return fmtErrors.Errorf("combined error: %v %v", err, errStatus) } return err } -func (r *RayServiceReconciler) getOrCreateRayClusterInstance(rayServiceInstance *rayv1alpha1.RayService) (*rayv1alpha1.RayCluster, error) { +func (r *RayServiceReconciler) getOrCreateRayClusterInstance(ctx context.Context, rayServiceInstance *rayv1alpha1.RayService) (*rayv1alpha1.RayCluster, error) { // Update ray cluster rayClusterNamespacedName := types.NamespacedName{ Namespace: rayServiceInstance.Namespace, @@ -189,13 +191,13 @@ func (r *RayServiceReconciler) getOrCreateRayClusterInstance(rayServiceInstance } rayClusterInstance := &rayv1alpha1.RayCluster{} - err := r.Get(context.TODO(), rayClusterNamespacedName, rayClusterInstance) + err := r.Get(ctx, rayClusterNamespacedName, rayClusterInstance) if err == nil { rayClusterInstance.Spec = rayServiceInstance.Spec.RayClusterSpec r.Log.Info("Update ray cluster spec") - if err := r.Update(context.TODO(), rayClusterInstance); err != nil { + if err := r.Update(ctx, rayClusterInstance); err != nil { r.Log.Error(err, "Fail to update ray cluster instance!") // Error updating the RayCluster object. return nil, client.IgnoreNotFound(err) @@ -208,7 +210,7 @@ func (r *RayServiceReconciler) getOrCreateRayClusterInstance(rayServiceInstance // Error construct the RayCluster object - requeue the request. return nil, err } - if err := r.Create(context.TODO(), rayClusterInstance); err != nil { + if err := r.Create(ctx, rayClusterInstance); err != nil { r.Log.Error(err, "unable to create rayCluster for rayService", "rayCluster", rayClusterInstance) // Error creating the RayCluster object - requeue the request. return nil, err @@ -244,10 +246,10 @@ func (r *RayServiceReconciler) constructRayClusterForRayService(rayService *rayv return rayCluster, nil } -func (r *RayServiceReconciler) fetchDashboardURL(rayCluster *rayv1alpha1.RayCluster) (string, error) { +func (r *RayServiceReconciler) fetchDashboardURL(ctx context.Context, rayCluster *rayv1alpha1.RayCluster) (string, error) { headServices := corev1.ServiceList{} filterLabels := client.MatchingLabels{common.RayClusterLabelKey: rayCluster.Name} - if err := r.List(context.TODO(), &headServices, client.InNamespace(rayCluster.Namespace), filterLabels); err != nil { + if err := r.List(ctx, &headServices, client.InNamespace(rayCluster.Namespace), filterLabels); err != nil { return "", err } @@ -287,7 +289,7 @@ func (r *RayServiceReconciler) fetchDashboardURL(rayCluster *rayv1alpha1.RayClus } } - // Create head service if there's no existing one in the cluster. + // Rely on RayCluster controller to create head service. if headServices.Items == nil || len(headServices.Items) == 0 { return "", fmtErrors.Errorf("No head service found") } @@ -305,7 +307,7 @@ func (r *RayServiceReconciler) checkIfNeedSubmitServeDeployment(rayServiceInstan } r.Log.Info("shouldUpdate value", "shouldUpdate", shouldUpdate) - log.V(1).Info("status check", "len(rayServiceInstance.Status.ServeStatuses.Statuses) ", len(rayServiceInstance.Status.ServeStatuses.Statuses), "len(existConfig.ServeConfigSpecs)", len(existConfig.ServeConfigSpecs)) + log.V(1).Info("status check", "len(rayServiceInstance.Status.ServeDeploymentStatuses.Statuses) ", len(rayServiceInstance.Status.ServeStatuses.Statuses), "len(existConfig.ServeConfigSpecs)", len(existConfig.ServeConfigSpecs)) return shouldUpdate } @@ -322,14 +324,14 @@ func (r *RayServiceReconciler) updateServeDeployment(rayServiceInstance *rayv1al } func (r *RayServiceReconciler) getAndCheckServeStatus(rayServiceInstance *rayv1alpha1.RayService, dashboardClient utils.RayDashboardClient) (bool, error) { - var serveStatuses *rayv1alpha1.ServeStatuses + var serveStatuses *rayv1alpha1.ServeDeploymentStatuses var err error if serveStatuses, err = dashboardClient.GetDeploymentsStatus(); err != nil { r.Log.Error(err, "fail to get deployment status") return false, err } - statusMap := make(map[string]rayv1alpha1.ServeStatus) + statusMap := make(map[string]rayv1alpha1.ServeDeploymentStatus) for _, status := range rayServiceInstance.Status.ServeStatuses.Statuses { statusMap[status.Name] = status @@ -344,7 +346,7 @@ func (r *RayServiceReconciler) getAndCheckServeStatus(rayServiceInstance *rayv1a if prevStatus.Status != "HEALTHY" { serveStatuses.Statuses[i].HealthLastUpdateTime = prevStatus.HealthLastUpdateTime - if time.Since(prevStatus.HealthLastUpdateTime.Time).Seconds() > 60 { + if time.Since(prevStatus.HealthLastUpdateTime.Time).Seconds() > RayServeDeploymentUnhealthSecondThreshold { isHealthy = false } } diff --git a/ray-operator/controllers/ray/utils/serve_httpclient.go b/ray-operator/controllers/ray/utils/serve_httpclient.go index a590dd82b8..6fa33a12c4 100644 --- a/ray-operator/controllers/ray/utils/serve_httpclient.go +++ b/ray-operator/controllers/ray/utils/serve_httpclient.go @@ -10,12 +10,41 @@ import ( ) var ( - DEPLOY_PATH = "/api/serve/deployments/" - STATUS_PATH = "/api/serve/deployments/status" + DeployPath = "/api/serve/deployments/" + StatusPath = "/api/serve/deployments/status" ) +// ServeConfigSpec defines the desired state of RayService, used by Ray Dashboard. +type ServeConfigSpec struct { + Name string `json:"name"` + ImportPath string `json:"import_path"` + InitArgs []string `json:"init_args,omitempty"` + InitKwargs map[string]string `json:"init_kwargs,omitempty"` + NumReplicas *int32 `json:"num_replicas,omitempty"` + RoutePrefix string `json:"route_prefix,omitempty"` + MaxConcurrentQueries *int32 `json:"max_concurrent_queries,omitempty"` + UserConfig map[string]string `json:"user_config,omitempty"` + AutoscalingConfig map[string]string `json:"autoscaling_config,omitempty"` + GracefulShutdownWaitLoopS *float64 `json:"graceful_shutdown_wait_loop_s,omitempty"` + GracefulShutdownTimeoutS *float64 `json:"graceful_shutdown_timeout_s,omitempty"` + HealthCheckPeriodS *float64 `json:"health_check_period_s,omitempty"` + HealthCheckTimeoutS *float64 `json:"health_check_timeout_s,omitempty"` + RayActorOptions RayActorOptionSpec `json:"ray_actor_options,omitempty"` +} + +// RayActorOptionSpec defines the desired state of RayActor, used by Ray Dashboard. +type RayActorOptionSpec struct { + RuntimeEnv map[string][]string `json:"runtime_env,omitempty"` + NumCpus *float64 `json:"num_cpus,omitempty"` + NumGpus *float64 `json:"num_gpus,omitempty"` + Memory *int32 `json:"memory,omitempty"` + ObjectStoreMemory *int32 `json:"object_store_memory,omitempty"` + Resources map[string]string `json:"resources,omitempty"` + AcceleratorType string `json:"accelerator_type,omitempty"` +} + type ServingClusterDeployments struct { - Deployments []rayv1alpha1.ServeConfigSpec `json:"deployments,omitempty"` + Deployments []ServeConfigSpec `json:"deployments,omitempty"` } type RayDashboardClient struct { @@ -29,7 +58,7 @@ func (r *RayDashboardClient) InitClient(url string) { } func (r *RayDashboardClient) GetDeployments() (string, error) { - req, err := http.NewRequest("GET", r.dashboardURL+DEPLOY_PATH, nil) + req, err := http.NewRequest("GET", r.dashboardURL+DeployPath, nil) if err != nil { return "", err } @@ -49,7 +78,7 @@ func (r *RayDashboardClient) GetDeployments() (string, error) { func (r *RayDashboardClient) UpdateDeployments(specs []rayv1alpha1.ServeConfigSpec) error { servingClusterDeployments := ServingClusterDeployments{ - Deployments: specs, + Deployments: r.convertServeConfig(specs), } deploymentJson, err := json.Marshal(servingClusterDeployments) @@ -58,7 +87,7 @@ func (r *RayDashboardClient) UpdateDeployments(specs []rayv1alpha1.ServeConfigSp return err } - req, err := http.NewRequest("PUT", r.dashboardURL+DEPLOY_PATH, bytes.NewBuffer(deploymentJson)) + req, err := http.NewRequest("PUT", r.dashboardURL+DeployPath, bytes.NewBuffer(deploymentJson)) if err != nil { return err } @@ -74,8 +103,8 @@ func (r *RayDashboardClient) UpdateDeployments(specs []rayv1alpha1.ServeConfigSp return nil } -func (r *RayDashboardClient) GetDeploymentsStatus() (*rayv1alpha1.ServeStatuses, error) { - req, err := http.NewRequest("GET", r.dashboardURL+STATUS_PATH, nil) +func (r *RayDashboardClient) GetDeploymentsStatus() (*rayv1alpha1.ServeDeploymentStatuses, error) { + req, err := http.NewRequest("GET", r.dashboardURL+StatusPath, nil) if err != nil { return nil, err } @@ -89,10 +118,43 @@ func (r *RayDashboardClient) GetDeploymentsStatus() (*rayv1alpha1.ServeStatuses, body, _ := ioutil.ReadAll(resp.Body) - var serveStatuses rayv1alpha1.ServeStatuses + var serveStatuses rayv1alpha1.ServeDeploymentStatuses if err = json.Unmarshal(body, &serveStatuses); err != nil { return nil, err } return &serveStatuses, nil } + +func (r *RayDashboardClient) convertServeConfig(specs []rayv1alpha1.ServeConfigSpec) []ServeConfigSpec { + serveConfigToSend := make([]ServeConfigSpec, len(specs)) + + for i, config := range specs { + serveConfigToSend[i] = ServeConfigSpec{ + Name: config.Name, + ImportPath: config.ImportPath, + InitArgs: config.InitArgs, + InitKwargs: config.InitKwargs, + NumReplicas: config.NumReplicas, + RoutePrefix: config.RoutePrefix, + MaxConcurrentQueries: config.MaxConcurrentQueries, + UserConfig: config.UserConfig, + AutoscalingConfig: config.AutoscalingConfig, + GracefulShutdownWaitLoopS: config.GracefulShutdownWaitLoopS, + GracefulShutdownTimeoutS: config.GracefulShutdownTimeoutS, + HealthCheckPeriodS: config.HealthCheckPeriodS, + HealthCheckTimeoutS: config.GracefulShutdownTimeoutS, + RayActorOptions: RayActorOptionSpec{ + RuntimeEnv: config.RayActorOptions.RuntimeEnv, + NumCpus: config.RayActorOptions.NumCpus, + NumGpus: config.RayActorOptions.NumGpus, + Memory: config.RayActorOptions.Memory, + ObjectStoreMemory: config.RayActorOptions.ObjectStoreMemory, + Resources: config.RayActorOptions.Resources, + AcceleratorType: config.RayActorOptions.AcceleratorType, + }, + } + } + + return serveConfigToSend +} From 8e6bee4519f6221353313da0ff8a2d90e8cdd9ef Mon Sep 17 00:00:00 2001 From: brucez-anyscale Date: Tue, 7 Jun 2022 17:42:07 -0700 Subject: [PATCH 10/19] address comments --- .../config/crd/bases/ray.io_rayservices.yaml | 169 +++--------------- .../config/manager/kustomization.yaml | 4 - 2 files changed, 26 insertions(+), 147 deletions(-) diff --git a/ray-operator/config/crd/bases/ray.io_rayservices.yaml b/ray-operator/config/crd/bases/ray.io_rayservices.yaml index 1f417b3d55..055a0cbecb 100644 --- a/ray-operator/config/crd/bases/ray.io_rayservices.yaml +++ b/ray-operator/config/crd/bases/ray.io_rayservices.yaml @@ -34,125 +34,6 @@ spec: spec: description: RayServiceSpec defines the desired state of RayService properties: - healthCheckConfig: - description: 'Probe describes a health check to be performed against - a container to determine whether it is alive ' - properties: - exec: - description: Exec specifies the action to take. - properties: - command: - description: 'Command is the command line to execute inside - the container, the working directory for the command ' - items: - type: string - type: array - type: object - failureThreshold: - description: Minimum consecutive failures for the probe to be - considered failed after having succeeded. - format: int32 - type: integer - grpc: - description: GRPC specifies an action involving a GRPC port. - properties: - port: - description: Port number of the gRPC service. Number must - be in the range 1 to 65535. - format: int32 - type: integer - service: - description: Service is the name of the service to place in - the gRPC HealthCheckRequest (see https://github. - type: string - required: - - port - type: object - httpGet: - description: HTTPGet specifies the http request to perform. - properties: - host: - description: Host name to connect to, defaults to the pod - IP. - type: string - httpHeaders: - description: Custom headers to set in the request. HTTP allows - repeated headers. - items: - description: HTTPHeader describes a custom header to be - used in HTTP probes - properties: - name: - description: The header field name - type: string - value: - description: The header field value - type: string - required: - - name - - value - type: object - type: array - path: - description: Path to access on the HTTP server. - type: string - port: - anyOf: - - type: integer - - type: string - description: Name or number of the port to access on the container. - Number must be in the range 1 to 65535. - x-kubernetes-int-or-string: true - scheme: - description: Scheme to use for connecting to the host. Defaults - to HTTP. - type: string - required: - - port - type: object - initialDelaySeconds: - description: Number of seconds after the container has started - before liveness probes are initiated. - format: int32 - type: integer - periodSeconds: - description: How often (in seconds) to perform the probe. Default - to 10 seconds. Minimum value is 1. - format: int32 - type: integer - successThreshold: - description: Minimum consecutive successes for the probe to be - considered successful after having failed. - format: int32 - type: integer - tcpSocket: - description: TCPSocket specifies an action involving a TCP port. - properties: - host: - description: 'Optional: Host name to connect to, defaults - to the pod IP.' - type: string - port: - anyOf: - - type: integer - - type: string - description: Number or name of the port to access on the container. - Number must be in the range 1 to 65535. - x-kubernetes-int-or-string: true - required: - - port - type: object - terminationGracePeriodSeconds: - description: Optional duration in seconds the pod needs to terminate - gracefully upon probe failure. - format: int64 - type: integer - timeoutSeconds: - description: Number of seconds after which the probe times out. - Defaults to 1 second. Minimum value is 1. - format: int32 - type: integer - type: object rayClusterConfig: description: 'EDIT THIS FILE! THIS IS SCAFFOLDING FOR YOU TO OWN! NOTE: json tags are required.' @@ -11587,74 +11468,76 @@ spec: - headGroupSpec type: object serveConfigs: + description: 'Important: Run "make" to regenerate code after modifying + this file' items: description: ServeConfigSpec defines the desired state of RayService properties: - autoscaling_config: + autoscalingConfig: additionalProperties: type: string type: object - graceful_shutdown_timeout_s: + gracefulShutdownTimeoutS: type: number - graceful_shutdown_wait_loop_s: + gracefulShutdownWaitLoopS: type: number - health_check_period_s: + healthCheckPeriodS: type: number - health_check_timeout_s: + healthCheckTimeoutS: type: number - import_path: + importPath: type: string - init_args: + initArgs: items: type: string type: array - init_kwargs: + initKwargs: additionalProperties: type: string type: object - max_concurrent_queries: + maxConcurrentQueries: format: int32 type: integer name: type: string - num_replicas: + numReplicas: format: int32 type: integer - ray_actor_options: + rayActorOptions: description: RayActorOptionSpec defines the desired state of RayActor properties: - accelerator_type: + acceleratorType: type: string memory: format: int32 type: integer - num_cpus: + numCpus: type: number - num_gpus: + numGpus: type: number - object_store_memory: + objectStoreMemory: format: int32 type: integer resources: additionalProperties: type: string type: object - runtime_env: + runtimeEnv: additionalProperties: items: type: string type: array type: object type: object - route_prefix: + routePrefix: type: string - user_config: + userConfig: additionalProperties: type: string type: object required: - - import_path + - importPath - name type: object type: array @@ -11696,14 +11579,14 @@ spec: state of cluster Important: Run "make" to regenerat' type: string type: object - serveStatuses: - description: ServeStatuses defines the desired states of all Serve - Deployments + serveDeploymentStatuses: + description: ServeDeploymentStatuses defines the desired states of + all Serve Deployments properties: statuses: items: - description: ServeStatus defines the desired state of Serve - Deployment + description: ServeDeploymentStatus defines the desired state + of Serve Deployment properties: healthLastUpdateTime: description: Keep track of how long the service is healthy. diff --git a/ray-operator/config/manager/kustomization.yaml b/ray-operator/config/manager/kustomization.yaml index 3006e30f68..7e3a0f40b3 100644 --- a/ray-operator/config/manager/kustomization.yaml +++ b/ray-operator/config/manager/kustomization.yaml @@ -3,7 +3,3 @@ resources: - service.yaml apiVersion: kustomize.config.k8s.io/v1beta1 kind: Kustomization -images: -- name: kuberay/operator - newName: brucezhang90/bruce-docker-repo - newTag: health_kuberay From 3744fb507569b4444669be63a23ad5bdf2a7c4d3 Mon Sep 17 00:00:00 2001 From: brucez-anyscale Date: Tue, 7 Jun 2022 23:04:55 -0700 Subject: [PATCH 11/19] address comments --- ray-operator/apis/ray/v1alpha1/rayservice_types.go | 8 ++++---- ray-operator/controllers/ray/utils/serve_httpclient.go | 8 ++++---- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/ray-operator/apis/ray/v1alpha1/rayservice_types.go b/ray-operator/apis/ray/v1alpha1/rayservice_types.go index 1271a8ce5a..6277cfd464 100644 --- a/ray-operator/apis/ray/v1alpha1/rayservice_types.go +++ b/ray-operator/apis/ray/v1alpha1/rayservice_types.go @@ -38,10 +38,10 @@ type ServeConfigSpec struct { MaxConcurrentQueries *int32 `json:"maxConcurrentQueries,omitempty"` UserConfig map[string]string `json:"userConfig,omitempty"` AutoscalingConfig map[string]string `json:"autoscalingConfig,omitempty"` - GracefulShutdownWaitLoopS *float64 `json:"gracefulShutdownWaitLoopS,omitempty"` - GracefulShutdownTimeoutS *float64 `json:"gracefulShutdownTimeoutS,omitempty"` - HealthCheckPeriodS *float64 `json:"healthCheckPeriodS,omitempty"` - HealthCheckTimeoutS *float64 `json:"healthCheckTimeoutS,omitempty"` + GracefulShutdownWaitLoopS *int32 `json:"gracefulShutdownWaitLoopS,omitempty"` + GracefulShutdownTimeoutS *int32 `json:"gracefulShutdownTimeoutS,omitempty"` + HealthCheckPeriodS *int32 `json:"healthCheckPeriodS,omitempty"` + HealthCheckTimeoutS *int32 `json:"healthCheckTimeoutS,omitempty"` RayActorOptions RayActorOptionSpec `json:"rayActorOptions,omitempty"` } diff --git a/ray-operator/controllers/ray/utils/serve_httpclient.go b/ray-operator/controllers/ray/utils/serve_httpclient.go index 6fa33a12c4..537bdea1d3 100644 --- a/ray-operator/controllers/ray/utils/serve_httpclient.go +++ b/ray-operator/controllers/ray/utils/serve_httpclient.go @@ -25,10 +25,10 @@ type ServeConfigSpec struct { MaxConcurrentQueries *int32 `json:"max_concurrent_queries,omitempty"` UserConfig map[string]string `json:"user_config,omitempty"` AutoscalingConfig map[string]string `json:"autoscaling_config,omitempty"` - GracefulShutdownWaitLoopS *float64 `json:"graceful_shutdown_wait_loop_s,omitempty"` - GracefulShutdownTimeoutS *float64 `json:"graceful_shutdown_timeout_s,omitempty"` - HealthCheckPeriodS *float64 `json:"health_check_period_s,omitempty"` - HealthCheckTimeoutS *float64 `json:"health_check_timeout_s,omitempty"` + GracefulShutdownWaitLoopS *int32 `json:"graceful_shutdown_wait_loop_s,omitempty"` + GracefulShutdownTimeoutS *int32 `json:"graceful_shutdown_timeout_s,omitempty"` + HealthCheckPeriodS *int32 `json:"health_check_period_s,omitempty"` + HealthCheckTimeoutS *int32 `json:"health_check_timeout_s,omitempty"` RayActorOptions RayActorOptionSpec `json:"ray_actor_options,omitempty"` } From 7e7f8215bd9429f916796b3dfc2a64b1958c4881 Mon Sep 17 00:00:00 2001 From: brucez-anyscale Date: Wed, 8 Jun 2022 14:44:51 -0700 Subject: [PATCH 12/19] address comments --- .../apis/ray/v1alpha1/rayservice_types.go | 16 +++-- .../controllers/ray/rayservice_controller.go | 69 ++++++++++++------- ray-operator/go.mod | 3 +- ray-operator/go.sum | 2 + 4 files changed, 60 insertions(+), 30 deletions(-) diff --git a/ray-operator/apis/ray/v1alpha1/rayservice_types.go b/ray-operator/apis/ray/v1alpha1/rayservice_types.go index 6277cfd464..3ed1d193e6 100644 --- a/ray-operator/apis/ray/v1alpha1/rayservice_types.go +++ b/ray-operator/apis/ray/v1alpha1/rayservice_types.go @@ -60,20 +60,24 @@ type RayActorOptionSpec struct { type RayServiceStatus struct { // Important: Run "make" to regenerate code after modifying this file ServiceStatus ServiceStatus `json:"serviceStatus,omitempty"` - ServeStatuses ServeDeploymentStatuses `json:"serveDeploymentStatuses,omitempty"` + ServeStatuses []ServeDeploymentStatus `json:"serveDeploymentStatuses,omitempty"` + RayClusterName string `json:"rayClusterName,omitempty"` RayClusterStatus RayClusterStatus `json:"rayClusterStatus,omitempty"` } -// ServeDeploymentStatuses defines the desired states of all Serve Deployments +// ServeDeploymentStatuses defines the current states of all Serve Deployments type ServeDeploymentStatuses struct { Statuses []ServeDeploymentStatus `json:"statuses,omitempty"` } -// ServeDeploymentStatus defines the desired state of Serve Deployment +// ServeDeploymentStatus defines the current state of Serve Deployment type ServeDeploymentStatus struct { - Name string `json:"name,omitempty"` - Status string `json:"status,omitempty"` - Message string `json:"message,omitempty"` + // Name, Status, Message are from Ray Dashboard to represent the state of a serve deployment. + Name string `json:"name,omitempty"` + // TODO: change status type to enum + Status string `json:"status,omitempty"` + Message string `json:"message,omitempty"` + LastUpdateTime metav1.Time `json:"lastUpdateTime,omitempty"` // Keep track of how long the service is healthy. // Update when Serve Deployment is healthy or first time convert to unhealthy from healthy. HealthLastUpdateTime metav1.Time `json:"healthLastUpdateTime,omitempty"` diff --git a/ray-operator/controllers/ray/rayservice_controller.go b/ray-operator/controllers/ray/rayservice_controller.go index 559415eb33..a69c621905 100644 --- a/ray-operator/controllers/ray/rayservice_controller.go +++ b/ray-operator/controllers/ray/rayservice_controller.go @@ -6,15 +6,17 @@ import ( "reflect" "time" + "github.com/google/uuid" + cmap "github.com/orcaman/concurrent-map" + + "github.com/go-logr/logr" + fmtErrors "github.com/pkg/errors" "github.com/ray-project/kuberay/ray-operator/controllers/ray/common" "github.com/ray-project/kuberay/ray-operator/controllers/ray/utils" corev1 "k8s.io/api/core/v1" "k8s.io/apimachinery/pkg/api/errors" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "k8s.io/apimachinery/pkg/types" - - "github.com/go-logr/logr" - fmtErrors "github.com/pkg/errors" "k8s.io/client-go/tools/record" "sigs.k8s.io/controller-runtime/pkg/manager" @@ -27,11 +29,15 @@ import ( ) var ( - rayServiceLog = logf.Log.WithName("rayservice-controller") + rayServiceLog = logf.Log.WithName("rayservice-controller") +) + +const ( RayServiceDefaultRequeueDuration = 2 * time.Second RayServiceRestartRequeueDuration = 10 * time.Second RayServeDeploymentUnhealthSecondThreshold = 60.0 - rayClusterSuffix = "-raycluster" + rayClusterSuffix = "-raycluster-" + servicePortName = "dashboard" ) // RayServiceReconciler reconciles a RayService object @@ -41,17 +47,17 @@ type RayServiceReconciler struct { Log logr.Logger Recorder record.EventRecorder // Now Ray dashboard does not cache serve deployment config. To avoid updating the same config repeatedly, cache the Serve Deployment config in this map. - ServeDeploymentConfigMap map[types.NamespacedName]rayv1alpha1.RayServiceSpec + ServeDeploymentConfigs cmap.ConcurrentMap } // NewRayServiceReconciler returns a new reconcile.Reconciler func NewRayServiceReconciler(mgr manager.Manager) *RayServiceReconciler { return &RayServiceReconciler{ - Client: mgr.GetClient(), - Scheme: mgr.GetScheme(), - Log: ctrl.Log.WithName("controllers").WithName("RayService"), - Recorder: mgr.GetEventRecorderFor("rayservice-controller"), - ServeDeploymentConfigMap: make(map[types.NamespacedName]rayv1alpha1.RayServiceSpec), + Client: mgr.GetClient(), + Scheme: mgr.GetScheme(), + Log: ctrl.Log.WithName("controllers").WithName("RayService"), + Recorder: mgr.GetEventRecorderFor("rayservice-controller"), + ServeDeploymentConfigs: cmap.New(), } } @@ -89,7 +95,6 @@ func (r *RayServiceReconciler) Reconcile(ctx context.Context, request ctrl.Reque var rayServiceInstance *rayv1alpha1.RayService if rayServiceInstance, err = r.getRayServiceInstance(ctx, request); err != nil { - err = r.updateState(ctx, rayServiceInstance, rayv1alpha1.FailToGetRayService, err) return ctrl.Result{}, client.IgnoreNotFound(err) } @@ -133,12 +138,13 @@ func (r *RayServiceReconciler) Reconcile(ctx context.Context, request ctrl.Reque return ctrl.Result{}, err } } else { + rayServiceInstance.Status.RayClusterName = "" rayServiceInstance.Status.ServiceStatus = rayv1alpha1.Restarting if err := r.Status().Update(ctx, rayServiceInstance); err != nil { return ctrl.Result{}, err } - delete(r.ServeDeploymentConfigMap, request.NamespacedName) + r.ServeDeploymentConfigs.Remove(request.NamespacedName.String()) // restart raycluster if err := r.Delete(ctx, rayClusterInstance); err != nil { @@ -185,9 +191,16 @@ func (r *RayServiceReconciler) updateState(ctx context.Context, rayServiceInstan func (r *RayServiceReconciler) getOrCreateRayClusterInstance(ctx context.Context, rayServiceInstance *rayv1alpha1.RayService) (*rayv1alpha1.RayCluster, error) { // Update ray cluster + var rayClusterInstanceName string + if rayServiceInstance.Status.RayClusterName != "" { + rayClusterInstanceName = rayServiceInstance.Status.RayClusterName + } else { + rayClusterInstanceName = rayServiceInstance.Name + rayClusterSuffix + uuid.New().String() + } + rayClusterNamespacedName := types.NamespacedName{ Namespace: rayServiceInstance.Namespace, - Name: rayServiceInstance.Name + rayClusterSuffix, + Name: rayClusterInstanceName, } rayClusterInstance := &rayv1alpha1.RayCluster{} @@ -197,6 +210,7 @@ func (r *RayServiceReconciler) getOrCreateRayClusterInstance(ctx context.Context rayClusterInstance.Spec = rayServiceInstance.Spec.RayClusterSpec r.Log.Info("Update ray cluster spec") + // TODO: Check which config fields can be handled by RayCluster controller. For other fields, need to start a new RayCluster. if err := r.Update(ctx, rayClusterInstance); err != nil { r.Log.Error(err, "Fail to update ray cluster instance!") // Error updating the RayCluster object. @@ -215,11 +229,13 @@ func (r *RayServiceReconciler) getOrCreateRayClusterInstance(ctx context.Context // Error creating the RayCluster object - requeue the request. return nil, err } + // Update RayCluster name in RayService state. + rayServiceInstance.Status.RayClusterName = rayClusterInstanceName r.Log.V(1).Info("created rayCluster for rayService run", "rayCluster", rayClusterInstance) } else { r.Log.Error(err, "Get request rayCluster instance error!") // Error reading the RayCluster object - requeue the request. - return nil, client.IgnoreNotFound(err) + return nil, err } return rayClusterInstance, nil @@ -264,7 +280,7 @@ func (r *RayServiceReconciler) fetchDashboardURL(ctx context.Context, rayCluster dashboardPort := int32(-1) for _, servicePort := range servicePorts { - if servicePort.Name == "dashboard" { + if servicePort.Name == servicePortName { dashboardPort = servicePort.Port break } @@ -298,16 +314,22 @@ func (r *RayServiceReconciler) fetchDashboardURL(ctx context.Context, rayCluster } func (r *RayServiceReconciler) checkIfNeedSubmitServeDeployment(rayServiceInstance *rayv1alpha1.RayService, request ctrl.Request) bool { - existConfig, exist := r.ServeDeploymentConfigMap[request.NamespacedName] + existConfigObj, exist := r.ServeDeploymentConfigs.Get(request.NamespacedName.String()) + + if !exist { + r.Log.Info("shouldUpdate value, config does not exist") + return true + } + + existConfig, ok := existConfigObj.(rayv1alpha1.RayServiceSpec) shouldUpdate := false - if !exist || !reflect.DeepEqual(existConfig, rayServiceInstance.Spec) || len(rayServiceInstance.Status.ServeStatuses.Statuses) != len(existConfig.ServeConfigSpecs) { + if !ok || !reflect.DeepEqual(existConfig, rayServiceInstance.Spec) || len(rayServiceInstance.Status.ServeStatuses) != len(existConfig.ServeConfigSpecs) { shouldUpdate = true } - r.Log.Info("shouldUpdate value", "shouldUpdate", shouldUpdate) - log.V(1).Info("status check", "len(rayServiceInstance.Status.ServeDeploymentStatuses.Statuses) ", len(rayServiceInstance.Status.ServeStatuses.Statuses), "len(existConfig.ServeConfigSpecs)", len(existConfig.ServeConfigSpecs)) + r.Log.Info("shouldUpdate value", "shouldUpdate", shouldUpdate) return shouldUpdate } @@ -319,7 +341,7 @@ func (r *RayServiceReconciler) updateServeDeployment(rayServiceInstance *rayv1al return err } - r.ServeDeploymentConfigMap[request.NamespacedName] = rayServiceInstance.Spec + r.ServeDeploymentConfigs.Set(request.NamespacedName.String(), rayServiceInstance.Spec) return nil } @@ -333,12 +355,13 @@ func (r *RayServiceReconciler) getAndCheckServeStatus(rayServiceInstance *rayv1a statusMap := make(map[string]rayv1alpha1.ServeDeploymentStatus) - for _, status := range rayServiceInstance.Status.ServeStatuses.Statuses { + for _, status := range rayServiceInstance.Status.ServeStatuses { statusMap[status.Name] = status } isHealthy := true for i := 0; i < len(serveStatuses.Statuses); i++ { + serveStatuses.Statuses[i].LastUpdateTime = metav1.Now() serveStatuses.Statuses[i].HealthLastUpdateTime = metav1.Now() if serveStatuses.Statuses[i].Status != "HEALTHY" { prevStatus, exist := statusMap[serveStatuses.Statuses[i].Name] @@ -354,7 +377,7 @@ func (r *RayServiceReconciler) getAndCheckServeStatus(rayServiceInstance *rayv1a } } - rayServiceInstance.Status.ServeStatuses = *serveStatuses + rayServiceInstance.Status.ServeStatuses = serveStatuses.Statuses r.Log.Info("getAndCheckServeStatus ", "statusMap", statusMap, "serveStatuses", serveStatuses) diff --git a/ray-operator/go.mod b/ray-operator/go.mod index 90d1ebbfc5..77d320ef15 100644 --- a/ray-operator/go.mod +++ b/ray-operator/go.mod @@ -6,6 +6,8 @@ require ( github.com/go-logr/logr v1.2.0 github.com/onsi/ginkgo v1.16.5 github.com/onsi/gomega v1.17.0 + github.com/orcaman/concurrent-map v1.0.0 + github.com/pkg/errors v0.9.1 github.com/prometheus/client_golang v1.11.0 github.com/sirupsen/logrus v1.8.1 github.com/stretchr/testify v1.7.0 @@ -46,7 +48,6 @@ require ( github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect github.com/modern-go/reflect2 v1.0.2 // indirect github.com/nxadm/tail v1.4.8 // indirect - github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/prometheus/client_model v0.2.0 // indirect github.com/prometheus/common v0.28.0 // indirect diff --git a/ray-operator/go.sum b/ray-operator/go.sum index d94173e268..6729d58d9e 100644 --- a/ray-operator/go.sum +++ b/ray-operator/go.sum @@ -366,6 +366,8 @@ github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1y github.com/onsi/gomega v1.17.0 h1:9Luw4uT5HTjHTN8+aNcSThgH1vdXnmdJ8xIfZ4wyTRE= github.com/onsi/gomega v1.17.0/go.mod h1:HnhC7FXeEQY45zxNK3PPoIUhzk/80Xly9PcubAlGdZY= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= +github.com/orcaman/concurrent-map v1.0.0 h1:I/2A2XPCb4IuQWcQhBhSwGfiuybl/J0ev9HDbW65HOY= +github.com/orcaman/concurrent-map v1.0.0/go.mod h1:Lu3tH6HLW3feq74c2GC+jIMS/K2CFcDWnWD9XkenwhI= github.com/pascaldekloe/goe v0.0.0-20180627143212-57f6aae5913c/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc= github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic= github.com/pelletier/go-toml v1.9.3/go.mod h1:u1nR/EPcESfeI/szUZKdtJ0xRNbUoANCkoOuaOx1Y+c= From 850131793638904fccc2e13716e2635903c8c288 Mon Sep 17 00:00:00 2001 From: brucez-anyscale Date: Wed, 8 Jun 2022 15:16:56 -0700 Subject: [PATCH 13/19] address comments --- .../apis/ray/v1alpha1/rayservice_types.go | 2 +- .../controllers/ray/rayservice_controller.go | 57 ++++++------------- 2 files changed, 19 insertions(+), 40 deletions(-) diff --git a/ray-operator/apis/ray/v1alpha1/rayservice_types.go b/ray-operator/apis/ray/v1alpha1/rayservice_types.go index 3ed1d193e6..c99130b0b6 100644 --- a/ray-operator/apis/ray/v1alpha1/rayservice_types.go +++ b/ray-operator/apis/ray/v1alpha1/rayservice_types.go @@ -10,7 +10,6 @@ import ( type ServiceStatus string const ( - FailToGetRayService ServiceStatus = "FailToGetRayService" FailToGetOrCreateRayCluster ServiceStatus = "FailToGetOrCreateRayCluster" WaitForDashboard ServiceStatus = "WaitForDashboard" FailServeDeploy ServiceStatus = "FailServeDeploy" @@ -28,6 +27,7 @@ type RayServiceSpec struct { } // ServeConfigSpec defines the desired state of RayService +// Reference to https://docs.ray.io/en/latest/ray-core/package-ref.html#ray-remote. type ServeConfigSpec struct { Name string `json:"name"` ImportPath string `json:"importPath"` diff --git a/ray-operator/controllers/ray/rayservice_controller.go b/ray-operator/controllers/ray/rayservice_controller.go index a69c621905..15cc9ac0be 100644 --- a/ray-operator/controllers/ray/rayservice_controller.go +++ b/ray-operator/controllers/ray/rayservice_controller.go @@ -11,7 +11,6 @@ import ( "github.com/go-logr/logr" fmtErrors "github.com/pkg/errors" - "github.com/ray-project/kuberay/ray-operator/controllers/ray/common" "github.com/ray-project/kuberay/ray-operator/controllers/ray/utils" corev1 "k8s.io/api/core/v1" "k8s.io/apimachinery/pkg/api/errors" @@ -263,54 +262,34 @@ func (r *RayServiceReconciler) constructRayClusterForRayService(rayService *rayv } func (r *RayServiceReconciler) fetchDashboardURL(ctx context.Context, rayCluster *rayv1alpha1.RayCluster) (string, error) { - headServices := corev1.ServiceList{} - filterLabels := client.MatchingLabels{common.RayClusterLabelKey: rayCluster.Name} - if err := r.List(ctx, &headServices, client.InNamespace(rayCluster.Namespace), filterLabels); err != nil { + var headService *corev1.Service + if err := r.Get(ctx, client.ObjectKey{Name: rayCluster.Name, Namespace: rayCluster.Namespace}, headService); err != nil { return "", err } - dashboardURL := "" + r.Log.Info("reconcileServices ", "head service found", headService.Name) + // TODO: compare diff and reconcile the object. For example. ServiceType might be changed or port might be modified + servicePorts := headService.Spec.Ports - if headServices.Items != nil { - if len(headServices.Items) == 1 { - r.Log.Info("reconcileServices ", "head service found", headServices.Items[0].Name) - // TODO: compare diff and reconcile the object. For example. ServiceType might be changed or port might be modified - servicePorts := headServices.Items[0].Spec.Ports + dashboardPort := int32(-1) - dashboardPort := int32(-1) - - for _, servicePort := range servicePorts { - if servicePort.Name == servicePortName { - dashboardPort = servicePort.Port - break - } - } - - if dashboardPort == int32(-1) { - return "", fmtErrors.Errorf("dashboard port not found") - } - - dashboardURL = fmt.Sprintf("%s.%s.svc.cluster.local:%v", - headServices.Items[0].Name, - headServices.Items[0].Namespace, - dashboardPort) - return dashboardURL, nil - } - - // This should never happen. - // We add the protection here just in case controller has race issue or user manually create service with same label. - if len(headServices.Items) > 1 { - r.Log.Info("reconcileServices ", "Duplicates head service found", len(headServices.Items)) - return "", fmtErrors.Errorf("Duplicates head service found %v", len(headServices.Items)) + for _, servicePort := range servicePorts { + if servicePort.Name == servicePortName { + dashboardPort = servicePort.Port + break } } - // Rely on RayCluster controller to create head service. - if headServices.Items == nil || len(headServices.Items) == 0 { - return "", fmtErrors.Errorf("No head service found") + if dashboardPort == int32(-1) { + return "", fmtErrors.Errorf("dashboard port not found") } - return "", nil + dashboardURL := fmt.Sprintf("%s.%s.svc.cluster.local:%v", + headService.Name, + headService.Namespace, + dashboardPort) + + return dashboardURL, nil } func (r *RayServiceReconciler) checkIfNeedSubmitServeDeployment(rayServiceInstance *rayv1alpha1.RayService, request ctrl.Request) bool { From 5772e23ea79ff50bb8e5c929d096b7d181ee5f47 Mon Sep 17 00:00:00 2001 From: brucez-anyscale Date: Wed, 8 Jun 2022 17:44:04 -0700 Subject: [PATCH 14/19] address comments --- .../config/crd/bases/ray.io_rayservices.yaml | 58 +++++++++++-------- .../config/manager/kustomization.yaml | 2 +- .../samples/ray_v1alpha1_rayservice.yaml | 2 + .../controllers/ray/rayservice_controller.go | 23 ++++---- 4 files changed, 49 insertions(+), 36 deletions(-) diff --git a/ray-operator/config/crd/bases/ray.io_rayservices.yaml b/ray-operator/config/crd/bases/ray.io_rayservices.yaml index 055a0cbecb..c0624a811c 100644 --- a/ray-operator/config/crd/bases/ray.io_rayservices.yaml +++ b/ray-operator/config/crd/bases/ray.io_rayservices.yaml @@ -11472,19 +11472,24 @@ spec: this file' items: description: ServeConfigSpec defines the desired state of RayService + Reference to https://docs.ray. properties: autoscalingConfig: additionalProperties: type: string type: object gracefulShutdownTimeoutS: - type: number + format: int32 + type: integer gracefulShutdownWaitLoopS: - type: number + format: int32 + type: integer healthCheckPeriodS: - type: number + format: int32 + type: integer healthCheckTimeoutS: - type: number + format: int32 + type: integer importPath: type: string initArgs: @@ -11545,6 +11550,8 @@ spec: status: description: RayServiceStatus defines the observed state of RayService properties: + rayClusterName: + type: string rayClusterStatus: description: RayClusterStatus defines the observed state of RayCluster properties: @@ -11580,27 +11587,28 @@ spec: type: string type: object serveDeploymentStatuses: - description: ServeDeploymentStatuses defines the desired states of - all Serve Deployments - properties: - statuses: - items: - description: ServeDeploymentStatus defines the desired state - of Serve Deployment - properties: - healthLastUpdateTime: - description: Keep track of how long the service is healthy. - format: date-time - type: string - message: - type: string - name: - type: string - status: - type: string - type: object - type: array - type: object + items: + description: ServeDeploymentStatus defines the current state of + Serve Deployment + properties: + healthLastUpdateTime: + description: Keep track of how long the service is healthy. + format: date-time + type: string + lastUpdateTime: + format: date-time + type: string + message: + type: string + name: + description: Name, Status, Message are from Ray Dashboard to + represent the state of a serve deployment. + type: string + status: + description: 'TODO: change status type to enum' + type: string + type: object + type: array serviceStatus: description: 'Important: Run "make" to regenerate code after modifying this file' diff --git a/ray-operator/config/manager/kustomization.yaml b/ray-operator/config/manager/kustomization.yaml index 7e3a0f40b3..61c3432c12 100644 --- a/ray-operator/config/manager/kustomization.yaml +++ b/ray-operator/config/manager/kustomization.yaml @@ -2,4 +2,4 @@ resources: - manager.yaml - service.yaml apiVersion: kustomize.config.k8s.io/v1beta1 -kind: Kustomization +kind: Kustomization \ No newline at end of file diff --git a/ray-operator/config/samples/ray_v1alpha1_rayservice.yaml b/ray-operator/config/samples/ray_v1alpha1_rayservice.yaml index e25f295d4e..51cd74b1ea 100644 --- a/ray-operator/config/samples/ray_v1alpha1_rayservice.yaml +++ b/ray-operator/config/samples/ray_v1alpha1_rayservice.yaml @@ -40,6 +40,8 @@ spec: headGroupSpec: # Kubernetes Service Type, valid values are 'ClusterIP', 'NodePort' and 'LoadBalancer' serviceType: ClusterIP + # the pod replicas in this group typed head (assuming there could be more than 1 in the future) + replicas: 1 # logical group name, for this called head-group, also can be functional # pod type head or worker # rayNodeType: head # Not needed since it is under the headgroup diff --git a/ray-operator/controllers/ray/rayservice_controller.go b/ray-operator/controllers/ray/rayservice_controller.go index 15cc9ac0be..4a291b4304 100644 --- a/ray-operator/controllers/ray/rayservice_controller.go +++ b/ray-operator/controllers/ray/rayservice_controller.go @@ -6,7 +6,8 @@ import ( "reflect" "time" - "github.com/google/uuid" + "k8s.io/apimachinery/pkg/util/rand" + cmap "github.com/orcaman/concurrent-map" "github.com/go-logr/logr" @@ -103,6 +104,8 @@ func (r *RayServiceReconciler) Reconcile(ctx context.Context, request ctrl.Reque return ctrl.Result{}, client.IgnoreNotFound(err) } + log.Info("Updated RayCluster") + rayServiceInstance.Status.RayClusterStatus = rayClusterInstance.Status var clientURL string @@ -183,7 +186,7 @@ func (r *RayServiceReconciler) getRayServiceInstance(ctx context.Context, reques func (r *RayServiceReconciler) updateState(ctx context.Context, rayServiceInstance *rayv1alpha1.RayService, status rayv1alpha1.ServiceStatus, err error) error { rayServiceInstance.Status.ServiceStatus = status if errStatus := r.Status().Update(ctx, rayServiceInstance); errStatus != nil { - return fmtErrors.Errorf("combined error: %v %v", err, errStatus) + return fmtErrors.Errorf("combined error: \n %v \n %v", err, errStatus) } return err } @@ -194,9 +197,11 @@ func (r *RayServiceReconciler) getOrCreateRayClusterInstance(ctx context.Context if rayServiceInstance.Status.RayClusterName != "" { rayClusterInstanceName = rayServiceInstance.Status.RayClusterName } else { - rayClusterInstanceName = rayServiceInstance.Name + rayClusterSuffix + uuid.New().String() + rayClusterInstanceName = fmt.Sprintf("%s%s%s", rayServiceInstance.Name, rayClusterSuffix, rand.String(5)) } + r.Log.Info("getOrCreateRayClusterInstance", "rayClusterInstanceName", rayClusterInstanceName) + rayClusterNamespacedName := types.NamespacedName{ Namespace: rayServiceInstance.Namespace, Name: rayClusterInstanceName, @@ -217,7 +222,7 @@ func (r *RayServiceReconciler) getOrCreateRayClusterInstance(ctx context.Context } } else if errors.IsNotFound(err) { r.Log.Info("Not found rayCluster, creating rayCluster!") - rayClusterInstance, err = r.constructRayClusterForRayService(rayServiceInstance) + rayClusterInstance, err = r.constructRayClusterForRayService(rayServiceInstance, rayClusterInstanceName) if err != nil { r.Log.Error(err, "unable to construct rayCluster from spec") // Error construct the RayCluster object - requeue the request. @@ -240,14 +245,12 @@ func (r *RayServiceReconciler) getOrCreateRayClusterInstance(ctx context.Context return rayClusterInstance, nil } -func (r *RayServiceReconciler) constructRayClusterForRayService(rayService *rayv1alpha1.RayService) (*rayv1alpha1.RayCluster, error) { - name := fmt.Sprintf("%s%s", rayService.Name, rayClusterSuffix) - +func (r *RayServiceReconciler) constructRayClusterForRayService(rayService *rayv1alpha1.RayService, rayClusterName string) (*rayv1alpha1.RayCluster, error) { rayCluster := &rayv1alpha1.RayCluster{ ObjectMeta: metav1.ObjectMeta{ Labels: rayService.Labels, Annotations: rayService.Annotations, - Name: name, + Name: rayClusterName, Namespace: rayService.Namespace, }, Spec: *rayService.Spec.RayClusterSpec.DeepCopy(), @@ -262,8 +265,8 @@ func (r *RayServiceReconciler) constructRayClusterForRayService(rayService *rayv } func (r *RayServiceReconciler) fetchDashboardURL(ctx context.Context, rayCluster *rayv1alpha1.RayCluster) (string, error) { - var headService *corev1.Service - if err := r.Get(ctx, client.ObjectKey{Name: rayCluster.Name, Namespace: rayCluster.Namespace}, headService); err != nil { + headService := &corev1.Service{} + if err := r.Get(ctx, client.ObjectKey{Name: utils.GenerateServiceName(rayCluster.Name), Namespace: rayCluster.Namespace}, headService); err != nil { return "", err } From 51653a3d27e1c6a31472f51e2ea9de1907143e2e Mon Sep 17 00:00:00 2001 From: brucez-anyscale Date: Thu, 9 Jun 2022 00:07:17 -0700 Subject: [PATCH 15/19] add unit tests --- .../ray/v1alpha1/rayservice_types_test.go | 143 ++++++++++++++ .../controllers/ray/rayservice_controller.go | 12 +- .../ray/rayservice_controller_test.go | 179 ++++++++++++++++++ 3 files changed, 328 insertions(+), 6 deletions(-) create mode 100644 ray-operator/apis/ray/v1alpha1/rayservice_types_test.go create mode 100644 ray-operator/controllers/ray/rayservice_controller_test.go diff --git a/ray-operator/apis/ray/v1alpha1/rayservice_types_test.go b/ray-operator/apis/ray/v1alpha1/rayservice_types_test.go new file mode 100644 index 0000000000..03f6823c71 --- /dev/null +++ b/ray-operator/apis/ray/v1alpha1/rayservice_types_test.go @@ -0,0 +1,143 @@ +package v1alpha1 + +import ( + "encoding/json" + "testing" + + corev1 "k8s.io/api/core/v1" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/utils/pointer" +) + +var numReplicas int32 +var numCpus float64 + +var myRayService = &RayService{ + ObjectMeta: metav1.ObjectMeta{ + Name: "rayservice-sample", + Namespace: "default", + }, + Spec: RayServiceSpec{ + ServeConfigSpecs: []ServeConfigSpec{ + { + Name: "shallow", + ImportPath: "test_env.shallow_import.ShallowClass", + NumReplicas: &numReplicas, + RoutePrefix: "/shallow", + RayActorOptions: RayActorOptionSpec{ + NumCpus: &numCpus, + RuntimeEnv: map[string][]string{ + "py_modules": { + "https://github.com/shrekris-anyscale/test_deploy_group/archive/HEAD.zip", + "https://github.com/shrekris-anyscale/test_module/archive/HEAD.zip", + }, + }, + }, + }, + { + Name: "deep", + ImportPath: "test_env.subdir1.subdir2.deep_import.DeepClass", + NumReplicas: &numReplicas, + RoutePrefix: "/deep", + RayActorOptions: RayActorOptionSpec{ + NumCpus: &numCpus, + }, + }, + }, + RayClusterSpec: RayClusterSpec{ + RayVersion: "1.0", + HeadGroupSpec: HeadGroupSpec{ + Replicas: pointer.Int32Ptr(1), + RayStartParams: map[string]string{ + "port": "6379", + "object-manager-port": "12345", + "node-manager-port": "12346", + "object-store-memory": "100000000", + "redis-password": "LetMeInRay", + "num-cpus": "1", + }, + Template: corev1.PodTemplateSpec{ + ObjectMeta: metav1.ObjectMeta{ + Namespace: "default", + Labels: map[string]string{ + "rayCluster": "raycluster-sample", + "groupName": "headgroup", + }, + }, + Spec: corev1.PodSpec{ + Containers: []corev1.Container{ + { + Name: "ray-head", + Image: "rayproject/autoscaler", + Command: []string{"python"}, + Args: []string{"/opt/code.py"}, + Env: []corev1.EnvVar{ + { + Name: "MY_POD_IP", + ValueFrom: &corev1.EnvVarSource{ + FieldRef: &corev1.ObjectFieldSelector{ + FieldPath: "status.podIP", + }, + }, + }, + }, + }, + }, + }, + }, + }, + WorkerGroupSpecs: []WorkerGroupSpec{ + { + Replicas: pointer.Int32Ptr(3), + MinReplicas: pointer.Int32Ptr(0), + MaxReplicas: pointer.Int32Ptr(10000), + GroupName: "small-group", + RayStartParams: map[string]string{ + "port": "6379", + "redis-password": "LetMeInRay", + "num-cpus": "1", + }, + Template: corev1.PodTemplateSpec{ + ObjectMeta: metav1.ObjectMeta{ + Namespace: "default", + Labels: map[string]string{ + "rayCluster": "raycluster-sample", + "groupName": "small-group", + }, + }, + Spec: corev1.PodSpec{ + Containers: []corev1.Container{ + { + Name: "ray-worker", + Image: "rayproject/autoscaler", + Command: []string{"echo"}, + Args: []string{"Hello Ray"}, + Env: []corev1.EnvVar{ + { + Name: "MY_POD_IP", + ValueFrom: &corev1.EnvVarSource{ + FieldRef: &corev1.ObjectFieldSelector{ + FieldPath: "status.podIP", + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, +} + +func TestMarshallingRayService(t *testing.T) { + numReplicas = 1 + numCpus = 0.1 + // marshal successfully + _, err := json.Marshal(&myRayService) + if err != nil { + t.Fatalf("Expected `%v` but got `%v`", nil, err) + } +} diff --git a/ray-operator/controllers/ray/rayservice_controller.go b/ray-operator/controllers/ray/rayservice_controller.go index 4a291b4304..054488d4ca 100644 --- a/ray-operator/controllers/ray/rayservice_controller.go +++ b/ray-operator/controllers/ray/rayservice_controller.go @@ -33,11 +33,11 @@ var ( ) const ( - RayServiceDefaultRequeueDuration = 2 * time.Second - RayServiceRestartRequeueDuration = 10 * time.Second - RayServeDeploymentUnhealthSecondThreshold = 60.0 - rayClusterSuffix = "-raycluster-" - servicePortName = "dashboard" + RayServiceDefaultRequeueDuration = 2 * time.Second + RayServiceRestartRequeueDuration = 10 * time.Second + RayServeDeploymentUnhealthySecondThreshold = 60.0 + rayClusterSuffix = "-raycluster-" + servicePortName = "dashboard" ) // RayServiceReconciler reconciles a RayService object @@ -351,7 +351,7 @@ func (r *RayServiceReconciler) getAndCheckServeStatus(rayServiceInstance *rayv1a if prevStatus.Status != "HEALTHY" { serveStatuses.Statuses[i].HealthLastUpdateTime = prevStatus.HealthLastUpdateTime - if time.Since(prevStatus.HealthLastUpdateTime.Time).Seconds() > RayServeDeploymentUnhealthSecondThreshold { + if time.Since(prevStatus.HealthLastUpdateTime.Time).Seconds() > RayServeDeploymentUnhealthySecondThreshold { isHealthy = false } } diff --git a/ray-operator/controllers/ray/rayservice_controller_test.go b/ray-operator/controllers/ray/rayservice_controller_test.go new file mode 100644 index 0000000000..ab8ecd3d2c --- /dev/null +++ b/ray-operator/controllers/ray/rayservice_controller_test.go @@ -0,0 +1,179 @@ +/* + +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 ray + +import ( + "context" + "time" + + . "github.com/onsi/ginkgo" + . "github.com/onsi/gomega" + rayiov1alpha1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1alpha1" + + corev1 "k8s.io/api/core/v1" + "k8s.io/utils/pointer" + + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "sigs.k8s.io/controller-runtime/pkg/client" + // +kubebuilder:scaffold:imports +) + +var _ = Context("Inside the default namespace", func() { + ctx := context.TODO() + var workerPods corev1.PodList + var enableInTreeAutoscaling = true + + var numReplicas int32 + var numCpus float64 + numReplicas = 1 + numCpus = 0.1 + + myRayService := &rayiov1alpha1.RayService{ + ObjectMeta: metav1.ObjectMeta{ + Name: "raycluster-sample", + Namespace: "default", + }, + Spec: rayiov1alpha1.RayServiceSpec{ + ServeConfigSpecs: []rayiov1alpha1.ServeConfigSpec{ + { + Name: "shallow", + ImportPath: "test_env.shallow_import.ShallowClass", + NumReplicas: &numReplicas, + RoutePrefix: "/shallow", + RayActorOptions: rayiov1alpha1.RayActorOptionSpec{ + NumCpus: &numCpus, + RuntimeEnv: map[string][]string{ + "py_modules": { + "https://github.com/shrekris-anyscale/test_deploy_group/archive/HEAD.zip", + "https://github.com/shrekris-anyscale/test_module/archive/HEAD.zip", + }, + }, + }, + }, + }, + RayClusterSpec: rayiov1alpha1.RayClusterSpec{ + RayVersion: "1.0", + EnableInTreeAutoscaling: &enableInTreeAutoscaling, + HeadGroupSpec: rayiov1alpha1.HeadGroupSpec{ + ServiceType: "ClusterIP", + Replicas: pointer.Int32Ptr(1), + RayStartParams: map[string]string{ + "port": "6379", + "object-manager-port": "12345", + "node-manager-port": "12346", + "object-store-memory": "100000000", + "redis-password": "LetMeInRay", + "num-cpus": "1", + }, + Template: corev1.PodTemplateSpec{ + Spec: corev1.PodSpec{ + ServiceAccountName: "head-service-account", + Containers: []corev1.Container{ + { + Name: "ray-head", + Image: "rayproject/autoscaler", + Command: []string{"python"}, + Args: []string{"/opt/code.py"}, + Env: []corev1.EnvVar{ + { + Name: "MY_POD_IP", + ValueFrom: &corev1.EnvVarSource{ + FieldRef: &corev1.ObjectFieldSelector{ + FieldPath: "status.podIP", + }, + }, + }, + }, + }, + }, + }, + }, + }, + WorkerGroupSpecs: []rayiov1alpha1.WorkerGroupSpec{ + { + Replicas: pointer.Int32Ptr(3), + MinReplicas: pointer.Int32Ptr(0), + MaxReplicas: pointer.Int32Ptr(10000), + GroupName: "small-group", + RayStartParams: map[string]string{ + "port": "6379", + "redis-password": "LetMeInRay", + "num-cpus": "1", + }, + Template: corev1.PodTemplateSpec{ + Spec: corev1.PodSpec{ + Containers: []corev1.Container{ + { + Name: "ray-worker", + Image: "rayproject/autoscaler", + Command: []string{"echo"}, + Args: []string{"Hello Ray"}, + Env: []corev1.EnvVar{ + { + Name: "MY_POD_IP", + ValueFrom: &corev1.EnvVarSource{ + FieldRef: &corev1.ObjectFieldSelector{ + FieldPath: "status.podIP", + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + } + + Describe("When creating a rayservice", func() { + It("should create a rayservice object", func() { + err := k8sClient.Create(ctx, myRayService) + Expect(err).NotTo(HaveOccurred(), "failed to create test RayService resource") + }) + + It("should see a rayservice object", func() { + Eventually( + getResourceFunc(ctx, client.ObjectKey{Name: myRayService.Name, Namespace: "default"}, myRayService), + time.Second*3, time.Millisecond*500).Should(BeNil(), "My myRayService = %v", myRayService.Name) + }) + + It("should see one serve deployment", func() { + Eventually( + getResourceFunc(ctx, client.ObjectKey{Name: myRayService.Name, Namespace: "default"}, myRayService), + time.Second*30, time.Millisecond*500).Should(BeNil(), "My myRayService = %v", myRayService.Name) + Expect(len(myRayService.Status.ServeStatuses), 1) + Expect(myRayService.Status.ServeStatuses[0].Name, "shallow") + }) + + It("should update a rayservice object", func() { + // adding a scale strategy + Eventually( + getResourceFunc(ctx, client.ObjectKey{Name: myRayService.Name, Namespace: "default"}, myRayService), + time.Second*3, time.Millisecond*500).Should(BeNil(), "My myRayService = %v", myRayService.Name) + + podToDelete1 := workerPods.Items[0] + rep := new(int32) + *rep = 1 + myRayService.Spec.RayClusterSpec.WorkerGroupSpecs[0].Replicas = rep + myRayService.Spec.RayClusterSpec.WorkerGroupSpecs[0].ScaleStrategy.WorkersToDelete = []string{podToDelete1.Name} + + Expect(k8sClient.Update(ctx, myRayService)).Should(Succeed(), "failed to update test RayService resource") + }) + }) +}) From 9157ac2c733e12c84f6d1e0b4c1618ef52930f90 Mon Sep 17 00:00:00 2001 From: brucez-anyscale Date: Thu, 9 Jun 2022 01:48:37 -0700 Subject: [PATCH 16/19] fix ut --- .../ray/rayservice_controller_test.go | 37 ++++++++++++++++--- ray-operator/controllers/ray/suite_test.go | 5 ++- 2 files changed, 35 insertions(+), 7 deletions(-) diff --git a/ray-operator/controllers/ray/rayservice_controller_test.go b/ray-operator/controllers/ray/rayservice_controller_test.go index ab8ecd3d2c..207b8b3da6 100644 --- a/ray-operator/controllers/ray/rayservice_controller_test.go +++ b/ray-operator/controllers/ray/rayservice_controller_test.go @@ -17,13 +17,17 @@ package ray import ( "context" + "fmt" "time" + "github.com/ray-project/kuberay/ray-operator/controllers/ray/common" + . "github.com/onsi/ginkgo" . "github.com/onsi/gomega" rayiov1alpha1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1alpha1" corev1 "k8s.io/api/core/v1" + v1 "k8s.io/api/core/v1" "k8s.io/utils/pointer" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" @@ -43,7 +47,7 @@ var _ = Context("Inside the default namespace", func() { myRayService := &rayiov1alpha1.RayService{ ObjectMeta: metav1.ObjectMeta{ - Name: "raycluster-sample", + Name: "rayservice-sample", Namespace: "default", }, Spec: rayiov1alpha1.RayServiceSpec{ @@ -153,12 +157,24 @@ var _ = Context("Inside the default namespace", func() { time.Second*3, time.Millisecond*500).Should(BeNil(), "My myRayService = %v", myRayService.Name) }) - It("should see one serve deployment", func() { + It("should create a raycluster object", func() { Eventually( - getResourceFunc(ctx, client.ObjectKey{Name: myRayService.Name, Namespace: "default"}, myRayService), - time.Second*30, time.Millisecond*500).Should(BeNil(), "My myRayService = %v", myRayService.Name) - Expect(len(myRayService.Status.ServeStatuses), 1) - Expect(myRayService.Status.ServeStatuses[0].Name, "shallow") + getRayClusterNameFunc(ctx, myRayService), + time.Second*15, time.Millisecond*500).Should(Not(BeEmpty()), "My RayCluster name = %v", myRayService.Status.RayClusterName) + myRayCluster := &rayiov1alpha1.RayCluster{} + Eventually( + getResourceFunc(ctx, client.ObjectKey{Name: myRayService.Status.RayClusterName, Namespace: "default"}, myRayCluster), + time.Second*3, time.Millisecond*500).Should(BeNil(), "My myRayCluster = %v", myRayCluster.Name) + }) + + It("should create more than 1 worker", func() { + filterLabels := client.MatchingLabels{common.RayClusterLabelKey: myRayService.Status.RayClusterName, common.RayNodeGroupLabelKey: "small-group"} + Eventually( + listResourceFunc(ctx, &workerPods, filterLabels, &client.ListOptions{Namespace: "default"}), + time.Second*15, time.Millisecond*500).Should(Equal(3), fmt.Sprintf("workerGroup %v", workerPods.Items)) + if len(workerPods.Items) > 0 { + Expect(workerPods.Items[0].Status.Phase).Should(Or(Equal(v1.PodRunning), Equal(v1.PodPending))) + } }) It("should update a rayservice object", func() { @@ -177,3 +193,12 @@ var _ = Context("Inside the default namespace", func() { }) }) }) + +func getRayClusterNameFunc(ctx context.Context, rayService *rayiov1alpha1.RayService) func() (string, error) { + return func() (string, error) { + if err := k8sClient.Get(ctx, client.ObjectKey{Name: rayService.Name, Namespace: "default"}, rayService); err != nil { + return "", err + } + return rayService.Status.RayClusterName, nil + } +} diff --git a/ray-operator/controllers/ray/suite_test.go b/ray-operator/controllers/ray/suite_test.go index 5d2aaf335d..5e4d1b6df9 100644 --- a/ray-operator/controllers/ray/suite_test.go +++ b/ray-operator/controllers/ray/suite_test.go @@ -84,7 +84,10 @@ var _ = BeforeSuite(func(done Done) { Expect(err).NotTo(HaveOccurred(), "failed to create manager") err = NewReconciler(mgr).SetupWithManager(mgr, 1) - Expect(err).NotTo(HaveOccurred(), "failed to setup controller") + Expect(err).NotTo(HaveOccurred(), "failed to setup RayCluster controller") + + err = NewRayServiceReconciler(mgr).SetupWithManager(mgr) + Expect(err).NotTo(HaveOccurred(), "failed to setup RayService controller") go func() { err = mgr.Start(ctrl.SetupSignalHandler()) From d91d49f0d2d1a7f75d6fbfe696304afaefea001f Mon Sep 17 00:00:00 2001 From: brucez-anyscale Date: Thu, 9 Jun 2022 10:49:08 -0700 Subject: [PATCH 17/19] Update yaml --- ray-operator/config/manager/kustomization.yaml | 3 ++- .../config/samples/ray_v1alpha1_rayservice.yaml | 12 ++++++------ 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/ray-operator/config/manager/kustomization.yaml b/ray-operator/config/manager/kustomization.yaml index 61c3432c12..7477b240f1 100644 --- a/ray-operator/config/manager/kustomization.yaml +++ b/ray-operator/config/manager/kustomization.yaml @@ -2,4 +2,5 @@ resources: - manager.yaml - service.yaml apiVersion: kustomize.config.k8s.io/v1beta1 -kind: Kustomization \ No newline at end of file +kind: Kustomization + diff --git a/ray-operator/config/samples/ray_v1alpha1_rayservice.yaml b/ray-operator/config/samples/ray_v1alpha1_rayservice.yaml index 51cd74b1ea..42ff125407 100644 --- a/ray-operator/config/samples/ray_v1alpha1_rayservice.yaml +++ b/ray-operator/config/samples/ray_v1alpha1_rayservice.yaml @@ -12,8 +12,8 @@ spec: numCpus: 0.1 runtimeEnv: py_modules: - - "https://github.com/shrekris-anyscale/test_deploy_group/archive/HEAD.zip" - - "https://github.com/shrekris-anyscale/test_module/archive/HEAD.zip" + - "https://github.com/ray-project/test_deploy_group/archive/67971777e225600720f91f618cdfe71fc47f60ee.zip" + - "https://github.com/ray-project/test_module/archive/aa6f366f7daa78c98408c27d917a983caa9f888b.zip" - name: deep importPath: "test_env.subdir1.subdir2.deep_import.DeepClass" numReplicas: 1 @@ -22,8 +22,8 @@ spec: numCpus: 0.1 runtimeEnv: py_modules: - - "https://github.com/shrekris-anyscale/test_deploy_group/archive/HEAD.zip" - - "https://github.com/shrekris-anyscale/test_module/archive/HEAD.zip" + - "https://github.com/ray-project/test_deploy_group/archive/67971777e225600720f91f618cdfe71fc47f60ee.zip" + - "https://github.com/ray-project/test_module/archive/aa6f366f7daa78c98408c27d917a983caa9f888b.zip" - name: one importPath: "test_module.test.one" numReplicas: 1 @@ -31,8 +31,8 @@ spec: numCpus: 0.1 runtimeEnv: py_modules: - - "https://github.com/shrekris-anyscale/test_deploy_group/archive/HEAD.zip" - - "https://github.com/shrekris-anyscale/test_module/archive/HEAD.zip" + - "https://github.com/ray-project/test_deploy_group/archive/67971777e225600720f91f618cdfe71fc47f60ee.zip" + - "https://github.com/ray-project/test_module/archive/aa6f366f7daa78c98408c27d917a983caa9f888b.zip" rayClusterConfig: rayVersion: '1.12.1' # should match the Ray version in the image of the containers ######################headGroupSpecs################################# From 6c4be798450c1b4008a2dca0066a571941e1a966 Mon Sep 17 00:00:00 2001 From: brucez-anyscale Date: Thu, 9 Jun 2022 14:29:07 -0700 Subject: [PATCH 18/19] update ut --- .../ray/v1alpha1/rayservice_types_test.go | 271 ++++++++++++++++-- .../ray/rayservice_controller_test.go | 102 +++++-- 2 files changed, 336 insertions(+), 37 deletions(-) diff --git a/ray-operator/apis/ray/v1alpha1/rayservice_types_test.go b/ray-operator/apis/ray/v1alpha1/rayservice_types_test.go index 03f6823c71..8298070ccc 100644 --- a/ray-operator/apis/ray/v1alpha1/rayservice_types_test.go +++ b/ray-operator/apis/ray/v1alpha1/rayservice_types_test.go @@ -4,6 +4,9 @@ import ( "encoding/json" "testing" + "github.com/stretchr/testify/require" + "k8s.io/apimachinery/pkg/api/resource" + corev1 "k8s.io/api/core/v1" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "k8s.io/utils/pointer" @@ -28,8 +31,8 @@ var myRayService = &RayService{ NumCpus: &numCpus, RuntimeEnv: map[string][]string{ "py_modules": { - "https://github.com/shrekris-anyscale/test_deploy_group/archive/HEAD.zip", - "https://github.com/shrekris-anyscale/test_module/archive/HEAD.zip", + "https://github.com/ray-project/test_deploy_group/archive/67971777e225600720f91f618cdfe71fc47f60ee.zip", + "https://github.com/ray-project/test_module/archive/aa6f366f7daa78c98408c27d917a983caa9f888b.zip", }, }, }, @@ -41,36 +44,58 @@ var myRayService = &RayService{ RoutePrefix: "/deep", RayActorOptions: RayActorOptionSpec{ NumCpus: &numCpus, + RuntimeEnv: map[string][]string{ + "py_modules": { + "https://github.com/ray-project/test_deploy_group/archive/67971777e225600720f91f618cdfe71fc47f60ee.zip", + "https://github.com/ray-project/test_module/archive/aa6f366f7daa78c98408c27d917a983caa9f888b.zip", + }, + }, + }, + }, + { + Name: "one", + ImportPath: "test_module.test.one", + NumReplicas: &numReplicas, + RayActorOptions: RayActorOptionSpec{ + NumCpus: &numCpus, + RuntimeEnv: map[string][]string{ + "py_modules": { + "https://github.com/ray-project/test_deploy_group/archive/67971777e225600720f91f618cdfe71fc47f60ee.zip", + "https://github.com/ray-project/test_module/archive/aa6f366f7daa78c98408c27d917a983caa9f888b.zip", + }, + }, }, }, }, RayClusterSpec: RayClusterSpec{ - RayVersion: "1.0", + RayVersion: "1.12.1", HeadGroupSpec: HeadGroupSpec{ - Replicas: pointer.Int32Ptr(1), + ServiceType: corev1.ServiceTypeClusterIP, + Replicas: pointer.Int32Ptr(1), RayStartParams: map[string]string{ "port": "6379", - "object-manager-port": "12345", - "node-manager-port": "12346", "object-store-memory": "100000000", - "redis-password": "LetMeInRay", + "dashboard-host": "0.0.0.0", "num-cpus": "1", + "node-ip-address": "127.0.0.1", + "block": "true", }, Template: corev1.PodTemplateSpec{ ObjectMeta: metav1.ObjectMeta{ - Namespace: "default", Labels: map[string]string{ - "rayCluster": "raycluster-sample", - "groupName": "headgroup", + "rayCluster": "raycluster-sample", + "rayNodeType": "head", + "groupName": "headgroup", + }, + Annotations: map[string]string{ + "key": "value", }, }, Spec: corev1.PodSpec{ Containers: []corev1.Container{ { - Name: "ray-head", - Image: "rayproject/autoscaler", - Command: []string{"python"}, - Args: []string{"/opt/code.py"}, + Name: "ray-head", + Image: "rayproject/ray:1.12.1", Env: []corev1.EnvVar{ { Name: "MY_POD_IP", @@ -81,6 +106,30 @@ var myRayService = &RayService{ }, }, }, + Resources: corev1.ResourceRequirements{ + Limits: corev1.ResourceList{ + corev1.ResourceCPU: resource.MustParse("1"), + corev1.ResourceMemory: resource.MustParse("2Gi"), + }, + Requests: corev1.ResourceList{ + corev1.ResourceCPU: resource.MustParse("1"), + corev1.ResourceMemory: resource.MustParse("2Gi"), + }, + }, + Ports: []corev1.ContainerPort{ + { + Name: "gcs-server", + ContainerPort: 6379, + }, + { + Name: "dashboard", + ContainerPort: 8265, + }, + { + Name: "head", + ContainerPort: 10001, + }, + }, }, }, }, @@ -93,9 +142,8 @@ var myRayService = &RayService{ MaxReplicas: pointer.Int32Ptr(10000), GroupName: "small-group", RayStartParams: map[string]string{ - "port": "6379", - "redis-password": "LetMeInRay", - "num-cpus": "1", + "port": "6379", + "num-cpus": "1", }, Template: corev1.PodTemplateSpec{ ObjectMeta: metav1.ObjectMeta{ @@ -109,7 +157,7 @@ var myRayService = &RayService{ Containers: []corev1.Container{ { Name: "ray-worker", - Image: "rayproject/autoscaler", + Image: "rayproject/ray:1.12.1", Command: []string{"echo"}, Args: []string{"Hello Ray"}, Env: []corev1.EnvVar{ @@ -132,12 +180,197 @@ var myRayService = &RayService{ }, } +var expected = `{ + "metadata":{ + "name":"rayservice-sample", + "namespace":"default", + "creationTimestamp":null + }, + "spec":{ + "serveConfigs":[ + { + "name":"shallow", + "importPath":"test_env.shallow_import.ShallowClass", + "numReplicas":1, + "routePrefix":"/shallow", + "rayActorOptions":{ + "runtimeEnv":{ + "py_modules":[ + "https://github.com/ray-project/test_deploy_group/archive/67971777e225600720f91f618cdfe71fc47f60ee.zip", + "https://github.com/ray-project/test_module/archive/aa6f366f7daa78c98408c27d917a983caa9f888b.zip" + ] + }, + "numCpus":0.1 + } + }, + { + "name":"deep", + "importPath":"test_env.subdir1.subdir2.deep_import.DeepClass", + "numReplicas":1, + "routePrefix":"/deep", + "rayActorOptions":{ + "runtimeEnv":{ + "py_modules":[ + "https://github.com/ray-project/test_deploy_group/archive/67971777e225600720f91f618cdfe71fc47f60ee.zip", + "https://github.com/ray-project/test_module/archive/aa6f366f7daa78c98408c27d917a983caa9f888b.zip" + ] + }, + "numCpus":0.1 + } + }, + { + "name":"one", + "importPath":"test_module.test.one", + "numReplicas":1, + "rayActorOptions":{ + "runtimeEnv":{ + "py_modules":[ + "https://github.com/ray-project/test_deploy_group/archive/67971777e225600720f91f618cdfe71fc47f60ee.zip", + "https://github.com/ray-project/test_module/archive/aa6f366f7daa78c98408c27d917a983caa9f888b.zip" + ] + }, + "numCpus":0.1 + } + } + ], + "rayClusterConfig":{ + "headGroupSpec":{ + "serviceType":"ClusterIP", + "replicas":1, + "rayStartParams":{ + "block":"true", + "dashboard-host":"0.0.0.0", + "node-ip-address":"127.0.0.1", + "num-cpus":"1", + "object-store-memory":"100000000", + "port":"6379" + }, + "template":{ + "metadata":{ + "creationTimestamp":null, + "labels":{ + "groupName":"headgroup", + "rayCluster":"raycluster-sample", + "rayNodeType":"head" + }, + "annotations":{ + "key":"value" + } + }, + "spec":{ + "containers":[ + { + "name":"ray-head", + "image":"rayproject/ray:1.12.1", + "ports":[ + { + "name":"gcs-server", + "containerPort":6379 + }, + { + "name":"dashboard", + "containerPort":8265 + }, + { + "name":"head", + "containerPort":10001 + } + ], + "env":[ + { + "name":"MY_POD_IP", + "valueFrom":{ + "fieldRef":{ + "fieldPath":"status.podIP" + } + } + } + ], + "resources":{ + "limits":{ + "cpu":"1", + "memory":"2Gi" + }, + "requests":{ + "cpu":"1", + "memory":"2Gi" + } + } + } + ] + } + } + }, + "workerGroupSpecs":[ + { + "groupName":"small-group", + "replicas":3, + "minReplicas":0, + "maxReplicas":10000, + "rayStartParams":{ + "num-cpus":"1", + "port":"6379" + }, + "template":{ + "metadata":{ + "namespace":"default", + "creationTimestamp":null, + "labels":{ + "groupName":"small-group", + "rayCluster":"raycluster-sample" + } + }, + "spec":{ + "containers":[ + { + "name":"ray-worker", + "image":"rayproject/ray:1.12.1", + "command":[ + "echo" + ], + "args":[ + "Hello Ray" + ], + "env":[ + { + "name":"MY_POD_IP", + "valueFrom":{ + "fieldRef":{ + "fieldPath":"status.podIP" + } + } + } + ], + "resources":{ + + } + } + ] + } + }, + "scaleStrategy":{ + + } + } + ], + "rayVersion":"1.12.1" + } + }, + "status":{ + "rayClusterStatus":{ + "lastUpdateTime":null + } + } +}` + func TestMarshallingRayService(t *testing.T) { numReplicas = 1 numCpus = 0.1 // marshal successfully - _, err := json.Marshal(&myRayService) + myRayServiceJson, err := json.Marshal(&myRayService) if err != nil { t.Fatalf("Expected `%v` but got `%v`", nil, err) } + + require.JSONEq(t, expected, string(myRayServiceJson)) } diff --git a/ray-operator/controllers/ray/rayservice_controller_test.go b/ray-operator/controllers/ray/rayservice_controller_test.go index 207b8b3da6..3d407c55c3 100644 --- a/ray-operator/controllers/ray/rayservice_controller_test.go +++ b/ray-operator/controllers/ray/rayservice_controller_test.go @@ -20,6 +20,8 @@ import ( "fmt" "time" + "k8s.io/apimachinery/pkg/api/resource" + "github.com/ray-project/kuberay/ray-operator/controllers/ray/common" . "github.com/onsi/ginkgo" @@ -38,7 +40,6 @@ import ( var _ = Context("Inside the default namespace", func() { ctx := context.TODO() var workerPods corev1.PodList - var enableInTreeAutoscaling = true var numReplicas int32 var numCpus float64 @@ -61,36 +62,71 @@ var _ = Context("Inside the default namespace", func() { NumCpus: &numCpus, RuntimeEnv: map[string][]string{ "py_modules": { - "https://github.com/shrekris-anyscale/test_deploy_group/archive/HEAD.zip", - "https://github.com/shrekris-anyscale/test_module/archive/HEAD.zip", + "https://github.com/ray-project/test_deploy_group/archive/67971777e225600720f91f618cdfe71fc47f60ee.zip", + "https://github.com/ray-project/test_module/archive/aa6f366f7daa78c98408c27d917a983caa9f888b.zip", + }, + }, + }, + }, + { + Name: "deep", + ImportPath: "test_env.subdir1.subdir2.deep_import.DeepClass", + NumReplicas: &numReplicas, + RoutePrefix: "/deep", + RayActorOptions: rayiov1alpha1.RayActorOptionSpec{ + NumCpus: &numCpus, + RuntimeEnv: map[string][]string{ + "py_modules": { + "https://github.com/ray-project/test_deploy_group/archive/67971777e225600720f91f618cdfe71fc47f60ee.zip", + "https://github.com/ray-project/test_module/archive/aa6f366f7daa78c98408c27d917a983caa9f888b.zip", + }, + }, + }, + }, + { + Name: "one", + ImportPath: "test_module.test.one", + NumReplicas: &numReplicas, + RayActorOptions: rayiov1alpha1.RayActorOptionSpec{ + NumCpus: &numCpus, + RuntimeEnv: map[string][]string{ + "py_modules": { + "https://github.com/ray-project/test_deploy_group/archive/67971777e225600720f91f618cdfe71fc47f60ee.zip", + "https://github.com/ray-project/test_module/archive/aa6f366f7daa78c98408c27d917a983caa9f888b.zip", }, }, }, }, }, RayClusterSpec: rayiov1alpha1.RayClusterSpec{ - RayVersion: "1.0", - EnableInTreeAutoscaling: &enableInTreeAutoscaling, + RayVersion: "1.12.1", HeadGroupSpec: rayiov1alpha1.HeadGroupSpec{ - ServiceType: "ClusterIP", + ServiceType: corev1.ServiceTypeClusterIP, Replicas: pointer.Int32Ptr(1), RayStartParams: map[string]string{ "port": "6379", - "object-manager-port": "12345", - "node-manager-port": "12346", "object-store-memory": "100000000", - "redis-password": "LetMeInRay", + "dashboard-host": "0.0.0.0", "num-cpus": "1", + "node-ip-address": "127.0.0.1", + "block": "true", }, Template: corev1.PodTemplateSpec{ + ObjectMeta: metav1.ObjectMeta{ + Labels: map[string]string{ + "rayCluster": "raycluster-sample", + "rayNodeType": "head", + "groupName": "headgroup", + }, + Annotations: map[string]string{ + "key": "value", + }, + }, Spec: corev1.PodSpec{ - ServiceAccountName: "head-service-account", Containers: []corev1.Container{ { - Name: "ray-head", - Image: "rayproject/autoscaler", - Command: []string{"python"}, - Args: []string{"/opt/code.py"}, + Name: "ray-head", + Image: "rayproject/ray:1.12.1", Env: []corev1.EnvVar{ { Name: "MY_POD_IP", @@ -101,6 +137,30 @@ var _ = Context("Inside the default namespace", func() { }, }, }, + Resources: corev1.ResourceRequirements{ + Limits: corev1.ResourceList{ + corev1.ResourceCPU: resource.MustParse("1"), + corev1.ResourceMemory: resource.MustParse("2Gi"), + }, + Requests: corev1.ResourceList{ + corev1.ResourceCPU: resource.MustParse("1"), + corev1.ResourceMemory: resource.MustParse("2Gi"), + }, + }, + Ports: []corev1.ContainerPort{ + { + Name: "gcs-server", + ContainerPort: 6379, + }, + { + Name: "dashboard", + ContainerPort: 8265, + }, + { + Name: "head", + ContainerPort: 10001, + }, + }, }, }, }, @@ -113,16 +173,22 @@ var _ = Context("Inside the default namespace", func() { MaxReplicas: pointer.Int32Ptr(10000), GroupName: "small-group", RayStartParams: map[string]string{ - "port": "6379", - "redis-password": "LetMeInRay", - "num-cpus": "1", + "port": "6379", + "num-cpus": "1", }, Template: corev1.PodTemplateSpec{ + ObjectMeta: metav1.ObjectMeta{ + Namespace: "default", + Labels: map[string]string{ + "rayCluster": "raycluster-sample", + "groupName": "small-group", + }, + }, Spec: corev1.PodSpec{ Containers: []corev1.Container{ { Name: "ray-worker", - Image: "rayproject/autoscaler", + Image: "rayproject/ray:1.12.1", Command: []string{"echo"}, Args: []string{"Hello Ray"}, Env: []corev1.EnvVar{ From 1e3e1c0a74ff7e2904a081255092b6b86f5b6f7f Mon Sep 17 00:00:00 2001 From: brucez-anyscale Date: Thu, 9 Jun 2022 15:55:32 -0700 Subject: [PATCH 19/19] update --- ray-operator/apis/ray/v1alpha1/rayservice_types_test.go | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/ray-operator/apis/ray/v1alpha1/rayservice_types_test.go b/ray-operator/apis/ray/v1alpha1/rayservice_types_test.go index 8298070ccc..d275a3e43b 100644 --- a/ray-operator/apis/ray/v1alpha1/rayservice_types_test.go +++ b/ray-operator/apis/ray/v1alpha1/rayservice_types_test.go @@ -12,8 +12,8 @@ import ( "k8s.io/utils/pointer" ) -var numReplicas int32 -var numCpus float64 +var numReplicas int32 = 1 +var numCpus = 0.1 var myRayService = &RayService{ ObjectMeta: metav1.ObjectMeta{ @@ -364,8 +364,6 @@ var expected = `{ }` func TestMarshallingRayService(t *testing.T) { - numReplicas = 1 - numCpus = 0.1 // marshal successfully myRayServiceJson, err := json.Marshal(&myRayService) if err != nil {