Documentation ¶
Overview ¶
+groupName=ray.io
Package v1alpha1 contains API Schema definitions for the ray v1alpha1 API group +kubebuilder:object:generate=true +groupName=ray.io
Index ¶
- Variables
- func IsJobTerminal(status JobStatus) bool
- func Resource(resource string) schema.GroupResource
- type AppStatus
- type AutoscalerOptions
- type ClusterState
- type EventReason
- type HeadGroupSpec
- type HeadInfo
- type JobDeploymentStatus
- type JobStatus
- type RayCluster
- type RayClusterList
- type RayClusterSpec
- type RayClusterStatus
- type RayJob
- type RayJobList
- type RayJobSpec
- type RayJobStatus
- type RayNodeType
- type RayService
- type RayServiceList
- type RayServiceSpec
- type RayServiceStatus
- type RayServiceStatuses
- type ScaleStrategy
- type ServeDeploymentStatus
- type ServiceStatus
- type UpscalingMode
- type WorkerGroupSpec
Constants ¶
This section is empty.
Variables ¶
var ( // GroupVersion is group version used to register these objects GroupVersion = schema.GroupVersion{Group: "ray.io", Version: "v1alpha1"} // 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 )
var ApplicationStatusEnum = struct { NOT_STARTED string DEPLOYING string RUNNING string DEPLOY_FAILED string DELETING string UNHEALTHY string }{ NOT_STARTED: "NOT_STARTED", DEPLOYING: "DEPLOYING", RUNNING: "RUNNING", DEPLOY_FAILED: "DEPLOY_FAILED", DELETING: "DELETING", UNHEALTHY: "UNHEALTHY", }
These statuses should match Ray Serve's application statuses See `enum ApplicationStatus` in https://sourcegraph.com/github.com/ray-project/ray/-/blob/src/ray/protobuf/serve.proto for more details.
var DeploymentStatusEnum = struct { UPDATING string HEALTHY string UNHEALTHY string }{ UPDATING: "UPDATING", HEALTHY: "HEALTHY", UNHEALTHY: "UNHEALTHY", }
These statuses should match Ray Serve's deployment statuses
var SchemeGroupVersion = GroupVersion
SchemeGroupVersion is group version used to register these objects.
Functions ¶
func IsJobTerminal ¶
This function should be synchronized with the function `is_terminal()` in Ray Job.
func Resource ¶
func Resource(resource string) schema.GroupResource
Types ¶
type AppStatus ¶
type AppStatus struct { // 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"` Deployments map[string]ServeDeploymentStatus `json:"serveDeploymentStatuses,omitempty"` Status string `json:"status,omitempty"` Message string `json:"message,omitempty"` }
func (*AppStatus) DeepCopy ¶
DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new AppStatus.
func (*AppStatus) DeepCopyInto ¶
DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil.
type AutoscalerOptions ¶
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 *corev1.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 *corev1.PullPolicy `json:"imagePullPolicy,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 *corev1.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). It is not read by the KubeRay operator but by the Ray autoscaler. 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. // It is not read by the KubeRay operator but by the Ray autoscaler. UpscalingMode *UpscalingMode `json:"upscalingMode,omitempty"` // Optional list of environment variables to set in the autoscaler container. Env []corev1.EnvVar `json:"env,omitempty"` // Optional list of sources to populate environment variables in the autoscaler container. EnvFrom []corev1.EnvFromSource `json:"envFrom,omitempty"` // Optional list of volumeMounts. This is needed for enabling TLS for the autoscaler container. VolumeMounts []corev1.VolumeMount `json:"volumeMounts,omitempty"` }
AutoscalerOptions specifies optional configuration for the Ray autoscaler.
func (*AutoscalerOptions) DeepCopy ¶
func (in *AutoscalerOptions) DeepCopy() *AutoscalerOptions
DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new AutoscalerOptions.
func (*AutoscalerOptions) DeepCopyInto ¶
func (in *AutoscalerOptions) DeepCopyInto(out *AutoscalerOptions)
DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil.
type ClusterState ¶
type ClusterState string
The overall state of the Ray cluster.
const ( Ready ClusterState = "ready" Unhealthy ClusterState = "unhealthy" Failed ClusterState = "failed" Suspended ClusterState = "suspended" )
type EventReason ¶
type EventReason string
const ( RayConfigError EventReason = "RayConfigError" PodReconciliationError EventReason = "PodReconciliationError" )
type HeadGroupSpec ¶
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 corev1.ServiceType `json:"serviceType,omitempty"` // HeadService is the Kubernetes service of the head pod. HeadService *corev1.Service `json:"headService,omitempty"` // EnableIngress indicates whether operator should create ingress object for head service or not. EnableIngress *bool `json:"enableIngress,omitempty"` // RayStartParams are the params of the start command: node-manager-port, object-store-memory, ... RayStartParams map[string]string `json:"rayStartParams"` // Template is the exact pod template used in K8s depoyments, statefulsets, etc. Template corev1.PodTemplateSpec `json:"template"` }
HeadGroupSpec are the spec for the head pod
func (*HeadGroupSpec) DeepCopy ¶
func (in *HeadGroupSpec) DeepCopy() *HeadGroupSpec
DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new HeadGroupSpec.
func (*HeadGroupSpec) DeepCopyInto ¶
func (in *HeadGroupSpec) DeepCopyInto(out *HeadGroupSpec)
DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil.
type HeadInfo ¶
type HeadInfo struct { PodIP string `json:"podIP,omitempty"` ServiceIP string `json:"serviceIP,omitempty"` }
HeadInfo gives info about head
func (*HeadInfo) DeepCopy ¶
DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new HeadInfo.
func (*HeadInfo) DeepCopyInto ¶
DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil.
type JobDeploymentStatus ¶
type JobDeploymentStatus string
JobDeploymentStatus indicates RayJob status including RayCluster lifecycle management and Job submission
const ( JobDeploymentStatusNew JobDeploymentStatus = "" JobDeploymentStatusInitializing JobDeploymentStatus = "Initializing" JobDeploymentStatusRunning JobDeploymentStatus = "Running" JobDeploymentStatusComplete JobDeploymentStatus = "Complete" JobDeploymentStatusSuspended JobDeploymentStatus = "Suspended" )
type JobStatus ¶
type JobStatus string
JobStatus is the Ray Job Status.
type RayCluster ¶
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"` }
RayCluster is the Schema for the RayClusters API +kubebuilder:object:root=true +kubebuilder:resource:categories=all +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="cpus",type=string,JSONPath=".status.desiredCPU",priority=0 +kubebuilder:printcolumn:name="memory",type=string,JSONPath=".status.desiredMemory",priority=0 +kubebuilder:printcolumn:name="gpus",type=string,JSONPath=".status.desiredGPU",priority=0 +kubebuilder:printcolumn:name="tpus",type=string,JSONPath=".status.desiredTPU",priority=1 +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
func (*RayCluster) DeepCopy ¶
func (in *RayCluster) DeepCopy() *RayCluster
DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new RayCluster.
func (*RayCluster) DeepCopyInto ¶
func (in *RayCluster) DeepCopyInto(out *RayCluster)
DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil.
func (*RayCluster) DeepCopyObject ¶
func (in *RayCluster) DeepCopyObject() runtime.Object
DeepCopyObject is an autogenerated deepcopy function, copying the receiver, creating a new runtime.Object.
type RayClusterList ¶
type RayClusterList struct { metav1.TypeMeta `json:",inline"` metav1.ListMeta `json:"metadata,omitempty"` Items []RayCluster `json:"items"` }
RayClusterList contains a list of RayCluster
func (*RayClusterList) DeepCopy ¶
func (in *RayClusterList) DeepCopy() *RayClusterList
DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new RayClusterList.
func (*RayClusterList) DeepCopyInto ¶
func (in *RayClusterList) DeepCopyInto(out *RayClusterList)
DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil.
func (*RayClusterList) DeepCopyObject ¶
func (in *RayClusterList) DeepCopyObject() runtime.Object
DeepCopyObject is an autogenerated deepcopy function, copying the receiver, creating a new runtime.Object.
type RayClusterSpec ¶
type RayClusterSpec struct { // 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"` // Suspend indicates whether a RayCluster should be suspended. // A suspended RayCluster will have head pods and worker pods deleted. Suspend *bool `json:"suspend,omitempty"` HeadServiceAnnotations map[string]string `json:"headServiceAnnotations,omitempty"` // 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"` // RayVersion is used to determine the command for the Kubernetes Job managed by RayJob RayVersion string `json:"rayVersion,omitempty"` // WorkerGroupSpecs are the specs for the worker pods WorkerGroupSpecs []WorkerGroupSpec `json:"workerGroupSpecs,omitempty"` }
RayClusterSpec defines the desired state of RayCluster
func (*RayClusterSpec) DeepCopy ¶
func (in *RayClusterSpec) DeepCopy() *RayClusterSpec
DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new RayClusterSpec.
func (*RayClusterSpec) DeepCopyInto ¶
func (in *RayClusterSpec) DeepCopyInto(out *RayClusterSpec)
DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil.
type RayClusterStatus ¶
type RayClusterStatus struct { // 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"` // DesiredCPU indicates total desired CPUs for the cluster DesiredCPU resource.Quantity `json:"desiredCPU,omitempty"` // DesiredMemory indicates total desired memory for the cluster DesiredMemory resource.Quantity `json:"desiredMemory,omitempty"` // DesiredGPU indicates total desired GPUs for the cluster DesiredGPU resource.Quantity `json:"desiredGPU,omitempty"` // DesiredTPU indicates total desired TPUs for the cluster DesiredTPU resource.Quantity `json:"desiredTPU,omitempty"` // Head info Head HeadInfo `json:"head,omitempty"` // 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 // // Deprecated: the State field is replaced by the Conditions field. State ClusterState `json:"state,omitempty"` // Reason provides more information about current State Reason string `json:"reason,omitempty"` // ReadyWorkerReplicas indicates how many worker replicas are ready in the cluster ReadyWorkerReplicas int32 `json:"readyWorkerReplicas,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"` // 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. ObservedGeneration int64 `json:"observedGeneration,omitempty"` }
RayClusterStatus defines the observed state of RayCluster
func (*RayClusterStatus) DeepCopy ¶
func (in *RayClusterStatus) DeepCopy() *RayClusterStatus
DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new RayClusterStatus.
func (*RayClusterStatus) DeepCopyInto ¶
func (in *RayClusterStatus) DeepCopyInto(out *RayClusterStatus)
DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil.
type RayJob ¶
type RayJob struct { metav1.TypeMeta `json:",inline"` metav1.ObjectMeta `json:"metadata,omitempty"` Spec RayJobSpec `json:"spec,omitempty"` Status RayJobStatus `json:"status,omitempty"` }
+kubebuilder:object:root=true +kubebuilder:resource:categories=all +kubebuilder:subresource:status +genclient RayJob is the Schema for the rayjobs API
func (*RayJob) DeepCopy ¶
DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new RayJob.
func (*RayJob) DeepCopyInto ¶
DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil.
func (*RayJob) DeepCopyObject ¶
DeepCopyObject is an autogenerated deepcopy function, copying the receiver, creating a new runtime.Object.
type RayJobList ¶
type RayJobList struct { metav1.TypeMeta `json:",inline"` metav1.ListMeta `json:"metadata,omitempty"` Items []RayJob `json:"items"` }
RayJobList contains a list of RayJob
func (*RayJobList) DeepCopy ¶
func (in *RayJobList) DeepCopy() *RayJobList
DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new RayJobList.
func (*RayJobList) DeepCopyInto ¶
func (in *RayJobList) DeepCopyInto(out *RayJobList)
DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil.
func (*RayJobList) DeepCopyObject ¶
func (in *RayJobList) DeepCopyObject() runtime.Object
DeepCopyObject is an autogenerated deepcopy function, copying the receiver, creating a new runtime.Object.
type RayJobSpec ¶
type RayJobSpec struct { // SubmitterPodTemplate is the template for the pod that will run `ray job submit`. SubmitterPodTemplate *corev1.PodTemplateSpec `json:"submitterPodTemplate,omitempty"` // Metadata is data to store along with this job. Metadata map[string]string `json:"metadata,omitempty"` // RayClusterSpec is the cluster template to run the job RayClusterSpec *RayClusterSpec `json:"rayClusterSpec,omitempty"` // ClusterSelector is used to select running rayclusters by labels ClusterSelector map[string]string `json:"clusterSelector,omitempty"` // INSERT ADDITIONAL SPEC FIELDS - desired state of cluster // Important: Run "make" to regenerate code after modifying this file Entrypoint string `json:"entrypoint"` // RuntimeEnvYAML represents the runtime environment configuration // provided as a multi-line YAML string. RuntimeEnvYAML string `json:"runtimeEnvYAML,omitempty"` // If jobId is not set, a new jobId will be auto-generated. JobId string `json:"jobId,omitempty"` // EntrypointResources specifies the custom resources and quantities to reserve for the // entrypoint command. EntrypointResources string `json:"entrypointResources,omitempty"` // TTLSecondsAfterFinished is the TTL to clean up RayCluster. // It's only working when ShutdownAfterJobFinishes set to true. // +kubebuilder:default:=0 TTLSecondsAfterFinished int32 `json:"ttlSecondsAfterFinished,omitempty"` // EntrypointNumCpus specifies the number of cpus to reserve for the entrypoint command. EntrypointNumCpus float32 `json:"entrypointNumCpus,omitempty"` // EntrypointNumGpus specifies the number of gpus to reserve for the entrypoint command. EntrypointNumGpus float32 `json:"entrypointNumGpus,omitempty"` // ShutdownAfterJobFinishes will determine whether to delete the ray cluster once rayJob succeed or failed. ShutdownAfterJobFinishes bool `json:"shutdownAfterJobFinishes,omitempty"` // Suspend specifies whether the RayJob controller should create a RayCluster instance // If a job is applied with the suspend field set to true, // the RayCluster will not be created and will wait for the transition to false. // If the RayCluster is already created, it will be deleted. // In case of transition to false a new RayCluster will be created. Suspend bool `json:"suspend,omitempty"` }
RayJobSpec defines the desired state of RayJob
func (*RayJobSpec) DeepCopy ¶
func (in *RayJobSpec) DeepCopy() *RayJobSpec
DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new RayJobSpec.
func (*RayJobSpec) DeepCopyInto ¶
func (in *RayJobSpec) DeepCopyInto(out *RayJobSpec)
DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil.
type RayJobStatus ¶
type RayJobStatus struct { // INSERT ADDITIONAL STATUS FIELD - define observed state of cluster // Important: Run "make" to regenerate code after modifying this file JobId string `json:"jobId,omitempty"` RayClusterName string `json:"rayClusterName,omitempty"` DashboardURL string `json:"dashboardURL,omitempty"` JobStatus JobStatus `json:"jobStatus,omitempty"` JobDeploymentStatus JobDeploymentStatus `json:"jobDeploymentStatus,omitempty"` Message string `json:"message,omitempty"` // Represents time when the job was acknowledged by the Ray cluster. // It is not guaranteed to be set in happens-before order across separate operations. // It is represented in RFC3339 form StartTime *metav1.Time `json:"startTime,omitempty"` // Represents time when the job was ended. EndTime *metav1.Time `json:"endTime,omitempty"` RayClusterStatus RayClusterStatus `json:"rayClusterStatus,omitempty"` // observedGeneration is the most recent generation observed for this RayJob. It corresponds to the // RayJob's generation, which is updated on mutation by the API Server. ObservedGeneration int64 `json:"observedGeneration,omitempty"` }
RayJobStatus defines the observed state of RayJob
func (*RayJobStatus) DeepCopy ¶
func (in *RayJobStatus) DeepCopy() *RayJobStatus
DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new RayJobStatus.
func (*RayJobStatus) DeepCopyInto ¶
func (in *RayJobStatus) DeepCopyInto(out *RayJobStatus)
DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil.
type RayNodeType ¶
type RayNodeType string
RayNodeType the type of a ray node: head/worker
const ( HeadNode RayNodeType = "head" WorkerNode RayNodeType = "worker" // RedisCleanupNode is a Pod managed by a Kubernetes Job that cleans up Redis data after // a RayCluster with GCS fault tolerance enabled is deleted. RedisCleanupNode RayNodeType = "redis-cleanup" )
type RayService ¶
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 +kubebuilder:resource:categories=all +kubebuilder:subresource:status +genclient RayService is the Schema for the rayservices API
func (*RayService) DeepCopy ¶
func (in *RayService) DeepCopy() *RayService
DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new RayService.
func (*RayService) DeepCopyInto ¶
func (in *RayService) DeepCopyInto(out *RayService)
DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil.
func (*RayService) DeepCopyObject ¶
func (in *RayService) DeepCopyObject() runtime.Object
DeepCopyObject is an autogenerated deepcopy function, copying the receiver, creating a new runtime.Object.
type RayServiceList ¶
type RayServiceList struct { metav1.TypeMeta `json:",inline"` metav1.ListMeta `json:"metadata,omitempty"` Items []RayService `json:"items"` }
RayServiceList contains a list of RayService
func (*RayServiceList) DeepCopy ¶
func (in *RayServiceList) DeepCopy() *RayServiceList
DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new RayServiceList.
func (*RayServiceList) DeepCopyInto ¶
func (in *RayServiceList) DeepCopyInto(out *RayServiceList)
DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil.
func (*RayServiceList) DeepCopyObject ¶
func (in *RayServiceList) DeepCopyObject() runtime.Object
DeepCopyObject is an autogenerated deepcopy function, copying the receiver, creating a new runtime.Object.
type RayServiceSpec ¶
type RayServiceSpec struct { // ServeService is the Kubernetes service for head node and worker nodes who have healthy http proxy to serve traffics. ServeService *corev1.Service `json:"serveService,omitempty"` // Deprecated: This field is not used anymore. ref: https://github.com/ray-project/kuberay/issues/1685 ServiceUnhealthySecondThreshold *int32 `json:"serviceUnhealthySecondThreshold,omitempty"` // Deprecated: This field is not used anymore. ref: https://github.com/ray-project/kuberay/issues/1685 DeploymentUnhealthySecondThreshold *int32 `json:"deploymentUnhealthySecondThreshold,omitempty"` // Important: Run "make" to regenerate code after modifying this file // Defines the applications and deployments to deploy, should be a YAML multi-line scalar string. ServeConfigV2 string `json:"serveConfigV2,omitempty"` RayClusterSpec RayClusterSpec `json:"rayClusterConfig,omitempty"` }
RayServiceSpec defines the desired state of RayService
func (*RayServiceSpec) DeepCopy ¶
func (in *RayServiceSpec) DeepCopy() *RayServiceSpec
DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new RayServiceSpec.
func (*RayServiceSpec) DeepCopyInto ¶
func (in *RayServiceSpec) DeepCopyInto(out *RayServiceSpec)
DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil.
type RayServiceStatus ¶
type RayServiceStatus struct { // Important: Run "make" to regenerate code after modifying this file Applications map[string]AppStatus `json:"applicationStatuses,omitempty"` RayClusterName string `json:"rayClusterName,omitempty"` RayClusterStatus RayClusterStatus `json:"rayClusterStatus,omitempty"` }
func (*RayServiceStatus) DeepCopy ¶
func (in *RayServiceStatus) DeepCopy() *RayServiceStatus
DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new RayServiceStatus.
func (*RayServiceStatus) DeepCopyInto ¶
func (in *RayServiceStatus) DeepCopyInto(out *RayServiceStatus)
DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil.
type RayServiceStatuses ¶
type RayServiceStatuses struct { // LastUpdateTime represents the timestamp when the RayService status was last updated. LastUpdateTime *metav1.Time `json:"lastUpdateTime,omitempty"` // ServiceStatus indicates the current RayService status. ServiceStatus ServiceStatus `json:"serviceStatus,omitempty"` ActiveServiceStatus RayServiceStatus `json:"activeServiceStatus,omitempty"` // Pending Service Status indicates a RayCluster will be created or is being created. PendingServiceStatus RayServiceStatus `json:"pendingServiceStatus,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. ObservedGeneration int64 `json:"observedGeneration,omitempty"` }
RayServiceStatuses defines the observed state of RayService +kubebuilder:printcolumn:name="ServiceStatus",type=string,JSONPath=".status.serviceStatus"
func (*RayServiceStatuses) DeepCopy ¶
func (in *RayServiceStatuses) DeepCopy() *RayServiceStatuses
DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new RayServiceStatuses.
func (*RayServiceStatuses) DeepCopyInto ¶
func (in *RayServiceStatuses) DeepCopyInto(out *RayServiceStatuses)
DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil.
type ScaleStrategy ¶
type ScaleStrategy struct { // WorkersToDelete workers to be deleted WorkersToDelete []string `json:"workersToDelete,omitempty"` }
ScaleStrategy to remove workers
func (*ScaleStrategy) DeepCopy ¶
func (in *ScaleStrategy) DeepCopy() *ScaleStrategy
DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new ScaleStrategy.
func (*ScaleStrategy) DeepCopyInto ¶
func (in *ScaleStrategy) DeepCopyInto(out *ScaleStrategy)
DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil.
type ServeDeploymentStatus ¶
type ServeDeploymentStatus struct { // 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"` // Name, Status, Message are from Ray Dashboard and represent a Serve deployment's state. // TODO: change status type to enum Status string `json:"status,omitempty"` Message string `json:"message,omitempty"` }
ServeDeploymentStatus defines the current state of a Serve deployment
func (*ServeDeploymentStatus) DeepCopy ¶
func (in *ServeDeploymentStatus) DeepCopy() *ServeDeploymentStatus
DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new ServeDeploymentStatus.
func (*ServeDeploymentStatus) DeepCopyInto ¶
func (in *ServeDeploymentStatus) DeepCopyInto(out *ServeDeploymentStatus)
DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil.
type ServiceStatus ¶
type ServiceStatus string
const ( FailedToGetOrCreateRayCluster ServiceStatus = "FailedToGetOrCreateRayCluster" WaitForServeDeploymentReady ServiceStatus = "WaitForServeDeploymentReady" FailedToGetServeDeploymentStatus ServiceStatus = "FailedToGetServeDeploymentStatus" Running ServiceStatus = "Running" Restarting ServiceStatus = "Restarting" FailedToUpdateIngress ServiceStatus = "FailedToUpdateIngress" FailedToUpdateServingPodLabel ServiceStatus = "FailedToUpdateServingPodLabel" FailedToUpdateService ServiceStatus = "FailedToUpdateService" )
type UpscalingMode ¶
type UpscalingMode string
+kubebuilder:validation:Enum=Default;Aggressive;Conservative
type WorkerGroupSpec ¶
type WorkerGroupSpec struct { // we can have multiple worker groups, we distinguish them by name GroupName string `json:"groupName"` // Replicas is the number of desired Pods for this worker group. See https://github.com/ray-project/kuberay/pull/1443 for more details about the reason for making this field optional. // +kubebuilder:default:=0 Replicas *int32 `json:"replicas,omitempty"` // MinReplicas denotes the minimum number of desired Pods for this worker group. // +kubebuilder:default:=0 MinReplicas *int32 `json:"minReplicas"` // MaxReplicas denotes the maximum number of desired Pods for this worker group, and the default value is maxInt32. // +kubebuilder:default:=2147483647 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 corev1.PodTemplateSpec `json:"template"` // ScaleStrategy defines which pods to remove ScaleStrategy ScaleStrategy `json:"scaleStrategy,omitempty"` }
WorkerGroupSpec are the specs for the worker pods
func (*WorkerGroupSpec) DeepCopy ¶
func (in *WorkerGroupSpec) DeepCopy() *WorkerGroupSpec
DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new WorkerGroupSpec.
func (*WorkerGroupSpec) DeepCopyInto ¶
func (in *WorkerGroupSpec) DeepCopyInto(out *WorkerGroupSpec)
DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil.