Skip to content

Commit

Permalink
RayClusterProvisioned status should be set while cluster is being pro…
Browse files Browse the repository at this point in the history
…visioned for the first time (ray-project#2304)

Signed-off-by: Andrew Sy Kim <[email protected]>
  • Loading branch information
andrewsykim authored Aug 14, 2024
1 parent 678ec25 commit 8e3296e
Show file tree
Hide file tree
Showing 4 changed files with 19 additions and 9 deletions.
1 change: 1 addition & 0 deletions ray-operator/apis/ray/v1/raycluster_types.go
Original file line number Diff line number Diff line change
Expand Up @@ -169,6 +169,7 @@ type RayClusterConditionType string
// Custom Reason for RayClusterCondition
const (
AllPodRunningAndReadyFirstTime = "AllPodRunningAndReadyFirstTime"
RayClusterPodsProvisioning = "RayClusterPodsProvisioning"
HeadPodNotFound = "HeadPodNotFound"
HeadPodRunningAndReady = "HeadPodRunningAndReady"
// UnknownReason says that the reason for the condition is unknown.
Expand Down
11 changes: 9 additions & 2 deletions ray-operator/controllers/ray/raycluster_controller.go
Original file line number Diff line number Diff line change
Expand Up @@ -1210,16 +1210,23 @@ func (r *RayClusterReconciler) calculateStatus(ctx context.Context, instance *ra
meta.SetStatusCondition(&newInstance.Status.Conditions, headPodReadyCondition)
}

if meta.FindStatusCondition(newInstance.Status.Conditions, string(rayv1.RayClusterProvisioned)) == nil {
if !meta.IsStatusConditionTrue(newInstance.Status.Conditions, string(rayv1.RayClusterProvisioned)) {
// RayClusterProvisioned indicates whether all Ray Pods are ready when the RayCluster is first created.
// Note RayClusterProvisioned StatusCondition will not be added to Raycluster until all Ray Pods are ready for the first time.
// Note RayClusterProvisioned StatusCondition will not be updated after all Ray Pods are ready for the first time.
if utils.CheckAllPodsRunning(ctx, runtimePods) {
meta.SetStatusCondition(&newInstance.Status.Conditions, metav1.Condition{
Type: string(rayv1.RayClusterProvisioned),
Status: metav1.ConditionTrue,
Reason: rayv1.AllPodRunningAndReadyFirstTime,
Message: "All Ray Pods are ready for the first time",
})
} else {
meta.SetStatusCondition(&newInstance.Status.Conditions, metav1.Condition{
Type: string(rayv1.RayClusterProvisioned),
Status: metav1.ConditionFalse,
Reason: rayv1.RayClusterPodsProvisioning,
Message: "RayCluster Pods are being provisioned for first time",
})
}
}

Expand Down
10 changes: 5 additions & 5 deletions ray-operator/controllers/ray/raycluster_controller_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -898,15 +898,15 @@ var _ = Context("Inside the default namespace", func() {
time.Second*3, time.Millisecond*500).Should(BeTrue())

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

By("Update the worker pod to Running")
workerPod.Status.Phase = corev1.PodRunning
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1829,15 +1829,17 @@ func TestRayClusterProvisionedCondition(t *testing.T) {
_ = fakeClient.Status().Update(ctx, headPod)
_ = fakeClient.Status().Update(ctx, workerPod)
testRayCluster, _ = r.calculateStatus(ctx, testRayCluster, nil)
assert.Nil(t, meta.FindStatusCondition(testRayCluster.Status.Conditions, string(rayv1.RayClusterProvisioned)))
rayClusterProvisionedCondition := meta.FindStatusCondition(testRayCluster.Status.Conditions, string(rayv1.RayClusterProvisioned))
assert.Equal(t, rayClusterProvisionedCondition.Status, metav1.ConditionFalse)
assert.Equal(t, rayClusterProvisionedCondition.Reason, rayv1.RayClusterPodsProvisioning)

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

Expand Down

0 comments on commit 8e3296e

Please sign in to comment.