diff --git a/ray-operator/Makefile b/ray-operator/Makefile index c826f6ad9f..bcb569b1cb 100644 --- a/ray-operator/Makefile +++ b/ray-operator/Makefile @@ -43,8 +43,13 @@ help: ## Display this help. manifests: controller-gen ## Generate WebhookConfiguration, ClusterRole and CustomResourceDefinition objects. $(CONTROLLER_GEN) $(CRD_OPTIONS) rbac:roleName=kuberay-operator webhook paths="./..." output:crd:artifacts:config=config/crd/bases -generate: controller-gen ## Generate code containing DeepCopy, DeepCopyInto, and DeepCopyObject method implementations. +## Generate code containing DeepCopy, DeepCopyInto, and DeepCopyObject method implementations. +generate: controller-gen conversion-gen $(CONTROLLER_GEN) object:headerFile="hack/boilerplate.go.txt" paths="./..." + $(CONVERSION_GEN) \ + --input-dirs "./apis/ray/v1beta1" \ + --go-header-file "./hack/boilerplate.go.txt" \ + --output-file-base "zz_generated.conversion" --v 2 helm: manifests kustomize ## Sync the CRDs into the Helm chart rm -r ../helm-chart/kuberay-operator/crds/ @@ -117,6 +122,10 @@ GOFUMPT = $(shell pwd)/bin/gofumpt gofumpt: ## Download gofumpt locally if necessary. $(call go-get-tool,$(GOFUMPT),mvdan.cc/gofumpt@latest) +CONVERSION_GEN = $(shell pwd)/bin/conversion-gen +conversion-gen: ## Download conversion-gen locally if necessary. + $(call go-get-tool,$(CONVERSION_GEN),k8s.io/code-generator/cmd/conversion-gen@v0.26.1) + # go-get-tool will 'go get' any package $2 and install it to $1. PROJECT_DIR := $(shell dirname $(abspath $(lastword $(MAKEFILE_LIST)))) define go-get-tool diff --git a/ray-operator/PROJECT b/ray-operator/PROJECT index b0a117e1d9..a55c44dafe 100644 --- a/ray-operator/PROJECT +++ b/ray-operator/PROJECT @@ -1,3 +1,7 @@ +# Code generated by tool. DO NOT EDIT. +# This file is used to track the info used to scaffold your project +# and allow the plugins properly work. +# More info: https://book.kubebuilder.io/reference/project-config.html domain: io layout: - go.kubebuilder.io/v3 @@ -23,6 +27,9 @@ resources: kind: RayService path: github.com/ray-project/kuberay/ray-operator/apis/ray/v1alpha1 version: v1alpha1 + webhooks: + conversion: true + webhookVersion: v1 - api: crdVersion: v1 namespaced: true @@ -32,4 +39,20 @@ resources: kind: RayJob path: github.com/ray-project/kuberay/ray-operator/apis/ray/v1alpha1 version: v1alpha1 +- api: + crdVersion: v1 + namespaced: true + domain: io + group: ray + kind: RayService + path: github.com/ray-project/kuberay/ray-operator/apis/ray/v1beta1 + version: v1beta1 +- api: + crdVersion: v1 + namespaced: true + domain: io + group: ray + kind: RayCluster + path: github.com/ray-project/kuberay/ray-operator/apis/ray/v1beta1 + version: v1beta1 version: "3" diff --git a/ray-operator/apis/ray/v1alpha1/doc.go b/ray-operator/apis/ray/v1alpha1/doc.go index 5027bb804f..d4bac839ac 100644 --- a/ray-operator/apis/ray/v1alpha1/doc.go +++ b/ray-operator/apis/ray/v1alpha1/doc.go @@ -1,2 +1,3 @@ // +groupName=ray.io +// +kubebuilder:object:generate=true package v1alpha1 diff --git a/ray-operator/apis/ray/v1alpha1/raycluster_types.go b/ray-operator/apis/ray/v1alpha1/raycluster_types.go index 03aac6bf87..73a6654a3a 100644 --- a/ray-operator/apis/ray/v1alpha1/raycluster_types.go +++ b/ray-operator/apis/ray/v1alpha1/raycluster_types.go @@ -7,7 +7,10 @@ import ( // 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. -// var app appsv1.Deployment{} + +// Hub marks this type as a conversion hub. +func (*RayCluster) Hub() {} + // RayClusterSpec defines the desired state of RayCluster type RayClusterSpec struct { // INSERT ADDITIONAL SPEC FIELDS - desired state of cluster @@ -155,6 +158,7 @@ const ( // RayCluster is the Schema for the RayClusters API // +kubebuilder:object:root=true // +kubebuilder:subresource:status +// +kubebuilder:storageversion // +kubebuilder:printcolumn:name="desired workers",type=integer,JSONPath=".status.desiredWorkerReplicas",priority=0 // +kubebuilder:printcolumn:name="available workers",type=integer,JSONPath=".status.availableWorkerReplicas",priority=0 // +kubebuilder:printcolumn:name="status",type="string",JSONPath=".status.state",priority=0 diff --git a/ray-operator/apis/ray/v1alpha1/rayservice_types.go b/ray-operator/apis/ray/v1alpha1/rayservice_types.go index 4c1618703a..508255e2ea 100644 --- a/ray-operator/apis/ray/v1alpha1/rayservice_types.go +++ b/ray-operator/apis/ray/v1alpha1/rayservice_types.go @@ -7,6 +7,9 @@ import ( // 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. +// Hub marks this type as a conversion hub. +func (*RayService) Hub() {} + type ServiceStatus string const ( @@ -142,9 +145,10 @@ type ServeDeploymentStatus struct { HealthLastUpdateTime *metav1.Time `json:"healthLastUpdateTime,omitempty"` } -//+kubebuilder:object:root=true -//+kubebuilder:subresource:status -//+genclient +// +kubebuilder:object:root=true +// +kubebuilder:subresource:status +// +kubebuilder:storageversion +// +genclient // RayService is the Schema for the rayservices API type RayService struct { metav1.TypeMeta `json:",inline"` diff --git a/ray-operator/apis/ray/v1alpha1/rayservice_webhook.go b/ray-operator/apis/ray/v1alpha1/rayservice_webhook.go new file mode 100644 index 0000000000..a99fec5c46 --- /dev/null +++ b/ray-operator/apis/ray/v1alpha1/rayservice_webhook.go @@ -0,0 +1,17 @@ +package v1alpha1 + +import ( + ctrl "sigs.k8s.io/controller-runtime" + logf "sigs.k8s.io/controller-runtime/pkg/log" +) + +// log is for logging in this package. +var rayservicelog = logf.Log.WithName("rayservice-resource") + +func (r *RayService) SetupWebhookWithManager(mgr ctrl.Manager) error { + return ctrl.NewWebhookManagedBy(mgr). + For(r). + Complete() +} + +// TODO(user): EDIT THIS FILE! THIS IS SCAFFOLDING FOR YOU TO OWN! diff --git a/ray-operator/apis/ray/v1beta1/doc.go b/ray-operator/apis/ray/v1beta1/doc.go new file mode 100644 index 0000000000..b4c47d0550 --- /dev/null +++ b/ray-operator/apis/ray/v1beta1/doc.go @@ -0,0 +1,3 @@ +// +groupName=ray.io +// +k8s:conversion-gen=github.com/ray-project/kuberay/ray-operator/apis/ray/v1alpha1 +package v1beta1 diff --git a/ray-operator/apis/ray/v1beta1/groupversion_info.go b/ray-operator/apis/ray/v1beta1/groupversion_info.go new file mode 100644 index 0000000000..5a9afbcf72 --- /dev/null +++ b/ray-operator/apis/ray/v1beta1/groupversion_info.go @@ -0,0 +1,23 @@ +// Package v1beta1 contains API Schema definitions for the ray v1beta1 API group +// +kubebuilder:object:generate=true +// +groupName=ray.io +package v1beta1 + +import ( + "k8s.io/apimachinery/pkg/runtime" + "k8s.io/apimachinery/pkg/runtime/schema" + "sigs.k8s.io/controller-runtime/pkg/scheme" +) + +var ( + // GroupVersion is group version used to register these objects + GroupVersion = schema.GroupVersion{Group: "ray.io", Version: "v1beta1"} + + // SchemeBuilder is used to add go types to the GroupVersionKind scheme + SchemeBuilder = &scheme.Builder{GroupVersion: GroupVersion} + + // AddToScheme adds the types in this group-version to the given scheme. + AddToScheme = SchemeBuilder.AddToScheme + + localSchemeBuilder = runtime.NewSchemeBuilder(SchemeBuilder.AddToScheme) +) diff --git a/ray-operator/apis/ray/v1beta1/raycluster_conversion.go b/ray-operator/apis/ray/v1beta1/raycluster_conversion.go new file mode 100644 index 0000000000..e410e7564f --- /dev/null +++ b/ray-operator/apis/ray/v1beta1/raycluster_conversion.go @@ -0,0 +1,24 @@ +package v1beta1 + +import ( + rayv1alpha1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1alpha1" + "sigs.k8s.io/controller-runtime/pkg/conversion" +) + +// ConvertTo converts this version (v1beta1) to the Hub version (v1alpha1). +func (src *RayCluster) ConvertTo(dstRaw conversion.Hub) error { + dst := dstRaw.(*rayv1alpha1.RayCluster) + if err := Convert_v1beta1_RayCluster_To_v1alpha1_RayCluster(src, dst, nil); err != nil { + return err + } + return nil +} + +// ConvertFrom converts from the Hub version (v1alpha1) to this version (v1beta1). +func (dst *RayCluster) ConvertFrom(srcRaw conversion.Hub) error { + src := srcRaw.(*rayv1alpha1.RayCluster) + if err := Convert_v1alpha1_RayCluster_To_v1beta1_RayCluster(src, dst, nil); err != nil { + return err + } + return nil +} diff --git a/ray-operator/apis/ray/v1beta1/raycluster_types.go b/ray-operator/apis/ray/v1beta1/raycluster_types.go new file mode 100644 index 0000000000..b923b837fa --- /dev/null +++ b/ray-operator/apis/ray/v1beta1/raycluster_types.go @@ -0,0 +1,193 @@ +package v1beta1 + +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. + +// RayClusterSpec defines the desired state of RayCluster +type RayClusterSpec struct { + // INSERT ADDITIONAL SPEC FIELDS - desired state of cluster + // Important: Run "make" to regenerate code after modifying this file + // HeadGroupSpecs are the spec for the head pod + HeadGroupSpec HeadGroupSpec `json:"headGroupSpec"` + // WorkerGroupSpecs are the specs for the worker pods + WorkerGroupSpecs []WorkerGroupSpec `json:"workerGroupSpecs,omitempty"` + // RayVersion is the version of ray being used. This determines the autoscaler's image version. + RayVersion string `json:"rayVersion,omitempty"` + // EnableInTreeAutoscaling indicates whether operator should create in tree autoscaling configs + EnableInTreeAutoscaling *bool `json:"enableInTreeAutoscaling,omitempty"` + // AutoscalerOptions specifies optional configuration for the Ray autoscaler. + AutoscalerOptions *AutoscalerOptions `json:"autoscalerOptions,omitempty"` + HeadServiceAnnotations map[string]string `json:"headServiceAnnotations,omitempty"` +} + +// HeadGroupSpec are the spec for the head pod +type HeadGroupSpec struct { + // ServiceType is Kubernetes service type of the head service. it will be used by the workers to connect to the head pod + ServiceType v1.ServiceType `json:"serviceType,omitempty"` + // HeadService is the Kubernetes service of the head pod. + HeadService *v1.Service `json:"headService,omitempty"` + // EnableIngress indicates whether operator should create ingress object for head service or not. + EnableIngress *bool `json:"enableIngress,omitempty"` + // HeadGroupSpec.Replicas is deprecated and ignored; there can only be one head pod per Ray cluster. + Replicas *int32 `json:"replicas,omitempty"` + // RayStartParams are the params of the start command: node-manager-port, object-store-memory, ... + RayStartParams map[string]string `json:"rayStartParams"` + // Template is the eaxct pod template used in K8s depoyments, statefulsets, etc. + Template v1.PodTemplateSpec `json:"template"` +} + +// WorkerGroupSpec are the specs for the worker pods +type WorkerGroupSpec struct { + // we can have multiple worker groups, we distinguish them by name + GroupName string `json:"groupName"` + // Replicas Number of desired pods in this pod group. This is a pointer to distinguish between explicit + // zero and not specified. Defaults to 1. + Replicas *int32 `json:"replicas"` + // MinReplicas defaults to 1 + MinReplicas *int32 `json:"minReplicas"` + // MaxReplicas defaults to maxInt32 + MaxReplicas *int32 `json:"maxReplicas"` + // RayStartParams are the params of the start command: address, object-store-memory, ... + RayStartParams map[string]string `json:"rayStartParams"` + // Template is a pod template for the worker + Template v1.PodTemplateSpec `json:"template"` + // ScaleStrategy defines which pods to remove + ScaleStrategy ScaleStrategy `json:"scaleStrategy,omitempty"` +} + +// ScaleStrategy to remove workers +type ScaleStrategy struct { + // WorkersToDelete workers to be deleted + WorkersToDelete []string `json:"workersToDelete,omitempty"` +} + +// AutoscalerOptions specifies optional configuration for the Ray autoscaler. +type AutoscalerOptions struct { + // Resources specifies optional resource request and limit overrides for the autoscaler container. + // Default values: 500m CPU request and limit. 512Mi memory request and limit. + Resources *v1.ResourceRequirements `json:"resources,omitempty"` + // Image optionally overrides the autoscaler's container image. This override is for provided for autoscaler testing and development. + Image *string `json:"image,omitempty"` + // ImagePullPolicy optionally overrides the autoscaler container's image pull policy. This override is for provided for autoscaler testing and development. + ImagePullPolicy *v1.PullPolicy `json:"imagePullPolicy,omitempty"` + // Optional list of environment variables to set in the autoscaler container. + Env []v1.EnvVar `json:"env,omitempty"` + // Optional list of sources to populate environment variables in the autoscaler container. + EnvFrom []v1.EnvFromSource `json:"envFrom,omitempty"` + // Optional list of volumeMounts. This is needed for enabling TLS for the autoscaler container. + VolumeMounts []v1.VolumeMount `json:"volumeMounts,omitempty"` + // SecurityContext defines the security options the container should be run with. + // If set, the fields of SecurityContext override the equivalent fields of PodSecurityContext. + // More info: https://kubernetes.io/docs/tasks/configure-pod-container/security-context/ + SecurityContext *v1.SecurityContext `json:"securityContext,omitempty"` + // IdleTimeoutSeconds is the number of seconds to wait before scaling down a worker pod which is not using Ray resources. + // Defaults to 60 (one minute). + IdleTimeoutSeconds *int32 `json:"idleTimeoutSeconds,omitempty"` + // UpscalingMode is "Conservative", "Default", or "Aggressive." + // Conservative: Upscaling is rate-limited; the number of pending worker pods is at most the size of the Ray cluster. + // Default: Upscaling is not rate-limited. + // Aggressive: An alias for Default; upscaling is not rate-limited. + UpscalingMode *UpscalingMode `json:"upscalingMode,omitempty"` +} + +// +kubebuilder:validation:Enum=Default;Aggressive;Conservative +type UpscalingMode string + +// The overall state of the Ray cluster. +type ClusterState string + +const ( + Ready ClusterState = "ready" + Unhealthy ClusterState = "unhealthy" + Failed ClusterState = "failed" +) + +// RayClusterStatus defines the observed state of RayCluster +type RayClusterStatus struct { + // INSERT ADDITIONAL STATUS FIELD - define observed state of cluster + // Important: Run "make" to regenerate code after modifying this file + // Status reflects the status of the cluster + State ClusterState `json:"state,omitempty"` + // AvailableWorkerReplicas indicates how many replicas are available in the cluster + AvailableWorkerReplicas int32 `json:"availableWorkerReplicas,omitempty"` + // DesiredWorkerReplicas indicates overall desired replicas claimed by the user at the cluster level. + DesiredWorkerReplicas int32 `json:"desiredWorkerReplicas,omitempty"` + // MinWorkerReplicas indicates sum of minimum replicas of each node group. + MinWorkerReplicas int32 `json:"minWorkerReplicas,omitempty"` + // MaxWorkerReplicas indicates sum of maximum replicas of each node group. + MaxWorkerReplicas int32 `json:"maxWorkerReplicas,omitempty"` + // LastUpdateTime indicates last update timestamp for this cluster status. + // +nullable + LastUpdateTime *metav1.Time `json:"lastUpdateTime,omitempty"` + // Service Endpoints + Endpoints map[string]string `json:"endpoints,omitempty"` + // Head info + Head HeadInfo `json:"head,omitempty"` + // Reason provides more information about current State + Reason string `json:"reason,omitempty"` + // observedGeneration is the most recent generation observed for this RayCluster. It corresponds to the + // RayCluster's generation, which is updated on mutation by the API Server. + // +optional + ObservedGeneration int64 `json:"observedGeneration,omitempty"` +} + +// HeadInfo gives info about head +type HeadInfo struct { + PodIP string `json:"podIP,omitempty"` + ServiceIP string `json:"serviceIP,omitempty"` +} + +// RayNodeType the type of a ray node: head/worker +type RayNodeType string + +const ( + // HeadNode means that this pod will be ray cluster head + HeadNode RayNodeType = "head" + // WorkerNode means that this pod will be ray cluster worker + WorkerNode RayNodeType = "worker" +) + +// RayCluster is the Schema for the RayClusters API +// +kubebuilder:object:root=true +// +kubebuilder:subresource:status +// +kubebuilder:printcolumn:name="desired workers",type=integer,JSONPath=".status.desiredWorkerReplicas",priority=0 +// +kubebuilder:printcolumn:name="available workers",type=integer,JSONPath=".status.availableWorkerReplicas",priority=0 +// +kubebuilder:printcolumn:name="status",type="string",JSONPath=".status.state",priority=0 +// +kubebuilder:printcolumn:name="age",type="date",JSONPath=".metadata.creationTimestamp",priority=0 +// +kubebuilder:printcolumn:name="head pod IP",type="string",JSONPath=".status.head.podIP",priority=1 +// +kubebuilder:printcolumn:name="head service IP",type="string",JSONPath=".status.head.serviceIP",priority=1 +// +genclient +type RayCluster struct { + // Standard object metadata. + metav1.TypeMeta `json:",inline"` + metav1.ObjectMeta `json:"metadata,omitempty"` + + // Specification of the desired behavior of the RayCluster. + Spec RayClusterSpec `json:"spec,omitempty"` + Status RayClusterStatus `json:"status,omitempty"` +} + +//+kubebuilder:object:root=true + +// RayClusterList contains a list of RayCluster +type RayClusterList struct { + metav1.TypeMeta `json:",inline"` + metav1.ListMeta `json:"metadata,omitempty"` + Items []RayCluster `json:"items"` +} + +func init() { + SchemeBuilder.Register(&RayCluster{}, &RayClusterList{}) +} + +type EventReason string + +const ( + RayConfigError EventReason = "RayConfigError" + PodReconciliationError EventReason = "PodReconciliationError" +) diff --git a/ray-operator/apis/ray/v1beta1/rayservice_conversion.go b/ray-operator/apis/ray/v1beta1/rayservice_conversion.go new file mode 100644 index 0000000000..f725eb5a2b --- /dev/null +++ b/ray-operator/apis/ray/v1beta1/rayservice_conversion.go @@ -0,0 +1,24 @@ +package v1beta1 + +import ( + rayv1alpha1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1alpha1" + "sigs.k8s.io/controller-runtime/pkg/conversion" +) + +// ConvertTo converts this version (v1beta1) to the Hub version (v1alpha1). +func (src *RayService) ConvertTo(dstRaw conversion.Hub) error { + dst := dstRaw.(*rayv1alpha1.RayService) + if err := Convert_v1beta1_RayService_To_v1alpha1_RayService(src, dst, nil); err != nil { + return err + } + return nil +} + +// ConvertFrom converts from the Hub version (v1alpha1) to this version (v1beta1). +func (dst *RayService) ConvertFrom(srcRaw conversion.Hub) error { + src := srcRaw.(*rayv1alpha1.RayService) + if err := Convert_v1alpha1_RayService_To_v1beta1_RayService(src, dst, nil); err != nil { + return err + } + return nil +} diff --git a/ray-operator/apis/ray/v1beta1/rayservice_types.go b/ray-operator/apis/ray/v1beta1/rayservice_types.go new file mode 100644 index 0000000000..f4ad37141b --- /dev/null +++ b/ray-operator/apis/ray/v1beta1/rayservice_types.go @@ -0,0 +1,171 @@ +package v1beta1 + +import ( + 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. + +// Hub marks this type as a conversion hub. +func (*RayService) Hub() {} + +type ServiceStatus string + +const ( + FailedToGetOrCreateRayCluster ServiceStatus = "FailedToGetOrCreateRayCluster" + WaitForDashboard ServiceStatus = "WaitForDashboard" + WaitForServeDeploymentReady ServiceStatus = "WaitForServeDeploymentReady" + FailedToGetServeDeploymentStatus ServiceStatus = "FailedToGetServeDeploymentStatus" + Running ServiceStatus = "Running" + Restarting ServiceStatus = "Restarting" + FailedToUpdateIngress ServiceStatus = "FailedToUpdateIngress" + FailedToUpdateServingPodLabel ServiceStatus = "FailedToUpdateServingPodLabel" + FailedToUpdateService ServiceStatus = "FailedToUpdateService" +) + +// These statuses should match Ray Serve's application statuses +var ApplicationStatusEnum = struct { + NOT_STARTED string + DEPLOYING string + RUNNING string + DEPLOY_FAILED string +}{ + NOT_STARTED: "NOT_STARTED", + DEPLOYING: "DEPLOYING", + RUNNING: "RUNNING", + DEPLOY_FAILED: "DEPLOY_FAILED", +} + +// These statuses should match Ray Serve's deployment statuses +var DeploymentStatusEnum = struct { + UPDATING string + HEALTHY string + UNHEALTHY string +}{ + UPDATING: "UPDATING", + HEALTHY: "HEALTHY", + UNHEALTHY: "UNHEALTHY", +} + +// RayServiceSpec defines the desired state of RayService +type RayServiceSpec struct { + // Important: Run "make" to regenerate code after modifying this file + ServeDeploymentGraphSpec ServeDeploymentGraphSpec `json:"serveConfig,omitempty"` + RayClusterSpec RayClusterSpec `json:"rayClusterConfig,omitempty"` + ServiceUnhealthySecondThreshold *int32 `json:"serviceUnhealthySecondThreshold,omitempty"` + DeploymentUnhealthySecondThreshold *int32 `json:"deploymentUnhealthySecondThreshold,omitempty"` +} + +type ServeDeploymentGraphSpec struct { + ImportPath string `json:"importPath"` + RuntimeEnv string `json:"runtimeEnv,omitempty"` + ServeConfigSpecs []ServeConfigSpec `json:"deployments,omitempty"` + Port int `json:"port,omitempty"` +} + +// ServeConfigSpec defines the desired state of RayService +// Reference to http://rayserve.org +type ServeConfigSpec struct { + Name string `json:"name"` + NumReplicas *int32 `json:"numReplicas,omitempty"` + RoutePrefix string `json:"routePrefix,omitempty"` + MaxConcurrentQueries *int32 `json:"maxConcurrentQueries,omitempty"` + UserConfig string `json:"userConfig,omitempty"` + AutoscalingConfig string `json:"autoscalingConfig,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"` +} + +// RayActorOptionSpec defines the desired state of RayActor +type RayActorOptionSpec struct { + RuntimeEnv string `json:"runtimeEnv,omitempty"` + NumCpus *float64 `json:"numCpus,omitempty"` + NumGpus *float64 `json:"numGpus,omitempty"` + Memory *int32 `json:"memory,omitempty"` + ObjectStoreMemory *int32 `json:"objectStoreMemory,omitempty"` + Resources string `json:"resources,omitempty"` + AcceleratorType string `json:"acceleratorType,omitempty"` +} + +// RayServiceStatuses defines the observed state of RayService +// +kubebuilder:printcolumn:name="ServiceStatus",type=string,JSONPath=".status.serviceStatus" +type RayServiceStatuses struct { + ActiveServiceStatus RayServiceStatus `json:"activeServiceStatus,omitempty"` + // Pending Service Status indicates a RayCluster will be created or is being created. + PendingServiceStatus RayServiceStatus `json:"pendingServiceStatus,omitempty"` + // ServiceStatus indicates the current RayService status. + ServiceStatus ServiceStatus `json:"serviceStatus,omitempty"` + // observedGeneration is the most recent generation observed for this RayService. It corresponds to the + // RayService's generation, which is updated on mutation by the API Server. + // +optional + ObservedGeneration int64 `json:"observedGeneration,omitempty"` +} + +type RayServiceStatus struct { + // Important: Run "make" to regenerate code after modifying this file + ApplicationStatus AppStatus `json:"appStatus,omitempty"` + ServeStatuses []ServeDeploymentStatus `json:"serveDeploymentStatuses,omitempty"` + DashboardStatus DashboardStatus `json:"dashboardStatus,omitempty"` + RayClusterName string `json:"rayClusterName,omitempty"` + RayClusterStatus RayClusterStatus `json:"rayClusterStatus,omitempty"` +} + +// DashboardStatus defines the current states of Ray Dashboard +type DashboardStatus struct { + IsHealthy bool `json:"isHealthy,omitempty"` + LastUpdateTime *metav1.Time `json:"lastUpdateTime,omitempty"` + // Keep track of how long the dashboard is healthy. + // Update when Dashboard is responsive or first time convert to non-responsive from responsive. + HealthLastUpdateTime *metav1.Time `json:"healthLastUpdateTime,omitempty"` +} + +type AppStatus struct { + 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"` +} + +// ServeDeploymentStatus defines the current state of a Serve deployment +type ServeDeploymentStatus struct { + // Name, Status, Message are from Ray Dashboard and represent a Serve deployment's state. + 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"` +} + +// +kubebuilder:object:root=true +// +kubebuilder:subresource:status +// +genclient +// RayService is the Schema for the rayservices API +type RayService struct { + metav1.TypeMeta `json:",inline"` + metav1.ObjectMeta `json:"metadata,omitempty"` + + Spec RayServiceSpec `json:"spec,omitempty"` + Status RayServiceStatuses `json:"status,omitempty"` +} + +//+kubebuilder:object:root=true + +// RayServiceList contains a list of RayService +type RayServiceList struct { + metav1.TypeMeta `json:",inline"` + metav1.ListMeta `json:"metadata,omitempty"` + Items []RayService `json:"items"` +} + +func init() { + SchemeBuilder.Register(&RayService{}, &RayServiceList{}) +} diff --git a/ray-operator/apis/ray/v1beta1/zz_generated.conversion.go b/ray-operator/apis/ray/v1beta1/zz_generated.conversion.go new file mode 100644 index 0000000000..e1c9f6001b --- /dev/null +++ b/ray-operator/apis/ray/v1beta1/zz_generated.conversion.go @@ -0,0 +1,842 @@ +//go:build !ignore_autogenerated +// +build !ignore_autogenerated + +// Code generated by conversion-gen. DO NOT EDIT. + +package v1beta1 + +import ( + unsafe "unsafe" + + v1alpha1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1alpha1" + corev1 "k8s.io/api/core/v1" + v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + conversion "k8s.io/apimachinery/pkg/conversion" + runtime "k8s.io/apimachinery/pkg/runtime" +) + +func init() { + localSchemeBuilder.Register(RegisterConversions) +} + +// RegisterConversions adds conversion functions to the given scheme. +// Public to allow building arbitrary schemes. +func RegisterConversions(s *runtime.Scheme) error { + if err := s.AddGeneratedConversionFunc((*AppStatus)(nil), (*v1alpha1.AppStatus)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1beta1_AppStatus_To_v1alpha1_AppStatus(a.(*AppStatus), b.(*v1alpha1.AppStatus), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*v1alpha1.AppStatus)(nil), (*AppStatus)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1alpha1_AppStatus_To_v1beta1_AppStatus(a.(*v1alpha1.AppStatus), b.(*AppStatus), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*AutoscalerOptions)(nil), (*v1alpha1.AutoscalerOptions)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1beta1_AutoscalerOptions_To_v1alpha1_AutoscalerOptions(a.(*AutoscalerOptions), b.(*v1alpha1.AutoscalerOptions), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*v1alpha1.AutoscalerOptions)(nil), (*AutoscalerOptions)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1alpha1_AutoscalerOptions_To_v1beta1_AutoscalerOptions(a.(*v1alpha1.AutoscalerOptions), b.(*AutoscalerOptions), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*DashboardStatus)(nil), (*v1alpha1.DashboardStatus)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1beta1_DashboardStatus_To_v1alpha1_DashboardStatus(a.(*DashboardStatus), b.(*v1alpha1.DashboardStatus), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*v1alpha1.DashboardStatus)(nil), (*DashboardStatus)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1alpha1_DashboardStatus_To_v1beta1_DashboardStatus(a.(*v1alpha1.DashboardStatus), b.(*DashboardStatus), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*HeadGroupSpec)(nil), (*v1alpha1.HeadGroupSpec)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1beta1_HeadGroupSpec_To_v1alpha1_HeadGroupSpec(a.(*HeadGroupSpec), b.(*v1alpha1.HeadGroupSpec), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*v1alpha1.HeadGroupSpec)(nil), (*HeadGroupSpec)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1alpha1_HeadGroupSpec_To_v1beta1_HeadGroupSpec(a.(*v1alpha1.HeadGroupSpec), b.(*HeadGroupSpec), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*HeadInfo)(nil), (*v1alpha1.HeadInfo)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1beta1_HeadInfo_To_v1alpha1_HeadInfo(a.(*HeadInfo), b.(*v1alpha1.HeadInfo), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*v1alpha1.HeadInfo)(nil), (*HeadInfo)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1alpha1_HeadInfo_To_v1beta1_HeadInfo(a.(*v1alpha1.HeadInfo), b.(*HeadInfo), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*RayActorOptionSpec)(nil), (*v1alpha1.RayActorOptionSpec)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1beta1_RayActorOptionSpec_To_v1alpha1_RayActorOptionSpec(a.(*RayActorOptionSpec), b.(*v1alpha1.RayActorOptionSpec), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*v1alpha1.RayActorOptionSpec)(nil), (*RayActorOptionSpec)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1alpha1_RayActorOptionSpec_To_v1beta1_RayActorOptionSpec(a.(*v1alpha1.RayActorOptionSpec), b.(*RayActorOptionSpec), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*RayCluster)(nil), (*v1alpha1.RayCluster)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1beta1_RayCluster_To_v1alpha1_RayCluster(a.(*RayCluster), b.(*v1alpha1.RayCluster), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*v1alpha1.RayCluster)(nil), (*RayCluster)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1alpha1_RayCluster_To_v1beta1_RayCluster(a.(*v1alpha1.RayCluster), b.(*RayCluster), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*RayClusterList)(nil), (*v1alpha1.RayClusterList)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1beta1_RayClusterList_To_v1alpha1_RayClusterList(a.(*RayClusterList), b.(*v1alpha1.RayClusterList), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*v1alpha1.RayClusterList)(nil), (*RayClusterList)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1alpha1_RayClusterList_To_v1beta1_RayClusterList(a.(*v1alpha1.RayClusterList), b.(*RayClusterList), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*RayClusterSpec)(nil), (*v1alpha1.RayClusterSpec)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1beta1_RayClusterSpec_To_v1alpha1_RayClusterSpec(a.(*RayClusterSpec), b.(*v1alpha1.RayClusterSpec), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*v1alpha1.RayClusterSpec)(nil), (*RayClusterSpec)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1alpha1_RayClusterSpec_To_v1beta1_RayClusterSpec(a.(*v1alpha1.RayClusterSpec), b.(*RayClusterSpec), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*RayClusterStatus)(nil), (*v1alpha1.RayClusterStatus)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1beta1_RayClusterStatus_To_v1alpha1_RayClusterStatus(a.(*RayClusterStatus), b.(*v1alpha1.RayClusterStatus), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*v1alpha1.RayClusterStatus)(nil), (*RayClusterStatus)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1alpha1_RayClusterStatus_To_v1beta1_RayClusterStatus(a.(*v1alpha1.RayClusterStatus), b.(*RayClusterStatus), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*RayService)(nil), (*v1alpha1.RayService)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1beta1_RayService_To_v1alpha1_RayService(a.(*RayService), b.(*v1alpha1.RayService), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*v1alpha1.RayService)(nil), (*RayService)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1alpha1_RayService_To_v1beta1_RayService(a.(*v1alpha1.RayService), b.(*RayService), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*RayServiceList)(nil), (*v1alpha1.RayServiceList)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1beta1_RayServiceList_To_v1alpha1_RayServiceList(a.(*RayServiceList), b.(*v1alpha1.RayServiceList), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*v1alpha1.RayServiceList)(nil), (*RayServiceList)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1alpha1_RayServiceList_To_v1beta1_RayServiceList(a.(*v1alpha1.RayServiceList), b.(*RayServiceList), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*RayServiceSpec)(nil), (*v1alpha1.RayServiceSpec)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1beta1_RayServiceSpec_To_v1alpha1_RayServiceSpec(a.(*RayServiceSpec), b.(*v1alpha1.RayServiceSpec), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*v1alpha1.RayServiceSpec)(nil), (*RayServiceSpec)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1alpha1_RayServiceSpec_To_v1beta1_RayServiceSpec(a.(*v1alpha1.RayServiceSpec), b.(*RayServiceSpec), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*RayServiceStatus)(nil), (*v1alpha1.RayServiceStatus)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1beta1_RayServiceStatus_To_v1alpha1_RayServiceStatus(a.(*RayServiceStatus), b.(*v1alpha1.RayServiceStatus), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*v1alpha1.RayServiceStatus)(nil), (*RayServiceStatus)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1alpha1_RayServiceStatus_To_v1beta1_RayServiceStatus(a.(*v1alpha1.RayServiceStatus), b.(*RayServiceStatus), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*RayServiceStatuses)(nil), (*v1alpha1.RayServiceStatuses)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1beta1_RayServiceStatuses_To_v1alpha1_RayServiceStatuses(a.(*RayServiceStatuses), b.(*v1alpha1.RayServiceStatuses), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*v1alpha1.RayServiceStatuses)(nil), (*RayServiceStatuses)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1alpha1_RayServiceStatuses_To_v1beta1_RayServiceStatuses(a.(*v1alpha1.RayServiceStatuses), b.(*RayServiceStatuses), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*ScaleStrategy)(nil), (*v1alpha1.ScaleStrategy)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1beta1_ScaleStrategy_To_v1alpha1_ScaleStrategy(a.(*ScaleStrategy), b.(*v1alpha1.ScaleStrategy), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*v1alpha1.ScaleStrategy)(nil), (*ScaleStrategy)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1alpha1_ScaleStrategy_To_v1beta1_ScaleStrategy(a.(*v1alpha1.ScaleStrategy), b.(*ScaleStrategy), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*ServeConfigSpec)(nil), (*v1alpha1.ServeConfigSpec)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1beta1_ServeConfigSpec_To_v1alpha1_ServeConfigSpec(a.(*ServeConfigSpec), b.(*v1alpha1.ServeConfigSpec), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*v1alpha1.ServeConfigSpec)(nil), (*ServeConfigSpec)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1alpha1_ServeConfigSpec_To_v1beta1_ServeConfigSpec(a.(*v1alpha1.ServeConfigSpec), b.(*ServeConfigSpec), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*ServeDeploymentGraphSpec)(nil), (*v1alpha1.ServeDeploymentGraphSpec)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1beta1_ServeDeploymentGraphSpec_To_v1alpha1_ServeDeploymentGraphSpec(a.(*ServeDeploymentGraphSpec), b.(*v1alpha1.ServeDeploymentGraphSpec), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*v1alpha1.ServeDeploymentGraphSpec)(nil), (*ServeDeploymentGraphSpec)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1alpha1_ServeDeploymentGraphSpec_To_v1beta1_ServeDeploymentGraphSpec(a.(*v1alpha1.ServeDeploymentGraphSpec), b.(*ServeDeploymentGraphSpec), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*ServeDeploymentStatus)(nil), (*v1alpha1.ServeDeploymentStatus)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1beta1_ServeDeploymentStatus_To_v1alpha1_ServeDeploymentStatus(a.(*ServeDeploymentStatus), b.(*v1alpha1.ServeDeploymentStatus), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*v1alpha1.ServeDeploymentStatus)(nil), (*ServeDeploymentStatus)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1alpha1_ServeDeploymentStatus_To_v1beta1_ServeDeploymentStatus(a.(*v1alpha1.ServeDeploymentStatus), b.(*ServeDeploymentStatus), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*WorkerGroupSpec)(nil), (*v1alpha1.WorkerGroupSpec)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1beta1_WorkerGroupSpec_To_v1alpha1_WorkerGroupSpec(a.(*WorkerGroupSpec), b.(*v1alpha1.WorkerGroupSpec), scope) + }); err != nil { + return err + } + if err := s.AddGeneratedConversionFunc((*v1alpha1.WorkerGroupSpec)(nil), (*WorkerGroupSpec)(nil), func(a, b interface{}, scope conversion.Scope) error { + return Convert_v1alpha1_WorkerGroupSpec_To_v1beta1_WorkerGroupSpec(a.(*v1alpha1.WorkerGroupSpec), b.(*WorkerGroupSpec), scope) + }); err != nil { + return err + } + return nil +} + +func autoConvert_v1beta1_AppStatus_To_v1alpha1_AppStatus(in *AppStatus, out *v1alpha1.AppStatus, s conversion.Scope) error { + out.Status = in.Status + out.Message = in.Message + out.LastUpdateTime = (*v1.Time)(unsafe.Pointer(in.LastUpdateTime)) + out.HealthLastUpdateTime = (*v1.Time)(unsafe.Pointer(in.HealthLastUpdateTime)) + return nil +} + +// Convert_v1beta1_AppStatus_To_v1alpha1_AppStatus is an autogenerated conversion function. +func Convert_v1beta1_AppStatus_To_v1alpha1_AppStatus(in *AppStatus, out *v1alpha1.AppStatus, s conversion.Scope) error { + return autoConvert_v1beta1_AppStatus_To_v1alpha1_AppStatus(in, out, s) +} + +func autoConvert_v1alpha1_AppStatus_To_v1beta1_AppStatus(in *v1alpha1.AppStatus, out *AppStatus, s conversion.Scope) error { + out.Status = in.Status + out.Message = in.Message + out.LastUpdateTime = (*v1.Time)(unsafe.Pointer(in.LastUpdateTime)) + out.HealthLastUpdateTime = (*v1.Time)(unsafe.Pointer(in.HealthLastUpdateTime)) + return nil +} + +// Convert_v1alpha1_AppStatus_To_v1beta1_AppStatus is an autogenerated conversion function. +func Convert_v1alpha1_AppStatus_To_v1beta1_AppStatus(in *v1alpha1.AppStatus, out *AppStatus, s conversion.Scope) error { + return autoConvert_v1alpha1_AppStatus_To_v1beta1_AppStatus(in, out, s) +} + +func autoConvert_v1beta1_AutoscalerOptions_To_v1alpha1_AutoscalerOptions(in *AutoscalerOptions, out *v1alpha1.AutoscalerOptions, s conversion.Scope) error { + out.Resources = (*corev1.ResourceRequirements)(unsafe.Pointer(in.Resources)) + out.Image = (*string)(unsafe.Pointer(in.Image)) + out.ImagePullPolicy = (*corev1.PullPolicy)(unsafe.Pointer(in.ImagePullPolicy)) + out.Env = *(*[]corev1.EnvVar)(unsafe.Pointer(&in.Env)) + out.EnvFrom = *(*[]corev1.EnvFromSource)(unsafe.Pointer(&in.EnvFrom)) + out.VolumeMounts = *(*[]corev1.VolumeMount)(unsafe.Pointer(&in.VolumeMounts)) + out.SecurityContext = (*corev1.SecurityContext)(unsafe.Pointer(in.SecurityContext)) + out.IdleTimeoutSeconds = (*int32)(unsafe.Pointer(in.IdleTimeoutSeconds)) + out.UpscalingMode = (*v1alpha1.UpscalingMode)(unsafe.Pointer(in.UpscalingMode)) + return nil +} + +// Convert_v1beta1_AutoscalerOptions_To_v1alpha1_AutoscalerOptions is an autogenerated conversion function. +func Convert_v1beta1_AutoscalerOptions_To_v1alpha1_AutoscalerOptions(in *AutoscalerOptions, out *v1alpha1.AutoscalerOptions, s conversion.Scope) error { + return autoConvert_v1beta1_AutoscalerOptions_To_v1alpha1_AutoscalerOptions(in, out, s) +} + +func autoConvert_v1alpha1_AutoscalerOptions_To_v1beta1_AutoscalerOptions(in *v1alpha1.AutoscalerOptions, out *AutoscalerOptions, s conversion.Scope) error { + out.Resources = (*corev1.ResourceRequirements)(unsafe.Pointer(in.Resources)) + out.Image = (*string)(unsafe.Pointer(in.Image)) + out.ImagePullPolicy = (*corev1.PullPolicy)(unsafe.Pointer(in.ImagePullPolicy)) + out.Env = *(*[]corev1.EnvVar)(unsafe.Pointer(&in.Env)) + out.EnvFrom = *(*[]corev1.EnvFromSource)(unsafe.Pointer(&in.EnvFrom)) + out.VolumeMounts = *(*[]corev1.VolumeMount)(unsafe.Pointer(&in.VolumeMounts)) + out.SecurityContext = (*corev1.SecurityContext)(unsafe.Pointer(in.SecurityContext)) + out.IdleTimeoutSeconds = (*int32)(unsafe.Pointer(in.IdleTimeoutSeconds)) + out.UpscalingMode = (*UpscalingMode)(unsafe.Pointer(in.UpscalingMode)) + return nil +} + +// Convert_v1alpha1_AutoscalerOptions_To_v1beta1_AutoscalerOptions is an autogenerated conversion function. +func Convert_v1alpha1_AutoscalerOptions_To_v1beta1_AutoscalerOptions(in *v1alpha1.AutoscalerOptions, out *AutoscalerOptions, s conversion.Scope) error { + return autoConvert_v1alpha1_AutoscalerOptions_To_v1beta1_AutoscalerOptions(in, out, s) +} + +func autoConvert_v1beta1_DashboardStatus_To_v1alpha1_DashboardStatus(in *DashboardStatus, out *v1alpha1.DashboardStatus, s conversion.Scope) error { + out.IsHealthy = in.IsHealthy + out.LastUpdateTime = (*v1.Time)(unsafe.Pointer(in.LastUpdateTime)) + out.HealthLastUpdateTime = (*v1.Time)(unsafe.Pointer(in.HealthLastUpdateTime)) + return nil +} + +// Convert_v1beta1_DashboardStatus_To_v1alpha1_DashboardStatus is an autogenerated conversion function. +func Convert_v1beta1_DashboardStatus_To_v1alpha1_DashboardStatus(in *DashboardStatus, out *v1alpha1.DashboardStatus, s conversion.Scope) error { + return autoConvert_v1beta1_DashboardStatus_To_v1alpha1_DashboardStatus(in, out, s) +} + +func autoConvert_v1alpha1_DashboardStatus_To_v1beta1_DashboardStatus(in *v1alpha1.DashboardStatus, out *DashboardStatus, s conversion.Scope) error { + out.IsHealthy = in.IsHealthy + out.LastUpdateTime = (*v1.Time)(unsafe.Pointer(in.LastUpdateTime)) + out.HealthLastUpdateTime = (*v1.Time)(unsafe.Pointer(in.HealthLastUpdateTime)) + return nil +} + +// Convert_v1alpha1_DashboardStatus_To_v1beta1_DashboardStatus is an autogenerated conversion function. +func Convert_v1alpha1_DashboardStatus_To_v1beta1_DashboardStatus(in *v1alpha1.DashboardStatus, out *DashboardStatus, s conversion.Scope) error { + return autoConvert_v1alpha1_DashboardStatus_To_v1beta1_DashboardStatus(in, out, s) +} + +func autoConvert_v1beta1_HeadGroupSpec_To_v1alpha1_HeadGroupSpec(in *HeadGroupSpec, out *v1alpha1.HeadGroupSpec, s conversion.Scope) error { + out.ServiceType = corev1.ServiceType(in.ServiceType) + out.HeadService = (*corev1.Service)(unsafe.Pointer(in.HeadService)) + out.EnableIngress = (*bool)(unsafe.Pointer(in.EnableIngress)) + out.Replicas = (*int32)(unsafe.Pointer(in.Replicas)) + out.RayStartParams = *(*map[string]string)(unsafe.Pointer(&in.RayStartParams)) + out.Template = in.Template + return nil +} + +// Convert_v1beta1_HeadGroupSpec_To_v1alpha1_HeadGroupSpec is an autogenerated conversion function. +func Convert_v1beta1_HeadGroupSpec_To_v1alpha1_HeadGroupSpec(in *HeadGroupSpec, out *v1alpha1.HeadGroupSpec, s conversion.Scope) error { + return autoConvert_v1beta1_HeadGroupSpec_To_v1alpha1_HeadGroupSpec(in, out, s) +} + +func autoConvert_v1alpha1_HeadGroupSpec_To_v1beta1_HeadGroupSpec(in *v1alpha1.HeadGroupSpec, out *HeadGroupSpec, s conversion.Scope) error { + out.ServiceType = corev1.ServiceType(in.ServiceType) + out.HeadService = (*corev1.Service)(unsafe.Pointer(in.HeadService)) + out.EnableIngress = (*bool)(unsafe.Pointer(in.EnableIngress)) + out.Replicas = (*int32)(unsafe.Pointer(in.Replicas)) + out.RayStartParams = *(*map[string]string)(unsafe.Pointer(&in.RayStartParams)) + out.Template = in.Template + return nil +} + +// Convert_v1alpha1_HeadGroupSpec_To_v1beta1_HeadGroupSpec is an autogenerated conversion function. +func Convert_v1alpha1_HeadGroupSpec_To_v1beta1_HeadGroupSpec(in *v1alpha1.HeadGroupSpec, out *HeadGroupSpec, s conversion.Scope) error { + return autoConvert_v1alpha1_HeadGroupSpec_To_v1beta1_HeadGroupSpec(in, out, s) +} + +func autoConvert_v1beta1_HeadInfo_To_v1alpha1_HeadInfo(in *HeadInfo, out *v1alpha1.HeadInfo, s conversion.Scope) error { + out.PodIP = in.PodIP + out.ServiceIP = in.ServiceIP + return nil +} + +// Convert_v1beta1_HeadInfo_To_v1alpha1_HeadInfo is an autogenerated conversion function. +func Convert_v1beta1_HeadInfo_To_v1alpha1_HeadInfo(in *HeadInfo, out *v1alpha1.HeadInfo, s conversion.Scope) error { + return autoConvert_v1beta1_HeadInfo_To_v1alpha1_HeadInfo(in, out, s) +} + +func autoConvert_v1alpha1_HeadInfo_To_v1beta1_HeadInfo(in *v1alpha1.HeadInfo, out *HeadInfo, s conversion.Scope) error { + out.PodIP = in.PodIP + out.ServiceIP = in.ServiceIP + return nil +} + +// Convert_v1alpha1_HeadInfo_To_v1beta1_HeadInfo is an autogenerated conversion function. +func Convert_v1alpha1_HeadInfo_To_v1beta1_HeadInfo(in *v1alpha1.HeadInfo, out *HeadInfo, s conversion.Scope) error { + return autoConvert_v1alpha1_HeadInfo_To_v1beta1_HeadInfo(in, out, s) +} + +func autoConvert_v1beta1_RayActorOptionSpec_To_v1alpha1_RayActorOptionSpec(in *RayActorOptionSpec, out *v1alpha1.RayActorOptionSpec, s conversion.Scope) error { + out.RuntimeEnv = in.RuntimeEnv + out.NumCpus = (*float64)(unsafe.Pointer(in.NumCpus)) + out.NumGpus = (*float64)(unsafe.Pointer(in.NumGpus)) + out.Memory = (*int32)(unsafe.Pointer(in.Memory)) + out.ObjectStoreMemory = (*int32)(unsafe.Pointer(in.ObjectStoreMemory)) + out.Resources = in.Resources + out.AcceleratorType = in.AcceleratorType + return nil +} + +// Convert_v1beta1_RayActorOptionSpec_To_v1alpha1_RayActorOptionSpec is an autogenerated conversion function. +func Convert_v1beta1_RayActorOptionSpec_To_v1alpha1_RayActorOptionSpec(in *RayActorOptionSpec, out *v1alpha1.RayActorOptionSpec, s conversion.Scope) error { + return autoConvert_v1beta1_RayActorOptionSpec_To_v1alpha1_RayActorOptionSpec(in, out, s) +} + +func autoConvert_v1alpha1_RayActorOptionSpec_To_v1beta1_RayActorOptionSpec(in *v1alpha1.RayActorOptionSpec, out *RayActorOptionSpec, s conversion.Scope) error { + out.RuntimeEnv = in.RuntimeEnv + out.NumCpus = (*float64)(unsafe.Pointer(in.NumCpus)) + out.NumGpus = (*float64)(unsafe.Pointer(in.NumGpus)) + out.Memory = (*int32)(unsafe.Pointer(in.Memory)) + out.ObjectStoreMemory = (*int32)(unsafe.Pointer(in.ObjectStoreMemory)) + out.Resources = in.Resources + out.AcceleratorType = in.AcceleratorType + return nil +} + +// Convert_v1alpha1_RayActorOptionSpec_To_v1beta1_RayActorOptionSpec is an autogenerated conversion function. +func Convert_v1alpha1_RayActorOptionSpec_To_v1beta1_RayActorOptionSpec(in *v1alpha1.RayActorOptionSpec, out *RayActorOptionSpec, s conversion.Scope) error { + return autoConvert_v1alpha1_RayActorOptionSpec_To_v1beta1_RayActorOptionSpec(in, out, s) +} + +func autoConvert_v1beta1_RayCluster_To_v1alpha1_RayCluster(in *RayCluster, out *v1alpha1.RayCluster, s conversion.Scope) error { + out.ObjectMeta = in.ObjectMeta + if err := Convert_v1beta1_RayClusterSpec_To_v1alpha1_RayClusterSpec(&in.Spec, &out.Spec, s); err != nil { + return err + } + if err := Convert_v1beta1_RayClusterStatus_To_v1alpha1_RayClusterStatus(&in.Status, &out.Status, s); err != nil { + return err + } + return nil +} + +// Convert_v1beta1_RayCluster_To_v1alpha1_RayCluster is an autogenerated conversion function. +func Convert_v1beta1_RayCluster_To_v1alpha1_RayCluster(in *RayCluster, out *v1alpha1.RayCluster, s conversion.Scope) error { + return autoConvert_v1beta1_RayCluster_To_v1alpha1_RayCluster(in, out, s) +} + +func autoConvert_v1alpha1_RayCluster_To_v1beta1_RayCluster(in *v1alpha1.RayCluster, out *RayCluster, s conversion.Scope) error { + out.ObjectMeta = in.ObjectMeta + if err := Convert_v1alpha1_RayClusterSpec_To_v1beta1_RayClusterSpec(&in.Spec, &out.Spec, s); err != nil { + return err + } + if err := Convert_v1alpha1_RayClusterStatus_To_v1beta1_RayClusterStatus(&in.Status, &out.Status, s); err != nil { + return err + } + return nil +} + +// Convert_v1alpha1_RayCluster_To_v1beta1_RayCluster is an autogenerated conversion function. +func Convert_v1alpha1_RayCluster_To_v1beta1_RayCluster(in *v1alpha1.RayCluster, out *RayCluster, s conversion.Scope) error { + return autoConvert_v1alpha1_RayCluster_To_v1beta1_RayCluster(in, out, s) +} + +func autoConvert_v1beta1_RayClusterList_To_v1alpha1_RayClusterList(in *RayClusterList, out *v1alpha1.RayClusterList, s conversion.Scope) error { + out.ListMeta = in.ListMeta + out.Items = *(*[]v1alpha1.RayCluster)(unsafe.Pointer(&in.Items)) + return nil +} + +// Convert_v1beta1_RayClusterList_To_v1alpha1_RayClusterList is an autogenerated conversion function. +func Convert_v1beta1_RayClusterList_To_v1alpha1_RayClusterList(in *RayClusterList, out *v1alpha1.RayClusterList, s conversion.Scope) error { + return autoConvert_v1beta1_RayClusterList_To_v1alpha1_RayClusterList(in, out, s) +} + +func autoConvert_v1alpha1_RayClusterList_To_v1beta1_RayClusterList(in *v1alpha1.RayClusterList, out *RayClusterList, s conversion.Scope) error { + out.ListMeta = in.ListMeta + out.Items = *(*[]RayCluster)(unsafe.Pointer(&in.Items)) + return nil +} + +// Convert_v1alpha1_RayClusterList_To_v1beta1_RayClusterList is an autogenerated conversion function. +func Convert_v1alpha1_RayClusterList_To_v1beta1_RayClusterList(in *v1alpha1.RayClusterList, out *RayClusterList, s conversion.Scope) error { + return autoConvert_v1alpha1_RayClusterList_To_v1beta1_RayClusterList(in, out, s) +} + +func autoConvert_v1beta1_RayClusterSpec_To_v1alpha1_RayClusterSpec(in *RayClusterSpec, out *v1alpha1.RayClusterSpec, s conversion.Scope) error { + if err := Convert_v1beta1_HeadGroupSpec_To_v1alpha1_HeadGroupSpec(&in.HeadGroupSpec, &out.HeadGroupSpec, s); err != nil { + return err + } + out.WorkerGroupSpecs = *(*[]v1alpha1.WorkerGroupSpec)(unsafe.Pointer(&in.WorkerGroupSpecs)) + out.RayVersion = in.RayVersion + out.EnableInTreeAutoscaling = (*bool)(unsafe.Pointer(in.EnableInTreeAutoscaling)) + out.AutoscalerOptions = (*v1alpha1.AutoscalerOptions)(unsafe.Pointer(in.AutoscalerOptions)) + out.HeadServiceAnnotations = *(*map[string]string)(unsafe.Pointer(&in.HeadServiceAnnotations)) + return nil +} + +// Convert_v1beta1_RayClusterSpec_To_v1alpha1_RayClusterSpec is an autogenerated conversion function. +func Convert_v1beta1_RayClusterSpec_To_v1alpha1_RayClusterSpec(in *RayClusterSpec, out *v1alpha1.RayClusterSpec, s conversion.Scope) error { + return autoConvert_v1beta1_RayClusterSpec_To_v1alpha1_RayClusterSpec(in, out, s) +} + +func autoConvert_v1alpha1_RayClusterSpec_To_v1beta1_RayClusterSpec(in *v1alpha1.RayClusterSpec, out *RayClusterSpec, s conversion.Scope) error { + if err := Convert_v1alpha1_HeadGroupSpec_To_v1beta1_HeadGroupSpec(&in.HeadGroupSpec, &out.HeadGroupSpec, s); err != nil { + return err + } + out.WorkerGroupSpecs = *(*[]WorkerGroupSpec)(unsafe.Pointer(&in.WorkerGroupSpecs)) + out.RayVersion = in.RayVersion + out.EnableInTreeAutoscaling = (*bool)(unsafe.Pointer(in.EnableInTreeAutoscaling)) + out.AutoscalerOptions = (*AutoscalerOptions)(unsafe.Pointer(in.AutoscalerOptions)) + out.HeadServiceAnnotations = *(*map[string]string)(unsafe.Pointer(&in.HeadServiceAnnotations)) + return nil +} + +// Convert_v1alpha1_RayClusterSpec_To_v1beta1_RayClusterSpec is an autogenerated conversion function. +func Convert_v1alpha1_RayClusterSpec_To_v1beta1_RayClusterSpec(in *v1alpha1.RayClusterSpec, out *RayClusterSpec, s conversion.Scope) error { + return autoConvert_v1alpha1_RayClusterSpec_To_v1beta1_RayClusterSpec(in, out, s) +} + +func autoConvert_v1beta1_RayClusterStatus_To_v1alpha1_RayClusterStatus(in *RayClusterStatus, out *v1alpha1.RayClusterStatus, s conversion.Scope) error { + out.State = v1alpha1.ClusterState(in.State) + out.AvailableWorkerReplicas = in.AvailableWorkerReplicas + out.DesiredWorkerReplicas = in.DesiredWorkerReplicas + out.MinWorkerReplicas = in.MinWorkerReplicas + out.MaxWorkerReplicas = in.MaxWorkerReplicas + out.LastUpdateTime = (*v1.Time)(unsafe.Pointer(in.LastUpdateTime)) + out.Endpoints = *(*map[string]string)(unsafe.Pointer(&in.Endpoints)) + if err := Convert_v1beta1_HeadInfo_To_v1alpha1_HeadInfo(&in.Head, &out.Head, s); err != nil { + return err + } + out.Reason = in.Reason + out.ObservedGeneration = in.ObservedGeneration + return nil +} + +// Convert_v1beta1_RayClusterStatus_To_v1alpha1_RayClusterStatus is an autogenerated conversion function. +func Convert_v1beta1_RayClusterStatus_To_v1alpha1_RayClusterStatus(in *RayClusterStatus, out *v1alpha1.RayClusterStatus, s conversion.Scope) error { + return autoConvert_v1beta1_RayClusterStatus_To_v1alpha1_RayClusterStatus(in, out, s) +} + +func autoConvert_v1alpha1_RayClusterStatus_To_v1beta1_RayClusterStatus(in *v1alpha1.RayClusterStatus, out *RayClusterStatus, s conversion.Scope) error { + out.State = ClusterState(in.State) + out.AvailableWorkerReplicas = in.AvailableWorkerReplicas + out.DesiredWorkerReplicas = in.DesiredWorkerReplicas + out.MinWorkerReplicas = in.MinWorkerReplicas + out.MaxWorkerReplicas = in.MaxWorkerReplicas + out.LastUpdateTime = (*v1.Time)(unsafe.Pointer(in.LastUpdateTime)) + out.Endpoints = *(*map[string]string)(unsafe.Pointer(&in.Endpoints)) + if err := Convert_v1alpha1_HeadInfo_To_v1beta1_HeadInfo(&in.Head, &out.Head, s); err != nil { + return err + } + out.Reason = in.Reason + out.ObservedGeneration = in.ObservedGeneration + return nil +} + +// Convert_v1alpha1_RayClusterStatus_To_v1beta1_RayClusterStatus is an autogenerated conversion function. +func Convert_v1alpha1_RayClusterStatus_To_v1beta1_RayClusterStatus(in *v1alpha1.RayClusterStatus, out *RayClusterStatus, s conversion.Scope) error { + return autoConvert_v1alpha1_RayClusterStatus_To_v1beta1_RayClusterStatus(in, out, s) +} + +func autoConvert_v1beta1_RayService_To_v1alpha1_RayService(in *RayService, out *v1alpha1.RayService, s conversion.Scope) error { + out.ObjectMeta = in.ObjectMeta + if err := Convert_v1beta1_RayServiceSpec_To_v1alpha1_RayServiceSpec(&in.Spec, &out.Spec, s); err != nil { + return err + } + if err := Convert_v1beta1_RayServiceStatuses_To_v1alpha1_RayServiceStatuses(&in.Status, &out.Status, s); err != nil { + return err + } + return nil +} + +// Convert_v1beta1_RayService_To_v1alpha1_RayService is an autogenerated conversion function. +func Convert_v1beta1_RayService_To_v1alpha1_RayService(in *RayService, out *v1alpha1.RayService, s conversion.Scope) error { + return autoConvert_v1beta1_RayService_To_v1alpha1_RayService(in, out, s) +} + +func autoConvert_v1alpha1_RayService_To_v1beta1_RayService(in *v1alpha1.RayService, out *RayService, s conversion.Scope) error { + out.ObjectMeta = in.ObjectMeta + if err := Convert_v1alpha1_RayServiceSpec_To_v1beta1_RayServiceSpec(&in.Spec, &out.Spec, s); err != nil { + return err + } + if err := Convert_v1alpha1_RayServiceStatuses_To_v1beta1_RayServiceStatuses(&in.Status, &out.Status, s); err != nil { + return err + } + return nil +} + +// Convert_v1alpha1_RayService_To_v1beta1_RayService is an autogenerated conversion function. +func Convert_v1alpha1_RayService_To_v1beta1_RayService(in *v1alpha1.RayService, out *RayService, s conversion.Scope) error { + return autoConvert_v1alpha1_RayService_To_v1beta1_RayService(in, out, s) +} + +func autoConvert_v1beta1_RayServiceList_To_v1alpha1_RayServiceList(in *RayServiceList, out *v1alpha1.RayServiceList, s conversion.Scope) error { + out.ListMeta = in.ListMeta + out.Items = *(*[]v1alpha1.RayService)(unsafe.Pointer(&in.Items)) + return nil +} + +// Convert_v1beta1_RayServiceList_To_v1alpha1_RayServiceList is an autogenerated conversion function. +func Convert_v1beta1_RayServiceList_To_v1alpha1_RayServiceList(in *RayServiceList, out *v1alpha1.RayServiceList, s conversion.Scope) error { + return autoConvert_v1beta1_RayServiceList_To_v1alpha1_RayServiceList(in, out, s) +} + +func autoConvert_v1alpha1_RayServiceList_To_v1beta1_RayServiceList(in *v1alpha1.RayServiceList, out *RayServiceList, s conversion.Scope) error { + out.ListMeta = in.ListMeta + out.Items = *(*[]RayService)(unsafe.Pointer(&in.Items)) + return nil +} + +// Convert_v1alpha1_RayServiceList_To_v1beta1_RayServiceList is an autogenerated conversion function. +func Convert_v1alpha1_RayServiceList_To_v1beta1_RayServiceList(in *v1alpha1.RayServiceList, out *RayServiceList, s conversion.Scope) error { + return autoConvert_v1alpha1_RayServiceList_To_v1beta1_RayServiceList(in, out, s) +} + +func autoConvert_v1beta1_RayServiceSpec_To_v1alpha1_RayServiceSpec(in *RayServiceSpec, out *v1alpha1.RayServiceSpec, s conversion.Scope) error { + if err := Convert_v1beta1_ServeDeploymentGraphSpec_To_v1alpha1_ServeDeploymentGraphSpec(&in.ServeDeploymentGraphSpec, &out.ServeDeploymentGraphSpec, s); err != nil { + return err + } + if err := Convert_v1beta1_RayClusterSpec_To_v1alpha1_RayClusterSpec(&in.RayClusterSpec, &out.RayClusterSpec, s); err != nil { + return err + } + out.ServiceUnhealthySecondThreshold = (*int32)(unsafe.Pointer(in.ServiceUnhealthySecondThreshold)) + out.DeploymentUnhealthySecondThreshold = (*int32)(unsafe.Pointer(in.DeploymentUnhealthySecondThreshold)) + return nil +} + +// Convert_v1beta1_RayServiceSpec_To_v1alpha1_RayServiceSpec is an autogenerated conversion function. +func Convert_v1beta1_RayServiceSpec_To_v1alpha1_RayServiceSpec(in *RayServiceSpec, out *v1alpha1.RayServiceSpec, s conversion.Scope) error { + return autoConvert_v1beta1_RayServiceSpec_To_v1alpha1_RayServiceSpec(in, out, s) +} + +func autoConvert_v1alpha1_RayServiceSpec_To_v1beta1_RayServiceSpec(in *v1alpha1.RayServiceSpec, out *RayServiceSpec, s conversion.Scope) error { + if err := Convert_v1alpha1_ServeDeploymentGraphSpec_To_v1beta1_ServeDeploymentGraphSpec(&in.ServeDeploymentGraphSpec, &out.ServeDeploymentGraphSpec, s); err != nil { + return err + } + if err := Convert_v1alpha1_RayClusterSpec_To_v1beta1_RayClusterSpec(&in.RayClusterSpec, &out.RayClusterSpec, s); err != nil { + return err + } + out.ServiceUnhealthySecondThreshold = (*int32)(unsafe.Pointer(in.ServiceUnhealthySecondThreshold)) + out.DeploymentUnhealthySecondThreshold = (*int32)(unsafe.Pointer(in.DeploymentUnhealthySecondThreshold)) + return nil +} + +// Convert_v1alpha1_RayServiceSpec_To_v1beta1_RayServiceSpec is an autogenerated conversion function. +func Convert_v1alpha1_RayServiceSpec_To_v1beta1_RayServiceSpec(in *v1alpha1.RayServiceSpec, out *RayServiceSpec, s conversion.Scope) error { + return autoConvert_v1alpha1_RayServiceSpec_To_v1beta1_RayServiceSpec(in, out, s) +} + +func autoConvert_v1beta1_RayServiceStatus_To_v1alpha1_RayServiceStatus(in *RayServiceStatus, out *v1alpha1.RayServiceStatus, s conversion.Scope) error { + if err := Convert_v1beta1_AppStatus_To_v1alpha1_AppStatus(&in.ApplicationStatus, &out.ApplicationStatus, s); err != nil { + return err + } + out.ServeStatuses = *(*[]v1alpha1.ServeDeploymentStatus)(unsafe.Pointer(&in.ServeStatuses)) + if err := Convert_v1beta1_DashboardStatus_To_v1alpha1_DashboardStatus(&in.DashboardStatus, &out.DashboardStatus, s); err != nil { + return err + } + out.RayClusterName = in.RayClusterName + if err := Convert_v1beta1_RayClusterStatus_To_v1alpha1_RayClusterStatus(&in.RayClusterStatus, &out.RayClusterStatus, s); err != nil { + return err + } + return nil +} + +// Convert_v1beta1_RayServiceStatus_To_v1alpha1_RayServiceStatus is an autogenerated conversion function. +func Convert_v1beta1_RayServiceStatus_To_v1alpha1_RayServiceStatus(in *RayServiceStatus, out *v1alpha1.RayServiceStatus, s conversion.Scope) error { + return autoConvert_v1beta1_RayServiceStatus_To_v1alpha1_RayServiceStatus(in, out, s) +} + +func autoConvert_v1alpha1_RayServiceStatus_To_v1beta1_RayServiceStatus(in *v1alpha1.RayServiceStatus, out *RayServiceStatus, s conversion.Scope) error { + if err := Convert_v1alpha1_AppStatus_To_v1beta1_AppStatus(&in.ApplicationStatus, &out.ApplicationStatus, s); err != nil { + return err + } + out.ServeStatuses = *(*[]ServeDeploymentStatus)(unsafe.Pointer(&in.ServeStatuses)) + if err := Convert_v1alpha1_DashboardStatus_To_v1beta1_DashboardStatus(&in.DashboardStatus, &out.DashboardStatus, s); err != nil { + return err + } + out.RayClusterName = in.RayClusterName + if err := Convert_v1alpha1_RayClusterStatus_To_v1beta1_RayClusterStatus(&in.RayClusterStatus, &out.RayClusterStatus, s); err != nil { + return err + } + return nil +} + +// Convert_v1alpha1_RayServiceStatus_To_v1beta1_RayServiceStatus is an autogenerated conversion function. +func Convert_v1alpha1_RayServiceStatus_To_v1beta1_RayServiceStatus(in *v1alpha1.RayServiceStatus, out *RayServiceStatus, s conversion.Scope) error { + return autoConvert_v1alpha1_RayServiceStatus_To_v1beta1_RayServiceStatus(in, out, s) +} + +func autoConvert_v1beta1_RayServiceStatuses_To_v1alpha1_RayServiceStatuses(in *RayServiceStatuses, out *v1alpha1.RayServiceStatuses, s conversion.Scope) error { + if err := Convert_v1beta1_RayServiceStatus_To_v1alpha1_RayServiceStatus(&in.ActiveServiceStatus, &out.ActiveServiceStatus, s); err != nil { + return err + } + if err := Convert_v1beta1_RayServiceStatus_To_v1alpha1_RayServiceStatus(&in.PendingServiceStatus, &out.PendingServiceStatus, s); err != nil { + return err + } + out.ServiceStatus = v1alpha1.ServiceStatus(in.ServiceStatus) + out.ObservedGeneration = in.ObservedGeneration + return nil +} + +// Convert_v1beta1_RayServiceStatuses_To_v1alpha1_RayServiceStatuses is an autogenerated conversion function. +func Convert_v1beta1_RayServiceStatuses_To_v1alpha1_RayServiceStatuses(in *RayServiceStatuses, out *v1alpha1.RayServiceStatuses, s conversion.Scope) error { + return autoConvert_v1beta1_RayServiceStatuses_To_v1alpha1_RayServiceStatuses(in, out, s) +} + +func autoConvert_v1alpha1_RayServiceStatuses_To_v1beta1_RayServiceStatuses(in *v1alpha1.RayServiceStatuses, out *RayServiceStatuses, s conversion.Scope) error { + if err := Convert_v1alpha1_RayServiceStatus_To_v1beta1_RayServiceStatus(&in.ActiveServiceStatus, &out.ActiveServiceStatus, s); err != nil { + return err + } + if err := Convert_v1alpha1_RayServiceStatus_To_v1beta1_RayServiceStatus(&in.PendingServiceStatus, &out.PendingServiceStatus, s); err != nil { + return err + } + out.ServiceStatus = ServiceStatus(in.ServiceStatus) + out.ObservedGeneration = in.ObservedGeneration + return nil +} + +// Convert_v1alpha1_RayServiceStatuses_To_v1beta1_RayServiceStatuses is an autogenerated conversion function. +func Convert_v1alpha1_RayServiceStatuses_To_v1beta1_RayServiceStatuses(in *v1alpha1.RayServiceStatuses, out *RayServiceStatuses, s conversion.Scope) error { + return autoConvert_v1alpha1_RayServiceStatuses_To_v1beta1_RayServiceStatuses(in, out, s) +} + +func autoConvert_v1beta1_ScaleStrategy_To_v1alpha1_ScaleStrategy(in *ScaleStrategy, out *v1alpha1.ScaleStrategy, s conversion.Scope) error { + out.WorkersToDelete = *(*[]string)(unsafe.Pointer(&in.WorkersToDelete)) + return nil +} + +// Convert_v1beta1_ScaleStrategy_To_v1alpha1_ScaleStrategy is an autogenerated conversion function. +func Convert_v1beta1_ScaleStrategy_To_v1alpha1_ScaleStrategy(in *ScaleStrategy, out *v1alpha1.ScaleStrategy, s conversion.Scope) error { + return autoConvert_v1beta1_ScaleStrategy_To_v1alpha1_ScaleStrategy(in, out, s) +} + +func autoConvert_v1alpha1_ScaleStrategy_To_v1beta1_ScaleStrategy(in *v1alpha1.ScaleStrategy, out *ScaleStrategy, s conversion.Scope) error { + out.WorkersToDelete = *(*[]string)(unsafe.Pointer(&in.WorkersToDelete)) + return nil +} + +// Convert_v1alpha1_ScaleStrategy_To_v1beta1_ScaleStrategy is an autogenerated conversion function. +func Convert_v1alpha1_ScaleStrategy_To_v1beta1_ScaleStrategy(in *v1alpha1.ScaleStrategy, out *ScaleStrategy, s conversion.Scope) error { + return autoConvert_v1alpha1_ScaleStrategy_To_v1beta1_ScaleStrategy(in, out, s) +} + +func autoConvert_v1beta1_ServeConfigSpec_To_v1alpha1_ServeConfigSpec(in *ServeConfigSpec, out *v1alpha1.ServeConfigSpec, s conversion.Scope) error { + out.Name = in.Name + out.NumReplicas = (*int32)(unsafe.Pointer(in.NumReplicas)) + out.RoutePrefix = in.RoutePrefix + out.MaxConcurrentQueries = (*int32)(unsafe.Pointer(in.MaxConcurrentQueries)) + out.UserConfig = in.UserConfig + out.AutoscalingConfig = in.AutoscalingConfig + out.GracefulShutdownWaitLoopS = (*int32)(unsafe.Pointer(in.GracefulShutdownWaitLoopS)) + out.GracefulShutdownTimeoutS = (*int32)(unsafe.Pointer(in.GracefulShutdownTimeoutS)) + out.HealthCheckPeriodS = (*int32)(unsafe.Pointer(in.HealthCheckPeriodS)) + out.HealthCheckTimeoutS = (*int32)(unsafe.Pointer(in.HealthCheckTimeoutS)) + if err := Convert_v1beta1_RayActorOptionSpec_To_v1alpha1_RayActorOptionSpec(&in.RayActorOptions, &out.RayActorOptions, s); err != nil { + return err + } + return nil +} + +// Convert_v1beta1_ServeConfigSpec_To_v1alpha1_ServeConfigSpec is an autogenerated conversion function. +func Convert_v1beta1_ServeConfigSpec_To_v1alpha1_ServeConfigSpec(in *ServeConfigSpec, out *v1alpha1.ServeConfigSpec, s conversion.Scope) error { + return autoConvert_v1beta1_ServeConfigSpec_To_v1alpha1_ServeConfigSpec(in, out, s) +} + +func autoConvert_v1alpha1_ServeConfigSpec_To_v1beta1_ServeConfigSpec(in *v1alpha1.ServeConfigSpec, out *ServeConfigSpec, s conversion.Scope) error { + out.Name = in.Name + out.NumReplicas = (*int32)(unsafe.Pointer(in.NumReplicas)) + out.RoutePrefix = in.RoutePrefix + out.MaxConcurrentQueries = (*int32)(unsafe.Pointer(in.MaxConcurrentQueries)) + out.UserConfig = in.UserConfig + out.AutoscalingConfig = in.AutoscalingConfig + out.GracefulShutdownWaitLoopS = (*int32)(unsafe.Pointer(in.GracefulShutdownWaitLoopS)) + out.GracefulShutdownTimeoutS = (*int32)(unsafe.Pointer(in.GracefulShutdownTimeoutS)) + out.HealthCheckPeriodS = (*int32)(unsafe.Pointer(in.HealthCheckPeriodS)) + out.HealthCheckTimeoutS = (*int32)(unsafe.Pointer(in.HealthCheckTimeoutS)) + if err := Convert_v1alpha1_RayActorOptionSpec_To_v1beta1_RayActorOptionSpec(&in.RayActorOptions, &out.RayActorOptions, s); err != nil { + return err + } + return nil +} + +// Convert_v1alpha1_ServeConfigSpec_To_v1beta1_ServeConfigSpec is an autogenerated conversion function. +func Convert_v1alpha1_ServeConfigSpec_To_v1beta1_ServeConfigSpec(in *v1alpha1.ServeConfigSpec, out *ServeConfigSpec, s conversion.Scope) error { + return autoConvert_v1alpha1_ServeConfigSpec_To_v1beta1_ServeConfigSpec(in, out, s) +} + +func autoConvert_v1beta1_ServeDeploymentGraphSpec_To_v1alpha1_ServeDeploymentGraphSpec(in *ServeDeploymentGraphSpec, out *v1alpha1.ServeDeploymentGraphSpec, s conversion.Scope) error { + out.ImportPath = in.ImportPath + out.RuntimeEnv = in.RuntimeEnv + out.ServeConfigSpecs = *(*[]v1alpha1.ServeConfigSpec)(unsafe.Pointer(&in.ServeConfigSpecs)) + out.Port = in.Port + return nil +} + +// Convert_v1beta1_ServeDeploymentGraphSpec_To_v1alpha1_ServeDeploymentGraphSpec is an autogenerated conversion function. +func Convert_v1beta1_ServeDeploymentGraphSpec_To_v1alpha1_ServeDeploymentGraphSpec(in *ServeDeploymentGraphSpec, out *v1alpha1.ServeDeploymentGraphSpec, s conversion.Scope) error { + return autoConvert_v1beta1_ServeDeploymentGraphSpec_To_v1alpha1_ServeDeploymentGraphSpec(in, out, s) +} + +func autoConvert_v1alpha1_ServeDeploymentGraphSpec_To_v1beta1_ServeDeploymentGraphSpec(in *v1alpha1.ServeDeploymentGraphSpec, out *ServeDeploymentGraphSpec, s conversion.Scope) error { + out.ImportPath = in.ImportPath + out.RuntimeEnv = in.RuntimeEnv + out.ServeConfigSpecs = *(*[]ServeConfigSpec)(unsafe.Pointer(&in.ServeConfigSpecs)) + out.Port = in.Port + return nil +} + +// Convert_v1alpha1_ServeDeploymentGraphSpec_To_v1beta1_ServeDeploymentGraphSpec is an autogenerated conversion function. +func Convert_v1alpha1_ServeDeploymentGraphSpec_To_v1beta1_ServeDeploymentGraphSpec(in *v1alpha1.ServeDeploymentGraphSpec, out *ServeDeploymentGraphSpec, s conversion.Scope) error { + return autoConvert_v1alpha1_ServeDeploymentGraphSpec_To_v1beta1_ServeDeploymentGraphSpec(in, out, s) +} + +func autoConvert_v1beta1_ServeDeploymentStatus_To_v1alpha1_ServeDeploymentStatus(in *ServeDeploymentStatus, out *v1alpha1.ServeDeploymentStatus, s conversion.Scope) error { + out.Name = in.Name + out.Status = in.Status + out.Message = in.Message + out.LastUpdateTime = (*v1.Time)(unsafe.Pointer(in.LastUpdateTime)) + out.HealthLastUpdateTime = (*v1.Time)(unsafe.Pointer(in.HealthLastUpdateTime)) + return nil +} + +// Convert_v1beta1_ServeDeploymentStatus_To_v1alpha1_ServeDeploymentStatus is an autogenerated conversion function. +func Convert_v1beta1_ServeDeploymentStatus_To_v1alpha1_ServeDeploymentStatus(in *ServeDeploymentStatus, out *v1alpha1.ServeDeploymentStatus, s conversion.Scope) error { + return autoConvert_v1beta1_ServeDeploymentStatus_To_v1alpha1_ServeDeploymentStatus(in, out, s) +} + +func autoConvert_v1alpha1_ServeDeploymentStatus_To_v1beta1_ServeDeploymentStatus(in *v1alpha1.ServeDeploymentStatus, out *ServeDeploymentStatus, s conversion.Scope) error { + out.Name = in.Name + out.Status = in.Status + out.Message = in.Message + out.LastUpdateTime = (*v1.Time)(unsafe.Pointer(in.LastUpdateTime)) + out.HealthLastUpdateTime = (*v1.Time)(unsafe.Pointer(in.HealthLastUpdateTime)) + return nil +} + +// Convert_v1alpha1_ServeDeploymentStatus_To_v1beta1_ServeDeploymentStatus is an autogenerated conversion function. +func Convert_v1alpha1_ServeDeploymentStatus_To_v1beta1_ServeDeploymentStatus(in *v1alpha1.ServeDeploymentStatus, out *ServeDeploymentStatus, s conversion.Scope) error { + return autoConvert_v1alpha1_ServeDeploymentStatus_To_v1beta1_ServeDeploymentStatus(in, out, s) +} + +func autoConvert_v1beta1_WorkerGroupSpec_To_v1alpha1_WorkerGroupSpec(in *WorkerGroupSpec, out *v1alpha1.WorkerGroupSpec, s conversion.Scope) error { + out.GroupName = in.GroupName + out.Replicas = (*int32)(unsafe.Pointer(in.Replicas)) + out.MinReplicas = (*int32)(unsafe.Pointer(in.MinReplicas)) + out.MaxReplicas = (*int32)(unsafe.Pointer(in.MaxReplicas)) + out.RayStartParams = *(*map[string]string)(unsafe.Pointer(&in.RayStartParams)) + out.Template = in.Template + if err := Convert_v1beta1_ScaleStrategy_To_v1alpha1_ScaleStrategy(&in.ScaleStrategy, &out.ScaleStrategy, s); err != nil { + return err + } + return nil +} + +// Convert_v1beta1_WorkerGroupSpec_To_v1alpha1_WorkerGroupSpec is an autogenerated conversion function. +func Convert_v1beta1_WorkerGroupSpec_To_v1alpha1_WorkerGroupSpec(in *WorkerGroupSpec, out *v1alpha1.WorkerGroupSpec, s conversion.Scope) error { + return autoConvert_v1beta1_WorkerGroupSpec_To_v1alpha1_WorkerGroupSpec(in, out, s) +} + +func autoConvert_v1alpha1_WorkerGroupSpec_To_v1beta1_WorkerGroupSpec(in *v1alpha1.WorkerGroupSpec, out *WorkerGroupSpec, s conversion.Scope) error { + out.GroupName = in.GroupName + out.Replicas = (*int32)(unsafe.Pointer(in.Replicas)) + out.MinReplicas = (*int32)(unsafe.Pointer(in.MinReplicas)) + out.MaxReplicas = (*int32)(unsafe.Pointer(in.MaxReplicas)) + out.RayStartParams = *(*map[string]string)(unsafe.Pointer(&in.RayStartParams)) + out.Template = in.Template + if err := Convert_v1alpha1_ScaleStrategy_To_v1beta1_ScaleStrategy(&in.ScaleStrategy, &out.ScaleStrategy, s); err != nil { + return err + } + return nil +} + +// Convert_v1alpha1_WorkerGroupSpec_To_v1beta1_WorkerGroupSpec is an autogenerated conversion function. +func Convert_v1alpha1_WorkerGroupSpec_To_v1beta1_WorkerGroupSpec(in *v1alpha1.WorkerGroupSpec, out *WorkerGroupSpec, s conversion.Scope) error { + return autoConvert_v1alpha1_WorkerGroupSpec_To_v1beta1_WorkerGroupSpec(in, out, s) +} diff --git a/ray-operator/apis/ray/v1beta1/zz_generated.deepcopy.go b/ray-operator/apis/ray/v1beta1/zz_generated.deepcopy.go new file mode 100644 index 0000000000..0f79fbd104 --- /dev/null +++ b/ray-operator/apis/ray/v1beta1/zz_generated.deepcopy.go @@ -0,0 +1,615 @@ +//go:build !ignore_autogenerated +// +build !ignore_autogenerated + +// Code generated by controller-gen. DO NOT EDIT. + +package v1beta1 + +import ( + "k8s.io/api/core/v1" + "k8s.io/apimachinery/pkg/runtime" +) + +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *AppStatus) DeepCopyInto(out *AppStatus) { + *out = *in + if in.LastUpdateTime != nil { + in, out := &in.LastUpdateTime, &out.LastUpdateTime + *out = (*in).DeepCopy() + } + if in.HealthLastUpdateTime != nil { + in, out := &in.HealthLastUpdateTime, &out.HealthLastUpdateTime + *out = (*in).DeepCopy() + } +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new AppStatus. +func (in *AppStatus) DeepCopy() *AppStatus { + if in == nil { + return nil + } + out := new(AppStatus) + in.DeepCopyInto(out) + return out +} + +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *AutoscalerOptions) DeepCopyInto(out *AutoscalerOptions) { + *out = *in + if in.Resources != nil { + in, out := &in.Resources, &out.Resources + *out = new(v1.ResourceRequirements) + (*in).DeepCopyInto(*out) + } + if in.Image != nil { + in, out := &in.Image, &out.Image + *out = new(string) + **out = **in + } + if in.ImagePullPolicy != nil { + in, out := &in.ImagePullPolicy, &out.ImagePullPolicy + *out = new(v1.PullPolicy) + **out = **in + } + if in.Env != nil { + in, out := &in.Env, &out.Env + *out = make([]v1.EnvVar, len(*in)) + for i := range *in { + (*in)[i].DeepCopyInto(&(*out)[i]) + } + } + if in.EnvFrom != nil { + in, out := &in.EnvFrom, &out.EnvFrom + *out = make([]v1.EnvFromSource, len(*in)) + for i := range *in { + (*in)[i].DeepCopyInto(&(*out)[i]) + } + } + if in.VolumeMounts != nil { + in, out := &in.VolumeMounts, &out.VolumeMounts + *out = make([]v1.VolumeMount, len(*in)) + for i := range *in { + (*in)[i].DeepCopyInto(&(*out)[i]) + } + } + if in.SecurityContext != nil { + in, out := &in.SecurityContext, &out.SecurityContext + *out = new(v1.SecurityContext) + (*in).DeepCopyInto(*out) + } + if in.IdleTimeoutSeconds != nil { + in, out := &in.IdleTimeoutSeconds, &out.IdleTimeoutSeconds + *out = new(int32) + **out = **in + } + if in.UpscalingMode != nil { + in, out := &in.UpscalingMode, &out.UpscalingMode + *out = new(UpscalingMode) + **out = **in + } +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new AutoscalerOptions. +func (in *AutoscalerOptions) DeepCopy() *AutoscalerOptions { + if in == nil { + return nil + } + out := new(AutoscalerOptions) + in.DeepCopyInto(out) + return out +} + +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *DashboardStatus) DeepCopyInto(out *DashboardStatus) { + *out = *in + if in.LastUpdateTime != nil { + in, out := &in.LastUpdateTime, &out.LastUpdateTime + *out = (*in).DeepCopy() + } + if in.HealthLastUpdateTime != nil { + in, out := &in.HealthLastUpdateTime, &out.HealthLastUpdateTime + *out = (*in).DeepCopy() + } +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new DashboardStatus. +func (in *DashboardStatus) DeepCopy() *DashboardStatus { + if in == nil { + return nil + } + out := new(DashboardStatus) + in.DeepCopyInto(out) + return out +} + +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *HeadGroupSpec) DeepCopyInto(out *HeadGroupSpec) { + *out = *in + if in.HeadService != nil { + in, out := &in.HeadService, &out.HeadService + *out = new(v1.Service) + (*in).DeepCopyInto(*out) + } + if in.EnableIngress != nil { + in, out := &in.EnableIngress, &out.EnableIngress + *out = new(bool) + **out = **in + } + if in.Replicas != nil { + in, out := &in.Replicas, &out.Replicas + *out = new(int32) + **out = **in + } + if in.RayStartParams != nil { + in, out := &in.RayStartParams, &out.RayStartParams + *out = make(map[string]string, len(*in)) + for key, val := range *in { + (*out)[key] = val + } + } + in.Template.DeepCopyInto(&out.Template) +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new HeadGroupSpec. +func (in *HeadGroupSpec) DeepCopy() *HeadGroupSpec { + if in == nil { + return nil + } + out := new(HeadGroupSpec) + in.DeepCopyInto(out) + return out +} + +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *HeadInfo) DeepCopyInto(out *HeadInfo) { + *out = *in +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new HeadInfo. +func (in *HeadInfo) DeepCopy() *HeadInfo { + if in == nil { + return nil + } + out := new(HeadInfo) + in.DeepCopyInto(out) + return out +} + +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *RayActorOptionSpec) DeepCopyInto(out *RayActorOptionSpec) { + *out = *in + if in.NumCpus != nil { + in, out := &in.NumCpus, &out.NumCpus + *out = new(float64) + **out = **in + } + if in.NumGpus != nil { + in, out := &in.NumGpus, &out.NumGpus + *out = new(float64) + **out = **in + } + if in.Memory != nil { + in, out := &in.Memory, &out.Memory + *out = new(int32) + **out = **in + } + if in.ObjectStoreMemory != nil { + in, out := &in.ObjectStoreMemory, &out.ObjectStoreMemory + *out = new(int32) + **out = **in + } +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new RayActorOptionSpec. +func (in *RayActorOptionSpec) DeepCopy() *RayActorOptionSpec { + if in == nil { + return nil + } + out := new(RayActorOptionSpec) + in.DeepCopyInto(out) + return out +} + +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *RayCluster) DeepCopyInto(out *RayCluster) { + *out = *in + out.TypeMeta = in.TypeMeta + in.ObjectMeta.DeepCopyInto(&out.ObjectMeta) + in.Spec.DeepCopyInto(&out.Spec) + in.Status.DeepCopyInto(&out.Status) +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new RayCluster. +func (in *RayCluster) DeepCopy() *RayCluster { + if in == nil { + return nil + } + out := new(RayCluster) + in.DeepCopyInto(out) + return out +} + +// DeepCopyObject is an autogenerated deepcopy function, copying the receiver, creating a new runtime.Object. +func (in *RayCluster) DeepCopyObject() runtime.Object { + if c := in.DeepCopy(); c != nil { + return c + } + return nil +} + +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *RayClusterList) DeepCopyInto(out *RayClusterList) { + *out = *in + out.TypeMeta = in.TypeMeta + in.ListMeta.DeepCopyInto(&out.ListMeta) + if in.Items != nil { + in, out := &in.Items, &out.Items + *out = make([]RayCluster, len(*in)) + for i := range *in { + (*in)[i].DeepCopyInto(&(*out)[i]) + } + } +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new RayClusterList. +func (in *RayClusterList) DeepCopy() *RayClusterList { + if in == nil { + return nil + } + out := new(RayClusterList) + in.DeepCopyInto(out) + return out +} + +// DeepCopyObject is an autogenerated deepcopy function, copying the receiver, creating a new runtime.Object. +func (in *RayClusterList) DeepCopyObject() runtime.Object { + if c := in.DeepCopy(); c != nil { + return c + } + return nil +} + +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *RayClusterSpec) DeepCopyInto(out *RayClusterSpec) { + *out = *in + in.HeadGroupSpec.DeepCopyInto(&out.HeadGroupSpec) + if in.WorkerGroupSpecs != nil { + in, out := &in.WorkerGroupSpecs, &out.WorkerGroupSpecs + *out = make([]WorkerGroupSpec, len(*in)) + for i := range *in { + (*in)[i].DeepCopyInto(&(*out)[i]) + } + } + if in.EnableInTreeAutoscaling != nil { + in, out := &in.EnableInTreeAutoscaling, &out.EnableInTreeAutoscaling + *out = new(bool) + **out = **in + } + if in.AutoscalerOptions != nil { + in, out := &in.AutoscalerOptions, &out.AutoscalerOptions + *out = new(AutoscalerOptions) + (*in).DeepCopyInto(*out) + } + if in.HeadServiceAnnotations != nil { + in, out := &in.HeadServiceAnnotations, &out.HeadServiceAnnotations + *out = make(map[string]string, len(*in)) + for key, val := range *in { + (*out)[key] = val + } + } +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new RayClusterSpec. +func (in *RayClusterSpec) DeepCopy() *RayClusterSpec { + if in == nil { + return nil + } + out := new(RayClusterSpec) + in.DeepCopyInto(out) + return out +} + +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *RayClusterStatus) DeepCopyInto(out *RayClusterStatus) { + *out = *in + if in.LastUpdateTime != nil { + in, out := &in.LastUpdateTime, &out.LastUpdateTime + *out = (*in).DeepCopy() + } + if in.Endpoints != nil { + in, out := &in.Endpoints, &out.Endpoints + *out = make(map[string]string, len(*in)) + for key, val := range *in { + (*out)[key] = val + } + } + out.Head = in.Head +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new RayClusterStatus. +func (in *RayClusterStatus) DeepCopy() *RayClusterStatus { + if in == nil { + return nil + } + out := new(RayClusterStatus) + in.DeepCopyInto(out) + return out +} + +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *RayService) DeepCopyInto(out *RayService) { + *out = *in + out.TypeMeta = in.TypeMeta + in.ObjectMeta.DeepCopyInto(&out.ObjectMeta) + in.Spec.DeepCopyInto(&out.Spec) + in.Status.DeepCopyInto(&out.Status) +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new RayService. +func (in *RayService) DeepCopy() *RayService { + if in == nil { + return nil + } + out := new(RayService) + in.DeepCopyInto(out) + return out +} + +// DeepCopyObject is an autogenerated deepcopy function, copying the receiver, creating a new runtime.Object. +func (in *RayService) DeepCopyObject() runtime.Object { + if c := in.DeepCopy(); c != nil { + return c + } + return nil +} + +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *RayServiceList) DeepCopyInto(out *RayServiceList) { + *out = *in + out.TypeMeta = in.TypeMeta + in.ListMeta.DeepCopyInto(&out.ListMeta) + if in.Items != nil { + in, out := &in.Items, &out.Items + *out = make([]RayService, len(*in)) + for i := range *in { + (*in)[i].DeepCopyInto(&(*out)[i]) + } + } +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new RayServiceList. +func (in *RayServiceList) DeepCopy() *RayServiceList { + if in == nil { + return nil + } + out := new(RayServiceList) + in.DeepCopyInto(out) + return out +} + +// DeepCopyObject is an autogenerated deepcopy function, copying the receiver, creating a new runtime.Object. +func (in *RayServiceList) DeepCopyObject() runtime.Object { + if c := in.DeepCopy(); c != nil { + return c + } + return nil +} + +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *RayServiceSpec) DeepCopyInto(out *RayServiceSpec) { + *out = *in + in.ServeDeploymentGraphSpec.DeepCopyInto(&out.ServeDeploymentGraphSpec) + in.RayClusterSpec.DeepCopyInto(&out.RayClusterSpec) + if in.ServiceUnhealthySecondThreshold != nil { + in, out := &in.ServiceUnhealthySecondThreshold, &out.ServiceUnhealthySecondThreshold + *out = new(int32) + **out = **in + } + if in.DeploymentUnhealthySecondThreshold != nil { + in, out := &in.DeploymentUnhealthySecondThreshold, &out.DeploymentUnhealthySecondThreshold + *out = new(int32) + **out = **in + } +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new RayServiceSpec. +func (in *RayServiceSpec) DeepCopy() *RayServiceSpec { + if in == nil { + return nil + } + out := new(RayServiceSpec) + in.DeepCopyInto(out) + return out +} + +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *RayServiceStatus) DeepCopyInto(out *RayServiceStatus) { + *out = *in + in.ApplicationStatus.DeepCopyInto(&out.ApplicationStatus) + if in.ServeStatuses != nil { + in, out := &in.ServeStatuses, &out.ServeStatuses + *out = make([]ServeDeploymentStatus, len(*in)) + for i := range *in { + (*in)[i].DeepCopyInto(&(*out)[i]) + } + } + in.DashboardStatus.DeepCopyInto(&out.DashboardStatus) + in.RayClusterStatus.DeepCopyInto(&out.RayClusterStatus) +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new RayServiceStatus. +func (in *RayServiceStatus) DeepCopy() *RayServiceStatus { + if in == nil { + return nil + } + out := new(RayServiceStatus) + in.DeepCopyInto(out) + return out +} + +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *RayServiceStatuses) DeepCopyInto(out *RayServiceStatuses) { + *out = *in + in.ActiveServiceStatus.DeepCopyInto(&out.ActiveServiceStatus) + in.PendingServiceStatus.DeepCopyInto(&out.PendingServiceStatus) +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new RayServiceStatuses. +func (in *RayServiceStatuses) DeepCopy() *RayServiceStatuses { + if in == nil { + return nil + } + out := new(RayServiceStatuses) + in.DeepCopyInto(out) + return out +} + +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *ScaleStrategy) DeepCopyInto(out *ScaleStrategy) { + *out = *in + if in.WorkersToDelete != nil { + in, out := &in.WorkersToDelete, &out.WorkersToDelete + *out = make([]string, len(*in)) + copy(*out, *in) + } +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new ScaleStrategy. +func (in *ScaleStrategy) DeepCopy() *ScaleStrategy { + if in == nil { + return nil + } + out := new(ScaleStrategy) + in.DeepCopyInto(out) + return out +} + +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *ServeConfigSpec) DeepCopyInto(out *ServeConfigSpec) { + *out = *in + if in.NumReplicas != nil { + in, out := &in.NumReplicas, &out.NumReplicas + *out = new(int32) + **out = **in + } + if in.MaxConcurrentQueries != nil { + in, out := &in.MaxConcurrentQueries, &out.MaxConcurrentQueries + *out = new(int32) + **out = **in + } + if in.GracefulShutdownWaitLoopS != nil { + in, out := &in.GracefulShutdownWaitLoopS, &out.GracefulShutdownWaitLoopS + *out = new(int32) + **out = **in + } + if in.GracefulShutdownTimeoutS != nil { + in, out := &in.GracefulShutdownTimeoutS, &out.GracefulShutdownTimeoutS + *out = new(int32) + **out = **in + } + if in.HealthCheckPeriodS != nil { + in, out := &in.HealthCheckPeriodS, &out.HealthCheckPeriodS + *out = new(int32) + **out = **in + } + if in.HealthCheckTimeoutS != nil { + in, out := &in.HealthCheckTimeoutS, &out.HealthCheckTimeoutS + *out = new(int32) + **out = **in + } + in.RayActorOptions.DeepCopyInto(&out.RayActorOptions) +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new ServeConfigSpec. +func (in *ServeConfigSpec) DeepCopy() *ServeConfigSpec { + if in == nil { + return nil + } + out := new(ServeConfigSpec) + in.DeepCopyInto(out) + return out +} + +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *ServeDeploymentGraphSpec) DeepCopyInto(out *ServeDeploymentGraphSpec) { + *out = *in + if in.ServeConfigSpecs != nil { + in, out := &in.ServeConfigSpecs, &out.ServeConfigSpecs + *out = make([]ServeConfigSpec, len(*in)) + for i := range *in { + (*in)[i].DeepCopyInto(&(*out)[i]) + } + } +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new ServeDeploymentGraphSpec. +func (in *ServeDeploymentGraphSpec) DeepCopy() *ServeDeploymentGraphSpec { + if in == nil { + return nil + } + out := new(ServeDeploymentGraphSpec) + in.DeepCopyInto(out) + return out +} + +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *ServeDeploymentStatus) DeepCopyInto(out *ServeDeploymentStatus) { + *out = *in + if in.LastUpdateTime != nil { + in, out := &in.LastUpdateTime, &out.LastUpdateTime + *out = (*in).DeepCopy() + } + if in.HealthLastUpdateTime != nil { + in, out := &in.HealthLastUpdateTime, &out.HealthLastUpdateTime + *out = (*in).DeepCopy() + } +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new ServeDeploymentStatus. +func (in *ServeDeploymentStatus) DeepCopy() *ServeDeploymentStatus { + if in == nil { + return nil + } + out := new(ServeDeploymentStatus) + in.DeepCopyInto(out) + return out +} + +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *WorkerGroupSpec) DeepCopyInto(out *WorkerGroupSpec) { + *out = *in + if in.Replicas != nil { + in, out := &in.Replicas, &out.Replicas + *out = new(int32) + **out = **in + } + if in.MinReplicas != nil { + in, out := &in.MinReplicas, &out.MinReplicas + *out = new(int32) + **out = **in + } + if in.MaxReplicas != nil { + in, out := &in.MaxReplicas, &out.MaxReplicas + *out = new(int32) + **out = **in + } + if in.RayStartParams != nil { + in, out := &in.RayStartParams, &out.RayStartParams + *out = make(map[string]string, len(*in)) + for key, val := range *in { + (*out)[key] = val + } + } + in.Template.DeepCopyInto(&out.Template) + in.ScaleStrategy.DeepCopyInto(&out.ScaleStrategy) +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new WorkerGroupSpec. +func (in *WorkerGroupSpec) DeepCopy() *WorkerGroupSpec { + if in == nil { + return nil + } + out := new(WorkerGroupSpec) + in.DeepCopyInto(out) + return out +} diff --git a/ray-operator/config/certmanager/certificate.yaml b/ray-operator/config/certmanager/certificate.yaml new file mode 100644 index 0000000000..71210bf4c9 --- /dev/null +++ b/ray-operator/config/certmanager/certificate.yaml @@ -0,0 +1,39 @@ +# The following manifests contain a self-signed issuer CR and a certificate CR. +# More document can be found at https://docs.cert-manager.io +# WARNING: Targets CertManager v1.0. Check https://cert-manager.io/docs/installation/upgrading/ for breaking changes. +apiVersion: cert-manager.io/v1 +kind: Issuer +metadata: + labels: + app.kubernetes.io/name: issuer + app.kubernetes.io/instance: selfsigned-issuer + app.kubernetes.io/component: certificate + app.kubernetes.io/created-by: ray-operator + app.kubernetes.io/part-of: ray-operator + app.kubernetes.io/managed-by: kustomize + name: selfsigned-issuer + namespace: system +spec: + selfSigned: {} +--- +apiVersion: cert-manager.io/v1 +kind: Certificate +metadata: + labels: + app.kubernetes.io/name: certificate + app.kubernetes.io/instance: serving-cert + app.kubernetes.io/component: certificate + app.kubernetes.io/created-by: ray-operator + app.kubernetes.io/part-of: ray-operator + app.kubernetes.io/managed-by: kustomize + name: serving-cert # this name should match the one appeared in kustomizeconfig.yaml + namespace: system +spec: + # $(SERVICE_NAME) and $(SERVICE_NAMESPACE) will be substituted by kustomize + dnsNames: + - $(SERVICE_NAME).$(SERVICE_NAMESPACE).svc + - $(SERVICE_NAME).$(SERVICE_NAMESPACE).svc.cluster.local + issuerRef: + kind: Issuer + name: selfsigned-issuer + secretName: webhook-server-cert # this secret will not be prefixed, since it's not managed by kustomize diff --git a/ray-operator/config/certmanager/kustomization.yaml b/ray-operator/config/certmanager/kustomization.yaml new file mode 100644 index 0000000000..bebea5a595 --- /dev/null +++ b/ray-operator/config/certmanager/kustomization.yaml @@ -0,0 +1,5 @@ +resources: +- certificate.yaml + +configurations: +- kustomizeconfig.yaml diff --git a/ray-operator/config/certmanager/kustomizeconfig.yaml b/ray-operator/config/certmanager/kustomizeconfig.yaml new file mode 100644 index 0000000000..e631f77736 --- /dev/null +++ b/ray-operator/config/certmanager/kustomizeconfig.yaml @@ -0,0 +1,16 @@ +# This configuration is for teaching kustomize how to update name ref and var substitution +nameReference: +- kind: Issuer + group: cert-manager.io + fieldSpecs: + - kind: Certificate + group: cert-manager.io + path: spec/issuerRef/name + +varReference: +- kind: Certificate + group: cert-manager.io + path: spec/commonName +- kind: Certificate + group: cert-manager.io + path: spec/dnsNames diff --git a/ray-operator/config/crd/bases/ray.io_rayclusters.yaml b/ray-operator/config/crd/bases/ray.io_rayclusters.yaml index 81cdc69864..6ebb298497 100644 --- a/ray-operator/config/crd/bases/ray.io_rayclusters.yaml +++ b/ray-operator/config/crd/bases/ray.io_rayclusters.yaml @@ -11635,6 +11635,11625 @@ spec: storage: true subresources: status: {} + - additionalPrinterColumns: + - jsonPath: .status.desiredWorkerReplicas + name: desired workers + type: integer + - jsonPath: .status.availableWorkerReplicas + name: available workers + type: integer + - jsonPath: .status.state + name: status + type: string + - jsonPath: .metadata.creationTimestamp + name: age + type: date + - jsonPath: .status.head.podIP + name: head pod IP + priority: 1 + type: string + - jsonPath: .status.head.serviceIP + name: head service IP + priority: 1 + type: string + name: v1beta1 + schema: + openAPIV3Schema: + description: RayCluster is the Schema for the RayClusters API + properties: + apiVersion: + description: APIVersion defines the versioned schema of this representation + of an object. + type: string + kind: + description: Kind is a string value representing the REST resource this + object represents. + type: string + metadata: + type: object + spec: + description: Specification of the desired behavior of the RayCluster. + properties: + autoscalerOptions: + description: AutoscalerOptions specifies optional configuration for + the Ray autoscaler. + properties: + env: + description: Optional list of environment variables to set in + the autoscaler container. + 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: Optional list of sources to populate environment + variables in the autoscaler 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 + 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 + imagePullPolicy: + description: ImagePullPolicy optionally overrides the autoscaler + container's image pull policy. + type: string + resources: + description: Resources specifies optional resource request and + limit overrides 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 + 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 + upscalingMode: + description: UpscalingMode is "Conservative", "Default", or "Aggressive. + enum: + - Default + - Aggressive + - Conservative + type: string + volumeMounts: + description: Optional list of volumeMounts. This is needed for + enabling TLS for the autoscaler container. + 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 + 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 + headService: + description: HeadService is the Kubernetes service of the head + pod. + properties: + apiVersion: + description: APIVersion defines the versioned schema of this + representation of an object. + type: string + kind: + description: Kind is a string value representing the REST + resource this object represents. + type: string + 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: Spec defines the behavior of a service. https://git.k8s. + properties: + allocateLoadBalancerNodePorts: + description: allocateLoadBalancerNodePorts defines if + NodePorts will be automatically allocated for services + with + type: boolean + clusterIP: + description: clusterIP is the IP address of the service + and is usually assigned randomly. + type: string + clusterIPs: + description: ClusterIPs is a list of IP addresses assigned + to this service, and are usually assigned randomly. + items: + type: string + type: array + x-kubernetes-list-type: atomic + externalIPs: + description: externalIPs is a list of IP addresses for + which nodes in the cluster will also accept traffic + for th + items: + type: string + type: array + externalName: + description: externalName is the external reference that + discovery mechanisms will return as an alias for this + se + type: string + externalTrafficPolicy: + description: externalTrafficPolicy denotes if this Service + desires to route external traffic to node-local or clu + type: string + healthCheckNodePort: + description: healthCheckNodePort specifies the healthcheck + nodePort for the service. + format: int32 + type: integer + internalTrafficPolicy: + description: InternalTrafficPolicy specifies if the cluster + internal traffic should be routed to all endpoints or + type: string + ipFamilies: + description: IPFamilies is a list of IP families (e.g. + IPv4, IPv6) assigned to this service. + items: + description: IPFamily represents the IP Family (IPv4 + or IPv6). + type: string + type: array + x-kubernetes-list-type: atomic + ipFamilyPolicy: + description: IPFamilyPolicy represents the dual-stack-ness + requested or required by this Service. + type: string + loadBalancerClass: + description: loadBalancerClass is the class of the load + balancer implementation this Service belongs to. + type: string + loadBalancerIP: + description: 'Only applies to Service Type: LoadBalancer + LoadBalancer will get created with the IP specified + in th' + type: string + loadBalancerSourceRanges: + description: If specified and supported by the platform, + this will restrict traffic through the cloud-provider + lo + items: + type: string + type: array + ports: + description: 'The list of ports that are exposed by this + service. More info: https://kubernetes.' + items: + description: ServicePort contains information on service's + port. + properties: + appProtocol: + description: The application protocol for this port. + This field follows standard Kubernetes label syntax. + type: string + name: + description: The name of this port within the service. + This must be a DNS_LABEL. + type: string + nodePort: + description: The port on each node on which this + service is exposed when type is NodePort or LoadBalancer. + format: int32 + type: integer + port: + description: The port that will be exposed by this + service. + format: int32 + type: integer + protocol: + default: TCP + description: The IP protocol for this port. Supports + "TCP", "UDP", and "SCTP". Default is TCP. + type: string + targetPort: + anyOf: + - type: integer + - type: string + description: Number or name of the port to access + on the pods targeted by the service. + x-kubernetes-int-or-string: true + required: + - port + type: object + type: array + x-kubernetes-list-map-keys: + - port + - protocol + x-kubernetes-list-type: map + publishNotReadyAddresses: + description: publishNotReadyAddresses indicates that any + agent which deals with endpoints for this Service should + type: boolean + selector: + additionalProperties: + type: string + description: Route service traffic to pods with label + keys and values matching this selector. + type: object + x-kubernetes-map-type: atomic + sessionAffinity: + description: Supports "ClientIP" and "None". Used to maintain + session affinity. + type: string + sessionAffinityConfig: + description: sessionAffinityConfig contains the configurations + of session affinity. + properties: + clientIP: + description: clientIP contains the configurations + of Client IP based session affinity. + properties: + timeoutSeconds: + description: timeoutSeconds specifies the seconds + of ClientIP type session sticky time. + format: int32 + type: integer + type: object + type: object + type: + description: type determines how the Service is exposed. + Defaults to ClusterIP. + type: string + type: object + status: + description: Most recently observed status of the service. + Populated by the system. Read-only. + properties: + conditions: + description: Current service state + items: + description: Condition contains details for one aspect + of the current state of this API Resource. + properties: + lastTransitionTime: + description: lastTransitionTime is the last time + the condition transitioned from one status to + another. + format: date-time + type: string + message: + description: message is a human readable message + indicating details about the transition. + maxLength: 32768 + type: string + observedGeneration: + description: observedGeneration represents the .metadata.generation + that the condition was set based upon. + format: int64 + minimum: 0 + type: integer + reason: + description: reason contains a programmatic identifier + indicating the reason for the condition's last + transition. + maxLength: 1024 + minLength: 1 + pattern: ^[A-Za-z]([A-Za-z0-9_,:]*[A-Za-z0-9_])?$ + type: string + status: + description: status of the condition, one of True, + False, Unknown. + enum: + - "True" + - "False" + - Unknown + type: string + type: + description: type of condition in CamelCase or in + foo.example.com/CamelCase. --- Many .condition. + maxLength: 316 + pattern: ^([a-z0-9]([-a-z0-9]*[a-z0-9])?(\.[a-z0-9]([-a-z0-9]*[a-z0-9])?)*/)?(([A-Za-z0-9][-A-Za-z0-9_.]*)?[A-Za-z0-9])$ + type: string + required: + - lastTransitionTime + - message + - reason + - status + - type + type: object + type: array + x-kubernetes-list-map-keys: + - type + x-kubernetes-list-type: map + loadBalancer: + description: LoadBalancer contains the current status + of the load-balancer, if one is present. + properties: + ingress: + description: Ingress is a list containing ingress + points for the load-balancer. + items: + description: 'LoadBalancerIngress represents the + status of a load-balancer ingress point: traffic + intended for the' + properties: + hostname: + description: Hostname is set for load-balancer + ingress points that are DNS based (typically + AWS load-balancers) + type: string + ip: + description: IP is set for load-balancer ingress + points that are IP based (typically GCE or + OpenStack load-balanc + type: string + ports: + description: Ports is a list of records of service + ports If used, every port defined in the service + should have a + items: + properties: + error: + description: Error is to record the problem + with the service port The format of + the error shall comply with the f + maxLength: 316 + pattern: ^([a-z0-9]([-a-z0-9]*[a-z0-9])?(\.[a-z0-9]([-a-z0-9]*[a-z0-9])?)*/)?(([A-Za-z0-9][-A-Za-z0-9_.]*)?[A-Za-z0-9])$ + type: string + port: + description: Port is the port number of + the service port of which status is + recorded here + format: int32 + type: integer + protocol: + default: TCP + description: Protocol is the protocol + of the service port of which status + is recorded here The supported values + a + type: string + required: + - port + - protocol + type: object + type: array + x-kubernetes-list-type: atomic + type: object + type: array + type: object + type: object + type: object + rayStartParams: + additionalProperties: + type: string + description: 'RayStartParams are the params of the start command: + node-manager-port, object-store-memory, ...' + type: object + replicas: + description: HeadGroupSpec.Replicas is deprecated and ignored; + there can only be one head pod per Ray cluster. + 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 + - template + type: object + headServiceAnnotations: + additionalProperties: + type: string + type: object + rayVersion: + description: RayVersion is the version of ray being used. This determines + the autoscaler's image version. + 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 is 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 + status: + 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 + endpoints: + additionalProperties: + type: string + description: Service Endpoints + type: object + head: + description: Head info + properties: + podIP: + type: string + serviceIP: + type: string + type: object + 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 + observedGeneration: + description: observedGeneration is the most recent generation observed + for this RayCluster. + format: int64 + type: integer + reason: + description: Reason provides more information about current State + type: string + state: + description: 'INSERT ADDITIONAL STATUS FIELD - define observed state + of cluster Important: Run "make" to regenerat' + type: string + type: object + type: object + served: true + storage: false + subresources: + status: {} status: acceptedNames: kind: "" diff --git a/ray-operator/config/crd/bases/ray.io_rayservices.yaml b/ray-operator/config/crd/bases/ray.io_rayservices.yaml index 0317f5e882..ece24bffa7 100644 --- a/ray-operator/config/crd/bases/ray.io_rayservices.yaml +++ b/ray-operator/config/crd/bases/ray.io_rayservices.yaml @@ -38,8 +38,7 @@ spec: format: int32 type: integer rayClusterConfig: - description: 'EDIT THIS FILE! THIS IS SCAFFOLDING FOR YOU TO OWN! - NOTE: json tags are required.' + description: RayClusterSpec defines the desired state of RayCluster properties: autoscalerOptions: description: AutoscalerOptions specifies optional configuration @@ -12395,6 +12394,12384 @@ spec: storage: true subresources: status: {} + - name: v1beta1 + schema: + openAPIV3Schema: + description: RayService is the Schema for the rayservices API + properties: + apiVersion: + description: APIVersion defines the versioned schema of this representation + of an object. + type: string + kind: + description: Kind is a string value representing the REST resource this + object represents. + type: string + metadata: + type: object + spec: + description: RayServiceSpec defines the desired state of RayService + properties: + deploymentUnhealthySecondThreshold: + format: int32 + type: integer + rayClusterConfig: + description: RayClusterSpec defines the desired state of RayCluster + properties: + autoscalerOptions: + description: AutoscalerOptions specifies optional configuration + for the Ray autoscaler. + properties: + env: + description: Optional list of environment variables to set + in the autoscaler container. + 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: Optional list of sources to populate environment + variables in the autoscaler 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 + 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 + imagePullPolicy: + description: ImagePullPolicy optionally overrides the autoscaler + container's image pull policy. + type: string + resources: + description: Resources specifies optional resource request + and limit overrides 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 + 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 + upscalingMode: + description: UpscalingMode is "Conservative", "Default", or + "Aggressive. + enum: + - Default + - Aggressive + - Conservative + type: string + volumeMounts: + description: Optional list of volumeMounts. This is needed + for enabling TLS for the autoscaler container. + 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 + 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 + headService: + description: HeadService is the Kubernetes service of the + head pod. + properties: + apiVersion: + description: APIVersion defines the versioned schema of + this representation of an object. + type: string + kind: + description: Kind is a string value representing the REST + resource this object represents. + type: string + 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: Spec defines the behavior of a service. https://git.k8s. + properties: + allocateLoadBalancerNodePorts: + description: allocateLoadBalancerNodePorts defines + if NodePorts will be automatically allocated for + services with + type: boolean + clusterIP: + description: clusterIP is the IP address of the service + and is usually assigned randomly. + type: string + clusterIPs: + description: ClusterIPs is a list of IP addresses + assigned to this service, and are usually assigned + randomly. + items: + type: string + type: array + x-kubernetes-list-type: atomic + externalIPs: + description: externalIPs is a list of IP addresses + for which nodes in the cluster will also accept + traffic for th + items: + type: string + type: array + externalName: + description: externalName is the external reference + that discovery mechanisms will return as an alias + for this se + type: string + externalTrafficPolicy: + description: externalTrafficPolicy denotes if this + Service desires to route external traffic to node-local + or clu + type: string + healthCheckNodePort: + description: healthCheckNodePort specifies the healthcheck + nodePort for the service. + format: int32 + type: integer + internalTrafficPolicy: + description: InternalTrafficPolicy specifies if the + cluster internal traffic should be routed to all + endpoints or + type: string + ipFamilies: + description: IPFamilies is a list of IP families (e.g. + IPv4, IPv6) assigned to this service. + items: + description: IPFamily represents the IP Family (IPv4 + or IPv6). + type: string + type: array + x-kubernetes-list-type: atomic + ipFamilyPolicy: + description: IPFamilyPolicy represents the dual-stack-ness + requested or required by this Service. + type: string + loadBalancerClass: + description: loadBalancerClass is the class of the + load balancer implementation this Service belongs + to. + type: string + loadBalancerIP: + description: 'Only applies to Service Type: LoadBalancer + LoadBalancer will get created with the IP specified + in th' + type: string + loadBalancerSourceRanges: + description: If specified and supported by the platform, + this will restrict traffic through the cloud-provider + lo + items: + type: string + type: array + ports: + description: 'The list of ports that are exposed by + this service. More info: https://kubernetes.' + items: + description: ServicePort contains information on + service's port. + properties: + appProtocol: + description: The application protocol for this + port. This field follows standard Kubernetes + label syntax. + type: string + name: + description: The name of this port within the + service. This must be a DNS_LABEL. + type: string + nodePort: + description: The port on each node on which + this service is exposed when type is NodePort + or LoadBalancer. + format: int32 + type: integer + port: + description: The port that will be exposed by + this service. + format: int32 + type: integer + protocol: + default: TCP + description: The IP protocol for this port. + Supports "TCP", "UDP", and "SCTP". Default + is TCP. + type: string + targetPort: + anyOf: + - type: integer + - type: string + description: Number or name of the port to access + on the pods targeted by the service. + x-kubernetes-int-or-string: true + required: + - port + type: object + type: array + x-kubernetes-list-map-keys: + - port + - protocol + x-kubernetes-list-type: map + publishNotReadyAddresses: + description: publishNotReadyAddresses indicates that + any agent which deals with endpoints for this Service + should + type: boolean + selector: + additionalProperties: + type: string + description: Route service traffic to pods with label + keys and values matching this selector. + type: object + x-kubernetes-map-type: atomic + sessionAffinity: + description: Supports "ClientIP" and "None". Used + to maintain session affinity. + type: string + sessionAffinityConfig: + description: sessionAffinityConfig contains the configurations + of session affinity. + properties: + clientIP: + description: clientIP contains the configurations + of Client IP based session affinity. + properties: + timeoutSeconds: + description: timeoutSeconds specifies the + seconds of ClientIP type session sticky + time. + format: int32 + type: integer + type: object + type: object + type: + description: type determines how the Service is exposed. + Defaults to ClusterIP. + type: string + type: object + status: + description: Most recently observed status of the service. + Populated by the system. Read-only. + properties: + conditions: + description: Current service state + items: + description: Condition contains details for one + aspect of the current state of this API Resource. + properties: + lastTransitionTime: + description: lastTransitionTime is the last + time the condition transitioned from one status + to another. + format: date-time + type: string + message: + description: message is a human readable message + indicating details about the transition. + maxLength: 32768 + type: string + observedGeneration: + description: observedGeneration represents the + .metadata.generation that the condition was + set based upon. + format: int64 + minimum: 0 + type: integer + reason: + description: reason contains a programmatic + identifier indicating the reason for the condition's + last transition. + maxLength: 1024 + minLength: 1 + pattern: ^[A-Za-z]([A-Za-z0-9_,:]*[A-Za-z0-9_])?$ + type: string + status: + description: status of the condition, one of + True, False, Unknown. + enum: + - "True" + - "False" + - Unknown + type: string + type: + description: type of condition in CamelCase + or in foo.example.com/CamelCase. --- Many + .condition. + maxLength: 316 + pattern: ^([a-z0-9]([-a-z0-9]*[a-z0-9])?(\.[a-z0-9]([-a-z0-9]*[a-z0-9])?)*/)?(([A-Za-z0-9][-A-Za-z0-9_.]*)?[A-Za-z0-9])$ + type: string + required: + - lastTransitionTime + - message + - reason + - status + - type + type: object + type: array + x-kubernetes-list-map-keys: + - type + x-kubernetes-list-type: map + loadBalancer: + description: LoadBalancer contains the current status + of the load-balancer, if one is present. + properties: + ingress: + description: Ingress is a list containing ingress + points for the load-balancer. + items: + description: 'LoadBalancerIngress represents + the status of a load-balancer ingress point: + traffic intended for the' + properties: + hostname: + description: Hostname is set for load-balancer + ingress points that are DNS based (typically + AWS load-balancers) + type: string + ip: + description: IP is set for load-balancer + ingress points that are IP based (typically + GCE or OpenStack load-balanc + type: string + ports: + description: Ports is a list of records + of service ports If used, every port defined + in the service should have a + items: + properties: + error: + description: Error is to record the + problem with the service port The + format of the error shall comply + with the f + maxLength: 316 + pattern: ^([a-z0-9]([-a-z0-9]*[a-z0-9])?(\.[a-z0-9]([-a-z0-9]*[a-z0-9])?)*/)?(([A-Za-z0-9][-A-Za-z0-9_.]*)?[A-Za-z0-9])$ + type: string + port: + description: Port is the port number + of the service port of which status + is recorded here + format: int32 + type: integer + protocol: + default: TCP + description: Protocol is the protocol + of the service port of which status + is recorded here The supported values + a + type: string + required: + - port + - protocol + type: object + type: array + x-kubernetes-list-type: atomic + type: object + type: array + type: object + type: object + type: object + rayStartParams: + additionalProperties: + type: string + description: 'RayStartParams are the params of the start command: + node-manager-port, object-store-memory, ...' + type: object + replicas: + description: HeadGroupSpec.Replicas is deprecated and ignored; + there can only be one head pod per Ray cluster. + 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 + - template + type: object + headServiceAnnotations: + additionalProperties: + type: string + type: object + rayVersion: + description: RayVersion is the version of ray being used. This + determines the autoscaler's image version. + 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 is 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 + serveConfig: + description: 'Important: Run "make" to regenerate code after modifying + this file' + properties: + deployments: + items: + description: ServeConfigSpec defines the desired state of RayService + Reference to http://rayserve.org + properties: + autoscalingConfig: + type: string + gracefulShutdownTimeoutS: + format: int32 + type: integer + gracefulShutdownWaitLoopS: + format: int32 + type: integer + healthCheckPeriodS: + format: int32 + type: integer + healthCheckTimeoutS: + format: int32 + type: integer + maxConcurrentQueries: + format: int32 + type: integer + name: + type: string + numReplicas: + format: int32 + type: integer + rayActorOptions: + description: RayActorOptionSpec defines the desired state + of RayActor + properties: + acceleratorType: + type: string + memory: + format: int32 + type: integer + numCpus: + type: number + numGpus: + type: number + objectStoreMemory: + format: int32 + type: integer + resources: + type: string + runtimeEnv: + type: string + type: object + routePrefix: + type: string + userConfig: + type: string + required: + - name + type: object + type: array + importPath: + type: string + port: + type: integer + runtimeEnv: + type: string + required: + - importPath + type: object + serviceUnhealthySecondThreshold: + format: int32 + type: integer + type: object + status: + description: RayServiceStatuses defines the observed state of RayService + properties: + activeServiceStatus: + properties: + appStatus: + description: 'Important: Run "make" to regenerate code after modifying + this file' + 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 + status: + type: string + type: object + dashboardStatus: + description: DashboardStatus defines the current states of Ray + Dashboard + properties: + healthLastUpdateTime: + description: Keep track of how long the dashboard is healthy. + format: date-time + type: string + isHealthy: + type: boolean + lastUpdateTime: + format: date-time + type: string + type: object + rayClusterName: + type: string + 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 + endpoints: + additionalProperties: + type: string + description: Service Endpoints + type: object + head: + description: Head info + properties: + podIP: + type: string + serviceIP: + type: string + type: object + 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 + observedGeneration: + description: observedGeneration is the most recent generation + observed for this RayCluster. + format: int64 + type: integer + reason: + description: Reason provides more information about current + State + type: string + state: + description: 'INSERT ADDITIONAL STATUS FIELD - define observed + state of cluster Important: Run "make" to regenerat' + type: string + type: object + serveDeploymentStatuses: + items: + description: ServeDeploymentStatus defines the current state + of a 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 + and represent a Serve deployment's state. + type: string + status: + description: 'TODO: change status type to enum' + type: string + type: object + type: array + type: object + observedGeneration: + description: observedGeneration is the most recent generation observed + for this RayService. + format: int64 + type: integer + pendingServiceStatus: + description: Pending Service Status indicates a RayCluster will be + created or is being created. + properties: + appStatus: + description: 'Important: Run "make" to regenerate code after modifying + this file' + 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 + status: + type: string + type: object + dashboardStatus: + description: DashboardStatus defines the current states of Ray + Dashboard + properties: + healthLastUpdateTime: + description: Keep track of how long the dashboard is healthy. + format: date-time + type: string + isHealthy: + type: boolean + lastUpdateTime: + format: date-time + type: string + type: object + rayClusterName: + type: string + 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 + endpoints: + additionalProperties: + type: string + description: Service Endpoints + type: object + head: + description: Head info + properties: + podIP: + type: string + serviceIP: + type: string + type: object + 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 + observedGeneration: + description: observedGeneration is the most recent generation + observed for this RayCluster. + format: int64 + type: integer + reason: + description: Reason provides more information about current + State + type: string + state: + description: 'INSERT ADDITIONAL STATUS FIELD - define observed + state of cluster Important: Run "make" to regenerat' + type: string + type: object + serveDeploymentStatuses: + items: + description: ServeDeploymentStatus defines the current state + of a 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 + and represent a Serve deployment's state. + type: string + status: + description: 'TODO: change status type to enum' + type: string + type: object + type: array + type: object + serviceStatus: + description: ServiceStatus indicates the current RayService status. + type: string + type: object + type: object + served: true + storage: false + subresources: + status: {} status: acceptedNames: kind: "" diff --git a/ray-operator/config/crd/patches/cainjection_in_ray_rayclusters.yaml b/ray-operator/config/crd/patches/cainjection_in_ray_rayclusters.yaml new file mode 100644 index 0000000000..7d4de64ad2 --- /dev/null +++ b/ray-operator/config/crd/patches/cainjection_in_ray_rayclusters.yaml @@ -0,0 +1,7 @@ +# The following patch adds a directive for certmanager to inject CA into the CRD +apiVersion: apiextensions.k8s.io/v1 +kind: CustomResourceDefinition +metadata: + annotations: + cert-manager.io/inject-ca-from: $(CERTIFICATE_NAMESPACE)/$(CERTIFICATE_NAME) + name: rayclusters.ray.io diff --git a/ray-operator/config/crd/patches/cainjection_in_ray_rayservices.yaml b/ray-operator/config/crd/patches/cainjection_in_ray_rayservices.yaml new file mode 100644 index 0000000000..d46ece7891 --- /dev/null +++ b/ray-operator/config/crd/patches/cainjection_in_ray_rayservices.yaml @@ -0,0 +1,7 @@ +# The following patch adds a directive for certmanager to inject CA into the CRD +apiVersion: apiextensions.k8s.io/v1 +kind: CustomResourceDefinition +metadata: + annotations: + cert-manager.io/inject-ca-from: $(CERTIFICATE_NAMESPACE)/$(CERTIFICATE_NAME) + name: rayservices.ray.io diff --git a/ray-operator/config/crd/patches/webhook_in_ray_rayclusters.yaml b/ray-operator/config/crd/patches/webhook_in_ray_rayclusters.yaml new file mode 100644 index 0000000000..52abb36fed --- /dev/null +++ b/ray-operator/config/crd/patches/webhook_in_ray_rayclusters.yaml @@ -0,0 +1,16 @@ +# The following patch enables a conversion webhook for the CRD +apiVersion: apiextensions.k8s.io/v1 +kind: CustomResourceDefinition +metadata: + name: rayclusters.ray.io +spec: + conversion: + strategy: Webhook + webhook: + clientConfig: + service: + namespace: system + name: webhook-service + path: /convert + conversionReviewVersions: + - v1 diff --git a/ray-operator/config/crd/patches/webhook_in_ray_rayservices.yaml b/ray-operator/config/crd/patches/webhook_in_ray_rayservices.yaml new file mode 100644 index 0000000000..16a33aa850 --- /dev/null +++ b/ray-operator/config/crd/patches/webhook_in_ray_rayservices.yaml @@ -0,0 +1,16 @@ +# The following patch enables a conversion webhook for the CRD +apiVersion: apiextensions.k8s.io/v1 +kind: CustomResourceDefinition +metadata: + name: rayservices.ray.io +spec: + conversion: + strategy: Webhook + webhook: + clientConfig: + service: + namespace: system + name: webhook-service + path: /convert + conversionReviewVersions: + - v1 diff --git a/ray-operator/config/default/manager_webhook_patch.yaml b/ray-operator/config/default/manager_webhook_patch.yaml new file mode 100644 index 0000000000..738de350b7 --- /dev/null +++ b/ray-operator/config/default/manager_webhook_patch.yaml @@ -0,0 +1,23 @@ +apiVersion: apps/v1 +kind: Deployment +metadata: + name: controller-manager + namespace: system +spec: + template: + spec: + containers: + - name: manager + ports: + - containerPort: 9443 + name: webhook-server + protocol: TCP + volumeMounts: + - mountPath: /tmp/k8s-webhook-server/serving-certs + name: cert + readOnly: true + volumes: + - name: cert + secret: + defaultMode: 420 + secretName: webhook-server-cert diff --git a/ray-operator/config/default/webhookcainjection_patch.yaml b/ray-operator/config/default/webhookcainjection_patch.yaml new file mode 100644 index 0000000000..89c3662802 --- /dev/null +++ b/ray-operator/config/default/webhookcainjection_patch.yaml @@ -0,0 +1,29 @@ +# This patch add annotation to admission webhook config and +# the variables $(CERTIFICATE_NAMESPACE) and $(CERTIFICATE_NAME) will be substituted by kustomize. +apiVersion: admissionregistration.k8s.io/v1 +kind: MutatingWebhookConfiguration +metadata: + labels: + app.kubernetes.io/name: mutatingwebhookconfiguration + app.kubernetes.io/instance: mutating-webhook-configuration + app.kubernetes.io/component: webhook + app.kubernetes.io/created-by: ray-operator + app.kubernetes.io/part-of: ray-operator + app.kubernetes.io/managed-by: kustomize + name: mutating-webhook-configuration + annotations: + cert-manager.io/inject-ca-from: $(CERTIFICATE_NAMESPACE)/$(CERTIFICATE_NAME) +--- +apiVersion: admissionregistration.k8s.io/v1 +kind: ValidatingWebhookConfiguration +metadata: + labels: + app.kubernetes.io/name: validatingwebhookconfiguration + app.kubernetes.io/instance: validating-webhook-configuration + app.kubernetes.io/component: webhook + app.kubernetes.io/created-by: ray-operator + app.kubernetes.io/part-of: ray-operator + app.kubernetes.io/managed-by: kustomize + name: validating-webhook-configuration + annotations: + cert-manager.io/inject-ca-from: $(CERTIFICATE_NAMESPACE)/$(CERTIFICATE_NAME) diff --git a/ray-operator/config/rbac/ray_raycluster_editor_role.yaml b/ray-operator/config/rbac/ray_raycluster_editor_role.yaml new file mode 100644 index 0000000000..44f0745032 --- /dev/null +++ b/ray-operator/config/rbac/ray_raycluster_editor_role.yaml @@ -0,0 +1,31 @@ +# permissions for end users to edit rayclusters. +apiVersion: rbac.authorization.k8s.io/v1 +kind: ClusterRole +metadata: + labels: + app.kubernetes.io/name: clusterrole + app.kubernetes.io/instance: raycluster-editor-role + app.kubernetes.io/component: rbac + app.kubernetes.io/created-by: ray-operator + app.kubernetes.io/part-of: ray-operator + app.kubernetes.io/managed-by: kustomize + name: raycluster-editor-role +rules: +- apiGroups: + - ray.io + resources: + - rayclusters + verbs: + - create + - delete + - get + - list + - patch + - update + - watch +- apiGroups: + - ray.io + resources: + - rayclusters/status + verbs: + - get diff --git a/ray-operator/config/rbac/ray_raycluster_viewer_role.yaml b/ray-operator/config/rbac/ray_raycluster_viewer_role.yaml new file mode 100644 index 0000000000..c62d08e80e --- /dev/null +++ b/ray-operator/config/rbac/ray_raycluster_viewer_role.yaml @@ -0,0 +1,27 @@ +# permissions for end users to view rayclusters. +apiVersion: rbac.authorization.k8s.io/v1 +kind: ClusterRole +metadata: + labels: + app.kubernetes.io/name: clusterrole + app.kubernetes.io/instance: raycluster-viewer-role + app.kubernetes.io/component: rbac + app.kubernetes.io/created-by: ray-operator + app.kubernetes.io/part-of: ray-operator + app.kubernetes.io/managed-by: kustomize + name: raycluster-viewer-role +rules: +- apiGroups: + - ray.io + resources: + - rayclusters + verbs: + - get + - list + - watch +- apiGroups: + - ray.io + resources: + - rayclusters/status + verbs: + - get diff --git a/ray-operator/config/webhook/kustomization.yaml b/ray-operator/config/webhook/kustomization.yaml new file mode 100644 index 0000000000..9cf26134e4 --- /dev/null +++ b/ray-operator/config/webhook/kustomization.yaml @@ -0,0 +1,6 @@ +resources: +- manifests.yaml +- service.yaml + +configurations: +- kustomizeconfig.yaml diff --git a/ray-operator/config/webhook/kustomizeconfig.yaml b/ray-operator/config/webhook/kustomizeconfig.yaml new file mode 100644 index 0000000000..25e21e3c96 --- /dev/null +++ b/ray-operator/config/webhook/kustomizeconfig.yaml @@ -0,0 +1,25 @@ +# the following config is for teaching kustomize where to look at when substituting vars. +# It requires kustomize v2.1.0 or newer to work properly. +nameReference: +- kind: Service + version: v1 + fieldSpecs: + - kind: MutatingWebhookConfiguration + group: admissionregistration.k8s.io + path: webhooks/clientConfig/service/name + - kind: ValidatingWebhookConfiguration + group: admissionregistration.k8s.io + path: webhooks/clientConfig/service/name + +namespace: +- kind: MutatingWebhookConfiguration + group: admissionregistration.k8s.io + path: webhooks/clientConfig/service/namespace + create: true +- kind: ValidatingWebhookConfiguration + group: admissionregistration.k8s.io + path: webhooks/clientConfig/service/namespace + create: true + +varReference: +- path: metadata/annotations diff --git a/ray-operator/config/webhook/service.yaml b/ray-operator/config/webhook/service.yaml new file mode 100644 index 0000000000..9e1489c71b --- /dev/null +++ b/ray-operator/config/webhook/service.yaml @@ -0,0 +1,20 @@ + +apiVersion: v1 +kind: Service +metadata: + labels: + app.kubernetes.io/name: service + app.kubernetes.io/instance: webhook-service + app.kubernetes.io/component: webhook + app.kubernetes.io/created-by: ray-operator + app.kubernetes.io/part-of: ray-operator + app.kubernetes.io/managed-by: kustomize + name: webhook-service + namespace: system +spec: + ports: + - port: 443 + protocol: TCP + targetPort: 9443 + selector: + control-plane: controller-manager diff --git a/ray-operator/go.mod b/ray-operator/go.mod index e4116b4c89..dea5ec84c9 100644 --- a/ray-operator/go.mod +++ b/ray-operator/go.mod @@ -16,7 +16,9 @@ require ( go.uber.org/zap v1.19.1 gopkg.in/natefinch/lumberjack.v2 v2.0.0 k8s.io/api v0.23.0 + k8s.io/apiextensions-apiserver v0.23.0 k8s.io/apimachinery v0.23.0 + k8s.io/apiserver v0.23.0 k8s.io/client-go v0.23.0 k8s.io/code-generator v0.23.0 k8s.io/utils v0.0.0-20210930125809-cb0fa318a74b @@ -75,8 +77,6 @@ require ( gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 // indirect gopkg.in/yaml.v2 v2.4.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect - k8s.io/apiextensions-apiserver v0.23.0 // indirect - k8s.io/apiserver v0.23.0 // indirect k8s.io/component-base v0.23.0 // indirect k8s.io/gengo v0.0.0-20210813121822-485abfe95c7c // indirect k8s.io/klog/v2 v2.30.0 // indirect diff --git a/ray-operator/go.sum b/ray-operator/go.sum index ee5027cfe9..788bcd4aba 100644 --- a/ray-operator/go.sum +++ b/ray-operator/go.sum @@ -46,6 +46,7 @@ github.com/Azure/go-autorest/autorest/date v0.3.0/go.mod h1:BI0uouVdmngYNUzGWeSY github.com/Azure/go-autorest/autorest/mocks v0.4.1/go.mod h1:LTp+uSrOhSkaKrUy935gNZuuIPPVsHlr9DSOxSayd+k= github.com/Azure/go-autorest/logger v0.2.1/go.mod h1:T9E3cAhj2VqvPOtCYAvby9aBXkZmbF5NWuPV8+WeEW8= github.com/Azure/go-autorest/tracing v0.6.0/go.mod h1:+vhtPC754Xsa23ID7GlGsrdKBpUA79WCAKPPZVC2DeU= +github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= github.com/NYTimes/gziphandler v0.0.0-20170623195520-56545f4a5d46/go.mod h1:3wb06e3pkSAbeQ52E9H9iFoQsEEwGN64994WTCIhntQ= @@ -390,7 +391,6 @@ github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXP github.com/prometheus/client_golang v0.9.3/go.mod h1:/TN21ttK/J9q6uSwhBd54HahCDft0ttaMvbicHlPoso= github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= github.com/prometheus/client_golang v1.7.1/go.mod h1:PY5Wy2awLA44sXw4AOSfFBetzPP4j5+D6mVACh+pe2M= -github.com/prometheus/client_golang v1.11.0 h1:HNkLOAEQMIDv/K+04rukrLx6ch7msSRwf3/SASFAGtQ= github.com/prometheus/client_golang v1.11.0/go.mod h1:Z6t4BnS23TR94PD6BsDNk8yVqroYurpAkEiz0P2BEV0= github.com/prometheus/client_golang v1.11.1 h1:+4eQaD7vAZ6DsfsxB15hbE0odUjGI5ARs9yskGu1v4s= github.com/prometheus/client_golang v1.11.1/go.mod h1:Z6t4BnS23TR94PD6BsDNk8yVqroYurpAkEiz0P2BEV0= diff --git a/ray-operator/main.go b/ray-operator/main.go index 4cd26994df..7ff7b5930b 100644 --- a/ray-operator/main.go +++ b/ray-operator/main.go @@ -22,6 +22,7 @@ import ( k8szap "sigs.k8s.io/controller-runtime/pkg/log/zap" rayv1alpha1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1alpha1" + rayv1beta1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1beta1" // +kubebuilder:scaffold:imports ) @@ -37,6 +38,7 @@ func init() { utilruntime.Must(clientgoscheme.AddToScheme(scheme)) utilruntime.Must(rayv1alpha1.AddToScheme(scheme)) batchscheduler.AddToScheme(scheme) + utilruntime.Must(rayv1beta1.AddToScheme(scheme)) // +kubebuilder:scaffold:scheme } @@ -143,6 +145,10 @@ func main() { setupLog.Error(err, "unable to create controller", "controller", "RayJob") os.Exit(1) } + if err = (&rayv1alpha1.RayService{}).SetupWebhookWithManager(mgr); err != nil { + setupLog.Error(err, "unable to create webhook", "webhook", "RayService") + os.Exit(1) + } // +kubebuilder:scaffold:builder if err := mgr.AddHealthzCheck("healthz", healthz.Ping); err != nil {