mirror of
				https://github.com/optim-enterprises-bv/kubernetes.git
				synced 2025-11-03 19:58:17 +00:00 
			
		
		
		
	RC/RS: Use ControllerRef to route watch events.
This is part of the completion of ControllerRef, as described here: https://github.com/kubernetes/community/blob/master/contributors/design-proposals/controller-ref.md#watches This also removes the need for the Pod->Controller mapping cache in RC and RS. This mapping is now persisted in the Pod's ControllerRef instead.
This commit is contained in:
		@@ -29,7 +29,6 @@ import (
 | 
			
		||||
	"k8s.io/apimachinery/pkg/api/errors"
 | 
			
		||||
	metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
 | 
			
		||||
	"k8s.io/apimachinery/pkg/labels"
 | 
			
		||||
	"k8s.io/apimachinery/pkg/runtime/schema"
 | 
			
		||||
	utilruntime "k8s.io/apimachinery/pkg/util/runtime"
 | 
			
		||||
	"k8s.io/apimachinery/pkg/util/wait"
 | 
			
		||||
	utiltrace "k8s.io/apiserver/pkg/util/trace"
 | 
			
		||||
@@ -56,9 +55,8 @@ const (
 | 
			
		||||
	statusUpdateRetries = 1
 | 
			
		||||
)
 | 
			
		||||
 | 
			
		||||
func getRCKind() schema.GroupVersionKind {
 | 
			
		||||
	return v1.SchemeGroupVersion.WithKind("ReplicationController")
 | 
			
		||||
}
 | 
			
		||||
// controllerKind contains the schema.GroupVersionKind for this controller type.
 | 
			
		||||
var controllerKind = v1.SchemeGroupVersion.WithKind("ReplicationController")
 | 
			
		||||
 | 
			
		||||
// ReplicationManager is responsible for synchronizing ReplicationController objects stored
 | 
			
		||||
// in the system with actual running pods.
 | 
			
		||||
@@ -85,14 +83,12 @@ type ReplicationManager struct {
 | 
			
		||||
	// Added as a member to the struct to allow injection for testing.
 | 
			
		||||
	podListerSynced cache.InformerSynced
 | 
			
		||||
 | 
			
		||||
	lookupCache *controller.MatchingCache
 | 
			
		||||
 | 
			
		||||
	// Controllers that need to be synced
 | 
			
		||||
	queue workqueue.RateLimitingInterface
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// NewReplicationManager configures a replication manager with the specified event recorder
 | 
			
		||||
func NewReplicationManager(podInformer coreinformers.PodInformer, rcInformer coreinformers.ReplicationControllerInformer, kubeClient clientset.Interface, burstReplicas int, lookupCacheSize int) *ReplicationManager {
 | 
			
		||||
func NewReplicationManager(podInformer coreinformers.PodInformer, rcInformer coreinformers.ReplicationControllerInformer, kubeClient clientset.Interface, burstReplicas int) *ReplicationManager {
 | 
			
		||||
	if kubeClient != nil && kubeClient.Core().RESTClient().GetRateLimiter() != nil {
 | 
			
		||||
		metrics.RegisterMetricAndTrackRateLimiterUsage("replication_controller", kubeClient.Core().RESTClient().GetRateLimiter())
 | 
			
		||||
	}
 | 
			
		||||
@@ -135,7 +131,6 @@ func NewReplicationManager(podInformer coreinformers.PodInformer, rcInformer cor
 | 
			
		||||
	rm.podListerSynced = podInformer.Informer().HasSynced
 | 
			
		||||
 | 
			
		||||
	rm.syncHandler = rm.syncReplicationController
 | 
			
		||||
	rm.lookupCache = controller.NewMatchingCache(lookupCacheSize)
 | 
			
		||||
	return rm
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
@@ -167,71 +162,19 @@ func (rm *ReplicationManager) Run(workers int, stopCh <-chan struct{}) {
 | 
			
		||||
	glog.Infof("Shutting down RC Manager")
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// getPodController returns the controller managing the given pod.
 | 
			
		||||
// TODO: Surface that we are ignoring multiple controllers for a single pod.
 | 
			
		||||
// TODO: use ownerReference.Controller to determine if the rc controls the pod.
 | 
			
		||||
func (rm *ReplicationManager) getPodController(pod *v1.Pod) *v1.ReplicationController {
 | 
			
		||||
	// look up in the cache, if cached and the cache is valid, just return cached value
 | 
			
		||||
	if obj, cached := rm.lookupCache.GetMatchingObject(pod); cached {
 | 
			
		||||
		controller, ok := obj.(*v1.ReplicationController)
 | 
			
		||||
		if !ok {
 | 
			
		||||
			// This should not happen
 | 
			
		||||
			utilruntime.HandleError(fmt.Errorf("lookup cache does not return a ReplicationController object"))
 | 
			
		||||
			return nil
 | 
			
		||||
		}
 | 
			
		||||
		if cached && rm.isCacheValid(pod, controller) {
 | 
			
		||||
			return controller
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// if not cached or cached value is invalid, search all the rc to find the matching one, and update cache
 | 
			
		||||
	controllers, err := rm.rcLister.GetPodControllers(pod)
 | 
			
		||||
// getPodControllers returns a list of ReplicationControllers matching the given pod.
 | 
			
		||||
func (rm *ReplicationManager) getPodControllers(pod *v1.Pod) []*v1.ReplicationController {
 | 
			
		||||
	rcs, err := rm.rcLister.GetPodControllers(pod)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		glog.V(4).Infof("No controllers found for pod %v, replication manager will avoid syncing", pod.Name)
 | 
			
		||||
		glog.V(4).Infof("No ReplicationControllers found for pod %v, controller will avoid syncing", pod.Name)
 | 
			
		||||
		return nil
 | 
			
		||||
	}
 | 
			
		||||
	// In theory, overlapping controllers is user error. This sorting will not prevent
 | 
			
		||||
	// oscillation of replicas in all cases, eg:
 | 
			
		||||
	// rc1 (older rc): [(k1=v1)], replicas=1 rc2: [(k2=v2)], replicas=2
 | 
			
		||||
	// pod: [(k1:v1), (k2:v2)] will wake both rc1 and rc2, and we will sync rc1.
 | 
			
		||||
	// pod: [(k2:v2)] will wake rc2 which creates a new replica.
 | 
			
		||||
	if len(controllers) > 1 {
 | 
			
		||||
		// More than two items in this list indicates user error. If two replication-controller
 | 
			
		||||
		// overlap, sort by creation timestamp, subsort by name, then pick
 | 
			
		||||
		// the first.
 | 
			
		||||
		utilruntime.HandleError(fmt.Errorf("user error! more than one replication controller is selecting pods with labels: %+v", pod.Labels))
 | 
			
		||||
		sort.Sort(OverlappingControllers(controllers))
 | 
			
		||||
	if len(rcs) > 1 {
 | 
			
		||||
		// ControllerRef will ensure we don't do anything crazy, but more than one
 | 
			
		||||
		// item in this list nevertheless constitutes user error.
 | 
			
		||||
		utilruntime.HandleError(fmt.Errorf("user error! more than one ReplicationController is selecting pods with labels: %+v", pod.Labels))
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// update lookup cache
 | 
			
		||||
	rm.lookupCache.Update(pod, controllers[0])
 | 
			
		||||
 | 
			
		||||
	return controllers[0]
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// isCacheValid check if the cache is valid
 | 
			
		||||
func (rm *ReplicationManager) isCacheValid(pod *v1.Pod, cachedRC *v1.ReplicationController) bool {
 | 
			
		||||
	_, err := rm.rcLister.ReplicationControllers(cachedRC.Namespace).Get(cachedRC.Name)
 | 
			
		||||
	// rc has been deleted or updated, cache is invalid
 | 
			
		||||
	if err != nil || !isControllerMatch(pod, cachedRC) {
 | 
			
		||||
		return false
 | 
			
		||||
	}
 | 
			
		||||
	return true
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// isControllerMatch take a Pod and ReplicationController, return whether the Pod and ReplicationController are matching
 | 
			
		||||
// TODO(mqliang): This logic is a copy from GetPodControllers(), remove the duplication
 | 
			
		||||
func isControllerMatch(pod *v1.Pod, rc *v1.ReplicationController) bool {
 | 
			
		||||
	if rc.Namespace != pod.Namespace {
 | 
			
		||||
		return false
 | 
			
		||||
	}
 | 
			
		||||
	selector := labels.Set(rc.Spec.Selector).AsSelectorPreValidated()
 | 
			
		||||
 | 
			
		||||
	// If an rc with a nil or empty selector creeps in, it should match nothing, not everything.
 | 
			
		||||
	if selector.Empty() || !selector.Matches(labels.Set(pod.Labels)) {
 | 
			
		||||
		return false
 | 
			
		||||
	}
 | 
			
		||||
	return true
 | 
			
		||||
	return rcs
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// callback when RC is updated
 | 
			
		||||
@@ -239,20 +182,6 @@ func (rm *ReplicationManager) updateRC(old, cur interface{}) {
 | 
			
		||||
	oldRC := old.(*v1.ReplicationController)
 | 
			
		||||
	curRC := cur.(*v1.ReplicationController)
 | 
			
		||||
 | 
			
		||||
	// We should invalidate the whole lookup cache if a RC's selector has been updated.
 | 
			
		||||
	//
 | 
			
		||||
	// Imagine that you have two RCs:
 | 
			
		||||
	// * old RC1
 | 
			
		||||
	// * new RC2
 | 
			
		||||
	// You also have a pod that is attached to RC2 (because it doesn't match RC1 selector).
 | 
			
		||||
	// Now imagine that you are changing RC1 selector so that it is now matching that pod,
 | 
			
		||||
	// in such case, we must invalidate the whole cache so that pod could be adopted by RC1
 | 
			
		||||
	//
 | 
			
		||||
	// This makes the lookup cache less helpful, but selector update does not happen often,
 | 
			
		||||
	// so it's not a big problem
 | 
			
		||||
	if !reflect.DeepEqual(oldRC.Spec.Selector, curRC.Spec.Selector) {
 | 
			
		||||
		rm.lookupCache.InvalidateAll()
 | 
			
		||||
	}
 | 
			
		||||
	// TODO: Remove when #31981 is resolved!
 | 
			
		||||
	glog.Infof("Observed updated replication controller %v. Desired pod count change: %d->%d", curRC.Name, *(oldRC.Spec.Replicas), *(curRC.Spec.Replicas))
 | 
			
		||||
 | 
			
		||||
@@ -275,19 +204,10 @@ func (rm *ReplicationManager) updateRC(old, cur interface{}) {
 | 
			
		||||
	rm.enqueueController(cur)
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// When a pod is created, enqueue the controller that manages it and update it's expectations.
 | 
			
		||||
// When a pod is created, enqueue the ReplicationController that manages it and update its expectations.
 | 
			
		||||
func (rm *ReplicationManager) addPod(obj interface{}) {
 | 
			
		||||
	pod := obj.(*v1.Pod)
 | 
			
		||||
 | 
			
		||||
	rc := rm.getPodController(pod)
 | 
			
		||||
	if rc == nil {
 | 
			
		||||
		return
 | 
			
		||||
	}
 | 
			
		||||
	rcKey, err := controller.KeyFunc(rc)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		utilruntime.HandleError(fmt.Errorf("Couldn't get key for replication controller %#v: %v", rc, err))
 | 
			
		||||
		return
 | 
			
		||||
	}
 | 
			
		||||
	glog.V(4).Infof("Pod %s created: %#v.", pod.Name, pod)
 | 
			
		||||
 | 
			
		||||
	if pod.DeletionTimestamp != nil {
 | 
			
		||||
		// on a restart of the controller manager, it's possible a new pod shows up in a state that
 | 
			
		||||
@@ -295,13 +215,38 @@ func (rm *ReplicationManager) addPod(obj interface{}) {
 | 
			
		||||
		rm.deletePod(pod)
 | 
			
		||||
		return
 | 
			
		||||
	}
 | 
			
		||||
	rm.expectations.CreationObserved(rcKey)
 | 
			
		||||
	rm.enqueueController(rc)
 | 
			
		||||
 | 
			
		||||
	// If it has a ControllerRef, that's all that matters.
 | 
			
		||||
	if controllerRef := controller.GetControllerOf(pod); controllerRef != nil {
 | 
			
		||||
		if controllerRef.Kind != controllerKind.Kind {
 | 
			
		||||
			// It's controlled by a different type of controller.
 | 
			
		||||
			return
 | 
			
		||||
		}
 | 
			
		||||
		rc, err := rm.rcLister.ReplicationControllers(pod.Namespace).Get(controllerRef.Name)
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			return
 | 
			
		||||
		}
 | 
			
		||||
		rsKey, err := controller.KeyFunc(rc)
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			return
 | 
			
		||||
		}
 | 
			
		||||
		rm.expectations.CreationObserved(rsKey)
 | 
			
		||||
		rm.enqueueController(rc)
 | 
			
		||||
		return
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// Otherwise, it's an orphan. Get a list of all matching ReplicationControllers and sync
 | 
			
		||||
	// them to see if anyone wants to adopt it.
 | 
			
		||||
	// DO NOT observe creation because no controller should be waiting for an
 | 
			
		||||
	// orphan.
 | 
			
		||||
	for _, rc := range rm.getPodControllers(pod) {
 | 
			
		||||
		rm.enqueueController(rc)
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// When a pod is updated, figure out what controller/s manage it and wake them
 | 
			
		||||
// When a pod is updated, figure out what ReplicationController/s manage it and wake them
 | 
			
		||||
// up. If the labels of the pod have changed we need to awaken both the old
 | 
			
		||||
// and new controller. old and cur must be *v1.Pod types.
 | 
			
		||||
// and new ReplicationController. old and cur must be *v1.Pod types.
 | 
			
		||||
func (rm *ReplicationManager) updatePod(old, cur interface{}) {
 | 
			
		||||
	curPod := cur.(*v1.Pod)
 | 
			
		||||
	oldPod := old.(*v1.Pod)
 | 
			
		||||
@@ -311,6 +256,7 @@ func (rm *ReplicationManager) updatePod(old, cur interface{}) {
 | 
			
		||||
		return
 | 
			
		||||
	}
 | 
			
		||||
	glog.V(4).Infof("Pod %s updated, objectMeta %+v -> %+v.", curPod.Name, oldPod.ObjectMeta, curPod.ObjectMeta)
 | 
			
		||||
 | 
			
		||||
	labelChanged := !reflect.DeepEqual(curPod.Labels, oldPod.Labels)
 | 
			
		||||
	if curPod.DeletionTimestamp != nil {
 | 
			
		||||
		// when a pod is deleted gracefully it's deletion timestamp is first modified to reflect a grace period,
 | 
			
		||||
@@ -326,34 +272,53 @@ func (rm *ReplicationManager) updatePod(old, cur interface{}) {
 | 
			
		||||
		return
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// Only need to get the old controller if the labels changed.
 | 
			
		||||
	// Enqueue the oldRC before the curRC to give curRC a chance to adopt the oldPod.
 | 
			
		||||
	if labelChanged {
 | 
			
		||||
		// If the old and new rc are the same, the first one that syncs
 | 
			
		||||
		// will set expectations preventing any damage from the second.
 | 
			
		||||
		if oldRC := rm.getPodController(oldPod); oldRC != nil {
 | 
			
		||||
			rm.enqueueController(oldRC)
 | 
			
		||||
	curControllerRef := controller.GetControllerOf(curPod)
 | 
			
		||||
	oldControllerRef := controller.GetControllerOf(oldPod)
 | 
			
		||||
	controllerRefChanged := !reflect.DeepEqual(curControllerRef, oldControllerRef)
 | 
			
		||||
	if controllerRefChanged &&
 | 
			
		||||
		oldControllerRef != nil && oldControllerRef.Kind == controllerKind.Kind {
 | 
			
		||||
		// The ControllerRef was changed. Sync the old controller, if any.
 | 
			
		||||
		rc, err := rm.rcLister.ReplicationControllers(oldPod.Namespace).Get(oldControllerRef.Name)
 | 
			
		||||
		if err == nil {
 | 
			
		||||
			rm.enqueueController(rc)
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	changedToReady := !v1.IsPodReady(oldPod) && v1.IsPodReady(curPod)
 | 
			
		||||
	if curRC := rm.getPodController(curPod); curRC != nil {
 | 
			
		||||
		rm.enqueueController(curRC)
 | 
			
		||||
	// If it has a ControllerRef, that's all that matters.
 | 
			
		||||
	if curControllerRef != nil {
 | 
			
		||||
		if curControllerRef.Kind != controllerKind.Kind {
 | 
			
		||||
			// It's controlled by a different type of controller.
 | 
			
		||||
			return
 | 
			
		||||
		}
 | 
			
		||||
		rc, err := rm.rcLister.ReplicationControllers(curPod.Namespace).Get(curControllerRef.Name)
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			return
 | 
			
		||||
		}
 | 
			
		||||
		rm.enqueueController(rc)
 | 
			
		||||
		// TODO: MinReadySeconds in the Pod will generate an Available condition to be added in
 | 
			
		||||
		// the Pod status which in turn will trigger a requeue of the owning replication controller
 | 
			
		||||
		// thus having its status updated with the newly available replica. For now, we can fake the
 | 
			
		||||
		// update by resyncing the controller MinReadySeconds after the it is requeued because a Pod
 | 
			
		||||
		// transitioned to Ready.
 | 
			
		||||
		// the Pod status which in turn will trigger a requeue of the owning ReplicationController thus
 | 
			
		||||
		// having its status updated with the newly available replica. For now, we can fake the
 | 
			
		||||
		// update by resyncing the controller MinReadySeconds after the it is requeued because
 | 
			
		||||
		// a Pod transitioned to Ready.
 | 
			
		||||
		// Note that this still suffers from #29229, we are just moving the problem one level
 | 
			
		||||
		// "closer" to kubelet (from the deployment to the replication controller manager).
 | 
			
		||||
		if changedToReady && curRC.Spec.MinReadySeconds > 0 {
 | 
			
		||||
			glog.V(2).Infof("ReplicationController %q will be enqueued after %ds for availability check", curRC.Name, curRC.Spec.MinReadySeconds)
 | 
			
		||||
			rm.enqueueControllerAfter(curRC, time.Duration(curRC.Spec.MinReadySeconds)*time.Second)
 | 
			
		||||
		// "closer" to kubelet (from the deployment to the ReplicationController controller).
 | 
			
		||||
		if !v1.IsPodReady(oldPod) && v1.IsPodReady(curPod) && rc.Spec.MinReadySeconds > 0 {
 | 
			
		||||
			glog.V(2).Infof("ReplicationController %q will be enqueued after %ds for availability check", rc.Name, rc.Spec.MinReadySeconds)
 | 
			
		||||
			rm.enqueueControllerAfter(rc, time.Duration(rc.Spec.MinReadySeconds)*time.Second)
 | 
			
		||||
		}
 | 
			
		||||
		return
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// Otherwise, it's an orphan. If anything changed, sync matching controllers
 | 
			
		||||
	// to see if anyone wants to adopt it now.
 | 
			
		||||
	if labelChanged || controllerRefChanged {
 | 
			
		||||
		for _, rc := range rm.getPodControllers(curPod) {
 | 
			
		||||
			rm.enqueueController(rc)
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// When a pod is deleted, enqueue the controller that manages the pod and update its expectations.
 | 
			
		||||
// When a pod is deleted, enqueue the ReplicationController that manages the pod and update its expectations.
 | 
			
		||||
// obj could be an *v1.Pod, or a DeletionFinalStateUnknown marker item.
 | 
			
		||||
func (rm *ReplicationManager) deletePod(obj interface{}) {
 | 
			
		||||
	pod, ok := obj.(*v1.Pod)
 | 
			
		||||
@@ -361,45 +326,50 @@ func (rm *ReplicationManager) deletePod(obj interface{}) {
 | 
			
		||||
	// When a delete is dropped, the relist will notice a pod in the store not
 | 
			
		||||
	// in the list, leading to the insertion of a tombstone object which contains
 | 
			
		||||
	// the deleted key/value. Note that this value might be stale. If the pod
 | 
			
		||||
	// changed labels the new rc will not be woken up till the periodic resync.
 | 
			
		||||
	// changed labels the new ReplicationController will not be woken up till the periodic resync.
 | 
			
		||||
	if !ok {
 | 
			
		||||
		tombstone, ok := obj.(cache.DeletedFinalStateUnknown)
 | 
			
		||||
		if !ok {
 | 
			
		||||
			utilruntime.HandleError(fmt.Errorf("Couldn't get object from tombstone %#v", obj))
 | 
			
		||||
			utilruntime.HandleError(fmt.Errorf("couldn't get object from tombstone %+v", obj))
 | 
			
		||||
			return
 | 
			
		||||
		}
 | 
			
		||||
		pod, ok = tombstone.Obj.(*v1.Pod)
 | 
			
		||||
		if !ok {
 | 
			
		||||
			utilruntime.HandleError(fmt.Errorf("Tombstone contained object that is not a pod %#v", obj))
 | 
			
		||||
			utilruntime.HandleError(fmt.Errorf("tombstone contained object that is not a pod %#v", obj))
 | 
			
		||||
			return
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
	glog.V(4).Infof("Pod %s/%s deleted through %v, timestamp %+v, labels %+v.", pod.Namespace, pod.Name, utilruntime.GetCaller(), pod.DeletionTimestamp, pod.Labels)
 | 
			
		||||
	if rc := rm.getPodController(pod); rc != nil {
 | 
			
		||||
		rcKey, err := controller.KeyFunc(rc)
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			utilruntime.HandleError(fmt.Errorf("Couldn't get key for replication controller %#v: %v", rc, err))
 | 
			
		||||
			return
 | 
			
		||||
		}
 | 
			
		||||
		rm.expectations.DeletionObserved(rcKey, controller.PodKey(pod))
 | 
			
		||||
		rm.enqueueController(rc)
 | 
			
		||||
	glog.V(4).Infof("Pod %s/%s deleted through %v, timestamp %+v: %#v.", pod.Namespace, pod.Name, utilruntime.GetCaller(), pod.DeletionTimestamp, pod)
 | 
			
		||||
 | 
			
		||||
	controllerRef := controller.GetControllerOf(pod)
 | 
			
		||||
	if controllerRef == nil {
 | 
			
		||||
		// No controller should care about orphans being deleted.
 | 
			
		||||
		return
 | 
			
		||||
	}
 | 
			
		||||
	if controllerRef.Kind != controllerKind.Kind {
 | 
			
		||||
		// It's controlled by a different type of controller.
 | 
			
		||||
		return
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	rc, err := rm.rcLister.ReplicationControllers(pod.Namespace).Get(controllerRef.Name)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		return
 | 
			
		||||
	}
 | 
			
		||||
	rsKey, err := controller.KeyFunc(rc)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		return
 | 
			
		||||
	}
 | 
			
		||||
	rm.expectations.DeletionObserved(rsKey, controller.PodKey(pod))
 | 
			
		||||
	rm.enqueueController(rc)
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// obj could be an *v1.ReplicationController, or a DeletionFinalStateUnknown marker item.
 | 
			
		||||
func (rm *ReplicationManager) enqueueController(obj interface{}) {
 | 
			
		||||
	key, err := controller.KeyFunc(obj)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		utilruntime.HandleError(fmt.Errorf("Couldn't get key for object %+v: %v", obj, err))
 | 
			
		||||
		utilruntime.HandleError(fmt.Errorf("couldn't get key for object %+v: %v", obj, err))
 | 
			
		||||
		return
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// TODO: Handle overlapping controllers better. Either disallow them at admission time or
 | 
			
		||||
	// deterministically avoid syncing controllers that fight over pods. Currently, we only
 | 
			
		||||
	// ensure that the same controller is synced for a given pod. When we periodically relist
 | 
			
		||||
	// all controllers there will still be some replica instability. One way to handle this is
 | 
			
		||||
	// by querying the store for all controllers that this rc overlaps, as well as all
 | 
			
		||||
	// controllers that overlap this rc, and sorting them.
 | 
			
		||||
	rm.queue.Add(key)
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
@@ -407,16 +377,9 @@ func (rm *ReplicationManager) enqueueController(obj interface{}) {
 | 
			
		||||
func (rm *ReplicationManager) enqueueControllerAfter(obj interface{}, after time.Duration) {
 | 
			
		||||
	key, err := controller.KeyFunc(obj)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		utilruntime.HandleError(fmt.Errorf("Couldn't get key for object %+v: %v", obj, err))
 | 
			
		||||
		utilruntime.HandleError(fmt.Errorf("couldn't get key for object %+v: %v", obj, err))
 | 
			
		||||
		return
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// TODO: Handle overlapping controllers better. Either disallow them at admission time or
 | 
			
		||||
	// deterministically avoid syncing controllers that fight over pods. Currently, we only
 | 
			
		||||
	// ensure that the same controller is synced for a given pod. When we periodically relist
 | 
			
		||||
	// all controllers there will still be some replica instability. One way to handle this is
 | 
			
		||||
	// by querying the store for all controllers that this rc overlaps, as well as all
 | 
			
		||||
	// controllers that overlap this rc, and sorting them.
 | 
			
		||||
	rm.queue.AddAfter(key, after)
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
@@ -481,8 +444,8 @@ func (rm *ReplicationManager) manageReplicas(filteredPods []*v1.Pod, rc *v1.Repl
 | 
			
		||||
				var err error
 | 
			
		||||
				boolPtr := func(b bool) *bool { return &b }
 | 
			
		||||
				controllerRef := &metav1.OwnerReference{
 | 
			
		||||
					APIVersion:         getRCKind().GroupVersion().String(),
 | 
			
		||||
					Kind:               getRCKind().Kind,
 | 
			
		||||
					APIVersion:         controllerKind.GroupVersion().String(),
 | 
			
		||||
					Kind:               controllerKind.Kind,
 | 
			
		||||
					Name:               rc.Name,
 | 
			
		||||
					UID:                rc.UID,
 | 
			
		||||
					BlockOwnerDeletion: boolPtr(true),
 | 
			
		||||
@@ -610,7 +573,7 @@ func (rm *ReplicationManager) syncReplicationController(key string) error {
 | 
			
		||||
		rm.queue.Add(key)
 | 
			
		||||
		return err
 | 
			
		||||
	}
 | 
			
		||||
	cm := controller.NewPodControllerRefManager(rm.podControl, rc, labels.Set(rc.Spec.Selector).AsSelectorPreValidated(), getRCKind())
 | 
			
		||||
	cm := controller.NewPodControllerRefManager(rm.podControl, rc, labels.Set(rc.Spec.Selector).AsSelectorPreValidated(), controllerKind)
 | 
			
		||||
	filteredPods, err = cm.ClaimPods(pods)
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		// Something went wrong with adoption or release.
 | 
			
		||||
 
 | 
			
		||||
		Reference in New Issue
	
	Block a user