Skip to content

Commit c9d35b2

Browse files
committed
Add test
Signed-off-by: win5923 <[email protected]>
1 parent bf87764 commit c9d35b2

File tree

4 files changed

+308
-5
lines changed

4 files changed

+308
-5
lines changed

ray-operator/controllers/ray/raycluster_controller_unit_test.go

Lines changed: 122 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -3643,3 +3643,125 @@ func TestReconcile_PodsWithAuthToken(t *testing.T) {
36433643
assert.True(t, authModeEnvFound, "Auth mode env vars not found")
36443644
}
36453645
}
3646+
3647+
func TestShouldRecreatePodsForUpgrade(t *testing.T) {
3648+
setupTest(t)
3649+
ctx := context.Background()
3650+
3651+
// Calculate template hashes for matching pods
3652+
headHash, err := common.GeneratePodTemplateHash(testRayCluster.Spec.HeadGroupSpec.Template)
3653+
require.NoError(t, err, "Failed to generate head template hash")
3654+
workerHash, err := common.GeneratePodTemplateHash(testRayCluster.Spec.WorkerGroupSpecs[0].Template)
3655+
require.NoError(t, err, "Failed to generate worker template hash")
3656+
3657+
// Helper function to create a pod with specific template hash
3658+
createPodWithHash := func(name string, nodeType rayv1.RayNodeType, groupName string, templateHash string) *corev1.Pod {
3659+
return &corev1.Pod{
3660+
ObjectMeta: metav1.ObjectMeta{
3661+
Name: name,
3662+
Namespace: namespaceStr,
3663+
Labels: map[string]string{
3664+
utils.RayNodeLabelKey: "yes",
3665+
utils.RayClusterLabelKey: instanceName,
3666+
utils.RayNodeTypeLabelKey: string(nodeType),
3667+
utils.RayNodeGroupLabelKey: groupName,
3668+
},
3669+
Annotations: map[string]string{
3670+
utils.PodTemplateHashKey: templateHash,
3671+
},
3672+
},
3673+
Spec: corev1.PodSpec{
3674+
Containers: []corev1.Container{
3675+
{Name: "ray-head", Image: "rayproject/ray:latest"},
3676+
},
3677+
},
3678+
Status: corev1.PodStatus{Phase: corev1.PodRunning},
3679+
}
3680+
}
3681+
3682+
tests := []struct {
3683+
name string
3684+
upgradeStrategy *rayv1.RayClusterUpgradeStrategy
3685+
pods []runtime.Object
3686+
expectedRecreate bool
3687+
}{
3688+
{
3689+
name: "No upgrade strategy",
3690+
upgradeStrategy: nil,
3691+
pods: testPods,
3692+
expectedRecreate: false,
3693+
},
3694+
{
3695+
name: "Upgrade strategy type is nil",
3696+
upgradeStrategy: &rayv1.RayClusterUpgradeStrategy{Type: nil},
3697+
pods: testPods,
3698+
expectedRecreate: false,
3699+
},
3700+
{
3701+
name: "Upgrade strategy type is None",
3702+
upgradeStrategy: &rayv1.RayClusterUpgradeStrategy{
3703+
Type: ptr.To(rayv1.RayClusterUpgradeNone),
3704+
},
3705+
pods: testPods,
3706+
expectedRecreate: false,
3707+
},
3708+
{
3709+
name: "Recreate strategy but no pods exist",
3710+
upgradeStrategy: &rayv1.RayClusterUpgradeStrategy{
3711+
Type: ptr.To(rayv1.Recreate),
3712+
},
3713+
pods: []runtime.Object{},
3714+
expectedRecreate: false,
3715+
},
3716+
{
3717+
name: "Recreate strategy with matching template hash",
3718+
upgradeStrategy: &rayv1.RayClusterUpgradeStrategy{
3719+
Type: ptr.To(rayv1.Recreate),
3720+
},
3721+
pods: []runtime.Object{
3722+
createPodWithHash("head-pod", rayv1.HeadNode, headGroupNameStr, headHash),
3723+
createPodWithHash("worker-pod", rayv1.WorkerNode, groupNameStr, workerHash),
3724+
},
3725+
expectedRecreate: false,
3726+
},
3727+
{
3728+
name: "Recreate strategy with mismatched head template hash",
3729+
upgradeStrategy: &rayv1.RayClusterUpgradeStrategy{
3730+
Type: ptr.To(rayv1.Recreate),
3731+
},
3732+
pods: []runtime.Object{
3733+
createPodWithHash("head-pod", rayv1.HeadNode, headGroupNameStr, "old-head-hash"),
3734+
createPodWithHash("worker-pod", rayv1.WorkerNode, groupNameStr, workerHash),
3735+
},
3736+
expectedRecreate: true,
3737+
},
3738+
{
3739+
name: "Recreate strategy with mismatched worker template hash",
3740+
upgradeStrategy: &rayv1.RayClusterUpgradeStrategy{
3741+
Type: ptr.To(rayv1.Recreate),
3742+
},
3743+
pods: []runtime.Object{
3744+
createPodWithHash("head-pod", rayv1.HeadNode, headGroupNameStr, headHash),
3745+
createPodWithHash("worker-pod", rayv1.WorkerNode, groupNameStr, "old-worker-hash"),
3746+
},
3747+
expectedRecreate: true,
3748+
},
3749+
}
3750+
3751+
for _, tc := range tests {
3752+
t.Run(tc.name, func(t *testing.T) {
3753+
cluster := testRayCluster.DeepCopy()
3754+
cluster.Spec.UpgradeStrategy = tc.upgradeStrategy
3755+
3756+
fakeClient := clientFake.NewClientBuilder().WithRuntimeObjects(tc.pods...).Build()
3757+
testRayClusterReconciler := &RayClusterReconciler{
3758+
Client: fakeClient,
3759+
Scheme: scheme.Scheme,
3760+
Recorder: &record.FakeRecorder{},
3761+
}
3762+
3763+
result := testRayClusterReconciler.shouldRecreatePodsForUpgrade(ctx, cluster)
3764+
assert.Equal(t, tc.expectedRecreate, result)
3765+
})
3766+
}
3767+
}

ray-operator/controllers/ray/utils/validation.go

Lines changed: 8 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -38,11 +38,14 @@ func ValidateRayClusterMetadata(metadata metav1.ObjectMeta) error {
3838
}
3939

4040
func ValidateRayClusterUpgradeOptions(instance *rayv1.RayCluster) error {
41-
if instance.Spec.UpgradeStrategy != nil {
42-
creatorCRDType := GetCRDType(instance.Labels[RayOriginatedFromCRDLabelKey])
43-
if creatorCRDType == RayJobCRD || creatorCRDType == RayServiceCRD {
44-
return fmt.Errorf("upgradeStrategy cannot be set when RayCluster is created by %s", creatorCRDType)
45-
}
41+
strategy := instance.Spec.UpgradeStrategy
42+
if strategy == nil || strategy.Type == nil || *strategy.Type == rayv1.RayClusterUpgradeNone {
43+
return nil
44+
}
45+
46+
creatorCRDType := GetCRDType(instance.Labels[RayOriginatedFromCRDLabelKey])
47+
if creatorCRDType == RayJobCRD || creatorCRDType == RayServiceCRD {
48+
return fmt.Errorf("upgradeStrategy cannot be set when RayCluster is created by %s", creatorCRDType)
4649
}
4750
return nil
4851
}

ray-operator/controllers/ray/utils/validation_test.go

Lines changed: 89 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1887,3 +1887,92 @@ func TestValidateClusterUpgradeOptions(t *testing.T) {
18871887
})
18881888
}
18891889
}
1890+
1891+
func TestValidateRayClusterUpgradeOptions(t *testing.T) {
1892+
tests := []struct {
1893+
upgradeStrategy *rayv1.RayClusterUpgradeStrategy
1894+
name string
1895+
originatedFromCRD string
1896+
errorMessage string
1897+
expectError bool
1898+
}{
1899+
{
1900+
name: "No upgrade strategy",
1901+
upgradeStrategy: nil,
1902+
expectError: false,
1903+
},
1904+
{
1905+
name: "Upgrade strategy set None for RayCluster",
1906+
upgradeStrategy: &rayv1.RayClusterUpgradeStrategy{
1907+
Type: ptr.To(rayv1.RayClusterUpgradeNone),
1908+
},
1909+
originatedFromCRD: string(RayClusterCRD),
1910+
expectError: false,
1911+
},
1912+
{
1913+
name: "Upgrade strategy set Recreate for RayCluster",
1914+
upgradeStrategy: &rayv1.RayClusterUpgradeStrategy{
1915+
Type: ptr.To(rayv1.Recreate),
1916+
},
1917+
originatedFromCRD: string(RayClusterCRD),
1918+
expectError: false,
1919+
},
1920+
{
1921+
name: "Upgrade strategy set Recreate for RayCluster created by RayJob",
1922+
upgradeStrategy: &rayv1.RayClusterUpgradeStrategy{
1923+
Type: ptr.To(rayv1.Recreate),
1924+
},
1925+
originatedFromCRD: string(RayJobCRD),
1926+
expectError: true,
1927+
errorMessage: "upgradeStrategy cannot be set when RayCluster is created by RayJob",
1928+
},
1929+
{
1930+
name: "Upgrade strategy set Recreate for RayCluster created by RayService",
1931+
upgradeStrategy: &rayv1.RayClusterUpgradeStrategy{
1932+
Type: ptr.To(rayv1.Recreate),
1933+
},
1934+
originatedFromCRD: string(RayServiceCRD),
1935+
expectError: true,
1936+
errorMessage: "upgradeStrategy cannot be set when RayCluster is created by RayService",
1937+
},
1938+
}
1939+
1940+
for _, tt := range tests {
1941+
t.Run(tt.name, func(t *testing.T) {
1942+
cluster := &rayv1.RayCluster{
1943+
ObjectMeta: metav1.ObjectMeta{
1944+
Name: "test-cluster",
1945+
Namespace: "default",
1946+
Labels: map[string]string{},
1947+
},
1948+
Spec: rayv1.RayClusterSpec{
1949+
UpgradeStrategy: tt.upgradeStrategy,
1950+
HeadGroupSpec: rayv1.HeadGroupSpec{
1951+
Template: corev1.PodTemplateSpec{
1952+
Spec: corev1.PodSpec{
1953+
Containers: []corev1.Container{
1954+
{Name: "ray-head", Image: "rayproject/ray:latest"},
1955+
},
1956+
},
1957+
},
1958+
},
1959+
},
1960+
}
1961+
1962+
if tt.originatedFromCRD != "" {
1963+
cluster.Labels[RayOriginatedFromCRDLabelKey] = tt.originatedFromCRD
1964+
}
1965+
1966+
err := ValidateRayClusterUpgradeOptions(cluster)
1967+
1968+
if tt.expectError {
1969+
require.Error(t, err, "Expected error for test case: %s", tt.name)
1970+
if tt.errorMessage != "" {
1971+
assert.Contains(t, err.Error(), tt.errorMessage, "Error message mismatch for test case: %s", tt.name)
1972+
}
1973+
} else {
1974+
require.NoError(t, err, "Unexpected error for test case: %s", tt.name)
1975+
}
1976+
})
1977+
}
1978+
}

ray-operator/test/e2e/raycluster_test.go

Lines changed: 89 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -9,6 +9,7 @@ import (
99
"k8s.io/apimachinery/pkg/api/errors"
1010
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
1111
"k8s.io/apimachinery/pkg/types"
12+
corev1ac "k8s.io/client-go/applyconfigurations/core/v1"
1213
"k8s.io/utils/ptr"
1314

1415
rayv1 "github.com/ray-project/kuberay/ray-operator/apis/ray/v1"
@@ -208,3 +209,91 @@ func TestRayClusterScalingDown(t *testing.T) {
208209
g.Expect(err).NotTo(HaveOccurred(), "Failed to remove finalizer from pod %s/%s", namespace.Name, pod.Name)
209210
}
210211
}
212+
213+
func TestRayClusterUpgradeStrategy(t *testing.T) {
214+
test := With(t)
215+
g := NewWithT(t)
216+
217+
// Create a namespace
218+
namespace := test.NewTestNamespace()
219+
220+
// Create RayCluster with Recreate upgrade strategy
221+
rayClusterAC := rayv1ac.RayCluster("raycluster-upgrade", namespace.Name).
222+
WithSpec(rayv1ac.RayClusterSpec().
223+
WithRayVersion(GetRayVersion()).
224+
WithUpgradeStrategy(rayv1ac.RayClusterUpgradeStrategy().
225+
WithType(rayv1.Recreate)).
226+
WithHeadGroupSpec(rayv1ac.HeadGroupSpec().
227+
WithRayStartParams(map[string]string{"dashboard-host": "0.0.0.0"}).
228+
WithTemplate(HeadPodTemplateApplyConfiguration())).
229+
WithWorkerGroupSpecs(rayv1ac.WorkerGroupSpec().
230+
WithReplicas(2).
231+
WithMinReplicas(1).
232+
WithMaxReplicas(5).
233+
WithGroupName("small-group").
234+
WithRayStartParams(map[string]string{"num-cpus": "1"}).
235+
WithTemplate(WorkerPodTemplateApplyConfiguration())))
236+
237+
rayCluster, err := test.Client().Ray().RayV1().RayClusters(namespace.Name).Apply(test.Ctx(), rayClusterAC, TestApplyOptions)
238+
g.Expect(err).NotTo(HaveOccurred())
239+
LogWithTimestamp(test.T(), "Created RayCluster %s/%s with Recreate upgrade strategy", rayCluster.Namespace, rayCluster.Name)
240+
241+
LogWithTimestamp(test.T(), "Waiting for RayCluster %s/%s to become ready", rayCluster.Namespace, rayCluster.Name)
242+
g.Eventually(RayCluster(test, rayCluster.Namespace, rayCluster.Name), TestTimeoutMedium).
243+
Should(WithTransform(RayClusterState, Equal(rayv1.Ready)))
244+
245+
// Get initial pods
246+
headPod, err := GetHeadPod(test, rayCluster)
247+
g.Expect(err).NotTo(HaveOccurred())
248+
initialHeadPodName := headPod.Name
249+
LogWithTimestamp(test.T(), "Initial head pod name: %s", initialHeadPodName)
250+
251+
workerPods, err := GetWorkerPods(test, rayCluster)
252+
g.Expect(err).NotTo(HaveOccurred())
253+
g.Expect(workerPods).To(HaveLen(2), "Should have 2 worker pods initially")
254+
255+
LogWithTimestamp(test.T(), "Updating RayCluster %s/%s pod template to trigger upgrade", rayCluster.Namespace, rayCluster.Name)
256+
// Update head pod template with a new environment variable to trigger recreation
257+
rayClusterAC.Spec.HeadGroupSpec.Template.Spec.Containers[0].
258+
WithEnv(corev1ac.EnvVar().
259+
WithName("TEST_UPGRADE_ENV").
260+
WithValue("upgraded"))
261+
262+
rayCluster, err = test.Client().Ray().RayV1().RayClusters(namespace.Name).Apply(test.Ctx(), rayClusterAC, TestApplyOptions)
263+
g.Expect(err).NotTo(HaveOccurred())
264+
LogWithTimestamp(test.T(), "Applied updated RayCluster spec")
265+
266+
LogWithTimestamp(test.T(), "Waiting for all pods to be deleted due to Recreate strategy")
267+
g.Eventually(func(gg Gomega) {
268+
pods, err := GetAllPods(test, rayCluster)
269+
gg.Expect(err).NotTo(HaveOccurred())
270+
gg.Expect(pods).To(BeEmpty(), "All pods should be deleted")
271+
}, TestTimeoutMedium).Should(Succeed())
272+
273+
LogWithTimestamp(test.T(), "Waiting for RayCluster %s/%s to become ready again", rayCluster.Namespace, rayCluster.Name)
274+
g.Eventually(RayCluster(test, rayCluster.Namespace, rayCluster.Name), TestTimeoutMedium).
275+
Should(WithTransform(RayClusterState, Equal(rayv1.Ready)))
276+
277+
// Verify new pods are created
278+
newHeadPod, err := GetHeadPod(test, rayCluster)
279+
g.Expect(err).NotTo(HaveOccurred())
280+
g.Expect(newHeadPod.Name).NotTo(Equal(initialHeadPodName), "Head pod should be recreated with a different name")
281+
LogWithTimestamp(test.T(), "New head pod name: %s", newHeadPod.Name)
282+
283+
// Verify the new environment variable is applied
284+
found := false
285+
for _, container := range newHeadPod.Spec.Containers {
286+
for _, env := range container.Env {
287+
if env.Name == "TEST_UPGRADE_ENV" && env.Value == "upgraded" {
288+
found = true
289+
break
290+
}
291+
}
292+
}
293+
g.Expect(found).To(BeTrue(), "New head pod should have the updated environment variable")
294+
295+
newWorkerPods, err := GetWorkerPods(test, rayCluster)
296+
g.Expect(err).NotTo(HaveOccurred())
297+
g.Expect(newWorkerPods).To(HaveLen(2), "Should have 2 worker pods after upgrade")
298+
LogWithTimestamp(test.T(), "Successfully verified RayCluster upgrade with Recreate strategy")
299+
}

0 commit comments

Comments
 (0)