Skip to content

Commit

Permalink
[Logging] Remove duplicate info in CR logs (#2531)
Browse files Browse the repository at this point in the history
  • Loading branch information
nadongjun authored Nov 13, 2024
1 parent f41539b commit df03863
Show file tree
Hide file tree
Showing 4 changed files with 41 additions and 48 deletions.
26 changes: 12 additions & 14 deletions ray-operator/controllers/ray/raycluster_controller.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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)

Expand Down Expand Up @@ -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.
Expand All @@ -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,
)
Expand Down Expand Up @@ -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
}

Expand Down Expand Up @@ -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)
}
Expand Down Expand Up @@ -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
}

Expand Down Expand Up @@ -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 {
Expand Down Expand Up @@ -725,15 +723,15 @@ 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)
return errstd.Join(utils.ErrFailedCreateHeadPod, err)
}
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++ {
Expand Down Expand Up @@ -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.
Expand Down Expand Up @@ -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.")
}
}
}
Expand Down Expand Up @@ -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
Expand Down
14 changes: 5 additions & 9 deletions ray-operator/controllers/ray/raycluster_controller_unit_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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{}
Expand Down Expand Up @@ -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))
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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
Expand All @@ -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")
Expand Down
Loading

0 comments on commit df03863

Please sign in to comment.