From df038633d2c0ca397427d78578d4f099a1031cb7 Mon Sep 17 00:00:00 2001 From: nadongjun Date: Thu, 14 Nov 2024 06:47:33 +0900 Subject: [PATCH] [Logging] Remove duplicate info in CR logs (#2531) --- .../controllers/ray/raycluster_controller.go | 26 ++++++------- .../ray/raycluster_controller_unit_test.go | 14 +++---- .../controllers/ray/rayjob_controller.go | 38 +++++++++---------- .../controllers/ray/rayservice_controller.go | 11 +++--- 4 files changed, 41 insertions(+), 48 deletions(-) diff --git a/ray-operator/controllers/ray/raycluster_controller.go b/ray-operator/controllers/ray/raycluster_controller.go index b16683bb37..436474c2ac 100644 --- a/ray-operator/controllers/ray/raycluster_controller.go +++ b/ray-operator/controllers/ray/raycluster_controller.go @@ -179,7 +179,7 @@ func (r *RayClusterReconciler) Reconcile(ctx context.Context, request ctrl.Reque // Try to fetch the RayCluster instance instance := &rayv1.RayCluster{} if err = r.Get(ctx, request.NamespacedName, instance); err == nil { - return r.rayClusterReconcile(ctx, request, instance) + return r.rayClusterReconcile(ctx, instance) } // No match found @@ -219,7 +219,7 @@ func (r *RayClusterReconciler) validateRayClusterStatus(instance *rayv1.RayClust return nil } -func (r *RayClusterReconciler) rayClusterReconcile(ctx context.Context, request ctrl.Request, instance *rayv1.RayCluster) (ctrl.Result, error) { +func (r *RayClusterReconciler) rayClusterReconcile(ctx context.Context, instance *rayv1.RayCluster) (ctrl.Result, error) { var reconcileErr error logger := ctrl.LoggerFrom(ctx) @@ -247,7 +247,7 @@ func (r *RayClusterReconciler) rayClusterReconcile(ctx context.Context, request "finalizer", utils.GCSFaultToleranceRedisCleanupFinalizer) controllerutil.AddFinalizer(instance, utils.GCSFaultToleranceRedisCleanupFinalizer) if err := r.Update(ctx, instance); err != nil { - err = fmt.Errorf("Failed to add the finalizer %s to the RayCluster: %w", utils.GCSFaultToleranceRedisCleanupFinalizer, err) + err = fmt.Errorf("failed to add the finalizer %s to the RayCluster: %w", utils.GCSFaultToleranceRedisCleanupFinalizer, err) return ctrl.Result{RequeueAfter: DefaultRequeueDuration}, err } // Only start the RayCluster reconciliation after the finalizer is added. @@ -256,7 +256,6 @@ func (r *RayClusterReconciler) rayClusterReconcile(ctx context.Context, request } else { logger.Info( "The RayCluster with GCS enabled is being deleted. Start to handle the Redis cleanup finalizer.", - "rayClusterName", instance.Name, "redisCleanupFinalizer", utils.GCSFaultToleranceRedisCleanupFinalizer, "deletionTimestamp", instance.ObjectMeta.DeletionTimestamp, ) @@ -337,7 +336,7 @@ func (r *RayClusterReconciler) rayClusterReconcile(ctx context.Context, request } if instance.DeletionTimestamp != nil && !instance.DeletionTimestamp.IsZero() { - logger.Info("RayCluster is being deleted, just ignore", "cluster name", request.Name) + logger.Info("RayCluster is being deleted, just ignore") return ctrl.Result{}, nil } @@ -365,7 +364,7 @@ func (r *RayClusterReconciler) rayClusterReconcile(ctx context.Context, request var updateErr error var inconsistent bool if calculateErr != nil { - logger.Info("Got error when calculating new status", "cluster name", request.Name, "error", calculateErr) + logger.Info("Got error when calculating new status", "error", calculateErr) } else { inconsistent, updateErr = r.updateRayClusterStatus(ctx, originalRayClusterInstance, newInstance) } @@ -395,11 +394,10 @@ func (r *RayClusterReconciler) rayClusterReconcile(ctx context.Context, request "Environment variable is not set, using default value of seconds", "environmentVariable", utils.RAYCLUSTER_DEFAULT_REQUEUE_SECONDS_ENV, "defaultValue", utils.RAYCLUSTER_DEFAULT_REQUEUE_SECONDS, - "clusterName", request.Name, ) requeueAfterSeconds = utils.RAYCLUSTER_DEFAULT_REQUEUE_SECONDS } - logger.Info("Unconditional requeue after", "cluster name", request.Name, "seconds", requeueAfterSeconds) + logger.Info("Unconditional requeue after", "seconds", requeueAfterSeconds) return ctrl.Result{RequeueAfter: time.Duration(requeueAfterSeconds) * time.Second}, nil } @@ -575,7 +573,7 @@ func (r *RayClusterReconciler) reconcileHeadService(ctx context.Context, instanc // TODO (kevin85421): Provide a detailed and actionable error message. For example, which port is missing? if len(headSvc.Spec.Ports) == 0 { logger.Info("Ray head service does not have any ports set up.", "serviceSpecification", headSvc.Spec) - return fmt.Errorf("Ray head service does not have any ports set up. Service specification: %v", headSvc.Spec) + return fmt.Errorf("ray head service does not have any ports set up. Service specification: %v", headSvc.Spec) } if err != nil { @@ -725,7 +723,7 @@ func (r *RayClusterReconciler) reconcilePods(ctx context.Context, instance *rayv } } else if len(headPods.Items) == 0 { // Create head Pod if it does not exist. - logger.Info("reconcilePods: Found 0 head Pods; creating a head Pod for the RayCluster.", "rayClusterName", instance.Name) + logger.Info("reconcilePods: Found 0 head Pods; creating a head Pod for the RayCluster.") common.CreatedClustersCounterInc(instance.Namespace) if err := r.createHeadPod(ctx, *instance); err != nil { common.FailedClustersCounterInc(instance.Namespace) @@ -733,7 +731,7 @@ func (r *RayClusterReconciler) reconcilePods(ctx context.Context, instance *rayv } common.SuccessfulClustersCounterInc(instance.Namespace) } else if len(headPods.Items) > 1 { - logger.Info("reconcilePods: Found more than one head Pods; deleting extra head Pods.", "nHeadPods", len(headPods.Items), "rayClusterName", instance.Name) + logger.Info("reconcilePods: Found more than one head Pods; deleting extra head Pods.", "nHeadPods", len(headPods.Items)) // TODO (kevin85421): In-place update may not be a good idea. itemLength := len(headPods.Items) for index := 0; index < itemLength; index++ { @@ -787,7 +785,7 @@ func (r *RayClusterReconciler) reconcilePods(ctx context.Context, instance *rayv // If we delete unhealthy Pods, we will not create new Pods in this reconciliation. if numDeletedUnhealthyWorkerPods > 0 { - return fmt.Errorf("Delete %d unhealthy worker Pods", numDeletedUnhealthyWorkerPods) + return fmt.Errorf("delete %d unhealthy worker Pods", numDeletedUnhealthyWorkerPods) } // Always remove the specified WorkersToDelete - regardless of the value of Replicas. @@ -877,7 +875,7 @@ func (r *RayClusterReconciler) reconcilePods(ctx context.Context, instance *rayv r.Recorder.Eventf(instance, corev1.EventTypeNormal, string(utils.DeletedWorkerPod), "Deleted Pod %s/%s", randomPodToDelete.Namespace, randomPodToDelete.Name) } } else { - logger.Info("Random Pod deletion is disabled for the cluster. The only decision-maker for Pod deletions is Autoscaler.", "rayClusterName", instance.Name) + logger.Info("Random Pod deletion is disabled for the cluster. The only decision-maker for Pod deletions is Autoscaler.") } } } @@ -1449,7 +1447,7 @@ func (r *RayClusterReconciler) updateEndpoints(ctx context.Context, instance *ra } } } else { - logger.Info("updateEndpoints: Unable to find a Service for this RayCluster. Not adding RayCluster status.endpoints", "rayClusterName", instance.Name, "serviceSelectors", filterLabels) + logger.Info("updateEndpoints: Unable to find a Service for this RayCluster. Not adding RayCluster status.endpoints", "serviceSelectors", filterLabels) } return nil diff --git a/ray-operator/controllers/ray/raycluster_controller_unit_test.go b/ray-operator/controllers/ray/raycluster_controller_unit_test.go index 4897f48290..e05cc0c29a 100644 --- a/ray-operator/controllers/ray/raycluster_controller_unit_test.go +++ b/ray-operator/controllers/ray/raycluster_controller_unit_test.go @@ -47,7 +47,6 @@ import ( "k8s.io/client-go/tools/record" "k8s.io/utils/ptr" - ctrl "sigs.k8s.io/controller-runtime" "sigs.k8s.io/controller-runtime/pkg/client" clientFake "sigs.k8s.io/controller-runtime/pkg/client/fake" "sigs.k8s.io/controller-runtime/pkg/client/interceptor" @@ -2390,8 +2389,7 @@ func Test_RedisCleanupFeatureFlag(t *testing.T) { assert.Equal(t, 1, len(rayClusterList.Items)) assert.Equal(t, 0, len(rayClusterList.Items[0].Finalizers)) - request := ctrl.Request{NamespacedName: types.NamespacedName{Name: cluster.Name, Namespace: cluster.Namespace}} - _, err = testRayClusterReconciler.rayClusterReconcile(ctx, request, cluster) + _, err = testRayClusterReconciler.rayClusterReconcile(ctx, cluster) if tc.enableGCSFTRedisCleanup == "false" { assert.Nil(t, err) podList := corev1.PodList{} @@ -2419,7 +2417,7 @@ func Test_RedisCleanupFeatureFlag(t *testing.T) { assert.Equal(t, 0, len(podList.Items)) // Reconcile the RayCluster again. The controller should create Pods. - _, err = testRayClusterReconciler.rayClusterReconcile(ctx, request, cluster) + _, err = testRayClusterReconciler.rayClusterReconcile(ctx, cluster) assert.Nil(t, err) err = fakeClient.List(ctx, &podList, client.InNamespace(namespaceStr)) @@ -2498,8 +2496,7 @@ func TestEvents_RedisCleanup(t *testing.T) { Scheme: newScheme, } - request := ctrl.Request{NamespacedName: types.NamespacedName{Name: cluster.Name, Namespace: cluster.Namespace}} - _, err := testRayClusterReconciler.rayClusterReconcile(ctx, request, cluster) + _, err := testRayClusterReconciler.rayClusterReconcile(ctx, cluster) assert.ErrorIs(t, err, tc.errInjected) var foundEvent bool @@ -2642,8 +2639,7 @@ func Test_RedisCleanup(t *testing.T) { assert.Nil(t, err, "Fail to get Job list") assert.Equal(t, 0, len(jobList.Items)) - request := ctrl.Request{NamespacedName: types.NamespacedName{Name: cluster.Name, Namespace: cluster.Namespace}} - _, err = testRayClusterReconciler.rayClusterReconcile(ctx, request, cluster) + _, err = testRayClusterReconciler.rayClusterReconcile(ctx, cluster) assert.Nil(t, err) // Check Job @@ -2670,7 +2666,7 @@ func Test_RedisCleanup(t *testing.T) { // Reconcile the RayCluster again. The controller should remove the finalizer and the RayCluster will be deleted. // See https://github.com/kubernetes-sigs/controller-runtime/blob/release-0.11/pkg/client/fake/client.go#L308-L310 for more details. - _, err = testRayClusterReconciler.rayClusterReconcile(ctx, request, cluster) + _, err = testRayClusterReconciler.rayClusterReconcile(ctx, cluster) assert.Nil(t, err, "Fail to reconcile RayCluster") err = fakeClient.List(ctx, &rayClusterList, client.InNamespace(namespaceStr)) assert.Nil(t, err, "Fail to get RayCluster list") diff --git a/ray-operator/controllers/ray/rayjob_controller.go b/ray-operator/controllers/ray/rayjob_controller.go index 0c54c9da14..f5b387b8fe 100644 --- a/ray-operator/controllers/ray/rayjob_controller.go +++ b/ray-operator/controllers/ray/rayjob_controller.go @@ -84,7 +84,7 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) if err := r.Get(ctx, request.NamespacedName, rayJobInstance); err != nil { if errors.IsNotFound(err) { // Request object not found, could have been deleted after reconcile request. Stop reconciliation. - logger.Info("RayJob resource not found. Ignoring since object must be deleted", "name", request.NamespacedName) + logger.Info("RayJob resource not found. Ignoring since object must be deleted") return ctrl.Result{}, nil } // Error reading the object - requeue the request. @@ -141,7 +141,7 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) // Please do NOT modify `originalRayJobInstance` in the following code. originalRayJobInstance := rayJobInstance.DeepCopy() - logger.Info("RayJob", "name", rayJobInstance.Name, "namespace", rayJobInstance.Namespace, "JobStatus", rayJobInstance.Status.JobStatus, "JobDeploymentStatus", rayJobInstance.Status.JobDeploymentStatus, "SubmissionMode", rayJobInstance.Spec.SubmissionMode) + logger.Info("RayJob", "JobStatus", rayJobInstance.Status.JobStatus, "JobDeploymentStatus", rayJobInstance.Status.JobDeploymentStatus, "SubmissionMode", rayJobInstance.Spec.SubmissionMode) switch rayJobInstance.Status.JobDeploymentStatus { case rayv1.JobDeploymentStatusNew: if !controllerutil.ContainsFinalizer(rayJobInstance, utils.RayJobStopJobFinalizer) { @@ -154,7 +154,7 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) } // Set `Status.JobDeploymentStatus` to `JobDeploymentStatusInitializing`, and initialize `Status.JobId` // and `Status.RayClusterName` prior to avoid duplicate job submissions and cluster creations. - logger.Info("JobDeploymentStatusNew", "RayJob", rayJobInstance.Name) + logger.Info("JobDeploymentStatusNew") if err = r.initRayJobStatusIfNeed(ctx, rayJobInstance); err != nil { return ctrl.Result{RequeueAfter: RayJobDefaultRequeueDuration}, err } @@ -199,7 +199,7 @@ func (r *RayJobReconciler) Reconcile(ctx context.Context, request ctrl.Request) } logger.Info("Both RayCluster and the submitter K8s Job are created. Transition the status from `Initializing` to `Running`.", "SubmissionMode", rayJobInstance.Spec.SubmissionMode, - "RayJob", rayJobInstance.Name, "RayCluster", rayJobInstance.Status.RayClusterName) + "RayCluster", rayJobInstance.Status.RayClusterName) rayJobInstance.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusRunning case rayv1.JobDeploymentStatusWaiting: // Try to get the Ray job id from rayJob.Spec.JobId @@ -447,7 +447,7 @@ func (r *RayJobReconciler) createK8sJobIfNeed(ctx context.Context, rayJobInstanc return err } - logger.Info("The submitter Kubernetes Job for RayJob already exists", "RayJob", rayJobInstance.Name, "Kubernetes Job", job.Name) + logger.Info("The submitter Kubernetes Job for RayJob already exists", "Kubernetes Job", job.Name) return nil } @@ -536,7 +536,7 @@ func (r *RayJobReconciler) createNewK8sJob(ctx context.Context, rayJobInstance * r.Recorder.Eventf(rayJobInstance, corev1.EventTypeWarning, string(utils.FailedToCreateRayJobSubmitter), "Failed to create new Kubernetes Job %s/%s: %v", job.Namespace, job.Name, err) return err } - logger.Info("Created submitter Kubernetes Job for RayJob", "RayJob", rayJobInstance.Name, "Kubernetes Job", job.Name) + logger.Info("Created submitter Kubernetes Job for RayJob", "Kubernetes Job", job.Name) r.Recorder.Eventf(rayJobInstance, corev1.EventTypeNormal, string(utils.CreatedRayJobSubmitter), "Created Kubernetes Job %s/%s", job.Namespace, job.Name) return nil } @@ -557,19 +557,19 @@ func (r *RayJobReconciler) deleteSubmitterJob(ctx context.Context, rayJobInstanc if err := r.Client.Get(ctx, namespacedName, job); err != nil { if errors.IsNotFound(err) { isJobDeleted = true - logger.Info("The submitter Kubernetes Job has been already deleted", "RayJob", rayJobInstance.Name, "Kubernetes Job", job.Name) + logger.Info("The submitter Kubernetes Job has been already deleted", "Kubernetes Job", job.Name) } else { return false, err } } else { if !job.DeletionTimestamp.IsZero() { - logger.Info("The deletion of submitter Kubernetes Job for RayJob is ongoing.", "RayJob", rayJobInstance.Name, "Submitter K8s Job", job.Name) + logger.Info("The deletion of submitter Kubernetes Job for RayJob is ongoing.", "Submitter K8s Job", job.Name) } else { if err := r.Client.Delete(ctx, job, client.PropagationPolicy(metav1.DeletePropagationBackground)); err != nil { r.Recorder.Eventf(rayJobInstance, corev1.EventTypeWarning, string(utils.FailedToDeleteRayJobSubmitter), "Failed to delete submitter K8s Job %s/%s: %v", job.Namespace, job.Name, err) return false, err } - logger.Info("The associated submitter Kubernetes Job for RayJob is deleted", "RayJob", rayJobInstance.Name, "Submitter K8s Job", job.Name) + logger.Info("The associated submitter Kubernetes Job for RayJob is deleted", "Submitter K8s Job", job.Name) r.Recorder.Eventf(rayJobInstance, corev1.EventTypeNormal, string(utils.DeletedRayJobSubmitter), "Deleted submitter K8s Job %s/%s", job.Namespace, job.Name) } } @@ -590,19 +590,19 @@ func (r *RayJobReconciler) deleteClusterResources(ctx context.Context, rayJobIns // If the cluster is not found, it means the cluster has been already deleted. // Don't return error to make this function idempotent. isClusterDeleted = true - logger.Info("The associated RayCluster for RayJob has been already deleted and it can not be found", "RayCluster", clusterIdentifier, "RayJob", rayJobInstance.Name) + logger.Info("The associated RayCluster for RayJob has been already deleted and it can not be found", "RayCluster", clusterIdentifier) } else { return false, err } } else { if !cluster.DeletionTimestamp.IsZero() { - logger.Info("The deletion of the associated RayCluster for RayJob is ongoing.", "RayJob", rayJobInstance.Name, "RayCluster", cluster.Name) + logger.Info("The deletion of the associated RayCluster for RayJob is ongoing.", "RayCluster", cluster.Name) } else { if err := r.Delete(ctx, &cluster); err != nil { r.Recorder.Eventf(rayJobInstance, corev1.EventTypeWarning, string(utils.FailedToDeleteRayCluster), "Failed to delete cluster %s/%s: %v", cluster.Namespace, cluster.Name, err) return false, err } - logger.Info("The associated RayCluster for RayJob is deleted", "RayCluster", clusterIdentifier, "RayJob", rayJobInstance.Name) + logger.Info("The associated RayCluster for RayJob is deleted", "RayCluster", clusterIdentifier) r.Recorder.Eventf(rayJobInstance, corev1.EventTypeNormal, string(utils.DeletedRayCluster), "Deleted cluster %s/%s", cluster.Namespace, cluster.Name) } } @@ -639,7 +639,7 @@ func (r *RayJobReconciler) initRayJobStatusIfNeed(ctx context.Context, rayJob *r logger := ctrl.LoggerFrom(ctx) shouldUpdateStatus := rayJob.Status.JobId == "" || rayJob.Status.RayClusterName == "" || rayJob.Status.JobStatus == "" // Please don't update `shouldUpdateStatus` below. - logger.Info("initRayJobStatusIfNeed", "shouldUpdateStatus", shouldUpdateStatus, "RayJob", rayJob.Name, "jobId", rayJob.Status.JobId, "rayClusterName", rayJob.Status.RayClusterName, "jobStatus", rayJob.Status.JobStatus) + logger.Info("initRayJobStatusIfNeed", "shouldUpdateStatus", shouldUpdateStatus, "jobId", rayJob.Status.JobId, "rayClusterName", rayJob.Status.RayClusterName, "jobStatus", rayJob.Status.JobStatus) if !shouldUpdateStatus { return nil } @@ -706,7 +706,7 @@ func (r *RayJobReconciler) getOrCreateRayClusterInstance(ctx context.Context, ra rayClusterInstance := &rayv1.RayCluster{} if err := r.Get(ctx, rayClusterNamespacedName, rayClusterInstance); err != nil { if errors.IsNotFound(err) { - logger.Info("RayCluster not found", "RayJob", rayJobInstance.Name, "RayCluster", rayClusterNamespacedName) + logger.Info("RayCluster not found", "RayCluster", rayClusterNamespacedName) if len(rayJobInstance.Spec.ClusterSelector) != 0 { err := fmt.Errorf("we have choosed the cluster selector mode, failed to find the cluster named %v, err: %w", rayClusterNamespacedName.Name, err) return nil, err @@ -726,12 +726,12 @@ func (r *RayJobReconciler) getOrCreateRayClusterInstance(ctx context.Context, ra return nil, err } } - logger.Info("Found the associated RayCluster for RayJob", "RayJob", rayJobInstance.Name, "RayCluster", rayClusterNamespacedName) + logger.Info("Found the associated RayCluster for RayJob", "RayCluster", rayClusterNamespacedName) // Verify that RayJob is not in cluster selector mode first to avoid nil pointer dereference error during spec comparison. // This is checked by ensuring len(rayJobInstance.Spec.ClusterSelector) equals 0. if len(rayJobInstance.Spec.ClusterSelector) == 0 && !utils.CompareJsonStruct(rayClusterInstance.Spec, *rayJobInstance.Spec.RayClusterSpec) { - logger.Info("Disregard changes in RayClusterSpec of RayJob", "RayJob", rayJobInstance.Name) + logger.Info("Disregard changes in RayClusterSpec of RayJob") } return rayClusterInstance, nil @@ -773,10 +773,10 @@ func (r *RayJobReconciler) updateStatusToSuspendingIfNeeded(ctx context.Context, rayv1.JobDeploymentStatusInitializing: {}, } if _, ok := validTransitions[rayJob.Status.JobDeploymentStatus]; !ok { - logger.Info("The current status is not allowed to transition to `Suspending`", "RayJob", rayJob.Name, "JobDeploymentStatus", rayJob.Status.JobDeploymentStatus) + logger.Info("The current status is not allowed to transition to `Suspending`", "JobDeploymentStatus", rayJob.Status.JobDeploymentStatus) return false } - logger.Info("Try to transition the status to `Suspending`", "oldStatus", rayJob.Status.JobDeploymentStatus, "RayJob", rayJob.Name) + logger.Info("Try to transition the status to `Suspending`", "oldStatus", rayJob.Status.JobDeploymentStatus) rayJob.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusSuspending return true } @@ -785,7 +785,7 @@ func (r *RayJobReconciler) checkK8sJobAndUpdateStatusIfNeeded(ctx context.Contex logger := ctrl.LoggerFrom(ctx) for _, cond := range job.Status.Conditions { if cond.Type == batchv1.JobFailed && cond.Status == corev1.ConditionTrue { - logger.Info("The submitter Kubernetes Job has failed. Attempting to transition the status to `Failed`.", "RayJob", rayJob.Name, "Submitter K8s Job", job.Name, "Reason", cond.Reason, "Message", cond.Message) + logger.Info("The submitter Kubernetes Job has failed. Attempting to transition the status to `Failed`.", "Submitter K8s Job", job.Name, "Reason", cond.Reason, "Message", cond.Message) rayJob.Status.JobDeploymentStatus = rayv1.JobDeploymentStatusFailed // The submitter Job needs to wait for the user code to finish and retrieve its logs. // Therefore, a failed Submitter Job indicates that the submission itself has failed or the user code has thrown an error. diff --git a/ray-operator/controllers/ray/rayservice_controller.go b/ray-operator/controllers/ray/rayservice_controller.go index fe5bda8230..2d52753583 100644 --- a/ray-operator/controllers/ray/rayservice_controller.go +++ b/ray-operator/controllers/ray/rayservice_controller.go @@ -635,7 +635,7 @@ func (r *RayServiceReconciler) createRayClusterInstanceIfNeeded(ctx context.Cont } else { clusterAction, err = getClusterAction(pendingRayCluster.Spec, rayServiceInstance.Spec.RayClusterSpec) if err != nil { - err = fmt.Errorf("Fail to generate hash for RayClusterSpec: %w", err) + err = fmt.Errorf("fail to generate hash for RayClusterSpec: %w", err) return nil, err } } @@ -672,7 +672,7 @@ func (r *RayServiceReconciler) updateRayClusterInstance(ctx context.Context, ray Name: rayClusterInstance.Name, }) if err != nil { - err = fmt.Errorf("Failed to get the current state of RayCluster, namespace: %s, name: %s: %w", rayClusterInstance.Namespace, rayClusterInstance.Name, err) + err = fmt.Errorf("failed to get the current state of RayCluster, namespace: %s, name: %s: %w", rayClusterInstance.Namespace, rayClusterInstance.Name, err) return err } @@ -843,12 +843,12 @@ func (r *RayServiceReconciler) updateServeDeployment(ctx context.Context, raySer configJson, err := json.Marshal(serveConfig) if err != nil { - return fmt.Errorf("Failed to marshal converted serve config into bytes: %w", err) + return fmt.Errorf("failed to marshal converted serve config into bytes: %w", err) } logger.Info("updateServeDeployment", "MULTI_APP json config", string(configJson)) if err := rayDashboardClient.UpdateDeployments(ctx, configJson); err != nil { err = fmt.Errorf( - "Fail to create / update Serve applications. If you observe this error consistently, "+ + "fail to create / update Serve applications. If you observe this error consistently, "+ "please check \"Issue 5: Fail to create / update Serve applications.\" in "+ "https://docs.ray.io/en/master/cluster/kubernetes/troubleshooting/rayservice-troubleshooting.html#kuberay-raysvc-troubleshoot for more details. "+ "err: %v", err) @@ -873,7 +873,7 @@ func (r *RayServiceReconciler) getAndCheckServeStatus(ctx context.Context, dashb var err error if serveAppStatuses, err = dashboardClient.GetMultiApplicationStatus(ctx); err != nil { err = fmt.Errorf( - "Failed to get Serve application statuses from the dashboard. "+ + "failed to get Serve application statuses from the dashboard. "+ "If you observe this error consistently, please check https://github.com/ray-project/kuberay/blob/master/docs/guidance/rayservice-troubleshooting.md for more details. "+ "err: %v", err) return false, err @@ -979,7 +979,6 @@ func (r *RayServiceReconciler) reconcileServices(ctx context.Context, rayService logger := ctrl.LoggerFrom(ctx) logger.Info( "reconcileServices", "serviceType", serviceType, - "RayService name", rayServiceInstance.Name, "RayService namespace", rayServiceInstance.Namespace, ) var newSvc *corev1.Service