Skip to content

Commit 8e3296e

Browse files
authored
RayClusterProvisioned status should be set while cluster is being provisioned for the first time (#2304)
Signed-off-by: Andrew Sy Kim <[email protected]>
1 parent 678ec25 commit 8e3296e

File tree

4 files changed

+19
-9
lines changed

4 files changed

+19
-9
lines changed

ray-operator/apis/ray/v1/raycluster_types.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -169,6 +169,7 @@ type RayClusterConditionType string
169169
// Custom Reason for RayClusterCondition
170170
const (
171171
AllPodRunningAndReadyFirstTime = "AllPodRunningAndReadyFirstTime"
172+
RayClusterPodsProvisioning = "RayClusterPodsProvisioning"
172173
HeadPodNotFound = "HeadPodNotFound"
173174
HeadPodRunningAndReady = "HeadPodRunningAndReady"
174175
// UnknownReason says that the reason for the condition is unknown.

ray-operator/controllers/ray/raycluster_controller.go

Lines changed: 9 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -1210,16 +1210,23 @@ func (r *RayClusterReconciler) calculateStatus(ctx context.Context, instance *ra
12101210
meta.SetStatusCondition(&newInstance.Status.Conditions, headPodReadyCondition)
12111211
}
12121212

1213-
if meta.FindStatusCondition(newInstance.Status.Conditions, string(rayv1.RayClusterProvisioned)) == nil {
1213+
if !meta.IsStatusConditionTrue(newInstance.Status.Conditions, string(rayv1.RayClusterProvisioned)) {
12141214
// RayClusterProvisioned indicates whether all Ray Pods are ready when the RayCluster is first created.
1215-
// Note RayClusterProvisioned StatusCondition will not be added to Raycluster until all Ray Pods are ready for the first time.
1215+
// Note RayClusterProvisioned StatusCondition will not be updated after all Ray Pods are ready for the first time.
12161216
if utils.CheckAllPodsRunning(ctx, runtimePods) {
12171217
meta.SetStatusCondition(&newInstance.Status.Conditions, metav1.Condition{
12181218
Type: string(rayv1.RayClusterProvisioned),
12191219
Status: metav1.ConditionTrue,
12201220
Reason: rayv1.AllPodRunningAndReadyFirstTime,
12211221
Message: "All Ray Pods are ready for the first time",
12221222
})
1223+
} else {
1224+
meta.SetStatusCondition(&newInstance.Status.Conditions, metav1.Condition{
1225+
Type: string(rayv1.RayClusterProvisioned),
1226+
Status: metav1.ConditionFalse,
1227+
Reason: rayv1.RayClusterPodsProvisioning,
1228+
Message: "RayCluster Pods are being provisioned for first time",
1229+
})
12231230
}
12241231
}
12251232

ray-operator/controllers/ray/raycluster_controller_test.go

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -898,15 +898,15 @@ var _ = Context("Inside the default namespace", func() {
898898
time.Second*3, time.Millisecond*500).Should(BeTrue())
899899

900900
By("Check RayCluster RayClusterProvisioned condition is false")
901-
// But the worker pod is not ready yet, RayClusterProvisioned condition should still be absent.
901+
// But the worker pod is not ready yet, RayClusterProvisioned condition should be false.
902902
Consistently(
903-
func() *metav1.Condition {
903+
func() bool {
904904
if err := getResourceFunc(ctx, client.ObjectKey{Name: rayCluster.Name, Namespace: namespace}, rayCluster)(); err != nil {
905-
return nil
905+
return false
906906
}
907-
return meta.FindStatusCondition(rayCluster.Status.Conditions, string(rayv1.RayClusterProvisioned))
907+
return meta.IsStatusConditionFalse(rayCluster.Status.Conditions, string(rayv1.RayClusterProvisioned))
908908
},
909-
time.Second*3, time.Millisecond*500).Should(BeNil())
909+
time.Second*3, time.Millisecond*500).Should(BeTrue())
910910

911911
By("Update the worker pod to Running")
912912
workerPod.Status.Phase = corev1.PodRunning

ray-operator/controllers/ray/raycluster_controller_unit_test.go

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -1829,15 +1829,17 @@ func TestRayClusterProvisionedCondition(t *testing.T) {
18291829
_ = fakeClient.Status().Update(ctx, headPod)
18301830
_ = fakeClient.Status().Update(ctx, workerPod)
18311831
testRayCluster, _ = r.calculateStatus(ctx, testRayCluster, nil)
1832-
assert.Nil(t, meta.FindStatusCondition(testRayCluster.Status.Conditions, string(rayv1.RayClusterProvisioned)))
1832+
rayClusterProvisionedCondition := meta.FindStatusCondition(testRayCluster.Status.Conditions, string(rayv1.RayClusterProvisioned))
1833+
assert.Equal(t, rayClusterProvisionedCondition.Status, metav1.ConditionFalse)
1834+
assert.Equal(t, rayClusterProvisionedCondition.Reason, rayv1.RayClusterPodsProvisioning)
18331835

18341836
// After a while, all Ray Pods are ready for the first time, RayClusterProvisioned condition should be added and set to True.
18351837
headPod.Status = ReadyStatus
18361838
workerPod.Status = ReadyStatus
18371839
_ = fakeClient.Status().Update(ctx, headPod)
18381840
_ = fakeClient.Status().Update(ctx, workerPod)
18391841
testRayCluster, _ = r.calculateStatus(ctx, testRayCluster, nil)
1840-
rayClusterProvisionedCondition := meta.FindStatusCondition(testRayCluster.Status.Conditions, string(rayv1.RayClusterProvisioned))
1842+
rayClusterProvisionedCondition = meta.FindStatusCondition(testRayCluster.Status.Conditions, string(rayv1.RayClusterProvisioned))
18411843
assert.Equal(t, rayClusterProvisionedCondition.Status, metav1.ConditionTrue)
18421844
assert.Equal(t, rayClusterProvisionedCondition.Reason, rayv1.AllPodRunningAndReadyFirstTime)
18431845

0 commit comments

Comments
 (0)