let RC manager utilize the GC
This commit is contained in:
@@ -26,6 +26,9 @@ import (
|
||||
|
||||
"github.com/golang/glog"
|
||||
"k8s.io/kubernetes/pkg/api"
|
||||
"k8s.io/kubernetes/pkg/api/errors"
|
||||
"k8s.io/kubernetes/pkg/api/unversioned"
|
||||
"k8s.io/kubernetes/pkg/api/v1"
|
||||
"k8s.io/kubernetes/pkg/client/cache"
|
||||
clientset "k8s.io/kubernetes/pkg/client/clientset_generated/internalclientset"
|
||||
unversionedcore "k8s.io/kubernetes/pkg/client/clientset_generated/internalclientset/typed/core/unversioned"
|
||||
@@ -36,6 +39,7 @@ import (
|
||||
"k8s.io/kubernetes/pkg/labels"
|
||||
"k8s.io/kubernetes/pkg/runtime"
|
||||
"k8s.io/kubernetes/pkg/util"
|
||||
utilerrors "k8s.io/kubernetes/pkg/util/errors"
|
||||
"k8s.io/kubernetes/pkg/util/metrics"
|
||||
utilruntime "k8s.io/kubernetes/pkg/util/runtime"
|
||||
"k8s.io/kubernetes/pkg/util/wait"
|
||||
@@ -65,6 +69,10 @@ const (
|
||||
statusUpdateRetries = 1
|
||||
)
|
||||
|
||||
func getRCKind() unversioned.GroupVersionKind {
|
||||
return v1.SchemeGroupVersion.WithKind("ReplicationController")
|
||||
}
|
||||
|
||||
// ReplicationManager is responsible for synchronizing ReplicationController objects stored
|
||||
// in the system with actual running pods.
|
||||
// TODO: this really should be called ReplicationController. The only reason why it's a Manager
|
||||
@@ -105,20 +113,24 @@ type ReplicationManager struct {
|
||||
|
||||
// Controllers that need to be synced
|
||||
queue *workqueue.Type
|
||||
|
||||
// garbageCollectorEnabled denotes if the garbage collector is enabled. RC
|
||||
// manager behaves differently if GC is enabled.
|
||||
garbageCollectorEnabled bool
|
||||
}
|
||||
|
||||
// NewReplicationManager creates a replication manager
|
||||
func NewReplicationManager(podInformer framework.SharedIndexInformer, kubeClient clientset.Interface, resyncPeriod controller.ResyncPeriodFunc, burstReplicas int, lookupCacheSize int) *ReplicationManager {
|
||||
func NewReplicationManager(podInformer framework.SharedIndexInformer, kubeClient clientset.Interface, resyncPeriod controller.ResyncPeriodFunc, burstReplicas int, lookupCacheSize int, garbageCollectorEnabled bool) *ReplicationManager {
|
||||
eventBroadcaster := record.NewBroadcaster()
|
||||
eventBroadcaster.StartLogging(glog.Infof)
|
||||
eventBroadcaster.StartRecordingToSink(&unversionedcore.EventSinkImpl{Interface: kubeClient.Core().Events("")})
|
||||
return newReplicationManagerInternal(
|
||||
return newReplicationManager(
|
||||
eventBroadcaster.NewRecorder(api.EventSource{Component: "replication-controller"}),
|
||||
podInformer, kubeClient, resyncPeriod, burstReplicas, lookupCacheSize)
|
||||
podInformer, kubeClient, resyncPeriod, burstReplicas, lookupCacheSize, garbageCollectorEnabled)
|
||||
}
|
||||
|
||||
// newReplicationManagerInternal configures a replication manager with the specified event recorder
|
||||
func newReplicationManagerInternal(eventRecorder record.EventRecorder, podInformer framework.SharedIndexInformer, kubeClient clientset.Interface, resyncPeriod controller.ResyncPeriodFunc, burstReplicas int, lookupCacheSize int) *ReplicationManager {
|
||||
// newReplicationManager configures a replication manager with the specified event recorder
|
||||
func newReplicationManager(eventRecorder record.EventRecorder, podInformer framework.SharedIndexInformer, kubeClient clientset.Interface, resyncPeriod controller.ResyncPeriodFunc, burstReplicas int, lookupCacheSize int, garbageCollectorEnabled bool) *ReplicationManager {
|
||||
if kubeClient != nil && kubeClient.Core().GetRESTClient().GetRateLimiter() != nil {
|
||||
metrics.RegisterMetricAndTrackRateLimiterUsage("replication_controller", kubeClient.Core().GetRESTClient().GetRateLimiter())
|
||||
}
|
||||
@@ -132,6 +144,7 @@ func newReplicationManagerInternal(eventRecorder record.EventRecorder, podInform
|
||||
burstReplicas: burstReplicas,
|
||||
expectations: controller.NewUIDTrackingControllerExpectations(controller.NewControllerExpectations()),
|
||||
queue: workqueue.New(),
|
||||
garbageCollectorEnabled: garbageCollectorEnabled,
|
||||
}
|
||||
|
||||
rm.rcStore.Indexer, rm.rcController = framework.NewIndexerInformer(
|
||||
@@ -147,43 +160,8 @@ func newReplicationManagerInternal(eventRecorder record.EventRecorder, podInform
|
||||
// TODO: Can we have much longer period here?
|
||||
FullControllerResyncPeriod,
|
||||
framework.ResourceEventHandlerFuncs{
|
||||
AddFunc: rm.enqueueController,
|
||||
UpdateFunc: func(old, cur interface{}) {
|
||||
oldRC := old.(*api.ReplicationController)
|
||||
curRC := cur.(*api.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()
|
||||
}
|
||||
|
||||
// You might imagine that we only really need to enqueue the
|
||||
// controller when Spec changes, but it is safer to sync any
|
||||
// time this function is triggered. That way a full informer
|
||||
// resync can requeue any controllers that don't yet have pods
|
||||
// but whose last attempts at creating a pod have failed (since
|
||||
// we don't block on creation of pods) instead of those
|
||||
// controllers stalling indefinitely. Enqueueing every time
|
||||
// does result in some spurious syncs (like when Status.Replica
|
||||
// is updated and the watch notification from it retriggers
|
||||
// this function), but in general extra resyncs shouldn't be
|
||||
// that bad as rcs that haven't met expectations yet won't
|
||||
// sync, and all the listing is done using local stores.
|
||||
if oldRC.Status.Replicas != curRC.Status.Replicas {
|
||||
glog.V(4).Infof("Observed updated replica count for rc: %v, %d->%d", curRC.Name, oldRC.Status.Replicas, curRC.Status.Replicas)
|
||||
}
|
||||
rm.enqueueController(cur)
|
||||
},
|
||||
AddFunc: rm.enqueueController,
|
||||
UpdateFunc: rm.updateRC,
|
||||
// This will enter the sync loop and no-op, because the controller has been deleted from the store.
|
||||
// Note that deleting a controller immediately after scaling it to 0 will not work. The recommended
|
||||
// way of achieving this is by performing a `stop` operation on the controller.
|
||||
@@ -212,7 +190,8 @@ func newReplicationManagerInternal(eventRecorder record.EventRecorder, podInform
|
||||
// NewReplicationManagerFromClientForIntegration creates a new ReplicationManager that runs its own informer. It disables event recording for use in integration tests.
|
||||
func NewReplicationManagerFromClientForIntegration(kubeClient clientset.Interface, resyncPeriod controller.ResyncPeriodFunc, burstReplicas int, lookupCacheSize int) *ReplicationManager {
|
||||
podInformer := informers.CreateSharedPodIndexInformer(kubeClient, resyncPeriod())
|
||||
rm := newReplicationManagerInternal(&record.FakeRecorder{}, podInformer, kubeClient, resyncPeriod, burstReplicas, lookupCacheSize)
|
||||
garbageCollectorEnabled := false
|
||||
rm := newReplicationManager(&record.FakeRecorder{}, podInformer, kubeClient, resyncPeriod, burstReplicas, lookupCacheSize, garbageCollectorEnabled)
|
||||
rm.internalPodInformer = podInformer
|
||||
return rm
|
||||
}
|
||||
@@ -220,7 +199,8 @@ func NewReplicationManagerFromClientForIntegration(kubeClient clientset.Interfac
|
||||
// NewReplicationManagerFromClient creates a new ReplicationManager that runs its own informer.
|
||||
func NewReplicationManagerFromClient(kubeClient clientset.Interface, resyncPeriod controller.ResyncPeriodFunc, burstReplicas int, lookupCacheSize int) *ReplicationManager {
|
||||
podInformer := informers.CreateSharedPodIndexInformer(kubeClient, resyncPeriod())
|
||||
rm := NewReplicationManager(podInformer, kubeClient, resyncPeriod, burstReplicas, lookupCacheSize)
|
||||
garbageCollectorEnabled := false
|
||||
rm := NewReplicationManager(podInformer, kubeClient, resyncPeriod, burstReplicas, lookupCacheSize, garbageCollectorEnabled)
|
||||
rm.internalPodInformer = podInformer
|
||||
|
||||
return rm
|
||||
@@ -255,13 +235,14 @@ func (rm *ReplicationManager) Run(workers int, stopCh <-chan struct{}) {
|
||||
|
||||
// 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 *api.Pod) *api.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.(*api.ReplicationController)
|
||||
if !ok {
|
||||
// This should not happen
|
||||
glog.Errorf("lookup cache does not retuen a ReplicationController object")
|
||||
glog.Errorf("lookup cache does not return a ReplicationController object")
|
||||
return nil
|
||||
}
|
||||
if cached && rm.isCacheValid(pod, controller) {
|
||||
@@ -320,6 +301,44 @@ func isControllerMatch(pod *api.Pod, rc *api.ReplicationController) bool {
|
||||
return true
|
||||
}
|
||||
|
||||
// callback when RC is updated
|
||||
func (rm *ReplicationManager) updateRC(old, cur interface{}) {
|
||||
oldRC := old.(*api.ReplicationController)
|
||||
curRC := cur.(*api.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()
|
||||
}
|
||||
|
||||
// You might imagine that we only really need to enqueue the
|
||||
// controller when Spec changes, but it is safer to sync any
|
||||
// time this function is triggered. That way a full informer
|
||||
// resync can requeue any controllers that don't yet have pods
|
||||
// but whose last attempts at creating a pod have failed (since
|
||||
// we don't block on creation of pods) instead of those
|
||||
// controllers stalling indefinitely. Enqueueing every time
|
||||
// does result in some spurious syncs (like when Status.Replica
|
||||
// is updated and the watch notification from it retriggers
|
||||
// this function), but in general extra resyncs shouldn't be
|
||||
// that bad as rcs that haven't met expectations yet won't
|
||||
// sync, and all the listing is done using local stores.
|
||||
if oldRC.Status.Replicas != curRC.Status.Replicas {
|
||||
glog.V(4).Infof("Observed updated replica count for rc: %v, %d->%d", curRC.Name, oldRC.Status.Replicas, curRC.Status.Replicas)
|
||||
}
|
||||
rm.enqueueController(cur)
|
||||
}
|
||||
|
||||
// When a pod is created, enqueue the controller that manages it and update it's expectations.
|
||||
func (rm *ReplicationManager) addPod(obj interface{}) {
|
||||
pod := obj.(*api.Pod)
|
||||
@@ -370,10 +389,8 @@ func (rm *ReplicationManager) updatePod(old, cur interface{}) {
|
||||
return
|
||||
}
|
||||
|
||||
if rc := rm.getPodController(curPod); rc != nil {
|
||||
rm.enqueueController(rc)
|
||||
}
|
||||
// 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.
|
||||
@@ -381,6 +398,10 @@ func (rm *ReplicationManager) updatePod(old, cur interface{}) {
|
||||
rm.enqueueController(oldRC)
|
||||
}
|
||||
}
|
||||
|
||||
if curRC := rm.getPodController(curPod); curRC != nil {
|
||||
rm.enqueueController(curRC)
|
||||
}
|
||||
}
|
||||
|
||||
// When a pod is deleted, enqueue the controller that manages the pod and update its expectations.
|
||||
@@ -481,7 +502,21 @@ func (rm *ReplicationManager) manageReplicas(filteredPods []*api.Pod, rc *api.Re
|
||||
for i := 0; i < diff; i++ {
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
if err := rm.podControl.CreatePods(rc.Namespace, rc.Spec.Template, rc); err != nil {
|
||||
var err error
|
||||
if rm.garbageCollectorEnabled {
|
||||
var trueVar = true
|
||||
controllerRef := &api.OwnerReference{
|
||||
APIVersion: getRCKind().GroupVersion().String(),
|
||||
Kind: getRCKind().Kind,
|
||||
Name: rc.Name,
|
||||
UID: rc.UID,
|
||||
Controller: &trueVar,
|
||||
}
|
||||
err = rm.podControl.CreatePodsWithControllerRef(rc.Namespace, rc.Spec.Template, rc, controllerRef)
|
||||
} else {
|
||||
err = rm.podControl.CreatePods(rc.Namespace, rc.Spec.Template, rc)
|
||||
}
|
||||
if err != nil {
|
||||
// Decrement the expected number of creates because the informer won't observe this pod
|
||||
glog.V(2).Infof("Failed creation, decrementing expectations for controller %q/%q", rc.Namespace, rc.Name)
|
||||
rm.expectations.CreationObserved(rcKey)
|
||||
@@ -580,7 +615,6 @@ func (rm *ReplicationManager) syncReplicationController(key string) error {
|
||||
trace.Step("ReplicationController restored")
|
||||
rcNeedsSync := rm.expectations.SatisfiedExpectations(rcKey)
|
||||
trace.Step("Expectations restored")
|
||||
podList, err := rm.podStore.Pods(rc.Namespace).List(labels.Set(rc.Spec.Selector).AsSelector())
|
||||
if err != nil {
|
||||
glog.Errorf("Error getting pods for rc %q: %v", key, err)
|
||||
rm.queue.Add(key)
|
||||
@@ -588,9 +622,59 @@ func (rm *ReplicationManager) syncReplicationController(key string) error {
|
||||
}
|
||||
trace.Step("Pods listed")
|
||||
|
||||
// TODO: Do this in a single pass, or use an index.
|
||||
filteredPods := controller.FilterActivePods(podList.Items)
|
||||
if rcNeedsSync {
|
||||
// TODO: Do the List and Filter in a single pass, or use an index.
|
||||
var filteredPods []*api.Pod
|
||||
if rm.garbageCollectorEnabled {
|
||||
// list all pods to include the pods that don't match the rc's selector
|
||||
// anymore but has the stale controller ref.
|
||||
podList, err := rm.podStore.Pods(rc.Namespace).List(labels.Everything())
|
||||
if err != nil {
|
||||
glog.Errorf("Error getting pods for rc %q: %v", key, err)
|
||||
rm.queue.Add(key)
|
||||
return err
|
||||
}
|
||||
cm := controller.NewPodControllerRefManager(rm.podControl, rc.ObjectMeta, labels.Set(rc.Spec.Selector).AsSelector(), getRCKind())
|
||||
matchesAndControlled, matchesNeedsController, controlledDoesNotMatch := cm.Classify(podList.Items)
|
||||
for _, pod := range matchesNeedsController {
|
||||
err := cm.AdoptPod(pod)
|
||||
// continue to next pod if adoption fails.
|
||||
if err != nil {
|
||||
// If the pod no longer exists, don't even log the error.
|
||||
if !errors.IsNotFound(err) {
|
||||
utilruntime.HandleError(err)
|
||||
}
|
||||
} else {
|
||||
matchesAndControlled = append(matchesAndControlled, pod)
|
||||
}
|
||||
}
|
||||
filteredPods = matchesAndControlled
|
||||
// remove the controllerRef for the pods that no longer have matching labels
|
||||
var errlist []error
|
||||
for _, pod := range controlledDoesNotMatch {
|
||||
err := cm.ReleasePod(pod)
|
||||
if err != nil {
|
||||
errlist = append(errlist, cm.ReleasePod(pod))
|
||||
}
|
||||
}
|
||||
if len(errlist) != 0 {
|
||||
aggregate := utilerrors.NewAggregate(errlist)
|
||||
// push the RC into work queue again. We need to try to free the
|
||||
// pods again otherwise they will stuck with the stale
|
||||
// controllerRef.
|
||||
rm.queue.Add(key)
|
||||
return aggregate
|
||||
}
|
||||
} else {
|
||||
podList, err := rm.podStore.Pods(rc.Namespace).List(labels.Set(rc.Spec.Selector).AsSelector())
|
||||
if err != nil {
|
||||
glog.Errorf("Error getting pods for rc %q: %v", key, err)
|
||||
rm.queue.Add(key)
|
||||
return err
|
||||
}
|
||||
filteredPods = controller.FilterActivePods(podList.Items)
|
||||
}
|
||||
|
||||
if rcNeedsSync && rc.DeletionTimestamp == nil {
|
||||
rm.manageReplicas(filteredPods, &rc)
|
||||
}
|
||||
trace.Step("manageReplicas done")
|
||||
|
Reference in New Issue
Block a user