Documentation
¶
Overview ¶
Package v1 contains API Schema definitions for the ray v1 API group +kubebuilder:object:generate=true +groupName=ray.io
Index ¶
- Constants
- Variables
- func IsJobDeploymentTerminal(status JobDeploymentStatus) bool
- func IsJobTerminal(status JobStatus) bool
- func Resource(resource string) schema.GroupResource
- type AppStatus
- type AuthMode
- type AuthOptions
- type AutoscalerOptions
- type AutoscalerVersion
- type ClusterState
- type ClusterUpgradeOptions
- type DeletionCondition
- type DeletionPolicy
- type DeletionPolicyType
- type DeletionRule
- type DeletionStrategy
- type EventReason
- type GcsFaultToleranceOptions
- type HeadGroupSpec
- type HeadInfo
- type JobDeploymentStatus
- type JobFailedReason
- type JobStatus
- type JobSubmissionMode
- type RayCluster
- type RayClusterConditionType
- type RayClusterList
- type RayClusterSpec
- type RayClusterStatus
- type RayJob
- type RayJobList
- type RayJobSpec
- type RayJobStatus
- type RayJobStatusInfo
- type RayNodeType
- type RayService
- type RayServiceConditionReason
- type RayServiceConditionType
- type RayServiceList
- type RayServiceSpec
- type RayServiceStatus
- type RayServiceStatuses
- type RayServiceUpgradeStrategy
- type RayServiceUpgradeType
- type RedisCredential
- type ScaleStrategy
- type ServeDeploymentStatus
- type ServiceStatus
- type SubmitterConfig
- type UpscalingMode
- type WorkerGroupSpec
Constants ¶
const ( AllPodRunningAndReadyFirstTime = "AllPodRunningAndReadyFirstTime" RayClusterPodsProvisioning = "RayClusterPodsProvisioning" HeadPodNotFound = "HeadPodNotFound" HeadPodRunningAndReady = "HeadPodRunningAndReady" // UnknownReason says that the reason for the condition is unknown. UnknownReason = "Unknown" )
Custom Reason for RayClusterCondition
Variables ¶
var ( // GroupVersion is group version used to register these objects GroupVersion = schema.GroupVersion{Group: "ray.io", Version: "v1"} // 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 AllJobStatuses = []JobStatus{ JobStatusNew, JobStatusPending, JobStatusRunning, JobStatusStopped, JobStatusSucceeded, JobStatusFailed, }
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 IsJobDeploymentTerminal ¶ added in v1.4.0
func IsJobDeploymentTerminal(status JobDeploymentStatus) bool
IsJobDeploymentTerminal returns true if the given JobDeploymentStatus is in a terminal state. Terminal states are either Complete or Failed.
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 {
// +optional
Deployments map[string]ServeDeploymentStatus `json:"serveDeploymentStatuses,omitempty"`
// +optional
Status string `json:"status,omitempty"`
// +optional
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 AuthMode ¶ added in v1.5.1
type AuthMode string
AuthMode describes the authentication mode for the Ray cluster.
type AuthOptions ¶ added in v1.5.1
type AuthOptions struct {
// Mode specifies the authentication mode.
// Supported values are "disabled" and "token".
// Defaults to "token".
// +kubebuilder:validation:Enum=disabled;token
// +optional
Mode AuthMode `json:"mode,omitempty"`
}
AuthOptions defines the authentication options for a RayCluster.
func (*AuthOptions) DeepCopy ¶ added in v1.5.1
func (in *AuthOptions) DeepCopy() *AuthOptions
DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new AuthOptions.
func (*AuthOptions) DeepCopyInto ¶ added in v1.5.1
func (in *AuthOptions) DeepCopyInto(out *AuthOptions)
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.
// +optional
Resources *corev1.ResourceRequirements `json:"resources,omitempty"`
// Image optionally overrides the autoscaler's container image. This override is provided for autoscaler testing and development.
// +optional
Image *string `json:"image,omitempty"`
// ImagePullPolicy optionally overrides the autoscaler container's image pull policy. This override is provided for autoscaler testing and development.
// +optional
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/
// +optional
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.
// +optional
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.
// +optional
UpscalingMode *UpscalingMode `json:"upscalingMode,omitempty"`
// Version is the version of the Ray autoscaler.
// Setting this to v1 will explicitly use autoscaler v1.
// Setting this to v2 will explicitly use autoscaler v2.
// If this isn't set, the Ray version determines the autoscaler version.
// In Ray 2.47.0 and later, the default autoscaler version is v2. It's v1 before that.
// +optional
Version *AutoscalerVersion `json:"version,omitempty"`
// Optional list of environment variables to set in the autoscaler container.
// +optional
Env []corev1.EnvVar `json:"env,omitempty"`
// Optional list of sources to populate environment variables in the autoscaler container.
// +optional
EnvFrom []corev1.EnvFromSource `json:"envFrom,omitempty"`
// Optional list of volumeMounts. This is needed for enabling TLS for the autoscaler container.
// +optional
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 AutoscalerVersion ¶ added in v1.4.0
type AutoscalerVersion string
+kubebuilder:validation:Enum=v1;v2
const ( AutoscalerVersionV1 AutoscalerVersion = "v1" AutoscalerVersionV2 AutoscalerVersion = "v2" )
type ClusterState ¶
type ClusterState string
The overall state of the Ray cluster.
const ( Ready ClusterState = "ready" // Failed is deprecated, but we keep it to avoid compilation errors in projects that import the KubeRay Golang module. Failed ClusterState = "failed" Suspended ClusterState = "suspended" )
type ClusterUpgradeOptions ¶ added in v1.5.0
type ClusterUpgradeOptions struct {
// The capacity of serve requests the upgraded cluster should scale to handle each interval.
// Defaults to 100%.
// +kubebuilder:default:=100
MaxSurgePercent *int32 `json:"maxSurgePercent,omitempty"`
// The percentage of traffic to switch to the upgraded RayCluster at a set interval after scaling by MaxSurgePercent.
StepSizePercent *int32 `json:"stepSizePercent"`
// The interval in seconds between transferring StepSize traffic from the old to new RayCluster.
IntervalSeconds *int32 `json:"intervalSeconds"`
// The name of the Gateway Class installed by the Kubernetes Cluster admin.
GatewayClassName string `json:"gatewayClassName"`
}
These options are currently only supported for the IncrementalUpgrade type.
func (*ClusterUpgradeOptions) DeepCopy ¶ added in v1.5.0
func (in *ClusterUpgradeOptions) DeepCopy() *ClusterUpgradeOptions
DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new ClusterUpgradeOptions.
func (*ClusterUpgradeOptions) DeepCopyInto ¶ added in v1.5.0
func (in *ClusterUpgradeOptions) DeepCopyInto(out *ClusterUpgradeOptions)
DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil.
type DeletionCondition ¶ added in v1.5.0
type DeletionCondition struct {
// JobStatus is the terminal status of the RayJob that triggers this condition. This field is required.
// For the initial implementation, only "SUCCEEDED" and "FAILED" are supported.
// +kubebuilder:validation:Enum=SUCCEEDED;FAILED
JobStatus JobStatus `json:"jobStatus"`
// TTLSeconds is the time in seconds from when the JobStatus
// reaches the specified terminal state to when this deletion action should be triggered.
// The value must be a non-negative integer.
// +kubebuilder:default=0
// +kubebuilder:validation:Minimum=0
// +optional
TTLSeconds int32 `json:"ttlSeconds,omitempty"`
}
DeletionCondition specifies the trigger conditions for a deletion action.
func (*DeletionCondition) DeepCopy ¶ added in v1.5.0
func (in *DeletionCondition) DeepCopy() *DeletionCondition
DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new DeletionCondition.
func (*DeletionCondition) DeepCopyInto ¶ added in v1.5.0
func (in *DeletionCondition) DeepCopyInto(out *DeletionCondition)
DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil.
type DeletionPolicy ¶ added in v1.3.0
type DeletionPolicy struct {
// Policy is the action to take when the condition is met.
// This field is logically required when using the legacy OnSuccess/OnFailure policies.
// It is marked as '+optional' at the API level to allow the 'deletionRules' field to be used instead.
// +kubebuilder:validation:Enum=DeleteCluster;DeleteWorkers;DeleteSelf;DeleteNone
// +optional
Policy *DeletionPolicyType `json:"policy,omitempty"`
}
DeletionPolicy is the legacy single-stage deletion policy. Deprecated: This struct is part of the legacy API. Use DeletionRule for new configurations.
func (*DeletionPolicy) DeepCopy ¶ added in v1.5.0
func (in *DeletionPolicy) DeepCopy() *DeletionPolicy
DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new DeletionPolicy.
func (*DeletionPolicy) DeepCopyInto ¶ added in v1.5.0
func (in *DeletionPolicy) DeepCopyInto(out *DeletionPolicy)
DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil.
type DeletionPolicyType ¶ added in v1.5.0
type DeletionPolicyType string
const ( DeleteCluster DeletionPolicyType = "DeleteCluster" // To delete the entire RayCluster custom resource on job completion. DeleteWorkers DeletionPolicyType = "DeleteWorkers" // To delete only the workers on job completion. DeleteSelf DeletionPolicyType = "DeleteSelf" // To delete the RayJob custom resource (and all associated resources) on job completion. DeleteNone DeletionPolicyType = "DeleteNone" // To delete no resources on job completion. )
type DeletionRule ¶ added in v1.5.0
type DeletionRule struct {
// Policy is the action to take when the condition is met. This field is required.
// +kubebuilder:validation:Enum=DeleteCluster;DeleteWorkers;DeleteSelf;DeleteNone
Policy DeletionPolicyType `json:"policy"`
// The condition under which this deletion rule is triggered. This field is required.
Condition DeletionCondition `json:"condition"`
}
DeletionRule defines a single deletion action and its trigger condition. This is the new, recommended way to define deletion behavior.
func (*DeletionRule) DeepCopy ¶ added in v1.5.0
func (in *DeletionRule) DeepCopy() *DeletionRule
DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new DeletionRule.
func (*DeletionRule) DeepCopyInto ¶ added in v1.5.0
func (in *DeletionRule) DeepCopyInto(out *DeletionRule)
DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil.
type DeletionStrategy ¶ added in v1.5.0
type DeletionStrategy struct {
// OnSuccess is the deletion policy for a successful RayJob.
// Deprecated: Use `deletionRules` instead for more flexible, multi-stage deletion strategies.
// This field will be removed in release 1.6.0.
// +optional
OnSuccess *DeletionPolicy `json:"onSuccess,omitempty"`
// OnFailure is the deletion policy for a failed RayJob.
// Deprecated: Use `deletionRules` instead for more flexible, multi-stage deletion strategies.
// This field will be removed in release 1.6.0.
// +optional
OnFailure *DeletionPolicy `json:"onFailure,omitempty"`
// DeletionRules is a list of deletion rules, processed based on their trigger conditions.
// While the rules can be used to define a sequence, if multiple rules are overdue (e.g., due to controller downtime),
// the most impactful rule (e.g., DeleteSelf) will be executed first to prioritize resource cleanup.
// +optional
// +listType=atomic
// +kubebuilder:validation:MinItems=1
DeletionRules []DeletionRule `json:"deletionRules,omitempty"`
}
DeletionStrategy configures automated cleanup after the RayJob reaches a terminal state. Two mutually exclusive styles are supported:
Legacy: provide both onSuccess and onFailure (deprecated; removal planned for 1.6.0). May be combined with shutdownAfterJobFinishes and (optionally) global TTLSecondsAfterFinished. Rules: provide deletionRules (non-empty list). Rules mode is incompatible with shutdownAfterJobFinishes, legacy fields, and the global TTLSecondsAfterFinished (use per‑rule condition.ttlSeconds instead).
Semantics:
- A non-empty deletionRules selects rules mode; empty lists are treated as unset.
- Legacy requires both onSuccess and onFailure; specifying only one is invalid.
- Global TTLSecondsAfterFinished > 0 requires shutdownAfterJobFinishes=true; therefore it cannot be used with rules mode or with legacy alone (no shutdown).
- Feature gate RayJobDeletionPolicy must be enabled when this block is present.
Validation:
- CRD XValidations prevent mixing legacy fields with deletionRules and enforce legacy completeness.
- Controller logic enforces rules vs shutdown exclusivity and TTL constraints.
- onSuccess/onFailure are deprecated; migration to deletionRules is encouraged.
+kubebuilder:validation:XValidation:rule="!((has(self.onSuccess) || has(self.onFailure)) && has(self.deletionRules))",message="legacy policies (onSuccess/onFailure) and deletionRules cannot be used together within the same deletionStrategy" +kubebuilder:validation:XValidation:rule="((has(self.onSuccess) && has(self.onFailure)) || has(self.deletionRules))",message="deletionStrategy requires either BOTH onSuccess and onFailure, OR the deletionRules field (cannot be empty)"
func (*DeletionStrategy) DeepCopy ¶ added in v1.5.0
func (in *DeletionStrategy) DeepCopy() *DeletionStrategy
DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new DeletionStrategy.
func (*DeletionStrategy) DeepCopyInto ¶ added in v1.5.0
func (in *DeletionStrategy) DeepCopyInto(out *DeletionStrategy)
DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil.
type EventReason ¶
type EventReason string
const ( RayConfigError EventReason = "RayConfigError" PodReconciliationError EventReason = "PodReconciliationError" )
type GcsFaultToleranceOptions ¶ added in v1.3.0
type GcsFaultToleranceOptions struct {
// +optional
RedisUsername *RedisCredential `json:"redisUsername,omitempty"`
// +optional
RedisPassword *RedisCredential `json:"redisPassword,omitempty"`
// +optional
ExternalStorageNamespace string `json:"externalStorageNamespace,omitempty"`
RedisAddress string `json:"redisAddress"`
}
GcsFaultToleranceOptions contains configs for GCS FT
func (*GcsFaultToleranceOptions) DeepCopy ¶ added in v1.3.0
func (in *GcsFaultToleranceOptions) DeepCopy() *GcsFaultToleranceOptions
DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new GcsFaultToleranceOptions.
func (*GcsFaultToleranceOptions) DeepCopyInto ¶ added in v1.3.0
func (in *GcsFaultToleranceOptions) DeepCopyInto(out *GcsFaultToleranceOptions)
DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil.
type HeadGroupSpec ¶
type HeadGroupSpec struct {
// Template is the exact pod template used in K8s deployments, statefulsets, etc.
Template corev1.PodTemplateSpec `json:"template"`
// HeadService is the Kubernetes service of the head pod.
// +optional
HeadService *corev1.Service `json:"headService,omitempty"`
// EnableIngress indicates whether operator should create ingress object for head service or not.
// +optional
EnableIngress *bool `json:"enableIngress,omitempty"`
// Resources specifies the resource quantities for the head group.
// These values override the resources passed to `rayStartParams` for the group, but
// have no effect on the resources set at the K8s Pod container level.
// +optional
Resources map[string]string `json:"resources,omitempty"`
// Labels specifies the Ray node labels for the head group.
// These labels will also be added to the Pods of this head group and override the `--labels`
// argument passed to `rayStartParams`.
// +optional
Labels map[string]string `json:"labels,omitempty"`
// RayStartParams are the params of the start command: node-manager-port, object-store-memory, ...
// +optional
RayStartParams map[string]string `json:"rayStartParams"`
// ServiceType is Kubernetes service type of the head service. it will be used by the workers to connect to the head pod
// +optional
ServiceType corev1.ServiceType `json:"serviceType,omitempty"`
}
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 {
// +optional
PodIP string `json:"podIP,omitempty"`
// +optional
ServiceIP string `json:"serviceIP,omitempty"`
// +optional
PodName string `json:"podName,omitempty"`
// +optional
ServiceName string `json:"serviceName,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" JobDeploymentStatusFailed JobDeploymentStatus = "Failed" JobDeploymentStatusValidationFailed JobDeploymentStatus = "ValidationFailed" JobDeploymentStatusSuspending JobDeploymentStatus = "Suspending" JobDeploymentStatusSuspended JobDeploymentStatus = "Suspended" JobDeploymentStatusRetrying JobDeploymentStatus = "Retrying" JobDeploymentStatusWaiting JobDeploymentStatus = "Waiting" )
type JobFailedReason ¶ added in v1.1.0
type JobFailedReason string
JobFailedReason indicates the reason the RayJob changes its JobDeploymentStatus to 'Failed'
const ( SubmissionFailed JobFailedReason = "SubmissionFailed" DeadlineExceeded JobFailedReason = "DeadlineExceeded" AppFailed JobFailedReason = "AppFailed" JobDeploymentStatusTransitionGracePeriodExceeded JobFailedReason = "JobDeploymentStatusTransitionGracePeriodExceeded" ValidationFailed JobFailedReason = "ValidationFailed" )
type JobStatus ¶
type JobStatus string
JobStatus is the Ray Job Status.
const ( JobStatusNew JobStatus = "" JobStatusPending JobStatus = "PENDING" JobStatusRunning JobStatus = "RUNNING" JobStatusStopped JobStatus = "STOPPED" JobStatusSucceeded JobStatus = "SUCCEEDED" JobStatusFailed JobStatus = "FAILED" )
NOTICE: AllJobStatuses should be kept in sync with all job statuses below.
type JobSubmissionMode ¶ added in v1.1.0
type JobSubmissionMode string
const ( K8sJobMode JobSubmissionMode = "K8sJobMode" // Submit job via Kubernetes Job HTTPMode JobSubmissionMode = "HTTPMode" // Submit job via HTTP request InteractiveMode JobSubmissionMode = "InteractiveMode" // Don't submit job in KubeRay. Instead, wait for user to submit job and provide the job submission ID. SidecarMode JobSubmissionMode = "SidecarMode" // Submit job via a sidecar container in the Ray head Pod )
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"`
// +optional
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: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="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 RayClusterConditionType ¶ added in v1.2.0
type RayClusterConditionType string
const ( // RayClusterProvisioned indicates whether all Ray Pods are ready for the first time. // After RayClusterProvisioned is set to true for the first time, it will not change anymore. RayClusterProvisioned RayClusterConditionType = "RayClusterProvisioned" // HeadPodReady indicates whether RayCluster's head Pod is ready for requests. HeadPodReady RayClusterConditionType = "HeadPodReady" // RayClusterReplicaFailure is added in a RayCluster when one of its pods fails to be created or deleted. RayClusterReplicaFailure RayClusterConditionType = "ReplicaFailure" // RayClusterSuspending is set to true when a user sets .Spec.Suspend to true, ensuring the atomicity of the suspend operation. RayClusterSuspending RayClusterConditionType = "RayClusterSuspending" // RayClusterSuspended is set to true when all Pods belonging to a suspending RayCluster are deleted. Note that RayClusterSuspending and RayClusterSuspended cannot both be true at the same time. RayClusterSuspended RayClusterConditionType = "RayClusterSuspended" )
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 {
// AuthOptions specifies the authentication options for the RayCluster.
// +optional
AuthOptions *AuthOptions `json:"authOptions,omitempty"`
// Suspend indicates whether a RayCluster should be suspended.
// A suspended RayCluster will have head pods and worker pods deleted.
// +optional
Suspend *bool `json:"suspend,omitempty"`
// ManagedBy is an optional configuration for the controller or entity that manages a RayCluster.
// The value must be either 'ray.io/kuberay-operator' or 'kueue.x-k8s.io/multikueue'.
// The kuberay-operator reconciles a RayCluster which doesn't have this field at all or
// the field value is the reserved string 'ray.io/kuberay-operator',
// but delegates reconciling the RayCluster with 'kueue.x-k8s.io/multikueue' to the Kueue.
// The field is immutable.
// +kubebuilder:validation:XValidation:rule="self == oldSelf",message="the managedBy field is immutable"
// +kubebuilder:validation:XValidation:rule="self in ['ray.io/kuberay-operator', 'kueue.x-k8s.io/multikueue']",message="the managedBy field value must be either 'ray.io/kuberay-operator' or 'kueue.x-k8s.io/multikueue'"
// +optional
ManagedBy *string `json:"managedBy,omitempty"`
// AutoscalerOptions specifies optional configuration for the Ray autoscaler.
// +optional
AutoscalerOptions *AutoscalerOptions `json:"autoscalerOptions,omitempty"`
// +optional
HeadServiceAnnotations map[string]string `json:"headServiceAnnotations,omitempty"`
// EnableInTreeAutoscaling indicates whether operator should create in tree autoscaling configs
// +optional
EnableInTreeAutoscaling *bool `json:"enableInTreeAutoscaling,omitempty"`
// GcsFaultToleranceOptions for enabling GCS FT
// +optional
GcsFaultToleranceOptions *GcsFaultToleranceOptions `json:"gcsFaultToleranceOptions,omitempty"`
// HeadGroupSpec is the spec for the head pod
HeadGroupSpec HeadGroupSpec `json:"headGroupSpec"`
// RayVersion is used to determine the command for the Kubernetes Job managed by RayJob
// +optional
RayVersion string `json:"rayVersion,omitempty"`
// WorkerGroupSpecs are the specs for the worker pods
// +optional
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 {
// 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.
// +optional
State ClusterState `json:"state,omitempty"`
// DesiredCPU indicates total desired CPUs for the cluster
// +optional
DesiredCPU resource.Quantity `json:"desiredCPU,omitempty"`
// DesiredMemory indicates total desired memory for the cluster
// +optional
DesiredMemory resource.Quantity `json:"desiredMemory,omitempty"`
// DesiredGPU indicates total desired GPUs for the cluster
// +optional
DesiredGPU resource.Quantity `json:"desiredGPU,omitempty"`
// DesiredTPU indicates total desired TPUs for the cluster
// +optional
DesiredTPU resource.Quantity `json:"desiredTPU,omitempty"`
// LastUpdateTime indicates last update timestamp for this cluster status.
// +nullable
LastUpdateTime *metav1.Time `json:"lastUpdateTime,omitempty"`
// StateTransitionTimes indicates the time of the last state transition for each state.
// +optional
StateTransitionTimes map[ClusterState]*metav1.Time `json:"stateTransitionTimes,omitempty"`
// Service Endpoints
// +optional
Endpoints map[string]string `json:"endpoints,omitempty"`
// Head info
// +optional
Head HeadInfo `json:"head,omitempty"`
// Reason provides more information about current State
// +optional
Reason string `json:"reason,omitempty"`
// Represents the latest available observations of a RayCluster's current state.
// +patchMergeKey=type
// +patchStrategy=merge
// +listType=map
// +listMapKey=type
// +optional
Conditions []metav1.Condition `json:"conditions,omitempty" patchStrategy:"merge" patchMergeKey:"type"`
// ReadyWorkerReplicas indicates the number of worker pods currently in the Ready state in the cluster.
// It actually reflects the number of Ready pods, although it is named "replicas" to maintain backward compatibility.
// +optional
ReadyWorkerReplicas int32 `json:"readyWorkerReplicas,omitempty"`
// AvailableWorkerReplicas indicates how many worker pods are currently available (i.e., running).
// It is named "replicas" to maintain backward compatibility.
// +optional
AvailableWorkerReplicas int32 `json:"availableWorkerReplicas,omitempty"`
// DesiredWorkerReplicas indicates the desired total number of worker Pods at the cluster level,
// calculated as the sum of `replicas * numOfHosts` for each worker group.
// It is named "replicas" to maintain backward compatibility.
// +optional
DesiredWorkerReplicas int32 `json:"desiredWorkerReplicas,omitempty"`
// MinWorkerReplicas indicates the minimum number of worker pods across all worker groups,
// calculated as the sum of `minReplicas * numOfHosts` for each worker group.
// It is named "replicas" to maintain backward compatibility.
// +optional
MinWorkerReplicas int32 `json:"minWorkerReplicas,omitempty"`
// MaxWorkerReplicas indicates the maximum number of worker pods across all worker groups,
// calculated as the sum of `maxReplicas * numOfHosts` for each worker group.
// It is named "replicas" to maintain backward compatibility.
// +optional
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.
// +optional
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"`
// +optional
Status RayJobStatus `json:"status,omitempty"`
}
+kubebuilder:object:root=true +kubebuilder:resource:categories=all +kubebuilder:subresource:status +kubebuilder:storageversion +kubebuilder:printcolumn:name="job status",type=string,JSONPath=".status.jobStatus",priority=0 +kubebuilder:printcolumn:name="deployment status",type=string,JSONPath=".status.jobDeploymentStatus",priority=0 +kubebuilder:printcolumn:name="ray cluster name",type="string",JSONPath=".status.rayClusterName",priority=0 +kubebuilder:printcolumn:name="start time",type=string,JSONPath=".status.startTime",priority=0 +kubebuilder:printcolumn:name="end time",type=string,JSONPath=".status.endTime",priority=0 +kubebuilder:printcolumn:name="age",type="date",JSONPath=".metadata.creationTimestamp",priority=0 +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 {
// ActiveDeadlineSeconds is the duration in seconds that the RayJob may be active before
// KubeRay actively tries to terminate the RayJob; value must be positive integer.
// +optional
ActiveDeadlineSeconds *int32 `json:"activeDeadlineSeconds,omitempty"`
// Specifies the number of retries before marking this job failed.
// Each retry creates a new RayCluster.
// +kubebuilder:default:=0
// +optional
BackoffLimit *int32 `json:"backoffLimit,omitempty"`
// RayClusterSpec is the cluster template to run the job
RayClusterSpec *RayClusterSpec `json:"rayClusterSpec,omitempty"`
// SubmitterPodTemplate is the template for the pod that will run `ray job submit`.
// +optional
SubmitterPodTemplate *corev1.PodTemplateSpec `json:"submitterPodTemplate,omitempty"`
// Metadata is data to store along with this job.
// +optional
Metadata map[string]string `json:"metadata,omitempty"`
// clusterSelector is used to select running rayclusters by labels
// +optional
ClusterSelector map[string]string `json:"clusterSelector,omitempty"`
// Configurations of submitter k8s job.
// +optional
SubmitterConfig *SubmitterConfig `json:"submitterConfig,omitempty"`
// ManagedBy is an optional configuration for the controller or entity that manages a RayJob.
// The value must be either 'ray.io/kuberay-operator' or 'kueue.x-k8s.io/multikueue'.
// The kuberay-operator reconciles a RayJob which doesn't have this field at all or
// the field value is the reserved string 'ray.io/kuberay-operator',
// but delegates reconciling the RayJob with 'kueue.x-k8s.io/multikueue' to the Kueue.
// The field is immutable.
// +kubebuilder:validation:XValidation:rule="self == oldSelf",message="the managedBy field is immutable"
// +kubebuilder:validation:XValidation:rule="self in ['ray.io/kuberay-operator', 'kueue.x-k8s.io/multikueue']",message="the managedBy field value must be either 'ray.io/kuberay-operator' or 'kueue.x-k8s.io/multikueue'"
// +optional
ManagedBy *string `json:"managedBy,omitempty"`
// DeletionStrategy automates post-completion cleanup.
// Choose one style or omit:
// - Legacy: both onSuccess & onFailure (deprecated; may combine with shutdownAfterJobFinishes and TTLSecondsAfterFinished).
// - Rules: deletionRules (non-empty) — incompatible with shutdownAfterJobFinishes, legacy fields, and global TTLSecondsAfterFinished (use per-rule condition.ttlSeconds).
// Global TTLSecondsAfterFinished > 0 requires shutdownAfterJobFinishes=true.
// Feature gate RayJobDeletionPolicy must be enabled when this field is set.
// +optional
DeletionStrategy *DeletionStrategy `json:"deletionStrategy,omitempty"`
// Entrypoint represents the command to start execution.
// +optional
Entrypoint string `json:"entrypoint,omitempty"`
// RuntimeEnvYAML represents the runtime environment configuration
// provided as a multi-line YAML string.
// +optional
RuntimeEnvYAML string `json:"runtimeEnvYAML,omitempty"`
// If jobId is not set, a new jobId will be auto-generated.
// +optional
JobId string `json:"jobId,omitempty"`
// SubmissionMode specifies how RayJob submits the Ray job to the RayCluster.
// In "K8sJobMode", the KubeRay operator creates a submitter Kubernetes Job to submit the Ray job.
// In "HTTPMode", the KubeRay operator sends a request to the RayCluster to create a Ray job.
// In "InteractiveMode", the KubeRay operator waits for a user to submit a job to the Ray cluster.
// In "SidecarMode", the KubeRay operator injects a container into the Ray head Pod that acts as the job submitter to submit the Ray job.
// +kubebuilder:default:=K8sJobMode
// +optional
SubmissionMode JobSubmissionMode `json:"submissionMode,omitempty"`
// EntrypointResources specifies the custom resources and quantities to reserve for the
// entrypoint command.
// +optional
EntrypointResources string `json:"entrypointResources,omitempty"`
// EntrypointNumCpus specifies the number of cpus to reserve for the entrypoint command.
// +optional
EntrypointNumCpus float32 `json:"entrypointNumCpus,omitempty"`
// EntrypointNumGpus specifies the number of gpus to reserve for the entrypoint command.
// +optional
EntrypointNumGpus float32 `json:"entrypointNumGpus,omitempty"`
// TTLSecondsAfterFinished is the TTL to clean up RayCluster.
// It's only working when ShutdownAfterJobFinishes set to true.
// +kubebuilder:default:=0
// +optional
TTLSecondsAfterFinished int32 `json:"ttlSecondsAfterFinished,omitempty"`
// ShutdownAfterJobFinishes will determine whether to delete the ray cluster once rayJob succeed or failed.
// +optional
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.
// +optional
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
// RayJobStatusInfo contains information about the Ray job retrieved from the Ray dashboard.
// +optional
RayJobStatusInfo RayJobStatusInfo `json:"rayJobInfo,omitempty"`
// +optional
JobId string `json:"jobId,omitempty"`
// +optional
RayClusterName string `json:"rayClusterName,omitempty"`
// +optional
DashboardURL string `json:"dashboardURL,omitempty"`
// +optional
JobStatus JobStatus `json:"jobStatus,omitempty"`
// +optional
JobDeploymentStatus JobDeploymentStatus `json:"jobDeploymentStatus,omitempty"`
// +optional
Reason JobFailedReason `json:"reason,omitempty"`
// +optional
Message string `json:"message,omitempty"`
// StartTime is the time when JobDeploymentStatus transitioned from 'New' to 'Initializing'.
// +optional
StartTime *metav1.Time `json:"startTime,omitempty"`
// EndTime is the time when JobDeploymentStatus transitioned to 'Complete' status.
// This occurs when the Ray job reaches a terminal state (SUCCEEDED, FAILED, STOPPED)
// or the submitter Job has failed.
// +optional
EndTime *metav1.Time `json:"endTime,omitempty"`
// Succeeded is the number of times this job succeeded.
// +kubebuilder:default:=0
// +optional
Succeeded *int32 `json:"succeeded,omitempty"`
// Failed is the number of times this job failed.
// +kubebuilder:default:=0
// +optional
Failed *int32 `json:"failed,omitempty"`
// RayClusterStatus is the status of the RayCluster running the job.
// +optional
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.
// +optional
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 RayJobStatusInfo ¶ added in v1.4.0
type RayJobStatusInfo struct {
StartTime *metav1.Time `json:"startTime,omitempty"`
EndTime *metav1.Time `json:"endTime,omitempty"`
}
`RayJobStatusInfo` is a subset of `RayJobInfo` from `dashboard_httpclient.py`. This subset is used to store information in the CR status.
TODO(kevin85421): We can consider exposing the whole `RayJobInfo` in the CR status after careful consideration. In that case, we can remove `RayJobStatusInfo`.
func (*RayJobStatusInfo) DeepCopy ¶ added in v1.4.0
func (in *RayJobStatusInfo) DeepCopy() *RayJobStatusInfo
DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new RayJobStatusInfo.
func (*RayJobStatusInfo) DeepCopyInto ¶ added in v1.4.0
func (in *RayJobStatusInfo) DeepCopyInto(out *RayJobStatusInfo)
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"`
// +optional
Status RayServiceStatuses `json:"status,omitempty"`
}
+kubebuilder:object:root=true +kubebuilder:resource:categories=all +kubebuilder:subresource:status +kubebuilder:storageversion +kubebuilder:printcolumn:name="service status",type=string,JSONPath=".status.serviceStatus" +kubebuilder:printcolumn:name="num serve endpoints",type=string,JSONPath=".status.numServeEndpoints" +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 RayServiceConditionReason ¶ added in v1.3.0
type RayServiceConditionReason string
const ( RayServiceInitializing RayServiceConditionReason = "Initializing" RayServiceInitializingTimeout RayServiceConditionReason = "InitializingTimeout" ZeroServeEndpoints RayServiceConditionReason = "ZeroServeEndpoints" NonZeroServeEndpoints RayServiceConditionReason = "NonZeroServeEndpoints" BothActivePendingClustersExist RayServiceConditionReason = "BothActivePendingClustersExist" NoPendingCluster RayServiceConditionReason = "NoPendingCluster" NoActiveCluster RayServiceConditionReason = "NoActiveCluster" )
type RayServiceConditionType ¶ added in v1.3.0
type RayServiceConditionType string
const ( // RayServiceReady means users can send requests to the underlying cluster and the number of serve endpoints is greater than 0. RayServiceReady RayServiceConditionType = "Ready" // UpgradeInProgress means the RayService is currently performing a zero-downtime upgrade. UpgradeInProgress RayServiceConditionType = "UpgradeInProgress" )
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 {
// RayClusterDeletionDelaySeconds specifies the delay, in seconds, before deleting old RayClusters.
// The default value is 60 seconds.
// +kubebuilder:validation:Minimum=0
// +optional
RayClusterDeletionDelaySeconds *int32 `json:"rayClusterDeletionDelaySeconds,omitempty"`
// Deprecated: This field is not used anymore. ref: https://github.com/ray-project/kuberay/issues/1685
// +optional
ServiceUnhealthySecondThreshold *int32 `json:"serviceUnhealthySecondThreshold,omitempty"`
// Deprecated: This field is not used anymore. ref: https://github.com/ray-project/kuberay/issues/1685
// +optional
DeploymentUnhealthySecondThreshold *int32 `json:"deploymentUnhealthySecondThreshold,omitempty"`
// ServeService is the Kubernetes service for head node and worker nodes who have healthy http proxy to serve traffics.
// +optional
ServeService *corev1.Service `json:"serveService,omitempty"`
// UpgradeStrategy defines the scaling policy used when upgrading the RayService.
// +optional
UpgradeStrategy *RayServiceUpgradeStrategy `json:"upgradeStrategy,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.
// +optional
ServeConfigV2 string `json:"serveConfigV2,omitempty"`
RayClusterSpec RayClusterSpec `json:"rayClusterConfig"`
// If the field is set to true, the value of the label `ray.io/serve` on the head Pod should always be false.
// Therefore, the head Pod's endpoint will not be added to the Kubernetes Serve service.
// +optional
ExcludeHeadPodFromServeSvc bool `json:"excludeHeadPodFromServeSvc,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
// +optional
Applications map[string]AppStatus `json:"applicationStatuses,omitempty"`
// TargetCapacity is the `target_capacity` percentage for all Serve replicas
// across the cluster for this RayService. The `num_replicas`, `min_replicas`, `max_replicas`,
// and `initial_replicas` for each deployment will be scaled by this percentage."
// +optional
TargetCapacity *int32 `json:"targetCapacity,omitempty"`
// TrafficRoutedPercent is the percentage of traffic that is routed to the Serve service
// for this RayService. TrafficRoutedPercent is updated to reflect the weight on the HTTPRoute
// created for this RayService during incremental upgrades to a new cluster.
// +optional
TrafficRoutedPercent *int32 `json:"trafficRoutedPercent,omitempty"`
// LastTrafficMigratedTime is the last time that TrafficRoutedPercent was updated to a new value
// for this RayService.
// +optional
LastTrafficMigratedTime *metav1.Time `json:"lastTrafficMigratedTime,omitempty"`
// +optional
RayClusterName string `json:"rayClusterName,omitempty"`
// +optional
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 {
// Represents the latest available observations of a RayService's current state.
// +patchMergeKey=type
// +patchStrategy=merge
// +listType=map
// +listMapKey=type
// +optional
Conditions []metav1.Condition `json:"conditions,omitempty" patchStrategy:"merge" patchMergeKey:"type"`
// LastUpdateTime represents the timestamp when the RayService status was last updated.
// +optional
LastUpdateTime *metav1.Time `json:"lastUpdateTime,omitempty"`
// Deprecated: `ServiceStatus` is deprecated - use `Conditions` instead. `Running` means the RayService is ready to
// serve requests. An empty `ServiceStatus` means the RayService is not ready to serve requests. The definition of
// `ServiceStatus` is equivalent to the `RayServiceReady` condition.
// +optional
ServiceStatus ServiceStatus `json:"serviceStatus,omitempty"`
// +optional
ActiveServiceStatus RayServiceStatus `json:"activeServiceStatus,omitempty"`
// Pending Service Status indicates a RayCluster will be created or is being created.
// +optional
PendingServiceStatus RayServiceStatus `json:"pendingServiceStatus,omitempty"`
// NumServeEndpoints indicates the number of Ray Pods that are actively serving or have been selected by the serve service.
// Ray Pods without a proxy actor or those that are unhealthy will not be counted.
// +optional
NumServeEndpoints int32 `json:"numServeEndpoints,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"`
}
RayServiceStatuses defines the observed state of RayService
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 RayServiceUpgradeStrategy ¶ added in v1.3.0
type RayServiceUpgradeStrategy struct {
// Type represents the strategy used when upgrading the RayService. Currently supports `NewCluster` and `None`.
// +optional
Type *RayServiceUpgradeType `json:"type,omitempty"`
// ClusterUpgradeOptions defines the behavior of a NewClusterWithIncrementalUpgrade type.
// RayServiceIncrementalUpgrade feature gate must be enabled to set ClusterUpgradeOptions.
ClusterUpgradeOptions *ClusterUpgradeOptions `json:"clusterUpgradeOptions,omitempty"`
}
func (*RayServiceUpgradeStrategy) DeepCopy ¶ added in v1.3.0
func (in *RayServiceUpgradeStrategy) DeepCopy() *RayServiceUpgradeStrategy
DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new RayServiceUpgradeStrategy.
func (*RayServiceUpgradeStrategy) DeepCopyInto ¶ added in v1.3.0
func (in *RayServiceUpgradeStrategy) DeepCopyInto(out *RayServiceUpgradeStrategy)
DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil.
type RayServiceUpgradeType ¶ added in v1.3.0
type RayServiceUpgradeType string
const ( // During upgrade, NewClusterWithIncrementalUpgrade strategy will create an upgraded cluster to gradually scale // and migrate traffic to using Gateway API. NewClusterWithIncrementalUpgrade RayServiceUpgradeType = "NewClusterWithIncrementalUpgrade" // During upgrade, NewCluster strategy will create new upgraded cluster and switch to it when it becomes ready NewCluster RayServiceUpgradeType = "NewCluster" // No new cluster will be created while the strategy is set to None None RayServiceUpgradeType = "None" )
type RedisCredential ¶ added in v1.3.0
type RedisCredential struct {
// +optional
ValueFrom *corev1.EnvVarSource `json:"valueFrom,omitempty"`
// +optional
Value string `json:"value,omitempty"`
}
RedisCredential is the redis username/password or a reference to the source containing the username/password
func (*RedisCredential) DeepCopy ¶ added in v1.3.0
func (in *RedisCredential) DeepCopy() *RedisCredential
DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new RedisCredential.
func (*RedisCredential) DeepCopyInto ¶ added in v1.3.0
func (in *RedisCredential) DeepCopyInto(out *RedisCredential)
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 {
// +optional
Status string `json:"status,omitempty"`
// +optional
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 ( // `Running` means the RayService is ready to serve requests. `NotRunning` means it is not ready. // The naming is a bit confusing, but to maintain backward compatibility, we use `Running` instead of `Ready`. // Since KubeRay v1.3.0, `ServiceStatus` is equivalent to the `RayServiceReady` condition. // `ServiceStatus` is deprecated - please use conditions instead. Running ServiceStatus = "Running" NotRunning ServiceStatus = "" )
type SubmitterConfig ¶ added in v1.2.0
type SubmitterConfig struct {
// BackoffLimit of the submitter k8s job.
// +optional
BackoffLimit *int32 `json:"backoffLimit,omitempty"`
}
func (*SubmitterConfig) DeepCopy ¶ added in v1.2.0
func (in *SubmitterConfig) DeepCopy() *SubmitterConfig
DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new SubmitterConfig.
func (*SubmitterConfig) DeepCopyInto ¶ added in v1.2.0
func (in *SubmitterConfig) DeepCopyInto(out *SubmitterConfig)
DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil.
type UpscalingMode ¶
type UpscalingMode string
+kubebuilder:validation:Enum=Default;Aggressive;Conservative
type WorkerGroupSpec ¶
type WorkerGroupSpec struct {
// Suspend indicates whether a worker group should be suspended.
// A suspended worker group will have all pods deleted.
// This is not a user-facing API and is only used by RayJob DeletionStrategy.
// +optional
Suspend *bool `json:"suspend,omitempty"`
// 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
// +optional
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"`
// IdleTimeoutSeconds denotes the number of seconds to wait before the v2 autoscaler terminates an idle worker pod of this type.
// This value is only used with the Ray Autoscaler enabled and defaults to the value set by the AutoscalingConfig if not specified for this worker group.
// +optional
IdleTimeoutSeconds *int32 `json:"idleTimeoutSeconds,omitempty"`
// Resources specifies the resource quantities for this worker group.
// These values override the resources passed to `rayStartParams` for the group, but
// have no effect on the resources set at the K8s Pod container level.
// +optional
Resources map[string]string `json:"resources,omitempty"`
// Labels specifies the Ray node labels for this worker group.
// These labels will also be added to the Pods of this worker group and override the `--labels`
// argument passed to `rayStartParams`.
// +optional
Labels map[string]string `json:"labels,omitempty"`
// RayStartParams are the params of the start command: address, object-store-memory, ...
// +optional
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
// +optional
ScaleStrategy ScaleStrategy `json:"scaleStrategy,omitempty"`
// NumOfHosts denotes the number of hosts to create per replica. The default value is 1.
// +kubebuilder:default:=1
// +optional
NumOfHosts int32 `json:"numOfHosts,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.