Skip to content

Commit

Permalink
[RayCluster][Feature] skip suspending worker groups if the in-tree au…
Browse files Browse the repository at this point in the history
…toscaler is enabled to prevent ray cluster from malfunctioning

Signed-off-by: Rueian <[email protected]>
  • Loading branch information
rueian committed Jan 17, 2025
1 parent 66e8688 commit 3155e13
Show file tree
Hide file tree
Showing 3 changed files with 5 additions and 6 deletions.
2 changes: 1 addition & 1 deletion ray-operator/controllers/ray/raycluster_controller.go
Original file line number Diff line number Diff line change
Expand Up @@ -267,7 +267,7 @@ func validateRayClusterSpec(instance *rayv1.RayCluster) error {
for _, workerGroup := range instance.Spec.WorkerGroupSpecs {
if workerGroup.Suspend != nil && *workerGroup.Suspend {
// TODO (rueian): This can be supported in future Ray. We should check the RayVersion once we know the version.
return fmt.Errorf("suspending worker groups is not supported with Autoscaler enabled")
return fmt.Errorf("suspending worker groups is not currently supported with Autoscaler enabled")
}
}
}
Expand Down
4 changes: 2 additions & 2 deletions ray-operator/controllers/ray/raycluster_controller_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -878,7 +878,7 @@ var _ = Context("Inside the default namespace", func() {
time.Second*3, time.Millisecond*500).Should(BeNil(), "Should be able to see RayCluster: %v", rayCluster.Name)
})

It("Check the number of Pods and add finalizers", func() {
It("Check the number of Pods", func() {
Eventually(listResourceFunc(ctx, &allPods, allFilters...), time.Second*3, time.Millisecond*500).
Should(Equal(4), fmt.Sprintf("all pods %v", allPods.Items))
})
Expand All @@ -889,7 +889,7 @@ var _ = Context("Inside the default namespace", func() {
Expect(err).NotTo(HaveOccurred(), "Failed to update RayCluster")
})

It("Worker pods should be not deleted and head pod should still be running", func() {
It("Worker pods should not be deleted and head pod should still be running", func() {
Consistently(listResourceFunc(ctx, &allPods, workerFilters...), time.Second*5, time.Millisecond*500).
Should(Equal(3), fmt.Sprintf("all pods %v", allPods.Items))
Consistently(listResourceFunc(ctx, &allPods, headFilters...), time.Second*5, time.Millisecond*500).
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3737,14 +3737,13 @@ func TestValidateRayClusterSpecSuspendingWorkerGroup(t *testing.T) {
},
},
}
workerGroupSpec := rayv1.WorkerGroupSpec{
workerGroupSpecSuspended := rayv1.WorkerGroupSpec{
Template: corev1.PodTemplateSpec{
Spec: corev1.PodSpec{
Containers: []corev1.Container{{Name: "ray-worker"}},
},
},
}
workerGroupSpecSuspended := *workerGroupSpec.DeepCopy()
workerGroupSpecSuspended.Suspend = ptr.To[bool](true)

tests := []struct {
Expand Down Expand Up @@ -3774,7 +3773,7 @@ func TestValidateRayClusterSpecSuspendingWorkerGroup(t *testing.T) {
},
},
expectError: true,
errorMessage: "suspending worker groups is not supported with Autoscaler enabled",
errorMessage: "suspending worker groups is not currently supported with Autoscaler enabled",
},
}

Expand Down

0 comments on commit 3155e13

Please sign in to comment.