refactor binding controller with controller-runtime (#52)

This commit is contained in:
Xianpao Chen 2020-12-04 11:47:06 +08:00 committed by GitHub
parent 2e7d385163
commit 28ecd3b0ff
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
4 changed files with 82 additions and 269 deletions

View File

@ -142,10 +142,6 @@ func startControllers(opts *options.Options, stopChan <-chan struct{}) {
klog.Fatalf("Failed to start member cluster controller. error: %v", err)
}
if err := binding.StartPropagationBindingController(controllerConfig, stopChan); err != nil {
klog.Fatalf("Failed to start binding controller. error: %v", err)
}
if err := execution.StartExecutionController(controllerConfig, stopChan); err != nil {
klog.Fatalf("Failed to start execution controller. error: %v", err)
}
@ -165,4 +161,14 @@ func setupControllers(mgr controllerruntime.Manager) {
if err := policyController.SetupWithManager(mgr); err != nil {
klog.Fatalf("Failed to setup policy controller: %v", err)
}
bindingController := &binding.PropagationBindingController{
Client: mgr.GetClient(),
DynamicClient: dynamicClientSet,
KarmadaClient: karmadaClient,
EventRecorder: mgr.GetEventRecorderFor(binding.ControllerName),
}
if err := bindingController.SetupWithManager(mgr); err != nil {
klog.Fatalf("Failed to setup binding controller: %v", err)
}
}

View File

@ -2,266 +2,88 @@ package binding
import (
"context"
"fmt"
"time"
corev1 "k8s.io/api/core/v1"
"k8s.io/apimachinery/pkg/api/errors"
apierrors "k8s.io/apimachinery/pkg/api/errors"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
"k8s.io/apimachinery/pkg/apis/meta/v1/unstructured"
"k8s.io/apimachinery/pkg/runtime"
utilruntime "k8s.io/apimachinery/pkg/util/runtime"
"k8s.io/apimachinery/pkg/util/wait"
"k8s.io/client-go/dynamic"
"k8s.io/client-go/kubernetes"
"k8s.io/client-go/kubernetes/scheme"
typedcorev1 "k8s.io/client-go/kubernetes/typed/core/v1"
"k8s.io/client-go/rest"
"k8s.io/client-go/tools/cache"
"k8s.io/client-go/tools/record"
"k8s.io/client-go/util/workqueue"
"k8s.io/klog/v2"
controllerruntime "sigs.k8s.io/controller-runtime"
"sigs.k8s.io/controller-runtime/pkg/client"
"github.com/huawei-cloudnative/karmada/pkg/apis/propagationstrategy/v1alpha1"
"github.com/huawei-cloudnative/karmada/pkg/controllers/util"
clientset "github.com/huawei-cloudnative/karmada/pkg/generated/clientset/versioned"
karmadaScheme "github.com/huawei-cloudnative/karmada/pkg/generated/clientset/versioned/scheme"
informers "github.com/huawei-cloudnative/karmada/pkg/generated/informers/externalversions"
listers "github.com/huawei-cloudnative/karmada/pkg/generated/listers/propagationstrategy/v1alpha1"
karmadaclientset "github.com/huawei-cloudnative/karmada/pkg/generated/clientset/versioned"
)
const (
controllerAgentName = "binding-controller"
)
// ControllerName is the controller name that will be used when reporting events.
const ControllerName = "binding-controller"
var controllerKind = v1alpha1.SchemeGroupVersion.WithKind("PropagationBinding")
// Controller is the controller implementation for binding resources
type Controller struct {
// karmadaClientSet is the clientset for our own API group.
karmadaClientSet clientset.Interface
// kubeClientSet is a standard kubernetes clientset.
kubeClientSet kubernetes.Interface
dynamicClientSet dynamic.Interface
karmadaInformerFactory informers.SharedInformerFactory
propagationBindingLister listers.PropagationBindingLister
propagationBindingSynced cache.InformerSynced
// workqueue is a rate limited work queue. This is used to queue work to be
// processed instead of performing it as soon as a change happens. This
// means we can ensure we only process a fixed amount of resources at a
// time, and makes it easy to ensure we are never processing the same item
// simultaneously in two different workers.
workqueue workqueue.RateLimitingInterface
// recorder is an event recorder for recording Event resources to the
// Kubernetes API.
eventRecorder record.EventRecorder
// PropagationBindingController is to sync PropagationBinding.
type PropagationBindingController struct {
client.Client // used to operate PropagationBinding resources.
DynamicClient dynamic.Interface // used to fetch arbitrary resources.
KarmadaClient karmadaclientset.Interface // used to create/update PropagationWork resources.
EventRecorder record.EventRecorder
}
// StartPropagationBindingController starts a new binding controller.
func StartPropagationBindingController(config *util.ControllerConfig, stopChan <-chan struct{}) error {
controller, err := newPropagationBindingController(config)
if err != nil {
return err
}
klog.Infof("Starting PropagationBinding controller")
// Reconcile performs a full reconciliation for the object referred to by the Request.
// The Controller will requeue the Request to be processed again if an error is non-nil or
// Result.Requeue is true, otherwise upon completion it will remove the work from the queue.
func (c *PropagationBindingController) Reconcile(req controllerruntime.Request) (controllerruntime.Result, error) {
klog.V(4).Infof("Reconciling PropagationBinding %s", req.NamespacedName.String())
go wait.Until(func() {
if err := controller.Run(1, stopChan); err != nil {
klog.Errorf("controller exit unexpected! will restart later, controller: %s, error: %v", controllerAgentName, err)
}
}, 1*time.Second, stopChan)
return nil
}
// newPropagationBindingController returns a new controller.
func newPropagationBindingController(config *util.ControllerConfig) (*Controller, error) {
headClusterConfig := rest.CopyConfig(config.HeadClusterConfig)
kubeClientSet := kubernetes.NewForConfigOrDie(headClusterConfig)
karmadaClientSet := clientset.NewForConfigOrDie(headClusterConfig)
dynamicClientSet, err := dynamic.NewForConfig(headClusterConfig)
if err != nil {
return nil, err
}
karmadaInformerFactory := informers.NewSharedInformerFactory(karmadaClientSet, 0)
propagationBindingInformer := karmadaInformerFactory.Propagationstrategy().V1alpha1().PropagationBindings()
// Add karmada types to the default Kubernetes Scheme so Events can be logged for karmada types.
utilruntime.Must(karmadaScheme.AddToScheme(scheme.Scheme))
// Create event broadcaster
klog.V(1).Infof("Creating event broadcaster for %s", controllerAgentName)
eventBroadcaster := record.NewBroadcaster()
eventBroadcaster.StartRecordingToSink(&typedcorev1.EventSinkImpl{Interface: kubeClientSet.CoreV1().Events("")})
controller := &Controller{
karmadaClientSet: karmadaClientSet,
kubeClientSet: kubeClientSet,
dynamicClientSet: dynamicClientSet,
karmadaInformerFactory: karmadaInformerFactory,
propagationBindingLister: propagationBindingInformer.Lister(),
propagationBindingSynced: propagationBindingInformer.Informer().HasSynced,
workqueue: workqueue.NewNamedRateLimitingQueue(workqueue.DefaultControllerRateLimiter(), controllerAgentName),
eventRecorder: eventBroadcaster.NewRecorder(scheme.Scheme, corev1.EventSource{Component: controllerAgentName}),
}
klog.Info("Setting up event handlers")
propagationBindingInformer.Informer().AddEventHandler(cache.ResourceEventHandlerFuncs{
AddFunc: func(obj interface{}) {
klog.Infof("Received add event. just add to queue.")
controller.enqueueEventResource(obj)
},
UpdateFunc: func(old, new interface{}) {
klog.Infof("Received update event. just add to queue.")
controller.enqueueEventResource(new)
},
DeleteFunc: func(obj interface{}) {
klog.Infof("Received delete event. Do nothing just log.")
},
})
return controller, nil
}
// Run will set up the event handlers for types we are interested in, as well
// as syncing informer caches and starting workers. It will block until stopCh
// is closed, at which point it will shutdown the workqueue and wait for
// workers to finish processing their current work items.
func (c *Controller) Run(workerNumber int, stopCh <-chan struct{}) error {
defer utilruntime.HandleCrash()
defer c.workqueue.ShutDown()
klog.Infof("Run controller: %s", controllerAgentName)
c.karmadaInformerFactory.Start(stopCh)
// Wait for the caches to be synced before starting workers
klog.Info("Waiting for informer caches to sync")
if ok := cache.WaitForCacheSync(stopCh, c.propagationBindingSynced); !ok {
return fmt.Errorf("failed to wait for caches to sync")
}
klog.Infof("Starting workers for controller. worker number: %d, controller: %s", workerNumber, controllerAgentName)
for i := 0; i < workerNumber; i++ {
go wait.Until(c.runWorker, time.Second, stopCh)
}
// Controller will block here until stopCh is closed.
<-stopCh
klog.Info("Shutting down workers")
return nil
}
// runWorker is a long-running function that will continually call the
// processNextWorkItem function in order to read and process a message on the
// workqueue.
func (c *Controller) runWorker() {
for c.processNextWorkItem() {
}
}
// processNextWorkItem will read a single work item off the workqueue and
// attempt to process it, by calling the syncHandler.
func (c *Controller) processNextWorkItem() bool {
obj, shutdown := c.workqueue.Get()
if shutdown {
return false
}
// We wrap this block in a func so we can defer c.workqueue.Done.
err := func(obj interface{}) error {
// We call Done here so the workqueue knows we have finished
// processing this item. We also must remember to call Forget if we
// do not want this work item being re-queued. For example, we do
// not call Forget if a transient error occurs, instead the item is
// put back on the workqueue and attempted again after a back-off
// period.
defer c.workqueue.Done(obj)
var key string
var ok bool
// We expect strings to come off the workqueue. These are of the
// form namespace/name. We do this as the delayed nature of the
// workqueue means the items in the informer cache may actually be
// more up to date that when the item was initially put onto the
// workqueue.
if key, ok = obj.(string); !ok {
// As the item in the workqueue is actually invalid, we call
// Forget here else we'd go into a loop of attempting to
// process a work item that is invalid.
c.workqueue.Forget(obj)
utilruntime.HandleError(fmt.Errorf("expected string in workqueue but got %#v", obj))
return nil
}
// Run the syncHandler, passing it the namespace/name string of the
// PropagateStrategy resource to be synced.
if err := c.syncHandler(key); err != nil {
// Put the item back on the workqueue to handle any transient errors.
c.workqueue.AddRateLimited(key)
return fmt.Errorf("error syncing '%s': %s, requeuing", key, err.Error())
}
// Finally, if no error occurs we Forget this item so it does not
// get queued again until another change happens.
c.workqueue.Forget(obj)
klog.Infof("Successfully synced '%s'", key)
return nil
}(obj)
if err != nil {
utilruntime.HandleError(err)
return true
}
return true
}
// syncHandler compares the actual state with the desired, and attempts to
// converge the two. It then updates the Status block of the PropagateStrategy resource
// with the current status of the resource.
func (c *Controller) syncHandler(key string) error {
// Convert the namespace/name string into a distinct namespace and name
namespace, name, err := cache.SplitMetaNamespaceKey(key)
if err != nil {
utilruntime.HandleError(fmt.Errorf("invalid resource key: %s", key))
return nil
}
// Get the resource with this namespace/name
propagationBinding, err := c.propagationBindingLister.PropagationBindings(namespace).Get(name)
if err != nil {
// The propagationBinding resource may no longer exist, in which case we stop
// processing.
binding := &v1alpha1.PropagationBinding{}
if err := c.Client.Get(context.TODO(), req.NamespacedName, binding); err != nil {
// The resource may no longer exist, in which case we stop processing.
if errors.IsNotFound(err) {
utilruntime.HandleError(fmt.Errorf("propagationBinding '%s' in work queue no longer exists", key))
return nil
return controllerruntime.Result{}, nil
}
return err
return controllerruntime.Result{Requeue: true}, err
}
klog.V(2).Infof("Sync propagationBinding: %s/%s", propagationBinding.Namespace, propagationBinding.Name)
err = c.transformBindingToWorks(propagationBinding)
if !binding.DeletionTimestamp.IsZero() {
// Do nothing, just return as we have added owner reference to PropagationWork.
// PropagationWork will be removed automatically by garbage collector.
return controllerruntime.Result{}, nil
}
return c.syncBinding(binding)
}
// syncBinding will sync propagationBinding to propagationWorks
func (c *PropagationBindingController) syncBinding(binding *v1alpha1.PropagationBinding) (controllerruntime.Result, error) {
err := c.transformBindingToWorks(binding)
if err != nil {
klog.Errorf("failed to transform propagationBinding %s/%s to propagationWorks. error: %+v",
propagationBinding.Namespace, propagationBinding.Name, err)
return err
binding.Namespace, binding.Name, err)
return controllerruntime.Result{Requeue: true}, err
}
return nil
return controllerruntime.Result{}, nil
}
// get clusterName list from bind clusters field
func (c *Controller) getBindingClusterNames(propagationBinding *v1alpha1.PropagationBinding) []string {
// SetupWithManager creates a controller and register to controller manager.
func (c *PropagationBindingController) SetupWithManager(mgr controllerruntime.Manager) error {
return controllerruntime.NewControllerManagedBy(mgr).For(&v1alpha1.PropagationBinding{}).Complete(c)
}
// getBindingClusterNames will get clusterName list from bind clusters field
func (c *PropagationBindingController) getBindingClusterNames(binding *v1alpha1.PropagationBinding) []string {
var clusterNames []string
for _, targetCluster := range propagationBinding.Spec.Clusters {
for _, targetCluster := range binding.Spec.Clusters {
clusterNames = append(clusterNames, targetCluster.Name)
}
return clusterNames
}
// delete irrelevant field from workload. such as uid, timestamp, status
func (c *Controller) removeIrrelevantField(workload *unstructured.Unstructured) {
// removeIrrelevantField will delete irrelevant field from workload. such as uid, timestamp, status
func (c *PropagationBindingController) removeIrrelevantField(workload *unstructured.Unstructured) {
unstructured.RemoveNestedField(workload.Object, "metadata", "creationTimestamp")
unstructured.RemoveNestedField(workload.Object, "metadata", "generation")
unstructured.RemoveNestedField(workload.Object, "metadata", "resourceVersion")
@ -270,18 +92,18 @@ func (c *Controller) removeIrrelevantField(workload *unstructured.Unstructured)
unstructured.RemoveNestedField(workload.Object, "status")
}
// transform propagationBinding resource to propagationWork resources
func (c *Controller) transformBindingToWorks(propagationBinding *v1alpha1.PropagationBinding) error {
workload, err := util.GetResourceStructure(c.dynamicClientSet, propagationBinding.Spec.Resource.APIVersion,
propagationBinding.Spec.Resource.Kind, propagationBinding.Spec.Resource.Namespace, propagationBinding.Spec.Resource.Name)
// transformBindingToWorks will transform propagationBinding to propagationWorks
func (c *PropagationBindingController) transformBindingToWorks(binding *v1alpha1.PropagationBinding) error {
workload, err := util.GetResourceStructure(c.DynamicClient, binding.Spec.Resource.APIVersion,
binding.Spec.Resource.Kind, binding.Spec.Resource.Namespace, binding.Spec.Resource.Name)
if err != nil {
klog.Errorf("failed to get resource. error: %v", err)
return err
}
clusterNames := c.getBindingClusterNames(propagationBinding)
clusterNames := c.getBindingClusterNames(binding)
err = c.ensurePropagationWork(workload, clusterNames, propagationBinding)
err = c.ensurePropagationWork(workload, clusterNames, binding)
if err != nil {
return err
}
@ -289,8 +111,8 @@ func (c *Controller) transformBindingToWorks(propagationBinding *v1alpha1.Propag
}
// ensurePropagationWork ensure PropagationWork to be created or updated
func (c *Controller) ensurePropagationWork(workload *unstructured.Unstructured, clusterNames []string,
propagationBinding *v1alpha1.PropagationBinding) error {
func (c *PropagationBindingController) ensurePropagationWork(workload *unstructured.Unstructured, clusterNames []string,
binding *v1alpha1.PropagationBinding) error {
c.removeIrrelevantField(workload)
formatWorkload, err := workload.MarshalJSON()
if err != nil {
@ -302,9 +124,9 @@ func (c *Controller) ensurePropagationWork(workload *unstructured.Unstructured,
}
propagationWork := v1alpha1.PropagationWork{
ObjectMeta: metav1.ObjectMeta{
Name: propagationBinding.Name,
Name: binding.Name,
OwnerReferences: []metav1.OwnerReference{
*metav1.NewControllerRef(propagationBinding, controllerKind),
*metav1.NewControllerRef(binding, controllerKind),
},
},
Spec: v1alpha1.PropagationWorkSpec{
@ -319,15 +141,14 @@ func (c *Controller) ensurePropagationWork(workload *unstructured.Unstructured,
}
for _, clusterNameMirrorNamespace := range clusterNames {
workGetResult, err := c.karmadaClientSet.PropagationstrategyV1alpha1().PropagationWorks(clusterNameMirrorNamespace).Get(context.TODO(), propagationWork.Name, metav1.GetOptions{})
workGetResult, err := c.KarmadaClient.PropagationstrategyV1alpha1().PropagationWorks(clusterNameMirrorNamespace).Get(context.TODO(), propagationWork.Name, metav1.GetOptions{})
if err != nil && apierrors.IsNotFound(err) {
workCreateResult, err := c.karmadaClientSet.PropagationstrategyV1alpha1().PropagationWorks(clusterNameMirrorNamespace).Create(context.TODO(), &propagationWork, metav1.CreateOptions{})
_, err := c.KarmadaClient.PropagationstrategyV1alpha1().PropagationWorks(clusterNameMirrorNamespace).Create(context.TODO(), &propagationWork, metav1.CreateOptions{})
if err != nil {
klog.Errorf("failed to create propagationWork %s/%s. error: %v", clusterNameMirrorNamespace, propagationWork.Name, err)
return err
}
klog.Infof("create propagationWork %s/%s success", clusterNameMirrorNamespace, propagationWork.Name)
klog.V(2).Infof("create propagationWork: %+v", workCreateResult)
continue
} else if err != nil && !apierrors.IsNotFound(err) {
klog.Errorf("failed to get propagationWork %s/%s. error: %v", clusterNameMirrorNamespace, propagationWork.Name, err)
@ -335,26 +156,12 @@ func (c *Controller) ensurePropagationWork(workload *unstructured.Unstructured,
}
workGetResult.Spec = propagationWork.Spec
workGetResult.ObjectMeta.OwnerReferences = propagationWork.ObjectMeta.OwnerReferences
workUpdateResult, err := c.karmadaClientSet.PropagationstrategyV1alpha1().PropagationWorks(clusterNameMirrorNamespace).Update(context.TODO(), workGetResult, metav1.UpdateOptions{})
_, err = c.KarmadaClient.PropagationstrategyV1alpha1().PropagationWorks(clusterNameMirrorNamespace).Update(context.TODO(), workGetResult, metav1.UpdateOptions{})
if err != nil {
klog.Errorf("failed to update propagationWork %s/%s. error: %v", clusterNameMirrorNamespace, propagationWork.Name, err)
return err
}
klog.Infof("update propagationWork %s/%s success", clusterNameMirrorNamespace, propagationWork.Name)
klog.V(2).Infof("update propagationWork: %+v", workUpdateResult)
}
return nil
}
// enqueueFoo takes a resource and converts it into a namespace/name
// string which is then put onto the work queue. This method should *not* be
// passed resources of any type other than propagationBinding.
func (c *Controller) enqueueEventResource(obj interface{}) {
var key string
var err error
if key, err = cache.MetaNamespaceKeyFunc(obj); err != nil {
utilruntime.HandleError(err)
return
}
c.workqueue.Add(key)
}

View File

@ -57,6 +57,7 @@ func (c *PropagationPolicyController) Reconcile(req controllerruntime.Request) (
return c.syncPolicy(policy)
}
// syncPolicy will fetch matched resource by policy, then transform them to propagationBindings
func (c *PropagationPolicyController) syncPolicy(policy *v1alpha1.PropagationPolicy) (controllerruntime.Result, error) {
workloads, err := c.fetchWorkloads(policy.Spec.ResourceSelectors)
if err != nil {
@ -83,8 +84,8 @@ func (c *PropagationPolicyController) fetchWorkloads(resourceSelectors []v1alpha
// todo: if resources repetitive, deduplication.
// todo: if namespaces, names, labelSelector is nil, need to do something
for _, resourceSelector := range resourceSelectors {
matchNamespaces := util.GetMatchItems(resourceSelector.Namespaces, resourceSelector.ExcludeNamespaces)
deduplicationNames := util.GetDeduplicationArray(resourceSelector.Names)
matchNamespaces := util.GetDifferenceSet(resourceSelector.Namespaces, resourceSelector.ExcludeNamespaces)
deduplicationNames := util.GetUniqueElements(resourceSelector.Names)
for _, namespace := range matchNamespaces {
if resourceSelector.LabelSelector == nil {
err := c.fetchWorkloadsWithOutLabelSelector(resourceSelector, namespace, deduplicationNames, &workloads)
@ -104,6 +105,7 @@ func (c *PropagationPolicyController) fetchWorkloads(resourceSelectors []v1alpha
return workloads, nil
}
// buildPropagationBinding will build propagationBinding by matched resources
func (c *PropagationPolicyController) buildPropagationBinding(policy *v1alpha1.PropagationPolicy, workloads []*unstructured.Unstructured) (controllerruntime.Result, error) {
targetCluster := c.getTargetClusters(policy.Spec.Placement)
@ -157,7 +159,7 @@ func (c *PropagationPolicyController) fetchWorkloadsWithOutLabelSelector(resourc
// getTargetClusters get targetClusters by placement
// TODO(RainbowMango): This is a dummy function and will be removed once scheduler on board.
func (c *PropagationPolicyController) getTargetClusters(placement v1alpha1.Placement) []v1alpha1.TargetCluster {
matchClusterNames := util.GetMatchItems(placement.ClusterAffinity.ClusterNames, placement.ClusterAffinity.ExcludeClusters)
matchClusterNames := util.GetDifferenceSet(placement.ClusterAffinity.ClusterNames, placement.ClusterAffinity.ExcludeClusters)
// todo: cluster labelSelector, fieldSelector, clusterTolerations
// todo: calc spread contraints. such as maximumClusters, minimumClusters
@ -168,7 +170,7 @@ func (c *PropagationPolicyController) getTargetClusters(placement v1alpha1.Place
return targetClusters
}
// create propagationBinding
// ensurePropagationBinding will ensure propagationBinding are created or updated.
func (c *PropagationPolicyController) ensurePropagationBinding(propagationPolicy *v1alpha1.PropagationPolicy, workload *unstructured.Unstructured, clusterNames []v1alpha1.TargetCluster) error {
bindingName := strings.ToLower(workload.GetNamespace() + "-" + workload.GetKind() + "-" + workload.GetName())
propagationBinding := v1alpha1.PropagationBinding{
@ -193,13 +195,12 @@ func (c *PropagationPolicyController) ensurePropagationBinding(propagationPolicy
bindingGetResult, err := c.KarmadaClient.PropagationstrategyV1alpha1().PropagationBindings(propagationBinding.Namespace).Get(context.TODO(), propagationBinding.Name, metav1.GetOptions{})
if err != nil && apierrors.IsNotFound(err) {
bindingCreateResult, err := c.KarmadaClient.PropagationstrategyV1alpha1().PropagationBindings(propagationBinding.Namespace).Create(context.TODO(), &propagationBinding, metav1.CreateOptions{})
_, err := c.KarmadaClient.PropagationstrategyV1alpha1().PropagationBindings(propagationBinding.Namespace).Create(context.TODO(), &propagationBinding, metav1.CreateOptions{})
if err != nil {
klog.Errorf("failed to create propagationBinding %s/%s. error: %v", propagationBinding.Namespace, propagationBinding.Name, err)
return err
}
klog.Infof("create propagationBinding %s/%s success", propagationBinding.Namespace, propagationBinding.Name)
klog.V(2).Infof("create propagationBinding: %+v", bindingCreateResult)
return nil
} else if err != nil && !apierrors.IsNotFound(err) {
klog.Errorf("failed to get propagationBinding %s/%s. error: %v", propagationBinding.Namespace, propagationBinding.Name, err)
@ -207,13 +208,12 @@ func (c *PropagationPolicyController) ensurePropagationBinding(propagationPolicy
}
bindingGetResult.Spec = propagationBinding.Spec
bindingGetResult.ObjectMeta.OwnerReferences = propagationBinding.ObjectMeta.OwnerReferences
bindingUpdateResult, err := c.KarmadaClient.PropagationstrategyV1alpha1().PropagationBindings(propagationBinding.Namespace).Update(context.TODO(), bindingGetResult, metav1.UpdateOptions{})
_, err = c.KarmadaClient.PropagationstrategyV1alpha1().PropagationBindings(propagationBinding.Namespace).Update(context.TODO(), bindingGetResult, metav1.UpdateOptions{})
if err != nil {
klog.Errorf("failed to update propagationBinding %s/%s. error: %v", propagationBinding.Namespace, propagationBinding.Name, err)
return err
}
klog.Infof("update propagationBinding %s/%s success", propagationBinding.Namespace, propagationBinding.Name)
klog.V(2).Infof("update propagationBinding: %+v", bindingUpdateResult)
return nil
}

View File

@ -68,8 +68,8 @@ func GetResourcesStructureByFilter(client dynamic.Interface, apiVersion, kind, n
return result, nil
}
// GetMatchItems get match item by compare include items and exclude items
func GetMatchItems(includeItems, excludeItems []string) []string {
// GetDifferenceSet get match item by compare include items and exclude items
func GetDifferenceSet(includeItems, excludeItems []string) []string {
if includeItems == nil {
includeItems = []string{}
}
@ -90,8 +90,8 @@ func GetMatchItems(includeItems, excludeItems []string) []string {
return matchItems.List()
}
// GetDeduplicationArray get deduplication array
func GetDeduplicationArray(list []string) []string {
// GetUniqueElements get deduplication array
func GetUniqueElements(list []string) []string {
if list == nil {
return []string{}
}