diff --git a/ray-operator/controllers/ray/common/association.go b/ray-operator/controllers/ray/common/association.go index 63eefa94bc4..c5b556fdc0a 100644 --- a/ray-operator/controllers/ray/common/association.go +++ b/ray-operator/controllers/ray/common/association.go @@ -40,13 +40,13 @@ func RayClusterHeadlessServiceListOptions(instance *rayv1.RayCluster) []client.L } } -func RayClusterHeadServiceListOptions(instance *rayv1.RayCluster) []client.ListOption { +func RayClusterHeadServiceListOptions(ctx context.Context, instance *rayv1.RayCluster) []client.ListOption { return []client.ListOption{ client.InNamespace(instance.Namespace), client.MatchingLabels(map[string]string{ utils.RayClusterLabelKey: instance.Name, utils.RayNodeTypeLabelKey: string(rayv1.HeadNode), - utils.RayIDLabelKey: utils.CheckLabel(utils.GenerateIdentifier(instance.Name, rayv1.HeadNode)), + utils.RayIDLabelKey: utils.CheckLabel(ctx, utils.GenerateIdentifier(instance.Name, rayv1.HeadNode)), }), } } diff --git a/ray-operator/controllers/ray/common/association_test.go b/ray-operator/controllers/ray/common/association_test.go index d54d06daabd..69c67e9518f 100644 --- a/ray-operator/controllers/ray/common/association_test.go +++ b/ray-operator/controllers/ray/common/association_test.go @@ -144,6 +144,8 @@ func TestRayClusterHeadlessServiceListOptions(t *testing.T) { } func TestRayClusterHeadServiceListOptions(t *testing.T) { + ctx := context.Background() + instance := rayv1.RayCluster{ ObjectMeta: metav1.ObjectMeta{ Name: "raycluster", @@ -151,7 +153,7 @@ func TestRayClusterHeadServiceListOptions(t *testing.T) { }, } - labels := HeadServiceLabels(instance) + labels := HeadServiceLabels(ctx, instance) delete(labels, utils.KubernetesCreatedByLabelKey) delete(labels, utils.KubernetesApplicationNameLabelKey) @@ -159,7 +161,7 @@ func TestRayClusterHeadServiceListOptions(t *testing.T) { client.InNamespace(instance.Namespace), client.MatchingLabels(labels), } - result := RayClusterHeadServiceListOptions(&instance) + result := RayClusterHeadServiceListOptions(ctx, &instance) if !reflect.DeepEqual(result, expected) { t.Errorf("Expected %v, got %v", expected, result) } diff --git a/ray-operator/controllers/ray/common/ingress.go b/ray-operator/controllers/ray/common/ingress.go index ef71c3f2701..1de5f784491 100644 --- a/ray-operator/controllers/ray/common/ingress.go +++ b/ray-operator/controllers/ray/common/ingress.go @@ -20,7 +20,7 @@ func BuildIngressForHeadService(ctx context.Context, cluster rayv1.RayCluster) ( labels := map[string]string{ utils.RayClusterLabelKey: cluster.Name, - utils.RayIDLabelKey: utils.CheckLabel(utils.GenerateIdentifier(cluster.Name, rayv1.HeadNode)), + utils.RayIDLabelKey: utils.CheckLabel(ctx, utils.GenerateIdentifier(cluster.Name, rayv1.HeadNode)), utils.KubernetesApplicationNameLabelKey: utils.ApplicationName, utils.KubernetesCreatedByLabelKey: utils.ComponentName, } diff --git a/ray-operator/controllers/ray/common/pod.go b/ray-operator/controllers/ray/common/pod.go index 98b89634fe1..10e3a73e683 100644 --- a/ray-operator/controllers/ray/common/pod.go +++ b/ray-operator/controllers/ray/common/pod.go @@ -172,7 +172,7 @@ func DefaultHeadPodTemplate(ctx context.Context, instance rayv1.RayCluster, head if podTemplate.Labels == nil { podTemplate.Labels = make(map[string]string) } - podTemplate.Labels = labelPod(rayv1.HeadNode, instance.Name, utils.RayNodeHeadGroupLabelValue, instance.Spec.HeadGroupSpec.Template.ObjectMeta.Labels) + podTemplate.Labels = labelPod(ctx, rayv1.HeadNode, instance.Name, utils.RayNodeHeadGroupLabelValue, instance.Spec.HeadGroupSpec.Template.ObjectMeta.Labels) headSpec.RayStartParams = setMissingRayStartParams(ctx, headSpec.RayStartParams, rayv1.HeadNode, headPort, "") initTemplateAnnotations(instance, &podTemplate) @@ -184,7 +184,7 @@ func DefaultHeadPodTemplate(ctx context.Context, instance rayv1.RayCluster, head headSpec.RayStartParams["no-monitor"] = "true" // set custom service account with proper roles bound. // utils.CheckName clips the name to match the behavior of reconcileAutoscalerServiceAccount - podTemplate.Spec.ServiceAccountName = utils.CheckName(utils.GetHeadGroupServiceAccountName(&instance)) + podTemplate.Spec.ServiceAccountName = utils.CheckName(ctx, utils.GetHeadGroupServiceAccountName(&instance)) // Use the same image as Ray head container by default. autoscalerImage := podTemplate.Spec.Containers[utils.RayContainerIndex].Image // inject autoscaler container into head pod @@ -311,7 +311,7 @@ func DefaultWorkerPodTemplate(ctx context.Context, instance rayv1.RayCluster, wo if podTemplate.Labels == nil { podTemplate.Labels = make(map[string]string) } - podTemplate.Labels = labelPod(rayv1.WorkerNode, instance.Name, workerSpec.GroupName, workerSpec.Template.ObjectMeta.Labels) + podTemplate.Labels = labelPod(ctx, rayv1.WorkerNode, instance.Name, workerSpec.GroupName, workerSpec.Template.ObjectMeta.Labels) workerSpec.RayStartParams = setMissingRayStartParams(ctx, workerSpec.RayStartParams, rayv1.WorkerNode, headPort, fqdnRayIP) initTemplateAnnotations(instance, &podTemplate) @@ -599,13 +599,13 @@ func getAutoscalerContainerIndex(pod corev1.Pod) (autoscalerContainerIndex int) // labelPod returns the labels for selecting the resources // belonging to the given RayCluster CR name. -func labelPod(rayNodeType rayv1.RayNodeType, rayClusterName string, groupName string, overrideLabels map[string]string) map[string]string { +func labelPod(ctx context.Context, rayNodeType rayv1.RayNodeType, rayClusterName string, groupName string, overrideLabels map[string]string) map[string]string { labels := map[string]string{ utils.RayNodeLabelKey: "yes", utils.RayClusterLabelKey: rayClusterName, utils.RayNodeTypeLabelKey: string(rayNodeType), utils.RayNodeGroupLabelKey: groupName, - utils.RayIDLabelKey: utils.CheckLabel(utils.GenerateIdentifier(rayClusterName, rayNodeType)), + utils.RayIDLabelKey: utils.CheckLabel(ctx, utils.GenerateIdentifier(rayClusterName, rayNodeType)), utils.KubernetesApplicationNameLabelKey: utils.ApplicationName, utils.KubernetesCreatedByLabelKey: utils.ComponentName, } diff --git a/ray-operator/controllers/ray/common/pod_test.go b/ray-operator/controllers/ray/common/pod_test.go index 39c2f3cf954..a720d85d9bc 100644 --- a/ray-operator/controllers/ray/common/pod_test.go +++ b/ray-operator/controllers/ray/common/pod_test.go @@ -1006,7 +1006,7 @@ func TestHeadPodTemplate_WithAutoscalingEnabled(t *testing.T) { // Repeat ServiceAccountName check with long cluster name. cluster.Name = longString(t) // 200 chars long podTemplateSpec = DefaultHeadPodTemplate(ctx, *cluster, cluster.Spec.HeadGroupSpec, podName, "6379") - assert.Equal(t, shortString(t), podTemplateSpec.Spec.ServiceAccountName) + assert.Equal(t, shortString(ctx, t), podTemplateSpec.Spec.ServiceAccountName) } func TestDefaultHeadPodTemplate_Autoscaling(t *testing.T) { diff --git a/ray-operator/controllers/ray/common/rbac.go b/ray-operator/controllers/ray/common/rbac.go index 2b2fba768d7..05b02394e51 100644 --- a/ray-operator/controllers/ray/common/rbac.go +++ b/ray-operator/controllers/ray/common/rbac.go @@ -1,6 +1,7 @@ package common import ( + "context" corev1 "k8s.io/api/core/v1" rbacv1 "k8s.io/api/rbac/v1" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" @@ -56,7 +57,7 @@ func BuildRole(cluster *rayv1.RayCluster) (*rbacv1.Role, error) { } // BuildRole -func BuildRoleBinding(cluster *rayv1.RayCluster) (*rbacv1.RoleBinding, error) { +func BuildRoleBinding(ctx context.Context, cluster *rayv1.RayCluster) (*rbacv1.RoleBinding, error) { serviceAccountName := utils.GetHeadGroupServiceAccountName(cluster) rb := &rbacv1.RoleBinding{ ObjectMeta: metav1.ObjectMeta{ @@ -72,7 +73,7 @@ func BuildRoleBinding(cluster *rayv1.RayCluster) (*rbacv1.RoleBinding, error) { { Kind: rbacv1.ServiceAccountKind, // Clip name for consistency with the function reconcileAutoscalerServiceAccount. - Name: utils.CheckName(serviceAccountName), + Name: utils.CheckName(ctx, serviceAccountName), Namespace: cluster.Namespace, }, }, @@ -80,7 +81,7 @@ func BuildRoleBinding(cluster *rayv1.RayCluster) (*rbacv1.RoleBinding, error) { APIGroup: rbacv1.GroupName, Kind: "Role", // Clip name for consistency with the function reconcileAutoscalerRole. - Name: utils.CheckName(cluster.Name), + Name: utils.CheckName(ctx, cluster.Name), }, } diff --git a/ray-operator/controllers/ray/common/rbac_test.go b/ray-operator/controllers/ray/common/rbac_test.go index 253ee6b318c..67d9992a0d7 100644 --- a/ray-operator/controllers/ray/common/rbac_test.go +++ b/ray-operator/controllers/ray/common/rbac_test.go @@ -1,6 +1,7 @@ package common import ( + "context" "testing" "github.com/stretchr/testify/assert" @@ -13,6 +14,8 @@ import ( // Test subject and role ref names in the function BuildRoleBinding. func TestBuildRoleBindingSubjectAndRoleRefName(t *testing.T) { + ctx := context.Background() + tests := []struct { name string input *rayv1.RayCluster @@ -70,15 +73,15 @@ func TestBuildRoleBindingSubjectAndRoleRefName(t *testing.T) { }, }, want: []string{ - shortString(t), // 50 chars long, truncated by utils.CheckName - shortString(t), + shortString(ctx, t), // 50 chars long, truncated by utils.CheckName + shortString(ctx, t), }, }, } for _, tc := range tests { t.Run(tc.name, func(t *testing.T) { - rb, err := BuildRoleBinding(tc.input) + rb, err := BuildRoleBinding(ctx, tc.input) require.NoError(t, err) got := []string{rb.Subjects[0].Name, rb.RoleRef.Name} assert.Equal(t, tc.want, got) diff --git a/ray-operator/controllers/ray/common/route.go b/ray-operator/controllers/ray/common/route.go index f41eb9e7dce..5979ffdaab3 100644 --- a/ray-operator/controllers/ray/common/route.go +++ b/ray-operator/controllers/ray/common/route.go @@ -1,6 +1,7 @@ package common import ( + "context" routev1 "github.com/openshift/api/route/v1" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "k8s.io/apimachinery/pkg/util/intstr" @@ -11,10 +12,10 @@ import ( // BuildRouteForHeadService Builds the Route (OpenShift) for head service dashboard. // This is used to expose dashboard and remote submit service apis or external traffic. -func BuildRouteForHeadService(cluster rayv1.RayCluster) (*routev1.Route, error) { +func BuildRouteForHeadService(ctx context.Context, cluster rayv1.RayCluster) (*routev1.Route, error) { labels := map[string]string{ utils.RayClusterLabelKey: cluster.Name, - utils.RayIDLabelKey: utils.CheckLabel(utils.GenerateIdentifier(cluster.Name, rayv1.HeadNode)), + utils.RayIDLabelKey: utils.CheckLabel(ctx, utils.GenerateIdentifier(cluster.Name, rayv1.HeadNode)), utils.KubernetesApplicationNameLabelKey: utils.ApplicationName, utils.KubernetesCreatedByLabelKey: utils.ComponentName, } diff --git a/ray-operator/controllers/ray/common/route_test.go b/ray-operator/controllers/ray/common/route_test.go index a7b33dacd55..8d0d68e8ef9 100644 --- a/ray-operator/controllers/ray/common/route_test.go +++ b/ray-operator/controllers/ray/common/route_test.go @@ -1,6 +1,7 @@ package common import ( + "context" "testing" "github.com/stretchr/testify/assert" @@ -41,7 +42,7 @@ var instanceWithRouteEnabled = &rayv1.RayCluster{ } func TestBuildRouteForHeadService(t *testing.T) { - route, err := BuildRouteForHeadService(*instanceWithRouteEnabled) + route, err := BuildRouteForHeadService(context.Background(), *instanceWithRouteEnabled) require.NoError(t, err) // Test name diff --git a/ray-operator/controllers/ray/common/service.go b/ray-operator/controllers/ray/common/service.go index 34189bea218..5984f736e8d 100644 --- a/ray-operator/controllers/ray/common/service.go +++ b/ray-operator/controllers/ray/common/service.go @@ -20,11 +20,11 @@ func getEnableRayHeadClusterIPService() bool { } // HeadServiceLabels returns the default labels for a cluster's head service. -func HeadServiceLabels(cluster rayv1.RayCluster) map[string]string { +func HeadServiceLabels(ctx context.Context, cluster rayv1.RayCluster) map[string]string { return map[string]string{ utils.RayClusterLabelKey: cluster.Name, utils.RayNodeTypeLabelKey: string(rayv1.HeadNode), - utils.RayIDLabelKey: utils.CheckLabel(utils.GenerateIdentifier(cluster.Name, rayv1.HeadNode)), + utils.RayIDLabelKey: utils.CheckLabel(ctx, utils.GenerateIdentifier(cluster.Name, rayv1.HeadNode)), utils.KubernetesApplicationNameLabelKey: utils.ApplicationName, utils.KubernetesCreatedByLabelKey: utils.ComponentName, } @@ -39,7 +39,7 @@ func BuildServiceForHeadPod(ctx context.Context, cluster rayv1.RayCluster, label labels = make(map[string]string) } - defaultLabels := HeadServiceLabels(cluster) + defaultLabels := HeadServiceLabels(ctx, cluster) // selector consists of *only* the keys in defaultLabels, updated with the values in labels if they exist selector := make(map[string]string) @@ -161,7 +161,7 @@ func BuildHeadServiceForRayService(ctx context.Context, rayService rayv1.RayServ utils.RayOriginatedFromCRNameLabelKey: rayService.Name, utils.RayOriginatedFromCRDLabelKey: utils.RayOriginatedFromCRDLabelValue(utils.RayServiceCRD), utils.RayNodeTypeLabelKey: string(rayv1.HeadNode), - utils.RayIDLabelKey: utils.CheckLabel(utils.GenerateIdentifier(rayService.Name, rayv1.HeadNode)), + utils.RayIDLabelKey: utils.CheckLabel(ctx, utils.GenerateIdentifier(rayService.Name, rayv1.HeadNode)), } return service, nil diff --git a/ray-operator/controllers/ray/common/service_test.go b/ray-operator/controllers/ray/common/service_test.go index bda8588a058..004c77ac290 100644 --- a/ray-operator/controllers/ray/common/service_test.go +++ b/ray-operator/controllers/ray/common/service_test.go @@ -289,6 +289,7 @@ func TestGetServicePortsWithMetricsPort(t *testing.T) { } func TestUserSpecifiedHeadService(t *testing.T) { + ctx := context.Background() // Use any RayCluster instance as a base for the test. testRayClusterWithHeadService := instanceWithWrongSvc.DeepCopy() @@ -332,7 +333,7 @@ func TestUserSpecifiedHeadService(t *testing.T) { // The selector field should only use the keys from the five default labels. The values should be updated with the values from the template labels. // The user-provided HeadService labels should be ignored for the purposes of the selector field. The user-provided Selector field should be ignored. - defaultLabels := HeadServiceLabels(*testRayClusterWithHeadService) + defaultLabels := HeadServiceLabels(ctx, *testRayClusterWithHeadService) // Make sure this test isn't spuriously passing. Check that RayClusterLabelKey is in the default labels. if _, ok := defaultLabels[utils.RayClusterLabelKey]; !ok { t.Errorf("utils.RayClusterLabelKey=%s should be in the default labels", utils.RayClusterLabelKey) diff --git a/ray-operator/controllers/ray/common/test_utils.go b/ray-operator/controllers/ray/common/test_utils.go index 0bdf383b97a..01084690e66 100644 --- a/ray-operator/controllers/ray/common/test_utils.go +++ b/ray-operator/controllers/ray/common/test_utils.go @@ -2,6 +2,7 @@ package common import ( "bytes" + "context" "testing" "github.com/stretchr/testify/assert" @@ -23,8 +24,8 @@ func longString(t *testing.T) string { // Clip the above string using utils.CheckName // to a string of length 50. -func shortString(t *testing.T) string { - result := utils.CheckName(longString(t)) +func shortString(ctx context.Context, t *testing.T) string { + result := utils.CheckName(ctx, longString(t)) // Confirm length. assert.Len(t, result, 50) return result diff --git a/ray-operator/controllers/ray/raycluster_controller.go b/ray-operator/controllers/ray/raycluster_controller.go index 92f10afe86d..9a8a5d6db7a 100644 --- a/ray-operator/controllers/ray/raycluster_controller.go +++ b/ray-operator/controllers/ray/raycluster_controller.go @@ -447,7 +447,7 @@ func (r *RayClusterReconciler) reconcileRouteOpenShift(ctx context.Context, inst } if len(headRoutes.Items) == 0 { - route, err := common.BuildRouteForHeadService(*instance) + route, err := common.BuildRouteForHeadService(ctx, *instance) if err != nil { return err } @@ -499,7 +499,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.RayClusterHeadServiceListOptions(instance) + filterLabels := common.RayClusterHeadServiceListOptions(ctx, instance) if err := r.List(ctx, &services, filterLabels...); err != nil { return err @@ -933,7 +933,7 @@ func (r *RayClusterReconciler) createHeadIngress(ctx context.Context, ingress *n logger := ctrl.LoggerFrom(ctx) // making sure the name is valid - ingress.Name = utils.CheckName(ingress.Name) + ingress.Name = utils.CheckName(ctx, ingress.Name) if err := controllerutil.SetControllerReference(instance, ingress, r.Scheme); err != nil { return err } @@ -1144,7 +1144,7 @@ func (r *RayClusterReconciler) buildRedisCleanupJob(ctx context.Context, instanc pod.Spec.RestartPolicy = corev1.RestartPolicyNever // Trim the job name to ensure it is within the 63-character limit. - jobName := utils.TrimJobName(fmt.Sprintf("%s-%s", instance.Name, "redis-cleanup")) + jobName := utils.TrimJobName(ctx, fmt.Sprintf("%s-%s", instance.Name, "redis-cleanup")) redisCleanupJob := batchv1.Job{ ObjectMeta: metav1.ObjectMeta{ @@ -1369,15 +1369,15 @@ func (r *RayClusterReconciler) calculateStatus(ctx context.Context, instance *ra func (r *RayClusterReconciler) getHeadServiceIPAndName(ctx context.Context, instance *rayv1.RayCluster) (string, string, error) { runtimeServices := corev1.ServiceList{} - if err := r.List(ctx, &runtimeServices, common.RayClusterHeadServiceListOptions(instance)...); err != nil { + if err := r.List(ctx, &runtimeServices, common.RayClusterHeadServiceListOptions(ctx, instance)...); err != nil { return "", "", err } if len(runtimeServices.Items) < 1 { - return "", "", fmt.Errorf("unable to find head service. cluster name %s, filter labels %v", instance.Name, common.RayClusterHeadServiceListOptions(instance)) + return "", "", fmt.Errorf("unable to find head service. cluster name %s, filter labels %v", instance.Name, common.RayClusterHeadServiceListOptions(ctx, instance)) } else if len(runtimeServices.Items) > 1 { - return "", "", fmt.Errorf("found multiple head services. cluster name %s, filter labels %v", instance.Name, common.RayClusterHeadServiceListOptions(instance)) + return "", "", fmt.Errorf("found multiple head services. cluster name %s, filter labels %v", instance.Name, common.RayClusterHeadServiceListOptions(ctx, instance)) } else if runtimeServices.Items[0].Spec.ClusterIP == "" { - return "", "", fmt.Errorf("head service IP is empty. cluster name %s, filter labels %v", instance.Name, common.RayClusterHeadServiceListOptions(instance)) + return "", "", fmt.Errorf("head service IP is empty. cluster name %s, filter labels %v", instance.Name, common.RayClusterHeadServiceListOptions(ctx, instance)) } else if runtimeServices.Items[0].Spec.ClusterIP == corev1.ClusterIPNone { // We return Head Pod IP if the Head service is headless. headPod, err := common.GetRayClusterHeadPod(ctx, r, instance) @@ -1399,7 +1399,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.RayClusterHeadServiceListOptions(instance) + filterLabels := common.RayClusterHeadServiceListOptions(ctx, instance) if err := r.List(ctx, &rayHeadSvc, filterLabels...); err != nil { return err } @@ -1488,7 +1488,7 @@ func (r *RayClusterReconciler) reconcileAutoscalerServiceAccount(ctx context.Con } // making sure the name is valid - serviceAccount.Name = utils.CheckName(serviceAccount.Name) + serviceAccount.Name = utils.CheckName(ctx, serviceAccount.Name) // Set controller reference if err := controllerutil.SetControllerReference(instance, serviceAccount, r.Scheme); err != nil { @@ -1531,7 +1531,7 @@ func (r *RayClusterReconciler) reconcileAutoscalerRole(ctx context.Context, inst } // making sure the name is valid - role.Name = utils.CheckName(role.Name) + role.Name = utils.CheckName(ctx, role.Name) // Set controller reference if err := controllerutil.SetControllerReference(instance, role, r.Scheme); err != nil { return err @@ -1567,13 +1567,13 @@ func (r *RayClusterReconciler) reconcileAutoscalerRoleBinding(ctx context.Contex } // Create role bindings for autoscaler if there's no existing one in the cluster. - roleBinding, err := common.BuildRoleBinding(instance) + roleBinding, err := common.BuildRoleBinding(ctx, instance) if err != nil { return err } // making sure the name is valid - roleBinding.Name = utils.CheckName(roleBinding.Name) + roleBinding.Name = utils.CheckName(ctx, roleBinding.Name) // Set controller reference if err := controllerutil.SetControllerReference(instance, roleBinding, r.Scheme); err != nil { return err diff --git a/ray-operator/controllers/ray/raycluster_controller_unit_test.go b/ray-operator/controllers/ray/raycluster_controller_unit_test.go index 89f41df4e21..04619665d5c 100644 --- a/ray-operator/controllers/ray/raycluster_controller_unit_test.go +++ b/ray-operator/controllers/ray/raycluster_controller_unit_test.go @@ -1010,6 +1010,8 @@ func TestReconcile_PodEvicted_DiffLess0_OK(t *testing.T) { func TestReconcileHeadService(t *testing.T) { setupTest(t) + ctx := context.Background() + // Create a new scheme with CRDs, Pod, Service schemes. newScheme := runtime.NewScheme() _ = rayv1.AddToScheme(newScheme) @@ -1024,7 +1026,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)), + utils.RayIDLabelKey: utils.CheckLabel(ctx, utils.GenerateIdentifier(cluster.Name, rayv1.HeadNode)), }, }, } @@ -1034,11 +1036,10 @@ func TestReconcileHeadService(t *testing.T) { // Initialize a fake client with newScheme and runtimeObjects. runtimeObjects := []runtime.Object{cluster} fakeClient := clientFake.NewClientBuilder().WithScheme(newScheme).WithRuntimeObjects(runtimeObjects...).Build() - ctx := context.TODO() 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)), + utils.RayIDLabelKey: utils.CheckLabel(ctx, utils.GenerateIdentifier(cluster.Name, rayv1.HeadNode)), }) // Initialize RayCluster reconciler. @@ -1415,6 +1416,8 @@ func TestGetHeadPodIPAndNameFromGetRayClusterHeadPod(t *testing.T) { func TestGetHeadServiceIPAndName(t *testing.T) { setupTest(t) + ctx := context.Background() + headServiceIP := "1.2.3.4" headService, err := common.BuildServiceForHeadPod(context.Background(), *testRayCluster, nil, nil) if err != nil { @@ -1429,7 +1432,7 @@ func TestGetHeadServiceIPAndName(t *testing.T) { ObjectMeta: metav1.ObjectMeta{ Name: "unexpectedExtraHeadService", Namespace: namespaceStr, - Labels: common.HeadServiceLabels(*testRayCluster), + Labels: common.HeadServiceLabels(ctx, *testRayCluster), }, } diff --git a/ray-operator/controllers/ray/suite_helpers_test.go b/ray-operator/controllers/ray/suite_helpers_test.go index f463ef559d9..1c6d15d26d8 100644 --- a/ray-operator/controllers/ray/suite_helpers_test.go +++ b/ray-operator/controllers/ray/suite_helpers_test.go @@ -2,7 +2,6 @@ package ray import ( "context" - "fmt" "log" "reflect" "time" @@ -232,7 +231,6 @@ func checkServeApplicationExists(ctx context.Context, rayService *rayv1.RayServi return false, err } for appName := range rayService.Status.ActiveServiceStatus.Applications { - fmt.Println("checkServeApplicationExists: appName", appName) if appName == serveAppName { return true, nil } diff --git a/ray-operator/controllers/ray/utils/util.go b/ray-operator/controllers/ray/utils/util.go index 495e90a8270..3d5e08bf1e6 100644 --- a/ray-operator/controllers/ray/utils/util.go +++ b/ray-operator/controllers/ray/utils/util.go @@ -158,7 +158,7 @@ func CheckRouteName(ctx context.Context, s string, n string) string { } // Pass through CheckName for remaining string validations - return CheckName(s) + return CheckName(ctx, s) } // PodName returns the value that should be used for a Pod's Name or GenerateName @@ -181,13 +181,14 @@ func PodName(prefix string, nodeType rayv1.RayNodeType, isGenerateName bool) str } // CheckName makes sure the name does not start with a numeric value and the total length is < 63 char -func CheckName(s string) string { +func CheckName(ctx context.Context, s string) string { + log := ctrl.LoggerFrom(ctx) maxLength := 50 // 63 - (max(8,6) + 5 ) // 6 to 8 char are consumed at the end with "-head-" or -worker- + 5 generated. if len(s) > maxLength { // shorten the name offset := int(math.Abs(float64(maxLength) - float64(len(s)))) - fmt.Printf("pod name is too long: len = %v, we will shorten it by offset = %v", len(s), offset) + log.Info("pod name is too long, we will shorten it by offset", "nameLength", len(s), "offset", offset) s = s[offset:] } @@ -198,7 +199,6 @@ func CheckName(s string) string { // cannot start with a punctuation if unicode.IsPunct(rune(s[0])) { - fmt.Println(s) s = "r" + s[1:] } @@ -206,24 +206,24 @@ func CheckName(s string) string { } // TrimJobName uses CheckLabel to trim Kubernetes job to constrains -func TrimJobName(jobName string) string { - return CheckLabel(jobName) +func TrimJobName(ctx context.Context, jobName string) string { + return CheckLabel(ctx, jobName) } // CheckLabel makes sure the label value does not start with a punctuation and the total length is < 63 char -func CheckLabel(s string) string { +func CheckLabel(ctx context.Context, s string) string { + log := ctrl.LoggerFrom(ctx) maxLength := 63 if len(s) > maxLength { // shorten the name offset := int(math.Abs(float64(maxLength) - float64(len(s)))) - fmt.Printf("label value is too long: len = %v, we will shorten it by offset = %v\n", len(s), offset) + log.Info("label value is too long, we will shorten it by offset", "labelLength", len(s), "offset", offset) s = s[offset:] } // cannot start with a punctuation if unicode.IsPunct(rune(s[0])) { - fmt.Println(s) s = "r" + s[1:] } diff --git a/ray-operator/controllers/ray/utils/util_test.go b/ray-operator/controllers/ray/utils/util_test.go index de14c602d02..216ea9c0fb2 100644 --- a/ray-operator/controllers/ray/utils/util_test.go +++ b/ray-operator/controllers/ray/utils/util_test.go @@ -156,6 +156,8 @@ func TestPodName(t *testing.T) { } func TestCheckName(t *testing.T) { + ctx := context.Background() + tests := []struct { name string input string @@ -180,7 +182,7 @@ func TestCheckName(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { - str := CheckName(test.input) + str := CheckName(ctx, test.input) if str != test.expected { t.Logf("expected: %q", test.expected) t.Logf("actual: %q", str)