From 229324c060d6a1dbb90ecbd6b9424ce412414c02 Mon Sep 17 00:00:00 2001 From: owenowenisme Date: Mon, 13 Jan 2025 01:01:27 +0000 Subject: [PATCH 1/9] moved client.matching label into associate.go Signed-off-by: owenowenisme --- ray-operator/controllers/ray/common/association.go | 8 ++++++++ .../controllers/ray/raycluster_controller.go | 13 +++++-------- 2 files changed, 13 insertions(+), 8 deletions(-) diff --git a/ray-operator/controllers/ray/common/association.go b/ray-operator/controllers/ray/common/association.go index 5566421a29..0c8fd3377d 100644 --- a/ray-operator/controllers/ray/common/association.go +++ b/ray-operator/controllers/ray/common/association.go @@ -184,3 +184,11 @@ func GetRayClusterHeadPod(ctx context.Context, reader client.Reader, instance *r } return &runtimePods.Items[0], nil } + +func GetFilteredRayClusterLabel(instance *rayv1.RayCluster) client.MatchingLabels { + return client.MatchingLabels{utils.RayClusterLabelKey: instance.Name} +} + +func GetFilteredRayClusterHeadPodLabel(instance *rayv1.RayCluster, nodeType string) client.MatchingLabels { + return client.MatchingLabels{utils.RayClusterLabelKey: instance.Name, utils.RayNodeTypeLabelKey: nodeType} +} diff --git a/ray-operator/controllers/ray/raycluster_controller.go b/ray-operator/controllers/ray/raycluster_controller.go index 794177651e..c0880b0307 100644 --- a/ray-operator/controllers/ray/raycluster_controller.go +++ b/ray-operator/controllers/ray/raycluster_controller.go @@ -347,7 +347,7 @@ func (r *RayClusterReconciler) rayClusterReconcile(ctx context.Context, instance } // We can start the Redis cleanup process now because the head Pod has been terminated. - filterLabels := client.MatchingLabels{utils.RayClusterLabelKey: instance.Name, utils.RayNodeTypeLabelKey: string(rayv1.RedisCleanupNode)} + filterLabels := common.GetFilteredRayClusterHeadPodLabel(instance, string(rayv1.RedisCleanupNode)) redisCleanupJobs := batchv1.JobList{} if err := r.List(ctx, &redisCleanupJobs, client.InNamespace(instance.Namespace), filterLabels); err != nil { return ctrl.Result{RequeueAfter: DefaultRequeueDuration}, err @@ -542,7 +542,7 @@ func (r *RayClusterReconciler) reconcileIngress(ctx context.Context, instance *r func (r *RayClusterReconciler) reconcileRouteOpenShift(ctx context.Context, instance *rayv1.RayCluster) error { logger := ctrl.LoggerFrom(ctx) headRoutes := routev1.RouteList{} - filterLabels := client.MatchingLabels{utils.RayClusterLabelKey: instance.Name} + filterLabels := common.GetFilteredRayClusterLabel(instance) if err := r.List(ctx, &headRoutes, client.InNamespace(instance.Namespace), filterLabels); err != nil { return err } @@ -573,7 +573,7 @@ func (r *RayClusterReconciler) reconcileRouteOpenShift(ctx context.Context, inst func (r *RayClusterReconciler) reconcileIngressKubernetes(ctx context.Context, instance *rayv1.RayCluster) error { logger := ctrl.LoggerFrom(ctx) headIngresses := networkingv1.IngressList{} - filterLabels := client.MatchingLabels{utils.RayClusterLabelKey: instance.Name} + filterLabels := common.GetFilteredRayClusterLabel(instance) if err := r.List(ctx, &headIngresses, client.InNamespace(instance.Namespace), filterLabels); err != nil { return err } @@ -605,7 +605,7 @@ func (r *RayClusterReconciler) reconcileIngressKubernetes(ctx context.Context, i func (r *RayClusterReconciler) reconcileHeadService(ctx context.Context, instance *rayv1.RayCluster) error { logger := ctrl.LoggerFrom(ctx) services := corev1.ServiceList{} - filterLabels := client.MatchingLabels{utils.RayClusterLabelKey: instance.Name, utils.RayNodeTypeLabelKey: string(rayv1.HeadNode)} + filterLabels := common.GetFilteredRayClusterHeadPodLabel(instance, string(rayv1.HeadNode)) if err := r.List(ctx, &services, client.InNamespace(instance.Namespace), filterLabels); err != nil { return err @@ -1510,10 +1510,7 @@ func (r *RayClusterReconciler) updateEndpoints(ctx context.Context, instance *ra // We assume we can find the right one by filtering Services with appropriate label selectors // and picking the first one. We may need to select by name in the future if the Service naming is stable. rayHeadSvc := corev1.ServiceList{} - filterLabels := client.MatchingLabels{ - utils.RayClusterLabelKey: instance.Name, - utils.RayNodeTypeLabelKey: "head", - } + filterLabels := common.GetFilteredRayClusterHeadPodLabel(instance, string(rayv1.HeadNode)) if err := r.List(ctx, &rayHeadSvc, client.InNamespace(instance.Namespace), filterLabels); err != nil { return err } From c2b44fd613bbbf9cf1dcf47922344d46666df20a Mon Sep 17 00:00:00 2001 From: owenowenisme Date: Tue, 14 Jan 2025 03:50:30 +0900 Subject: [PATCH 2/9] use existing AssociationOptions function Signed-off-by: owenowenisme --- .../controllers/ray/common/association.go | 18 ++++++++------- .../controllers/ray/raycluster_controller.go | 22 +++++++++---------- 2 files changed, 21 insertions(+), 19 deletions(-) diff --git a/ray-operator/controllers/ray/common/association.go b/ray-operator/controllers/ray/common/association.go index 0c8fd3377d..c9cd9d83d5 100644 --- a/ray-operator/controllers/ray/common/association.go +++ b/ray-operator/controllers/ray/common/association.go @@ -100,6 +100,16 @@ func RayClusterWorkerPodsAssociationOptions(instance *rayv1.RayCluster) Associat } } +func RayClusterRedisPodsAssociationOptions(instance *rayv1.RayCluster) AssociationOptions { + return AssociationOptions{ + client.InNamespace(instance.Namespace), + client.MatchingLabels{ + utils.RayClusterLabelKey: instance.Name, + utils.RayNodeTypeLabelKey: string(rayv1.RedisCleanupNode), + }, + } +} + func RayClusterGroupPodsAssociationOptions(instance *rayv1.RayCluster, group string) AssociationOptions { return AssociationOptions{ client.InNamespace(instance.Namespace), @@ -184,11 +194,3 @@ func GetRayClusterHeadPod(ctx context.Context, reader client.Reader, instance *r } return &runtimePods.Items[0], nil } - -func GetFilteredRayClusterLabel(instance *rayv1.RayCluster) client.MatchingLabels { - return client.MatchingLabels{utils.RayClusterLabelKey: instance.Name} -} - -func GetFilteredRayClusterHeadPodLabel(instance *rayv1.RayCluster, nodeType string) client.MatchingLabels { - return client.MatchingLabels{utils.RayClusterLabelKey: instance.Name, utils.RayNodeTypeLabelKey: nodeType} -} diff --git a/ray-operator/controllers/ray/raycluster_controller.go b/ray-operator/controllers/ray/raycluster_controller.go index c0880b0307..db6097310c 100644 --- a/ray-operator/controllers/ray/raycluster_controller.go +++ b/ray-operator/controllers/ray/raycluster_controller.go @@ -15,11 +15,11 @@ import ( metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "k8s.io/utils/ptr" + "github.com/ray-project/kubefray/ray-operator/controllers/ray/utils" configapi "github.com/ray-project/kuberay/ray-operator/apis/config/v1alpha1" "github.com/ray-project/kuberay/ray-operator/controllers/ray/batchscheduler" "github.com/ray-project/kuberay/ray-operator/controllers/ray/common" "github.com/ray-project/kuberay/ray-operator/controllers/ray/expectations" - "github.com/ray-project/kuberay/ray-operator/controllers/ray/utils" "github.com/ray-project/kuberay/ray-operator/pkg/features" batchv1 "k8s.io/api/batch/v1" @@ -347,9 +347,9 @@ func (r *RayClusterReconciler) rayClusterReconcile(ctx context.Context, instance } // We can start the Redis cleanup process now because the head Pod has been terminated. - filterLabels := common.GetFilteredRayClusterHeadPodLabel(instance, string(rayv1.RedisCleanupNode)) + filterLabels := common.RayClusterRedisPodsAssociationOptions(instance).ToListOptions() redisCleanupJobs := batchv1.JobList{} - if err := r.List(ctx, &redisCleanupJobs, client.InNamespace(instance.Namespace), filterLabels); err != nil { + if err := r.List(ctx, &redisCleanupJobs, filterLabels...); err != nil { return ctrl.Result{RequeueAfter: DefaultRequeueDuration}, err } @@ -542,8 +542,8 @@ func (r *RayClusterReconciler) reconcileIngress(ctx context.Context, instance *r func (r *RayClusterReconciler) reconcileRouteOpenShift(ctx context.Context, instance *rayv1.RayCluster) error { logger := ctrl.LoggerFrom(ctx) headRoutes := routev1.RouteList{} - filterLabels := common.GetFilteredRayClusterLabel(instance) - if err := r.List(ctx, &headRoutes, client.InNamespace(instance.Namespace), filterLabels); err != nil { + filterLabels := common.RayClusterAllPodsAssociationOptions(instance).ToListOptions() + if err := r.List(ctx, &headRoutes, filterLabels...); err != nil { return err } @@ -573,8 +573,8 @@ func (r *RayClusterReconciler) reconcileRouteOpenShift(ctx context.Context, inst func (r *RayClusterReconciler) reconcileIngressKubernetes(ctx context.Context, instance *rayv1.RayCluster) error { logger := ctrl.LoggerFrom(ctx) headIngresses := networkingv1.IngressList{} - filterLabels := common.GetFilteredRayClusterLabel(instance) - if err := r.List(ctx, &headIngresses, client.InNamespace(instance.Namespace), filterLabels); err != nil { + filterLabels := common.RayClusterAllPodsAssociationOptions(instance).ToListOptions() + if err := r.List(ctx, &headIngresses, filterLabels...); err != nil { return err } @@ -605,9 +605,9 @@ func (r *RayClusterReconciler) reconcileIngressKubernetes(ctx context.Context, i func (r *RayClusterReconciler) reconcileHeadService(ctx context.Context, instance *rayv1.RayCluster) error { logger := ctrl.LoggerFrom(ctx) services := corev1.ServiceList{} - filterLabels := common.GetFilteredRayClusterHeadPodLabel(instance, string(rayv1.HeadNode)) + filterLabels := common.RayClusterHeadPodsAssociationOptions(instance).ToListOptions() - if err := r.List(ctx, &services, client.InNamespace(instance.Namespace), filterLabels); err != nil { + if err := r.List(ctx, &services, filterLabels...); err != nil { return err } @@ -1510,8 +1510,8 @@ func (r *RayClusterReconciler) updateEndpoints(ctx context.Context, instance *ra // We assume we can find the right one by filtering Services with appropriate label selectors // and picking the first one. We may need to select by name in the future if the Service naming is stable. rayHeadSvc := corev1.ServiceList{} - filterLabels := common.GetFilteredRayClusterHeadPodLabel(instance, string(rayv1.HeadNode)) - if err := r.List(ctx, &rayHeadSvc, client.InNamespace(instance.Namespace), filterLabels); err != nil { + filterLabels := common.RayClusterAllPodsAssociationOptions(instance).ToListOptions() + if err := r.List(ctx, &rayHeadSvc, filterLabels...); err != nil { return err } From 0840c8446e7bad0a2209d1d1083ffe0b231fa4c1 Mon Sep 17 00:00:00 2001 From: owenowenisme Date: Tue, 14 Jan 2025 09:34:18 +0900 Subject: [PATCH 3/9] fix import path typo Signed-off-by: owenowenisme --- ray-operator/controllers/ray/raycluster_controller.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ray-operator/controllers/ray/raycluster_controller.go b/ray-operator/controllers/ray/raycluster_controller.go index db6097310c..88c87323a7 100644 --- a/ray-operator/controllers/ray/raycluster_controller.go +++ b/ray-operator/controllers/ray/raycluster_controller.go @@ -15,11 +15,11 @@ import ( metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "k8s.io/utils/ptr" - "github.com/ray-project/kubefray/ray-operator/controllers/ray/utils" configapi "github.com/ray-project/kuberay/ray-operator/apis/config/v1alpha1" "github.com/ray-project/kuberay/ray-operator/controllers/ray/batchscheduler" "github.com/ray-project/kuberay/ray-operator/controllers/ray/common" "github.com/ray-project/kuberay/ray-operator/controllers/ray/expectations" + "github.com/ray-project/kuberay/ray-operator/controllers/ray/utils" "github.com/ray-project/kuberay/ray-operator/pkg/features" batchv1 "k8s.io/api/batch/v1" From ebf254a9eeda2ed02dc62e0f1f85fb7beaea297b Mon Sep 17 00:00:00 2001 From: owenowenisme Date: Tue, 14 Jan 2025 18:25:30 +0000 Subject: [PATCH 4/9] added test for RayClusterRedisPodsAssociationOptions Signed-off-by: owenowenisme --- .../ray/common/association_test.go | 37 +++++++++++++++++++ 1 file changed, 37 insertions(+) diff --git a/ray-operator/controllers/ray/common/association_test.go b/ray-operator/controllers/ray/common/association_test.go index af79b2e397..bd34a664b7 100644 --- a/ray-operator/controllers/ray/common/association_test.go +++ b/ray-operator/controllers/ray/common/association_test.go @@ -292,3 +292,40 @@ func TestGetRayClusterHeadPod(t *testing.T) { assert.Nil(t, err) assert.Equal(t, ret, headPod) } + +func TestRayClusterRedisPodsAssociationOptions(t *testing.T) { + // Create a new scheme + newScheme := runtime.NewScheme() + _ = rayv1.AddToScheme(newScheme) + _ = corev1.AddToScheme(newScheme) + + instance := &rayv1.RayCluster{ + ObjectMeta: metav1.ObjectMeta{ + Name: "raycluster-example", + Namespace: "default", + }, + } + + _ = &corev1.Pod{ + ObjectMeta: metav1.ObjectMeta{ + Name: "redis-pod", + Namespace: instance.ObjectMeta.Namespace, + Labels: map[string]string{ + utils.RayClusterLabelKey: instance.Name, + utils.RayNodeTypeLabelKey: string(rayv1.RedisCleanupNode), + }, + }, + } + + expected := []client.ListOption{ + client.InNamespace(instance.ObjectMeta.Namespace), + client.MatchingLabels(map[string]string{ + utils.RayClusterLabelKey: instance.Name, + utils.RayNodeTypeLabelKey: string(rayv1.RedisCleanupNode), + }), + } + + listOpts := RayClusterRedisPodsAssociationOptions(instance).ToListOptions() + + assert.Equal(t, expected, listOpts) +} From f730064bb18a2b78108070a2ac905567efaa5d4b Mon Sep 17 00:00:00 2001 From: owenowenisme Date: Wed, 15 Jan 2025 03:09:40 +0000 Subject: [PATCH 5/9] use specific existing associateOption Signed-off-by: owenowenisme --- ray-operator/controllers/ray/common/association.go | 11 ++++++++++- .../controllers/ray/common/association_test.go | 2 +- ray-operator/controllers/ray/raycluster_controller.go | 9 ++++----- 3 files changed, 15 insertions(+), 7 deletions(-) diff --git a/ray-operator/controllers/ray/common/association.go b/ray-operator/controllers/ray/common/association.go index c9cd9d83d5..63eefa94bc 100644 --- a/ray-operator/controllers/ray/common/association.go +++ b/ray-operator/controllers/ray/common/association.go @@ -100,7 +100,7 @@ func RayClusterWorkerPodsAssociationOptions(instance *rayv1.RayCluster) Associat } } -func RayClusterRedisPodsAssociationOptions(instance *rayv1.RayCluster) AssociationOptions { +func RayClusterRedisCleanupJobAssociationOptions(instance *rayv1.RayCluster) AssociationOptions { return AssociationOptions{ client.InNamespace(instance.Namespace), client.MatchingLabels{ @@ -194,3 +194,12 @@ func GetRayClusterHeadPod(ctx context.Context, reader client.Reader, instance *r } return &runtimePods.Items[0], nil } + +func RayClusterNetworkResourcesOptions(instance *rayv1.RayCluster) AssociationOptions { + return AssociationOptions{ + client.InNamespace(instance.Namespace), + client.MatchingLabels{ + utils.RayClusterLabelKey: instance.Name, + }, + } +} diff --git a/ray-operator/controllers/ray/common/association_test.go b/ray-operator/controllers/ray/common/association_test.go index bd34a664b7..c9c6cb6ca8 100644 --- a/ray-operator/controllers/ray/common/association_test.go +++ b/ray-operator/controllers/ray/common/association_test.go @@ -325,7 +325,7 @@ func TestRayClusterRedisPodsAssociationOptions(t *testing.T) { }), } - listOpts := RayClusterRedisPodsAssociationOptions(instance).ToListOptions() + listOpts := RayClusterRedisCleanupJobAssociationOptions(instance).ToListOptions() assert.Equal(t, expected, listOpts) } diff --git a/ray-operator/controllers/ray/raycluster_controller.go b/ray-operator/controllers/ray/raycluster_controller.go index 88c87323a7..5dc2b008d1 100644 --- a/ray-operator/controllers/ray/raycluster_controller.go +++ b/ray-operator/controllers/ray/raycluster_controller.go @@ -346,8 +346,7 @@ func (r *RayClusterReconciler) rayClusterReconcile(ctx context.Context, instance return ctrl.Result{RequeueAfter: 10 * time.Second}, nil } - // We can start the Redis cleanup process now because the head Pod has been terminated. - filterLabels := common.RayClusterRedisPodsAssociationOptions(instance).ToListOptions() + filterLabels := common.RayClusterRedisCleanupJobAssociationOptions(instance).ToListOptions() redisCleanupJobs := batchv1.JobList{} if err := r.List(ctx, &redisCleanupJobs, filterLabels...); err != nil { return ctrl.Result{RequeueAfter: DefaultRequeueDuration}, err @@ -542,7 +541,7 @@ func (r *RayClusterReconciler) reconcileIngress(ctx context.Context, instance *r func (r *RayClusterReconciler) reconcileRouteOpenShift(ctx context.Context, instance *rayv1.RayCluster) error { logger := ctrl.LoggerFrom(ctx) headRoutes := routev1.RouteList{} - filterLabels := common.RayClusterAllPodsAssociationOptions(instance).ToListOptions() + filterLabels := common.RayClusterNetworkResourcesOptions(instance).ToListOptions() if err := r.List(ctx, &headRoutes, filterLabels...); err != nil { return err } @@ -605,7 +604,7 @@ func (r *RayClusterReconciler) reconcileIngressKubernetes(ctx context.Context, i func (r *RayClusterReconciler) reconcileHeadService(ctx context.Context, instance *rayv1.RayCluster) error { logger := ctrl.LoggerFrom(ctx) services := corev1.ServiceList{} - filterLabels := common.RayClusterHeadPodsAssociationOptions(instance).ToListOptions() + filterLabels := common.RayClusterHeadServiceListOptions(instance) if err := r.List(ctx, &services, filterLabels...); err != nil { return err @@ -1510,7 +1509,7 @@ func (r *RayClusterReconciler) updateEndpoints(ctx context.Context, instance *ra // We assume we can find the right one by filtering Services with appropriate label selectors // and picking the first one. We may need to select by name in the future if the Service naming is stable. rayHeadSvc := corev1.ServiceList{} - filterLabels := common.RayClusterAllPodsAssociationOptions(instance).ToListOptions() + filterLabels := common.RayClusterHeadServiceListOptions(instance) if err := r.List(ctx, &rayHeadSvc, filterLabels...); err != nil { return err } From 3305569029254701c9aeaa5be305089dd83ef796 Mon Sep 17 00:00:00 2001 From: owenowenisme Date: Wed, 15 Jan 2025 18:04:58 +0000 Subject: [PATCH 6/9] added TestRayClusterNetworkResourcesOptions Signed-off-by: owenowenisme --- .../ray/common/association_test.go | 42 ++++++++++++++++++- 1 file changed, 40 insertions(+), 2 deletions(-) diff --git a/ray-operator/controllers/ray/common/association_test.go b/ray-operator/controllers/ray/common/association_test.go index c9c6cb6ca8..6f9b2c2c82 100644 --- a/ray-operator/controllers/ray/common/association_test.go +++ b/ray-operator/controllers/ray/common/association_test.go @@ -5,6 +5,7 @@ import ( "reflect" "testing" + routev1 "github.com/openshift/api/route/v1" "github.com/stretchr/testify/assert" corev1 "k8s.io/api/core/v1" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" @@ -324,8 +325,45 @@ func TestRayClusterRedisPodsAssociationOptions(t *testing.T) { utils.RayNodeTypeLabelKey: string(rayv1.RedisCleanupNode), }), } + for k, v := range expected { + t.Log(k, v) + } + result := RayClusterRedisCleanupJobAssociationOptions(instance).ToListOptions() + + assert.Equal(t, expected, result) +} + +func TestRayClusterNetworkResourcesOptions(t *testing.T) { + newScheme := runtime.NewScheme() + _ = rayv1.AddToScheme(newScheme) + _ = corev1.AddToScheme(newScheme) + _ = routev1.AddToScheme(newScheme) + instance := &rayv1.RayCluster{ + ObjectMeta: metav1.ObjectMeta{ + Name: "raycluster-example", + Namespace: "default", + Annotations: map[string]string{ + IngressClassAnnotationKey: "nginx", + }, + }, + } + _ = &routev1.Route{ + ObjectMeta: metav1.ObjectMeta{ + Name: utils.GenerateRouteName(instance.Name), + Namespace: instance.Namespace, + Labels: map[string]string{ + utils.RayClusterLabelKey: instance.Name, + }, + }, + } + expected := []client.ListOption{ + client.InNamespace(instance.ObjectMeta.Namespace), + client.MatchingLabels(map[string]string{ + utils.RayClusterLabelKey: instance.Name, + }), + } - listOpts := RayClusterRedisCleanupJobAssociationOptions(instance).ToListOptions() + result := RayClusterNetworkResourcesOptions(instance).ToListOptions() - assert.Equal(t, expected, listOpts) + assert.Equal(t, expected, result) } From 0304b1f761970963ce48f713a941c89dbaccd4b2 Mon Sep 17 00:00:00 2001 From: owenowenisme Date: Fri, 17 Jan 2025 02:37:35 +0000 Subject: [PATCH 7/9] rename redis cleanup pod Signed-off-by: owenowenisme --- ray-operator/controllers/ray/common/association_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/ray-operator/controllers/ray/common/association_test.go b/ray-operator/controllers/ray/common/association_test.go index 6f9b2c2c82..0f74c0ab89 100644 --- a/ray-operator/controllers/ray/common/association_test.go +++ b/ray-operator/controllers/ray/common/association_test.go @@ -294,7 +294,7 @@ func TestGetRayClusterHeadPod(t *testing.T) { assert.Equal(t, ret, headPod) } -func TestRayClusterRedisPodsAssociationOptions(t *testing.T) { +func TestRayClusterRedisCleanupJobAssociationOptions(t *testing.T) { // Create a new scheme newScheme := runtime.NewScheme() _ = rayv1.AddToScheme(newScheme) @@ -309,7 +309,7 @@ func TestRayClusterRedisPodsAssociationOptions(t *testing.T) { _ = &corev1.Pod{ ObjectMeta: metav1.ObjectMeta{ - Name: "redis-pod", + Name: "redis-cleanup", Namespace: instance.ObjectMeta.Namespace, Labels: map[string]string{ utils.RayClusterLabelKey: instance.Name, From e8ddab159974b9eb33016eb11a4d33406a64a675 Mon Sep 17 00:00:00 2001 From: owenowenisme Date: Fri, 17 Jan 2025 02:38:22 +0000 Subject: [PATCH 8/9] added utils.RayIDLabelKey in testService in TestReconcileHeadService Signed-off-by: owenowenisme --- ray-operator/controllers/ray/raycluster_controller_unit_test.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/ray-operator/controllers/ray/raycluster_controller_unit_test.go b/ray-operator/controllers/ray/raycluster_controller_unit_test.go index 6beb74f980..8b476bb2b2 100644 --- a/ray-operator/controllers/ray/raycluster_controller_unit_test.go +++ b/ray-operator/controllers/ray/raycluster_controller_unit_test.go @@ -1013,6 +1013,7 @@ func TestReconcileHeadService(t *testing.T) { Labels: map[string]string{ utils.RayClusterLabelKey: cluster.Name, utils.RayNodeTypeLabelKey: string(rayv1.HeadNode), + utils.RayIDLabelKey: utils.CheckLabel(utils.GenerateIdentifier(cluster.Name, rayv1.HeadNode)), }, }, } @@ -1026,6 +1027,7 @@ func TestReconcileHeadService(t *testing.T) { headServiceSelector := labels.SelectorFromSet(map[string]string{ utils.RayClusterLabelKey: cluster.Name, utils.RayNodeTypeLabelKey: string(rayv1.HeadNode), + utils.RayIDLabelKey: utils.CheckLabel(utils.GenerateIdentifier(cluster.Name, rayv1.HeadNode)), }) // Initialize RayCluster reconciler. From bb016053174a3d669d2f5a7d06985d0e9ffaefae Mon Sep 17 00:00:00 2001 From: owenowenisme Date: Sat, 18 Jan 2025 09:13:58 +0900 Subject: [PATCH 9/9] remove debugging log and use RayClusterNetworkResourcesOptions correctly Signed-off-by: owenowenisme --- ray-operator/controllers/ray/common/association_test.go | 3 --- ray-operator/controllers/ray/raycluster_controller.go | 2 +- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/ray-operator/controllers/ray/common/association_test.go b/ray-operator/controllers/ray/common/association_test.go index 0f74c0ab89..506107a72d 100644 --- a/ray-operator/controllers/ray/common/association_test.go +++ b/ray-operator/controllers/ray/common/association_test.go @@ -325,9 +325,6 @@ func TestRayClusterRedisCleanupJobAssociationOptions(t *testing.T) { utils.RayNodeTypeLabelKey: string(rayv1.RedisCleanupNode), }), } - for k, v := range expected { - t.Log(k, v) - } result := RayClusterRedisCleanupJobAssociationOptions(instance).ToListOptions() assert.Equal(t, expected, result) diff --git a/ray-operator/controllers/ray/raycluster_controller.go b/ray-operator/controllers/ray/raycluster_controller.go index 5dc2b008d1..3485d33c8f 100644 --- a/ray-operator/controllers/ray/raycluster_controller.go +++ b/ray-operator/controllers/ray/raycluster_controller.go @@ -572,7 +572,7 @@ func (r *RayClusterReconciler) reconcileRouteOpenShift(ctx context.Context, inst func (r *RayClusterReconciler) reconcileIngressKubernetes(ctx context.Context, instance *rayv1.RayCluster) error { logger := ctrl.LoggerFrom(ctx) headIngresses := networkingv1.IngressList{} - filterLabels := common.RayClusterAllPodsAssociationOptions(instance).ToListOptions() + filterLabels := common.RayClusterNetworkResourcesOptions(instance).ToListOptions() if err := r.List(ctx, &headIngresses, filterLabels...); err != nil { return err }