From 3c6a9dad30e3a07cb9a72626fc002b8e558323e7 Mon Sep 17 00:00:00 2001 From: Kenny Han Date: Mon, 23 Jun 2025 03:28:40 +0000 Subject: [PATCH 01/34] init cron sheduling for ray jobs, transferring for local dev --- ray-operator/apis/ray/v1/rayjob_types.go | 8 ++ .../config/crd/bases/ray.io_rayjobs.yaml | 5 + .../controllers/ray/rayjob_controller.go | 29 +++- .../controllers/ray/utils/cron_helpers.go | 132 ++++++++++++++++++ ray-operator/go.mod | 1 + ray-operator/go.sum | 2 + .../applyconfiguration/ray/v1/rayjobspec.go | 9 ++ .../applyconfiguration/ray/v1/rayjobstatus.go | 9 ++ .../test/e2e/rayjob_scheduling_test.go | 105 ++++++++++++++ 9 files changed, 298 insertions(+), 2 deletions(-) create mode 100644 ray-operator/controllers/ray/utils/cron_helpers.go create mode 100644 ray-operator/test/e2e/rayjob_scheduling_test.go diff --git a/ray-operator/apis/ray/v1/rayjob_types.go b/ray-operator/apis/ray/v1/rayjob_types.go index 1f9f58d238d..4763da9cc53 100644 --- a/ray-operator/apis/ray/v1/rayjob_types.go +++ b/ray-operator/apis/ray/v1/rayjob_types.go @@ -54,6 +54,7 @@ const ( JobDeploymentStatusSuspended JobDeploymentStatus = "Suspended" JobDeploymentStatusRetrying JobDeploymentStatus = "Retrying" JobDeploymentStatusWaiting JobDeploymentStatus = "Waiting" + JobDeploymentStatusScheduled JobDeploymentStatus = "Scheduled" ) // IsJobDeploymentTerminal returns true if the given JobDeploymentStatus @@ -193,6 +194,9 @@ type RayJobSpec struct { // In case of transition to false a new RayCluster will be created. // +optional Suspend bool `json:"suspend,omitempty"` + // Schedule specifies a cron like string for cron scheduling + // +optional + Schedule string `json:"schedule,omitempty"` } // RayJobStatus defines the observed state of RayJob @@ -240,6 +244,10 @@ type RayJobStatus struct { // RayJob's generation, which is updated on mutation by the API Server. // +optional ObservedGeneration int64 `json:"observedGeneration,omitempty"` + + // Information when was the last time the job was successfully scheduled. + // +optional + LastScheduleTime *metav1.Time `json:"lastScheduleTime,omitempty" protobuf:"bytes,4,opt,name=lastScheduleTime"` } // +kubebuilder:object:root=true diff --git a/ray-operator/config/crd/bases/ray.io_rayjobs.yaml b/ray-operator/config/crd/bases/ray.io_rayjobs.yaml index f3dc33c7ef4..82051e0cd85 100644 --- a/ray-operator/config/crd/bases/ray.io_rayjobs.yaml +++ b/ray-operator/config/crd/bases/ray.io_rayjobs.yaml @@ -8067,6 +8067,8 @@ spec: type: object runtimeEnvYAML: type: string + schedule: + type: string shutdownAfterJobFinishes: type: boolean submissionMode: @@ -11770,6 +11772,9 @@ spec: type: string jobStatus: type: string + lastScheduleTime: + format: date-time + type: string message: type: string observedGeneration: diff --git a/ray-operator/controllers/ray/rayjob_controller.go b/ray-operator/controllers/ray/rayjob_controller.go index bc452c3bcb1..7adf8a2e43b 100644 --- a/ray-operator/controllers/ray/rayjob_controller.go +++ b/ray-operator/controllers/ray/rayjob_controller.go @@ -9,6 +9,7 @@ import ( "time" "github.com/go-logr/logr" + "github.com/robfig/cron/v3" batchv1 "k8s.io/api/batch/v1" corev1 "k8s.io/api/core/v1" "k8s.io/apimachinery/pkg/api/errors" @@ -34,6 +35,7 @@ const ( RayJobDefaultRequeueDuration = 3 * time.Second RayJobDefaultClusterSelectorKey = "ray.io/cluster" PythonUnbufferedEnvVarName = "PYTHONUNBUFFERED" + nextScheduleDelta = 100 * time.Millisecond ) // RayJobReconciler reconciles a RayJob object @@ -429,9 +431,32 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) return ctrl.Result{RequeueAfter: RayJobDefaultRequeueDuration}, err } } + if rayJobInstance.Spec.Schedule != "" { + // If the rayjob has cron scheduling then we should change status to schedule for the next job + rayJobInstance.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusNew + } else { + // If the RayJob is completed without scheduling, we should not requeue it. + return ctrl.Result{}, nil + } + case rayv1.JobDeploymentStatusScheduled: + cron_schedule, err := cron.ParseStandard(utils.FormatSchedule(rayJobInstance, r.Recorder)) + if err != nil { + // this is likely a user error in defining the spec value + // we should log the error and not reconcile this cronjob until an update to spec + logger.Error(err, "The cron schedule provided is unparseable") + r.Recorder.Eventf(rayJobInstance, corev1.EventTypeWarning, "UnparseableSchedule", "unparseable schedule: %q : %s", rayJobInstance.Spec.Schedule, err) + return ctrl.Result{}, nil + } - // If the RayJob is completed, we should not requeue it. - return ctrl.Result{}, nil + t := utils.NextScheduleTimeDuration(rayJobInstance, time.Now(), cron_schedule) + if *t <= nextScheduleDelta { + logger.Info("Next schedule (%v) is WITHIN the %v delta. Initiating schedule!\n", *t, nextScheduleDelta) + rayJobInstance.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusNew + + } else { + logger.Info("Next schedule (%v) is BEYOND the %v delta. Will wait.\n", *t, nextScheduleDelta) + return ctrl.Result{RequeueAfter: *t}, err + } default: logger.Info("Unknown JobDeploymentStatus", "JobDeploymentStatus", rayJobInstance.Status.JobDeploymentStatus) return ctrl.Result{RequeueAfter: RayJobDefaultRequeueDuration}, nil diff --git a/ray-operator/controllers/ray/utils/cron_helpers.go b/ray-operator/controllers/ray/utils/cron_helpers.go new file mode 100644 index 00000000000..c5c59d5f9bd --- /dev/null +++ b/ray-operator/controllers/ray/utils/cron_helpers.go @@ -0,0 +1,132 @@ +package utils + +import ( + "fmt" + "strings" + "time" + + rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1" + "github.com/robfig/cron/v3" + corev1 "k8s.io/api/core/v1" + "k8s.io/client-go/tools/record" +) + +type missedSchedulesType int + +const ( + noneMissed missedSchedulesType = iota + fewMissed + manyMissed + nextScheduleDelta = 100 * time.Millisecond +) + +func mostRecentScheduleTime(rj *rayv1.RayJob, now time.Time, schedule cron.Schedule, includeStartingDeadlineSeconds bool) (time.Time, *time.Time, missedSchedulesType, error) { + earliestTime := rj.ObjectMeta.CreationTimestamp.Time + missedSchedules := noneMissed + if rj.Status.LastScheduleTime != nil { + earliestTime = rj.Status.LastScheduleTime.Time + } + + t1 := schedule.Next(earliestTime) + t2 := schedule.Next(t1) + + if now.Before(t1) { + return earliestTime, nil, missedSchedules, nil + } + if now.Before(t2) { + return earliestTime, &t1, missedSchedules, nil + } + + // It is possible for cron.ParseStandard("59 23 31 2 *") to return an invalid schedule + // minute - 59, hour - 23, dom - 31, month - 2, and dow is optional, clearly 31 is invalid + // In this case the timeBetweenTwoSchedules will be 0, and we error out the invalid schedule + timeBetweenTwoSchedules := int64(t2.Sub(t1).Round(time.Second).Seconds()) + if timeBetweenTwoSchedules < 1 { + return earliestTime, nil, missedSchedules, fmt.Errorf("time difference between two schedules is less than 1 second") + } + // this logic used for calculating number of missed schedules does a rough + // approximation, by calculating a diff between two schedules (t1 and t2), + // and counting how many of these will fit in between last schedule and now + timeElapsed := int64(now.Sub(t1).Seconds()) + numberOfMissedSchedules := (timeElapsed / timeBetweenTwoSchedules) + 1 + + var mostRecentTime time.Time + // to get the most recent time accurate for regular schedules and the ones + // specified with @every form, we first need to calculate the potential earliest + // time by multiplying the initial number of missed schedules by its interval, + // this is critical to ensure @every starts at the correct time, this explains + // the numberOfMissedSchedules-1, the additional -1 serves there to go back + // in time one more time unit, and let the cron library calculate a proper + // schedule, for case where the schedule is not consistent, for example + // something like 30 6-16/4 * * 1-5 + potentialEarliest := t1.Add(time.Duration((numberOfMissedSchedules-1-1)*timeBetweenTwoSchedules) * time.Second) + for t := schedule.Next(potentialEarliest); !t.After(now); t = schedule.Next(t) { + mostRecentTime = t + } + + // An object might miss several starts. For example, if + // controller gets wedged on friday at 5:01pm when everyone has + // gone home, and someone comes in on tuesday AM and discovers + // the problem and restarts the controller, then all the hourly + // jobs, more than 80 of them for one hourly cronJob, should + // all start running with no further intervention (if the cronJob + // allows concurrency and late starts). + // + // However, if there is a bug somewhere, or incorrect clock + // on controller's server or apiservers (for setting creationTimestamp) + // then there could be so many missed start times (it could be off + // by decades or more), that it would eat up all the CPU and memory + // of this controller. In that case, we want to not try to list + // all the missed start times. + // + // I've somewhat arbitrarily picked 100, as more than 80, + // but less than "lots". + switch { + case numberOfMissedSchedules > 100: + missedSchedules = manyMissed + // inform about few missed, still + case numberOfMissedSchedules > 0: + missedSchedules = fewMissed + } + + if mostRecentTime.IsZero() { + return earliestTime, nil, missedSchedules, nil + } + return earliestTime, &mostRecentTime, missedSchedules, nil +} + +func FormatSchedule(rj *rayv1.RayJob, recorder record.EventRecorder) string { + if strings.Contains(rj.Spec.Schedule, "TZ") { + if recorder != nil { + recorder.Eventf(rj, corev1.EventTypeWarning, "UnsupportedSchedule", "CRON_TZ or TZ used in schedule %q is not officially supported, see https://kubernetes.io/docs/concepts/workloads/controllers/cron-jobs/ for more details", rj.Spec.Schedule) + } + + return rj.Spec.Schedule + } + + return rj.Spec.Schedule +} + +// nextScheduleTimeDuration returns the time duration to requeue based on +// the schedule and last schedule time. It adds a 100ms padding to the next requeue to account +// for Network Time Protocol(NTP) time skews. If the time drifts the adjustment, which in most +// realistic cases should be around 100s, the job will still be executed without missing +// the schedule. +func NextScheduleTimeDuration(rj *rayv1.RayJob, now time.Time, schedule cron.Schedule) *time.Duration { + earliestTime, mostRecentTime, missedSchedules, err := mostRecentScheduleTime(rj, now, schedule, false) + if err != nil { + // we still have to requeue at some point, so aim for the next scheduling slot from now + mostRecentTime = &now + } else if mostRecentTime == nil { + if missedSchedules == noneMissed { + // no missed schedules since earliestTime + mostRecentTime = &earliestTime + } else { + // if there are missed schedules since earliestTime, always use now + mostRecentTime = &now + } + } + + t := schedule.Next(*mostRecentTime).Add(nextScheduleDelta).Sub(now) + return &t +} diff --git a/ray-operator/go.mod b/ray-operator/go.mod index 5e08ec1ccbe..7a36cd440f0 100644 --- a/ray-operator/go.mod +++ b/ray-operator/go.mod @@ -13,6 +13,7 @@ require ( github.com/orcaman/concurrent-map/v2 v2.0.1 github.com/pkg/errors v0.9.1 github.com/prometheus/client_golang v1.22.0 + github.com/robfig/cron/v3 v3.0.1 github.com/stretchr/testify v1.10.0 go.uber.org/mock v0.5.2 go.uber.org/zap v1.27.0 diff --git a/ray-operator/go.sum b/ray-operator/go.sum index ae91463230d..7d8b4961f78 100644 --- a/ray-operator/go.sum +++ b/ray-operator/go.sum @@ -111,6 +111,8 @@ github.com/prometheus/common v0.62.0 h1:xasJaQlnWAeyHdUBeGjXmutelfJHWMRr+Fg4QszZ github.com/prometheus/common v0.62.0/go.mod h1:vyBcEuLSvWos9B1+CyL7JZ2up+uFzXhkqml0W5zIY1I= github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc= github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= +github.com/robfig/cron/v3 v3.0.1 h1:WdRxkvbJztn8LMz/QEvLN5sBU+xKpSqwwUO1Pjr4qDs= +github.com/robfig/cron/v3 v3.0.1/go.mod h1:eQICP3HwyT7UooqI/z+Ov+PtYAWygg1TEWWzGIFLtro= github.com/rogpeppe/go-internal v1.13.1 h1:KvO1DLK/DRN07sQ1LQKScxyZJuNnedQ5/wKSR38lUII= github.com/rogpeppe/go-internal v1.13.1/go.mod h1:uMEvuHeurkdAXX61udpOXGD/AzZDWNMNyH2VO9fmH0o= github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA= diff --git a/ray-operator/pkg/client/applyconfiguration/ray/v1/rayjobspec.go b/ray-operator/pkg/client/applyconfiguration/ray/v1/rayjobspec.go index 702361a84c3..c21f86bb814 100644 --- a/ray-operator/pkg/client/applyconfiguration/ray/v1/rayjobspec.go +++ b/ray-operator/pkg/client/applyconfiguration/ray/v1/rayjobspec.go @@ -29,6 +29,7 @@ type RayJobSpecApplyConfiguration struct { TTLSecondsAfterFinished *int32 `json:"ttlSecondsAfterFinished,omitempty"` ShutdownAfterJobFinishes *bool `json:"shutdownAfterJobFinishes,omitempty"` Suspend *bool `json:"suspend,omitempty"` + Schedule *string `json:"schedule,omitempty"` } // RayJobSpecApplyConfiguration constructs a declarative configuration of the RayJobSpec type for use with @@ -200,3 +201,11 @@ func (b *RayJobSpecApplyConfiguration) WithSuspend(value bool) *RayJobSpecApplyC b.Suspend = &value return b } + +// WithSchedule sets the Schedule field in the declarative configuration to the given value +// and returns the receiver, so that objects can be built by chaining "With" function invocations. +// If called multiple times, the Schedule field is set to the value of the last call. +func (b *RayJobSpecApplyConfiguration) WithSchedule(value string) *RayJobSpecApplyConfiguration { + b.Schedule = &value + return b +} diff --git a/ray-operator/pkg/client/applyconfiguration/ray/v1/rayjobstatus.go b/ray-operator/pkg/client/applyconfiguration/ray/v1/rayjobstatus.go index de1e5fdba42..858f0827051 100644 --- a/ray-operator/pkg/client/applyconfiguration/ray/v1/rayjobstatus.go +++ b/ray-operator/pkg/client/applyconfiguration/ray/v1/rayjobstatus.go @@ -24,6 +24,7 @@ type RayJobStatusApplyConfiguration struct { Failed *int32 `json:"failed,omitempty"` RayClusterStatus *RayClusterStatusApplyConfiguration `json:"rayClusterStatus,omitempty"` ObservedGeneration *int64 `json:"observedGeneration,omitempty"` + LastScheduleTime *metav1.Time `json:"lastScheduleTime,omitempty"` } // RayJobStatusApplyConfiguration constructs a declarative configuration of the RayJobStatus type for use with @@ -143,3 +144,11 @@ func (b *RayJobStatusApplyConfiguration) WithObservedGeneration(value int64) *Ra b.ObservedGeneration = &value return b } + +// WithLastScheduleTime sets the LastScheduleTime field in the declarative configuration to the given value +// and returns the receiver, so that objects can be built by chaining "With" function invocations. +// If called multiple times, the LastScheduleTime field is set to the value of the last call. +func (b *RayJobStatusApplyConfiguration) WithLastScheduleTime(value metav1.Time) *RayJobStatusApplyConfiguration { + b.LastScheduleTime = &value + return b +} diff --git a/ray-operator/test/e2e/rayjob_scheduling_test.go b/ray-operator/test/e2e/rayjob_scheduling_test.go new file mode 100644 index 00000000000..9bf20df1c59 --- /dev/null +++ b/ray-operator/test/e2e/rayjob_scheduling_test.go @@ -0,0 +1,105 @@ +package e2e + +import ( + "testing" + "time" + + . "github.com/onsi/gomega" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + + rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1" + rayv1ac "github.com/ray-project/kuberay/ray-operator/pkg/client/applyconfiguration/ray/v1" + . "github.com/ray-project/kuberay/ray-operator/test/support" +) + +func TestRayJobScheduling(t *testing.T) { + test := With(t) + g := NewWithT(t) + + // Create a namespace + namespace := test.NewTestNamespace() + + jobsAC := newConfigMap(namespace.Name, files(test, "counter.py", "fail.py")) + jobs, err := test.Client().Core().CoreV1().ConfigMaps(namespace.Name).Apply(test.Ctx(), jobsAC, TestApplyOptions) + g.Expect(err).NotTo(HaveOccurred()) + LogWithTimestamp(test.T(), "Created ConfigMap %s/%s successfully", jobs.Namespace, jobs.Name) + + test.T().Run("Sucessful RayJob scheduling WITHOUT deleting cluster after each job", func(_ *testing.T) { + rayJobAC := rayv1ac.RayJob("counter", namespace.Name). + WithSpec(rayv1ac.RayJobSpec(). + WithRayClusterSpec(newRayClusterSpec(mountConfigMap[rayv1ac.RayClusterSpecApplyConfiguration](jobs, "/home/ray/jobs"))). + WithEntrypoint("python /home/ray/jobs/counter.py"). + WithRuntimeEnvYAML(` + env_vars: + counter_name: test_counter + `). + WithShutdownAfterJobFinishes(true). + WithSchedule("*/1 * * * *"). + WithSubmitterPodTemplate(jobSubmitterPodTemplateApplyConfiguration())) + + rayJob, err := test.Client().Ray().RayV1().RayJobs(namespace.Name).Apply(test.Ctx(), rayJobAC, TestApplyOptions) + g.Expect(err).NotTo(HaveOccurred()) + LogWithTimestamp(test.T(), "Created RayJob with scheduling %s/%s successfully", rayJob.Namespace, rayJob.Name) + + numExpectedRuns := int32(3) + LogWithTimestamp(test.T(), "Waiting for %d successful runs of RayJob %s/%s", numExpectedRuns, rayJob.Namespace, rayJob.Name) + + g.Eventually(RayJob(test, rayJob.Namespace, rayJob.Name), TestTimeoutLong*2). // Give it a generous timeout (e.g., 2-3 minutes) + Should(WithTransform(RayJobSucceeded, Equal(numExpectedRuns))) + + expectedCompleteCounts := 3 + var lastScheduleTime *metav1.Time + for i := 0; i < expectedCompleteCounts; i++ { + LogWithTimestamp(test.T(), "--- Verifying Cycle %d: Waiting for JobDeploymentStatus %s ---", i+1, rayv1.JobDeploymentStatusComplete) + + // 1. Wait until the job reaches 'JobDeploymentStatusComplete' for the current cycle + g.Eventually(RayCluster(test, rayJob.Namespace, rayJob.Name), TestTimeoutMedium). + Should(WithTransform(RayJobDeploymentStatus, Equal(rayv1.JobDeploymentStatusComplete))) + + // 2. Get the latest RayJob instance to check its status fields + currentRayJob, err := GetRayJob(test, rayJob.Namespace, rayJob.Name) + g.Expect(err).NotTo(HaveOccurred()) + g.Expect(currentRayJob.Status.LastScheduleTime).NotTo(BeNil()) // Ensure LastScheduleTime is populated + + // 3. Verify LastScheduleTime advancement (if not the very first cycle) + if lastScheduleTime != nil { + durationSinceLastScheduled := currentRayJob.Status.LastScheduleTime.Time.Sub(lastScheduleTime.Time) + LogWithTimestamp(test.T(), "Observed LastScheduleTime advanced by: %v (expected ~1m)", durationSinceLastScheduled) + // Allow a buffer for timing variations (e.g., 1 minute +/- 15 seconds) + g.Expect(durationSinceLastScheduled).To(BeNumerically("~", 1*time.Minute, 15*time.Second)) + } + // Update lastScheduleTime for the next iteration + lastScheduleTime = currentRayJob.Status.LastScheduleTime + + // 4. If more cycles are expected, wait for the status to reset (from Complete to New/Scheduled) + // This is crucial for the next g.Eventually to "see" a state change from 'New'/'Scheduled' back to 'Complete'. + if i < expectedCompleteCounts-1 { + LogWithTimestamp(test.T(), "Waiting for JobDeploymentStatus to reset for next cycle %d", i+2) + g.Eventually(RayJob(test, rayJob.Namespace, rayJob.Name), TestTimeoutShort). // Should be a quick transition + Should(WithTransform(RayJobDeploymentStatus, SatisfyAny( + Equal(rayv1.JobDeploymentStatusNew), + Equal(rayv1.JobDeploymentStatusScheduled), + ))) + } + + } + LogWithTimestamp(test.T(), "Successfully observed RayJob %s/%s reach %s %d times.", rayJob.Name, rayJob.Namespace, rayv1.JobDeploymentStatusComplete, expectedCompleteCounts) + err = test.Client().Ray().RayV1().RayJobs(namespace.Name).Delete(test.Ctx(), rayJob.Name, metav1.DeleteOptions{}) + g.Expect(err).NotTo(HaveOccurred()) + LogWithTimestamp(test.T(), "Deleted RayJob %s/%s successfully", rayJob.Namespace, rayJob.Name) + + }) + + test.T().Run("Sucessful RayJob scheduling WITH deleting cluster after each job", func(_ *testing.T) { + + }) + + test.T().Run("Overlapping RayJobs", func(_ *testing.T) { + + }) + + test.T().Run("Bad Cron String", func(_ *testing.T) { + + }) + +} From 0d986a90cc1f4f7edb4731f4d82f81a72660ec83 Mon Sep 17 00:00:00 2001 From: DW-Han Date: Thu, 26 Jun 2025 08:05:22 +0000 Subject: [PATCH 02/34] initial cron scheduling for ray jobs --- docs/reference/api.md | 1 + go.mod | 1 + go.sum | 2 + .../kuberay-operator/crds/ray.io_rayjobs.yaml | 5 + ray-operator/apis/ray/v1/rayjob_types.go | 4 +- .../config/samples/ray-job.schedule.yaml | 149 ++++++++++++++++ .../controllers/ray/rayjob_controller.go | 46 +++-- .../controllers/ray/utils/cron_helper.go | 160 ++++++++++++++++++ ray-operator/local_deploy.sh | 92 ++++++++++ .../test/e2e/rayjob_scheduling_test.go | 151 ++++++++++++++++- 10 files changed, 596 insertions(+), 15 deletions(-) create mode 100644 ray-operator/config/samples/ray-job.schedule.yaml create mode 100644 ray-operator/controllers/ray/utils/cron_helper.go create mode 100755 ray-operator/local_deploy.sh diff --git a/docs/reference/api.md b/docs/reference/api.md index db0d31e9b85..ca4e2c71b37 100644 --- a/docs/reference/api.md +++ b/docs/reference/api.md @@ -220,6 +220,7 @@ _Appears in:_ | `ttlSecondsAfterFinished` _integer_ | TTLSecondsAfterFinished is the TTL to clean up RayCluster.
It's only working when ShutdownAfterJobFinishes set to true. | 0 | | | `shutdownAfterJobFinishes` _boolean_ | ShutdownAfterJobFinishes will determine whether to delete the ray cluster once rayJob succeed or failed. | | | | `suspend` _boolean_ | 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. | | | +| `schedule` _string_ | Schedule specifies a cron like string for cron scheduling | | | diff --git a/go.mod b/go.mod index 9bada76c68a..f7024c39de1 100644 --- a/go.mod +++ b/go.mod @@ -87,6 +87,7 @@ require ( github.com/prometheus/client_model v0.6.1 // indirect github.com/prometheus/common v0.62.0 // indirect github.com/prometheus/procfs v0.15.1 // indirect + github.com/robfig/cron/v3 v3.0.0 // indirect github.com/russross/blackfriday/v2 v2.1.0 // indirect github.com/x448/float16 v0.8.4 // indirect github.com/xlab/treeprint v1.2.0 // indirect diff --git a/go.sum b/go.sum index dddab9f7e86..47baee1d63f 100644 --- a/go.sum +++ b/go.sum @@ -184,6 +184,8 @@ github.com/prometheus/common v0.62.0 h1:xasJaQlnWAeyHdUBeGjXmutelfJHWMRr+Fg4QszZ github.com/prometheus/common v0.62.0/go.mod h1:vyBcEuLSvWos9B1+CyL7JZ2up+uFzXhkqml0W5zIY1I= github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc= github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= +github.com/robfig/cron/v3 v3.0.0 h1:kQ6Cb7aHOHTSzNVNEhmp8EcWKLb4CbiMW9h9VyIhO4E= +github.com/robfig/cron/v3 v3.0.0/go.mod h1:eQICP3HwyT7UooqI/z+Ov+PtYAWygg1TEWWzGIFLtro= github.com/rogpeppe/go-internal v1.13.1 h1:KvO1DLK/DRN07sQ1LQKScxyZJuNnedQ5/wKSR38lUII= github.com/rogpeppe/go-internal v1.13.1/go.mod h1:uMEvuHeurkdAXX61udpOXGD/AzZDWNMNyH2VO9fmH0o= github.com/rs/cors v1.11.1 h1:eU3gRzXLRK57F5rKMGMZURNdIG4EoAmX8k94r9wXWHA= diff --git a/helm-chart/kuberay-operator/crds/ray.io_rayjobs.yaml b/helm-chart/kuberay-operator/crds/ray.io_rayjobs.yaml index f3dc33c7ef4..82051e0cd85 100644 --- a/helm-chart/kuberay-operator/crds/ray.io_rayjobs.yaml +++ b/helm-chart/kuberay-operator/crds/ray.io_rayjobs.yaml @@ -8067,6 +8067,8 @@ spec: type: object runtimeEnvYAML: type: string + schedule: + type: string shutdownAfterJobFinishes: type: boolean submissionMode: @@ -11770,6 +11772,9 @@ spec: type: string jobStatus: type: string + lastScheduleTime: + format: date-time + type: string message: type: string observedGeneration: diff --git a/ray-operator/apis/ray/v1/rayjob_types.go b/ray-operator/apis/ray/v1/rayjob_types.go index 4763da9cc53..5425da2a196 100644 --- a/ray-operator/apis/ray/v1/rayjob_types.go +++ b/ray-operator/apis/ray/v1/rayjob_types.go @@ -21,6 +21,7 @@ const ( JobStatusStopped JobStatus = "STOPPED" JobStatusSucceeded JobStatus = "SUCCEEDED" JobStatusFailed JobStatus = "FAILED" + JobStatusScheduled JobStatus = "SCHEDULED" ) var AllJobStatuses = []JobStatus{ @@ -54,6 +55,7 @@ const ( JobDeploymentStatusSuspended JobDeploymentStatus = "Suspended" JobDeploymentStatusRetrying JobDeploymentStatus = "Retrying" JobDeploymentStatusWaiting JobDeploymentStatus = "Waiting" + JobDeploymentStatusScheduling JobDeploymentStatus = "Scheduling" JobDeploymentStatusScheduled JobDeploymentStatus = "Scheduled" ) @@ -245,7 +247,7 @@ type RayJobStatus struct { // +optional ObservedGeneration int64 `json:"observedGeneration,omitempty"` - // Information when was the last time the job was successfully scheduled. + // The last time the job was successfully scheduled. // +optional LastScheduleTime *metav1.Time `json:"lastScheduleTime,omitempty" protobuf:"bytes,4,opt,name=lastScheduleTime"` } diff --git a/ray-operator/config/samples/ray-job.schedule.yaml b/ray-operator/config/samples/ray-job.schedule.yaml new file mode 100644 index 00000000000..7ff703aed0f --- /dev/null +++ b/ray-operator/config/samples/ray-job.schedule.yaml @@ -0,0 +1,149 @@ +apiVersion: ray.io/v1 +kind: RayJob +metadata: + name: rayjob-schedule +spec: + schedule: "* * * * *" + # submissionMode specifies how RayJob submits the Ray job to the RayCluster. + # The default value is "K8sJobMode", meaning RayJob will submit the Ray job via a submitter Kubernetes Job. + # The alternative value is "HTTPMode", indicating that KubeRay will submit the Ray job by sending an HTTP request to the RayCluster. + # submissionMode: "K8sJobMode" + entrypoint: python /home/ray/samples/sample_code.py + # shutdownAfterJobFinishes specifies whether the RayCluster should be deleted after the RayJob finishes. Default is false. + # shutdownAfterJobFinishes: false + + # ttlSecondsAfterFinished specifies the number of seconds after which the RayCluster will be deleted after the RayJob finishes. + # ttlSecondsAfterFinished: 10 + + # 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. + # activeDeadlineSeconds: 120 + + # RuntimeEnvYAML represents the runtime environment configuration provided as a multi-line YAML string. + # See https://docs.ray.io/en/latest/ray-core/handling-dependencies.html for details. + # (New in KubeRay version 1.0.) + runtimeEnvYAML: | + pip: + - requests==2.26.0 + - pendulum==2.1.2 + env_vars: + counter_name: "test_counter" + + # 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 we will wait for the transition to false. + # If the RayCluster is already created, it will be deleted. In the case of transition to false, a new RayCluster will be created. + # suspend: false + + # rayClusterSpec specifies the RayCluster instance to be created by the RayJob controller. + rayClusterSpec: + rayVersion: '2.46.0' # should match the Ray version in the image of the containers + # Ray head pod template + headGroupSpec: + # The `rayStartParams` are used to configure the `ray start` command. + # See https://github.com/ray-project/kuberay/blob/master/docs/guidance/rayStartParams.md for the default settings of `rayStartParams` in KubeRay. + # See https://docs.ray.io/en/latest/cluster/cli.html#ray-start for all available options in `rayStartParams`. + rayStartParams: {} + #pod template + template: + spec: + containers: + - name: ray-head + image: rayproject/ray:2.46.0 + ports: + - containerPort: 6379 + name: gcs-server + - containerPort: 8265 # Ray dashboard + name: dashboard + - containerPort: 10001 + name: client + resources: + limits: + cpu: "1" + requests: + cpu: "200m" + volumeMounts: + - mountPath: /home/ray/samples + name: code-sample + volumes: + # You set volumes at the Pod level, then mount them into containers inside that Pod + - name: code-sample + configMap: + # Provide the name of the ConfigMap you want to mount. + name: ray-job-code-sample + # An array of keys from the ConfigMap to create as files + items: + - key: sample_code.py + path: sample_code.py + workerGroupSpecs: + # the pod replicas in this group typed worker + - replicas: 1 + minReplicas: 1 + maxReplicas: 5 + # logical group name, for this called small-group, also can be functional + groupName: small-group + # The `rayStartParams` are used to configure the `ray start` command. + # See https://github.com/ray-project/kuberay/blob/master/docs/guidance/rayStartParams.md for the default settings of `rayStartParams` in KubeRay. + # See https://docs.ray.io/en/latest/cluster/cli.html#ray-start for all available options in `rayStartParams`. + rayStartParams: {} + #pod template + template: + spec: + containers: + - name: ray-worker # must consist of lower case alphanumeric characters or '-', and must start and end with an alphanumeric character (e.g. 'my-name', or '123-abc' + image: rayproject/ray:2.46.0 + resources: + limits: + cpu: "1" + requests: + cpu: "200m" + # SubmitterPodTemplate is the template for the pod that will run the `ray job submit` command against the RayCluster. + # If SubmitterPodTemplate is specified, the first container is assumed to be the submitter container. + # submitterPodTemplate: + # spec: + # restartPolicy: Never + # containers: + # - name: my-custom-rayjob-submitter-pod + # image: rayproject/ray:2.46.0 + # # If Command is not specified, the correct command will be supplied at runtime using the RayJob spec `entrypoint` field. + # # Specifying Command is not recommended. + # # command: ["sh", "-c", "ray job submit --address=http://$RAY_DASHBOARD_ADDRESS --submission-id=$RAY_JOB_SUBMISSION_ID -- echo hello world"] + + +######################Ray code sample################################# +# this sample is from https://docs.ray.io/en/latest/cluster/job-submission.html#quick-start-example +# it is mounted into the container and executed to show the Ray job at work +--- +apiVersion: v1 +kind: ConfigMap +metadata: + name: ray-job-code-sample +data: + sample_code.py: | + import ray + import os + import requests + + ray.init() + + @ray.remote + class Counter: + def __init__(self): + # Used to verify runtimeEnv + self.name = os.getenv("counter_name") + assert self.name == "test_counter" + self.counter = 0 + + def inc(self): + self.counter += 1 + + def get_counter(self): + return "{} got {}".format(self.name, self.counter) + + counter = Counter.remote() + + for _ in range(5): + ray.get(counter.inc.remote()) + print(ray.get(counter.get_counter.remote())) + + # Verify that the correct runtime env was used for the job. + assert requests.__version__ == "2.26.0" diff --git a/ray-operator/controllers/ray/rayjob_controller.go b/ray-operator/controllers/ray/rayjob_controller.go index 7adf8a2e43b..b47c5757809 100644 --- a/ray-operator/controllers/ray/rayjob_controller.go +++ b/ray-operator/controllers/ray/rayjob_controller.go @@ -35,7 +35,7 @@ const ( RayJobDefaultRequeueDuration = 3 * time.Second RayJobDefaultClusterSelectorKey = "ray.io/cluster" PythonUnbufferedEnvVarName = "PYTHONUNBUFFERED" - nextScheduleDelta = 100 * time.Millisecond + ScheduleDelta = 1000 * time.Millisecond ) // RayJobReconciler reconciles a RayJob object @@ -321,7 +321,7 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) if jobInfo.EndTime != 0 { rayJobInstance.Status.RayJobStatusInfo.EndTime = &metav1.Time{Time: time.UnixMilli(utils.SafeUint64ToInt64(jobInfo.EndTime))} } - case rayv1.JobDeploymentStatusSuspending, rayv1.JobDeploymentStatusRetrying: + case rayv1.JobDeploymentStatusSuspending, rayv1.JobDeploymentStatusRetrying, rayv1.JobDeploymentStatusScheduling: // The `suspend` operation should be atomic. In other words, if users set the `suspend` flag to true and then immediately // set it back to false, either all of the RayJob's associated resources should be cleaned up, or no resources should be // cleaned up at all. To keep the atomicity, if a RayJob is in the `Suspending` status, we should delete all of its @@ -355,10 +355,12 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) // Reset the JobStatus to JobStatusNew and transition the JobDeploymentStatus to `Suspended`. rayJobInstance.Status.JobStatus = rayv1.JobStatusNew - if rayJobInstance.Status.JobDeploymentStatus == rayv1.JobDeploymentStatusSuspending { + switch rayJobInstance.Status.JobDeploymentStatus { + case rayv1.JobDeploymentStatusSuspending: rayJobInstance.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusSuspended - } - if rayJobInstance.Status.JobDeploymentStatus == rayv1.JobDeploymentStatusRetrying { + case rayv1.JobDeploymentStatusScheduling: + rayJobInstance.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusScheduled + case rayv1.JobDeploymentStatusRetrying: rayJobInstance.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusNew } case rayv1.JobDeploymentStatusSuspended: @@ -433,30 +435,47 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) } if rayJobInstance.Spec.Schedule != "" { // If the rayjob has cron scheduling then we should change status to schedule for the next job - rayJobInstance.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusNew + logger.Info("RayJob is scheduled again") + rayJobInstance.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusScheduling } else { // If the RayJob is completed without scheduling, we should not requeue it. + logger.Info("RayJob is not scheduled") return ctrl.Result{}, nil } case rayv1.JobDeploymentStatusScheduled: + if rayJobInstance.Status.JobStatus == rayv1.JobStatusScheduled { + logger.Info("We are have reached the new time for a job after reconciling") + rayJobInstance.Status.JobStatus = rayv1.JobStatusNew + rayJobInstance.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusNew + break + } + + logger.Info("We are Scheduling for the next RayJob") cron_schedule, err := cron.ParseStandard(utils.FormatSchedule(rayJobInstance, r.Recorder)) if err != nil { // this is likely a user error in defining the spec value // we should log the error and not reconcile this cronjob until an update to spec logger.Error(err, "The cron schedule provided is unparseable") r.Recorder.Eventf(rayJobInstance, corev1.EventTypeWarning, "UnparseableSchedule", "unparseable schedule: %q : %s", rayJobInstance.Spec.Schedule, err) - return ctrl.Result{}, nil + return ctrl.Result{}, err } + logger.Info("Successfully parsed cron schedule", "CronSchedule", cron_schedule) - t := utils.NextScheduleTimeDuration(rayJobInstance, time.Now(), cron_schedule) - if *t <= nextScheduleDelta { - logger.Info("Next schedule (%v) is WITHIN the %v delta. Initiating schedule!\n", *t, nextScheduleDelta) - rayJobInstance.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusNew + t1 := utils.NextScheduleTimeDuration(logger, rayJobInstance, time.Now(), cron_schedule) + t2 := utils.LastScheduleTimeDuration(logger, rayJobInstance, time.Now(), cron_schedule) + logger.Info("Got the time until we run the next RayJob", "NextScheduleTimeDuration", t1.Abs()) + if *t1 < ScheduleDelta || t2 < ScheduleDelta { + logger.Info("The current time is within the buffer window of a cron tick", "NextScheduleTimeDuration", t1, "LastScheduleTimeDuration", t2) + rayJobInstance.Status.JobStatus = rayv1.JobStatusScheduled } else { - logger.Info("Next schedule (%v) is BEYOND the %v delta. Will wait.\n", *t, nextScheduleDelta) - return ctrl.Result{RequeueAfter: *t}, err + logger.Info("We wait for until the next and reconcile", "Time till next job", t1, "Current Time", time.Now()) + + rayJobInstance.Status.LastScheduleTime = &metav1.Time{Time: time.Now()} + + return ctrl.Result{RequeueAfter: *t1}, nil } + default: logger.Info("Unknown JobDeploymentStatus", "JobDeploymentStatus", rayJobInstance.Status.JobDeploymentStatus) return ctrl.Result{RequeueAfter: RayJobDefaultRequeueDuration}, nil @@ -470,6 +489,7 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) return ctrl.Result{RequeueAfter: RayJobDefaultRequeueDuration}, err } emitRayJobMetrics(r.options.RayJobMetricsManager, rayJobInstance.Name, rayJobInstance.Namespace, originalRayJobInstance.Status, rayJobInstance.Status) + return ctrl.Result{RequeueAfter: RayJobDefaultRequeueDuration}, nil } diff --git a/ray-operator/controllers/ray/utils/cron_helper.go b/ray-operator/controllers/ray/utils/cron_helper.go new file mode 100644 index 00000000000..34460524a70 --- /dev/null +++ b/ray-operator/controllers/ray/utils/cron_helper.go @@ -0,0 +1,160 @@ +package utils + +import ( + "fmt" + "strings" + "time" + + "github.com/go-logr/logr" + rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1" + "github.com/robfig/cron/v3" + corev1 "k8s.io/api/core/v1" + "k8s.io/client-go/tools/record" +) + +type missedSchedulesType int + +const ( + noneMissed missedSchedulesType = iota + fewMissed + manyMissed + nextScheduleDelta = 100 * time.Millisecond +) + +func mostRecentScheduleTime(logger logr.Logger, rj *rayv1.RayJob, now time.Time, schedule cron.Schedule, includeStartingDeadlineSeconds bool) (time.Time, *time.Time, missedSchedulesType, error) { + earliestTime := rj.ObjectMeta.CreationTimestamp.Time + missedSchedules := noneMissed + if rj.Status.LastScheduleTime != nil { + earliestTime = rj.Status.LastScheduleTime.Time + } + + t1 := schedule.Next(earliestTime) + t2 := schedule.Next(t1) + + if now.Before(t1) { + return earliestTime, nil, missedSchedules, nil + } + if now.Before(t2) { + return earliestTime, &t1, missedSchedules, nil + } + + // It is possible for cron.ParseStandard("59 23 31 2 *") to return an invalid schedule + // minute - 59, hour - 23, dom - 31, month - 2, and dow is optional, clearly 31 is invalid + // In this case the timeBetweenTwoSchedules will be 0, and we error out the invalid schedule + timeBetweenTwoSchedules := int64(t2.Sub(t1).Round(time.Second).Seconds()) + if timeBetweenTwoSchedules < 1 { + return earliestTime, nil, missedSchedules, fmt.Errorf("time difference between two schedules is less than 1 second") + } + // this logic used for calculating number of missed schedules does a rough + // approximation, by calculating a diff between two schedules (t1 and t2), + // and counting how many of these will fit in between last schedule and now + timeElapsed := int64(now.Sub(t1).Seconds()) + numberOfMissedSchedules := (timeElapsed / timeBetweenTwoSchedules) + 1 + + var mostRecentTime time.Time + // to get the most recent time accurate for regular schedules and the ones + // specified with @every form, we first need to calculate the potential earliest + // time by multiplying the initial number of missed schedules by its interval, + // this is critical to ensure @every starts at the correct time, this explains + // the numberOfMissedSchedules-1, the additional -1 serves there to go back + // in time one more time unit, and let the cron library calculate a proper + // schedule, for case where the schedule is not consistent, for example + // something like 30 6-16/4 * * 1-5 + potentialEarliest := t1.Add(time.Duration((numberOfMissedSchedules-1-1)*timeBetweenTwoSchedules) * time.Second) + for t := schedule.Next(potentialEarliest); !t.After(now); t = schedule.Next(t) { + mostRecentTime = t + } + + // An object might miss several starts. For example, if + // controller gets wedged on friday at 5:01pm when everyone has + // gone home, and someone comes in on tuesday AM and discovers + // the problem and restarts the controller, then all the hourly + // jobs, more than 80 of them for one hourly cronJob, should + // all start running with no further intervention (if the cronJob + // allows concurrency and late starts). + // + // However, if there is a bug somewhere, or incorrect clock + // on controller's server or apiservers (for setting creationTimestamp) + // then there could be so many missed start times (it could be off + // by decades or more), that it would eat up all the CPU and memory + // of this controller. In that case, we want to not try to list + // all the missed start times. + // + // I've somewhat arbitrarily picked 100, as more than 80, + // but less than "lots". + switch { + case numberOfMissedSchedules > 100: + missedSchedules = manyMissed + // inform about few missed, still + case numberOfMissedSchedules > 0: + missedSchedules = fewMissed + } + + if mostRecentTime.IsZero() { + return earliestTime, nil, missedSchedules, nil + } + return earliestTime, &mostRecentTime, missedSchedules, nil +} + +func FormatSchedule(rj *rayv1.RayJob, recorder record.EventRecorder) string { + if strings.Contains(rj.Spec.Schedule, "TZ") { + if recorder != nil { + recorder.Eventf(rj, corev1.EventTypeWarning, "UnsupportedSchedule", "CRON_TZ or TZ used in schedule %q is not officially supported, see https://kubernetes.io/docs/concepts/workloads/controllers/cron-jobs/ for more details", rj.Spec.Schedule) + } + + return rj.Spec.Schedule + } + + return rj.Spec.Schedule +} + +// nextScheduleTimeDuration returns the time duration to requeue based on +// the schedule and last schedule time. It adds a 100ms padding to the next requeue to account +// for Network Time Protocol(NTP) time skews. If the time drifts the adjustment, which in most +// realistic cases should be around 100s, the job will still be executed without missing +// the schedule. +func NextScheduleTimeDuration(logger logr.Logger, rj *rayv1.RayJob, now time.Time, schedule cron.Schedule) *time.Duration { + + earliestTime, mostRecentTime, missedSchedules, err := mostRecentScheduleTime(logger, rj, now, schedule, false) + if err != nil { + // we still have to requeue at some point, so aim for the next scheduling slot from now + logger.Info("Error in mostRecentScheduleTime, we still have to requeue at some point, so aim for the next scheduling slot from now", "Error", err) + mostRecentTime = &now + } else if mostRecentTime == nil { + logger.Info("mostRecentTime doesnt exist", "mostRecentTime", mostRecentTime, "earliestTime", earliestTime) + if missedSchedules == noneMissed { + // no missed schedules since earliestTime + mostRecentTime = &earliestTime + } else { + // if there are missed schedules since earliestTime, always use now + mostRecentTime = &now + } + } + logger.Info("Successfully calculated earliestTime and mostRecentTime", "mostRecentTime", mostRecentTime, "earliestTime", earliestTime, "Next time to aim for", schedule.Next(*mostRecentTime)) + t := schedule.Next(*mostRecentTime).Add(nextScheduleDelta).Sub(now) + return &t +} + +func LastScheduleTimeDuration(logger logr.Logger, rj *rayv1.RayJob, now time.Time, schedule cron.Schedule) time.Duration { + // The mostRecentScheduleTime function is the core logic that calculates the most + // recent time a schedule should have executed based on the RayJob's creation time + // (or its last scheduled status) and the current time 'now'. + // The second return value, 'mostRecentTime', is precisely the "last previous cron time" + // that we want to find. We can ignore the other return values for this function's purpose. + earliestTime, mostRecentTime, missedSchedules, err := mostRecentScheduleTime(logger, rj, now, schedule, false) + if err != nil { + // we still have to requeue at some point, so aim for the next scheduling slot from now + logger.Info("Error in mostRecentScheduleTime, we still have to requeue at some point", "Error", err) + mostRecentTime = &now + } else if mostRecentTime == nil { + logger.Info("mostRecentTime doesnt exist", "mostRecentTime", mostRecentTime, "earliestTime", earliestTime) + if missedSchedules == noneMissed { + // no missed schedules since earliestTime + mostRecentTime = &earliestTime + } else { + // if there are missed schedules since earliestTime, always use now + mostRecentTime = &now + } + } + return now.Sub(*mostRecentTime) +} diff --git a/ray-operator/local_deploy.sh b/ray-operator/local_deploy.sh new file mode 100755 index 00000000000..54a2f3c7434 --- /dev/null +++ b/ray-operator/local_deploy.sh @@ -0,0 +1,92 @@ +#!/bin/bash + +# set -euxo pipefail: +# -e: Exit immediately if a command exits with a non-zero status. +# -u: Treat unset variables as an error when substituting. +# -x: Print commands and their arguments as they are executed (useful for debugging). +# -o pipefail: The return value of a pipeline is the status of the last command to exit with a non-zero status, +# or zero if all commands in the pipeline exit successfully. +set -euxo pipefail + +# --- Configuration Variables --- +# IMPORTANT: Customize these variables for your environment and project. +# Use your Docker registry and a unique image name. +# For Kind, if not pushing to a remote registry, you can use "kind-registry" or similar. +IMAGE_REPO="yourregistry" # e.g., "gcr.io/my-gcp-project" or "docker.io/myusername" +IMAGE_NAME="kuberay-operator" +# Set your desired image tag. A unique tag like a timestamp is recommended during development. +IMAGE_TAG="nightly" # Example: "my-custom-build-20250625-1030" or "nightly" + +KIND_CLUSTER_NAME="kind" # Your Kind cluster name +OPERATOR_NAMESPACE="default" # Or "kuberay-system", "kuberay-operator", etc. +HELM_RELEASE_NAME="kuberay-operator" # The Helm release name you use + +HELM_CHART_PATH="../helm-chart/kuberay-operator" # Path to your Helm chart + +# --- Script Logic --- + +echo "--- Checking for Kind Cluster ---" +# Check if the Kind cluster already exists +if ! kind get clusters | grep -q "${KIND_CLUSTER_NAME}"; then + echo "Kind cluster '${KIND_CLUSTER_NAME}' not found. Creating it..." + # You can customize your Kind cluster creation command here if needed. + kind create cluster --name "${KIND_CLUSTER_NAME}" --image=kindest/node:v1.24.0 +else + echo "Kind cluster '${KIND_CLUSTER_NAME}' already exists. Skipping creation." +fi + +# Step 2: Modify KubeRay source code (Manual Step) +# For example, add a log by adding setupLog.Info("Hello KubeRay") in the function `main` in `main.go`. + +echo "--- Building Docker Image ---" +FULL_IMAGE_NAME="${IMAGE_REPO}/${IMAGE_NAME}:${IMAGE_TAG}" +echo "Building image: ${FULL_IMAGE_NAME}" + +# Delete existing local image to ensure a fresh build and avoid caching issues. +# The '|| true' prevents the script from exiting if the image doesn't exist yet. +docker rmi -f "${FULL_IMAGE_NAME}" || true + +# Step 3: Build an image +# This command will copy the source code directory into the image, and build it. +# Command: IMG={IMG_REPO}:{IMG_TAG} make docker-build +make docker-build IMG="${FULL_IMAGE_NAME}" + +# To skip Go project compilation, run the following command instead: +# IMG=kuberay/operator:nightly make docker-image + +echo "--- Loading Image into Kind Cluster ---" +# Step 4: Load the custom KubeRay image into the Kind cluster. +# Command: kind load docker-image {IMG_REPO}:{IMG_TAG} +kind load docker-image "${FULL_IMAGE_NAME}" --name "${KIND_CLUSTER_NAME}" + +echo "--- Uninstalling and Installing KubeRay Operator ---" +# Check if the Helm release exists before trying to uninstall +if helm list --namespace "${OPERATOR_NAMESPACE}" | grep -q "${HELM_RELEASE_NAME}"; then + echo "Uninstalling existing Helm release: ${HELM_RELEASE_NAME}" + helm uninstall "${HELM_RELEASE_NAME}" --namespace "${OPERATOR_NAMESPACE}" + echo "Waiting for resources to be terminated..." + sleep 10 # Give Kubernetes some time to clean up +else + echo "Helm release '${HELM_RELEASE_NAME}' not found. Skipping uninstall." +fi + +# Step 6: Install KubeRay operator with the custom image via local Helm chart +# (Path: helm-chart/kuberay-operator) +# Command: helm install kuberay-operator --set image.repository={IMG_REPO} --set image.tag={IMG_TAG} ../helm-chart/kuberay-operator +echo "Installing new Helm release: ${HELM_RELEASE_NAME}" +helm install "${HELM_RELEASE_NAME}" "${HELM_CHART_PATH}" \ + --namespace "${OPERATOR_NAMESPACE}" \ + --set "image.repository=${IMAGE_REPO}/${IMAGE_NAME}" \ + --set "image.tag=${IMAGE_TAG}" + +echo "--- Waiting for Deployment Rollout ---" +# Wait for the operator deployment to successfully roll out. +# This ensures the new pod is running before we check logs. +kubectl rollout status deployment "${HELM_RELEASE_NAME}" --namespace "${OPERATOR_NAMESPACE}" --timeout=5m + +echo "--- Streaming Controller Logs (Ctrl+C to stop) ---" +# Step 7: Check the logs +# Note: This command directly targets the deployment for logs. +# kubectl logs -f deployments/"${HELM_RELEASE_NAME}" -n "${OPERATOR_NAMESPACE}" + +echo "--- Script Completed ---" diff --git a/ray-operator/test/e2e/rayjob_scheduling_test.go b/ray-operator/test/e2e/rayjob_scheduling_test.go index 9bf20df1c59..3217fc6e2a5 100644 --- a/ray-operator/test/e2e/rayjob_scheduling_test.go +++ b/ray-operator/test/e2e/rayjob_scheduling_test.go @@ -34,7 +34,7 @@ func TestRayJobScheduling(t *testing.T) { counter_name: test_counter `). WithShutdownAfterJobFinishes(true). - WithSchedule("*/1 * * * *"). + WithSchedule("*/3 * * * *"). WithSubmitterPodTemplate(jobSubmitterPodTemplateApplyConfiguration())) rayJob, err := test.Client().Ray().RayV1().RayJobs(namespace.Name).Apply(test.Ctx(), rayJobAC, TestApplyOptions) @@ -83,6 +83,7 @@ func TestRayJobScheduling(t *testing.T) { } } + LogWithTimestamp(test.T(), "Successfully observed RayJob %s/%s reach %s %d times.", rayJob.Name, rayJob.Namespace, rayv1.JobDeploymentStatusComplete, expectedCompleteCounts) err = test.Client().Ray().RayV1().RayJobs(namespace.Name).Delete(test.Ctx(), rayJob.Name, metav1.DeleteOptions{}) g.Expect(err).NotTo(HaveOccurred()) @@ -91,14 +92,162 @@ func TestRayJobScheduling(t *testing.T) { }) test.T().Run("Sucessful RayJob scheduling WITH deleting cluster after each job", func(_ *testing.T) { + rayJobAC := rayv1ac.RayJob("counter", namespace.Name). + WithSpec(rayv1ac.RayJobSpec(). + WithRayClusterSpec(newRayClusterSpec(mountConfigMap[rayv1ac.RayClusterSpecApplyConfiguration](jobs, "/home/ray/jobs"))). + WithEntrypoint("python /home/ray/jobs/counter.py"). + WithRuntimeEnvYAML(` + env_vars: + counter_name: test_counter + `). + WithShutdownAfterJobFinishes(true). + WithSchedule("*/3 * * * *"). + WithSubmitterPodTemplate(jobSubmitterPodTemplateApplyConfiguration())) + + rayJob, err := test.Client().Ray().RayV1().RayJobs(namespace.Name).Apply(test.Ctx(), rayJobAC, TestApplyOptions) + g.Expect(err).NotTo(HaveOccurred()) + LogWithTimestamp(test.T(), "Created RayJob with scheduling %s/%s successfully", rayJob.Namespace, rayJob.Name) + + numExpectedRuns := int32(3) + LogWithTimestamp(test.T(), "Waiting for %d successful runs of RayJob %s/%s", numExpectedRuns, rayJob.Namespace, rayJob.Name) + + g.Eventually(RayJob(test, rayJob.Namespace, rayJob.Name), TestTimeoutLong*2). // Give it a generous timeout (e.g., 2-3 minutes) + Should(WithTransform(RayJobSucceeded, Equal(numExpectedRuns))) + + expectedCompleteCounts := 3 + var lastScheduleTime *metav1.Time + for i := 0; i < expectedCompleteCounts; i++ { + LogWithTimestamp(test.T(), "--- Verifying Cycle %d: Waiting for JobDeploymentStatus %s ---", i+1, rayv1.JobDeploymentStatusComplete) + + // 1. Wait until the job reaches 'JobDeploymentStatusComplete' for the current cycle + g.Eventually(RayCluster(test, rayJob.Namespace, rayJob.Name), TestTimeoutMedium). + Should(WithTransform(RayJobDeploymentStatus, Equal(rayv1.JobDeploymentStatusComplete))) + + // 2. Get the latest RayJob instance to check its status fields + currentRayJob, err := GetRayJob(test, rayJob.Namespace, rayJob.Name) + g.Expect(err).NotTo(HaveOccurred()) + g.Expect(currentRayJob.Status.LastScheduleTime).NotTo(BeNil()) // Ensure LastScheduleTime is populated + + // 3. Verify LastScheduleTime advancement (if not the very first cycle) + if lastScheduleTime != nil { + durationSinceLastScheduled := currentRayJob.Status.LastScheduleTime.Time.Sub(lastScheduleTime.Time) + LogWithTimestamp(test.T(), "Observed LastScheduleTime advanced by: %v (expected ~1m)", durationSinceLastScheduled) + // Allow a buffer for timing variations (e.g., 1 minute +/- 15 seconds) + g.Expect(durationSinceLastScheduled).To(BeNumerically("~", 1*time.Minute, 15*time.Second)) + } + // Update lastScheduleTime for the next iteration + lastScheduleTime = currentRayJob.Status.LastScheduleTime + + // 4. If more cycles are expected, wait for the status to reset (from Complete to New/Scheduled) + // This is crucial for the next g.Eventually to "see" a state change from 'New'/'Scheduled' back to 'Complete'. + if i < expectedCompleteCounts-1 { + LogWithTimestamp(test.T(), "Waiting for JobDeploymentStatus to reset for next cycle %d", i+2) + g.Eventually(RayJob(test, rayJob.Namespace, rayJob.Name), TestTimeoutShort). // Should be a quick transition + Should(WithTransform(RayJobDeploymentStatus, SatisfyAny( + Equal(rayv1.JobDeploymentStatusNew), + Equal(rayv1.JobDeploymentStatusScheduled), + ))) + } + + } + + LogWithTimestamp(test.T(), "Successfully observed RayJob %s/%s reach %s %d times.", rayJob.Name, rayJob.Namespace, rayv1.JobDeploymentStatusComplete, expectedCompleteCounts) + err = test.Client().Ray().RayV1().RayJobs(namespace.Name).Delete(test.Ctx(), rayJob.Name, metav1.DeleteOptions{}) + g.Expect(err).NotTo(HaveOccurred()) + LogWithTimestamp(test.T(), "Deleted RayJob %s/%s successfully", rayJob.Namespace, rayJob.Name) }) test.T().Run("Overlapping RayJobs", func(_ *testing.T) { + rayJobAC := rayv1ac.RayJob("counter", namespace.Name). + WithSpec(rayv1ac.RayJobSpec(). + WithRayClusterSpec(newRayClusterSpec(mountConfigMap[rayv1ac.RayClusterSpecApplyConfiguration](jobs, "/home/ray/jobs"))). + WithEntrypoint("python /home/ray/jobs/long_running_counter.py"). + WithRuntimeEnvYAML(` + env_vars: + counter_name: test_counter + `). + WithShutdownAfterJobFinishes(false). + WithSchedule("*/1 * * * *"). + WithSubmitterPodTemplate(jobSubmitterPodTemplateApplyConfiguration())) + + rayJob, err := test.Client().Ray().RayV1().RayJobs(namespace.Name).Apply(test.Ctx(), rayJobAC, TestApplyOptions) + g.Expect(err).NotTo(HaveOccurred()) + LogWithTimestamp(test.T(), "Created RayJob with scheduling %s/%s successfully", rayJob.Namespace, rayJob.Name) + + numExpectedRuns := int32(3) + LogWithTimestamp(test.T(), "Waiting for %d successful runs of RayJob %s/%s", numExpectedRuns, rayJob.Namespace, rayJob.Name) + + g.Eventually(RayJob(test, rayJob.Namespace, rayJob.Name), TestTimeoutLong*2). // Give it a generous timeout (e.g., 2-3 minutes) + Should(WithTransform(RayJobSucceeded, Equal(numExpectedRuns))) + + expectedCompleteCounts := 3 + var lastScheduleTime *metav1.Time + for i := 0; i < expectedCompleteCounts; i++ { + LogWithTimestamp(test.T(), "--- Verifying Cycle %d: Waiting for JobDeploymentStatus %s ---", i+1, rayv1.JobDeploymentStatusComplete) + + // 1. Wait until the job reaches 'JobDeploymentStatusComplete' for the current cycle + g.Eventually(RayCluster(test, rayJob.Namespace, rayJob.Name), TestTimeoutMedium). + Should(WithTransform(RayJobDeploymentStatus, Equal(rayv1.JobDeploymentStatusComplete))) + + // 2. Get the latest RayJob instance to check its status fields + currentRayJob, err := GetRayJob(test, rayJob.Namespace, rayJob.Name) + g.Expect(err).NotTo(HaveOccurred()) + g.Expect(currentRayJob.Status.LastScheduleTime).NotTo(BeNil()) // Ensure LastScheduleTime is populated + + // 3. Verify LastScheduleTime advancement (if not the very first cycle) + if lastScheduleTime != nil { + durationSinceLastScheduled := currentRayJob.Status.LastScheduleTime.Time.Sub(lastScheduleTime.Time) + LogWithTimestamp(test.T(), "Observed LastScheduleTime advanced by: %v (expected ~1m)", durationSinceLastScheduled) + // Allow a buffer for timing variations (e.g., 1 minute +/- 15 seconds) + g.Expect(durationSinceLastScheduled).To(BeNumerically("~", 1*time.Minute, 15*time.Second)) + } + // Update lastScheduleTime for the next iteration + lastScheduleTime = currentRayJob.Status.LastScheduleTime + + // 4. If more cycles are expected, wait for the status to reset (from Complete to New/Scheduled) + // This is crucial for the next g.Eventually to "see" a state change from 'New'/'Scheduled' back to 'Complete'. + if i < expectedCompleteCounts-1 { + LogWithTimestamp(test.T(), "Waiting for JobDeploymentStatus to reset for next cycle %d", i+2) + g.Eventually(RayJob(test, rayJob.Namespace, rayJob.Name), TestTimeoutShort). // Should be a quick transition + Should(WithTransform(RayJobDeploymentStatus, SatisfyAny( + Equal(rayv1.JobDeploymentStatusNew), + Equal(rayv1.JobDeploymentStatusScheduled), + ))) + } + + } + + LogWithTimestamp(test.T(), "Successfully observed RayJob %s/%s reach %s %d times.", rayJob.Name, rayJob.Namespace, rayv1.JobDeploymentStatusComplete, expectedCompleteCounts) + err = test.Client().Ray().RayV1().RayJobs(namespace.Name).Delete(test.Ctx(), rayJob.Name, metav1.DeleteOptions{}) + g.Expect(err).NotTo(HaveOccurred()) + LogWithTimestamp(test.T(), "Deleted RayJob %s/%s successfully", rayJob.Namespace, rayJob.Name) }) test.T().Run("Bad Cron String", func(_ *testing.T) { + rayJobAC := rayv1ac.RayJob("counter", namespace.Name). + WithSpec(rayv1ac.RayJobSpec(). + WithRayClusterSpec(newRayClusterSpec(mountConfigMap[rayv1ac.RayClusterSpecApplyConfiguration](jobs, "/home/ray/jobs"))). + WithEntrypoint("python /home/ray/jobs/long_running_counter.py"). + WithRuntimeEnvYAML(` + env_vars: + counter_name: test_counter + `). + WithShutdownAfterJobFinishes(false). + WithSchedule("*(12*12)qw"). + WithSubmitterPodTemplate(jobSubmitterPodTemplateApplyConfiguration())) + + rayJob, err := test.Client().Ray().RayV1().RayJobs(namespace.Name).Apply(test.Ctx(), rayJobAC, TestApplyOptions) + g.Expect(err).NotTo(HaveOccurred()) + LogWithTimestamp(test.T(), "Created RayJob with scheduling %s/%s successfully", rayJob.Namespace, rayJob.Name) + + numExpectedRuns := int32(3) + LogWithTimestamp(test.T(), "Waiting for %d successful runs of RayJob %s/%s", numExpectedRuns, rayJob.Namespace, rayJob.Name) + + // `shedule` is not a valid cron string, so the RayJob controller will not do anything with the CR. + g.Consistently(RayJob(test, rayJob.Namespace, rayJob.Name), 5*time.Second). + Should(WithTransform(RayJobDeploymentStatus, Equal(rayv1.JobDeploymentStatusFailed))) }) From acdc061d6df5c63cc724371926453a86bdf431d8 Mon Sep 17 00:00:00 2001 From: Kenny Han Date: Thu, 26 Jun 2025 02:21:00 -0700 Subject: [PATCH 03/34] Delete ray-operator/controllers/ray/utils/cron_helpers.go Signed-off-by: Kenny Han --- .../controllers/ray/utils/cron_helpers.go | 132 ------------------ 1 file changed, 132 deletions(-) delete mode 100644 ray-operator/controllers/ray/utils/cron_helpers.go diff --git a/ray-operator/controllers/ray/utils/cron_helpers.go b/ray-operator/controllers/ray/utils/cron_helpers.go deleted file mode 100644 index c5c59d5f9bd..00000000000 --- a/ray-operator/controllers/ray/utils/cron_helpers.go +++ /dev/null @@ -1,132 +0,0 @@ -package utils - -import ( - "fmt" - "strings" - "time" - - rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1" - "github.com/robfig/cron/v3" - corev1 "k8s.io/api/core/v1" - "k8s.io/client-go/tools/record" -) - -type missedSchedulesType int - -const ( - noneMissed missedSchedulesType = iota - fewMissed - manyMissed - nextScheduleDelta = 100 * time.Millisecond -) - -func mostRecentScheduleTime(rj *rayv1.RayJob, now time.Time, schedule cron.Schedule, includeStartingDeadlineSeconds bool) (time.Time, *time.Time, missedSchedulesType, error) { - earliestTime := rj.ObjectMeta.CreationTimestamp.Time - missedSchedules := noneMissed - if rj.Status.LastScheduleTime != nil { - earliestTime = rj.Status.LastScheduleTime.Time - } - - t1 := schedule.Next(earliestTime) - t2 := schedule.Next(t1) - - if now.Before(t1) { - return earliestTime, nil, missedSchedules, nil - } - if now.Before(t2) { - return earliestTime, &t1, missedSchedules, nil - } - - // It is possible for cron.ParseStandard("59 23 31 2 *") to return an invalid schedule - // minute - 59, hour - 23, dom - 31, month - 2, and dow is optional, clearly 31 is invalid - // In this case the timeBetweenTwoSchedules will be 0, and we error out the invalid schedule - timeBetweenTwoSchedules := int64(t2.Sub(t1).Round(time.Second).Seconds()) - if timeBetweenTwoSchedules < 1 { - return earliestTime, nil, missedSchedules, fmt.Errorf("time difference between two schedules is less than 1 second") - } - // this logic used for calculating number of missed schedules does a rough - // approximation, by calculating a diff between two schedules (t1 and t2), - // and counting how many of these will fit in between last schedule and now - timeElapsed := int64(now.Sub(t1).Seconds()) - numberOfMissedSchedules := (timeElapsed / timeBetweenTwoSchedules) + 1 - - var mostRecentTime time.Time - // to get the most recent time accurate for regular schedules and the ones - // specified with @every form, we first need to calculate the potential earliest - // time by multiplying the initial number of missed schedules by its interval, - // this is critical to ensure @every starts at the correct time, this explains - // the numberOfMissedSchedules-1, the additional -1 serves there to go back - // in time one more time unit, and let the cron library calculate a proper - // schedule, for case where the schedule is not consistent, for example - // something like 30 6-16/4 * * 1-5 - potentialEarliest := t1.Add(time.Duration((numberOfMissedSchedules-1-1)*timeBetweenTwoSchedules) * time.Second) - for t := schedule.Next(potentialEarliest); !t.After(now); t = schedule.Next(t) { - mostRecentTime = t - } - - // An object might miss several starts. For example, if - // controller gets wedged on friday at 5:01pm when everyone has - // gone home, and someone comes in on tuesday AM and discovers - // the problem and restarts the controller, then all the hourly - // jobs, more than 80 of them for one hourly cronJob, should - // all start running with no further intervention (if the cronJob - // allows concurrency and late starts). - // - // However, if there is a bug somewhere, or incorrect clock - // on controller's server or apiservers (for setting creationTimestamp) - // then there could be so many missed start times (it could be off - // by decades or more), that it would eat up all the CPU and memory - // of this controller. In that case, we want to not try to list - // all the missed start times. - // - // I've somewhat arbitrarily picked 100, as more than 80, - // but less than "lots". - switch { - case numberOfMissedSchedules > 100: - missedSchedules = manyMissed - // inform about few missed, still - case numberOfMissedSchedules > 0: - missedSchedules = fewMissed - } - - if mostRecentTime.IsZero() { - return earliestTime, nil, missedSchedules, nil - } - return earliestTime, &mostRecentTime, missedSchedules, nil -} - -func FormatSchedule(rj *rayv1.RayJob, recorder record.EventRecorder) string { - if strings.Contains(rj.Spec.Schedule, "TZ") { - if recorder != nil { - recorder.Eventf(rj, corev1.EventTypeWarning, "UnsupportedSchedule", "CRON_TZ or TZ used in schedule %q is not officially supported, see https://kubernetes.io/docs/concepts/workloads/controllers/cron-jobs/ for more details", rj.Spec.Schedule) - } - - return rj.Spec.Schedule - } - - return rj.Spec.Schedule -} - -// nextScheduleTimeDuration returns the time duration to requeue based on -// the schedule and last schedule time. It adds a 100ms padding to the next requeue to account -// for Network Time Protocol(NTP) time skews. If the time drifts the adjustment, which in most -// realistic cases should be around 100s, the job will still be executed without missing -// the schedule. -func NextScheduleTimeDuration(rj *rayv1.RayJob, now time.Time, schedule cron.Schedule) *time.Duration { - earliestTime, mostRecentTime, missedSchedules, err := mostRecentScheduleTime(rj, now, schedule, false) - if err != nil { - // we still have to requeue at some point, so aim for the next scheduling slot from now - mostRecentTime = &now - } else if mostRecentTime == nil { - if missedSchedules == noneMissed { - // no missed schedules since earliestTime - mostRecentTime = &earliestTime - } else { - // if there are missed schedules since earliestTime, always use now - mostRecentTime = &now - } - } - - t := schedule.Next(*mostRecentTime).Add(nextScheduleDelta).Sub(now) - return &t -} From b4330c9fb4b77f5212b99751b8b0d0fd2d8ea91f Mon Sep 17 00:00:00 2001 From: DW-Han Date: Thu, 26 Jun 2025 09:56:14 +0000 Subject: [PATCH 04/34] updating dependencies --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index f7024c39de1..fcf935a5f21 100644 --- a/go.mod +++ b/go.mod @@ -87,7 +87,7 @@ require ( github.com/prometheus/client_model v0.6.1 // indirect github.com/prometheus/common v0.62.0 // indirect github.com/prometheus/procfs v0.15.1 // indirect - github.com/robfig/cron/v3 v3.0.0 // indirect + github.com/robfig/cron/v3 v3.0.1 // indirect github.com/russross/blackfriday/v2 v2.1.0 // indirect github.com/x448/float16 v0.8.4 // indirect github.com/xlab/treeprint v1.2.0 // indirect diff --git a/go.sum b/go.sum index 47baee1d63f..655b400f4dc 100644 --- a/go.sum +++ b/go.sum @@ -184,8 +184,8 @@ github.com/prometheus/common v0.62.0 h1:xasJaQlnWAeyHdUBeGjXmutelfJHWMRr+Fg4QszZ github.com/prometheus/common v0.62.0/go.mod h1:vyBcEuLSvWos9B1+CyL7JZ2up+uFzXhkqml0W5zIY1I= github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc= github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= -github.com/robfig/cron/v3 v3.0.0 h1:kQ6Cb7aHOHTSzNVNEhmp8EcWKLb4CbiMW9h9VyIhO4E= -github.com/robfig/cron/v3 v3.0.0/go.mod h1:eQICP3HwyT7UooqI/z+Ov+PtYAWygg1TEWWzGIFLtro= +github.com/robfig/cron/v3 v3.0.1 h1:WdRxkvbJztn8LMz/QEvLN5sBU+xKpSqwwUO1Pjr4qDs= +github.com/robfig/cron/v3 v3.0.1/go.mod h1:eQICP3HwyT7UooqI/z+Ov+PtYAWygg1TEWWzGIFLtro= github.com/rogpeppe/go-internal v1.13.1 h1:KvO1DLK/DRN07sQ1LQKScxyZJuNnedQ5/wKSR38lUII= github.com/rogpeppe/go-internal v1.13.1/go.mod h1:uMEvuHeurkdAXX61udpOXGD/AzZDWNMNyH2VO9fmH0o= github.com/rs/cors v1.11.1 h1:eU3gRzXLRK57F5rKMGMZURNdIG4EoAmX8k94r9wXWHA= From f7da0f3c7a98c14c6bdf068a8c0745a93babf136 Mon Sep 17 00:00:00 2001 From: Kenny Han Date: Thu, 26 Jun 2025 12:11:25 -0700 Subject: [PATCH 05/34] Delete ray-operator/test/e2e/rayjob_scheduling_test.go Signed-off-by: Kenny Han --- .../test/e2e/rayjob_scheduling_test.go | 254 ------------------ 1 file changed, 254 deletions(-) delete mode 100644 ray-operator/test/e2e/rayjob_scheduling_test.go diff --git a/ray-operator/test/e2e/rayjob_scheduling_test.go b/ray-operator/test/e2e/rayjob_scheduling_test.go deleted file mode 100644 index 3217fc6e2a5..00000000000 --- a/ray-operator/test/e2e/rayjob_scheduling_test.go +++ /dev/null @@ -1,254 +0,0 @@ -package e2e - -import ( - "testing" - "time" - - . "github.com/onsi/gomega" - metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" - - rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1" - rayv1ac "github.com/ray-project/kuberay/ray-operator/pkg/client/applyconfiguration/ray/v1" - . "github.com/ray-project/kuberay/ray-operator/test/support" -) - -func TestRayJobScheduling(t *testing.T) { - test := With(t) - g := NewWithT(t) - - // Create a namespace - namespace := test.NewTestNamespace() - - jobsAC := newConfigMap(namespace.Name, files(test, "counter.py", "fail.py")) - jobs, err := test.Client().Core().CoreV1().ConfigMaps(namespace.Name).Apply(test.Ctx(), jobsAC, TestApplyOptions) - g.Expect(err).NotTo(HaveOccurred()) - LogWithTimestamp(test.T(), "Created ConfigMap %s/%s successfully", jobs.Namespace, jobs.Name) - - test.T().Run("Sucessful RayJob scheduling WITHOUT deleting cluster after each job", func(_ *testing.T) { - rayJobAC := rayv1ac.RayJob("counter", namespace.Name). - WithSpec(rayv1ac.RayJobSpec(). - WithRayClusterSpec(newRayClusterSpec(mountConfigMap[rayv1ac.RayClusterSpecApplyConfiguration](jobs, "/home/ray/jobs"))). - WithEntrypoint("python /home/ray/jobs/counter.py"). - WithRuntimeEnvYAML(` - env_vars: - counter_name: test_counter - `). - WithShutdownAfterJobFinishes(true). - WithSchedule("*/3 * * * *"). - WithSubmitterPodTemplate(jobSubmitterPodTemplateApplyConfiguration())) - - rayJob, err := test.Client().Ray().RayV1().RayJobs(namespace.Name).Apply(test.Ctx(), rayJobAC, TestApplyOptions) - g.Expect(err).NotTo(HaveOccurred()) - LogWithTimestamp(test.T(), "Created RayJob with scheduling %s/%s successfully", rayJob.Namespace, rayJob.Name) - - numExpectedRuns := int32(3) - LogWithTimestamp(test.T(), "Waiting for %d successful runs of RayJob %s/%s", numExpectedRuns, rayJob.Namespace, rayJob.Name) - - g.Eventually(RayJob(test, rayJob.Namespace, rayJob.Name), TestTimeoutLong*2). // Give it a generous timeout (e.g., 2-3 minutes) - Should(WithTransform(RayJobSucceeded, Equal(numExpectedRuns))) - - expectedCompleteCounts := 3 - var lastScheduleTime *metav1.Time - for i := 0; i < expectedCompleteCounts; i++ { - LogWithTimestamp(test.T(), "--- Verifying Cycle %d: Waiting for JobDeploymentStatus %s ---", i+1, rayv1.JobDeploymentStatusComplete) - - // 1. Wait until the job reaches 'JobDeploymentStatusComplete' for the current cycle - g.Eventually(RayCluster(test, rayJob.Namespace, rayJob.Name), TestTimeoutMedium). - Should(WithTransform(RayJobDeploymentStatus, Equal(rayv1.JobDeploymentStatusComplete))) - - // 2. Get the latest RayJob instance to check its status fields - currentRayJob, err := GetRayJob(test, rayJob.Namespace, rayJob.Name) - g.Expect(err).NotTo(HaveOccurred()) - g.Expect(currentRayJob.Status.LastScheduleTime).NotTo(BeNil()) // Ensure LastScheduleTime is populated - - // 3. Verify LastScheduleTime advancement (if not the very first cycle) - if lastScheduleTime != nil { - durationSinceLastScheduled := currentRayJob.Status.LastScheduleTime.Time.Sub(lastScheduleTime.Time) - LogWithTimestamp(test.T(), "Observed LastScheduleTime advanced by: %v (expected ~1m)", durationSinceLastScheduled) - // Allow a buffer for timing variations (e.g., 1 minute +/- 15 seconds) - g.Expect(durationSinceLastScheduled).To(BeNumerically("~", 1*time.Minute, 15*time.Second)) - } - // Update lastScheduleTime for the next iteration - lastScheduleTime = currentRayJob.Status.LastScheduleTime - - // 4. If more cycles are expected, wait for the status to reset (from Complete to New/Scheduled) - // This is crucial for the next g.Eventually to "see" a state change from 'New'/'Scheduled' back to 'Complete'. - if i < expectedCompleteCounts-1 { - LogWithTimestamp(test.T(), "Waiting for JobDeploymentStatus to reset for next cycle %d", i+2) - g.Eventually(RayJob(test, rayJob.Namespace, rayJob.Name), TestTimeoutShort). // Should be a quick transition - Should(WithTransform(RayJobDeploymentStatus, SatisfyAny( - Equal(rayv1.JobDeploymentStatusNew), - Equal(rayv1.JobDeploymentStatusScheduled), - ))) - } - - } - - LogWithTimestamp(test.T(), "Successfully observed RayJob %s/%s reach %s %d times.", rayJob.Name, rayJob.Namespace, rayv1.JobDeploymentStatusComplete, expectedCompleteCounts) - err = test.Client().Ray().RayV1().RayJobs(namespace.Name).Delete(test.Ctx(), rayJob.Name, metav1.DeleteOptions{}) - g.Expect(err).NotTo(HaveOccurred()) - LogWithTimestamp(test.T(), "Deleted RayJob %s/%s successfully", rayJob.Namespace, rayJob.Name) - - }) - - test.T().Run("Sucessful RayJob scheduling WITH deleting cluster after each job", func(_ *testing.T) { - rayJobAC := rayv1ac.RayJob("counter", namespace.Name). - WithSpec(rayv1ac.RayJobSpec(). - WithRayClusterSpec(newRayClusterSpec(mountConfigMap[rayv1ac.RayClusterSpecApplyConfiguration](jobs, "/home/ray/jobs"))). - WithEntrypoint("python /home/ray/jobs/counter.py"). - WithRuntimeEnvYAML(` - env_vars: - counter_name: test_counter - `). - WithShutdownAfterJobFinishes(true). - WithSchedule("*/3 * * * *"). - WithSubmitterPodTemplate(jobSubmitterPodTemplateApplyConfiguration())) - - rayJob, err := test.Client().Ray().RayV1().RayJobs(namespace.Name).Apply(test.Ctx(), rayJobAC, TestApplyOptions) - g.Expect(err).NotTo(HaveOccurred()) - LogWithTimestamp(test.T(), "Created RayJob with scheduling %s/%s successfully", rayJob.Namespace, rayJob.Name) - - numExpectedRuns := int32(3) - LogWithTimestamp(test.T(), "Waiting for %d successful runs of RayJob %s/%s", numExpectedRuns, rayJob.Namespace, rayJob.Name) - - g.Eventually(RayJob(test, rayJob.Namespace, rayJob.Name), TestTimeoutLong*2). // Give it a generous timeout (e.g., 2-3 minutes) - Should(WithTransform(RayJobSucceeded, Equal(numExpectedRuns))) - - expectedCompleteCounts := 3 - var lastScheduleTime *metav1.Time - for i := 0; i < expectedCompleteCounts; i++ { - LogWithTimestamp(test.T(), "--- Verifying Cycle %d: Waiting for JobDeploymentStatus %s ---", i+1, rayv1.JobDeploymentStatusComplete) - - // 1. Wait until the job reaches 'JobDeploymentStatusComplete' for the current cycle - g.Eventually(RayCluster(test, rayJob.Namespace, rayJob.Name), TestTimeoutMedium). - Should(WithTransform(RayJobDeploymentStatus, Equal(rayv1.JobDeploymentStatusComplete))) - - // 2. Get the latest RayJob instance to check its status fields - currentRayJob, err := GetRayJob(test, rayJob.Namespace, rayJob.Name) - g.Expect(err).NotTo(HaveOccurred()) - g.Expect(currentRayJob.Status.LastScheduleTime).NotTo(BeNil()) // Ensure LastScheduleTime is populated - - // 3. Verify LastScheduleTime advancement (if not the very first cycle) - if lastScheduleTime != nil { - durationSinceLastScheduled := currentRayJob.Status.LastScheduleTime.Time.Sub(lastScheduleTime.Time) - LogWithTimestamp(test.T(), "Observed LastScheduleTime advanced by: %v (expected ~1m)", durationSinceLastScheduled) - // Allow a buffer for timing variations (e.g., 1 minute +/- 15 seconds) - g.Expect(durationSinceLastScheduled).To(BeNumerically("~", 1*time.Minute, 15*time.Second)) - } - // Update lastScheduleTime for the next iteration - lastScheduleTime = currentRayJob.Status.LastScheduleTime - - // 4. If more cycles are expected, wait for the status to reset (from Complete to New/Scheduled) - // This is crucial for the next g.Eventually to "see" a state change from 'New'/'Scheduled' back to 'Complete'. - if i < expectedCompleteCounts-1 { - LogWithTimestamp(test.T(), "Waiting for JobDeploymentStatus to reset for next cycle %d", i+2) - g.Eventually(RayJob(test, rayJob.Namespace, rayJob.Name), TestTimeoutShort). // Should be a quick transition - Should(WithTransform(RayJobDeploymentStatus, SatisfyAny( - Equal(rayv1.JobDeploymentStatusNew), - Equal(rayv1.JobDeploymentStatusScheduled), - ))) - } - - } - - LogWithTimestamp(test.T(), "Successfully observed RayJob %s/%s reach %s %d times.", rayJob.Name, rayJob.Namespace, rayv1.JobDeploymentStatusComplete, expectedCompleteCounts) - err = test.Client().Ray().RayV1().RayJobs(namespace.Name).Delete(test.Ctx(), rayJob.Name, metav1.DeleteOptions{}) - g.Expect(err).NotTo(HaveOccurred()) - LogWithTimestamp(test.T(), "Deleted RayJob %s/%s successfully", rayJob.Namespace, rayJob.Name) - - }) - - test.T().Run("Overlapping RayJobs", func(_ *testing.T) { - rayJobAC := rayv1ac.RayJob("counter", namespace.Name). - WithSpec(rayv1ac.RayJobSpec(). - WithRayClusterSpec(newRayClusterSpec(mountConfigMap[rayv1ac.RayClusterSpecApplyConfiguration](jobs, "/home/ray/jobs"))). - WithEntrypoint("python /home/ray/jobs/long_running_counter.py"). - WithRuntimeEnvYAML(` - env_vars: - counter_name: test_counter - `). - WithShutdownAfterJobFinishes(false). - WithSchedule("*/1 * * * *"). - WithSubmitterPodTemplate(jobSubmitterPodTemplateApplyConfiguration())) - - rayJob, err := test.Client().Ray().RayV1().RayJobs(namespace.Name).Apply(test.Ctx(), rayJobAC, TestApplyOptions) - g.Expect(err).NotTo(HaveOccurred()) - LogWithTimestamp(test.T(), "Created RayJob with scheduling %s/%s successfully", rayJob.Namespace, rayJob.Name) - - numExpectedRuns := int32(3) - LogWithTimestamp(test.T(), "Waiting for %d successful runs of RayJob %s/%s", numExpectedRuns, rayJob.Namespace, rayJob.Name) - - g.Eventually(RayJob(test, rayJob.Namespace, rayJob.Name), TestTimeoutLong*2). // Give it a generous timeout (e.g., 2-3 minutes) - Should(WithTransform(RayJobSucceeded, Equal(numExpectedRuns))) - - expectedCompleteCounts := 3 - var lastScheduleTime *metav1.Time - for i := 0; i < expectedCompleteCounts; i++ { - LogWithTimestamp(test.T(), "--- Verifying Cycle %d: Waiting for JobDeploymentStatus %s ---", i+1, rayv1.JobDeploymentStatusComplete) - - // 1. Wait until the job reaches 'JobDeploymentStatusComplete' for the current cycle - g.Eventually(RayCluster(test, rayJob.Namespace, rayJob.Name), TestTimeoutMedium). - Should(WithTransform(RayJobDeploymentStatus, Equal(rayv1.JobDeploymentStatusComplete))) - - // 2. Get the latest RayJob instance to check its status fields - currentRayJob, err := GetRayJob(test, rayJob.Namespace, rayJob.Name) - g.Expect(err).NotTo(HaveOccurred()) - g.Expect(currentRayJob.Status.LastScheduleTime).NotTo(BeNil()) // Ensure LastScheduleTime is populated - - // 3. Verify LastScheduleTime advancement (if not the very first cycle) - if lastScheduleTime != nil { - durationSinceLastScheduled := currentRayJob.Status.LastScheduleTime.Time.Sub(lastScheduleTime.Time) - LogWithTimestamp(test.T(), "Observed LastScheduleTime advanced by: %v (expected ~1m)", durationSinceLastScheduled) - // Allow a buffer for timing variations (e.g., 1 minute +/- 15 seconds) - g.Expect(durationSinceLastScheduled).To(BeNumerically("~", 1*time.Minute, 15*time.Second)) - } - // Update lastScheduleTime for the next iteration - lastScheduleTime = currentRayJob.Status.LastScheduleTime - - // 4. If more cycles are expected, wait for the status to reset (from Complete to New/Scheduled) - // This is crucial for the next g.Eventually to "see" a state change from 'New'/'Scheduled' back to 'Complete'. - if i < expectedCompleteCounts-1 { - LogWithTimestamp(test.T(), "Waiting for JobDeploymentStatus to reset for next cycle %d", i+2) - g.Eventually(RayJob(test, rayJob.Namespace, rayJob.Name), TestTimeoutShort). // Should be a quick transition - Should(WithTransform(RayJobDeploymentStatus, SatisfyAny( - Equal(rayv1.JobDeploymentStatusNew), - Equal(rayv1.JobDeploymentStatusScheduled), - ))) - } - - } - - LogWithTimestamp(test.T(), "Successfully observed RayJob %s/%s reach %s %d times.", rayJob.Name, rayJob.Namespace, rayv1.JobDeploymentStatusComplete, expectedCompleteCounts) - err = test.Client().Ray().RayV1().RayJobs(namespace.Name).Delete(test.Ctx(), rayJob.Name, metav1.DeleteOptions{}) - g.Expect(err).NotTo(HaveOccurred()) - LogWithTimestamp(test.T(), "Deleted RayJob %s/%s successfully", rayJob.Namespace, rayJob.Name) - - }) - - test.T().Run("Bad Cron String", func(_ *testing.T) { - rayJobAC := rayv1ac.RayJob("counter", namespace.Name). - WithSpec(rayv1ac.RayJobSpec(). - WithRayClusterSpec(newRayClusterSpec(mountConfigMap[rayv1ac.RayClusterSpecApplyConfiguration](jobs, "/home/ray/jobs"))). - WithEntrypoint("python /home/ray/jobs/long_running_counter.py"). - WithRuntimeEnvYAML(` - env_vars: - counter_name: test_counter - `). - WithShutdownAfterJobFinishes(false). - WithSchedule("*(12*12)qw"). - WithSubmitterPodTemplate(jobSubmitterPodTemplateApplyConfiguration())) - - rayJob, err := test.Client().Ray().RayV1().RayJobs(namespace.Name).Apply(test.Ctx(), rayJobAC, TestApplyOptions) - g.Expect(err).NotTo(HaveOccurred()) - LogWithTimestamp(test.T(), "Created RayJob with scheduling %s/%s successfully", rayJob.Namespace, rayJob.Name) - - numExpectedRuns := int32(3) - LogWithTimestamp(test.T(), "Waiting for %d successful runs of RayJob %s/%s", numExpectedRuns, rayJob.Namespace, rayJob.Name) - - // `shedule` is not a valid cron string, so the RayJob controller will not do anything with the CR. - g.Consistently(RayJob(test, rayJob.Namespace, rayJob.Name), 5*time.Second). - Should(WithTransform(RayJobDeploymentStatus, Equal(rayv1.JobDeploymentStatusFailed))) - - }) - -} From d2f294c1b0b17cc195e2b0a656fe686a9abc5c68 Mon Sep 17 00:00:00 2001 From: DW-Han Date: Mon, 7 Jul 2025 05:31:40 +0000 Subject: [PATCH 06/34] unit tests and cleaning up scheduling --- ray-operator/apis/ray/v1/rayjob_types.go | 2 +- .../config/samples/ray-job.schedule.yaml | 4 +- .../controllers/ray/rayjob_controller.go | 64 ++++++++----- .../ray/rayjob_controller_unit_test.go | 95 +++++++++++++++++++ .../ray/utils/{cron_helper.go => schedule.go} | 35 +++---- 5 files changed, 156 insertions(+), 44 deletions(-) rename ray-operator/controllers/ray/utils/{cron_helper.go => schedule.go} (80%) diff --git a/ray-operator/apis/ray/v1/rayjob_types.go b/ray-operator/apis/ray/v1/rayjob_types.go index 1a274d122a4..f8359b79635 100644 --- a/ray-operator/apis/ray/v1/rayjob_types.go +++ b/ray-operator/apis/ray/v1/rayjob_types.go @@ -258,7 +258,7 @@ type RayJobStatus struct { // The last time the job was successfully scheduled. // +optional - LastScheduleTime *metav1.Time `json:"lastScheduleTime,omitempty" protobuf:"bytes,4,opt,name=lastScheduleTime"` + LastScheduleTime *metav1.Time `json:"lastScheduleTime,omitempty"` } // +kubebuilder:object:root=true diff --git a/ray-operator/config/samples/ray-job.schedule.yaml b/ray-operator/config/samples/ray-job.schedule.yaml index 7ff703aed0f..c43618b628a 100644 --- a/ray-operator/config/samples/ray-job.schedule.yaml +++ b/ray-operator/config/samples/ray-job.schedule.yaml @@ -3,7 +3,9 @@ kind: RayJob metadata: name: rayjob-schedule spec: - schedule: "* * * * *" + # schedule specifires a cron scheduling string telling the rayjob when to start schedule and run new jobs + # Here it runs at every 5 minutes of every hour of every day of every week of every year + schedule: "*/5 * * * *" # submissionMode specifies how RayJob submits the Ray job to the RayCluster. # The default value is "K8sJobMode", meaning RayJob will submit the Ray job via a submitter Kubernetes Job. # The alternative value is "HTTPMode", indicating that KubeRay will submit the Ray job by sending an HTTP request to the RayCluster. diff --git a/ray-operator/controllers/ray/rayjob_controller.go b/ray-operator/controllers/ray/rayjob_controller.go index 3689003c4e8..fa5a6ce5385 100644 --- a/ray-operator/controllers/ray/rayjob_controller.go +++ b/ray-operator/controllers/ray/rayjob_controller.go @@ -35,7 +35,7 @@ const ( RayJobDefaultRequeueDuration = 3 * time.Second RayJobDefaultClusterSelectorKey = "ray.io/cluster" PythonUnbufferedEnvVarName = "PYTHONUNBUFFERED" - ScheduleDelta = 1000 * time.Millisecond + ScheduleDelta = 100 * time.Millisecond ) // RayJobReconciler reconciles a RayJob object @@ -90,6 +90,7 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) // Get RayJob instance var err error rayJobInstance := &rayv1.RayJob{} + if err := r.Get(ctx, request.NamespacedName, rayJobInstance); err != nil { if errors.IsNotFound(err) { // Request object not found, could have been deleted after reconcile request. Stop reconciliation. @@ -453,47 +454,39 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) return ctrl.Result{RequeueAfter: RayJobDefaultRequeueDuration}, err } } - if rayJobInstance.Spec.Schedule != "" { + if rayJobInstance.Spec.Schedule != "" && rayJobInstance.Status.JobDeploymentStatus != rayv1.JobDeploymentStatusFailed { // If the rayjob has cron scheduling then we should change status to schedule for the next job logger.Info("RayJob is scheduled again") rayJobInstance.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusScheduling } else { - // If the RayJob is completed without scheduling, we should not requeue it. + // NOTE: we do not requeue if the job fails, this could change + // If the RayJob is completed without scheduling or has failed, we should not requeue it. logger.Info("RayJob is not scheduled") return ctrl.Result{}, nil } case rayv1.JobDeploymentStatusScheduled: if rayJobInstance.Status.JobStatus == rayv1.JobStatusScheduled { - logger.Info("We are have reached the new time for a job after reconciling") + logger.Info("We have reached the new time for a job after reconciling") rayJobInstance.Status.JobStatus = rayv1.JobStatusNew rayJobInstance.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusNew break } - - logger.Info("We are Scheduling for the next RayJob") - cron_schedule, err := cron.ParseStandard(utils.FormatSchedule(rayJobInstance, r.Recorder)) + // We get the time from the current time to the previous and next cron schedule times + // We pass in time.Now() as a parameter so easier unit testing and consistency + t1, t2, err := r.getPreviousAndNextScheduleDistance(ctx, time.Now(), rayJobInstance) if err != nil { - // this is likely a user error in defining the spec value - // we should log the error and not reconcile this cronjob until an update to spec - logger.Error(err, "The cron schedule provided is unparseable") - r.Recorder.Eventf(rayJobInstance, corev1.EventTypeWarning, "UnparseableSchedule", "unparseable schedule: %q : %s", rayJobInstance.Spec.Schedule, err) + logger.Error(err, "Could not get the previous and next distances for a cron schedule") return ctrl.Result{}, err } - logger.Info("Successfully parsed cron schedule", "CronSchedule", cron_schedule) - - t1 := utils.NextScheduleTimeDuration(logger, rayJobInstance, time.Now(), cron_schedule) - t2 := utils.LastScheduleTimeDuration(logger, rayJobInstance, time.Now(), cron_schedule) - - logger.Info("Got the time until we run the next RayJob", "NextScheduleTimeDuration", t1.Abs()) - if *t1 < ScheduleDelta || t2 < ScheduleDelta { + // Checking if we are currently close enough to either the previous or next cron schedule times + if t1 < ScheduleDelta || t2 < ScheduleDelta { logger.Info("The current time is within the buffer window of a cron tick", "NextScheduleTimeDuration", t1, "LastScheduleTimeDuration", t2) rayJobInstance.Status.JobStatus = rayv1.JobStatusScheduled } else { - logger.Info("We wait for until the next and reconcile", "Time till next job", t1, "Current Time", time.Now()) - + logger.Info("We wait for until the next and reconcile", "Time till next job", t1, "Current Time", time.Now(), "LastScheduleTimeDuration", t2) rayJobInstance.Status.LastScheduleTime = &metav1.Time{Time: time.Now()} - return ctrl.Result{RequeueAfter: *t1}, nil + return ctrl.Result{RequeueAfter: t1}, nil } default: @@ -847,7 +840,14 @@ func initRayJobStatusIfNeed(ctx context.Context, rayJob *rayv1.RayJob) error { if rayJob.Status.JobStatus == "" { rayJob.Status.JobStatus = rayv1.JobStatusNew } - rayJob.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusInitializing + // if the rayjob is sheduled according to a cron string set the status to scheduling instead of initializing to begin with + // if know if this is the first job and not just another scheduled one we check the jobs count + if rayJob.Spec.Schedule != "" && rayJob.Status.Failed == nil && rayJob.Status.Succeeded == nil { + logger.Info("Initial schedule") + rayJob.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusScheduling + } else { + rayJob.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusInitializing + } rayJob.Status.StartTime = &metav1.Time{Time: time.Now()} return nil } @@ -939,6 +939,26 @@ func (r *RayJobReconciler) constructRayClusterForRayJob(rayJobInstance *rayv1.Ra return rayCluster, nil } +func (r *RayJobReconciler) getPreviousAndNextScheduleDistance(ctx context.Context, currentTime time.Time, rayJobInstance *rayv1.RayJob) (time.Duration, time.Duration, error) { + logger := ctrl.LoggerFrom(ctx) + logger.Info("We are Scheduling for the next RayJob") + cronSchedule, err := cron.ParseStandard(utils.FormatSchedule(rayJobInstance, r.Recorder)) + if err != nil { + // this is likely a user error in defining the spec value + // we should log the error and not reconcile this cronjob until an update to spec + logger.Error(err, "The cron schedule provided is unparseable") + r.Recorder.Eventf(rayJobInstance, corev1.EventTypeWarning, "UnparseableSchedule", "unparseable schedule: %q : %s", rayJobInstance.Spec.Schedule, err) + return 0, 0, err + } + logger.Info("Successfully parsed cron schedule", "CronSchedule", cronSchedule) + + t1 := utils.NextScheduleTimeDuration(logger, rayJobInstance, currentTime, cronSchedule) + t2 := utils.LastScheduleTimeDuration(logger, rayJobInstance, currentTime, cronSchedule) + + logger.Info("Got the time until we run the next RayJob", "NextScheduleTimeDuration", t1.Abs()) + return t1, t2, nil +} + func updateStatusToSuspendingIfNeeded(ctx context.Context, rayJob *rayv1.RayJob) bool { logger := ctrl.LoggerFrom(ctx) if !rayJob.Spec.Suspend { diff --git a/ray-operator/controllers/ray/rayjob_controller_unit_test.go b/ray-operator/controllers/ray/rayjob_controller_unit_test.go index 4e0eda26cb5..9d795d7f96e 100644 --- a/ray-operator/controllers/ray/rayjob_controller_unit_test.go +++ b/ray-operator/controllers/ray/rayjob_controller_unit_test.go @@ -624,3 +624,98 @@ func TestEmitRayJobExecutionDuration(t *testing.T) { }) } } + +func TestGetPreviousAndNextScheduleDistance(t *testing.T) { + // Test 1, the cron string is not valid + // Test 2, we are not within the the buffer period of a cron tick to run a ray job + // Test 3, we are within the buffer period of a cron tick to run a ray job + + newScheme := runtime.NewScheme() + _ = rayv1.AddToScheme(newScheme) + _ = corev1.AddToScheme(newScheme) // For events + + testCases := []struct { + name string + schedule string + initialLastTime *metav1.Time + currentTime time.Time + expectedErr bool + expectedNextDelta time.Duration + expectedPrevDelta time.Duration + isWithinBuffer bool + }{ + { + name: "Test 1: Invalid cron string", + schedule: "INVLAID * CRON * STRING * WOW", + currentTime: time.Now(), + expectedErr: true, + }, + { + name: "Test 2: Not within the buffer period - future schedule", + schedule: "0 0 * * *", // Every day at midnight + currentTime: time.Date(2025, time.July, 1, 10, 0, 0, 0, time.UTC), // 2025-07-01 10:00 AM + // Next schedule tick is 2025-07-02 00:00:00 (in 14 hours) + // Last schedule tick is 2025-07-01 00:00:00 (10 hours ago) + expectedNextDelta: 14 * time.Hour, + expectedPrevDelta: 10 * time.Hour, + expectedErr: false, + isWithinBuffer: false, + }, + { + name: "Test 3: Within the buffer period - next schedule", + schedule: "*/10 * * * *", // Every 10 minutes + currentTime: time.Date(2025, time.July, 1, 10, 10, 0, 0, time.UTC), // 2025-07-01 10:10:00 + // Next schedule tick is 2025-07-01 10:10:00 (in 10 minutes) + // Last schedule tick is 2025-07-01 10:00:00 (10 minutes ago) + expectedNextDelta: 10 * time.Minute, + expectedPrevDelta: 0 * time.Minute, + expectedErr: false, + isWithinBuffer: true, + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + rayJob := &rayv1.RayJob{ + ObjectMeta: metav1.ObjectMeta{ + Name: "test-rayjob", + Namespace: "default", + }, + Spec: rayv1.RayJobSpec{ + Schedule: tc.schedule, + }, + } + + fakeClient := clientFake.NewClientBuilder().WithScheme(newScheme).WithRuntimeObjects(rayJob).Build() + recorder := record.NewFakeRecorder(100) + + reconciler := &RayJobReconciler{ + Client: fakeClient, + Recorder: recorder, + Scheme: newScheme, + } + + // Call getPreviousAndNextScheduleDistance to get the next and pervious schedule ticks + nextDuration, prevDuration, err := reconciler.getPreviousAndNextScheduleDistance(context.Background(), tc.currentTime, rayJob) + + if tc.expectedErr { + require.Error(t, err) + assert.Contains(t, <-recorder.Events, "UnparseableSchedule") + } else { + require.NoError(t, err) + + // Asserting that + assert.InDelta(t, tc.expectedNextDelta.Seconds(), nextDuration.Seconds(), 1.0, "NextScheduleTimeDuration mismatch") + assert.InDelta(t, tc.expectedPrevDelta.Seconds(), prevDuration.Seconds(), 1.0, "LastScheduleTimeDuration mismatch") + + // Testing the ScheduleDelta logic and how it's called in reconcile + // Define ScheduleDelta within the test scope or as a global constant for testing + const ScheduleDelta = 100 * time.Millisecond + + isCurrentlyWithinBuffer := (nextDuration < ScheduleDelta) || (prevDuration < ScheduleDelta) + assert.Equal(t, tc.isWithinBuffer, isCurrentlyWithinBuffer, "isWithinBuffer check mismatch") + } + }) + } + +} diff --git a/ray-operator/controllers/ray/utils/cron_helper.go b/ray-operator/controllers/ray/utils/schedule.go similarity index 80% rename from ray-operator/controllers/ray/utils/cron_helper.go rename to ray-operator/controllers/ray/utils/schedule.go index 34460524a70..62ca76ba9da 100644 --- a/ray-operator/controllers/ray/utils/cron_helper.go +++ b/ray-operator/controllers/ray/utils/schedule.go @@ -18,10 +18,9 @@ const ( noneMissed missedSchedulesType = iota fewMissed manyMissed - nextScheduleDelta = 100 * time.Millisecond ) -func mostRecentScheduleTime(logger logr.Logger, rj *rayv1.RayJob, now time.Time, schedule cron.Schedule, includeStartingDeadlineSeconds bool) (time.Time, *time.Time, missedSchedulesType, error) { +func mostRecentScheduleTime(rj *rayv1.RayJob, now time.Time, schedule cron.Schedule) (time.Time, *time.Time, missedSchedulesType, error) { earliestTime := rj.ObjectMeta.CreationTimestamp.Time missedSchedules := noneMissed if rj.Status.LastScheduleTime != nil { @@ -108,14 +107,11 @@ func FormatSchedule(rj *rayv1.RayJob, recorder record.EventRecorder) string { return rj.Spec.Schedule } -// nextScheduleTimeDuration returns the time duration to requeue based on -// the schedule and last schedule time. It adds a 100ms padding to the next requeue to account -// for Network Time Protocol(NTP) time skews. If the time drifts the adjustment, which in most -// realistic cases should be around 100s, the job will still be executed without missing -// the schedule. -func NextScheduleTimeDuration(logger logr.Logger, rj *rayv1.RayJob, now time.Time, schedule cron.Schedule) *time.Duration { +// nextScheduleTimeDuration returns the time duration to requeue a cron schedule based on +// the schedule and last schedule time. +func NextScheduleTimeDuration(logger logr.Logger, rj *rayv1.RayJob, now time.Time, schedule cron.Schedule) time.Duration { - earliestTime, mostRecentTime, missedSchedules, err := mostRecentScheduleTime(logger, rj, now, schedule, false) + earliestTime, mostRecentTime, missedSchedules, err := mostRecentScheduleTime(rj, now, schedule) if err != nil { // we still have to requeue at some point, so aim for the next scheduling slot from now logger.Info("Error in mostRecentScheduleTime, we still have to requeue at some point, so aim for the next scheduling slot from now", "Error", err) @@ -131,28 +127,27 @@ func NextScheduleTimeDuration(logger logr.Logger, rj *rayv1.RayJob, now time.Tim } } logger.Info("Successfully calculated earliestTime and mostRecentTime", "mostRecentTime", mostRecentTime, "earliestTime", earliestTime, "Next time to aim for", schedule.Next(*mostRecentTime)) - t := schedule.Next(*mostRecentTime).Add(nextScheduleDelta).Sub(now) - return &t + t := schedule.Next(*mostRecentTime).Sub(now) + return t } +// The LastScheduleTimeDuration function returns the last previous cron time. +// It calculates the most recent time a schedule should have executed based +// on the RayJob's creation time (or its last scheduled status) and the current time 'now'. func LastScheduleTimeDuration(logger logr.Logger, rj *rayv1.RayJob, now time.Time, schedule cron.Schedule) time.Duration { - // The mostRecentScheduleTime function is the core logic that calculates the most - // recent time a schedule should have executed based on the RayJob's creation time - // (or its last scheduled status) and the current time 'now'. - // The second return value, 'mostRecentTime', is precisely the "last previous cron time" - // that we want to find. We can ignore the other return values for this function's purpose. - earliestTime, mostRecentTime, missedSchedules, err := mostRecentScheduleTime(logger, rj, now, schedule, false) + + earliestTime, mostRecentTime, missedSchedules, err := mostRecentScheduleTime(rj, now, schedule) if err != nil { - // we still have to requeue at some point, so aim for the next scheduling slot from now + // We still have to requeue at some point, so aim for the next scheduling slot from now logger.Info("Error in mostRecentScheduleTime, we still have to requeue at some point", "Error", err) mostRecentTime = &now } else if mostRecentTime == nil { logger.Info("mostRecentTime doesnt exist", "mostRecentTime", mostRecentTime, "earliestTime", earliestTime) if missedSchedules == noneMissed { - // no missed schedules since earliestTime + // No missed schedules since earliestTime mostRecentTime = &earliestTime } else { - // if there are missed schedules since earliestTime, always use now + // If there are missed schedules since earliestTime, always use now mostRecentTime = &now } } From 9b4db805d309bdd01fd1c8084e612ea3997c78ab Mon Sep 17 00:00:00 2001 From: DW-Han Date: Wed, 9 Jul 2025 20:13:55 +0000 Subject: [PATCH 07/34] adding cluster delete option and cleaning code --- .../config/samples/ray-job.schedule.yaml | 5 ++- .../controllers/ray/rayjob_controller.go | 33 ++++++++++++------- ray-operator/local_deploy.sh | 18 ++++------ 3 files changed, 32 insertions(+), 24 deletions(-) diff --git a/ray-operator/config/samples/ray-job.schedule.yaml b/ray-operator/config/samples/ray-job.schedule.yaml index c43618b628a..c31fdd87ed9 100644 --- a/ray-operator/config/samples/ray-job.schedule.yaml +++ b/ray-operator/config/samples/ray-job.schedule.yaml @@ -11,9 +11,12 @@ spec: # The alternative value is "HTTPMode", indicating that KubeRay will submit the Ray job by sending an HTTP request to the RayCluster. # submissionMode: "K8sJobMode" entrypoint: python /home/ray/samples/sample_code.py + # shutdownAfterJobFinishes specifies whether the RayCluster should be deleted after the RayJob finishes. Default is false. - # shutdownAfterJobFinishes: false + # NOTE that the expected behavor with schedule is that the cluster will be deleted and recreated at each schedule if set to true, and it will keep using the same cluster otherwise + shutdownAfterJobFinishes: true + # NOTE that if schedule is set will never end the rayjob since the expected behavior is to run at a schedule # ttlSecondsAfterFinished specifies the number of seconds after which the RayCluster will be deleted after the RayJob finishes. # ttlSecondsAfterFinished: 10 diff --git a/ray-operator/controllers/ray/rayjob_controller.go b/ray-operator/controllers/ray/rayjob_controller.go index fa5a6ce5385..ec5318f965b 100644 --- a/ray-operator/controllers/ray/rayjob_controller.go +++ b/ray-operator/controllers/ray/rayjob_controller.go @@ -331,9 +331,16 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) // TODO (kevin85421): Currently, Ray doesn't have a best practice to stop a Ray job gracefully. At this moment, // KubeRay doesn't stop the Ray job before suspending the RayJob. If users want to stop the Ray job by SIGTERM, // users need to set the Pod's preStop hook by themselves. - isClusterDeleted, err := r.deleteClusterResources(ctx, rayJobInstance) - if err != nil { - return ctrl.Result{RequeueAfter: RayJobDefaultRequeueDuration}, err + + // We set isClusterDeleted to true as default, i.e. we dont need to delete the cluster + isClusterDeleted := true + deleteCluster := rayJobInstance.Status.JobDeploymentStatus != rayv1.JobDeploymentStatusScheduling || rayJobInstance.Spec.ShutdownAfterJobFinishes + if deleteCluster { + isClusterDeleted, err = r.deleteClusterResources(ctx, rayJobInstance) + if err != nil { + return ctrl.Result{RequeueAfter: RayJobDefaultRequeueDuration}, err + } + } isJobDeleted, err := r.deleteSubmitterJob(ctx, rayJobInstance) if err != nil { @@ -345,14 +352,18 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) return ctrl.Result{RequeueAfter: RayJobDefaultRequeueDuration}, nil } - // Reset the RayCluster and Ray job related status. - rayJobInstance.Status.RayClusterStatus = rayv1.RayClusterStatus{} - rayJobInstance.Status.RayClusterName = "" + // Reset the RayCluster and Ray job related status. Done this way to be atomic. + if deleteCluster { + rayJobInstance.Status.RayClusterStatus = rayv1.RayClusterStatus{} + rayJobInstance.Status.RayClusterName = "" + + } rayJobInstance.Status.DashboardURL = "" rayJobInstance.Status.JobId = "" rayJobInstance.Status.Message = "" rayJobInstance.Status.Reason = "" rayJobInstance.Status.RayJobStatusInfo = rayv1.RayJobStatusInfo{} + // Reset the JobStatus to JobStatusNew and transition the JobDeploymentStatus to `Suspended`. rayJobInstance.Status.JobStatus = rayv1.JobStatusNew @@ -455,11 +466,10 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) } } if rayJobInstance.Spec.Schedule != "" && rayJobInstance.Status.JobDeploymentStatus != rayv1.JobDeploymentStatusFailed { - // If the rayjob has cron scheduling then we should change status to schedule for the next job logger.Info("RayJob is scheduled again") rayJobInstance.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusScheduling } else { - // NOTE: we do not requeue if the job fails, this could change + // NOTE: we do not requeue if the job fails even if scheduled, this could change // If the RayJob is completed without scheduling or has failed, we should not requeue it. logger.Info("RayJob is not scheduled") return ctrl.Result{}, nil @@ -478,8 +488,8 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) logger.Error(err, "Could not get the previous and next distances for a cron schedule") return ctrl.Result{}, err } - // Checking if we are currently close enough to either the previous or next cron schedule times - if t1 < ScheduleDelta || t2 < ScheduleDelta { + // Checking if we are currently within a buffer to the previous cron schedule time + if t2 <= ScheduleDelta { logger.Info("The current time is within the buffer window of a cron tick", "NextScheduleTimeDuration", t1, "LastScheduleTimeDuration", t2) rayJobInstance.Status.JobStatus = rayv1.JobStatusScheduled } else { @@ -844,7 +854,8 @@ func initRayJobStatusIfNeed(ctx context.Context, rayJob *rayv1.RayJob) error { // if know if this is the first job and not just another scheduled one we check the jobs count if rayJob.Spec.Schedule != "" && rayJob.Status.Failed == nil && rayJob.Status.Succeeded == nil { logger.Info("Initial schedule") - rayJob.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusScheduling + //NOTE + rayJob.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusScheduled } else { rayJob.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusInitializing } diff --git a/ray-operator/local_deploy.sh b/ray-operator/local_deploy.sh index 54a2f3c7434..c62c9ae1464 100755 --- a/ray-operator/local_deploy.sh +++ b/ray-operator/local_deploy.sh @@ -11,33 +11,27 @@ set -euxo pipefail # --- Configuration Variables --- # IMPORTANT: Customize these variables for your environment and project. # Use your Docker registry and a unique image name. -# For Kind, if not pushing to a remote registry, you can use "kind-registry" or similar. -IMAGE_REPO="yourregistry" # e.g., "gcr.io/my-gcp-project" or "docker.io/myusername" +IMAGE_REPO="yourregistry" IMAGE_NAME="kuberay-operator" # Set your desired image tag. A unique tag like a timestamp is recommended during development. -IMAGE_TAG="nightly" # Example: "my-custom-build-20250625-1030" or "nightly" - -KIND_CLUSTER_NAME="kind" # Your Kind cluster name -OPERATOR_NAMESPACE="default" # Or "kuberay-system", "kuberay-operator", etc. -HELM_RELEASE_NAME="kuberay-operator" # The Helm release name you use +IMAGE_TAG="nightly" +KIND_CLUSTER_NAME="kind" +OPERATOR_NAMESPACE="default" +HELM_RELEASE_NAME="kuberay-operator" HELM_CHART_PATH="../helm-chart/kuberay-operator" # Path to your Helm chart -# --- Script Logic --- + echo "--- Checking for Kind Cluster ---" # Check if the Kind cluster already exists if ! kind get clusters | grep -q "${KIND_CLUSTER_NAME}"; then echo "Kind cluster '${KIND_CLUSTER_NAME}' not found. Creating it..." - # You can customize your Kind cluster creation command here if needed. kind create cluster --name "${KIND_CLUSTER_NAME}" --image=kindest/node:v1.24.0 else echo "Kind cluster '${KIND_CLUSTER_NAME}' already exists. Skipping creation." fi -# Step 2: Modify KubeRay source code (Manual Step) -# For example, add a log by adding setupLog.Info("Hello KubeRay") in the function `main` in `main.go`. - echo "--- Building Docker Image ---" FULL_IMAGE_NAME="${IMAGE_REPO}/${IMAGE_NAME}:${IMAGE_TAG}" echo "Building image: ${FULL_IMAGE_NAME}" From d6f007642dea169d1ab2752cb8aff31e9cd31257 Mon Sep 17 00:00:00 2001 From: DW-Han Date: Fri, 11 Jul 2025 19:03:06 +0000 Subject: [PATCH 08/34] cleaning --- .../controllers/ray/rayjob_controller.go | 5 +- .../controllers/ray/utils/schedule.go | 8 ++ ray-operator/local_deploy.sh | 86 ------------------- 3 files changed, 10 insertions(+), 89 deletions(-) delete mode 100755 ray-operator/local_deploy.sh diff --git a/ray-operator/controllers/ray/rayjob_controller.go b/ray-operator/controllers/ray/rayjob_controller.go index ec5318f965b..d3ad0915184 100644 --- a/ray-operator/controllers/ray/rayjob_controller.go +++ b/ray-operator/controllers/ray/rayjob_controller.go @@ -850,11 +850,10 @@ func initRayJobStatusIfNeed(ctx context.Context, rayJob *rayv1.RayJob) error { if rayJob.Status.JobStatus == "" { rayJob.Status.JobStatus = rayv1.JobStatusNew } - // if the rayjob is sheduled according to a cron string set the status to scheduling instead of initializing to begin with - // if know if this is the first job and not just another scheduled one we check the jobs count + // if the rayjob is scheduled according to a cron string set the status to scheduling instead of initializing to begin with + // we check the job count to know if its the first job if rayJob.Spec.Schedule != "" && rayJob.Status.Failed == nil && rayJob.Status.Succeeded == nil { logger.Info("Initial schedule") - //NOTE rayJob.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusScheduled } else { rayJob.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusInitializing diff --git a/ray-operator/controllers/ray/utils/schedule.go b/ray-operator/controllers/ray/utils/schedule.go index 62ca76ba9da..cf6e18ad598 100644 --- a/ray-operator/controllers/ray/utils/schedule.go +++ b/ray-operator/controllers/ray/utils/schedule.go @@ -1,3 +1,11 @@ +/* +Portions of this file are derived from the CronJob reasource in the Kubernetes project, +Kubernetes project: https://github.com/kubernetes/kubernetes +Licensed under the Apache License, Version 2.0. +For more information on Kubernetes CronJob, refer to: +https://kubernetes.io/docs/concepts/workloads/controllers/cron-jobs/ +*/ + package utils import ( diff --git a/ray-operator/local_deploy.sh b/ray-operator/local_deploy.sh deleted file mode 100755 index c62c9ae1464..00000000000 --- a/ray-operator/local_deploy.sh +++ /dev/null @@ -1,86 +0,0 @@ -#!/bin/bash - -# set -euxo pipefail: -# -e: Exit immediately if a command exits with a non-zero status. -# -u: Treat unset variables as an error when substituting. -# -x: Print commands and their arguments as they are executed (useful for debugging). -# -o pipefail: The return value of a pipeline is the status of the last command to exit with a non-zero status, -# or zero if all commands in the pipeline exit successfully. -set -euxo pipefail - -# --- Configuration Variables --- -# IMPORTANT: Customize these variables for your environment and project. -# Use your Docker registry and a unique image name. -IMAGE_REPO="yourregistry" -IMAGE_NAME="kuberay-operator" -# Set your desired image tag. A unique tag like a timestamp is recommended during development. -IMAGE_TAG="nightly" - -KIND_CLUSTER_NAME="kind" -OPERATOR_NAMESPACE="default" -HELM_RELEASE_NAME="kuberay-operator" -HELM_CHART_PATH="../helm-chart/kuberay-operator" # Path to your Helm chart - - - -echo "--- Checking for Kind Cluster ---" -# Check if the Kind cluster already exists -if ! kind get clusters | grep -q "${KIND_CLUSTER_NAME}"; then - echo "Kind cluster '${KIND_CLUSTER_NAME}' not found. Creating it..." - kind create cluster --name "${KIND_CLUSTER_NAME}" --image=kindest/node:v1.24.0 -else - echo "Kind cluster '${KIND_CLUSTER_NAME}' already exists. Skipping creation." -fi - -echo "--- Building Docker Image ---" -FULL_IMAGE_NAME="${IMAGE_REPO}/${IMAGE_NAME}:${IMAGE_TAG}" -echo "Building image: ${FULL_IMAGE_NAME}" - -# Delete existing local image to ensure a fresh build and avoid caching issues. -# The '|| true' prevents the script from exiting if the image doesn't exist yet. -docker rmi -f "${FULL_IMAGE_NAME}" || true - -# Step 3: Build an image -# This command will copy the source code directory into the image, and build it. -# Command: IMG={IMG_REPO}:{IMG_TAG} make docker-build -make docker-build IMG="${FULL_IMAGE_NAME}" - -# To skip Go project compilation, run the following command instead: -# IMG=kuberay/operator:nightly make docker-image - -echo "--- Loading Image into Kind Cluster ---" -# Step 4: Load the custom KubeRay image into the Kind cluster. -# Command: kind load docker-image {IMG_REPO}:{IMG_TAG} -kind load docker-image "${FULL_IMAGE_NAME}" --name "${KIND_CLUSTER_NAME}" - -echo "--- Uninstalling and Installing KubeRay Operator ---" -# Check if the Helm release exists before trying to uninstall -if helm list --namespace "${OPERATOR_NAMESPACE}" | grep -q "${HELM_RELEASE_NAME}"; then - echo "Uninstalling existing Helm release: ${HELM_RELEASE_NAME}" - helm uninstall "${HELM_RELEASE_NAME}" --namespace "${OPERATOR_NAMESPACE}" - echo "Waiting for resources to be terminated..." - sleep 10 # Give Kubernetes some time to clean up -else - echo "Helm release '${HELM_RELEASE_NAME}' not found. Skipping uninstall." -fi - -# Step 6: Install KubeRay operator with the custom image via local Helm chart -# (Path: helm-chart/kuberay-operator) -# Command: helm install kuberay-operator --set image.repository={IMG_REPO} --set image.tag={IMG_TAG} ../helm-chart/kuberay-operator -echo "Installing new Helm release: ${HELM_RELEASE_NAME}" -helm install "${HELM_RELEASE_NAME}" "${HELM_CHART_PATH}" \ - --namespace "${OPERATOR_NAMESPACE}" \ - --set "image.repository=${IMAGE_REPO}/${IMAGE_NAME}" \ - --set "image.tag=${IMAGE_TAG}" - -echo "--- Waiting for Deployment Rollout ---" -# Wait for the operator deployment to successfully roll out. -# This ensures the new pod is running before we check logs. -kubectl rollout status deployment "${HELM_RELEASE_NAME}" --namespace "${OPERATOR_NAMESPACE}" --timeout=5m - -echo "--- Streaming Controller Logs (Ctrl+C to stop) ---" -# Step 7: Check the logs -# Note: This command directly targets the deployment for logs. -# kubectl logs -f deployments/"${HELM_RELEASE_NAME}" -n "${OPERATOR_NAMESPACE}" - -echo "--- Script Completed ---" From b6b398953a336ace75b02386d17320dbc8be3b99 Mon Sep 17 00:00:00 2001 From: DW-Han Date: Tue, 15 Jul 2025 01:05:46 +0000 Subject: [PATCH 09/34] cleaning and adding schedule util unit tests --- docs/reference/api.md | 2 +- ray-operator/apis/ray/v1/rayjob_types.go | 3 + .../config/samples/ray-job.schedule.yaml | 43 +- .../controllers/ray/rayjob_controller.go | 18 +- .../controllers/ray/utils/schedule.go | 10 +- .../controllers/ray/utils/schedule_test.go | 556 ++++++++++++++++++ ray-operator/go.mod | 1 + ray-operator/go.sum | 2 + 8 files changed, 581 insertions(+), 54 deletions(-) create mode 100644 ray-operator/controllers/ray/utils/schedule_test.go diff --git a/docs/reference/api.md b/docs/reference/api.md index 478db64e53b..8e0bdf25d10 100644 --- a/docs/reference/api.md +++ b/docs/reference/api.md @@ -253,7 +253,7 @@ _Appears in:_ | `ttlSecondsAfterFinished` _integer_ | TTLSecondsAfterFinished is the TTL to clean up RayCluster.
It's only working when ShutdownAfterJobFinishes set to true. | 0 | | | `shutdownAfterJobFinishes` _boolean_ | ShutdownAfterJobFinishes will determine whether to delete the ray cluster once rayJob succeed or failed. | | | | `suspend` _boolean_ | 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. | | | -| `schedule` _string_ | Schedule specifies a cron like string for cron scheduling | | | +| `schedule` _string_ | Schedule specifies a cron like string for cron scheduling
When set with shutdownAfterJobFinishes, if shutdownAfterJobFinishes is False
the schduled jobs will keep using the same cluster, if True it will delete
the cluster and create a new one on the next job. | | | diff --git a/ray-operator/apis/ray/v1/rayjob_types.go b/ray-operator/apis/ray/v1/rayjob_types.go index f8359b79635..822d298244d 100644 --- a/ray-operator/apis/ray/v1/rayjob_types.go +++ b/ray-operator/apis/ray/v1/rayjob_types.go @@ -206,6 +206,9 @@ type RayJobSpec struct { // +optional Suspend bool `json:"suspend,omitempty"` // Schedule specifies a cron like string for cron scheduling + // When set with shutdownAfterJobFinishes, if shutdownAfterJobFinishes is False + // the schduled jobs will keep using the same cluster, if True it will delete + // the cluster and create a new one on the next job. // +optional Schedule string `json:"schedule,omitempty"` } diff --git a/ray-operator/config/samples/ray-job.schedule.yaml b/ray-operator/config/samples/ray-job.schedule.yaml index c31fdd87ed9..ccf299a945f 100644 --- a/ray-operator/config/samples/ray-job.schedule.yaml +++ b/ray-operator/config/samples/ray-job.schedule.yaml @@ -6,27 +6,13 @@ spec: # schedule specifires a cron scheduling string telling the rayjob when to start schedule and run new jobs # Here it runs at every 5 minutes of every hour of every day of every week of every year schedule: "*/5 * * * *" - # submissionMode specifies how RayJob submits the Ray job to the RayCluster. - # The default value is "K8sJobMode", meaning RayJob will submit the Ray job via a submitter Kubernetes Job. - # The alternative value is "HTTPMode", indicating that KubeRay will submit the Ray job by sending an HTTP request to the RayCluster. - # submissionMode: "K8sJobMode" + entrypoint: python /home/ray/samples/sample_code.py # shutdownAfterJobFinishes specifies whether the RayCluster should be deleted after the RayJob finishes. Default is false. - # NOTE that the expected behavor with schedule is that the cluster will be deleted and recreated at each schedule if set to true, and it will keep using the same cluster otherwise + # NOTE that the expected behavior with schedule is that the cluster will be deleted and recreated at each schedule if set to true, and it will keep using the same cluster otherwise shutdownAfterJobFinishes: true - # NOTE that if schedule is set will never end the rayjob since the expected behavior is to run at a schedule - # ttlSecondsAfterFinished specifies the number of seconds after which the RayCluster will be deleted after the RayJob finishes. - # ttlSecondsAfterFinished: 10 - - # 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. - # activeDeadlineSeconds: 120 - - # RuntimeEnvYAML represents the runtime environment configuration provided as a multi-line YAML string. - # See https://docs.ray.io/en/latest/ray-core/handling-dependencies.html for details. - # (New in KubeRay version 1.0.) runtimeEnvYAML: | pip: - requests==2.26.0 @@ -34,21 +20,11 @@ spec: env_vars: counter_name: "test_counter" - # 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 we will wait for the transition to false. - # If the RayCluster is already created, it will be deleted. In the case of transition to false, a new RayCluster will be created. - # suspend: false - # rayClusterSpec specifies the RayCluster instance to be created by the RayJob controller. rayClusterSpec: - rayVersion: '2.46.0' # should match the Ray version in the image of the containers - # Ray head pod template + rayVersion: '2.46.0' headGroupSpec: - # The `rayStartParams` are used to configure the `ray start` command. - # See https://github.com/ray-project/kuberay/blob/master/docs/guidance/rayStartParams.md for the default settings of `rayStartParams` in KubeRay. - # See https://docs.ray.io/en/latest/cluster/cli.html#ray-start for all available options in `rayStartParams`. rayStartParams: {} - #pod template template: spec: containers: @@ -57,7 +33,7 @@ spec: ports: - containerPort: 6379 name: gcs-server - - containerPort: 8265 # Ray dashboard + - containerPort: 8265 name: dashboard - containerPort: 10001 name: client @@ -70,31 +46,22 @@ spec: - mountPath: /home/ray/samples name: code-sample volumes: - # You set volumes at the Pod level, then mount them into containers inside that Pod - name: code-sample configMap: - # Provide the name of the ConfigMap you want to mount. name: ray-job-code-sample - # An array of keys from the ConfigMap to create as files items: - key: sample_code.py path: sample_code.py workerGroupSpecs: - # the pod replicas in this group typed worker - replicas: 1 minReplicas: 1 maxReplicas: 5 - # logical group name, for this called small-group, also can be functional groupName: small-group - # The `rayStartParams` are used to configure the `ray start` command. - # See https://github.com/ray-project/kuberay/blob/master/docs/guidance/rayStartParams.md for the default settings of `rayStartParams` in KubeRay. - # See https://docs.ray.io/en/latest/cluster/cli.html#ray-start for all available options in `rayStartParams`. rayStartParams: {} - #pod template template: spec: containers: - - name: ray-worker # must consist of lower case alphanumeric characters or '-', and must start and end with an alphanumeric character (e.g. 'my-name', or '123-abc' + - name: ray-worker image: rayproject/ray:2.46.0 resources: limits: diff --git a/ray-operator/controllers/ray/rayjob_controller.go b/ray-operator/controllers/ray/rayjob_controller.go index d3ad0915184..5db3e8ee1ff 100644 --- a/ray-operator/controllers/ray/rayjob_controller.go +++ b/ray-operator/controllers/ray/rayjob_controller.go @@ -35,7 +35,8 @@ const ( RayJobDefaultRequeueDuration = 3 * time.Second RayJobDefaultClusterSelectorKey = "ray.io/cluster" PythonUnbufferedEnvVarName = "PYTHONUNBUFFERED" - ScheduleDelta = 100 * time.Millisecond + // The buffer period in which a scheduled rajob can run since the last cron tick + ScheduleBuffer = 100 * time.Millisecond ) // RayJobReconciler reconciles a RayJob object @@ -489,13 +490,12 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) return ctrl.Result{}, err } // Checking if we are currently within a buffer to the previous cron schedule time - if t2 <= ScheduleDelta { + if t2 <= ScheduleBuffer { logger.Info("The current time is within the buffer window of a cron tick", "NextScheduleTimeDuration", t1, "LastScheduleTimeDuration", t2) rayJobInstance.Status.JobStatus = rayv1.JobStatusScheduled } else { - logger.Info("We wait for until the next and reconcile", "Time till next job", t1, "Current Time", time.Now(), "LastScheduleTimeDuration", t2) - rayJobInstance.Status.LastScheduleTime = &metav1.Time{Time: time.Now()} - + logger.Info("Waiting until the next reconcile to determine schedule", "nextScheduleDuration", t1, "currentTime", time.Now(), "lastScheduleTimeDuration", t2) + // rayJobInstance.Status.LastScheduleTime = &metav1.Time{Time: time.Now()} return ctrl.Result{RequeueAfter: t1}, nil } @@ -853,7 +853,7 @@ func initRayJobStatusIfNeed(ctx context.Context, rayJob *rayv1.RayJob) error { // if the rayjob is scheduled according to a cron string set the status to scheduling instead of initializing to begin with // we check the job count to know if its the first job if rayJob.Spec.Schedule != "" && rayJob.Status.Failed == nil && rayJob.Status.Succeeded == nil { - logger.Info("Initial schedule") + logger.Info("Since this is a new schdueled job we enter the Scheduled state not Initalizing") rayJob.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusScheduled } else { rayJob.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusInitializing @@ -951,21 +951,19 @@ func (r *RayJobReconciler) constructRayClusterForRayJob(rayJobInstance *rayv1.Ra func (r *RayJobReconciler) getPreviousAndNextScheduleDistance(ctx context.Context, currentTime time.Time, rayJobInstance *rayv1.RayJob) (time.Duration, time.Duration, error) { logger := ctrl.LoggerFrom(ctx) - logger.Info("We are Scheduling for the next RayJob") + logger.Info("Calculating next schedule for the RayJob") cronSchedule, err := cron.ParseStandard(utils.FormatSchedule(rayJobInstance, r.Recorder)) if err != nil { // this is likely a user error in defining the spec value // we should log the error and not reconcile this cronjob until an update to spec - logger.Error(err, "The cron schedule provided is unparseable") r.Recorder.Eventf(rayJobInstance, corev1.EventTypeWarning, "UnparseableSchedule", "unparseable schedule: %q : %s", rayJobInstance.Spec.Schedule, err) - return 0, 0, err + return 0, 0, fmt.Errorf("the cron schedule provided is unparseable: %w", err) } logger.Info("Successfully parsed cron schedule", "CronSchedule", cronSchedule) t1 := utils.NextScheduleTimeDuration(logger, rayJobInstance, currentTime, cronSchedule) t2 := utils.LastScheduleTimeDuration(logger, rayJobInstance, currentTime, cronSchedule) - logger.Info("Got the time until we run the next RayJob", "NextScheduleTimeDuration", t1.Abs()) return t1, t2, nil } diff --git a/ray-operator/controllers/ray/utils/schedule.go b/ray-operator/controllers/ray/utils/schedule.go index cf6e18ad598..e522d28c8c5 100644 --- a/ray-operator/controllers/ray/utils/schedule.go +++ b/ray-operator/controllers/ray/utils/schedule.go @@ -28,7 +28,7 @@ const ( manyMissed ) -func mostRecentScheduleTime(rj *rayv1.RayJob, now time.Time, schedule cron.Schedule) (time.Time, *time.Time, missedSchedulesType, error) { +func MostRecentScheduleTime(rj *rayv1.RayJob, now time.Time, schedule cron.Schedule) (time.Time, *time.Time, missedSchedulesType, error) { earliestTime := rj.ObjectMeta.CreationTimestamp.Time missedSchedules := noneMissed if rj.Status.LastScheduleTime != nil { @@ -119,13 +119,13 @@ func FormatSchedule(rj *rayv1.RayJob, recorder record.EventRecorder) string { // the schedule and last schedule time. func NextScheduleTimeDuration(logger logr.Logger, rj *rayv1.RayJob, now time.Time, schedule cron.Schedule) time.Duration { - earliestTime, mostRecentTime, missedSchedules, err := mostRecentScheduleTime(rj, now, schedule) + earliestTime, mostRecentTime, missedSchedules, err := MostRecentScheduleTime(rj, now, schedule) if err != nil { // we still have to requeue at some point, so aim for the next scheduling slot from now logger.Info("Error in mostRecentScheduleTime, we still have to requeue at some point, so aim for the next scheduling slot from now", "Error", err) mostRecentTime = &now } else if mostRecentTime == nil { - logger.Info("mostRecentTime doesnt exist", "mostRecentTime", mostRecentTime, "earliestTime", earliestTime) + logger.Info("mostRecentTime doesnt exist") if missedSchedules == noneMissed { // no missed schedules since earliestTime mostRecentTime = &earliestTime @@ -144,13 +144,13 @@ func NextScheduleTimeDuration(logger logr.Logger, rj *rayv1.RayJob, now time.Tim // on the RayJob's creation time (or its last scheduled status) and the current time 'now'. func LastScheduleTimeDuration(logger logr.Logger, rj *rayv1.RayJob, now time.Time, schedule cron.Schedule) time.Duration { - earliestTime, mostRecentTime, missedSchedules, err := mostRecentScheduleTime(rj, now, schedule) + earliestTime, mostRecentTime, missedSchedules, err := MostRecentScheduleTime(rj, now, schedule) if err != nil { // We still have to requeue at some point, so aim for the next scheduling slot from now logger.Info("Error in mostRecentScheduleTime, we still have to requeue at some point", "Error", err) mostRecentTime = &now } else if mostRecentTime == nil { - logger.Info("mostRecentTime doesnt exist", "mostRecentTime", mostRecentTime, "earliestTime", earliestTime) + logger.Info("mostRecentTime doesnt exist") if missedSchedules == noneMissed { // No missed schedules since earliestTime mostRecentTime = &earliestTime diff --git a/ray-operator/controllers/ray/utils/schedule_test.go b/ray-operator/controllers/ray/utils/schedule_test.go new file mode 100644 index 00000000000..d2394cd35c5 --- /dev/null +++ b/ray-operator/controllers/ray/utils/schedule_test.go @@ -0,0 +1,556 @@ +package utils + +import ( + "reflect" + "testing" + "time" + + "github.com/go-logr/logr/testr" + rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1" + cron "github.com/robfig/cron/v3" + metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" +) + +func TestMostRecentScheduleTime(t *testing.T) { + metav1TopOfTheHour := metav1.NewTime(*topOfTheHour()) + metav1HalfPastTheHour := metav1.NewTime(*deltaTimeAfterTopOfTheHour(30 * time.Minute)) + // metav1MinuteAfterTopOfTheHour := metav1.NewTime(*deltaTimeAfterTopOfTheHour(1 * time.Minute)) + // oneMinute := int64(60) + // tenSeconds := int64(10) + + tests := []struct { + name string + cj *rayv1.RayJob + includeSDS bool + now time.Time + expectedEarliestTime time.Time + expectedRecentTime *time.Time + expectedTooManyMissed missedSchedulesType + wantErr bool + }{ + { + name: "now before next schedule", + cj: &rayv1.RayJob{ + ObjectMeta: metav1.ObjectMeta{ + CreationTimestamp: metav1TopOfTheHour, + }, + Spec: rayv1.RayJobSpec{ + Schedule: "0 * * * *", + }, + }, + now: topOfTheHour().Add(30 * time.Second), + expectedRecentTime: nil, + expectedEarliestTime: *topOfTheHour(), + }, + { + name: "now just after next schedule", + cj: &rayv1.RayJob{ + ObjectMeta: metav1.ObjectMeta{ + CreationTimestamp: metav1TopOfTheHour, + }, + Spec: rayv1.RayJobSpec{ + Schedule: "0 * * * *", + }, + }, + now: topOfTheHour().Add(61 * time.Minute), + expectedRecentTime: deltaTimeAfterTopOfTheHour(60 * time.Minute), + expectedEarliestTime: *topOfTheHour(), + }, + { + name: "missed 5 schedules", + cj: &rayv1.RayJob{ + ObjectMeta: metav1.ObjectMeta{ + CreationTimestamp: metav1.NewTime(*deltaTimeAfterTopOfTheHour(10 * time.Second)), + }, + Spec: rayv1.RayJobSpec{ + Schedule: "0 * * * *", + }, + }, + now: *deltaTimeAfterTopOfTheHour(301 * time.Minute), + expectedRecentTime: deltaTimeAfterTopOfTheHour(300 * time.Minute), + expectedEarliestTime: *deltaTimeAfterTopOfTheHour(10 * time.Second), + expectedTooManyMissed: fewMissed, + }, + { + name: "complex schedule", + cj: &rayv1.RayJob{ + ObjectMeta: metav1.ObjectMeta{ + CreationTimestamp: metav1TopOfTheHour, + }, + Spec: rayv1.RayJobSpec{ + Schedule: "30 6-16/4 * * 1-5", + }, + Status: rayv1.RayJobStatus{ + LastScheduleTime: &metav1HalfPastTheHour, + }, + }, + now: *deltaTimeAfterTopOfTheHour(24*time.Hour + 31*time.Minute), + expectedRecentTime: deltaTimeAfterTopOfTheHour(24*time.Hour + 30*time.Minute), + expectedEarliestTime: *deltaTimeAfterTopOfTheHour(30 * time.Minute), + expectedTooManyMissed: fewMissed, + }, + { + name: "another complex schedule", + cj: &rayv1.RayJob{ + ObjectMeta: metav1.ObjectMeta{ + CreationTimestamp: metav1TopOfTheHour, + }, + Spec: rayv1.RayJobSpec{ + Schedule: "30 10,11,12 * * 1-5", + }, + Status: rayv1.RayJobStatus{ + LastScheduleTime: &metav1HalfPastTheHour, + }, + }, + now: *deltaTimeAfterTopOfTheHour(30*time.Hour + 30*time.Minute), + expectedRecentTime: nil, + expectedEarliestTime: *deltaTimeAfterTopOfTheHour(30 * time.Minute), + expectedTooManyMissed: fewMissed, + }, + { + name: "complex schedule with longer diff between executions", + cj: &rayv1.RayJob{ + ObjectMeta: metav1.ObjectMeta{ + CreationTimestamp: metav1TopOfTheHour, + }, + Spec: rayv1.RayJobSpec{ + Schedule: "30 6-16/4 * * 1-5", + }, + Status: rayv1.RayJobStatus{ + LastScheduleTime: &metav1HalfPastTheHour, + }, + }, + now: *deltaTimeAfterTopOfTheHour(96*time.Hour + 31*time.Minute), + expectedRecentTime: deltaTimeAfterTopOfTheHour(96*time.Hour + 30*time.Minute), + expectedEarliestTime: *deltaTimeAfterTopOfTheHour(30 * time.Minute), + expectedTooManyMissed: fewMissed, + }, + { + name: "complex schedule with shorter diff between executions", + cj: &rayv1.RayJob{ + ObjectMeta: metav1.ObjectMeta{ + CreationTimestamp: metav1TopOfTheHour, + }, + Spec: rayv1.RayJobSpec{ + Schedule: "30 6-16/4 * * 1-5", + }, + }, + now: *deltaTimeAfterTopOfTheHour(24*time.Hour + 31*time.Minute), + expectedRecentTime: deltaTimeAfterTopOfTheHour(24*time.Hour + 30*time.Minute), + expectedEarliestTime: *topOfTheHour(), + expectedTooManyMissed: fewMissed, + }, + // { + // name: "@every schedule", + // cj: &rayv1.RayJob{ + // ObjectMeta: metav1.ObjectMeta{ + // CreationTimestamp: metav1.NewTime(*deltaTimeAfterTopOfTheHour(-59 * time.Minute)), + // }, + // Spec: rayv1.RayJobSpec{ + // Schedule: "@every 1h", + // StartingDeadlineSeconds: &tenSeconds, + // }, + // Status: rayv1.RayJobStatus{ + // LastScheduleTime: &metav1MinuteAfterTopOfTheHour, + // }, + // }, + // now: *deltaTimeAfterTopOfTheHour(7 * 24 * time.Hour), + // expectedRecentTime: deltaTimeAfterTopOfTheHour((6 * 24 * time.Hour) + 23*time.Hour + 1*time.Minute), + // expectedEarliestTime: *deltaTimeAfterTopOfTheHour(1 * time.Minute), + // expectedTooManyMissed: manyMissed, + // }, + { + name: "rogue cronjob", + cj: &rayv1.RayJob{ + ObjectMeta: metav1.ObjectMeta{ + CreationTimestamp: metav1.NewTime(*deltaTimeAfterTopOfTheHour(10 * time.Second)), + }, + Spec: rayv1.RayJobSpec{ + Schedule: "59 23 31 2 *", + }, + }, + now: *deltaTimeAfterTopOfTheHour(1 * time.Hour), + expectedRecentTime: nil, + wantErr: true, + }, + { + name: "earliestTime being CreationTimestamp and LastScheduleTime", + cj: &rayv1.RayJob{ + ObjectMeta: metav1.ObjectMeta{ + CreationTimestamp: metav1TopOfTheHour, + }, + Spec: rayv1.RayJobSpec{ + Schedule: "0 * * * *", + }, + Status: rayv1.RayJobStatus{ + LastScheduleTime: &metav1TopOfTheHour, + }, + }, + now: *deltaTimeAfterTopOfTheHour(30 * time.Second), + expectedEarliestTime: *topOfTheHour(), + expectedRecentTime: nil, + }, + { + name: "earliestTime being LastScheduleTime", + cj: &rayv1.RayJob{ + ObjectMeta: metav1.ObjectMeta{ + CreationTimestamp: metav1TopOfTheHour, + }, + Spec: rayv1.RayJobSpec{ + Schedule: "*/5 * * * *", + }, + Status: rayv1.RayJobStatus{ + LastScheduleTime: &metav1HalfPastTheHour, + }, + }, + now: *deltaTimeAfterTopOfTheHour(31 * time.Minute), + expectedEarliestTime: *deltaTimeAfterTopOfTheHour(30 * time.Minute), + expectedRecentTime: nil, + }, + // { + // name: "earliestTime being LastScheduleTime (within StartingDeadlineSeconds)", + // cj: &rayv1.RayJob{ + // ObjectMeta: metav1.ObjectMeta{ + // CreationTimestamp: metav1TopOfTheHour, + // }, + // Spec: rayv1.RayJobSpec{ + // Schedule: "*/5 * * * *", + // StartingDeadlineSeconds: &oneMinute, + // }, + // Status: rayv1.RayJobStatus{ + // LastScheduleTime: &metav1HalfPastTheHour, + // }, + // }, + // now: *deltaTimeAfterTopOfTheHour(31 * time.Minute), + // expectedEarliestTime: *deltaTimeAfterTopOfTheHour(30 * time.Minute), + // expectedRecentTime: nil, + // }, + // { + // name: "earliestTime being LastScheduleTime (outside StartingDeadlineSeconds)", + // cj: &rayv1.RayJob{ + // ObjectMeta: metav1.ObjectMeta{ + // CreationTimestamp: metav1TopOfTheHour, + // }, + // Spec: rayv1.RayJobSpec{ + // Schedule: "*/5 * * * *", + // StartingDeadlineSeconds: &oneMinute, + // }, + // Status: rayv1.RayJobStatus{ + // LastScheduleTime: &metav1HalfPastTheHour, + // }, + // }, + // includeSDS: true, + // now: *deltaTimeAfterTopOfTheHour(32 * time.Minute), + // expectedEarliestTime: *deltaTimeAfterTopOfTheHour(31 * time.Minute), + // expectedRecentTime: nil, + // }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + sched, err := cron.ParseStandard(tt.cj.Spec.Schedule) + if err != nil { + t.Errorf("error setting up the test, %s", err) + } + gotEarliestTime, gotRecentTime, gotTooManyMissed, err := MostRecentScheduleTime(tt.cj, tt.now, sched) + if tt.wantErr { + if err == nil { + t.Error("mostRecentScheduleTime() got no error when expected one") + } + return + } + if !tt.wantErr && err != nil { + t.Error("mostRecentScheduleTime() got error when none expected") + } + if gotEarliestTime.IsZero() { + t.Errorf("earliestTime should never be 0, want %v", tt.expectedEarliestTime) + } + if !gotEarliestTime.Equal(tt.expectedEarliestTime) { + t.Errorf("expectedEarliestTime - got %v, want %v", gotEarliestTime, tt.expectedEarliestTime) + } + if !reflect.DeepEqual(gotRecentTime, tt.expectedRecentTime) { + t.Errorf("expectedRecentTime - got %v, want %v", gotRecentTime, tt.expectedRecentTime) + } + if gotTooManyMissed != tt.expectedTooManyMissed { + t.Errorf("expectedNumberOfMisses - got %v, want %v", gotTooManyMissed, tt.expectedTooManyMissed) + } + }) + } +} + +func TestNextScheduleTimeDuration(t *testing.T) { + logger := testr.New(t) + metav1TopOfTheHour := metav1.NewTime(*topOfTheHour()) + metav1HalfPastTheHour := metav1.NewTime(*deltaTimeAfterTopOfTheHour(30 * time.Minute)) + metav1TwoHoursLater := metav1.NewTime(*deltaTimeAfterTopOfTheHour(2 * time.Hour)) + + tests := []struct { + name string + cj *rayv1.RayJob + now time.Time + expectedDuration time.Duration + }{ + { + name: "complex schedule skipping weekend", + cj: &rayv1.RayJob{ + ObjectMeta: metav1.ObjectMeta{ + CreationTimestamp: metav1TopOfTheHour, + }, + Spec: rayv1.RayJobSpec{ + Schedule: "30 6-16/4 * * 1-5", + }, + Status: rayv1.RayJobStatus{ + LastScheduleTime: &metav1HalfPastTheHour, + }, + }, + now: *deltaTimeAfterTopOfTheHour(24*time.Hour + 31*time.Minute), + expectedDuration: 3*time.Hour + 59*time.Minute, + }, + { + name: "another complex schedule skipping weekend", + cj: &rayv1.RayJob{ + ObjectMeta: metav1.ObjectMeta{ + CreationTimestamp: metav1TopOfTheHour, + }, + Spec: rayv1.RayJobSpec{ + Schedule: "30 10,11,12 * * 1-5", + }, + Status: rayv1.RayJobStatus{ + LastScheduleTime: &metav1HalfPastTheHour, + }, + }, + now: *deltaTimeAfterTopOfTheHour(30*time.Hour + 30*time.Minute), + expectedDuration: 66 * time.Hour, + }, + { + name: "once a week cronjob, missed two runs", + cj: &rayv1.RayJob{ + ObjectMeta: metav1.ObjectMeta{ + CreationTimestamp: metav1TopOfTheHour, + }, + Spec: rayv1.RayJobSpec{ + Schedule: "0 12 * * 4", + }, + Status: rayv1.RayJobStatus{ + LastScheduleTime: &metav1TwoHoursLater, + }, + }, + now: *deltaTimeAfterTopOfTheHour(19*24*time.Hour + 1*time.Hour + 30*time.Minute), + expectedDuration: 48*time.Hour + 30*time.Minute, + }, + { + name: "no previous run of a cronjob", + cj: &rayv1.RayJob{ + ObjectMeta: metav1.ObjectMeta{ + CreationTimestamp: metav1TopOfTheHour, + }, + Spec: rayv1.RayJobSpec{ + Schedule: "0 12 * * 5", + }, + }, + now: *deltaTimeAfterTopOfTheHour(6 * time.Hour), + expectedDuration: 20 * time.Hour, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + sched, err := cron.ParseStandard(tt.cj.Spec.Schedule) + if err != nil { + t.Errorf("error setting up the test, %s", err) + } + gotScheduleTimeDuration := NextScheduleTimeDuration(logger, tt.cj, tt.now, sched) + if gotScheduleTimeDuration < 0 { + t.Errorf("scheduleTimeDuration should never be less than 0, got %s", gotScheduleTimeDuration) + } + if !reflect.DeepEqual(&gotScheduleTimeDuration, &tt.expectedDuration) { + t.Errorf("scheduleTimeDuration - got %s, want %s, difference: %s", gotScheduleTimeDuration, tt.expectedDuration, gotScheduleTimeDuration-tt.expectedDuration) + } + }) + } +} + +func TestLastScheduleTimeDuration(t *testing.T) { + logger := testr.New(t) + metav1TopOfTheHour := metav1.NewTime(*topOfTheHour()) + metav1OneHourAgo := metav1.NewTime(*deltaTimeAfterTopOfTheHour(-1 * time.Hour)) + metav1YesterdayMidday := metav1.NewTime(metav1TopOfTheHour.Add(-28 * time.Hour)) + metav1TwoDaysAgo := metav1.NewTime(metav1TopOfTheHour.Add(-48 * time.Hour)) + metav1FourMonthsAgo := metav1.NewTime(metav1TopOfTheHour.AddDate(0, -4, 0)) + metav1FiveMonthsAgo := metav1.NewTime(metav1TopOfTheHour.AddDate(0, -5, 0)) + + tests := []struct { + name string + cj *rayv1.RayJob + now time.Time + expectedDuration time.Duration + }{ + { + name: "hourly job, last scheduled 30 minutes ago", + cj: &rayv1.RayJob{ + ObjectMeta: metav1.ObjectMeta{ + CreationTimestamp: metav1TopOfTheHour, + }, + Spec: rayv1.RayJobSpec{ + Schedule: "0 * * * *", + }, + Status: rayv1.RayJobStatus{ + LastScheduleTime: &metav1TopOfTheHour, + }, + }, + now: *deltaTimeAfterTopOfTheHour(30 * time.Minute), + expectedDuration: 30 * time.Minute, + }, + { + name: "daily job, last scheduled yesterday, now is midday next day", + cj: &rayv1.RayJob{ + ObjectMeta: metav1.ObjectMeta{ + CreationTimestamp: metav1TwoDaysAgo, + }, + Spec: rayv1.RayJobSpec{ + Schedule: "0 12 * * *", + }, + Status: rayv1.RayJobStatus{ + LastScheduleTime: &metav1YesterdayMidday, + }, + }, + now: *deltaTimeAfterTopOfTheHour(26*time.Hour + 5*time.Minute), + expectedDuration: 5 * time.Minute, + }, + { + name: "weekly job, no previous run, now before first schedule", + cj: &rayv1.RayJob{ + ObjectMeta: metav1.ObjectMeta{ + CreationTimestamp: metav1TopOfTheHour, + }, + Spec: rayv1.RayJobSpec{ + Schedule: "0 12 * * 5", + }, + Status: rayv1.RayJobStatus{ + LastScheduleTime: nil, + }, + }, + now: *deltaTimeAfterTopOfTheHour(24 * time.Hour), + expectedDuration: 24 * time.Hour, + }, + { + name: "weekly job, no previous run, now after first schedule (missed)", + cj: &rayv1.RayJob{ + ObjectMeta: metav1.ObjectMeta{ + CreationTimestamp: metav1TopOfTheHour, + }, + Spec: rayv1.RayJobSpec{ + Schedule: "0 10 * * 5", + }, + Status: rayv1.RayJobStatus{ + LastScheduleTime: nil, + }, + }, + now: *deltaTimeAfterTopOfTheHour(8 * 24 * time.Hour), + expectedDuration: 0 * time.Minute, + }, + { + name: "cronjob with lastScheduleTime equal to now", + cj: &rayv1.RayJob{ + ObjectMeta: metav1.ObjectMeta{ + CreationTimestamp: metav1TopOfTheHour, + }, + Spec: rayv1.RayJobSpec{ + Schedule: "0 * * * *", + }, + Status: rayv1.RayJobStatus{ + LastScheduleTime: &metav1TopOfTheHour, + }, + }, + now: *deltaTimeAfterTopOfTheHour(0 * time.Minute), + expectedDuration: 0 * time.Minute, + }, + { + name: "complex schedule, now before first next run after lastScheduleTime", + cj: &rayv1.RayJob{ + ObjectMeta: metav1.ObjectMeta{ + CreationTimestamp: metav1OneHourAgo, + }, + Spec: rayv1.RayJobSpec{ + Schedule: "30 6-16/4 * * 1-5", + }, + Status: rayv1.RayJobStatus{ + LastScheduleTime: func() *metav1.Time { + t := metav1.NewTime(*deltaTimeAfterTopOfTheHour(-90 * time.Minute)) + return &t + }(), + }, + }, + now: *deltaTimeAfterTopOfTheHour(-60 * time.Minute), + expectedDuration: 30 * time.Minute, + }, + { + name: "daily job, last scheduled today earlier, now is later", + cj: &rayv1.RayJob{ + ObjectMeta: metav1.ObjectMeta{ + CreationTimestamp: metav1TopOfTheHour, + }, + Spec: rayv1.RayJobSpec{ + Schedule: "0 16 * * *", + }, + Status: rayv1.RayJobStatus{ + LastScheduleTime: &metav1TopOfTheHour, + }, + }, + now: *deltaTimeAfterTopOfTheHour(1 * time.Hour), + expectedDuration: 1 * time.Hour, + }, + { + name: "monthly job, missed several months, now is far past last schedule", + cj: &rayv1.RayJob{ + ObjectMeta: metav1.ObjectMeta{ + CreationTimestamp: metav1FiveMonthsAgo, + }, + Spec: rayv1.RayJobSpec{ + Schedule: "0 0 1 * *", + }, + Status: rayv1.RayJobStatus{ + LastScheduleTime: &metav1FourMonthsAgo, + }, + }, + now: metav1TopOfTheHour.Add(1 * time.Hour), + expectedDuration: metav1TopOfTheHour.Add(1 * time.Hour).Sub(time.Date(2016, time.May, 1, 0, 0, 0, 0, time.UTC)), + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + sched, err := cron.ParseStandard(tt.cj.Spec.Schedule) + if err != nil { + t.Errorf("error setting up the test, %s", err) + return + } + t.Log(tt.now) + + gotScheduleTimeDuration := LastScheduleTimeDuration(logger, tt.cj, tt.now, sched) + + if gotScheduleTimeDuration < 0 { + t.Errorf("LastScheduleTimeDuration should never be less than 0, got %s", gotScheduleTimeDuration) + } + + if !reflect.DeepEqual(gotScheduleTimeDuration, tt.expectedDuration) { + t.Errorf("LastScheduleTimeDuration - got %s, want %s, difference: %s", gotScheduleTimeDuration, tt.expectedDuration, gotScheduleTimeDuration-tt.expectedDuration) + } + }) + } +} + +func topOfTheHour() *time.Time { + T1, err := time.Parse(time.RFC3339, "2016-05-19T10:00:00Z") + if err != nil { + panic("test setup error") + } + return &T1 +} + +func deltaTimeAfterTopOfTheHour(duration time.Duration) *time.Time { + T1, err := time.Parse(time.RFC3339, "2016-05-19T10:00:00Z") + if err != nil { + panic("test setup error") + } + t := T1.Add(duration) + return &t +} diff --git a/ray-operator/go.mod b/ray-operator/go.mod index 7b4971aed11..8c3a92423ea 100644 --- a/ray-operator/go.mod +++ b/ray-operator/go.mod @@ -13,6 +13,7 @@ require ( github.com/orcaman/concurrent-map/v2 v2.0.1 github.com/pkg/errors v0.9.1 github.com/prometheus/client_golang v1.22.0 + github.com/robfig/cron v1.2.0 github.com/robfig/cron/v3 v3.0.1 github.com/stretchr/testify v1.10.0 go.uber.org/mock v0.5.2 diff --git a/ray-operator/go.sum b/ray-operator/go.sum index cd176622933..fe91d9f7103 100644 --- a/ray-operator/go.sum +++ b/ray-operator/go.sum @@ -112,6 +112,8 @@ github.com/prometheus/common v0.62.0 h1:xasJaQlnWAeyHdUBeGjXmutelfJHWMRr+Fg4QszZ github.com/prometheus/common v0.62.0/go.mod h1:vyBcEuLSvWos9B1+CyL7JZ2up+uFzXhkqml0W5zIY1I= github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc= github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= +github.com/robfig/cron v1.2.0 h1:ZjScXvvxeQ63Dbyxy76Fj3AT3Ut0aKsyd2/tl3DTMuQ= +github.com/robfig/cron v1.2.0/go.mod h1:JGuDeoQd7Z6yL4zQhZ3OPEVHB7fL6Ka6skscFHfmt2k= github.com/robfig/cron/v3 v3.0.1 h1:WdRxkvbJztn8LMz/QEvLN5sBU+xKpSqwwUO1Pjr4qDs= github.com/robfig/cron/v3 v3.0.1/go.mod h1:eQICP3HwyT7UooqI/z+Ov+PtYAWygg1TEWWzGIFLtro= github.com/rogpeppe/go-internal v1.13.1 h1:KvO1DLK/DRN07sQ1LQKScxyZJuNnedQ5/wKSR38lUII= From e449c92f492bc8bc5b3c2a6b88a066da73ea8c9b Mon Sep 17 00:00:00 2001 From: DW-Han Date: Tue, 15 Jul 2025 20:03:57 +0000 Subject: [PATCH 10/34] cleaning up comment --- ray-operator/controllers/ray/rayjob_controller.go | 1 - 1 file changed, 1 deletion(-) diff --git a/ray-operator/controllers/ray/rayjob_controller.go b/ray-operator/controllers/ray/rayjob_controller.go index 5db3e8ee1ff..8ca72e78f4f 100644 --- a/ray-operator/controllers/ray/rayjob_controller.go +++ b/ray-operator/controllers/ray/rayjob_controller.go @@ -495,7 +495,6 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) rayJobInstance.Status.JobStatus = rayv1.JobStatusScheduled } else { logger.Info("Waiting until the next reconcile to determine schedule", "nextScheduleDuration", t1, "currentTime", time.Now(), "lastScheduleTimeDuration", t2) - // rayJobInstance.Status.LastScheduleTime = &metav1.Time{Time: time.Now()} return ctrl.Result{RequeueAfter: t1}, nil } From d6a2bfc35836a40111604060c63e92e489523132 Mon Sep 17 00:00:00 2001 From: DW-Han Date: Fri, 18 Jul 2025 00:30:53 +0000 Subject: [PATCH 11/34] seperate case for scheduling --- .../controllers/ray/rayjob_controller.go | 73 ++++++++++++------- 1 file changed, 48 insertions(+), 25 deletions(-) diff --git a/ray-operator/controllers/ray/rayjob_controller.go b/ray-operator/controllers/ray/rayjob_controller.go index 8ca72e78f4f..7ea1806d525 100644 --- a/ray-operator/controllers/ray/rayjob_controller.go +++ b/ray-operator/controllers/ray/rayjob_controller.go @@ -91,7 +91,6 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) // Get RayJob instance var err error rayJobInstance := &rayv1.RayJob{} - if err := r.Get(ctx, request.NamespacedName, rayJobInstance); err != nil { if errors.IsNotFound(err) { // Request object not found, could have been deleted after reconcile request. Stop reconciliation. @@ -323,7 +322,7 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) if jobInfo.EndTime != 0 { rayJobInstance.Status.RayJobStatusInfo.EndTime = &metav1.Time{Time: time.UnixMilli(utils.SafeUint64ToInt64(jobInfo.EndTime))} } - case rayv1.JobDeploymentStatusSuspending, rayv1.JobDeploymentStatusRetrying, rayv1.JobDeploymentStatusScheduling: + case rayv1.JobDeploymentStatusSuspending, rayv1.JobDeploymentStatusRetrying: // The `suspend` operation should be atomic. In other words, if users set the `suspend` flag to true and then immediately // set it back to false, either all of the RayJob's associated resources should be cleaned up, or no resources should be // cleaned up at all. To keep the atomicity, if a RayJob is in the `Suspending` status, we should delete all of its @@ -332,16 +331,9 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) // TODO (kevin85421): Currently, Ray doesn't have a best practice to stop a Ray job gracefully. At this moment, // KubeRay doesn't stop the Ray job before suspending the RayJob. If users want to stop the Ray job by SIGTERM, // users need to set the Pod's preStop hook by themselves. - - // We set isClusterDeleted to true as default, i.e. we dont need to delete the cluster - isClusterDeleted := true - deleteCluster := rayJobInstance.Status.JobDeploymentStatus != rayv1.JobDeploymentStatusScheduling || rayJobInstance.Spec.ShutdownAfterJobFinishes - if deleteCluster { - isClusterDeleted, err = r.deleteClusterResources(ctx, rayJobInstance) - if err != nil { - return ctrl.Result{RequeueAfter: RayJobDefaultRequeueDuration}, err - } - + isClusterDeleted, err := r.deleteClusterResources(ctx, rayJobInstance) + if err != nil { + return ctrl.Result{RequeueAfter: RayJobDefaultRequeueDuration}, err } isJobDeleted, err := r.deleteSubmitterJob(ctx, rayJobInstance) if err != nil { @@ -353,27 +345,21 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) return ctrl.Result{RequeueAfter: RayJobDefaultRequeueDuration}, nil } - // Reset the RayCluster and Ray job related status. Done this way to be atomic. - if deleteCluster { - rayJobInstance.Status.RayClusterStatus = rayv1.RayClusterStatus{} - rayJobInstance.Status.RayClusterName = "" - - } + // Reset the RayCluster and Ray job related status. + rayJobInstance.Status.RayClusterStatus = rayv1.RayClusterStatus{} + rayJobInstance.Status.RayClusterName = "" rayJobInstance.Status.DashboardURL = "" rayJobInstance.Status.JobId = "" rayJobInstance.Status.Message = "" rayJobInstance.Status.Reason = "" rayJobInstance.Status.RayJobStatusInfo = rayv1.RayJobStatusInfo{} - // Reset the JobStatus to JobStatusNew and transition the JobDeploymentStatus to `Suspended`. rayJobInstance.Status.JobStatus = rayv1.JobStatusNew - switch rayJobInstance.Status.JobDeploymentStatus { - case rayv1.JobDeploymentStatusSuspending: + if rayJobInstance.Status.JobDeploymentStatus == rayv1.JobDeploymentStatusSuspending { rayJobInstance.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusSuspended - case rayv1.JobDeploymentStatusScheduling: - rayJobInstance.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusScheduled - case rayv1.JobDeploymentStatusRetrying: + } + if rayJobInstance.Status.JobDeploymentStatus == rayv1.JobDeploymentStatusRetrying { rayJobInstance.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusNew } case rayv1.JobDeploymentStatusSuspended: @@ -475,6 +461,44 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) logger.Info("RayJob is not scheduled") return ctrl.Result{}, nil } + case rayv1.JobDeploymentStatusScheduling: + // We attempt to delete both the cluster and the job. If they're not fully deleted, + // we remain in this state and continue reconciling until resources are freed. + // `isClusterDeleted` is initially true for when we dont need to delete our cluster + isClusterDeleted := true + deleteCluster := rayJobInstance.Spec.ShutdownAfterJobFinishes + if deleteCluster { + isClusterDeleted, err = r.deleteClusterResources(ctx, rayJobInstance) + if err != nil { + return ctrl.Result{RequeueAfter: RayJobDefaultRequeueDuration}, err + } + } + + isJobDeleted, err := r.deleteSubmitterJob(ctx, rayJobInstance) + if err != nil { + return ctrl.Result{RequeueAfter: RayJobDefaultRequeueDuration}, err + } + + if !isClusterDeleted || !isJobDeleted { + logger.Info("The release of the compute resources has not been completed yet. " + + "Wait for the resources to be deleted before the status transitions to avoid a resource leak.") + return ctrl.Result{RequeueAfter: RayJobDefaultRequeueDuration}, nil + } + + if deleteCluster { + rayJobInstance.Status.RayClusterStatus = rayv1.RayClusterStatus{} + rayJobInstance.Status.RayClusterName = "" + + } + rayJobInstance.Status.DashboardURL = "" + rayJobInstance.Status.JobId = "" + rayJobInstance.Status.Message = "" + rayJobInstance.Status.Reason = "" + rayJobInstance.Status.RayJobStatusInfo = rayv1.RayJobStatusInfo{} + + rayJobInstance.Status.JobStatus = rayv1.JobStatusNew + rayJobInstance.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusScheduled + case rayv1.JobDeploymentStatusScheduled: if rayJobInstance.Status.JobStatus == rayv1.JobStatusScheduled { logger.Info("We have reached the new time for a job after reconciling") @@ -511,7 +535,6 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) return ctrl.Result{RequeueAfter: RayJobDefaultRequeueDuration}, err } emitRayJobMetrics(r.options.RayJobMetricsManager, rayJobInstance.Name, rayJobInstance.Namespace, originalRayJobInstance.Status, rayJobInstance.Status) - return ctrl.Result{RequeueAfter: RayJobDefaultRequeueDuration}, nil } From d369a9371787b76e4877fd6ffa42e5bf2e32250b Mon Sep 17 00:00:00 2001 From: DW-Han Date: Fri, 18 Jul 2025 02:34:00 +0000 Subject: [PATCH 12/34] cleaning doc string --- ray-operator/apis/ray/v1/rayjob_types.go | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/ray-operator/apis/ray/v1/rayjob_types.go b/ray-operator/apis/ray/v1/rayjob_types.go index 822d298244d..b9480e3c345 100644 --- a/ray-operator/apis/ray/v1/rayjob_types.go +++ b/ray-operator/apis/ray/v1/rayjob_types.go @@ -205,10 +205,9 @@ type RayJobSpec struct { // In case of transition to false a new RayCluster will be created. // +optional Suspend bool `json:"suspend,omitempty"` - // Schedule specifies a cron like string for cron scheduling - // When set with shutdownAfterJobFinishes, if shutdownAfterJobFinishes is False - // the schduled jobs will keep using the same cluster, if True it will delete - // the cluster and create a new one on the next job. + // Schedule specifies a cron like string for scheduling Ray jobs. + // When shutdownAfterJobFinishes is set to true, a new cluster is provisioned + // per scheduled job, otherwise the job is scheduled on an existing cluster. // +optional Schedule string `json:"schedule,omitempty"` } @@ -259,7 +258,7 @@ type RayJobStatus struct { // +optional ObservedGeneration int64 `json:"observedGeneration,omitempty"` - // The last time the job was successfully scheduled. + // lastScheduledTime is the last time the job was successfully scheduled. // +optional LastScheduleTime *metav1.Time `json:"lastScheduleTime,omitempty"` } From 7a9f4629844d6a4785da1a16b3afd27380309fbd Mon Sep 17 00:00:00 2001 From: DW-Han Date: Fri, 18 Jul 2025 02:51:42 +0000 Subject: [PATCH 13/34] updating api.md --- docs/reference/api.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/api.md b/docs/reference/api.md index 8e0bdf25d10..14b061d2f5b 100644 --- a/docs/reference/api.md +++ b/docs/reference/api.md @@ -253,7 +253,7 @@ _Appears in:_ | `ttlSecondsAfterFinished` _integer_ | TTLSecondsAfterFinished is the TTL to clean up RayCluster.
It's only working when ShutdownAfterJobFinishes set to true. | 0 | | | `shutdownAfterJobFinishes` _boolean_ | ShutdownAfterJobFinishes will determine whether to delete the ray cluster once rayJob succeed or failed. | | | | `suspend` _boolean_ | 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. | | | -| `schedule` _string_ | Schedule specifies a cron like string for cron scheduling
When set with shutdownAfterJobFinishes, if shutdownAfterJobFinishes is False
the schduled jobs will keep using the same cluster, if True it will delete
the cluster and create a new one on the next job. | | | +| `schedule` _string_ | Schedule specifies a cron like string for scheduling Ray jobs.
When shutdownAfterJobFinishes is set to true, a new cluster is provisioned
per scheduled job, otherwise the job is scheduled on an existing cluster. | | | From c06bbedfbcf548faa493f44ae29248b36b01ff0c Mon Sep 17 00:00:00 2001 From: DW-Han Date: Fri, 18 Jul 2025 19:51:22 +0000 Subject: [PATCH 14/34] cleaning up scheduling state, controller, etc. --- ray-operator/apis/ray/v1/rayjob_types.go | 1 - .../controllers/ray/rayjob_controller.go | 27 ++------- .../controllers/ray/utils/schedule_test.go | 59 ------------------- 3 files changed, 5 insertions(+), 82 deletions(-) diff --git a/ray-operator/apis/ray/v1/rayjob_types.go b/ray-operator/apis/ray/v1/rayjob_types.go index b9480e3c345..427f103cb0b 100644 --- a/ray-operator/apis/ray/v1/rayjob_types.go +++ b/ray-operator/apis/ray/v1/rayjob_types.go @@ -21,7 +21,6 @@ const ( JobStatusStopped JobStatus = "STOPPED" JobStatusSucceeded JobStatus = "SUCCEEDED" JobStatusFailed JobStatus = "FAILED" - JobStatusScheduled JobStatus = "SCHEDULED" ) var AllJobStatuses = []JobStatus{ diff --git a/ray-operator/controllers/ray/rayjob_controller.go b/ray-operator/controllers/ray/rayjob_controller.go index 7ea1806d525..f37b988cb93 100644 --- a/ray-operator/controllers/ray/rayjob_controller.go +++ b/ray-operator/controllers/ray/rayjob_controller.go @@ -452,34 +452,23 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) return ctrl.Result{RequeueAfter: RayJobDefaultRequeueDuration}, err } } - if rayJobInstance.Spec.Schedule != "" && rayJobInstance.Status.JobDeploymentStatus != rayv1.JobDeploymentStatusFailed { + if rayJobInstance.Spec.Schedule != "" { logger.Info("RayJob is scheduled again") rayJobInstance.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusScheduling } else { - // NOTE: we do not requeue if the job fails even if scheduled, this could change - // If the RayJob is completed without scheduling or has failed, we should not requeue it. + // If the RayJob is completed without scheduling, we should not requeue it. logger.Info("RayJob is not scheduled") return ctrl.Result{}, nil } case rayv1.JobDeploymentStatusScheduling: - // We attempt to delete both the cluster and the job. If they're not fully deleted, - // we remain in this state and continue reconciling until resources are freed. - // `isClusterDeleted` is initially true for when we dont need to delete our cluster - isClusterDeleted := true deleteCluster := rayJobInstance.Spec.ShutdownAfterJobFinishes - if deleteCluster { - isClusterDeleted, err = r.deleteClusterResources(ctx, rayJobInstance) - if err != nil { - return ctrl.Result{RequeueAfter: RayJobDefaultRequeueDuration}, err - } - } isJobDeleted, err := r.deleteSubmitterJob(ctx, rayJobInstance) if err != nil { return ctrl.Result{RequeueAfter: RayJobDefaultRequeueDuration}, err } - if !isClusterDeleted || !isJobDeleted { + if !isJobDeleted { logger.Info("The release of the compute resources has not been completed yet. " + "Wait for the resources to be deleted before the status transitions to avoid a resource leak.") return ctrl.Result{RequeueAfter: RayJobDefaultRequeueDuration}, nil @@ -498,14 +487,7 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) rayJobInstance.Status.JobStatus = rayv1.JobStatusNew rayJobInstance.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusScheduled - case rayv1.JobDeploymentStatusScheduled: - if rayJobInstance.Status.JobStatus == rayv1.JobStatusScheduled { - logger.Info("We have reached the new time for a job after reconciling") - rayJobInstance.Status.JobStatus = rayv1.JobStatusNew - rayJobInstance.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusNew - break - } // We get the time from the current time to the previous and next cron schedule times // We pass in time.Now() as a parameter so easier unit testing and consistency t1, t2, err := r.getPreviousAndNextScheduleDistance(ctx, time.Now(), rayJobInstance) @@ -516,7 +498,8 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) // Checking if we are currently within a buffer to the previous cron schedule time if t2 <= ScheduleBuffer { logger.Info("The current time is within the buffer window of a cron tick", "NextScheduleTimeDuration", t1, "LastScheduleTimeDuration", t2) - rayJobInstance.Status.JobStatus = rayv1.JobStatusScheduled + rayJobInstance.Status.JobStatus = rayv1.JobStatusNew + rayJobInstance.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusNew } else { logger.Info("Waiting until the next reconcile to determine schedule", "nextScheduleDuration", t1, "currentTime", time.Now(), "lastScheduleTimeDuration", t2) return ctrl.Result{RequeueAfter: t1}, nil diff --git a/ray-operator/controllers/ray/utils/schedule_test.go b/ray-operator/controllers/ray/utils/schedule_test.go index d2394cd35c5..168e008bdb2 100644 --- a/ray-operator/controllers/ray/utils/schedule_test.go +++ b/ray-operator/controllers/ray/utils/schedule_test.go @@ -14,9 +14,6 @@ import ( func TestMostRecentScheduleTime(t *testing.T) { metav1TopOfTheHour := metav1.NewTime(*topOfTheHour()) metav1HalfPastTheHour := metav1.NewTime(*deltaTimeAfterTopOfTheHour(30 * time.Minute)) - // metav1MinuteAfterTopOfTheHour := metav1.NewTime(*deltaTimeAfterTopOfTheHour(1 * time.Minute)) - // oneMinute := int64(60) - // tenSeconds := int64(10) tests := []struct { name string @@ -140,25 +137,6 @@ func TestMostRecentScheduleTime(t *testing.T) { expectedEarliestTime: *topOfTheHour(), expectedTooManyMissed: fewMissed, }, - // { - // name: "@every schedule", - // cj: &rayv1.RayJob{ - // ObjectMeta: metav1.ObjectMeta{ - // CreationTimestamp: metav1.NewTime(*deltaTimeAfterTopOfTheHour(-59 * time.Minute)), - // }, - // Spec: rayv1.RayJobSpec{ - // Schedule: "@every 1h", - // StartingDeadlineSeconds: &tenSeconds, - // }, - // Status: rayv1.RayJobStatus{ - // LastScheduleTime: &metav1MinuteAfterTopOfTheHour, - // }, - // }, - // now: *deltaTimeAfterTopOfTheHour(7 * 24 * time.Hour), - // expectedRecentTime: deltaTimeAfterTopOfTheHour((6 * 24 * time.Hour) + 23*time.Hour + 1*time.Minute), - // expectedEarliestTime: *deltaTimeAfterTopOfTheHour(1 * time.Minute), - // expectedTooManyMissed: manyMissed, - // }, { name: "rogue cronjob", cj: &rayv1.RayJob{ @@ -207,43 +185,6 @@ func TestMostRecentScheduleTime(t *testing.T) { expectedEarliestTime: *deltaTimeAfterTopOfTheHour(30 * time.Minute), expectedRecentTime: nil, }, - // { - // name: "earliestTime being LastScheduleTime (within StartingDeadlineSeconds)", - // cj: &rayv1.RayJob{ - // ObjectMeta: metav1.ObjectMeta{ - // CreationTimestamp: metav1TopOfTheHour, - // }, - // Spec: rayv1.RayJobSpec{ - // Schedule: "*/5 * * * *", - // StartingDeadlineSeconds: &oneMinute, - // }, - // Status: rayv1.RayJobStatus{ - // LastScheduleTime: &metav1HalfPastTheHour, - // }, - // }, - // now: *deltaTimeAfterTopOfTheHour(31 * time.Minute), - // expectedEarliestTime: *deltaTimeAfterTopOfTheHour(30 * time.Minute), - // expectedRecentTime: nil, - // }, - // { - // name: "earliestTime being LastScheduleTime (outside StartingDeadlineSeconds)", - // cj: &rayv1.RayJob{ - // ObjectMeta: metav1.ObjectMeta{ - // CreationTimestamp: metav1TopOfTheHour, - // }, - // Spec: rayv1.RayJobSpec{ - // Schedule: "*/5 * * * *", - // StartingDeadlineSeconds: &oneMinute, - // }, - // Status: rayv1.RayJobStatus{ - // LastScheduleTime: &metav1HalfPastTheHour, - // }, - // }, - // includeSDS: true, - // now: *deltaTimeAfterTopOfTheHour(32 * time.Minute), - // expectedEarliestTime: *deltaTimeAfterTopOfTheHour(31 * time.Minute), - // expectedRecentTime: nil, - // }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { From 6da922646084845febb5163be11bf7b588f36877 Mon Sep 17 00:00:00 2001 From: DW-Han Date: Mon, 21 Jul 2025 01:25:05 +0000 Subject: [PATCH 15/34] integration tests --- .../ray/rayjob_controller_scheduled_test.go | 242 ++++++++++++++++++ 1 file changed, 242 insertions(+) create mode 100644 ray-operator/controllers/ray/rayjob_controller_scheduled_test.go diff --git a/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go b/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go new file mode 100644 index 00000000000..17f81a124d3 --- /dev/null +++ b/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go @@ -0,0 +1,242 @@ +/* + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package ray + +import ( + "context" + "time" + + . "github.com/onsi/ginkgo/v2" + . "github.com/onsi/gomega" + batchv1 "k8s.io/api/batch/v1" + "k8s.io/apimachinery/pkg/api/errors" + apierrors "k8s.io/apimachinery/pkg/api/errors" + "k8s.io/client-go/util/retry" // For creating pointers to int32, string, bool etc. + "sigs.k8s.io/controller-runtime/pkg/client" + + rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1" + "github.com/ray-project/kuberay/ray-operator/controllers/ray/common" + // Assuming utils.RayOriginatedFromCRNameLabelKey, etc. +) + +func scheduledRayJobTemplate(name, namespace, schedule string) *rayv1.RayJob { + job := rayJobTemplate(name, namespace) + job.Spec.Schedule = schedule + return job +} + +var _ = Context("RayJob with schedule operation", func() { + // This Describe block focuses on the lifecycle of a RayJob configured with a schedule. + Describe("When creating a RayJob with a schedule field and NO cluster deletion", Ordered, func() { + ctx := context.Background() + namespace := "default" + cronSchedule := "0 0 0 0 0" + rayJob := scheduledRayJobTemplate("rayjob-scheduled-test", namespace, cronSchedule) + rayCluster := &rayv1.RayCluster{} + + It("should create a RayJob object with the schedule", func() { + err := k8sClient.Create(ctx, rayJob) + Expect(err).NotTo(HaveOccurred(), "failed to create test scheduled RayJob resource") + }) + + It("should have a JobDeploymentStatus reflecting its scheduled state", func() { + Eventually( + getRayJobDeploymentStatus(ctx, rayJob), + time.Second*5, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusScheduled), + "JobDeploymentStatus should be Scheduled") + }) + + It("should NOT create a raycluster object immediately", func() { + Consistently( + getRayClusterNameForRayJob(ctx, rayJob), + time.Second*3, time.Millisecond*500).Should(BeEmpty()) + }) + + It("should update the CronJob's schedule when the RayJob's schedule is modified", func() { + newCronSchedule := "*/1 * * * *" + updatedRayJob := &rayv1.RayJob{} + err := retry.RetryOnConflict(retry.DefaultRetry, func() error { + err := k8sClient.Get(ctx, client.ObjectKey{Name: rayJob.Name, Namespace: namespace}, updatedRayJob) + if err != nil { + return err + } + updatedRayJob.Spec.Schedule = newCronSchedule + return k8sClient.Update(ctx, updatedRayJob) + }) + Expect(err).NotTo(HaveOccurred(), "failed to update RayJob schedule") + }) + + It("should transition to the New state", func() { + Eventually( + getRayJobDeploymentStatus(ctx, rayJob), + time.Second*5, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusNew), + "JobDeploymentStatus should be New") + }) + It("should create a raycluster object", func() { + // Ray Cluster name can be present on RayJob's CRD + Eventually( + getRayClusterNameForRayJob(ctx, rayJob), + time.Second*15, time.Millisecond*500).Should(Not(BeEmpty())) + // The actual cluster instance and underlying resources SHOULD be created when suspend == false + Eventually( + + getResourceFunc(ctx, common.RayJobRayClusterNamespacedName(rayJob), rayCluster), + time.Second*3, time.Millisecond*500).Should(Succeed()) + }) + + It("should NOT create the underlying K8s job yet because the cluster is not ready", func() { + underlyingK8sJob := &batchv1.Job{} + Consistently( + // k8sClient client throws error if resource not found + func() bool { + err := getResourceFunc(ctx, common.RayJobK8sJobNamespacedName(rayJob), underlyingK8sJob)() + return errors.IsNotFound(err) + }, + time.Second*3, time.Millisecond*500).Should(BeTrue()) + }) + + It("should be able to update all Pods to Running", func() { + updateHeadPodToRunningAndReady(ctx, rayJob.Status.RayClusterName, namespace) + updateWorkerPodsToRunningAndReady(ctx, rayJob.Status.RayClusterName, namespace) + }) + + It("Dashboard URL should be set", func() { + Eventually( + getDashboardURLForRayJob(ctx, rayJob), + time.Second*3, time.Millisecond*500).Should(HavePrefix(rayJob.Name), "Dashboard URL = %v", rayJob.Status.DashboardURL) + }) + + It("should create the underlying Kubernetes Job object", func() { + underlyingK8sJob := &batchv1.Job{} + // The underlying Kubernetes Job should be created when the RayJob is created + Eventually( + getResourceFunc(ctx, common.RayJobK8sJobNamespacedName(rayJob), underlyingK8sJob), + time.Second*3, time.Millisecond*500).Should(Succeed(), "Expected Kubernetes job to be present") + }) + + It("should transition to the Initailizing", func() { + Eventually( + getRayJobDeploymentStatus(ctx, rayJob), + time.Second*5, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusInitializing), + "JobDeploymentStatus should be Initializing") + }) + + It("should transition to the Running", func() { + Eventually( + getRayJobDeploymentStatus(ctx, rayJob), + time.Second*5, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusRunning), + "JobDeploymentStatus should be Initializing") + }) + + It("should transition to the Complete", func() { + Eventually( + getRayJobDeploymentStatus(ctx, rayJob), + time.Second*5, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusComplete), + "JobDeploymentStatus should be Initializing") + }) + + It("should transition to the Scheduled", func() { + Eventually( + getRayJobDeploymentStatus(ctx, rayJob), + time.Second*5, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusScheduled), + "JobDeploymentStatus should be Initializing") + }) + + It("The raycluster object should still exist", func() { + Eventually( + + getResourceFunc(ctx, common.RayJobRayClusterNamespacedName(rayJob), rayCluster), + time.Second*3, time.Millisecond*500).Should(Succeed()) + }) + }) + + Describe("When creating a RayJob with a schedule field and WITH cluster deletion", Ordered, func() { + ctx := context.Background() + namespace := "default" + cronSchedule := "*/1 * * * *" + rayJob := scheduledRayJobTemplate("rayjob-scheduled-test", namespace, cronSchedule) + rayCluster := &rayv1.RayCluster{} + + It("should create a RayJob object with the schedule", func() { + err := k8sClient.Create(ctx, rayJob) + Expect(err).NotTo(HaveOccurred(), "failed to create test scheduled RayJob resource") + }) + + It("should have a JobDeploymentStatus reflecting its scheduled state", func() { + Eventually( + getRayJobDeploymentStatus(ctx, rayJob), + time.Second*5, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusScheduled), + "JobDeploymentStatus should be Scheduled") + }) + + It("should NOT create a RayCluster object immediately", func() { + rayCluster := &rayv1.RayCluster{} + Consistently( + func() bool { + err := k8sClient.Get(ctx, common.RayJobRayClusterNamespacedName(rayJob), rayCluster) + return apierrors.IsNotFound(err) + }, + time.Second*3, time.Millisecond*500).Should(BeTrue(), "RayCluster should NOT be created upon scheduled RayJob creation") + }) + + It("should update the CronJob's schedule when the RayJob's schedule is modified", func() { + newCronSchedule := "0 1 * * *" + updatedRayJob := &rayv1.RayJob{} + err := retry.RetryOnConflict(retry.DefaultRetry, func() error { + err := k8sClient.Get(ctx, client.ObjectKey{Name: rayJob.Name, Namespace: namespace}, updatedRayJob) + if err != nil { + return err + } + updatedRayJob.Spec.Schedule = newCronSchedule + return k8sClient.Update(ctx, updatedRayJob) + }) + Expect(err).NotTo(HaveOccurred(), "failed to update RayJob schedule") + }) + + It("should transition to the New state", func() { + Eventually( + getRayJobDeploymentStatus(ctx, rayJob), + time.Second*5, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusNew), + "JobDeploymentStatus should be New") + }) + + It("should create a raycluster object", func() { + // Ray Cluster name can be present on RayJob's CRD + Eventually( + getRayClusterNameForRayJob(ctx, rayJob), + time.Second*15, time.Millisecond*500).Should(Not(BeEmpty())) + // The actual cluster instance and underlying resources SHOULD be created when suspend == false + Eventually( + + getResourceFunc(ctx, common.RayJobRayClusterNamespacedName(rayJob), rayCluster), + time.Second*3, time.Millisecond*500).Should(Succeed()) + }) + + It("should transition to the Scheduled", func() { + Eventually( + getRayJobDeploymentStatus(ctx, rayJob), + time.Second*5, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusScheduled), + "JobDeploymentStatus should be Initializing") + }) + + It("The raycluster object should be deleted", func() { + Consistently( + getRayClusterNameForRayJob(ctx, rayJob), + time.Second*3, time.Millisecond*500).Should(BeEmpty()) + }) + + }) +}) From 93adf7cec3c1692262f5cecbd1d0591c1a5888a8 Mon Sep 17 00:00:00 2001 From: DW-Han Date: Mon, 21 Jul 2025 04:05:12 +0000 Subject: [PATCH 16/34] cleaning controller --- ray-operator/controllers/ray/rayjob_controller.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/ray-operator/controllers/ray/rayjob_controller.go b/ray-operator/controllers/ray/rayjob_controller.go index f37b988cb93..cf19cbd5f30 100644 --- a/ray-operator/controllers/ray/rayjob_controller.go +++ b/ray-operator/controllers/ray/rayjob_controller.go @@ -455,11 +455,11 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) if rayJobInstance.Spec.Schedule != "" { logger.Info("RayJob is scheduled again") rayJobInstance.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusScheduling - } else { - // If the RayJob is completed without scheduling, we should not requeue it. - logger.Info("RayJob is not scheduled") - return ctrl.Result{}, nil + break } + + // If the RayJob is completed, we should not requeue it. + return ctrl.Result{}, nil case rayv1.JobDeploymentStatusScheduling: deleteCluster := rayJobInstance.Spec.ShutdownAfterJobFinishes From 8055fa70af0a3c18e048472c3889dd6b98299e10 Mon Sep 17 00:00:00 2001 From: DW-Han Date: Mon, 21 Jul 2025 21:30:13 +0000 Subject: [PATCH 17/34] cleaning and lint --- docs/reference/api.md | 2 +- ray-operator/apis/ray/v1/rayjob_types.go | 17 +- .../apis/ray/v1/zz_generated.deepcopy.go | 4 + .../config/samples/ray-job.schedule.yaml | 4 +- .../controllers/ray/rayjob_controller.go | 2 +- .../ray/rayjob_controller_scheduled_test.go | 201 ++++++++---------- .../ray/rayjob_controller_unit_test.go | 9 +- .../controllers/ray/utils/schedule.go | 11 +- .../controllers/ray/utils/schedule_test.go | 19 +- .../applyconfiguration/ray/v1/rayjobspec.go | 18 +- .../applyconfiguration/ray/v1/rayjobstatus.go | 18 +- 11 files changed, 137 insertions(+), 168 deletions(-) diff --git a/docs/reference/api.md b/docs/reference/api.md index 14b061d2f5b..694b215c231 100644 --- a/docs/reference/api.md +++ b/docs/reference/api.md @@ -248,12 +248,12 @@ _Appears in:_ | `jobId` _string_ | If jobId is not set, a new jobId will be auto-generated. | | | | `submissionMode` _[JobSubmissionMode](#jobsubmissionmode)_ | 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. | K8sJobMode | | | `entrypointResources` _string_ | EntrypointResources specifies the custom resources and quantities to reserve for the
entrypoint command. | | | +| `schedule` _string_ | Schedule specifies a cron like string for scheduling Ray jobs.
When shutdownAfterJobFinishes is set to true, a new cluster is provisioned
per scheduled job, otherwise the job is scheduled on an existing cluster. | | | | `entrypointNumCpus` _float_ | EntrypointNumCpus specifies the number of cpus to reserve for the entrypoint command. | | | | `entrypointNumGpus` _float_ | EntrypointNumGpus specifies the number of gpus to reserve for the entrypoint command. | | | | `ttlSecondsAfterFinished` _integer_ | TTLSecondsAfterFinished is the TTL to clean up RayCluster.
It's only working when ShutdownAfterJobFinishes set to true. | 0 | | | `shutdownAfterJobFinishes` _boolean_ | ShutdownAfterJobFinishes will determine whether to delete the ray cluster once rayJob succeed or failed. | | | | `suspend` _boolean_ | 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. | | | -| `schedule` _string_ | Schedule specifies a cron like string for scheduling Ray jobs.
When shutdownAfterJobFinishes is set to true, a new cluster is provisioned
per scheduled job, otherwise the job is scheduled on an existing cluster. | | | diff --git a/ray-operator/apis/ray/v1/rayjob_types.go b/ray-operator/apis/ray/v1/rayjob_types.go index 427f103cb0b..bb600f61dfe 100644 --- a/ray-operator/apis/ray/v1/rayjob_types.go +++ b/ray-operator/apis/ray/v1/rayjob_types.go @@ -183,6 +183,11 @@ type RayJobSpec struct { // entrypoint command. // +optional EntrypointResources string `json:"entrypointResources,omitempty"` + // Schedule specifies a cron like string for scheduling Ray jobs. + // When shutdownAfterJobFinishes is set to true, a new cluster is provisioned + // per scheduled job, otherwise the job is scheduled on an existing cluster. + // +optional + Schedule string `json:"schedule,omitempty"` // EntrypointNumCpus specifies the number of cpus to reserve for the entrypoint command. // +optional EntrypointNumCpus float32 `json:"entrypointNumCpus,omitempty"` @@ -204,11 +209,6 @@ type RayJobSpec struct { // In case of transition to false a new RayCluster will be created. // +optional Suspend bool `json:"suspend,omitempty"` - // Schedule specifies a cron like string for scheduling Ray jobs. - // When shutdownAfterJobFinishes is set to true, a new cluster is provisioned - // per scheduled job, otherwise the job is scheduled on an existing cluster. - // +optional - Schedule string `json:"schedule,omitempty"` } // RayJobStatus defines the observed state of RayJob @@ -240,6 +240,9 @@ type RayJobStatus struct { // or the submitter Job has failed. // +optional EndTime *metav1.Time `json:"endTime,omitempty"` + // lastScheduledTime is the last time the job was successfully scheduled. + // +optional + LastScheduleTime *metav1.Time `json:"lastScheduleTime,omitempty"` // Succeeded is the number of times this job succeeded. // +kubebuilder:default:=0 // +optional @@ -256,10 +259,6 @@ type RayJobStatus struct { // RayJob's generation, which is updated on mutation by the API Server. // +optional ObservedGeneration int64 `json:"observedGeneration,omitempty"` - - // lastScheduledTime is the last time the job was successfully scheduled. - // +optional - LastScheduleTime *metav1.Time `json:"lastScheduleTime,omitempty"` } // +kubebuilder:object:root=true diff --git a/ray-operator/apis/ray/v1/zz_generated.deepcopy.go b/ray-operator/apis/ray/v1/zz_generated.deepcopy.go index 05e119b2165..8c436ea68b4 100644 --- a/ray-operator/apis/ray/v1/zz_generated.deepcopy.go +++ b/ray-operator/apis/ray/v1/zz_generated.deepcopy.go @@ -518,6 +518,10 @@ func (in *RayJobStatus) DeepCopyInto(out *RayJobStatus) { **out = **in } in.RayClusterStatus.DeepCopyInto(&out.RayClusterStatus) + if in.LastScheduleTime != nil { + in, out := &in.LastScheduleTime, &out.LastScheduleTime + *out = (*in).DeepCopy() + } } // DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new RayJobStatus. diff --git a/ray-operator/config/samples/ray-job.schedule.yaml b/ray-operator/config/samples/ray-job.schedule.yaml index ccf299a945f..ec22a9b2097 100644 --- a/ray-operator/config/samples/ray-job.schedule.yaml +++ b/ray-operator/config/samples/ray-job.schedule.yaml @@ -6,9 +6,9 @@ spec: # schedule specifires a cron scheduling string telling the rayjob when to start schedule and run new jobs # Here it runs at every 5 minutes of every hour of every day of every week of every year schedule: "*/5 * * * *" - + entrypoint: python /home/ray/samples/sample_code.py - + # shutdownAfterJobFinishes specifies whether the RayCluster should be deleted after the RayJob finishes. Default is false. # NOTE that the expected behavior with schedule is that the cluster will be deleted and recreated at each schedule if set to true, and it will keep using the same cluster otherwise shutdownAfterJobFinishes: true diff --git a/ray-operator/controllers/ray/rayjob_controller.go b/ray-operator/controllers/ray/rayjob_controller.go index cf19cbd5f30..3c49eeb1cc4 100644 --- a/ray-operator/controllers/ray/rayjob_controller.go +++ b/ray-operator/controllers/ray/rayjob_controller.go @@ -858,7 +858,7 @@ func initRayJobStatusIfNeed(ctx context.Context, rayJob *rayv1.RayJob) error { // if the rayjob is scheduled according to a cron string set the status to scheduling instead of initializing to begin with // we check the job count to know if its the first job if rayJob.Spec.Schedule != "" && rayJob.Status.Failed == nil && rayJob.Status.Succeeded == nil { - logger.Info("Since this is a new schdueled job we enter the Scheduled state not Initalizing") + logger.Info("Since this is a new schdueled job we enter the Scheduled state not Initializing") rayJob.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusScheduled } else { rayJob.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusInitializing diff --git a/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go b/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go index 17f81a124d3..f9c2e73ef84 100644 --- a/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go +++ b/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go @@ -21,15 +21,14 @@ import ( . "github.com/onsi/ginkgo/v2" . "github.com/onsi/gomega" - batchv1 "k8s.io/api/batch/v1" - "k8s.io/apimachinery/pkg/api/errors" + // batchv1 "k8s.io/api/batch/v1" + // "k8s.io/apimachinery/pkg/api/errors" apierrors "k8s.io/apimachinery/pkg/api/errors" "k8s.io/client-go/util/retry" // For creating pointers to int32, string, bool etc. "sigs.k8s.io/controller-runtime/pkg/client" rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1" "github.com/ray-project/kuberay/ray-operator/controllers/ray/common" - // Assuming utils.RayOriginatedFromCRNameLabelKey, etc. ) func scheduledRayJobTemplate(name, namespace, schedule string) *rayv1.RayJob { @@ -44,8 +43,8 @@ var _ = Context("RayJob with schedule operation", func() { ctx := context.Background() namespace := "default" cronSchedule := "0 0 0 0 0" - rayJob := scheduledRayJobTemplate("rayjob-scheduled-test", namespace, cronSchedule) - rayCluster := &rayv1.RayCluster{} + rayJob := scheduledRayJobTemplate("rayjob-scheduled-no-deletion", namespace, cronSchedule) + // rayCluster := &rayv1.RayCluster{} It("should create a RayJob object with the schedule", func() { err := k8sClient.Create(ctx, rayJob) @@ -59,12 +58,6 @@ var _ = Context("RayJob with schedule operation", func() { "JobDeploymentStatus should be Scheduled") }) - It("should NOT create a raycluster object immediately", func() { - Consistently( - getRayClusterNameForRayJob(ctx, rayJob), - time.Second*3, time.Millisecond*500).Should(BeEmpty()) - }) - It("should update the CronJob's schedule when the RayJob's schedule is modified", func() { newCronSchedule := "*/1 * * * *" updatedRayJob := &rayv1.RayJob{} @@ -79,96 +72,96 @@ var _ = Context("RayJob with schedule operation", func() { Expect(err).NotTo(HaveOccurred(), "failed to update RayJob schedule") }) + // The cron job runs every minute so it will take at most 1 minute to run It("should transition to the New state", func() { Eventually( getRayJobDeploymentStatus(ctx, rayJob), - time.Second*5, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusNew), + time.Second*60).Should(Equal(rayv1.JobDeploymentStatusNew), "JobDeploymentStatus should be New") }) - It("should create a raycluster object", func() { - // Ray Cluster name can be present on RayJob's CRD - Eventually( - getRayClusterNameForRayJob(ctx, rayJob), - time.Second*15, time.Millisecond*500).Should(Not(BeEmpty())) - // The actual cluster instance and underlying resources SHOULD be created when suspend == false - Eventually( - - getResourceFunc(ctx, common.RayJobRayClusterNamespacedName(rayJob), rayCluster), - time.Second*3, time.Millisecond*500).Should(Succeed()) - }) - - It("should NOT create the underlying K8s job yet because the cluster is not ready", func() { - underlyingK8sJob := &batchv1.Job{} - Consistently( - // k8sClient client throws error if resource not found - func() bool { - err := getResourceFunc(ctx, common.RayJobK8sJobNamespacedName(rayJob), underlyingK8sJob)() - return errors.IsNotFound(err) - }, - time.Second*3, time.Millisecond*500).Should(BeTrue()) - }) - - It("should be able to update all Pods to Running", func() { - updateHeadPodToRunningAndReady(ctx, rayJob.Status.RayClusterName, namespace) - updateWorkerPodsToRunningAndReady(ctx, rayJob.Status.RayClusterName, namespace) - }) - - It("Dashboard URL should be set", func() { - Eventually( - getDashboardURLForRayJob(ctx, rayJob), - time.Second*3, time.Millisecond*500).Should(HavePrefix(rayJob.Name), "Dashboard URL = %v", rayJob.Status.DashboardURL) - }) - - It("should create the underlying Kubernetes Job object", func() { - underlyingK8sJob := &batchv1.Job{} - // The underlying Kubernetes Job should be created when the RayJob is created - Eventually( - getResourceFunc(ctx, common.RayJobK8sJobNamespacedName(rayJob), underlyingK8sJob), - time.Second*3, time.Millisecond*500).Should(Succeed(), "Expected Kubernetes job to be present") - }) - - It("should transition to the Initailizing", func() { - Eventually( - getRayJobDeploymentStatus(ctx, rayJob), - time.Second*5, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusInitializing), - "JobDeploymentStatus should be Initializing") - }) - - It("should transition to the Running", func() { - Eventually( - getRayJobDeploymentStatus(ctx, rayJob), - time.Second*5, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusRunning), - "JobDeploymentStatus should be Initializing") - }) - - It("should transition to the Complete", func() { - Eventually( - getRayJobDeploymentStatus(ctx, rayJob), - time.Second*5, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusComplete), - "JobDeploymentStatus should be Initializing") - }) - - It("should transition to the Scheduled", func() { - Eventually( - getRayJobDeploymentStatus(ctx, rayJob), - time.Second*5, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusScheduled), - "JobDeploymentStatus should be Initializing") - }) - - It("The raycluster object should still exist", func() { - Eventually( - - getResourceFunc(ctx, common.RayJobRayClusterNamespacedName(rayJob), rayCluster), - time.Second*3, time.Millisecond*500).Should(Succeed()) - }) + // It("should create a raycluster object", func() { + // // Ray Cluster name can be present on RayJob's CRD + // Eventually( + // getRayClusterNameForRayJob(ctx, rayJob), + // time.Second*15, time.Millisecond*500).Should(Not(BeEmpty())) + // // The actual cluster instance and underlying resources SHOULD be created when suspend == false + // Eventually( + + // getResourceFunc(ctx, common.RayJobRayClusterNamespacedName(rayJob), rayCluster), + // time.Second*3, time.Millisecond*500).Should(Succeed()) + // }) + + // It("should NOT create the underlying K8s job yet because the cluster is not ready", func() { + // underlyingK8sJob := &batchv1.Job{} + // Consistently( + // // k8sClient client throws error if resource not found + // func() bool { + // err := getResourceFunc(ctx, common.RayJobK8sJobNamespacedName(rayJob), underlyingK8sJob)() + // return errors.IsNotFound(err) + // }, + // time.Second*3, time.Millisecond*500).Should(BeTrue()) + // }) + + // It("should be able to update all Pods to Running", func() { + // updateHeadPodToRunningAndReady(ctx, rayJob.Status.RayClusterName, namespace) + // updateWorkerPodsToRunningAndReady(ctx, rayJob.Status.RayClusterName, namespace) + // }) + + // It("Dashboard URL should be set", func() { + // Eventually( + // getDashboardURLForRayJob(ctx, rayJob), + // time.Second*3, time.Millisecond*500).Should(HavePrefix(rayJob.Name), "Dashboard URL = %v", rayJob.Status.DashboardURL) + // }) + + // It("should create the underlying Kubernetes Job object", func() { + // underlyingK8sJob := &batchv1.Job{} + // // The underlying Kubernetes Job should be created when the RayJob is created + // Eventually( + // getResourceFunc(ctx, common.RayJobK8sJobNamespacedName(rayJob), underlyingK8sJob), + // time.Second*3, time.Millisecond*500).Should(Succeed(), "Expected Kubernetes job to be present") + // }) + + // It("should transition to the Initailizing", func() { + // Eventually( + // getRayJobDeploymentStatus(ctx, rayJob), + // time.Second*5, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusInitializing), + // "JobDeploymentStatus should be Initializing") + // }) + + // It("should transition to the Running", func() { + // Eventually( + // getRayJobDeploymentStatus(ctx, rayJob), + // time.Second*5, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusRunning), + // "JobDeploymentStatus should be Initializing") + // }) + + // It("should transition to the Complete", func() { + // Eventually( + // getRayJobDeploymentStatus(ctx, rayJob), + // time.Second*5, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusComplete), + // "JobDeploymentStatus should be Initializing") + // }) + + // It("should transition to the Scheduled", func() { + // Eventually( + // getRayJobDeploymentStatus(ctx, rayJob), + // time.Second*5, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusScheduled), + // "JobDeploymentStatus should be Initializing") + // }) + + // It("The raycluster object should still exist", func() { + // Eventually( + + // getResourceFunc(ctx, common.RayJobRayClusterNamespacedName(rayJob), rayCluster), + // time.Second*3, time.Millisecond*500).Should(Succeed()) + // }) }) Describe("When creating a RayJob with a schedule field and WITH cluster deletion", Ordered, func() { ctx := context.Background() namespace := "default" cronSchedule := "*/1 * * * *" - rayJob := scheduledRayJobTemplate("rayjob-scheduled-test", namespace, cronSchedule) - rayCluster := &rayv1.RayCluster{} + rayJob := scheduledRayJobTemplate("rayjob-scheduled-with-deletion", namespace, cronSchedule) It("should create a RayJob object with the schedule", func() { err := k8sClient.Create(ctx, rayJob) @@ -209,34 +202,8 @@ var _ = Context("RayJob with schedule operation", func() { It("should transition to the New state", func() { Eventually( getRayJobDeploymentStatus(ctx, rayJob), - time.Second*5, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusNew), + time.Second*60).Should(Equal(rayv1.JobDeploymentStatusNew), "JobDeploymentStatus should be New") }) - - It("should create a raycluster object", func() { - // Ray Cluster name can be present on RayJob's CRD - Eventually( - getRayClusterNameForRayJob(ctx, rayJob), - time.Second*15, time.Millisecond*500).Should(Not(BeEmpty())) - // The actual cluster instance and underlying resources SHOULD be created when suspend == false - Eventually( - - getResourceFunc(ctx, common.RayJobRayClusterNamespacedName(rayJob), rayCluster), - time.Second*3, time.Millisecond*500).Should(Succeed()) - }) - - It("should transition to the Scheduled", func() { - Eventually( - getRayJobDeploymentStatus(ctx, rayJob), - time.Second*5, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusScheduled), - "JobDeploymentStatus should be Initializing") - }) - - It("The raycluster object should be deleted", func() { - Consistently( - getRayClusterNameForRayJob(ctx, rayJob), - time.Second*3, time.Millisecond*500).Should(BeEmpty()) - }) - }) }) diff --git a/ray-operator/controllers/ray/rayjob_controller_unit_test.go b/ray-operator/controllers/ray/rayjob_controller_unit_test.go index 9d795d7f96e..0f4127cbb59 100644 --- a/ray-operator/controllers/ray/rayjob_controller_unit_test.go +++ b/ray-operator/controllers/ray/rayjob_controller_unit_test.go @@ -635,13 +635,13 @@ func TestGetPreviousAndNextScheduleDistance(t *testing.T) { _ = corev1.AddToScheme(newScheme) // For events testCases := []struct { + currentTime time.Time + initialLastTime *metav1.Time name string schedule string - initialLastTime *metav1.Time - currentTime time.Time - expectedErr bool expectedNextDelta time.Duration expectedPrevDelta time.Duration + expectedErr bool isWithinBuffer bool }{ { @@ -695,7 +695,7 @@ func TestGetPreviousAndNextScheduleDistance(t *testing.T) { Scheme: newScheme, } - // Call getPreviousAndNextScheduleDistance to get the next and pervious schedule ticks + // Call getPreviousAndNextScheduleDistance to get the next and previous schedule ticks nextDuration, prevDuration, err := reconciler.getPreviousAndNextScheduleDistance(context.Background(), tc.currentTime, rayJob) if tc.expectedErr { @@ -717,5 +717,4 @@ func TestGetPreviousAndNextScheduleDistance(t *testing.T) { } }) } - } diff --git a/ray-operator/controllers/ray/utils/schedule.go b/ray-operator/controllers/ray/utils/schedule.go index e522d28c8c5..9b4df49596a 100644 --- a/ray-operator/controllers/ray/utils/schedule.go +++ b/ray-operator/controllers/ray/utils/schedule.go @@ -14,21 +14,22 @@ import ( "time" "github.com/go-logr/logr" - rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1" "github.com/robfig/cron/v3" corev1 "k8s.io/api/core/v1" "k8s.io/client-go/tools/record" + + rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1" ) -type missedSchedulesType int +type MissedSchedulesType int const ( - noneMissed missedSchedulesType = iota + noneMissed MissedSchedulesType = iota fewMissed manyMissed ) -func MostRecentScheduleTime(rj *rayv1.RayJob, now time.Time, schedule cron.Schedule) (time.Time, *time.Time, missedSchedulesType, error) { +func MostRecentScheduleTime(rj *rayv1.RayJob, now time.Time, schedule cron.Schedule) (time.Time, *time.Time, MissedSchedulesType, error) { earliestTime := rj.ObjectMeta.CreationTimestamp.Time missedSchedules := noneMissed if rj.Status.LastScheduleTime != nil { @@ -118,7 +119,6 @@ func FormatSchedule(rj *rayv1.RayJob, recorder record.EventRecorder) string { // nextScheduleTimeDuration returns the time duration to requeue a cron schedule based on // the schedule and last schedule time. func NextScheduleTimeDuration(logger logr.Logger, rj *rayv1.RayJob, now time.Time, schedule cron.Schedule) time.Duration { - earliestTime, mostRecentTime, missedSchedules, err := MostRecentScheduleTime(rj, now, schedule) if err != nil { // we still have to requeue at some point, so aim for the next scheduling slot from now @@ -143,7 +143,6 @@ func NextScheduleTimeDuration(logger logr.Logger, rj *rayv1.RayJob, now time.Tim // It calculates the most recent time a schedule should have executed based // on the RayJob's creation time (or its last scheduled status) and the current time 'now'. func LastScheduleTimeDuration(logger logr.Logger, rj *rayv1.RayJob, now time.Time, schedule cron.Schedule) time.Duration { - earliestTime, mostRecentTime, missedSchedules, err := MostRecentScheduleTime(rj, now, schedule) if err != nil { // We still have to requeue at some point, so aim for the next scheduling slot from now diff --git a/ray-operator/controllers/ray/utils/schedule_test.go b/ray-operator/controllers/ray/utils/schedule_test.go index 168e008bdb2..a92264a48b8 100644 --- a/ray-operator/controllers/ray/utils/schedule_test.go +++ b/ray-operator/controllers/ray/utils/schedule_test.go @@ -6,9 +6,10 @@ import ( "time" "github.com/go-logr/logr/testr" - rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1" cron "github.com/robfig/cron/v3" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" + + rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1" ) func TestMostRecentScheduleTime(t *testing.T) { @@ -16,13 +17,13 @@ func TestMostRecentScheduleTime(t *testing.T) { metav1HalfPastTheHour := metav1.NewTime(*deltaTimeAfterTopOfTheHour(30 * time.Minute)) tests := []struct { - name string - cj *rayv1.RayJob - includeSDS bool now time.Time expectedEarliestTime time.Time + cj *rayv1.RayJob expectedRecentTime *time.Time - expectedTooManyMissed missedSchedulesType + name string + expectedTooManyMissed MissedSchedulesType + includeSDS bool wantErr bool }{ { @@ -225,9 +226,9 @@ func TestNextScheduleTimeDuration(t *testing.T) { metav1TwoHoursLater := metav1.NewTime(*deltaTimeAfterTopOfTheHour(2 * time.Hour)) tests := []struct { - name string - cj *rayv1.RayJob now time.Time + cj *rayv1.RayJob + name string expectedDuration time.Duration }{ { @@ -319,9 +320,9 @@ func TestLastScheduleTimeDuration(t *testing.T) { metav1FiveMonthsAgo := metav1.NewTime(metav1TopOfTheHour.AddDate(0, -5, 0)) tests := []struct { - name string - cj *rayv1.RayJob now time.Time + cj *rayv1.RayJob + name string expectedDuration time.Duration }{ { diff --git a/ray-operator/pkg/client/applyconfiguration/ray/v1/rayjobspec.go b/ray-operator/pkg/client/applyconfiguration/ray/v1/rayjobspec.go index 8a4dded8d40..cebba3a2efe 100644 --- a/ray-operator/pkg/client/applyconfiguration/ray/v1/rayjobspec.go +++ b/ray-operator/pkg/client/applyconfiguration/ray/v1/rayjobspec.go @@ -24,12 +24,12 @@ type RayJobSpecApplyConfiguration struct { JobId *string `json:"jobId,omitempty"` SubmissionMode *rayv1.JobSubmissionMode `json:"submissionMode,omitempty"` EntrypointResources *string `json:"entrypointResources,omitempty"` + Schedule *string `json:"schedule,omitempty"` EntrypointNumCpus *float32 `json:"entrypointNumCpus,omitempty"` EntrypointNumGpus *float32 `json:"entrypointNumGpus,omitempty"` TTLSecondsAfterFinished *int32 `json:"ttlSecondsAfterFinished,omitempty"` ShutdownAfterJobFinishes *bool `json:"shutdownAfterJobFinishes,omitempty"` Suspend *bool `json:"suspend,omitempty"` - Schedule *string `json:"schedule,omitempty"` } // RayJobSpecApplyConfiguration constructs a declarative configuration of the RayJobSpec type for use with @@ -162,6 +162,14 @@ func (b *RayJobSpecApplyConfiguration) WithEntrypointResources(value string) *Ra return b } +// WithSchedule sets the Schedule field in the declarative configuration to the given value +// and returns the receiver, so that objects can be built by chaining "With" function invocations. +// If called multiple times, the Schedule field is set to the value of the last call. +func (b *RayJobSpecApplyConfiguration) WithSchedule(value string) *RayJobSpecApplyConfiguration { + b.Schedule = &value + return b +} + // WithEntrypointNumCpus sets the EntrypointNumCpus field in the declarative configuration to the given value // and returns the receiver, so that objects can be built by chaining "With" function invocations. // If called multiple times, the EntrypointNumCpus field is set to the value of the last call. @@ -201,11 +209,3 @@ func (b *RayJobSpecApplyConfiguration) WithSuspend(value bool) *RayJobSpecApplyC b.Suspend = &value return b } - -// WithSchedule sets the Schedule field in the declarative configuration to the given value -// and returns the receiver, so that objects can be built by chaining "With" function invocations. -// If called multiple times, the Schedule field is set to the value of the last call. -func (b *RayJobSpecApplyConfiguration) WithSchedule(value string) *RayJobSpecApplyConfiguration { - b.Schedule = &value - return b -} diff --git a/ray-operator/pkg/client/applyconfiguration/ray/v1/rayjobstatus.go b/ray-operator/pkg/client/applyconfiguration/ray/v1/rayjobstatus.go index 858f0827051..acdc9a688cf 100644 --- a/ray-operator/pkg/client/applyconfiguration/ray/v1/rayjobstatus.go +++ b/ray-operator/pkg/client/applyconfiguration/ray/v1/rayjobstatus.go @@ -20,11 +20,11 @@ type RayJobStatusApplyConfiguration struct { Message *string `json:"message,omitempty"` StartTime *metav1.Time `json:"startTime,omitempty"` EndTime *metav1.Time `json:"endTime,omitempty"` + LastScheduleTime *metav1.Time `json:"lastScheduleTime,omitempty"` Succeeded *int32 `json:"succeeded,omitempty"` Failed *int32 `json:"failed,omitempty"` RayClusterStatus *RayClusterStatusApplyConfiguration `json:"rayClusterStatus,omitempty"` ObservedGeneration *int64 `json:"observedGeneration,omitempty"` - LastScheduleTime *metav1.Time `json:"lastScheduleTime,omitempty"` } // RayJobStatusApplyConfiguration constructs a declarative configuration of the RayJobStatus type for use with @@ -113,6 +113,14 @@ func (b *RayJobStatusApplyConfiguration) WithEndTime(value metav1.Time) *RayJobS return b } +// WithLastScheduleTime sets the LastScheduleTime field in the declarative configuration to the given value +// and returns the receiver, so that objects can be built by chaining "With" function invocations. +// If called multiple times, the LastScheduleTime field is set to the value of the last call. +func (b *RayJobStatusApplyConfiguration) WithLastScheduleTime(value metav1.Time) *RayJobStatusApplyConfiguration { + b.LastScheduleTime = &value + return b +} + // WithSucceeded sets the Succeeded field in the declarative configuration to the given value // and returns the receiver, so that objects can be built by chaining "With" function invocations. // If called multiple times, the Succeeded field is set to the value of the last call. @@ -144,11 +152,3 @@ func (b *RayJobStatusApplyConfiguration) WithObservedGeneration(value int64) *Ra b.ObservedGeneration = &value return b } - -// WithLastScheduleTime sets the LastScheduleTime field in the declarative configuration to the given value -// and returns the receiver, so that objects can be built by chaining "With" function invocations. -// If called multiple times, the LastScheduleTime field is set to the value of the last call. -func (b *RayJobStatusApplyConfiguration) WithLastScheduleTime(value metav1.Time) *RayJobStatusApplyConfiguration { - b.LastScheduleTime = &value - return b -} From 23f5e28d7fc001c93860fb6b15a9f9a53d3fb728 Mon Sep 17 00:00:00 2001 From: DW-Han Date: Mon, 21 Jul 2025 21:35:37 +0000 Subject: [PATCH 18/34] deepcopy function --- ray-operator/apis/ray/v1/zz_generated.deepcopy.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/ray-operator/apis/ray/v1/zz_generated.deepcopy.go b/ray-operator/apis/ray/v1/zz_generated.deepcopy.go index 8c436ea68b4..fc37bc8f695 100644 --- a/ray-operator/apis/ray/v1/zz_generated.deepcopy.go +++ b/ray-operator/apis/ray/v1/zz_generated.deepcopy.go @@ -507,6 +507,10 @@ func (in *RayJobStatus) DeepCopyInto(out *RayJobStatus) { in, out := &in.EndTime, &out.EndTime *out = (*in).DeepCopy() } + if in.LastScheduleTime != nil { + in, out := &in.LastScheduleTime, &out.LastScheduleTime + *out = (*in).DeepCopy() + } if in.Succeeded != nil { in, out := &in.Succeeded, &out.Succeeded *out = new(int32) @@ -518,10 +522,6 @@ func (in *RayJobStatus) DeepCopyInto(out *RayJobStatus) { **out = **in } in.RayClusterStatus.DeepCopyInto(&out.RayClusterStatus) - if in.LastScheduleTime != nil { - in, out := &in.LastScheduleTime, &out.LastScheduleTime - *out = (*in).DeepCopy() - } } // DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new RayJobStatus. From 1a032b90d908c7d6d73d627a844b0a9d59f9b8e4 Mon Sep 17 00:00:00 2001 From: DW-Han Date: Tue, 22 Jul 2025 23:53:02 +0000 Subject: [PATCH 19/34] working integration tests and cleaning --- .../ray/rayjob_controller_scheduled_test.go | 324 +++++++++++------- .../controllers/ray/utils/schedule.go | 2 +- 2 files changed, 202 insertions(+), 124 deletions(-) diff --git a/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go b/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go index f9c2e73ef84..b1c71f0c7b2 100644 --- a/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go +++ b/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go @@ -21,30 +21,28 @@ import ( . "github.com/onsi/ginkgo/v2" . "github.com/onsi/gomega" - // batchv1 "k8s.io/api/batch/v1" - // "k8s.io/apimachinery/pkg/api/errors" + batchv1 "k8s.io/api/batch/v1" + corev1 "k8s.io/api/core/v1" + "k8s.io/apimachinery/pkg/api/errors" apierrors "k8s.io/apimachinery/pkg/api/errors" - "k8s.io/client-go/util/retry" // For creating pointers to int32, string, bool etc. "sigs.k8s.io/controller-runtime/pkg/client" rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1" "github.com/ray-project/kuberay/ray-operator/controllers/ray/common" + "github.com/ray-project/kuberay/ray-operator/controllers/ray/utils" ) -func scheduledRayJobTemplate(name, namespace, schedule string) *rayv1.RayJob { - job := rayJobTemplate(name, namespace) - job.Spec.Schedule = schedule - return job -} - var _ = Context("RayJob with schedule operation", func() { - // This Describe block focuses on the lifecycle of a RayJob configured with a schedule. Describe("When creating a RayJob with a schedule field and NO cluster deletion", Ordered, func() { + // The states should transition from Scheduled -> ... -> Initializing -> Running -> Complete -> Scheduled + // In the last scheduled state the cluster should still exist since ShutdownAfterJobFinishes is False ctx := context.Background() namespace := "default" - cronSchedule := "0 0 0 0 0" - rayJob := scheduledRayJobTemplate("rayjob-scheduled-no-deletion", namespace, cronSchedule) - // rayCluster := &rayv1.RayCluster{} + cronSchedule := "*/1 * * * *" + rayJob := rayJobTemplate("rayjob-scheduled-no-deletion", namespace) + rayJob.Spec.Schedule = cronSchedule + rayJob.Spec.ShutdownAfterJobFinishes = false + rayCluster := &rayv1.RayCluster{} It("should create a RayJob object with the schedule", func() { err := k8sClient.Create(ctx, rayJob) @@ -58,110 +56,113 @@ var _ = Context("RayJob with schedule operation", func() { "JobDeploymentStatus should be Scheduled") }) - It("should update the CronJob's schedule when the RayJob's schedule is modified", func() { - newCronSchedule := "*/1 * * * *" - updatedRayJob := &rayv1.RayJob{} - err := retry.RetryOnConflict(retry.DefaultRetry, func() error { - err := k8sClient.Get(ctx, client.ObjectKey{Name: rayJob.Name, Namespace: namespace}, updatedRayJob) - if err != nil { - return err - } - updatedRayJob.Spec.Schedule = newCronSchedule - return k8sClient.Update(ctx, updatedRayJob) - }) - Expect(err).NotTo(HaveOccurred(), "failed to update RayJob schedule") + // The cron job runs every minute so it will take at most 1 minute to run + It("should transition to the Initializing state", func() { + Eventually( + getRayJobDeploymentStatus(ctx, rayJob), + time.Second*60).Should(Equal(rayv1.JobDeploymentStatusInitializing), + "JobDeploymentStatus should be Initializing") }) - // The cron job runs every minute so it will take at most 1 minute to run - It("should transition to the New state", func() { + It("should create a raycluster object", func() { + Eventually( + getRayClusterNameForRayJob(ctx, rayJob), + time.Second*15, time.Millisecond*500).Should(Not(BeEmpty())) + Eventually( + + getResourceFunc(ctx, common.RayJobRayClusterNamespacedName(rayJob), rayCluster), + time.Second*3, time.Millisecond*500).Should(Succeed()) + }) + + It("should NOT create the underlying K8s job yet because the cluster is not ready", func() { + underlyingK8sJob := &batchv1.Job{} + Consistently( + // k8sClient client throws error if resource not found + func() bool { + err := getResourceFunc(ctx, common.RayJobK8sJobNamespacedName(rayJob), underlyingK8sJob)() + return errors.IsNotFound(err) + }, + time.Second*3, time.Millisecond*500).Should(BeTrue()) + }) + + It("should be able to update all Pods to Running", func() { + updateHeadPodToRunningAndReady(ctx, rayJob.Status.RayClusterName, namespace) + updateWorkerPodsToRunningAndReady(ctx, rayJob.Status.RayClusterName, namespace) + }) + + It("Dashboard URL should be set", func() { + Eventually( + getDashboardURLForRayJob(ctx, rayJob), + time.Second*3, time.Millisecond*500).Should(HavePrefix(rayJob.Name), "Dashboard URL = %v", rayJob.Status.DashboardURL) + }) + + It("should create the underlying Kubernetes Job object", func() { + underlyingK8sJob := &batchv1.Job{} + // The underlying Kubernetes Job should be created when the RayJob is created + Eventually( + getResourceFunc(ctx, common.RayJobK8sJobNamespacedName(rayJob), underlyingK8sJob), + time.Second*3, time.Millisecond*500).Should(Succeed(), "Expected Kubernetes job to be present") + }) + + It("should transition to the Running", func() { Eventually( getRayJobDeploymentStatus(ctx, rayJob), - time.Second*60).Should(Equal(rayv1.JobDeploymentStatusNew), - "JobDeploymentStatus should be New") - }) - // It("should create a raycluster object", func() { - // // Ray Cluster name can be present on RayJob's CRD - // Eventually( - // getRayClusterNameForRayJob(ctx, rayJob), - // time.Second*15, time.Millisecond*500).Should(Not(BeEmpty())) - // // The actual cluster instance and underlying resources SHOULD be created when suspend == false - // Eventually( - - // getResourceFunc(ctx, common.RayJobRayClusterNamespacedName(rayJob), rayCluster), - // time.Second*3, time.Millisecond*500).Should(Succeed()) - // }) - - // It("should NOT create the underlying K8s job yet because the cluster is not ready", func() { - // underlyingK8sJob := &batchv1.Job{} - // Consistently( - // // k8sClient client throws error if resource not found - // func() bool { - // err := getResourceFunc(ctx, common.RayJobK8sJobNamespacedName(rayJob), underlyingK8sJob)() - // return errors.IsNotFound(err) - // }, - // time.Second*3, time.Millisecond*500).Should(BeTrue()) - // }) - - // It("should be able to update all Pods to Running", func() { - // updateHeadPodToRunningAndReady(ctx, rayJob.Status.RayClusterName, namespace) - // updateWorkerPodsToRunningAndReady(ctx, rayJob.Status.RayClusterName, namespace) - // }) - - // It("Dashboard URL should be set", func() { - // Eventually( - // getDashboardURLForRayJob(ctx, rayJob), - // time.Second*3, time.Millisecond*500).Should(HavePrefix(rayJob.Name), "Dashboard URL = %v", rayJob.Status.DashboardURL) - // }) - - // It("should create the underlying Kubernetes Job object", func() { - // underlyingK8sJob := &batchv1.Job{} - // // The underlying Kubernetes Job should be created when the RayJob is created - // Eventually( - // getResourceFunc(ctx, common.RayJobK8sJobNamespacedName(rayJob), underlyingK8sJob), - // time.Second*3, time.Millisecond*500).Should(Succeed(), "Expected Kubernetes job to be present") - // }) - - // It("should transition to the Initailizing", func() { - // Eventually( - // getRayJobDeploymentStatus(ctx, rayJob), - // time.Second*5, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusInitializing), - // "JobDeploymentStatus should be Initializing") - // }) - - // It("should transition to the Running", func() { - // Eventually( - // getRayJobDeploymentStatus(ctx, rayJob), - // time.Second*5, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusRunning), - // "JobDeploymentStatus should be Initializing") - // }) - - // It("should transition to the Complete", func() { - // Eventually( - // getRayJobDeploymentStatus(ctx, rayJob), - // time.Second*5, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusComplete), - // "JobDeploymentStatus should be Initializing") - // }) - - // It("should transition to the Scheduled", func() { - // Eventually( - // getRayJobDeploymentStatus(ctx, rayJob), - // time.Second*5, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusScheduled), - // "JobDeploymentStatus should be Initializing") - // }) - - // It("The raycluster object should still exist", func() { - // Eventually( - - // getResourceFunc(ctx, common.RayJobRayClusterNamespacedName(rayJob), rayCluster), - // time.Second*3, time.Millisecond*500).Should(Succeed()) - // }) + time.Second*5, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusRunning), + "JobDeploymentStatus should be Running") + }) + + It("RayJobs's JobDeploymentStatus transitions to Scheduled after Job is Complete.", func() { + // Update fake dashboard client to return job info with "Succeeded" status. + getJobInfo := func(context.Context, string) (*utils.RayJobInfo, error) { //nolint:unparam // This is a mock function so parameters are required + return &utils.RayJobInfo{JobStatus: rayv1.JobStatusSucceeded, EndTime: uint64(time.Now().UnixMilli())}, nil + } + fakeRayDashboardClient.GetJobInfoMock.Store(&getJobInfo) + defer fakeRayDashboardClient.GetJobInfoMock.Store(nil) + + // RayJob transitions to Complete if and only if the corresponding submitter Kubernetes Job is Complete or Failed. + Consistently( + getRayJobDeploymentStatus(ctx, rayJob), + time.Second*3, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusRunning), "JobDeploymentStatus = %v", rayJob.Status.JobDeploymentStatus) + + // Update the submitter Kubernetes Job to Complete. + namespacedName := common.RayJobK8sJobNamespacedName(rayJob) + job := &batchv1.Job{} + err := k8sClient.Get(ctx, namespacedName, job) + Expect(err).NotTo(HaveOccurred(), "failed to get Kubernetes Job") + + // Update the submitter Kubernetes Job to Complete. + conditions := []batchv1.JobCondition{ + {Type: batchv1.JobComplete, Status: corev1.ConditionTrue}, + } + job.Status.Conditions = conditions + Expect(k8sClient.Status().Update(ctx, job)).Should(Succeed()) + + // RayJob transitions to Complete. + Eventually( + getRayJobDeploymentStatus(ctx, rayJob), + time.Second*5, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusScheduled), "jobDeploymentStatus = %v", rayJob.Status.JobDeploymentStatus) + }) + + It("The raycluster object should still exist", func() { + Eventually( + func() bool { + err := getResourceFunc(ctx, client.ObjectKey{Name: rayJob.Status.RayClusterName, Namespace: namespace}, rayCluster)() + return err == nil + }, + time.Second*15, time.Millisecond*500).Should(BeTrue(), "Expected RayCluster to still exist") + }) }) Describe("When creating a RayJob with a schedule field and WITH cluster deletion", Ordered, func() { + // The states should transition from Scheduled -> ... -> Initializing -> Running -> Complete -> Scheduled + // In the last scheduled state the cluster should not exist since ShutdownAfterJobFinishes is true ctx := context.Background() namespace := "default" cronSchedule := "*/1 * * * *" - rayJob := scheduledRayJobTemplate("rayjob-scheduled-with-deletion", namespace, cronSchedule) + rayJob := rayJobTemplate("rayjob-scheduled-with-deletion", namespace) + rayJob.Spec.Schedule = cronSchedule + rayJob.Spec.ShutdownAfterJobFinishes = true + rayCluster := &rayv1.RayCluster{} It("should create a RayJob object with the schedule", func() { err := k8sClient.Create(ctx, rayJob) @@ -176,7 +177,6 @@ var _ = Context("RayJob with schedule operation", func() { }) It("should NOT create a RayCluster object immediately", func() { - rayCluster := &rayv1.RayCluster{} Consistently( func() bool { err := k8sClient.Get(ctx, common.RayJobRayClusterNamespacedName(rayJob), rayCluster) @@ -185,25 +185,103 @@ var _ = Context("RayJob with schedule operation", func() { time.Second*3, time.Millisecond*500).Should(BeTrue(), "RayCluster should NOT be created upon scheduled RayJob creation") }) - It("should update the CronJob's schedule when the RayJob's schedule is modified", func() { - newCronSchedule := "0 1 * * *" - updatedRayJob := &rayv1.RayJob{} - err := retry.RetryOnConflict(retry.DefaultRetry, func() error { - err := k8sClient.Get(ctx, client.ObjectKey{Name: rayJob.Name, Namespace: namespace}, updatedRayJob) - if err != nil { - return err - } - updatedRayJob.Spec.Schedule = newCronSchedule - return k8sClient.Update(ctx, updatedRayJob) - }) - Expect(err).NotTo(HaveOccurred(), "failed to update RayJob schedule") + It("should transition to the Initializing state", func() { + Eventually( + getRayJobDeploymentStatus(ctx, rayJob), + time.Second*60).Should(Equal(rayv1.JobDeploymentStatusInitializing), + "JobDeploymentStatus should be Initializing") + }) + + It("should create a raycluster object", func() { + Eventually( + getRayClusterNameForRayJob(ctx, rayJob), + time.Second*15, time.Millisecond*500).Should(Not(BeEmpty())) + Eventually( + + getResourceFunc(ctx, common.RayJobRayClusterNamespacedName(rayJob), rayCluster), + time.Second*3, time.Millisecond*500).Should(Succeed()) + }) + + It("should NOT create the underlying K8s job yet because the cluster is not ready", func() { + underlyingK8sJob := &batchv1.Job{} + Consistently( + // k8sClient client throws error if resource not found + func() bool { + err := getResourceFunc(ctx, common.RayJobK8sJobNamespacedName(rayJob), underlyingK8sJob)() + return errors.IsNotFound(err) + }, + time.Second*3, time.Millisecond*500).Should(BeTrue()) + }) + + It("should be able to update all Pods to Running", func() { + updateHeadPodToRunningAndReady(ctx, rayJob.Status.RayClusterName, namespace) + updateWorkerPodsToRunningAndReady(ctx, rayJob.Status.RayClusterName, namespace) + }) + + It("Dashboard URL should be set", func() { + Eventually( + getDashboardURLForRayJob(ctx, rayJob), + time.Second*3, time.Millisecond*500).Should(HavePrefix(rayJob.Name), "Dashboard URL = %v", rayJob.Status.DashboardURL) + }) + + It("should create the underlying Kubernetes Job object", func() { + underlyingK8sJob := &batchv1.Job{} + // The underlying Kubernetes Job should be created when the RayJob is created + Eventually( + getResourceFunc(ctx, common.RayJobK8sJobNamespacedName(rayJob), underlyingK8sJob), + time.Second*3, time.Millisecond*500).Should(Succeed(), "Expected Kubernetes job to be present") }) - It("should transition to the New state", func() { + It("should transition to the Running", func() { Eventually( getRayJobDeploymentStatus(ctx, rayJob), - time.Second*60).Should(Equal(rayv1.JobDeploymentStatusNew), - "JobDeploymentStatus should be New") + time.Second*5, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusRunning), + "JobDeploymentStatus should be Running") + }) + + It("RayJobs's JobDeploymentStatus transitions to Scheduled after Job is Complete.", func() { + // Update fake dashboard client to return job info with "Succeeded" status. + getJobInfo := func(context.Context, string) (*utils.RayJobInfo, error) { //nolint:unparam // This is a mock function so parameters are required + return &utils.RayJobInfo{JobStatus: rayv1.JobStatusSucceeded, EndTime: uint64(time.Now().UnixMilli())}, nil + } + fakeRayDashboardClient.GetJobInfoMock.Store(&getJobInfo) + defer fakeRayDashboardClient.GetJobInfoMock.Store(nil) + + // RayJob transitions to Complete if and only if the corresponding submitter Kubernetes Job is Complete or Failed. + Consistently( + getRayJobDeploymentStatus(ctx, rayJob), + time.Second*3, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusRunning), "JobDeploymentStatus = %v", rayJob.Status.JobDeploymentStatus) + + // Update the submitter Kubernetes Job to Complete. + namespacedName := common.RayJobK8sJobNamespacedName(rayJob) + job := &batchv1.Job{} + err := k8sClient.Get(ctx, namespacedName, job) + Expect(err).NotTo(HaveOccurred(), "failed to get Kubernetes Job") + + // Update the submitter Kubernetes Job to Complete. + conditions := []batchv1.JobCondition{ + {Type: batchv1.JobComplete, Status: corev1.ConditionTrue}, + } + job.Status.Conditions = conditions + Expect(k8sClient.Status().Update(ctx, job)).Should(Succeed()) + + // RayJob transitions to Complete. + Eventually( + getRayJobDeploymentStatus(ctx, rayJob), + time.Second*5, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusScheduled), "jobDeploymentStatus = %v", rayJob.Status.JobDeploymentStatus) + }) + + It("If shutdownAfterJobFinishes is true, RayCluster should be deleted along with the submitter Job.", func() { + Eventually( + func() bool { + return apierrors.IsNotFound(getResourceFunc(ctx, client.ObjectKey{Name: rayJob.Status.RayClusterName, Namespace: namespace}, rayCluster)()) + }, + time.Second*3, time.Millisecond*500).Should(BeTrue()) + namespacedName := common.RayJobK8sJobNamespacedName(rayJob) + job := &batchv1.Job{} + Consistently( + getResourceFunc(ctx, namespacedName, job), + time.Second*3, time.Millisecond*500).ShouldNot(Succeed()) }) }) }) diff --git a/ray-operator/controllers/ray/utils/schedule.go b/ray-operator/controllers/ray/utils/schedule.go index 9b4df49596a..52401ccf1f2 100644 --- a/ray-operator/controllers/ray/utils/schedule.go +++ b/ray-operator/controllers/ray/utils/schedule.go @@ -134,7 +134,7 @@ func NextScheduleTimeDuration(logger logr.Logger, rj *rayv1.RayJob, now time.Tim mostRecentTime = &now } } - logger.Info("Successfully calculated earliestTime and mostRecentTime", "mostRecentTime", mostRecentTime, "earliestTime", earliestTime, "Next time to aim for", schedule.Next(*mostRecentTime)) + logger.Info("Successfully calculated earliestTime and mostRecentTime", "mostRecentTime", mostRecentTime, "Current Time", now, "Next time to aim for", schedule.Next(*mostRecentTime)) t := schedule.Next(*mostRecentTime).Sub(now) return t } From 95cd767c18ddb5698be1a764cc0d4bd233fe494e Mon Sep 17 00:00:00 2001 From: DW-Han Date: Wed, 23 Jul 2025 04:39:03 +0000 Subject: [PATCH 20/34] making tests more air tight --- .../controllers/ray/rayjob_controller.go | 5 +- .../ray/rayjob_controller_scheduled_test.go | 82 +++++++++++-------- 2 files changed, 53 insertions(+), 34 deletions(-) diff --git a/ray-operator/controllers/ray/rayjob_controller.go b/ray-operator/controllers/ray/rayjob_controller.go index 3c49eeb1cc4..bf1bd083b91 100644 --- a/ray-operator/controllers/ray/rayjob_controller.go +++ b/ray-operator/controllers/ray/rayjob_controller.go @@ -957,14 +957,15 @@ func (r *RayJobReconciler) constructRayClusterForRayJob(rayJobInstance *rayv1.Ra func (r *RayJobReconciler) getPreviousAndNextScheduleDistance(ctx context.Context, currentTime time.Time, rayJobInstance *rayv1.RayJob) (time.Duration, time.Duration, error) { logger := ctrl.LoggerFrom(ctx) logger.Info("Calculating next schedule for the RayJob") - cronSchedule, err := cron.ParseStandard(utils.FormatSchedule(rayJobInstance, r.Recorder)) + formatedCron := utils.FormatSchedule(rayJobInstance, r.Recorder) + cronSchedule, err := cron.ParseStandard(formatedCron) if err != nil { // this is likely a user error in defining the spec value // we should log the error and not reconcile this cronjob until an update to spec r.Recorder.Eventf(rayJobInstance, corev1.EventTypeWarning, "UnparseableSchedule", "unparseable schedule: %q : %s", rayJobInstance.Spec.Schedule, err) return 0, 0, fmt.Errorf("the cron schedule provided is unparseable: %w", err) } - logger.Info("Successfully parsed cron schedule", "CronSchedule", cronSchedule) + logger.Info("Successfully parsed cron schedule", "CronSchedule", formatedCron) t1 := utils.NextScheduleTimeDuration(logger, rayJobInstance, currentTime, cronSchedule) t2 := utils.LastScheduleTimeDuration(logger, rayJobInstance, currentTime, cronSchedule) diff --git a/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go b/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go index b1c71f0c7b2..8caa4918a3d 100644 --- a/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go +++ b/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go @@ -44,20 +44,37 @@ var _ = Context("RayJob with schedule operation", func() { rayJob.Spec.ShutdownAfterJobFinishes = false rayCluster := &rayv1.RayCluster{} + It("Verify RayJob spec", func() { + Expect(rayJob.Spec.ShutdownAfterJobFinishes).To(BeFalse()) + }) + It("should create a RayJob object with the schedule", func() { err := k8sClient.Create(ctx, rayJob) Expect(err).NotTo(HaveOccurred(), "failed to create test scheduled RayJob resource") }) - It("should have a JobDeploymentStatus reflecting its scheduled state", func() { + // It("should NOT create a raycluster object immediately", func() { + // Consistently( + // getRayClusterNameForRayJob(ctx, rayJob), + // time.Second*3, time.Millisecond*500).Should(BeEmpty()) + // }) + + // We dont control the time till next schedule so it could schedule then immediately run the job which which can cause errors without the Or + It("should have a JobDeploymentStatus reflecting its scheduled, new, or ", func() { Eventually( getRayJobDeploymentStatus(ctx, rayJob), - time.Second*5, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusScheduled), - "JobDeploymentStatus should be Scheduled") + time.Second*5).Should( + Or( + Equal(rayv1.JobDeploymentStatusScheduled), + Equal(rayv1.JobDeploymentStatusNew), + Equal(rayv1.JobDeploymentStatusInitializing), + ), + "JobDeploymentStatus should be Scheduled, New or Initializing", + ) }) // The cron job runs every minute so it will take at most 1 minute to run - It("should transition to the Initializing state", func() { + It("should transition to the Initializing", func() { Eventually( getRayJobDeploymentStatus(ctx, rayJob), time.Second*60).Should(Equal(rayv1.JobDeploymentStatusInitializing), @@ -137,7 +154,7 @@ var _ = Context("RayJob with schedule operation", func() { job.Status.Conditions = conditions Expect(k8sClient.Status().Update(ctx, job)).Should(Succeed()) - // RayJob transitions to Complete. + // RayJob transitions to Scheduled. Eventually( getRayJobDeploymentStatus(ctx, rayJob), time.Second*5, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusScheduled), "jobDeploymentStatus = %v", rayJob.Status.JobDeploymentStatus) @@ -164,28 +181,34 @@ var _ = Context("RayJob with schedule operation", func() { rayJob.Spec.ShutdownAfterJobFinishes = true rayCluster := &rayv1.RayCluster{} - It("should create a RayJob object with the schedule", func() { - err := k8sClient.Create(ctx, rayJob) - Expect(err).NotTo(HaveOccurred(), "failed to create test scheduled RayJob resource") + It("Verify RayJob spec", func() { + Expect(rayJob.Spec.ShutdownAfterJobFinishes).To(BeTrue()) }) - It("should have a JobDeploymentStatus reflecting its scheduled state", func() { + It("Create a RayJob custom resource", func() { + err := k8sClient.Create(ctx, rayJob) + Expect(err).NotTo(HaveOccurred(), "Failed to create RayJob") Eventually( - getRayJobDeploymentStatus(ctx, rayJob), - time.Second*5, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusScheduled), - "JobDeploymentStatus should be Scheduled") + getResourceFunc(ctx, client.ObjectKey{Name: rayJob.Name, Namespace: namespace}, rayJob), + time.Second*3, time.Millisecond*500).Should(Succeed(), "Should be able to see RayJob: %v", rayJob.Name) }) - It("should NOT create a RayCluster object immediately", func() { - Consistently( - func() bool { - err := k8sClient.Get(ctx, common.RayJobRayClusterNamespacedName(rayJob), rayCluster) - return apierrors.IsNotFound(err) - }, - time.Second*3, time.Millisecond*500).Should(BeTrue(), "RayCluster should NOT be created upon scheduled RayJob creation") + // We dont control the time till next schedule so it could schedule then immediately run the job which wi + It("should have a JobDeploymentStatus reflecting its scheduled, new, or ", func() { + Eventually( + getRayJobDeploymentStatus(ctx, rayJob), + time.Second*5).Should( + Or( + Equal(rayv1.JobDeploymentStatusScheduled), + Equal(rayv1.JobDeploymentStatusNew), + Equal(rayv1.JobDeploymentStatusInitializing), + ), + "JobDeploymentStatus should be Scheduled, New or Initializing", + ) }) - It("should transition to the Initializing state", func() { + // The cron job runs every minute so it will take at most 1 minute to run + It("should transition to the Initializing", func() { Eventually( getRayJobDeploymentStatus(ctx, rayJob), time.Second*60).Should(Equal(rayv1.JobDeploymentStatusInitializing), @@ -264,24 +287,19 @@ var _ = Context("RayJob with schedule operation", func() { } job.Status.Conditions = conditions Expect(k8sClient.Status().Update(ctx, job)).Should(Succeed()) - - // RayJob transitions to Complete. - Eventually( - getRayJobDeploymentStatus(ctx, rayJob), - time.Second*5, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusScheduled), "jobDeploymentStatus = %v", rayJob.Status.JobDeploymentStatus) - }) - - It("If shutdownAfterJobFinishes is true, RayCluster should be deleted along with the submitter Job.", func() { Eventually( func() bool { return apierrors.IsNotFound(getResourceFunc(ctx, client.ObjectKey{Name: rayJob.Status.RayClusterName, Namespace: namespace}, rayCluster)()) }, - time.Second*3, time.Millisecond*500).Should(BeTrue()) - namespacedName := common.RayJobK8sJobNamespacedName(rayJob) - job := &batchv1.Job{} + time.Second*30, time.Millisecond*500).Should(BeTrue()) + Consistently( getResourceFunc(ctx, namespacedName, job), - time.Second*3, time.Millisecond*500).ShouldNot(Succeed()) + time.Second*30, time.Millisecond*500).ShouldNot(Succeed()) + // RayJob transitions to Scheduled. + Eventually( + getRayJobDeploymentStatus(ctx, rayJob), + time.Second*5, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusScheduled), "jobDeploymentStatus = %v", rayJob.Status.JobDeploymentStatus) }) }) }) From 5f176a3f0bcfe6d2fb9d4ece5c4d0da42fabefb8 Mon Sep 17 00:00:00 2001 From: DW-Han Date: Wed, 23 Jul 2025 05:02:36 +0000 Subject: [PATCH 21/34] making tests more air tight --- .../ray/rayjob_controller_scheduled_test.go | 140 ------------------ 1 file changed, 140 deletions(-) diff --git a/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go b/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go index 8caa4918a3d..2293d55bc0f 100644 --- a/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go +++ b/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go @@ -24,7 +24,6 @@ import ( batchv1 "k8s.io/api/batch/v1" corev1 "k8s.io/api/core/v1" "k8s.io/apimachinery/pkg/api/errors" - apierrors "k8s.io/apimachinery/pkg/api/errors" "sigs.k8s.io/controller-runtime/pkg/client" rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1" @@ -53,12 +52,6 @@ var _ = Context("RayJob with schedule operation", func() { Expect(err).NotTo(HaveOccurred(), "failed to create test scheduled RayJob resource") }) - // It("should NOT create a raycluster object immediately", func() { - // Consistently( - // getRayClusterNameForRayJob(ctx, rayJob), - // time.Second*3, time.Millisecond*500).Should(BeEmpty()) - // }) - // We dont control the time till next schedule so it could schedule then immediately run the job which which can cause errors without the Or It("should have a JobDeploymentStatus reflecting its scheduled, new, or ", func() { Eventually( @@ -169,137 +162,4 @@ var _ = Context("RayJob with schedule operation", func() { time.Second*15, time.Millisecond*500).Should(BeTrue(), "Expected RayCluster to still exist") }) }) - - Describe("When creating a RayJob with a schedule field and WITH cluster deletion", Ordered, func() { - // The states should transition from Scheduled -> ... -> Initializing -> Running -> Complete -> Scheduled - // In the last scheduled state the cluster should not exist since ShutdownAfterJobFinishes is true - ctx := context.Background() - namespace := "default" - cronSchedule := "*/1 * * * *" - rayJob := rayJobTemplate("rayjob-scheduled-with-deletion", namespace) - rayJob.Spec.Schedule = cronSchedule - rayJob.Spec.ShutdownAfterJobFinishes = true - rayCluster := &rayv1.RayCluster{} - - It("Verify RayJob spec", func() { - Expect(rayJob.Spec.ShutdownAfterJobFinishes).To(BeTrue()) - }) - - It("Create a RayJob custom resource", func() { - err := k8sClient.Create(ctx, rayJob) - Expect(err).NotTo(HaveOccurred(), "Failed to create RayJob") - Eventually( - getResourceFunc(ctx, client.ObjectKey{Name: rayJob.Name, Namespace: namespace}, rayJob), - time.Second*3, time.Millisecond*500).Should(Succeed(), "Should be able to see RayJob: %v", rayJob.Name) - }) - - // We dont control the time till next schedule so it could schedule then immediately run the job which wi - It("should have a JobDeploymentStatus reflecting its scheduled, new, or ", func() { - Eventually( - getRayJobDeploymentStatus(ctx, rayJob), - time.Second*5).Should( - Or( - Equal(rayv1.JobDeploymentStatusScheduled), - Equal(rayv1.JobDeploymentStatusNew), - Equal(rayv1.JobDeploymentStatusInitializing), - ), - "JobDeploymentStatus should be Scheduled, New or Initializing", - ) - }) - - // The cron job runs every minute so it will take at most 1 minute to run - It("should transition to the Initializing", func() { - Eventually( - getRayJobDeploymentStatus(ctx, rayJob), - time.Second*60).Should(Equal(rayv1.JobDeploymentStatusInitializing), - "JobDeploymentStatus should be Initializing") - }) - - It("should create a raycluster object", func() { - Eventually( - getRayClusterNameForRayJob(ctx, rayJob), - time.Second*15, time.Millisecond*500).Should(Not(BeEmpty())) - Eventually( - - getResourceFunc(ctx, common.RayJobRayClusterNamespacedName(rayJob), rayCluster), - time.Second*3, time.Millisecond*500).Should(Succeed()) - }) - - It("should NOT create the underlying K8s job yet because the cluster is not ready", func() { - underlyingK8sJob := &batchv1.Job{} - Consistently( - // k8sClient client throws error if resource not found - func() bool { - err := getResourceFunc(ctx, common.RayJobK8sJobNamespacedName(rayJob), underlyingK8sJob)() - return errors.IsNotFound(err) - }, - time.Second*3, time.Millisecond*500).Should(BeTrue()) - }) - - It("should be able to update all Pods to Running", func() { - updateHeadPodToRunningAndReady(ctx, rayJob.Status.RayClusterName, namespace) - updateWorkerPodsToRunningAndReady(ctx, rayJob.Status.RayClusterName, namespace) - }) - - It("Dashboard URL should be set", func() { - Eventually( - getDashboardURLForRayJob(ctx, rayJob), - time.Second*3, time.Millisecond*500).Should(HavePrefix(rayJob.Name), "Dashboard URL = %v", rayJob.Status.DashboardURL) - }) - - It("should create the underlying Kubernetes Job object", func() { - underlyingK8sJob := &batchv1.Job{} - // The underlying Kubernetes Job should be created when the RayJob is created - Eventually( - getResourceFunc(ctx, common.RayJobK8sJobNamespacedName(rayJob), underlyingK8sJob), - time.Second*3, time.Millisecond*500).Should(Succeed(), "Expected Kubernetes job to be present") - }) - - It("should transition to the Running", func() { - Eventually( - getRayJobDeploymentStatus(ctx, rayJob), - time.Second*5, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusRunning), - "JobDeploymentStatus should be Running") - }) - - It("RayJobs's JobDeploymentStatus transitions to Scheduled after Job is Complete.", func() { - // Update fake dashboard client to return job info with "Succeeded" status. - getJobInfo := func(context.Context, string) (*utils.RayJobInfo, error) { //nolint:unparam // This is a mock function so parameters are required - return &utils.RayJobInfo{JobStatus: rayv1.JobStatusSucceeded, EndTime: uint64(time.Now().UnixMilli())}, nil - } - fakeRayDashboardClient.GetJobInfoMock.Store(&getJobInfo) - defer fakeRayDashboardClient.GetJobInfoMock.Store(nil) - - // RayJob transitions to Complete if and only if the corresponding submitter Kubernetes Job is Complete or Failed. - Consistently( - getRayJobDeploymentStatus(ctx, rayJob), - time.Second*3, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusRunning), "JobDeploymentStatus = %v", rayJob.Status.JobDeploymentStatus) - - // Update the submitter Kubernetes Job to Complete. - namespacedName := common.RayJobK8sJobNamespacedName(rayJob) - job := &batchv1.Job{} - err := k8sClient.Get(ctx, namespacedName, job) - Expect(err).NotTo(HaveOccurred(), "failed to get Kubernetes Job") - - // Update the submitter Kubernetes Job to Complete. - conditions := []batchv1.JobCondition{ - {Type: batchv1.JobComplete, Status: corev1.ConditionTrue}, - } - job.Status.Conditions = conditions - Expect(k8sClient.Status().Update(ctx, job)).Should(Succeed()) - Eventually( - func() bool { - return apierrors.IsNotFound(getResourceFunc(ctx, client.ObjectKey{Name: rayJob.Status.RayClusterName, Namespace: namespace}, rayCluster)()) - }, - time.Second*30, time.Millisecond*500).Should(BeTrue()) - - Consistently( - getResourceFunc(ctx, namespacedName, job), - time.Second*30, time.Millisecond*500).ShouldNot(Succeed()) - // RayJob transitions to Scheduled. - Eventually( - getRayJobDeploymentStatus(ctx, rayJob), - time.Second*5, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusScheduled), "jobDeploymentStatus = %v", rayJob.Status.JobDeploymentStatus) - }) - }) }) From c86ea08d25dd7f4c5797699200a770af8ea872d3 Mon Sep 17 00:00:00 2001 From: DW-Han Date: Thu, 24 Jul 2025 05:37:01 +0000 Subject: [PATCH 22/34] cleaning tests --- .../controllers/ray/rayjob_controller.go | 3 +- .../ray/rayjob_controller_scheduled_test.go | 87 +++++++++++++++++-- 2 files changed, 80 insertions(+), 10 deletions(-) diff --git a/ray-operator/controllers/ray/rayjob_controller.go b/ray-operator/controllers/ray/rayjob_controller.go index bf1bd083b91..ac555f88cb5 100644 --- a/ray-operator/controllers/ray/rayjob_controller.go +++ b/ray-operator/controllers/ray/rayjob_controller.go @@ -497,7 +497,8 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) } // Checking if we are currently within a buffer to the previous cron schedule time if t2 <= ScheduleBuffer { - logger.Info("The current time is within the buffer window of a cron tick", "NextScheduleTimeDuration", t1, "LastScheduleTimeDuration", t2) + logger.Info("The current time is within the buffer window of a cron tick", "NextScheduleTimeDuration", t1, "LastScheduleTimeDuration", t2, "Previous LastScheduleTime", rayJobInstance.Status.LastScheduleTime) + rayJobInstance.Status.LastScheduleTime = &metav1.Time{Time: time.Now()} rayJobInstance.Status.JobStatus = rayv1.JobStatusNew rayJobInstance.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusNew } else { diff --git a/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go b/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go index 2293d55bc0f..972de175de4 100644 --- a/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go +++ b/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go @@ -24,6 +24,7 @@ import ( batchv1 "k8s.io/api/batch/v1" corev1 "k8s.io/api/core/v1" "k8s.io/apimachinery/pkg/api/errors" + "k8s.io/apimachinery/pkg/types" "sigs.k8s.io/controller-runtime/pkg/client" rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1" @@ -45,6 +46,7 @@ var _ = Context("RayJob with schedule operation", func() { It("Verify RayJob spec", func() { Expect(rayJob.Spec.ShutdownAfterJobFinishes).To(BeFalse()) + Expect(rayJob.Spec.Schedule).To(Not(BeEmpty())) }) It("should create a RayJob object with the schedule", func() { @@ -52,17 +54,26 @@ var _ = Context("RayJob with schedule operation", func() { Expect(err).NotTo(HaveOccurred(), "failed to create test scheduled RayJob resource") }) - // We dont control the time till next schedule so it could schedule then immediately run the job which which can cause errors without the Or - It("should have a JobDeploymentStatus reflecting its scheduled, new, or ", func() { - Eventually( - getRayJobDeploymentStatus(ctx, rayJob), - time.Second*5).Should( + // Since the test can be run at any time, the RayJob might immediately + // transition past "Scheduled" to "New" or "Initializing" as it triggers a run. + // We first check if we have already passed the "Scheduled" state this + // accommodates the race condition if we only checked "Scheduled" and avoiding test flakiness. + It("should have a JobDeploymentStatus reflecting its scheduled, new, or initializing state", func() { + getStatusFunc := getRayJobDeploymentStatus(ctx, rayJob) + + currentStatus, err := getStatusFunc() + Expect(err).NotTo(HaveOccurred(), "Failed to get RayJob status initially") + + if currentStatus == rayv1.JobDeploymentStatusInitializing || currentStatus == rayv1.JobDeploymentStatusNew { + Expect(currentStatus).To(Or(Equal(rayv1.JobDeploymentStatusInitializing), Equal(rayv1.JobDeploymentStatusNew)), "RayJob was already Initializing or New") + return + } + // If it's not Initializing, then it should be sheduled + Eventually(getStatusFunc, time.Second*5, time.Millisecond*500).Should( Or( Equal(rayv1.JobDeploymentStatusScheduled), - Equal(rayv1.JobDeploymentStatusNew), - Equal(rayv1.JobDeploymentStatusInitializing), ), - "JobDeploymentStatus should be Scheduled, New or Initializing", + "JobDeploymentStatus should be Scheduled, New or Initializing within 5 seconds", ) }) @@ -70,7 +81,7 @@ var _ = Context("RayJob with schedule operation", func() { It("should transition to the Initializing", func() { Eventually( getRayJobDeploymentStatus(ctx, rayJob), - time.Second*60).Should(Equal(rayv1.JobDeploymentStatusInitializing), + time.Second*60, time.Microsecond*500).Should(Equal(rayv1.JobDeploymentStatusInitializing), "JobDeploymentStatus should be Initializing") }) @@ -161,5 +172,63 @@ var _ = Context("RayJob with schedule operation", func() { }, time.Second*15, time.Millisecond*500).Should(BeTrue(), "Expected RayCluster to still exist") }) + + It("should have a JobDeploymentStatus reflecting its scheduled, new, or initializing state", func() { + getStatusFunc := getRayJobDeploymentStatus(ctx, rayJob) + + currentStatus, err := getStatusFunc() + Expect(err).NotTo(HaveOccurred(), "Failed to get RayJob status initially") + + if currentStatus == rayv1.JobDeploymentStatusInitializing || currentStatus == rayv1.JobDeploymentStatusNew { + Expect(currentStatus).To(Equal(rayv1.JobDeploymentStatusInitializing), "RayJob was already Initializing") + return + } + // If it's not Initializing, then it should be sheduled + Eventually( + getStatusFunc, + time.Second*5, + time.Millisecond*500, + ).Should( + Or( + Equal(rayv1.JobDeploymentStatusScheduled), + ), + "JobDeploymentStatus should be Scheduled, New or Initializing within 5 seconds", + ) + }) + + // We are checking if the LastScheduleTime is correctly set + It("should have LastScheduleTime updated in its status", func() { + rayJobLookupKey := types.NamespacedName{Name: rayJob.Name, Namespace: rayJob.Namespace} + fetchedRayJob := &rayv1.RayJob{} + + var lastScheduleTime *time.Time + Eventually(func() bool { + err := k8sClient.Get(ctx, rayJobLookupKey, fetchedRayJob) + if err != nil { + return false + } + if fetchedRayJob.Status.LastScheduleTime != nil { + lastScheduleTime = &fetchedRayJob.Status.LastScheduleTime.Time + return true + } + return false + }, time.Second*10, time.Millisecond*500).Should(BeTrue(), "expected LastScheduleTime to be set") + + Expect(lastScheduleTime).ToNot(BeNil(), "LastScheduleTime should not be nil") + Expect(*lastScheduleTime).ToNot(BeZero(), "LastScheduleTime should not be a zero time") + + Expect(lastScheduleTime.After(time.Now().Add(-15*time.Second))).To(BeTrue(), "LastScheduleTime should be within the last 15 seconds") + Expect(lastScheduleTime.Before(time.Now().Add(5*time.Second))).To(BeTrue(), "LastScheduleTime should not be in the future") + + GinkgoWriter.Printf("Validated LastScheduleTime: %s\n", lastScheduleTime.String()) + }) + + // The cron job runs every minute so it will take at most 1 minute to run + It("should transition to the Initializing", func() { + Eventually( + getRayJobDeploymentStatus(ctx, rayJob), + time.Second*60, time.Microsecond*500).Should(Equal(rayv1.JobDeploymentStatusInitializing), + "JobDeploymentStatus should be Initializing") + }) }) }) From 05c47c7d12f1ec388b09825a017e18f0ad68bda9 Mon Sep 17 00:00:00 2001 From: DW-Han Date: Thu, 24 Jul 2025 05:39:51 +0000 Subject: [PATCH 23/34] cleaning rayjob controller --- ray-operator/controllers/ray/rayjob_controller.go | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/ray-operator/controllers/ray/rayjob_controller.go b/ray-operator/controllers/ray/rayjob_controller.go index ac555f88cb5..b2f0a50ac00 100644 --- a/ray-operator/controllers/ray/rayjob_controller.go +++ b/ray-operator/controllers/ray/rayjob_controller.go @@ -461,8 +461,6 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) // If the RayJob is completed, we should not requeue it. return ctrl.Result{}, nil case rayv1.JobDeploymentStatusScheduling: - deleteCluster := rayJobInstance.Spec.ShutdownAfterJobFinishes - isJobDeleted, err := r.deleteSubmitterJob(ctx, rayJobInstance) if err != nil { return ctrl.Result{RequeueAfter: RayJobDefaultRequeueDuration}, err @@ -474,7 +472,7 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) return ctrl.Result{RequeueAfter: RayJobDefaultRequeueDuration}, nil } - if deleteCluster { + if rayJobInstance.Spec.ShutdownAfterJobFinishes { rayJobInstance.Status.RayClusterStatus = rayv1.RayClusterStatus{} rayJobInstance.Status.RayClusterName = "" From f6794911629f2fbe7da0528dc88941cf6feaa2aa Mon Sep 17 00:00:00 2001 From: DW-Han Date: Thu, 24 Jul 2025 08:03:41 +0000 Subject: [PATCH 24/34] cleaning test --- .../ray/rayjob_controller_scheduled_test.go | 54 +++++++++---------- 1 file changed, 27 insertions(+), 27 deletions(-) diff --git a/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go b/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go index 972de175de4..3af493a3e64 100644 --- a/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go +++ b/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go @@ -95,6 +95,33 @@ var _ = Context("RayJob with schedule operation", func() { time.Second*3, time.Millisecond*500).Should(Succeed()) }) + // We are checking if the LastScheduleTime is correctly set + It("should have LastScheduleTime updated in its status", func() { + rayJobLookupKey := types.NamespacedName{Name: rayJob.Name, Namespace: rayJob.Namespace} + fetchedRayJob := &rayv1.RayJob{} + + var lastScheduleTime *time.Time + Eventually(func() bool { + err := k8sClient.Get(ctx, rayJobLookupKey, fetchedRayJob) + if err != nil { + return false + } + if fetchedRayJob.Status.LastScheduleTime != nil { + lastScheduleTime = &fetchedRayJob.Status.LastScheduleTime.Time + return true + } + return false + }, time.Second*10, time.Millisecond*500).Should(BeTrue(), "expected LastScheduleTime to be set") + + Expect(lastScheduleTime).ToNot(BeNil(), "LastScheduleTime should not be nil") + Expect(*lastScheduleTime).ToNot(BeZero(), "LastScheduleTime should not be a zero time") + + Expect(lastScheduleTime.After(time.Now().Add(-15*time.Second))).To(BeTrue(), "LastScheduleTime should be within the last 15 seconds") + Expect(lastScheduleTime.Before(time.Now().Add(5*time.Second))).To(BeTrue(), "LastScheduleTime should not be in the future") + + GinkgoWriter.Printf("Validated LastScheduleTime: %s\n", lastScheduleTime.String()) + }) + It("should NOT create the underlying K8s job yet because the cluster is not ready", func() { underlyingK8sJob := &batchv1.Job{} Consistently( @@ -196,33 +223,6 @@ var _ = Context("RayJob with schedule operation", func() { ) }) - // We are checking if the LastScheduleTime is correctly set - It("should have LastScheduleTime updated in its status", func() { - rayJobLookupKey := types.NamespacedName{Name: rayJob.Name, Namespace: rayJob.Namespace} - fetchedRayJob := &rayv1.RayJob{} - - var lastScheduleTime *time.Time - Eventually(func() bool { - err := k8sClient.Get(ctx, rayJobLookupKey, fetchedRayJob) - if err != nil { - return false - } - if fetchedRayJob.Status.LastScheduleTime != nil { - lastScheduleTime = &fetchedRayJob.Status.LastScheduleTime.Time - return true - } - return false - }, time.Second*10, time.Millisecond*500).Should(BeTrue(), "expected LastScheduleTime to be set") - - Expect(lastScheduleTime).ToNot(BeNil(), "LastScheduleTime should not be nil") - Expect(*lastScheduleTime).ToNot(BeZero(), "LastScheduleTime should not be a zero time") - - Expect(lastScheduleTime.After(time.Now().Add(-15*time.Second))).To(BeTrue(), "LastScheduleTime should be within the last 15 seconds") - Expect(lastScheduleTime.Before(time.Now().Add(5*time.Second))).To(BeTrue(), "LastScheduleTime should not be in the future") - - GinkgoWriter.Printf("Validated LastScheduleTime: %s\n", lastScheduleTime.String()) - }) - // The cron job runs every minute so it will take at most 1 minute to run It("should transition to the Initializing", func() { Eventually( From 226113fde728d9b3823dc5cd7698d3ca63405182 Mon Sep 17 00:00:00 2001 From: DW-Han Date: Thu, 24 Jul 2025 08:25:42 +0000 Subject: [PATCH 25/34] cleaning test --- helm-chart/ray-cluster/README.md | 1 + .../ray/rayjob_controller_scheduled_test.go | 31 ------------------- 2 files changed, 1 insertion(+), 31 deletions(-) diff --git a/helm-chart/ray-cluster/README.md b/helm-chart/ray-cluster/README.md index 57aef2f5f33..678a40ad765 100644 --- a/helm-chart/ray-cluster/README.md +++ b/helm-chart/ray-cluster/README.md @@ -78,6 +78,7 @@ helm uninstall raycluster | nameOverride | string | `"kuberay"` | String to partially override release name. | | fullnameOverride | string | `""` | String to fully override release name. | | imagePullSecrets | list | `[]` | Secrets with credentials to pull images from a private registry | +| gcsFaultTolerance.enabled | bool | `false` | | | common.containerEnv | list | `[]` | containerEnv specifies environment variables for the Ray head and worker containers. Follows standard K8s container env schema. | | head.initContainers | list | `[]` | Init containers to add to the head pod | | head.labels | object | `{}` | Labels for the head pod | diff --git a/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go b/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go index 3af493a3e64..34263d0fa50 100644 --- a/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go +++ b/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go @@ -199,36 +199,5 @@ var _ = Context("RayJob with schedule operation", func() { }, time.Second*15, time.Millisecond*500).Should(BeTrue(), "Expected RayCluster to still exist") }) - - It("should have a JobDeploymentStatus reflecting its scheduled, new, or initializing state", func() { - getStatusFunc := getRayJobDeploymentStatus(ctx, rayJob) - - currentStatus, err := getStatusFunc() - Expect(err).NotTo(HaveOccurred(), "Failed to get RayJob status initially") - - if currentStatus == rayv1.JobDeploymentStatusInitializing || currentStatus == rayv1.JobDeploymentStatusNew { - Expect(currentStatus).To(Equal(rayv1.JobDeploymentStatusInitializing), "RayJob was already Initializing") - return - } - // If it's not Initializing, then it should be sheduled - Eventually( - getStatusFunc, - time.Second*5, - time.Millisecond*500, - ).Should( - Or( - Equal(rayv1.JobDeploymentStatusScheduled), - ), - "JobDeploymentStatus should be Scheduled, New or Initializing within 5 seconds", - ) - }) - - // The cron job runs every minute so it will take at most 1 minute to run - It("should transition to the Initializing", func() { - Eventually( - getRayJobDeploymentStatus(ctx, rayJob), - time.Second*60, time.Microsecond*500).Should(Equal(rayv1.JobDeploymentStatusInitializing), - "JobDeploymentStatus should be Initializing") - }) }) }) From 75cf551d27d37f80f3ccbb71f667e1e39367e94c Mon Sep 17 00:00:00 2001 From: DW-Han Date: Thu, 24 Jul 2025 09:30:47 +0000 Subject: [PATCH 26/34] returning to scheduled state --- .../ray/rayjob_controller_scheduled_test.go | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go b/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go index 34263d0fa50..1d5a008f2c2 100644 --- a/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go +++ b/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go @@ -199,5 +199,24 @@ var _ = Context("RayJob with schedule operation", func() { }, time.Second*15, time.Millisecond*500).Should(BeTrue(), "Expected RayCluster to still exist") }) + + It("should have a JobDeploymentStatus reflecting its scheduled, new, or initializing state", func() { + getStatusFunc := getRayJobDeploymentStatus(ctx, rayJob) + + currentStatus, err := getStatusFunc() + Expect(err).NotTo(HaveOccurred(), "Failed to get RayJob status initially") + + if currentStatus == rayv1.JobDeploymentStatusInitializing || currentStatus == rayv1.JobDeploymentStatusNew { + Expect(currentStatus).To(Or(Equal(rayv1.JobDeploymentStatusInitializing), Equal(rayv1.JobDeploymentStatusNew)), "RayJob was already Initializing or New") + return + } + // If it's not Initializing, then it should be sheduled + Eventually(getStatusFunc, time.Second*5, time.Millisecond*500).Should( + Or( + Equal(rayv1.JobDeploymentStatusScheduled), + ), + "JobDeploymentStatus should be Scheduled, New or Initializing within 5 seconds", + ) + }) }) }) From baa17d611869badbec8b70c9cd330a3f42b7a301 Mon Sep 17 00:00:00 2001 From: DW-Han Date: Fri, 25 Jul 2025 00:07:04 +0000 Subject: [PATCH 27/34] cleaning test and controller --- .../controllers/ray/rayjob_controller.go | 5 +- .../ray/rayjob_controller_scheduled_test.go | 52 +------------------ 2 files changed, 4 insertions(+), 53 deletions(-) diff --git a/ray-operator/controllers/ray/rayjob_controller.go b/ray-operator/controllers/ray/rayjob_controller.go index b2f0a50ac00..c7cc77e87b0 100644 --- a/ray-operator/controllers/ray/rayjob_controller.go +++ b/ray-operator/controllers/ray/rayjob_controller.go @@ -856,11 +856,10 @@ func initRayJobStatusIfNeed(ctx context.Context, rayJob *rayv1.RayJob) error { } // if the rayjob is scheduled according to a cron string set the status to scheduling instead of initializing to begin with // we check the job count to know if its the first job + + rayJob.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusInitializing if rayJob.Spec.Schedule != "" && rayJob.Status.Failed == nil && rayJob.Status.Succeeded == nil { - logger.Info("Since this is a new schdueled job we enter the Scheduled state not Initializing") rayJob.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusScheduled - } else { - rayJob.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusInitializing } rayJob.Status.StartTime = &metav1.Time{Time: time.Now()} return nil diff --git a/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go b/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go index 1d5a008f2c2..19695033feb 100644 --- a/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go +++ b/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go @@ -53,30 +53,6 @@ var _ = Context("RayJob with schedule operation", func() { err := k8sClient.Create(ctx, rayJob) Expect(err).NotTo(HaveOccurred(), "failed to create test scheduled RayJob resource") }) - - // Since the test can be run at any time, the RayJob might immediately - // transition past "Scheduled" to "New" or "Initializing" as it triggers a run. - // We first check if we have already passed the "Scheduled" state this - // accommodates the race condition if we only checked "Scheduled" and avoiding test flakiness. - It("should have a JobDeploymentStatus reflecting its scheduled, new, or initializing state", func() { - getStatusFunc := getRayJobDeploymentStatus(ctx, rayJob) - - currentStatus, err := getStatusFunc() - Expect(err).NotTo(HaveOccurred(), "Failed to get RayJob status initially") - - if currentStatus == rayv1.JobDeploymentStatusInitializing || currentStatus == rayv1.JobDeploymentStatusNew { - Expect(currentStatus).To(Or(Equal(rayv1.JobDeploymentStatusInitializing), Equal(rayv1.JobDeploymentStatusNew)), "RayJob was already Initializing or New") - return - } - // If it's not Initializing, then it should be sheduled - Eventually(getStatusFunc, time.Second*5, time.Millisecond*500).Should( - Or( - Equal(rayv1.JobDeploymentStatusScheduled), - ), - "JobDeploymentStatus should be Scheduled, New or Initializing within 5 seconds", - ) - }) - // The cron job runs every minute so it will take at most 1 minute to run It("should transition to the Initializing", func() { Eventually( @@ -95,7 +71,7 @@ var _ = Context("RayJob with schedule operation", func() { time.Second*3, time.Millisecond*500).Should(Succeed()) }) - // We are checking if the LastScheduleTime is correctly set + // We are checking if LastScheduleTime is correctly set It("should have LastScheduleTime updated in its status", func() { rayJobLookupKey := types.NamespacedName{Name: rayJob.Name, Namespace: rayJob.Namespace} fetchedRayJob := &rayv1.RayJob{} @@ -113,13 +89,8 @@ var _ = Context("RayJob with schedule operation", func() { return false }, time.Second*10, time.Millisecond*500).Should(BeTrue(), "expected LastScheduleTime to be set") - Expect(lastScheduleTime).ToNot(BeNil(), "LastScheduleTime should not be nil") - Expect(*lastScheduleTime).ToNot(BeZero(), "LastScheduleTime should not be a zero time") - Expect(lastScheduleTime.After(time.Now().Add(-15*time.Second))).To(BeTrue(), "LastScheduleTime should be within the last 15 seconds") Expect(lastScheduleTime.Before(time.Now().Add(5*time.Second))).To(BeTrue(), "LastScheduleTime should not be in the future") - - GinkgoWriter.Printf("Validated LastScheduleTime: %s\n", lastScheduleTime.String()) }) It("should NOT create the underlying K8s job yet because the cluster is not ready", func() { @@ -159,7 +130,7 @@ var _ = Context("RayJob with schedule operation", func() { "JobDeploymentStatus should be Running") }) - It("RayJobs's JobDeploymentStatus transitions to Scheduled after Job is Complete.", func() { + It("RayJobs's JobDeploymentStatus transitions to Scheduled after Complete.", func() { // Update fake dashboard client to return job info with "Succeeded" status. getJobInfo := func(context.Context, string) (*utils.RayJobInfo, error) { //nolint:unparam // This is a mock function so parameters are required return &utils.RayJobInfo{JobStatus: rayv1.JobStatusSucceeded, EndTime: uint64(time.Now().UnixMilli())}, nil @@ -199,24 +170,5 @@ var _ = Context("RayJob with schedule operation", func() { }, time.Second*15, time.Millisecond*500).Should(BeTrue(), "Expected RayCluster to still exist") }) - - It("should have a JobDeploymentStatus reflecting its scheduled, new, or initializing state", func() { - getStatusFunc := getRayJobDeploymentStatus(ctx, rayJob) - - currentStatus, err := getStatusFunc() - Expect(err).NotTo(HaveOccurred(), "Failed to get RayJob status initially") - - if currentStatus == rayv1.JobDeploymentStatusInitializing || currentStatus == rayv1.JobDeploymentStatusNew { - Expect(currentStatus).To(Or(Equal(rayv1.JobDeploymentStatusInitializing), Equal(rayv1.JobDeploymentStatusNew)), "RayJob was already Initializing or New") - return - } - // If it's not Initializing, then it should be sheduled - Eventually(getStatusFunc, time.Second*5, time.Millisecond*500).Should( - Or( - Equal(rayv1.JobDeploymentStatusScheduled), - ), - "JobDeploymentStatus should be Scheduled, New or Initializing within 5 seconds", - ) - }) }) }) From 4e03d8213974afb522d15d3de6ab795c858c95cb Mon Sep 17 00:00:00 2001 From: DW-Han Date: Fri, 25 Jul 2025 01:57:42 +0000 Subject: [PATCH 28/34] cleaning --- ray-operator/controllers/ray/rayjob_controller_scheduled_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go b/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go index 19695033feb..1dd961222ad 100644 --- a/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go +++ b/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go @@ -53,6 +53,7 @@ var _ = Context("RayJob with schedule operation", func() { err := k8sClient.Create(ctx, rayJob) Expect(err).NotTo(HaveOccurred(), "failed to create test scheduled RayJob resource") }) + // The cron job runs every minute so it will take at most 1 minute to run It("should transition to the Initializing", func() { Eventually( From 00932b86fa98f2bc6520b5aba4dc5abbd70c0370 Mon Sep 17 00:00:00 2001 From: DW-Han Date: Mon, 28 Jul 2025 16:29:14 +0000 Subject: [PATCH 29/34] no cluster creation at start of schedule and cleaning tests --- .../controllers/ray/rayjob_controller.go | 13 +++--- .../ray/rayjob_controller_scheduled_test.go | 45 +++++++++++++++++-- .../controllers/ray/suite_helpers_test.go | 11 +++++ 3 files changed, 59 insertions(+), 10 deletions(-) diff --git a/ray-operator/controllers/ray/rayjob_controller.go b/ray-operator/controllers/ray/rayjob_controller.go index c7cc77e87b0..f5cc48537f0 100644 --- a/ray-operator/controllers/ray/rayjob_controller.go +++ b/ray-operator/controllers/ray/rayjob_controller.go @@ -163,6 +163,12 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) logger.Info("RayJob", "JobStatus", rayJobInstance.Status.JobStatus, "JobDeploymentStatus", rayJobInstance.Status.JobDeploymentStatus, "SubmissionMode", rayJobInstance.Spec.SubmissionMode) switch rayJobInstance.Status.JobDeploymentStatus { case rayv1.JobDeploymentStatusNew: + // We check the LastScheduleTime to know if its the first job + if rayJobInstance.Spec.Schedule != "" && rayJobInstance.Status.LastScheduleTime == nil { + rayJobInstance.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusScheduled + break + } + if !controllerutil.ContainsFinalizer(rayJobInstance, utils.RayJobStopJobFinalizer) { logger.Info("Add a finalizer", "finalizer", utils.RayJobStopJobFinalizer) controllerutil.AddFinalizer(rayJobInstance, utils.RayJobStopJobFinalizer) @@ -453,7 +459,7 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) } } if rayJobInstance.Spec.Schedule != "" { - logger.Info("RayJob is scheduled again") + logger.Info("Rescheduling RayJob") rayJobInstance.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusScheduling break } @@ -854,13 +860,8 @@ func initRayJobStatusIfNeed(ctx context.Context, rayJob *rayv1.RayJob) error { if rayJob.Status.JobStatus == "" { rayJob.Status.JobStatus = rayv1.JobStatusNew } - // if the rayjob is scheduled according to a cron string set the status to scheduling instead of initializing to begin with - // we check the job count to know if its the first job rayJob.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusInitializing - if rayJob.Spec.Schedule != "" && rayJob.Status.Failed == nil && rayJob.Status.Succeeded == nil { - rayJob.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusScheduled - } rayJob.Status.StartTime = &metav1.Time{Time: time.Now()} return nil } diff --git a/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go b/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go index 1dd961222ad..449edbb70c1 100644 --- a/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go +++ b/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go @@ -38,7 +38,7 @@ var _ = Context("RayJob with schedule operation", func() { // In the last scheduled state the cluster should still exist since ShutdownAfterJobFinishes is False ctx := context.Background() namespace := "default" - cronSchedule := "*/1 * * * *" + cronSchedule := "0 0 1 1 *" rayJob := rayJobTemplate("rayjob-scheduled-no-deletion", namespace) rayJob.Spec.Schedule = cronSchedule rayJob.Spec.ShutdownAfterJobFinishes = false @@ -54,11 +54,48 @@ var _ = Context("RayJob with schedule operation", func() { Expect(err).NotTo(HaveOccurred(), "failed to create test scheduled RayJob resource") }) + It("should start in the Scheduled state", func() { + Eventually( + getRayJobDeploymentStatus(ctx, rayJob), + time.Second*60, time.Microsecond*500).Should(Equal(rayv1.JobDeploymentStatusScheduled), + "JobDeploymentStatus should be Initializing") + }) + + It("should NOT create a raycluster object while scheduled", func() { + Consistently( + getRayClusterNameForRayJob(ctx, rayJob), + time.Second*3, time.Millisecond*500).Should(BeEmpty()) + }) + + It("should update schedule string in its spec", func() { + rayJobLookupKey := types.NamespacedName{Name: rayJob.Name, Namespace: rayJob.Namespace} + fetchedRayJob := &rayv1.RayJob{} + newSchedule := "*/1 * * * *" + + Eventually(func() bool { + err := k8sClient.Get(ctx, rayJobLookupKey, fetchedRayJob) + return err == nil + }, time.Second*5, time.Millisecond*200).Should(BeTrue(), "expected to fetch the RayJob before update") + + fetchedRayJob.Spec.Schedule = newSchedule + + err := updateRayJobScheduleField(ctx, fetchedRayJob, newSchedule) + Expect(err).NotTo(HaveOccurred(), "failed to update RayJob's schedule in spec") + + Eventually(func() bool { + err := k8sClient.Get(ctx, rayJobLookupKey, fetchedRayJob) + if err != nil { + return false + } + return fetchedRayJob.Spec.Schedule == newSchedule + }, time.Second*10, time.Millisecond*500).Should(BeTrue(), "expected RayJob's Spec.Schedule to be updated and match") + }) + // The cron job runs every minute so it will take at most 1 minute to run It("should transition to the Initializing", func() { Eventually( getRayJobDeploymentStatus(ctx, rayJob), - time.Second*60, time.Microsecond*500).Should(Equal(rayv1.JobDeploymentStatusInitializing), + time.Second*70, time.Microsecond*500).Should(Equal(rayv1.JobDeploymentStatusInitializing), "JobDeploymentStatus should be Initializing") }) @@ -95,11 +132,11 @@ var _ = Context("RayJob with schedule operation", func() { }) It("should NOT create the underlying K8s job yet because the cluster is not ready", func() { - underlyingK8sJob := &batchv1.Job{} + k8sJob := &batchv1.Job{} Consistently( // k8sClient client throws error if resource not found func() bool { - err := getResourceFunc(ctx, common.RayJobK8sJobNamespacedName(rayJob), underlyingK8sJob)() + err := getResourceFunc(ctx, common.RayJobK8sJobNamespacedName(rayJob), k8sJob)() return errors.IsNotFound(err) }, time.Second*3, time.Millisecond*500).Should(BeTrue()) diff --git a/ray-operator/controllers/ray/suite_helpers_test.go b/ray-operator/controllers/ray/suite_helpers_test.go index f463ef559d9..4c14f93a850 100644 --- a/ray-operator/controllers/ray/suite_helpers_test.go +++ b/ray-operator/controllers/ray/suite_helpers_test.go @@ -319,6 +319,17 @@ func updateRayJobSuspendField(ctx context.Context, rayJob *rayv1.RayJob, suspend }) } +func updateRayJobScheduleField(ctx context.Context, rayJob *rayv1.RayJob, schedule string) error { + return retry.RetryOnConflict(retry.DefaultRetry, func() error { + err := k8sClient.Get(ctx, client.ObjectKey{Namespace: rayJob.Namespace, Name: rayJob.Name}, rayJob) + if err != nil { + return err + } + rayJob.Spec.Schedule = schedule + return k8sClient.Update(ctx, rayJob) + }) +} + func setJobIdOnRayJob(ctx context.Context, rayJob *rayv1.RayJob, jobId string) error { return retry.RetryOnConflict(retry.DefaultRetry, func() error { err := k8sClient.Get(ctx, client.ObjectKey{Namespace: rayJob.Namespace, Name: rayJob.Name}, rayJob) From 26fe74fdb0a17be86ad830373336cb12a1548866 Mon Sep 17 00:00:00 2001 From: DW-Han Date: Mon, 28 Jul 2025 19:28:40 +0000 Subject: [PATCH 30/34] cleaning schedule distance function and other cleaning --- ray-operator/controllers/ray/rayjob_controller.go | 15 +++++++-------- .../ray/rayjob_controller_unit_test.go | 14 ++++---------- 2 files changed, 11 insertions(+), 18 deletions(-) diff --git a/ray-operator/controllers/ray/rayjob_controller.go b/ray-operator/controllers/ray/rayjob_controller.go index f5cc48537f0..c646a9ba418 100644 --- a/ray-operator/controllers/ray/rayjob_controller.go +++ b/ray-operator/controllers/ray/rayjob_controller.go @@ -163,12 +163,6 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) logger.Info("RayJob", "JobStatus", rayJobInstance.Status.JobStatus, "JobDeploymentStatus", rayJobInstance.Status.JobDeploymentStatus, "SubmissionMode", rayJobInstance.Spec.SubmissionMode) switch rayJobInstance.Status.JobDeploymentStatus { case rayv1.JobDeploymentStatusNew: - // We check the LastScheduleTime to know if its the first job - if rayJobInstance.Spec.Schedule != "" && rayJobInstance.Status.LastScheduleTime == nil { - rayJobInstance.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusScheduled - break - } - if !controllerutil.ContainsFinalizer(rayJobInstance, utils.RayJobStopJobFinalizer) { logger.Info("Add a finalizer", "finalizer", utils.RayJobStopJobFinalizer) controllerutil.AddFinalizer(rayJobInstance, utils.RayJobStopJobFinalizer) @@ -177,6 +171,11 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) return ctrl.Result{RequeueAfter: RayJobDefaultRequeueDuration}, err } } + // We check the LastScheduleTime to know if its the first job + if rayJobInstance.Spec.Schedule != "" && rayJobInstance.Status.LastScheduleTime == nil { + rayJobInstance.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusScheduled + break + } // Set `Status.JobDeploymentStatus` to `JobDeploymentStatusInitializing`, and initialize `Status.JobId` // and `Status.RayClusterName` prior to avoid duplicate job submissions and cluster creations. logger.Info("JobDeploymentStatusNew") @@ -494,7 +493,7 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) case rayv1.JobDeploymentStatusScheduled: // We get the time from the current time to the previous and next cron schedule times // We pass in time.Now() as a parameter so easier unit testing and consistency - t1, t2, err := r.getPreviousAndNextScheduleDistance(ctx, time.Now(), rayJobInstance) + t1, t2, err := r.getNextAndPreviousScheduleDistance(ctx, time.Now(), rayJobInstance) if err != nil { logger.Error(err, "Could not get the previous and next distances for a cron schedule") return ctrl.Result{}, err @@ -953,7 +952,7 @@ func (r *RayJobReconciler) constructRayClusterForRayJob(rayJobInstance *rayv1.Ra return rayCluster, nil } -func (r *RayJobReconciler) getPreviousAndNextScheduleDistance(ctx context.Context, currentTime time.Time, rayJobInstance *rayv1.RayJob) (time.Duration, time.Duration, error) { +func (r *RayJobReconciler) getNextAndPreviousScheduleDistance(ctx context.Context, currentTime time.Time, rayJobInstance *rayv1.RayJob) (time.Duration, time.Duration, error) { logger := ctrl.LoggerFrom(ctx) logger.Info("Calculating next schedule for the RayJob") formatedCron := utils.FormatSchedule(rayJobInstance, r.Recorder) diff --git a/ray-operator/controllers/ray/rayjob_controller_unit_test.go b/ray-operator/controllers/ray/rayjob_controller_unit_test.go index 0f4127cbb59..f461cde5e17 100644 --- a/ray-operator/controllers/ray/rayjob_controller_unit_test.go +++ b/ray-operator/controllers/ray/rayjob_controller_unit_test.go @@ -625,14 +625,14 @@ func TestEmitRayJobExecutionDuration(t *testing.T) { } } -func TestGetPreviousAndNextScheduleDistance(t *testing.T) { +func TestGetNextAndPreviousScheduleDistance(t *testing.T) { // Test 1, the cron string is not valid // Test 2, we are not within the the buffer period of a cron tick to run a ray job // Test 3, we are within the buffer period of a cron tick to run a ray job newScheme := runtime.NewScheme() _ = rayv1.AddToScheme(newScheme) - _ = corev1.AddToScheme(newScheme) // For events + _ = corev1.AddToScheme(newScheme) testCases := []struct { currentTime time.Time @@ -695,8 +695,7 @@ func TestGetPreviousAndNextScheduleDistance(t *testing.T) { Scheme: newScheme, } - // Call getPreviousAndNextScheduleDistance to get the next and previous schedule ticks - nextDuration, prevDuration, err := reconciler.getPreviousAndNextScheduleDistance(context.Background(), tc.currentTime, rayJob) + nextDuration, prevDuration, err := reconciler.getNextAndPreviousScheduleDistance(context.Background(), tc.currentTime, rayJob) if tc.expectedErr { require.Error(t, err) @@ -704,15 +703,10 @@ func TestGetPreviousAndNextScheduleDistance(t *testing.T) { } else { require.NoError(t, err) - // Asserting that assert.InDelta(t, tc.expectedNextDelta.Seconds(), nextDuration.Seconds(), 1.0, "NextScheduleTimeDuration mismatch") assert.InDelta(t, tc.expectedPrevDelta.Seconds(), prevDuration.Seconds(), 1.0, "LastScheduleTimeDuration mismatch") - // Testing the ScheduleDelta logic and how it's called in reconcile - // Define ScheduleDelta within the test scope or as a global constant for testing - const ScheduleDelta = 100 * time.Millisecond - - isCurrentlyWithinBuffer := (nextDuration < ScheduleDelta) || (prevDuration < ScheduleDelta) + isCurrentlyWithinBuffer := (nextDuration < ScheduleBuffer) || (prevDuration < ScheduleBuffer) assert.Equal(t, tc.isWithinBuffer, isCurrentlyWithinBuffer, "isWithinBuffer check mismatch") } }) From f5f26faefe9b9fcdc16c7b0aa660b370fec378ee Mon Sep 17 00:00:00 2001 From: DW-Han Date: Mon, 28 Jul 2025 19:35:36 +0000 Subject: [PATCH 31/34] cleaning comments --- .../controllers/ray/rayjob_controller_scheduled_test.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go b/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go index 449edbb70c1..e71054da550 100644 --- a/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go +++ b/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go @@ -155,7 +155,7 @@ var _ = Context("RayJob with schedule operation", func() { It("should create the underlying Kubernetes Job object", func() { underlyingK8sJob := &batchv1.Job{} - // The underlying Kubernetes Job should be created when the RayJob is created + // The underlying Kubernetes Job should be created when the RayJob is scheduled to run Eventually( getResourceFunc(ctx, common.RayJobK8sJobNamespacedName(rayJob), underlyingK8sJob), time.Second*3, time.Millisecond*500).Should(Succeed(), "Expected Kubernetes job to be present") @@ -187,7 +187,6 @@ var _ = Context("RayJob with schedule operation", func() { err := k8sClient.Get(ctx, namespacedName, job) Expect(err).NotTo(HaveOccurred(), "failed to get Kubernetes Job") - // Update the submitter Kubernetes Job to Complete. conditions := []batchv1.JobCondition{ {Type: batchv1.JobComplete, Status: corev1.ConditionTrue}, } From c88ee44fd85512a290dd8a7ce291e8c11cde0824 Mon Sep 17 00:00:00 2001 From: DW-Han Date: Mon, 28 Jul 2025 22:39:33 +0000 Subject: [PATCH 32/34] cleaning --- .../ray/rayjob_controller_scheduled_test.go | 52 ++++--------------- .../controllers/ray/suite_helpers_test.go | 16 ++++++ 2 files changed, 27 insertions(+), 41 deletions(-) diff --git a/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go b/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go index e71054da550..06ff6c7c3f0 100644 --- a/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go +++ b/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go @@ -58,7 +58,7 @@ var _ = Context("RayJob with schedule operation", func() { Eventually( getRayJobDeploymentStatus(ctx, rayJob), time.Second*60, time.Microsecond*500).Should(Equal(rayv1.JobDeploymentStatusScheduled), - "JobDeploymentStatus should be Initializing") + "JobDeploymentStatus should be Scheduled") }) It("should NOT create a raycluster object while scheduled", func() { @@ -72,23 +72,12 @@ var _ = Context("RayJob with schedule operation", func() { fetchedRayJob := &rayv1.RayJob{} newSchedule := "*/1 * * * *" - Eventually(func() bool { - err := k8sClient.Get(ctx, rayJobLookupKey, fetchedRayJob) - return err == nil - }, time.Second*5, time.Millisecond*200).Should(BeTrue(), "expected to fetch the RayJob before update") + err := k8sClient.Get(ctx, rayJobLookupKey, fetchedRayJob) + Expect(err).NotTo(HaveOccurred(), "failed to get RayJob before schedule update") fetchedRayJob.Spec.Schedule = newSchedule - - err := updateRayJobScheduleField(ctx, fetchedRayJob, newSchedule) + err = updateRayJobScheduleField(ctx, fetchedRayJob, newSchedule) Expect(err).NotTo(HaveOccurred(), "failed to update RayJob's schedule in spec") - - Eventually(func() bool { - err := k8sClient.Get(ctx, rayJobLookupKey, fetchedRayJob) - if err != nil { - return false - } - return fetchedRayJob.Spec.Schedule == newSchedule - }, time.Second*10, time.Millisecond*500).Should(BeTrue(), "expected RayJob's Spec.Schedule to be updated and match") }) // The cron job runs every minute so it will take at most 1 minute to run @@ -109,26 +98,12 @@ var _ = Context("RayJob with schedule operation", func() { time.Second*3, time.Millisecond*500).Should(Succeed()) }) - // We are checking if LastScheduleTime is correctly set + // We are checking if LastScheduleTime is set It("should have LastScheduleTime updated in its status", func() { - rayJobLookupKey := types.NamespacedName{Name: rayJob.Name, Namespace: rayJob.Namespace} - fetchedRayJob := &rayv1.RayJob{} - - var lastScheduleTime *time.Time - Eventually(func() bool { - err := k8sClient.Get(ctx, rayJobLookupKey, fetchedRayJob) - if err != nil { - return false - } - if fetchedRayJob.Status.LastScheduleTime != nil { - lastScheduleTime = &fetchedRayJob.Status.LastScheduleTime.Time - return true - } - return false - }, time.Second*10, time.Millisecond*500).Should(BeTrue(), "expected LastScheduleTime to be set") - - Expect(lastScheduleTime.After(time.Now().Add(-15*time.Second))).To(BeTrue(), "LastScheduleTime should be within the last 15 seconds") - Expect(lastScheduleTime.Before(time.Now().Add(5*time.Second))).To(BeTrue(), "LastScheduleTime should not be in the future") + Eventually( + getLastScheduleTime(ctx, k8sClient, rayJob), + time.Second*10, time.Millisecond*500, + ).ShouldNot(BeNil(), "expected LastScheduleTime to be set") }) It("should NOT create the underlying K8s job yet because the cluster is not ready", func() { @@ -154,10 +129,10 @@ var _ = Context("RayJob with schedule operation", func() { }) It("should create the underlying Kubernetes Job object", func() { - underlyingK8sJob := &batchv1.Job{} + k8sJob := &batchv1.Job{} // The underlying Kubernetes Job should be created when the RayJob is scheduled to run Eventually( - getResourceFunc(ctx, common.RayJobK8sJobNamespacedName(rayJob), underlyingK8sJob), + getResourceFunc(ctx, common.RayJobK8sJobNamespacedName(rayJob), k8sJob), time.Second*3, time.Millisecond*500).Should(Succeed(), "Expected Kubernetes job to be present") }) @@ -176,11 +151,6 @@ var _ = Context("RayJob with schedule operation", func() { fakeRayDashboardClient.GetJobInfoMock.Store(&getJobInfo) defer fakeRayDashboardClient.GetJobInfoMock.Store(nil) - // RayJob transitions to Complete if and only if the corresponding submitter Kubernetes Job is Complete or Failed. - Consistently( - getRayJobDeploymentStatus(ctx, rayJob), - time.Second*3, time.Millisecond*500).Should(Equal(rayv1.JobDeploymentStatusRunning), "JobDeploymentStatus = %v", rayJob.Status.JobDeploymentStatus) - // Update the submitter Kubernetes Job to Complete. namespacedName := common.RayJobK8sJobNamespacedName(rayJob) job := &batchv1.Job{} diff --git a/ray-operator/controllers/ray/suite_helpers_test.go b/ray-operator/controllers/ray/suite_helpers_test.go index 4c14f93a850..62970837256 100644 --- a/ray-operator/controllers/ray/suite_helpers_test.go +++ b/ray-operator/controllers/ray/suite_helpers_test.go @@ -205,6 +205,22 @@ func getPendingRayClusterWorkerGroupSpecsFunc(ctx context.Context, rayService *r } } +func getLastScheduleTime(ctx context.Context, k8sClient client.Client, rayJob *rayv1.RayJob) func() (*time.Time, error) { + return func() (*time.Time, error) { + rayJobLookupKey := client.ObjectKey{Name: rayJob.Name, Namespace: rayJob.Namespace} + fetchedRayJob := &rayv1.RayJob{} + + if err := k8sClient.Get(ctx, rayJobLookupKey, fetchedRayJob); err != nil { + return nil, err + } + + if fetchedRayJob.Status.LastScheduleTime != nil { + return &fetchedRayJob.Status.LastScheduleTime.Time, nil + } + return nil, nil + } +} + func checkServiceHealth(ctx context.Context, rayService *rayv1.RayService) func() (bool, error) { return func() (bool, error) { if err := k8sClient.Get(ctx, client.ObjectKey{Name: rayService.Name, Namespace: rayService.Namespace}, rayService); err != nil { From 2a57524551475d8861e5da6cad88fec5721eae8e Mon Sep 17 00:00:00 2001 From: DW-Han Date: Tue, 29 Jul 2025 22:23:34 +0000 Subject: [PATCH 33/34] clean commit --- ray-operator/controllers/ray/rayjob_controller_scheduled_test.go | 1 - 1 file changed, 1 deletion(-) diff --git a/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go b/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go index 06ff6c7c3f0..2a93db55461 100644 --- a/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go +++ b/ray-operator/controllers/ray/rayjob_controller_scheduled_test.go @@ -93,7 +93,6 @@ var _ = Context("RayJob with schedule operation", func() { getRayClusterNameForRayJob(ctx, rayJob), time.Second*15, time.Millisecond*500).Should(Not(BeEmpty())) Eventually( - getResourceFunc(ctx, common.RayJobRayClusterNamespacedName(rayJob), rayCluster), time.Second*3, time.Millisecond*500).Should(Succeed()) }) From e4c2a3e21ced1ddaffdc9e3d271d86af79a25e2d Mon Sep 17 00:00:00 2001 From: DW-Han Date: Wed, 30 Jul 2025 00:20:46 +0000 Subject: [PATCH 34/34] commiting changes --- ray-operator/controllers/ray/rayjob_controller.go | 3 --- ray-operator/controllers/ray/rayjob_controller_unit_test.go | 4 ---- 2 files changed, 7 deletions(-) diff --git a/ray-operator/controllers/ray/rayjob_controller.go b/ray-operator/controllers/ray/rayjob_controller.go index c646a9ba418..50e8647dc6b 100644 --- a/ray-operator/controllers/ray/rayjob_controller.go +++ b/ray-operator/controllers/ray/rayjob_controller.go @@ -859,7 +859,6 @@ func initRayJobStatusIfNeed(ctx context.Context, rayJob *rayv1.RayJob) error { if rayJob.Status.JobStatus == "" { rayJob.Status.JobStatus = rayv1.JobStatusNew } - rayJob.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusInitializing rayJob.Status.StartTime = &metav1.Time{Time: time.Now()} return nil @@ -954,7 +953,6 @@ func (r *RayJobReconciler) constructRayClusterForRayJob(rayJobInstance *rayv1.Ra func (r *RayJobReconciler) getNextAndPreviousScheduleDistance(ctx context.Context, currentTime time.Time, rayJobInstance *rayv1.RayJob) (time.Duration, time.Duration, error) { logger := ctrl.LoggerFrom(ctx) - logger.Info("Calculating next schedule for the RayJob") formatedCron := utils.FormatSchedule(rayJobInstance, r.Recorder) cronSchedule, err := cron.ParseStandard(formatedCron) if err != nil { @@ -963,7 +961,6 @@ func (r *RayJobReconciler) getNextAndPreviousScheduleDistance(ctx context.Contex r.Recorder.Eventf(rayJobInstance, corev1.EventTypeWarning, "UnparseableSchedule", "unparseable schedule: %q : %s", rayJobInstance.Spec.Schedule, err) return 0, 0, fmt.Errorf("the cron schedule provided is unparseable: %w", err) } - logger.Info("Successfully parsed cron schedule", "CronSchedule", formatedCron) t1 := utils.NextScheduleTimeDuration(logger, rayJobInstance, currentTime, cronSchedule) t2 := utils.LastScheduleTimeDuration(logger, rayJobInstance, currentTime, cronSchedule) diff --git a/ray-operator/controllers/ray/rayjob_controller_unit_test.go b/ray-operator/controllers/ray/rayjob_controller_unit_test.go index f461cde5e17..221b817c80d 100644 --- a/ray-operator/controllers/ray/rayjob_controller_unit_test.go +++ b/ray-operator/controllers/ray/rayjob_controller_unit_test.go @@ -626,10 +626,6 @@ func TestEmitRayJobExecutionDuration(t *testing.T) { } func TestGetNextAndPreviousScheduleDistance(t *testing.T) { - // Test 1, the cron string is not valid - // Test 2, we are not within the the buffer period of a cron tick to run a ray job - // Test 3, we are within the buffer period of a cron tick to run a ray job - newScheme := runtime.NewScheme() _ = rayv1.AddToScheme(newScheme) _ = corev1.AddToScheme(newScheme)