From 1314b9fd18c903afb10355590ca464017628ad96 Mon Sep 17 00:00:00 2001 From: xianlubird Date: Mon, 26 May 2025 15:20:44 +0800 Subject: [PATCH] Support custom resource configuration for the submit pod --- ray-operator/apis/ray/v1/rayjob_types.go | 9 +++++++++ .../config/crd/bases/ray.io_rayjobs.yaml | 19 +++++++++++++++++++ ray-operator/controllers/ray/common/job.go | 13 +++++++++++-- .../controllers/ray/rayjob_controller.go | 2 +- 4 files changed, 40 insertions(+), 3 deletions(-) diff --git a/ray-operator/apis/ray/v1/rayjob_types.go b/ray-operator/apis/ray/v1/rayjob_types.go index 3f044d8a4c6..dc630b765c6 100644 --- a/ray-operator/apis/ray/v1/rayjob_types.go +++ b/ray-operator/apis/ray/v1/rayjob_types.go @@ -96,6 +96,15 @@ type SubmitterConfig struct { // BackoffLimit of the submitter k8s job. // +optional BackoffLimit *int32 `json:"backoffLimit,omitempty"` + // +optional + Resources *SubmitterResources `json:"resources,omitempty"` +} + +type SubmitterResources struct { + // +optional + Requests *corev1.ResourceList `json:"requests,omitempty"` + // +optional + Limits *corev1.ResourceList `json:"limits,omitempty"` } // RayJobSpec defines the desired state of RayJob diff --git a/ray-operator/config/crd/bases/ray.io_rayjobs.yaml b/ray-operator/config/crd/bases/ray.io_rayjobs.yaml index 37ea7e676e2..9138cf0e1e5 100644 --- a/ray-operator/config/crd/bases/ray.io_rayjobs.yaml +++ b/ray-operator/config/crd/bases/ray.io_rayjobs.yaml @@ -8077,6 +8077,25 @@ spec: backoffLimit: format: int32 type: integer + resources: + 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 + 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 + type: object + type: object type: object submitterPodTemplate: properties: diff --git a/ray-operator/controllers/ray/common/job.go b/ray-operator/controllers/ray/common/job.go index b47421bade1..9cc246d1b81 100644 --- a/ray-operator/controllers/ray/common/job.go +++ b/ray-operator/controllers/ray/common/job.go @@ -123,8 +123,8 @@ func GetK8sJobCommand(rayJobInstance *rayv1.RayJob) ([]string, error) { } // GetDefaultSubmitterTemplate creates a default submitter template for the Ray job. -func GetDefaultSubmitterTemplate(rayClusterInstance *rayv1.RayCluster) corev1.PodTemplateSpec { - return corev1.PodTemplateSpec{ +func GetDefaultSubmitterTemplate(rayClusterInstance *rayv1.RayCluster, rayJob *rayv1.RayJob) corev1.PodTemplateSpec { + pod := corev1.PodTemplateSpec{ Spec: corev1.PodSpec{ Containers: []corev1.Container{ { @@ -146,4 +146,13 @@ func GetDefaultSubmitterTemplate(rayClusterInstance *rayv1.RayCluster) corev1.Po RestartPolicy: corev1.RestartPolicyNever, }, } + if rayJob != nil && rayJob.Spec.SubmitterConfig != nil && rayJob.Spec.SubmitterConfig.Resources != nil { + if rayJob.Spec.SubmitterConfig.Resources.Requests != nil { + pod.Spec.Containers[0].Resources.Requests = *rayJob.Spec.SubmitterConfig.Resources.Requests + } + if rayJob.Spec.SubmitterConfig.Resources.Limits != nil { + pod.Spec.Containers[0].Resources.Limits = *rayJob.Spec.SubmitterConfig.Resources.Limits + } + } + return pod } diff --git a/ray-operator/controllers/ray/rayjob_controller.go b/ray-operator/controllers/ray/rayjob_controller.go index b7a700ecd40..64e82ded0c9 100644 --- a/ray-operator/controllers/ray/rayjob_controller.go +++ b/ray-operator/controllers/ray/rayjob_controller.go @@ -527,7 +527,7 @@ func getSubmitterTemplate(ctx context.Context, rayJobInstance *rayv1.RayJob, ray // Set the default value for the optional field SubmitterPodTemplate if not provided. if rayJobInstance.Spec.SubmitterPodTemplate == nil { - submitterTemplate = common.GetDefaultSubmitterTemplate(rayClusterInstance) + submitterTemplate = common.GetDefaultSubmitterTemplate(rayClusterInstance, rayJobInstance) logger.Info("default submitter template is used") } else { submitterTemplate = *rayJobInstance.Spec.SubmitterPodTemplate.DeepCopy()