-
Notifications
You must be signed in to change notification settings - Fork 648
[RayJob] Enhance RayJob DeletionStrategy to Support Multi-Stage Deletion #4040
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Conversation
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I suggest combining the controller logic in the same PR as the CRD, as the controller implementation may shed some light on how the API should change
e586f52 to
600a4dc
Compare
600a4dc to
35c6ad7
Compare
Signed-off-by: wei-chenglai <[email protected]>
d853ad6 to
ac38a47
Compare
|
Hi @rueian @Future-Outlier @kevin85421 @andrewsykim , please help take a look when you are available. The PR is ready for review. Thanks! |
ac38a47 to
3010927
Compare
|
/retest |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can you provide an example called ray-job.deletion-strategy.yaml in the folder ray-operator/config/samples/, so users can easily play the deletion policy feature?
for example:
apiVersion: ray.io/v1
kind: RayJob
metadata:
name: rayjob-sample
spec:
deletionStrategy:
deletionRules:
- policy: DeleteCluster
condition:
jobStatus: SUCCEEDED
ttlSecondsAfterFinished: 60
- policy: DeleteWorkers
condition:
jobStatus: FAILED
ttlSecondsAfterFinished: 60
entrypoint: python /home/ray/samples/sample_code.py
... other can be the same as `ray-job.sample.yaml`
240aa4d to
31867ad
Compare
|
Let’s wait for the other maintainers to take a look. It looks good to me, but I wasn’t involved in the design-doc discussions, so I’m worried that I might have missed something. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
todo:
check retry scenario to avoid deletion policy accidentally delete it.
Signed-off-by: Wei-Cheng Lai <[email protected]>
94c880f to
e952550
Compare
|
Thanks for the review! I have added a sample for |
| # 3. After 90 seconds, the RayJob custom resource itself is deleted, removing it from the Kubernetes API server. | ||
| deletionRules: | ||
| - condition: | ||
| jobStatus: FAILED |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
For my own education, is there a way to indicate a policy (e.g. DeleteWorkers) regardless of the job status, or do you need to specify two rules with both conditions for each?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
No, we need to specify two rules with both conditions for each now.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I see, that's unfortunately really verbose. I wonder if we can make a case where if jobStatus is null, we apply the deletion policy after TTL, regardless of the status.
deletionRules:
- condition
ttlSeconds: 30
policy: DeleteWorkers
What do you think @seanlaii @rueian @Future-Outlier ?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
How about using a list for status such as:
// +kubebuilder:validation:MinItems=1
// +kubebuilder:validation:Items:Enum=SUCCEEDED;FAILED
// +listType=set
JobStatuses []JobStatus `json:"jobStatuses"``
I prefer avoiding null == all for clarity and unexpected expanding if we add other supported status.
What do you think?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I would think the null case actually helps to keep the policy working when new statuses are introduced. Actually I think a list of JobStatuses is good too, but if jobStatuses is null I still think we should treat it as "any statuses after TTL".
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think a []JobStatus is verbose and complicates the implementation, but it might be a safer option.
If we go with null == all, then we need to have tests to make sure what all means explicitly. Specifically, the tests should ensure that if someday we expand the deletion policy to non-terminal statuses, the null case should not apply to RUNNING or other statuses.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I agree with the safer option, since it will be easier to maintain in the future.
And since @seanlaii is the user of this PR, I think supporting the user’s use case is more important for now.
If many users want it, we can consider it in the future — but we must always maintain backward compatibility.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
cc @andrewsykim do you agree?
Thank you!
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I am ok with current implementation or []JobStatus, no strong preference
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I am also okay with the current implementation. We can have a new PR for changing to either []JobStatus or null == all with tests preventing unexpected future changes.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
E2E tests for DeletionStrategy: https://buildkite.com/ray-project/ray-ecosystem-ci-kuberay-ci/builds/11135/steps/canvas?sid=019988b7-c45f-448a-8bf5-daebdddd6444.
d3f26cc to
0f9dd7b
Compare
| switch policy { | ||
| case rayv1.DeleteWorkers: | ||
| if err := r.Get(ctx, clusterIdentifier, cluster); err != nil { | ||
| if errors.IsNotFound(err) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Or when a RayCluster has a deletionTimestamp set, it should be treated as deleted as well.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks, updated!
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Pull Request Overview
This PR enhances the RayJob DeletionStrategy to support multi-stage deletion through the introduction of DeletionRules. The new system provides more granular control over resource cleanup timing compared to the legacy onSuccess/onFailure policies, while maintaining full backward compatibility.
- Introduction of
DeletionRulesAPI for multi-stage deletion with per-rule TTL controls - Enhanced controller logic to handle rule-based deletion with impact-aware priority handling
- Comprehensive validation system to prevent configuration conflicts between legacy and new APIs
Reviewed Changes
Copilot reviewed 21 out of 21 changed files in this pull request and generated 4 comments.
Show a summary per file
| File | Description |
|---|---|
| ray-operator/apis/ray/v1/rayjob_types.go | Adds new DeletionRule, DeletionCondition types and updates DeletionStrategy |
| ray-operator/controllers/ray/rayjob_controller.go | Implements multi-stage deletion handling with impact-aware rule execution |
| ray-operator/controllers/ray/utils/validation.go | Adds comprehensive validation for deletion rules and legacy policy conflicts |
| ray-operator/test/e2erayjob/rayjob_deletion_strategy_test.go | Comprehensive end-to-end tests covering all deletion scenarios |
| ray-operator/pkg/client/applyconfiguration/ray/v1/*.go | Generated client configuration for new DeletionRule types |
Tip: Customize your code reviews with copilot-instructions.md. Create the file or learn how to get started.
| namespace := test.NewTestNamespace() | ||
|
|
||
| // Job scripts - using existing counter.py for successful jobs and fail.py for failed jobs | ||
| // Note: This test suite requires the RayJobDeletionPolicy feature gate to be enabled |
Copilot
AI
Oct 7, 2025
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The comment should specify how to enable the feature gate for users running these tests manually, as this is important setup information that's not immediately obvious.
| // Note: This test suite requires the RayJobDeletionPolicy feature gate to be enabled | |
| // Note: This test suite requires the RayJobDeletionPolicy feature gate to be enabled. | |
| // To enable it when running tests manually, start the Ray operator with the following environment variable: | |
| // FEATURE_GATES=RayJobDeletionPolicy=true | |
| // or add "--feature-gates=RayJobDeletionPolicy=true" to the operator's startup arguments, depending on your deployment method. |
| func requeueDelayFor(t time.Time) time.Duration { | ||
| return time.Until(t) + 2*time.Second |
Copilot
AI
Oct 7, 2025
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The hardcoded 2-second buffer should be defined as a named constant to make it easier to adjust and understand its purpose.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I will find time do a final pass for this PR this week.
(I want to think about any concurrency edge case will happen or not)
thank you!
graph TD
A[RayJob Reaches Terminal State] --> B{DeletionStrategy Configured?}
B -->|No| C[Use Default Behavior]
B -->|Yes| D{Feature Gate Enabled?}
D -->|No| E[Error: Feature Gate Required]
D -->|Yes| F{DeletionRules or Legacy?}
F -->|Legacy onSuccess/onFailure| G[handleLegacyDeletionPolicy]
F -->|DeletionRules| H[handleDeletionRules]
G --> I[Check TTLSecondsAfterFinished]
I --> J{TTL Met?}
J -->|No| K[Requeue for Later]
J -->|Yes| L[Execute Single Policy]
H --> M[Process Each Rule]
M --> N{Rule Matches JobStatus?}
N -->|No| O[Skip Rule]
N -->|Yes| P{TTL Met?}
P -->|No| Q[Add to Pending Rules]
P -->|Yes| R{Action Completed?}
R -->|Yes| S[Skip Completed Rule]
R -->|No| T[Add to Overdue Rules]
Q --> U[Calculate Next Requeue Time]
T --> V{Overdue Rules Exist?}
V -->|No| U
V -->|Yes| W[selectMostImpactfulRule]
W --> X[Execute Most Impactful Policy]
X --> Y[DeleteSelf Priority: 4]
Y --> Z[DeleteCluster Priority: 3]
Z --> AA[DeleteWorkers Priority: 2]
AA --> BB[DeleteNone Priority: 1]
BB --> CC[executeDeletionPolicy]
CC --> DD{Policy Type}
DD -->|DeleteCluster| EE[Delete RayCluster CR]
DD -->|DeleteWorkers| FF[Suspend Worker Groups]
DD -->|DeleteSelf| GG[Delete RayJob CR]
DD -->|DeleteNone| HH[No Action]
EE --> II[Requeue for Next Rule]
FF --> II
GG --> JJ[Terminal - No More Rules]
HH --> II
II --> KK[Continue Processing Rules]
KK --> M
L --> LL[Single Policy Execution]
LL --> CC
Co-authored-by: Copilot <[email protected]> Signed-off-by: Wei-Cheng Lai <[email protected]>
| continue | ||
| } | ||
|
|
||
| deletionTime := rayJob.Status.EndTime.Add(time.Duration(rule.Condition.TTLSeconds) * time.Second) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Hi, @seanlaii
is it guaranteed that rayJob.Status.EndTime will not be nil?
Should we check if rayJob.Status.EndTime is not nil?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, it will always be non-nil if it is in terminal state. We set it here: https://github.com/ray-project/kuberay/blob/master/ray-operator/controllers/ray/rayjob_controller.go#L918.
Additionally, the original implementation assumes that it is non-nil as well: https://github.com/ray-project/kuberay/blob/master/ray-operator/controllers/ray/rayjob_controller.go#L391.
Why are these changes needed?
DeletionRulein RayJob API for multi-stage deletion.DeletionStrategyLogic and Tests.DeletionStrategy.Related issue number
Closes #4019 #4020 #4021
Checks
Design
API Changes
Old
New
Validation
onSuccess/onFailure) and new (deletionRules) APIs cannot be used simultaneously.DeleteWorkershappens beforeDeleteCluster, andDeleteClusterhappens beforeDeleteSelf.Controller
The RayJob controller has been updated to support the new
deletionRules.DeleteClusteroverDeleteWorkers). This ensures that only the most impactful operation is executed, instead of executing all the operation.User Impact
CRD & Runtime Impact (Fully Backward Compatible)
Go Client / Controller Impact (Minor Breaking Change)
deletionRulesto be the sole policy-defining field, the legacyonSuccessandonFailurefields were changed fromDeletionPolicyto*DeletionPolicy(a pointer). This makes them truly optional in the API.nilbefore dereferencing the pointer.Migration
Legacy Configuration
New Configuration
Multi-Stage Configuration
Delete workers after 30 seconds, and delete the cluster after 60 seconds.