Merge pull request #4986 from whitewindmills/fix-aggregate-binding-status
Integrate UpdateStatus function
This commit is contained in:
commit
3ff49ad1d7
|
@ -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
|
|
||||||
})
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -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
|
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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
|
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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 {
|
||||||
|
|
|
@ -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
|
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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
|
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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
|
||||||
}
|
}
|
||||||
|
|
|
@ -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)
|
||||||
|
|
|
@ -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
|
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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
|
||||||
|
}
|
|
@ -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
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue