Merge pull request #4986 from whitewindmills/fix-aggregate-binding-status

Integrate UpdateStatus function
This commit is contained in:
karmada-bot 2024-06-11 19:48:39 +08:00 committed by GitHub
commit 3ff49ad1d7
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
11 changed files with 225 additions and 267 deletions

View File

@ -32,6 +32,7 @@ import (
"k8s.io/client-go/util/retry" "k8s.io/client-go/util/retry"
"k8s.io/klog/v2" "k8s.io/klog/v2"
"sigs.k8s.io/controller-runtime/pkg/client" "sigs.k8s.io/controller-runtime/pkg/client"
"sigs.k8s.io/controller-runtime/pkg/controller/controllerutil"
autoscalingv1alpha1 "github.com/karmada-io/karmada/pkg/apis/autoscaling/v1alpha1" autoscalingv1alpha1 "github.com/karmada-io/karmada/pkg/apis/autoscaling/v1alpha1"
"github.com/karmada-io/karmada/pkg/metrics" "github.com/karmada-io/karmada/pkg/metrics"
@ -226,52 +227,55 @@ func (c *ScalingJob) addFailedExecutionHistory(
cronFHPA *autoscalingv1alpha1.CronFederatedHPA, errMsg string) error { cronFHPA *autoscalingv1alpha1.CronFederatedHPA, errMsg string) error {
_, nextExecutionTime := c.scheduler.NextRun() _, nextExecutionTime := c.scheduler.NextRun()
// Add success history record, return false if there is no such rule // Add failed history record
addFailedHistoryFunc := func() bool { addFailedHistoryFunc := func(index int) {
exists := false failedExecution := autoscalingv1alpha1.FailedExecution{
for index, rule := range cronFHPA.Status.ExecutionHistories { ScheduleTime: cronFHPA.Status.ExecutionHistories[index].NextExecutionTime,
if rule.RuleName != c.rule.Name { ExecutionTime: &metav1.Time{Time: time.Now()},
continue Message: errMsg,
}
failedExecution := autoscalingv1alpha1.FailedExecution{
ScheduleTime: rule.NextExecutionTime,
ExecutionTime: &metav1.Time{Time: time.Now()},
Message: errMsg,
}
historyLimits := helper.GetCronFederatedHPAFailedHistoryLimits(c.rule)
if len(rule.FailedExecutions) > historyLimits-1 {
rule.FailedExecutions = rule.FailedExecutions[:historyLimits-1]
}
cronFHPA.Status.ExecutionHistories[index].FailedExecutions =
append([]autoscalingv1alpha1.FailedExecution{failedExecution}, rule.FailedExecutions...)
cronFHPA.Status.ExecutionHistories[index].NextExecutionTime = &metav1.Time{Time: nextExecutionTime}
exists = true
break
} }
historyLimits := helper.GetCronFederatedHPAFailedHistoryLimits(c.rule)
return exists if len(cronFHPA.Status.ExecutionHistories[index].FailedExecutions) > historyLimits-1 {
cronFHPA.Status.ExecutionHistories[index].FailedExecutions = cronFHPA.Status.ExecutionHistories[index].FailedExecutions[:historyLimits-1]
}
cronFHPA.Status.ExecutionHistories[index].FailedExecutions =
append([]autoscalingv1alpha1.FailedExecution{failedExecution}, cronFHPA.Status.ExecutionHistories[index].FailedExecutions...)
cronFHPA.Status.ExecutionHistories[index].NextExecutionTime = &metav1.Time{Time: nextExecutionTime}
} }
return retry.RetryOnConflict(retry.DefaultRetry, func() (err error) { index := c.findExecutionHistory(cronFHPA.Status.ExecutionHistories)
// If this history not exist, it means the rule is suspended or deleted, so just ignore it. if index < 0 {
if exists := addFailedHistoryFunc(); !exists { // The failed history does not exist, it means the rule deleted, so just ignore it.
return nil return nil
} }
updateErr := c.client.Status().Update(context.Background(), cronFHPA) var operationResult controllerutil.OperationResult
if updateErr == nil { if err := retry.RetryOnConflict(retry.DefaultRetry, func() (err error) {
klog.V(4).Infof("CronFederatedHPA(%s/%s) status has been updated successfully", cronFHPA.Namespace, cronFHPA.Name) operationResult, err = helper.UpdateStatus(context.Background(), c.client, cronFHPA, func() error {
addFailedHistoryFunc(index)
return nil return nil
} })
return err
}); err != nil {
klog.Errorf("Failed to add failed history record to CronFederatedHPA(%s/%s): %v", cronFHPA.Namespace, cronFHPA.Name, err)
return err
}
updated := &autoscalingv1alpha1.CronFederatedHPA{} if operationResult == controllerutil.OperationResultUpdatedStatusOnly {
if err = c.client.Get(context.Background(), client.ObjectKey{Namespace: cronFHPA.Namespace, Name: cronFHPA.Name}, updated); err == nil { klog.V(4).Infof("CronFederatedHPA(%s/%s) status has been updated successfully", cronFHPA.Namespace, cronFHPA.Name)
cronFHPA = updated }
} else {
klog.Errorf("Get CronFederatedHPA(%s/%s) failed: %v", cronFHPA.Namespace, cronFHPA.Name, err) return nil
}
// findExecutionHistory finds the history record, returns -1 if there is no such rule.
func (c *ScalingJob) findExecutionHistory(histories []autoscalingv1alpha1.ExecutionHistory) int {
for index, rule := range histories {
if rule.RuleName == c.rule.Name {
return index
} }
return updateErr }
}) return -1
} }
func (c *ScalingJob) addSuccessExecutionHistory( func (c *ScalingJob) addSuccessExecutionHistory(
@ -279,52 +283,45 @@ func (c *ScalingJob) addSuccessExecutionHistory(
appliedReplicas, appliedMinReplicas, appliedMaxReplicas *int32) error { appliedReplicas, appliedMinReplicas, appliedMaxReplicas *int32) error {
_, nextExecutionTime := c.scheduler.NextRun() _, nextExecutionTime := c.scheduler.NextRun()
// Add success history record, return false if there is no such rule // Add success history record
addSuccessHistoryFunc := func() bool { addSuccessHistoryFunc := func(index int) {
exists := false successExecution := autoscalingv1alpha1.SuccessfulExecution{
for index, rule := range cronFHPA.Status.ExecutionHistories { ScheduleTime: cronFHPA.Status.ExecutionHistories[index].NextExecutionTime,
if rule.RuleName != c.rule.Name { ExecutionTime: &metav1.Time{Time: time.Now()},
continue AppliedReplicas: appliedReplicas,
} AppliedMaxReplicas: appliedMaxReplicas,
successExecution := autoscalingv1alpha1.SuccessfulExecution{ AppliedMinReplicas: appliedMinReplicas,
ScheduleTime: rule.NextExecutionTime,
ExecutionTime: &metav1.Time{Time: time.Now()},
AppliedReplicas: appliedReplicas,
AppliedMaxReplicas: appliedMaxReplicas,
AppliedMinReplicas: appliedMinReplicas,
}
historyLimits := helper.GetCronFederatedHPASuccessHistoryLimits(c.rule)
if len(rule.SuccessfulExecutions) > historyLimits-1 {
rule.SuccessfulExecutions = rule.SuccessfulExecutions[:historyLimits-1]
}
cronFHPA.Status.ExecutionHistories[index].SuccessfulExecutions =
append([]autoscalingv1alpha1.SuccessfulExecution{successExecution}, rule.SuccessfulExecutions...)
cronFHPA.Status.ExecutionHistories[index].NextExecutionTime = &metav1.Time{Time: nextExecutionTime}
exists = true
break
} }
historyLimits := helper.GetCronFederatedHPASuccessHistoryLimits(c.rule)
return exists if len(cronFHPA.Status.ExecutionHistories[index].SuccessfulExecutions) > historyLimits-1 {
cronFHPA.Status.ExecutionHistories[index].SuccessfulExecutions = cronFHPA.Status.ExecutionHistories[index].SuccessfulExecutions[:historyLimits-1]
}
cronFHPA.Status.ExecutionHistories[index].SuccessfulExecutions =
append([]autoscalingv1alpha1.SuccessfulExecution{successExecution}, cronFHPA.Status.ExecutionHistories[index].SuccessfulExecutions...)
cronFHPA.Status.ExecutionHistories[index].NextExecutionTime = &metav1.Time{Time: nextExecutionTime}
} }
return retry.RetryOnConflict(retry.DefaultRetry, func() (err error) { index := c.findExecutionHistory(cronFHPA.Status.ExecutionHistories)
// If this history not exist, it means the rule deleted, so just ignore it. if index < 0 {
if exists := addSuccessHistoryFunc(); !exists { // The success history does not exist, it means the rule deleted, so just ignore it.
return nil
}
var operationResult controllerutil.OperationResult
if err := retry.RetryOnConflict(retry.DefaultRetry, func() (err error) {
operationResult, err = helper.UpdateStatus(context.Background(), c.client, cronFHPA, func() error {
addSuccessHistoryFunc(index)
return nil return nil
} })
return err
}); err != nil {
klog.Errorf("Failed to add success history record to CronFederatedHPA(%s/%s): %v", cronFHPA.Namespace, cronFHPA.Name, err)
return err
}
updateErr := c.client.Status().Update(context.Background(), cronFHPA) if operationResult == controllerutil.OperationResultUpdatedStatusOnly {
if updateErr == nil { klog.V(4).Infof("CronFederatedHPA(%s/%s) status has been updated successfully", cronFHPA.Namespace, cronFHPA.Name)
klog.V(4).Infof("CronFederatedHPA(%s/%s) status has been updated successfully", cronFHPA.Namespace, cronFHPA.Name) }
return err
}
updated := &autoscalingv1alpha1.CronFederatedHPA{} return nil
if err = c.client.Get(context.Background(), client.ObjectKey{Namespace: cronFHPA.Namespace, Name: cronFHPA.Name}, updated); err == nil {
cronFHPA = updated
} else {
klog.Errorf("Get CronFederatedHPA(%s/%s) failed: %v", cronFHPA.Namespace, cronFHPA.Name, err)
}
return updateErr
})
} }

View File

@ -258,18 +258,11 @@ func (c *Controller) updateAppliedCondition(work *workv1alpha1.Work, status meta
} }
return retry.RetryOnConflict(retry.DefaultRetry, func() (err error) { return retry.RetryOnConflict(retry.DefaultRetry, func() (err error) {
meta.SetStatusCondition(&work.Status.Conditions, newWorkAppliedCondition) _, err = helper.UpdateStatus(context.Background(), c.Client, work, func() error {
updateErr := c.Status().Update(context.TODO(), work) meta.SetStatusCondition(&work.Status.Conditions, newWorkAppliedCondition)
if updateErr == nil {
return nil return nil
} })
updated := &workv1alpha1.Work{} return err
if err = c.Get(context.TODO(), client.ObjectKey{Namespace: work.Namespace, Name: work.Name}, updated); err == nil {
work = updated
} else {
klog.Errorf("Failed to get the updated work(%s/%s), err: %v", work.Namespace, work.Name, err)
}
return updateErr
}) })
} }

View File

@ -160,20 +160,11 @@ func (c *StatusController) collectQuotaStatus(quota *policyv1alpha1.FederatedRes
} }
return retry.RetryOnConflict(retry.DefaultRetry, func() error { return retry.RetryOnConflict(retry.DefaultRetry, func() error {
quota.Status = *quotaStatus _, err = helper.UpdateStatus(context.Background(), c.Client, quota, func() error {
updateErr := c.Status().Update(context.TODO(), quota) quota.Status = *quotaStatus
if updateErr == nil {
return nil return nil
} })
return err
updated := &policyv1alpha1.FederatedResourceQuota{}
if err = c.Get(context.TODO(), client.ObjectKey{Namespace: quota.Namespace, Name: quota.Name}, updated); err == nil {
quota = updated
} else {
klog.Errorf("Failed to get updated federatedResourceQuota(%s): %v", klog.KObj(quota).String(), err)
}
return updateErr
}) })
} }

View File

@ -32,6 +32,7 @@ import (
"github.com/karmada-io/karmada/pkg/events" "github.com/karmada-io/karmada/pkg/events"
"github.com/karmada-io/karmada/pkg/util" "github.com/karmada-io/karmada/pkg/util"
"github.com/karmada-io/karmada/pkg/util/helper"
"github.com/karmada-io/karmada/pkg/util/names" "github.com/karmada-io/karmada/pkg/util/names"
) )
@ -153,24 +154,15 @@ func (c *ServiceImportController) updateServiceStatus(svcImport *mcsv1alpha1.Ser
} }
err := retry.RetryOnConflict(retry.DefaultRetry, func() (err error) { err := retry.RetryOnConflict(retry.DefaultRetry, func() (err error) {
derivedService.Status = corev1.ServiceStatus{ _, err = helper.UpdateStatus(context.Background(), c.Client, derivedService, func() error {
LoadBalancer: corev1.LoadBalancerStatus{ derivedService.Status = corev1.ServiceStatus{
Ingress: ingress, LoadBalancer: corev1.LoadBalancerStatus{
}, Ingress: ingress,
} },
updateErr := c.Status().Update(context.TODO(), derivedService) }
if updateErr == nil {
return nil return nil
} })
return err
updated := &corev1.Service{}
if err = c.Get(context.TODO(), client.ObjectKey{Namespace: derivedService.Namespace, Name: derivedService.Name}, updated); err == nil {
derivedService = updated
} else {
klog.Errorf("Failed to get updated service %s/%s: %v", derivedService.Namespace, derivedService.Name, err)
}
return updateErr
}) })
if err != nil { if err != nil {

View File

@ -127,18 +127,11 @@ func (c *EndpointsliceDispatchController) updateEndpointSliceDispatched(mcs *net
} }
return retry.RetryOnConflict(retry.DefaultRetry, func() (err error) { return retry.RetryOnConflict(retry.DefaultRetry, func() (err error) {
meta.SetStatusCondition(&mcs.Status.Conditions, EndpointSliceCollected) _, err = helper.UpdateStatus(context.Background(), c.Client, mcs, func() error {
updateErr := c.Status().Update(context.TODO(), mcs) meta.SetStatusCondition(&mcs.Status.Conditions, EndpointSliceCollected)
if updateErr == nil {
return nil return nil
} })
updated := &networkingv1alpha1.MultiClusterService{} return err
if err = c.Get(context.TODO(), client.ObjectKey{Namespace: mcs.Namespace, Name: mcs.Name}, updated); err == nil {
mcs = updated
} else {
klog.Errorf("Failed to get updated MultiClusterService %s/%s: %v", mcs.Namespace, mcs.Name, err)
}
return updateErr
}) })
} }

View File

@ -521,18 +521,11 @@ func (c *MCSController) updateMultiClusterServiceStatus(mcs *networkingv1alpha1.
} }
return retry.RetryOnConflict(retry.DefaultRetry, func() (err error) { return retry.RetryOnConflict(retry.DefaultRetry, func() (err error) {
meta.SetStatusCondition(&mcs.Status.Conditions, serviceAppliedCondition) _, err = helper.UpdateStatus(context.Background(), c.Client, mcs, func() error {
updateErr := c.Status().Update(context.TODO(), mcs) meta.SetStatusCondition(&mcs.Status.Conditions, serviceAppliedCondition)
if updateErr == nil {
return nil return nil
} })
updated := &networkingv1alpha1.MultiClusterService{} return err
if err = c.Get(context.TODO(), client.ObjectKey{Namespace: mcs.Namespace, Name: mcs.Name}, updated); err == nil {
mcs = updated
} else {
klog.Errorf("Failed to get updated MultiClusterService %s/%s: %v", mcs.Namespace, mcs.Name, err)
}
return updateErr
}) })
} }

View File

@ -18,10 +18,8 @@ package remediation
import ( import (
"context" "context"
"reflect"
apierrors "k8s.io/apimachinery/pkg/api/errors" apierrors "k8s.io/apimachinery/pkg/api/errors"
"k8s.io/apimachinery/pkg/types"
"k8s.io/client-go/util/retry" "k8s.io/client-go/util/retry"
"k8s.io/klog/v2" "k8s.io/klog/v2"
controllerruntime "sigs.k8s.io/controller-runtime" controllerruntime "sigs.k8s.io/controller-runtime"
@ -33,6 +31,7 @@ import (
clusterv1alpha1 "github.com/karmada-io/karmada/pkg/apis/cluster/v1alpha1" clusterv1alpha1 "github.com/karmada-io/karmada/pkg/apis/cluster/v1alpha1"
remedyv1alpha1 "github.com/karmada-io/karmada/pkg/apis/remedy/v1alpha1" remedyv1alpha1 "github.com/karmada-io/karmada/pkg/apis/remedy/v1alpha1"
"github.com/karmada-io/karmada/pkg/sharedcli/ratelimiterflag" "github.com/karmada-io/karmada/pkg/sharedcli/ratelimiterflag"
"github.com/karmada-io/karmada/pkg/util/helper"
) )
// ControllerName is the controller name that will be used when reporting events. // ControllerName is the controller name that will be used when reporting events.
@ -70,26 +69,13 @@ func (c *RemedyController) Reconcile(ctx context.Context, req controllerruntime.
} }
actions := calculateActions(clusterRelatedRemedies, cluster) actions := calculateActions(clusterRelatedRemedies, cluster)
err = retry.RetryOnConflict(retry.DefaultRetry, func() error { if err = retry.RetryOnConflict(retry.DefaultRetry, func() error {
if reflect.DeepEqual(actions, cluster.Status.RemedyActions) { _, err = helper.UpdateStatus(ctx, c.Client, cluster, func() error {
cluster.Status.RemedyActions = actions
return nil return nil
} })
cluster.Status.RemedyActions = actions return err
updateErr := c.Client.Status().Update(ctx, cluster) }); err != nil {
if updateErr == nil {
return nil
}
updatedCluster := &clusterv1alpha1.Cluster{}
err = c.Client.Get(ctx, types.NamespacedName{Name: cluster.Name}, updatedCluster)
if err == nil {
cluster = updatedCluster
} else {
klog.Errorf("Failed to get updated cluster(%s): %v", cluster.Name, err)
}
return updateErr
})
if err != nil {
klog.Errorf("Failed to sync cluster(%s) remedy actions: %v", cluster.Name, err) klog.Errorf("Failed to sync cluster(%s) remedy actions: %v", cluster.Name, err)
return controllerruntime.Result{}, err return controllerruntime.Result{}, err
} }

View File

@ -277,19 +277,11 @@ func (c *ClusterStatusController) updateStatusIfNeeded(cluster *clusterv1alpha1.
if !equality.Semantic.DeepEqual(cluster.Status, currentClusterStatus) { if !equality.Semantic.DeepEqual(cluster.Status, currentClusterStatus) {
klog.V(4).Infof("Start to update cluster status: %s", cluster.Name) klog.V(4).Infof("Start to update cluster status: %s", cluster.Name)
err := retry.RetryOnConflict(retry.DefaultRetry, func() (err error) { err := retry.RetryOnConflict(retry.DefaultRetry, func() (err error) {
cluster.Status = currentClusterStatus _, err = helper.UpdateStatus(context.Background(), c.Client, cluster, func() error {
updateErr := c.Status().Update(context.TODO(), cluster) cluster.Status = currentClusterStatus
if updateErr == nil {
return nil return nil
} })
return err
updated := &clusterv1alpha1.Cluster{}
if err = c.Get(context.TODO(), client.ObjectKey{Namespace: cluster.Namespace, Name: cluster.Name}, updated); err == nil {
cluster = updated
} else {
klog.Errorf("Failed to get updated cluster %s: %v", cluster.Name, err)
}
return updateErr
}) })
if err != nil { if err != nil {
klog.Errorf("Failed to update health status of the member cluster: %v, err is : %v", cluster.Name, err) klog.Errorf("Failed to update health status of the member cluster: %v, err is : %v", cluster.Name, err)

View File

@ -326,22 +326,11 @@ func (c *WorkStatusController) updateAppliedCondition(work *workv1alpha1.Work, s
} }
err := retry.RetryOnConflict(retry.DefaultRetry, func() (err error) { err := retry.RetryOnConflict(retry.DefaultRetry, func() (err error) {
workStatus := work.Status.DeepCopy() _, err = helper.UpdateStatus(context.Background(), c.Client, work, func() error {
meta.SetStatusCondition(&work.Status.Conditions, newWorkAppliedCondition) meta.SetStatusCondition(&work.Status.Conditions, newWorkAppliedCondition)
if reflect.DeepEqual(*workStatus, work.Status) {
return nil return nil
} })
updateErr := c.Status().Update(context.TODO(), work) return err
if updateErr == nil {
return nil
}
updated := &workv1alpha1.Work{}
if err = c.Get(context.TODO(), client.ObjectKey{Namespace: work.Namespace, Name: work.Name}, updated); err == nil {
work = updated
} else {
klog.Errorf("Failed to get updated work %s/%s: %s", work.Namespace, work.Name, err.Error())
}
return updateErr
}) })
if err != nil { if err != nil {
@ -386,25 +375,12 @@ func (c *WorkStatusController) reflectStatus(work *workv1alpha1.Work, clusterObj
Health: resourceHealth, Health: resourceHealth,
} }
workCopy := work.DeepCopy()
return retry.RetryOnConflict(retry.DefaultRetry, func() (err error) { return retry.RetryOnConflict(retry.DefaultRetry, func() (err error) {
manifestStatuses := c.mergeStatus(workCopy.Status.ManifestStatuses, manifestStatus) _, err = helper.UpdateStatus(context.Background(), c.Client, work, func() error {
if reflect.DeepEqual(workCopy.Status.ManifestStatuses, manifestStatuses) { work.Status.ManifestStatuses = c.mergeStatus(work.Status.ManifestStatuses, manifestStatus)
return nil return nil
} })
workCopy.Status.ManifestStatuses = manifestStatuses return err
updateErr := c.Status().Update(context.TODO(), workCopy)
if updateErr == nil {
return nil
}
updated := &workv1alpha1.Work{}
if err = c.Get(context.TODO(), client.ObjectKey{Namespace: workCopy.Namespace, Name: workCopy.Name}, updated); err == nil {
workCopy = updated
} else {
klog.Errorf("Failed to get updated work %s/%s: %v", workCopy.Namespace, workCopy.Name, err)
}
return updateErr
}) })
} }

69
pkg/util/helper/status.go Executable file
View File

@ -0,0 +1,69 @@
/*
Copyright 2024 The Karmada Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package helper
import (
"context"
"fmt"
"k8s.io/apimachinery/pkg/api/equality"
"sigs.k8s.io/controller-runtime/pkg/client"
"sigs.k8s.io/controller-runtime/pkg/controller/controllerutil"
)
// UpdateStatus updates the given object's status in the Kubernetes
// cluster. The object's desired state must be reconciled with the existing
// state inside the passed in callback MutateFn.
//
// The MutateFn is called when updating an object's status.
//
// It returns the executed operation and an error.
//
// Note: changes to any sub-resource other than status will be ignored.
// Changes to the status sub-resource will only be applied if the object
// already exist.
func UpdateStatus(ctx context.Context, c client.Client, obj client.Object, f controllerutil.MutateFn) (controllerutil.OperationResult, error) {
key := client.ObjectKeyFromObject(obj)
if err := c.Get(ctx, key, obj); err != nil {
return controllerutil.OperationResultNone, err
}
existing := obj.DeepCopyObject()
if err := mutate(f, key, obj); err != nil {
return controllerutil.OperationResultNone, err
}
if equality.Semantic.DeepEqual(existing, obj) {
return controllerutil.OperationResultNone, nil
}
if err := c.Status().Update(ctx, obj); err != nil {
return controllerutil.OperationResultNone, err
}
return controllerutil.OperationResultUpdatedStatusOnly, nil
}
// mutate wraps a MutateFn and applies validation to its result.
func mutate(f controllerutil.MutateFn, key client.ObjectKey, obj client.Object) error {
if err := f(); err != nil {
return err
}
if newKey := client.ObjectKeyFromObject(obj); key != newKey {
return fmt.Errorf("MutateFn cannot mutate object name and/or object namespace")
}
return nil
}

View File

@ -20,7 +20,6 @@ import (
"context" "context"
"encoding/json" "encoding/json"
"fmt" "fmt"
"reflect"
"sort" "sort"
corev1 "k8s.io/api/core/v1" corev1 "k8s.io/api/core/v1"
@ -34,6 +33,7 @@ import (
"k8s.io/client-go/util/retry" "k8s.io/client-go/util/retry"
"k8s.io/klog/v2" "k8s.io/klog/v2"
"sigs.k8s.io/controller-runtime/pkg/client" "sigs.k8s.io/controller-runtime/pkg/client"
"sigs.k8s.io/controller-runtime/pkg/controller/controllerutil"
workv1alpha1 "github.com/karmada-io/karmada/pkg/apis/work/v1alpha1" workv1alpha1 "github.com/karmada-io/karmada/pkg/apis/work/v1alpha1"
workv1alpha2 "github.com/karmada-io/karmada/pkg/apis/work/v1alpha2" workv1alpha2 "github.com/karmada-io/karmada/pkg/apis/work/v1alpha2"
@ -73,41 +73,28 @@ func AggregateResourceBindingWorkStatus(
fullyAppliedCondition := generateFullyAppliedCondition(binding.Spec, aggregatedStatuses) fullyAppliedCondition := generateFullyAppliedCondition(binding.Spec, aggregatedStatuses)
err = retry.RetryOnConflict(retry.DefaultRetry, func() (err error) { var operationResult controllerutil.OperationResult
currentBindingStatus := binding.Status.DeepCopy() if err = retry.RetryOnConflict(retry.DefaultRetry, func() error {
operationResult, err = UpdateStatus(context.Background(), c, binding, func() error {
binding.Status.AggregatedStatus = aggregatedStatuses binding.Status.AggregatedStatus = aggregatedStatuses
// set binding status with the newest condition // set binding status with the newest condition
meta.SetStatusCondition(&binding.Status.Conditions, fullyAppliedCondition) meta.SetStatusCondition(&binding.Status.Conditions, fullyAppliedCondition)
if reflect.DeepEqual(binding.Status, *currentBindingStatus) {
klog.V(4).Infof("New aggregatedStatuses are equal with old resourceBinding(%s/%s) AggregatedStatus, no update required.",
binding.Namespace, binding.Name)
return nil return nil
} })
return err
updateErr := c.Status().Update(context.TODO(), binding) }); err != nil {
if updateErr == nil {
return nil
}
updated := &workv1alpha2.ResourceBinding{}
if err = c.Get(context.TODO(), client.ObjectKey{Namespace: binding.Namespace, Name: binding.Name}, updated); err == nil {
binding = updated
} else {
klog.Errorf("Failed to get updated binding %s/%s: %v", binding.Namespace, binding.Name, err)
}
return updateErr
})
if err != nil {
eventRecorder.Event(binding, corev1.EventTypeWarning, events.EventReasonAggregateStatusFailed, err.Error()) eventRecorder.Event(binding, corev1.EventTypeWarning, events.EventReasonAggregateStatusFailed, err.Error())
eventRecorder.Event(resourceTemplate, corev1.EventTypeWarning, events.EventReasonAggregateStatusFailed, err.Error()) eventRecorder.Event(resourceTemplate, corev1.EventTypeWarning, events.EventReasonAggregateStatusFailed, err.Error())
return err return err
} }
msg := fmt.Sprintf("Update resourceBinding(%s/%s) with AggregatedStatus successfully.", binding.Namespace, binding.Name) if operationResult == controllerutil.OperationResultUpdatedStatusOnly {
eventRecorder.Event(binding, corev1.EventTypeNormal, events.EventReasonAggregateStatusSucceed, msg) msg := fmt.Sprintf("Update ResourceBinding(%s/%s) with AggregatedStatus successfully.", binding.Namespace, binding.Name)
eventRecorder.Event(resourceTemplate, corev1.EventTypeNormal, events.EventReasonAggregateStatusSucceed, msg) eventRecorder.Event(binding, corev1.EventTypeNormal, events.EventReasonAggregateStatusSucceed, msg)
eventRecorder.Event(resourceTemplate, corev1.EventTypeNormal, events.EventReasonAggregateStatusSucceed, msg)
} else {
klog.Infof("New aggregatedStatuses are equal with old ResourceBinding(%s/%s) AggregatedStatus, no update required.", binding.Namespace, binding.Name)
}
return nil return nil
} }
@ -131,40 +118,29 @@ func AggregateClusterResourceBindingWorkStatus(
fullyAppliedCondition := generateFullyAppliedCondition(binding.Spec, aggregatedStatuses) fullyAppliedCondition := generateFullyAppliedCondition(binding.Spec, aggregatedStatuses)
err = retry.RetryOnConflict(retry.DefaultRetry, func() (err error) { var operationResult controllerutil.OperationResult
currentBindingStatus := binding.Status.DeepCopy() if err = retry.RetryOnConflict(retry.DefaultRetry, func() error {
operationResult, err = UpdateStatus(context.Background(), c, binding, func() error {
binding.Status.AggregatedStatus = aggregatedStatuses binding.Status.AggregatedStatus = aggregatedStatuses
// set binding status with the newest condition // set binding status with the newest condition
meta.SetStatusCondition(&binding.Status.Conditions, fullyAppliedCondition) meta.SetStatusCondition(&binding.Status.Conditions, fullyAppliedCondition)
if reflect.DeepEqual(binding.Status, *currentBindingStatus) {
klog.Infof("New aggregatedStatuses are equal with old clusterResourceBinding(%s) AggregatedStatus, no update required.", binding.Name)
return nil return nil
} })
return err
updateErr := c.Status().Update(context.TODO(), binding) }); err != nil {
if updateErr == nil {
return nil
}
updated := &workv1alpha2.ClusterResourceBinding{}
if err = c.Get(context.TODO(), client.ObjectKey{Name: binding.Name}, updated); err == nil {
binding = updated
} else {
klog.Errorf("Failed to get updated binding %s/%s: %v", binding.Namespace, binding.Name, err)
}
return updateErr
})
if err != nil {
eventRecorder.Event(binding, corev1.EventTypeWarning, events.EventReasonAggregateStatusFailed, err.Error()) eventRecorder.Event(binding, corev1.EventTypeWarning, events.EventReasonAggregateStatusFailed, err.Error())
eventRecorder.Event(resourceTemplate, corev1.EventTypeWarning, events.EventReasonAggregateStatusFailed, err.Error()) eventRecorder.Event(resourceTemplate, corev1.EventTypeWarning, events.EventReasonAggregateStatusFailed, err.Error())
return err return err
} }
msg := fmt.Sprintf("Update clusterResourceBinding(%s) with AggregatedStatus successfully.", binding.Name) if operationResult == controllerutil.OperationResultUpdatedStatusOnly {
eventRecorder.Event(binding, corev1.EventTypeNormal, events.EventReasonAggregateStatusSucceed, msg) msg := fmt.Sprintf("Update ClusterResourceBinding(%s) with AggregatedStatus successfully.", binding.Name)
eventRecorder.Event(resourceTemplate, corev1.EventTypeNormal, events.EventReasonAggregateStatusSucceed, msg) eventRecorder.Event(binding, corev1.EventTypeNormal, events.EventReasonAggregateStatusSucceed, msg)
eventRecorder.Event(resourceTemplate, corev1.EventTypeNormal, events.EventReasonAggregateStatusSucceed, msg)
} else {
klog.Infof("New aggregatedStatuses are equal with old ClusterResourceBinding(%s) AggregatedStatus, no update required.", binding.Name)
}
return nil return nil
} }