VPA: refactor in-place and eviction logic

This commit refactors inplace logic outside of the pods eviction restriction and separates them into their own files.
Also this commit adds PatchResourceTarget to calculators to allow them to explictly specify to the caller which
resource/subresource they should be patched to. This commit also creates a utils subpackage in order to prevent
dependency cycles in the unit tests, and adds various unit tests. Lastly, this commit adds a rateLimiter specifically
for limiting inPlaceResize API calls.

Signed-off-by: Max Cao <macao@redhat.com>
This commit is contained in:
Max Cao 2025-04-03 12:56:02 -07:00
parent 15883dce79
commit 9eac8fc5c5
No known key found for this signature in database
GPG Key ID: 4EAEC3318EC211D2
20 changed files with 1852 additions and 1223 deletions

View File

@ -54,6 +54,10 @@ type fakePatchCalculator struct {
err error
}
func (*fakePatchCalculator) PatchResourceTarget() patch.PatchResourceTarget {
return patch.Pod
}
func (c *fakePatchCalculator) CalculatePatches(_ *apiv1.Pod, _ *vpa_types.VerticalPodAutoscaler) (
[]resource_admission.PatchRecord, error) {
return c.patches, c.err

View File

@ -23,7 +23,22 @@ import (
vpa_types "k8s.io/autoscaler/vertical-pod-autoscaler/pkg/apis/autoscaling.k8s.io/v1"
)
// PatchResourceTarget is the type of resource that can be patched.
type PatchResourceTarget string
const (
// Pod refers to the pod resource itself.
Pod PatchResourceTarget = "Pod"
// Resize refers to the resize subresource of the pod.
Resize PatchResourceTarget = "Resize"
// Future subresources can be added here.
// e.g. Status PatchResourceTarget = "Status"
)
// Calculator is capable of calculating required patches for pod.
type Calculator interface {
CalculatePatches(pod *core.Pod, vpa *vpa_types.VerticalPodAutoscaler) ([]resource.PatchRecord, error)
// PatchResourceTarget returns the resource this calculator should calculate patches for.
PatchResourceTarget() PatchResourceTarget
}

View File

@ -31,6 +31,10 @@ func (*observedContainers) CalculatePatches(pod *core.Pod, _ *vpa_types.Vertical
return []resource_admission.PatchRecord{GetAddAnnotationPatch(annotations.VpaObservedContainersLabel, vpaObservedContainersValue)}, nil
}
func (*observedContainers) PatchResourceTarget() PatchResourceTarget {
return Pod
}
// NewObservedContainersCalculator returns calculator for
// observed containers patches.
func NewObservedContainersCalculator() Calculator {

View File

@ -47,6 +47,10 @@ func NewResourceUpdatesCalculator(recommendationProvider recommendation.Provider
}
}
func (*resourcesUpdatesPatchCalculator) PatchResourceTarget() PatchResourceTarget {
return Pod
}
func (c *resourcesUpdatesPatchCalculator) CalculatePatches(pod *core.Pod, vpa *vpa_types.VerticalPodAutoscaler) ([]resource_admission.PatchRecord, error) {
result := []resource_admission.PatchRecord{}

View File

@ -1,570 +0,0 @@
/*
Copyright 2017 The Kubernetes 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 eviction
import (
"context"
"encoding/json"
"fmt"
"time"
appsv1 "k8s.io/api/apps/v1"
apiv1 "k8s.io/api/core/v1"
policyv1 "k8s.io/api/policy/v1"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
k8stypes "k8s.io/apimachinery/pkg/types"
appsinformer "k8s.io/client-go/informers/apps/v1"
coreinformer "k8s.io/client-go/informers/core/v1"
kube_client "k8s.io/client-go/kubernetes"
"k8s.io/client-go/tools/cache"
"k8s.io/client-go/tools/record"
"k8s.io/klog/v2"
resource_updates "k8s.io/autoscaler/vertical-pod-autoscaler/pkg/admission-controller/resource"
"k8s.io/autoscaler/vertical-pod-autoscaler/pkg/admission-controller/resource/pod/patch"
"k8s.io/autoscaler/vertical-pod-autoscaler/pkg/features"
vpa_types "k8s.io/autoscaler/vertical-pod-autoscaler/pkg/apis/autoscaling.k8s.io/v1"
)
const (
resyncPeriod time.Duration = 1 * time.Minute
)
// PodsEvictionRestriction controls pods evictions. It ensures that we will not evict too
// many pods from one replica set. For replica set will allow to evict one pod or more if
// evictionToleranceFraction is configured.
type PodsEvictionRestriction interface {
// Evict sends eviction instruction to the api client.
// Returns error if pod cannot be evicted or if client returned error.
Evict(pod *apiv1.Pod, vpa *vpa_types.VerticalPodAutoscaler, eventRecorder record.EventRecorder) error
// CanEvict checks if pod can be safely evicted
CanEvict(pod *apiv1.Pod) bool
// InPlaceUpdate updates the pod resources in-place
InPlaceUpdate(pod *apiv1.Pod, vpa *vpa_types.VerticalPodAutoscaler, eventRecorder record.EventRecorder) error
// CanInPlaceUpdate checks if the pod can be updated in-place
CanInPlaceUpdate(pod *apiv1.Pod) bool
}
type podsEvictionRestrictionImpl struct {
client kube_client.Interface
podToReplicaCreatorMap map[string]podReplicaCreator
creatorToSingleGroupStatsMap map[podReplicaCreator]singleGroupStats
patchCalculators []patch.Calculator
}
type singleGroupStats struct {
configured int
pending int
running int
evictionTolerance int
evicted int
inPlaceUpdating int
}
// PodsEvictionRestrictionFactory creates PodsEvictionRestriction
type PodsEvictionRestrictionFactory interface {
// NewPodsEvictionRestriction creates PodsEvictionRestriction for given set of pods,
// controlled by a single VPA object.
NewPodsEvictionRestriction(pods []*apiv1.Pod, vpa *vpa_types.VerticalPodAutoscaler, patchCalculators []patch.Calculator) PodsEvictionRestriction
}
type podsEvictionRestrictionFactoryImpl struct {
client kube_client.Interface
rcInformer cache.SharedIndexInformer // informer for Replication Controllers
ssInformer cache.SharedIndexInformer // informer for Stateful Sets
rsInformer cache.SharedIndexInformer // informer for Replica Sets
dsInformer cache.SharedIndexInformer // informer for Daemon Sets
minReplicas int
evictionToleranceFraction float64
}
type controllerKind string
const (
replicationController controllerKind = "ReplicationController"
statefulSet controllerKind = "StatefulSet"
replicaSet controllerKind = "ReplicaSet"
daemonSet controllerKind = "DaemonSet"
job controllerKind = "Job"
)
type podReplicaCreator struct {
Namespace string
Name string
Kind controllerKind
}
// CanEvict checks if pod can be safely evicted
func (e *podsEvictionRestrictionImpl) CanEvict(pod *apiv1.Pod) bool {
cr, present := e.podToReplicaCreatorMap[GetPodID(pod)]
if present {
singleGroupStats, present := e.creatorToSingleGroupStatsMap[cr]
if pod.Status.Phase == apiv1.PodPending {
return true
}
if present {
shouldBeAlive := singleGroupStats.configured - singleGroupStats.evictionTolerance
actuallyAlive := singleGroupStats.running - (singleGroupStats.evicted + singleGroupStats.inPlaceUpdating)
klog.V(4).InfoS("Pod disruption tolerance",
"pod", klog.KObj(pod),
"running", singleGroupStats.running,
"configured", singleGroupStats.configured,
"tolerance", singleGroupStats.evictionTolerance,
"evicted", singleGroupStats.evicted,
"updating", singleGroupStats.inPlaceUpdating)
if IsInPlaceUpdating(pod) {
if (actuallyAlive - 1) > shouldBeAlive { // -1 because this pod is the one being in-place updated
if pod.Status.Resize == apiv1.PodResizeStatusInfeasible || pod.Status.Resize == apiv1.PodResizeStatusDeferred {
klog.InfoS("Attempted in-place resize was impossible, should now evict", "pod", klog.KObj(pod), "resizePolicy", pod.Status.Resize)
return true
}
}
klog.V(4).InfoS("Would be able to evict, but already resizing", "pod", klog.KObj(pod))
return false
}
if actuallyAlive > shouldBeAlive {
return true
}
// If all pods are running and eviction tolerance is small evict 1 pod.
if singleGroupStats.running == singleGroupStats.configured &&
singleGroupStats.evictionTolerance == 0 &&
singleGroupStats.evicted == 0 &&
singleGroupStats.inPlaceUpdating == 0 {
return true
}
}
}
return false
}
// Evict sends eviction instruction to api client. Returns error if pod cannot be evicted or if client returned error
// Does not check if pod was actually evicted after eviction grace period.
func (e *podsEvictionRestrictionImpl) Evict(podToEvict *apiv1.Pod, vpa *vpa_types.VerticalPodAutoscaler, eventRecorder record.EventRecorder) error {
cr, present := e.podToReplicaCreatorMap[GetPodID(podToEvict)]
if !present {
return fmt.Errorf("pod not suitable for eviction %s/%s: not in replicated pods map", podToEvict.Namespace, podToEvict.Name)
}
if !e.CanEvict(podToEvict) {
return fmt.Errorf("cannot evict pod %s/%s: eviction budget exceeded", podToEvict.Namespace, podToEvict.Name)
}
eviction := &policyv1.Eviction{
ObjectMeta: metav1.ObjectMeta{
Namespace: podToEvict.Namespace,
Name: podToEvict.Name,
},
}
err := e.client.CoreV1().Pods(podToEvict.Namespace).EvictV1(context.TODO(), eviction)
if err != nil {
klog.ErrorS(err, "Failed to evict pod", "pod", klog.KObj(podToEvict))
return err
}
eventRecorder.Event(podToEvict, apiv1.EventTypeNormal, "EvictedByVPA",
"Pod was evicted by VPA Updater to apply resource recommendation.")
eventRecorder.Event(vpa, apiv1.EventTypeNormal, "EvictedPod",
"VPA Updater evicted Pod "+podToEvict.Name+" to apply resource recommendation.")
if podToEvict.Status.Phase != apiv1.PodPending {
singleGroupStats, present := e.creatorToSingleGroupStatsMap[cr]
if !present {
return fmt.Errorf("Internal error - cannot find stats for replication group %v", cr)
}
singleGroupStats.evicted = singleGroupStats.evicted + 1
e.creatorToSingleGroupStatsMap[cr] = singleGroupStats
}
return nil
}
// NewPodsEvictionRestrictionFactory creates PodsEvictionRestrictionFactory
func NewPodsEvictionRestrictionFactory(client kube_client.Interface, minReplicas int,
evictionToleranceFraction float64) (PodsEvictionRestrictionFactory, error) {
rcInformer, err := setUpInformer(client, replicationController)
if err != nil {
return nil, fmt.Errorf("Failed to create rcInformer: %v", err)
}
ssInformer, err := setUpInformer(client, statefulSet)
if err != nil {
return nil, fmt.Errorf("Failed to create ssInformer: %v", err)
}
rsInformer, err := setUpInformer(client, replicaSet)
if err != nil {
return nil, fmt.Errorf("Failed to create rsInformer: %v", err)
}
dsInformer, err := setUpInformer(client, daemonSet)
if err != nil {
return nil, fmt.Errorf("Failed to create dsInformer: %v", err)
}
return &podsEvictionRestrictionFactoryImpl{
client: client,
rcInformer: rcInformer, // informer for Replication Controllers
ssInformer: ssInformer, // informer for Replica Sets
rsInformer: rsInformer, // informer for Stateful Sets
dsInformer: dsInformer, // informer for Daemon Sets
minReplicas: minReplicas,
evictionToleranceFraction: evictionToleranceFraction}, nil
}
// NewPodsEvictionRestriction creates PodsEvictionRestriction for a given set of pods,
// controlled by a single VPA object.
func (f *podsEvictionRestrictionFactoryImpl) NewPodsEvictionRestriction(pods []*apiv1.Pod, vpa *vpa_types.VerticalPodAutoscaler, patchCalculators []patch.Calculator) PodsEvictionRestriction {
// We can evict pod only if it is a part of replica set
// For each replica set we can evict only a fraction of pods.
// Evictions may be later limited by pod disruption budget if configured.
livePods := make(map[podReplicaCreator][]*apiv1.Pod)
for _, pod := range pods {
creator, err := getPodReplicaCreator(pod)
if err != nil {
klog.ErrorS(err, "Failed to obtain replication info for pod", "pod", klog.KObj(pod))
continue
}
if creator == nil {
klog.V(0).InfoS("Pod is not managed by any controller", "pod", klog.KObj(pod))
continue
}
livePods[*creator] = append(livePods[*creator], pod)
}
podToReplicaCreatorMap := make(map[string]podReplicaCreator)
creatorToSingleGroupStatsMap := make(map[podReplicaCreator]singleGroupStats)
// Use per-VPA minReplicas if present, fall back to the global setting.
required := f.minReplicas
if vpa.Spec.UpdatePolicy != nil && vpa.Spec.UpdatePolicy.MinReplicas != nil {
required = int(*vpa.Spec.UpdatePolicy.MinReplicas)
klog.V(3).InfoS("Overriding minReplicas from global to per-VPA value", "globalMinReplicas", f.minReplicas, "vpaMinReplicas", required, "vpa", klog.KObj(vpa))
}
for creator, replicas := range livePods {
actual := len(replicas)
if actual < required {
klog.V(2).InfoS("Too few replicas", "kind", creator.Kind, "object", klog.KRef(creator.Namespace, creator.Name), "livePods", actual, "requiredPods", required, "globalMinReplicas", f.minReplicas)
continue
}
var configured int
if creator.Kind == job {
// Job has no replicas configuration, so we will use actual number of live pods as replicas count.
configured = actual
} else {
var err error
configured, err = f.getReplicaCount(creator)
if err != nil {
klog.ErrorS(err, "Failed to obtain replication info", "kind", creator.Kind, "object", klog.KRef(creator.Namespace, creator.Name))
continue
}
}
singleGroup := singleGroupStats{}
singleGroup.configured = configured
singleGroup.evictionTolerance = int(float64(configured) * f.evictionToleranceFraction)
for _, pod := range replicas {
podToReplicaCreatorMap[GetPodID(pod)] = creator
if pod.Status.Phase == apiv1.PodPending {
singleGroup.pending = singleGroup.pending + 1
}
if IsInPlaceUpdating(pod) {
singleGroup.inPlaceUpdating = singleGroup.inPlaceUpdating + 1
}
}
singleGroup.running = len(replicas) - singleGroup.pending
creatorToSingleGroupStatsMap[creator] = singleGroup
}
return &podsEvictionRestrictionImpl{
client: f.client,
podToReplicaCreatorMap: podToReplicaCreatorMap,
creatorToSingleGroupStatsMap: creatorToSingleGroupStatsMap,
patchCalculators: patchCalculators,
}
}
func getPodReplicaCreator(pod *apiv1.Pod) (*podReplicaCreator, error) {
creator := managingControllerRef(pod)
if creator == nil {
return nil, nil
}
podReplicaCreator := &podReplicaCreator{
Namespace: pod.Namespace,
Name: creator.Name,
Kind: controllerKind(creator.Kind),
}
return podReplicaCreator, nil
}
// GetPodID returns a string that uniquely identifies a pod by namespace and name
func GetPodID(pod *apiv1.Pod) string {
if pod == nil {
return ""
}
return pod.Namespace + "/" + pod.Name
}
func (f *podsEvictionRestrictionFactoryImpl) getReplicaCount(creator podReplicaCreator) (int, error) {
switch creator.Kind {
case replicationController:
rcObj, exists, err := f.rcInformer.GetStore().GetByKey(creator.Namespace + "/" + creator.Name)
if err != nil {
return 0, fmt.Errorf("replication controller %s/%s is not available, err: %v", creator.Namespace, creator.Name, err)
}
if !exists {
return 0, fmt.Errorf("replication controller %s/%s does not exist", creator.Namespace, creator.Name)
}
rc, ok := rcObj.(*apiv1.ReplicationController)
if !ok {
return 0, fmt.Errorf("Failed to parse Replication Controller")
}
if rc.Spec.Replicas == nil || *rc.Spec.Replicas == 0 {
return 0, fmt.Errorf("replication controller %s/%s has no replicas config", creator.Namespace, creator.Name)
}
return int(*rc.Spec.Replicas), nil
case replicaSet:
rsObj, exists, err := f.rsInformer.GetStore().GetByKey(creator.Namespace + "/" + creator.Name)
if err != nil {
return 0, fmt.Errorf("replica set %s/%s is not available, err: %v", creator.Namespace, creator.Name, err)
}
if !exists {
return 0, fmt.Errorf("replica set %s/%s does not exist", creator.Namespace, creator.Name)
}
rs, ok := rsObj.(*appsv1.ReplicaSet)
if !ok {
return 0, fmt.Errorf("Failed to parse Replicaset")
}
if rs.Spec.Replicas == nil || *rs.Spec.Replicas == 0 {
return 0, fmt.Errorf("replica set %s/%s has no replicas config", creator.Namespace, creator.Name)
}
return int(*rs.Spec.Replicas), nil
case statefulSet:
ssObj, exists, err := f.ssInformer.GetStore().GetByKey(creator.Namespace + "/" + creator.Name)
if err != nil {
return 0, fmt.Errorf("stateful set %s/%s is not available, err: %v", creator.Namespace, creator.Name, err)
}
if !exists {
return 0, fmt.Errorf("stateful set %s/%s does not exist", creator.Namespace, creator.Name)
}
ss, ok := ssObj.(*appsv1.StatefulSet)
if !ok {
return 0, fmt.Errorf("Failed to parse StatefulSet")
}
if ss.Spec.Replicas == nil || *ss.Spec.Replicas == 0 {
return 0, fmt.Errorf("stateful set %s/%s has no replicas config", creator.Namespace, creator.Name)
}
return int(*ss.Spec.Replicas), nil
case daemonSet:
dsObj, exists, err := f.dsInformer.GetStore().GetByKey(creator.Namespace + "/" + creator.Name)
if err != nil {
return 0, fmt.Errorf("daemon set %s/%s is not available, err: %v", creator.Namespace, creator.Name, err)
}
if !exists {
return 0, fmt.Errorf("daemon set %s/%s does not exist", creator.Namespace, creator.Name)
}
ds, ok := dsObj.(*appsv1.DaemonSet)
if !ok {
return 0, fmt.Errorf("Failed to parse DaemonSet")
}
if ds.Status.NumberReady == 0 {
return 0, fmt.Errorf("daemon set %s/%s has no number ready pods", creator.Namespace, creator.Name)
}
return int(ds.Status.NumberReady), nil
}
return 0, nil
}
func managingControllerRef(pod *apiv1.Pod) *metav1.OwnerReference {
var managingController metav1.OwnerReference
for _, ownerReference := range pod.ObjectMeta.GetOwnerReferences() {
if *ownerReference.Controller {
managingController = ownerReference
break
}
}
return &managingController
}
func setUpInformer(kubeClient kube_client.Interface, kind controllerKind) (cache.SharedIndexInformer, error) {
var informer cache.SharedIndexInformer
switch kind {
case replicationController:
informer = coreinformer.NewReplicationControllerInformer(kubeClient, apiv1.NamespaceAll,
resyncPeriod, cache.Indexers{cache.NamespaceIndex: cache.MetaNamespaceIndexFunc})
case replicaSet:
informer = appsinformer.NewReplicaSetInformer(kubeClient, apiv1.NamespaceAll,
resyncPeriod, cache.Indexers{cache.NamespaceIndex: cache.MetaNamespaceIndexFunc})
case statefulSet:
informer = appsinformer.NewStatefulSetInformer(kubeClient, apiv1.NamespaceAll,
resyncPeriod, cache.Indexers{cache.NamespaceIndex: cache.MetaNamespaceIndexFunc})
case daemonSet:
informer = appsinformer.NewDaemonSetInformer(kubeClient, apiv1.NamespaceAll,
resyncPeriod, cache.Indexers{cache.NamespaceIndex: cache.MetaNamespaceIndexFunc})
default:
return nil, fmt.Errorf("Unknown controller kind: %v", kind)
}
stopCh := make(chan struct{})
go informer.Run(stopCh)
synced := cache.WaitForCacheSync(stopCh, informer.HasSynced)
if !synced {
return nil, fmt.Errorf("Failed to sync %v cache.", kind)
}
return informer, nil
}
// CanInPlaceUpdate performs the same checks
func (e *podsEvictionRestrictionImpl) CanInPlaceUpdate(pod *apiv1.Pod) bool {
if !features.Enabled(features.InPlaceOrRecreate) {
return false
}
cr, present := e.podToReplicaCreatorMap[GetPodID(pod)]
if present {
if IsInPlaceUpdating(pod) {
return false
}
for _, container := range pod.Spec.Containers {
// If some of these are populated, we know it at least understands resizing
if container.ResizePolicy == nil {
klog.InfoS("Can't resize pod, container resize policy does not exist; is InPlacePodVerticalScaling enabled?", "pod", klog.KObj(pod))
return false
}
}
singleGroupStats, present := e.creatorToSingleGroupStatsMap[cr]
// TODO: Rename evictionTolerance to disruptionTolerance?
if present {
shouldBeAlive := singleGroupStats.configured - singleGroupStats.evictionTolerance
actuallyAlive := singleGroupStats.running - (singleGroupStats.evicted + singleGroupStats.inPlaceUpdating)
eligibleForInPlaceUpdate := false
if actuallyAlive > shouldBeAlive {
eligibleForInPlaceUpdate = true
}
// If all pods are running, no pods are being evicted or updated, and eviction tolerance is small, we can resize in-place
if singleGroupStats.running == singleGroupStats.configured &&
singleGroupStats.evictionTolerance == 0 &&
singleGroupStats.evicted == 0 && singleGroupStats.inPlaceUpdating == 0 {
eligibleForInPlaceUpdate = true
}
klog.V(4).InfoS("Pod disruption tolerance",
"pod", klog.KObj(pod),
"configuredPods", singleGroupStats.configured,
"runningPods", singleGroupStats.running,
"evictedPods", singleGroupStats.evicted,
"inPlaceUpdatingPods", singleGroupStats.inPlaceUpdating,
"evictionTolerance", singleGroupStats.evictionTolerance,
"eligibleForInPlaceUpdate", eligibleForInPlaceUpdate,
)
return eligibleForInPlaceUpdate
}
}
return false
}
// InPlaceUpdate sends calculates patches and sends resize request to api client. Returns error if pod cannot be in-place updated or if client returned error.
// Does not check if pod was actually in-place updated after grace period.
func (e *podsEvictionRestrictionImpl) InPlaceUpdate(podToUpdate *apiv1.Pod, vpa *vpa_types.VerticalPodAutoscaler, eventRecorder record.EventRecorder) error {
cr, present := e.podToReplicaCreatorMap[GetPodID(podToUpdate)]
if !present {
return fmt.Errorf("pod not suitable for eviction %v: not in replicated pods map", podToUpdate.Name)
}
// separate patches since we have to patch resize and spec separately
resourcePatches := []resource_updates.PatchRecord{}
annotationPatches := []resource_updates.PatchRecord{}
if podToUpdate.Annotations == nil {
annotationPatches = append(annotationPatches, patch.GetAddEmptyAnnotationsPatch())
}
for i, calculator := range e.patchCalculators {
p, err := calculator.CalculatePatches(podToUpdate, vpa)
if err != nil {
return err
}
klog.V(4).InfoS("Calculated patches for pod", "pod", klog.KObj(podToUpdate), "patches", p)
// TODO(maxcao13): change how this works later, this is gross and depends on the resource calculator being first in the slice
// we may not even want the updater to patch pod annotations at all
if i == 0 {
resourcePatches = append(resourcePatches, p...)
} else {
annotationPatches = append(annotationPatches, p...)
}
}
if len(resourcePatches) > 0 {
patch, err := json.Marshal(resourcePatches)
if err != nil {
return err
}
res, err := e.client.CoreV1().Pods(podToUpdate.Namespace).Patch(context.TODO(), podToUpdate.Name, k8stypes.JSONPatchType, patch, metav1.PatchOptions{}, "resize")
if err != nil {
return err
}
klog.V(4).InfoS("In-place patched pod /resize subresource using patches ", "pod", klog.KObj(res), "patches", string(patch))
if len(annotationPatches) > 0 {
patch, err := json.Marshal(annotationPatches)
if err != nil {
return err
}
res, err = e.client.CoreV1().Pods(podToUpdate.Namespace).Patch(context.TODO(), podToUpdate.Name, k8stypes.JSONPatchType, patch, metav1.PatchOptions{})
if err != nil {
return err
}
klog.V(4).InfoS("Patched pod annotations", "pod", klog.KObj(res), "patches", string(patch))
}
} else {
return fmt.Errorf("no resource patches were calculated to apply")
}
// TODO(maxcao13): If this keeps getting called on the same object with the same reason, it is considered a patch request.
// And we fail to have the corresponding rbac for it. So figure out if we need this later.
// Do we even need to emit an event? The node might reject the resize request. If so, should we rename this to InPlaceResizeAttempted?
// eventRecorder.Event(podToUpdate, apiv1.EventTypeNormal, "InPlaceResizedByVPA", "Pod was resized in place by VPA Updater.")
if podToUpdate.Status.Phase == apiv1.PodRunning {
singleGroupStats, present := e.creatorToSingleGroupStatsMap[cr]
if !present {
klog.InfoS("Internal error - cannot find stats for replication group", "pod", klog.KObj(podToUpdate), "podReplicaCreator", cr)
} else {
singleGroupStats.inPlaceUpdating = singleGroupStats.inPlaceUpdating + 1
e.creatorToSingleGroupStatsMap[cr] = singleGroupStats
}
} else {
klog.InfoS("Attempted to in-place update, but pod was not running", "pod", klog.KObj(podToUpdate), "phase", podToUpdate.Status.Phase)
}
return nil
}
// TODO(maxcao13): Switch to conditions after 1.33 is released: https://github.com/kubernetes/enhancements/pull/5089
// IsInPlaceUpdating checks whether or not the given pod is currently in the middle of an in-place update
func IsInPlaceUpdating(podToCheck *apiv1.Pod) (isUpdating bool) {
return podToCheck.Status.Resize != ""
}

View File

@ -1,84 +0,0 @@
/*
Copyright 2025 The Kubernetes 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 inplace
import (
"fmt"
core "k8s.io/api/core/v1"
"k8s.io/klog/v2"
"k8s.io/autoscaler/vertical-pod-autoscaler/pkg/admission-controller/resource/pod/recommendation"
vpa_types "k8s.io/autoscaler/vertical-pod-autoscaler/pkg/apis/autoscaling.k8s.io/v1"
"k8s.io/autoscaler/vertical-pod-autoscaler/pkg/utils/limitrange"
vpa_api_util "k8s.io/autoscaler/vertical-pod-autoscaler/pkg/utils/vpa"
)
type inPlaceRecommendationProvider struct {
limitsRangeCalculator limitrange.LimitRangeCalculator
recommendationProcessor vpa_api_util.RecommendationProcessor
}
// NewInPlaceRecommendationProvider constructs the recommendation provider that can be used to determine recommendations for pods.
func NewInPlaceRecommendationProvider(calculator limitrange.LimitRangeCalculator,
recommendationProcessor vpa_api_util.RecommendationProcessor) recommendation.Provider {
return &inPlaceRecommendationProvider{
limitsRangeCalculator: calculator,
recommendationProcessor: recommendationProcessor,
}
}
// GetContainersResourcesForPod returns recommended request for a given pod.
// The returned slice corresponds 1-1 to containers in the Pod.
func (p *inPlaceRecommendationProvider) GetContainersResourcesForPod(pod *core.Pod, vpa *vpa_types.VerticalPodAutoscaler) ([]vpa_api_util.ContainerResources, vpa_api_util.ContainerToAnnotationsMap, error) {
if vpa == nil || pod == nil {
klog.V(2).InfoS("Can't calculate recommendations, one of VPA or Pod is nil", "vpa", vpa, "pod", pod)
return nil, nil, nil
}
klog.V(2).InfoS("Updating requirements for pod", "pod", pod.Name)
recommendedPodResources := &vpa_types.RecommendedPodResources{}
if vpa.Status.Recommendation != nil {
var err error
// ignore annotations as they are cannot be used when patching resize subresource
recommendedPodResources, _, err = p.recommendationProcessor.Apply(vpa, pod)
if err != nil {
klog.V(2).InfoS("Cannot process recommendation for pod", "pod", klog.KObj(pod))
return nil, nil, err
}
}
containerLimitRange, err := p.limitsRangeCalculator.GetContainerLimitRangeItem(pod.Namespace)
if err != nil {
return nil, nil, fmt.Errorf("error getting containerLimitRange: %s", err)
}
var resourcePolicy *vpa_types.PodResourcePolicy
if vpa.Spec.UpdatePolicy == nil || vpa.Spec.UpdatePolicy.UpdateMode == nil || *vpa.Spec.UpdatePolicy.UpdateMode != vpa_types.UpdateModeOff {
resourcePolicy = vpa.Spec.ResourcePolicy
}
containerResources := recommendation.GetContainersResources(pod, resourcePolicy, *recommendedPodResources, containerLimitRange, false, nil)
// Ensure that we are not propagating empty resource key if any.
for _, resource := range containerResources {
if resource.RemoveEmptyResourceKeyIfAny() {
klog.InfoS("An empty resource key was found and purged", "pod", klog.KObj(pod), "vpa", klog.KObj(vpa))
}
}
return containerResources, nil, nil
}

View File

@ -33,6 +33,10 @@ func (*inPlaceUpdate) CalculatePatches(pod *core.Pod, _ *vpa_types.VerticalPodAu
return []resource_admission.PatchRecord{patch.GetAddAnnotationPatch(annotations.VpaInPlaceUpdatedLabel, vpaInPlaceUpdatedValue)}, nil
}
func (*inPlaceUpdate) PatchResourceTarget() patch.PatchResourceTarget {
return patch.Pod
}
// NewInPlaceUpdatedCalculator returns calculator for
// observed containers patches.
func NewInPlaceUpdatedCalculator() patch.Calculator {

View File

@ -33,13 +33,18 @@ type resourcesInplaceUpdatesPatchCalculator struct {
}
// NewResourceInPlaceUpdatesCalculator returns a calculator for
// resource in-place update patches.
// in-place resource update patches.
func NewResourceInPlaceUpdatesCalculator(recommendationProvider recommendation.Provider) patch.Calculator {
return &resourcesInplaceUpdatesPatchCalculator{
recommendationProvider: recommendationProvider,
}
}
// PatchResourceTarget returns the resize subresource to apply calculator patches.
func (*resourcesInplaceUpdatesPatchCalculator) PatchResourceTarget() patch.PatchResourceTarget {
return patch.Resize
}
// CalculatePatches calculates a JSON patch from a VPA's recommendation to send to the pod "resize" subresource as an in-place resize.
func (c *resourcesInplaceUpdatesPatchCalculator) CalculatePatches(pod *core.Pod, vpa *vpa_types.VerticalPodAutoscaler) ([]resource_admission.PatchRecord, error) {
result := []resource_admission.PatchRecord{}

View File

@ -30,7 +30,8 @@ import (
"k8s.io/apimachinery/pkg/labels"
kube_client "k8s.io/client-go/kubernetes"
"k8s.io/client-go/kubernetes/fake"
"k8s.io/utils/clock"
utils "k8s.io/autoscaler/vertical-pod-autoscaler/pkg/updater/utils"
corescheme "k8s.io/client-go/kubernetes/scheme"
clientv1 "k8s.io/client-go/kubernetes/typed/core/v1"
@ -47,24 +48,13 @@ import (
"k8s.io/autoscaler/vertical-pod-autoscaler/pkg/features"
"k8s.io/autoscaler/vertical-pod-autoscaler/pkg/target"
controllerfetcher "k8s.io/autoscaler/vertical-pod-autoscaler/pkg/target/controller_fetcher"
"k8s.io/autoscaler/vertical-pod-autoscaler/pkg/updater/eviction"
"k8s.io/autoscaler/vertical-pod-autoscaler/pkg/updater/priority"
restriction "k8s.io/autoscaler/vertical-pod-autoscaler/pkg/updater/restriction"
metrics_updater "k8s.io/autoscaler/vertical-pod-autoscaler/pkg/utils/metrics/updater"
"k8s.io/autoscaler/vertical-pod-autoscaler/pkg/utils/status"
vpa_api_util "k8s.io/autoscaler/vertical-pod-autoscaler/pkg/utils/vpa"
)
// TODO: Make these configurable by flags
const (
// DeferredResizeUpdateTimeout defines the duration during which an in-place resize request
// is considered deferred. If the resize is not completed within this time, it falls back to eviction.
DeferredResizeUpdateTimeout = 5 * time.Minute
// InProgressResizeUpdateTimeout defines the duration during which an in-place resize request
// is considered in progress. If the resize is not completed within this time, it falls back to eviction.
InProgressResizeUpdateTimeout = 1 * time.Hour
)
// Updater performs updates on pods if recommended by Vertical Pod Autoscaler
type Updater interface {
// RunOnce represents single iteration in the main-loop of Updater
@ -72,29 +62,27 @@ type Updater interface {
}
type updater struct {
vpaLister vpa_lister.VerticalPodAutoscalerLister
podLister v1lister.PodLister
eventRecorder record.EventRecorder
evictionFactory eviction.PodsEvictionRestrictionFactory
recommendationProcessor vpa_api_util.RecommendationProcessor
evictionAdmission priority.PodEvictionAdmission
priorityProcessor priority.PriorityProcessor
evictionRateLimiter *rate.Limiter
selectorFetcher target.VpaTargetSelectorFetcher
useAdmissionControllerStatus bool
statusValidator status.Validator
controllerFetcher controllerfetcher.ControllerFetcher
ignoredNamespaces []string
patchCalculators []patch.Calculator
clock clock.Clock
lastInPlaceUpdateAttemptTimeMap map[string]time.Time
vpaLister vpa_lister.VerticalPodAutoscalerLister
podLister v1lister.PodLister
eventRecorder record.EventRecorder
restrictionFactory restriction.PodsRestrictionFactory
recommendationProcessor vpa_api_util.RecommendationProcessor
evictionAdmission priority.PodEvictionAdmission
priorityProcessor priority.PriorityProcessor
evictionRateLimiter *rate.Limiter
inPlaceRateLimiter *rate.Limiter
selectorFetcher target.VpaTargetSelectorFetcher
useAdmissionControllerStatus bool
statusValidator status.Validator
controllerFetcher controllerfetcher.ControllerFetcher
ignoredNamespaces []string
}
// NewUpdater creates Updater with given configuration
func NewUpdater(
kubeClient kube_client.Interface,
vpaClient *vpa_clientset.Clientset,
minReplicasForEvicition int,
minReplicasForEviction int,
evictionRateLimit float64,
evictionRateBurst int,
evictionToleranceFraction float64,
@ -110,17 +98,26 @@ func NewUpdater(
patchCalculators []patch.Calculator,
) (Updater, error) {
evictionRateLimiter := getRateLimiter(evictionRateLimit, evictionRateBurst)
factory, err := eviction.NewPodsEvictionRestrictionFactory(kubeClient, minReplicasForEvicition, evictionToleranceFraction)
// TODO: Create in-place rate limits for the in-place rate limiter
inPlaceRateLimiter := getRateLimiter(evictionRateLimit, evictionRateBurst)
factory, err := restriction.NewPodsRestrictionFactory(
kubeClient,
minReplicasForEviction,
evictionToleranceFraction,
patchCalculators,
)
if err != nil {
return nil, fmt.Errorf("Failed to create eviction restriction factory: %v", err)
return nil, fmt.Errorf("Failed to create restriction factory: %v", err)
}
return &updater{
vpaLister: vpa_api_util.NewVpasLister(vpaClient, make(chan struct{}), namespace),
podLister: newPodLister(kubeClient, namespace),
eventRecorder: newEventRecorder(kubeClient),
evictionFactory: factory,
restrictionFactory: factory,
recommendationProcessor: recommendationProcessor,
evictionRateLimiter: evictionRateLimiter,
inPlaceRateLimiter: inPlaceRateLimiter,
evictionAdmission: evictionAdmission,
priorityProcessor: priorityProcessor,
selectorFetcher: selectorFetcher,
@ -131,10 +128,7 @@ func NewUpdater(
status.AdmissionControllerStatusName,
statusNamespace,
),
ignoredNamespaces: ignoredNamespaces,
patchCalculators: patchCalculators,
clock: &clock.RealClock{},
lastInPlaceUpdateAttemptTimeMap: make(map[string]time.Time),
ignoredNamespaces: ignoredNamespaces,
}, nil
}
@ -241,14 +235,21 @@ func (u *updater) RunOnce(ctx context.Context) {
for vpa, livePods := range controlledPods {
vpaSize := len(livePods)
controlledPodsCounter.Add(vpaSize, vpaSize)
evictionLimiter := u.evictionFactory.NewPodsEvictionRestriction(livePods, vpa, u.patchCalculators)
creatorToSingleGroupStatsMap, podToReplicaCreatorMap, err := u.restrictionFactory.GetCreatorMaps(livePods, vpa)
if err != nil {
klog.ErrorS(err, "Failed to get creator maps")
continue
}
evictionLimiter := u.restrictionFactory.NewPodsEvictionRestriction(creatorToSingleGroupStatsMap, podToReplicaCreatorMap)
inPlaceLimiter := u.restrictionFactory.NewPodsInPlaceRestriction(creatorToSingleGroupStatsMap, podToReplicaCreatorMap)
podsForInPlace := make([]*apiv1.Pod, 0)
podsForEviction := make([]*apiv1.Pod, 0)
updateMode := vpa_api_util.GetUpdateMode(vpa)
if updateMode == vpa_types.UpdateModeInPlaceOrRecreate && features.Enabled(features.InPlaceOrRecreate) {
podsForInPlace = u.getPodsUpdateOrder(filterNonInPlaceUpdatablePods(livePods, evictionLimiter), vpa)
podsForInPlace = u.getPodsUpdateOrder(filterNonInPlaceUpdatablePods(livePods, inPlaceLimiter), vpa)
inPlaceUpdatablePodsCounter.Add(vpaSize, len(podsForInPlace))
} else {
if updateMode == vpa_types.UpdateModeInPlaceOrRecreate {
@ -265,18 +266,27 @@ func (u *updater) RunOnce(ctx context.Context) {
for _, pod := range podsForInPlace {
withInPlaceUpdatable = true
fallBackToEviction, err := u.AttemptInPlaceUpdate(ctx, vpa, pod, evictionLimiter)
decision := inPlaceLimiter.CanInPlaceUpdate(pod)
if decision == utils.InPlaceDeferred {
klog.V(0).InfoS("In-place update deferred", "pod", klog.KObj(pod))
continue
} else if decision == utils.InPlaceEvict {
podsForEviction = append(podsForEviction, pod)
continue
}
err = u.inPlaceRateLimiter.Wait(ctx)
if err != nil {
klog.V(0).InfoS("In-place rate limiter wait failed for in-place resize", "error", err)
return
}
err := inPlaceLimiter.InPlaceUpdate(pod, vpa, u.eventRecorder)
if err != nil {
klog.V(0).InfoS("In-place update failed", "error", err, "pod", klog.KObj(pod))
return
}
if fallBackToEviction {
klog.V(4).InfoS("Falling back to eviction for pod", "pod", klog.KObj(pod))
podsForEviction = append(podsForEviction, pod)
} else {
withInPlaceUpdated = true
metrics_updater.AddInPlaceUpdatedPod(vpaSize)
}
withInPlaceUpdated = true
metrics_updater.AddInPlaceUpdatedPod(vpaSize)
}
for _, pod := range podsForEviction {
@ -315,17 +325,17 @@ func (u *updater) RunOnce(ctx context.Context) {
timer.ObserveStep("EvictPods")
}
func getRateLimiter(evictionRateLimit float64, evictionRateLimitBurst int) *rate.Limiter {
var evictionRateLimiter *rate.Limiter
if evictionRateLimit <= 0 {
func getRateLimiter(rateLimit float64, rateLimitBurst int) *rate.Limiter {
var rateLimiter *rate.Limiter
if rateLimit <= 0 {
// As a special case if the rate is set to rate.Inf, the burst rate is ignored
// see https://github.com/golang/time/blob/master/rate/rate.go#L37
evictionRateLimiter = rate.NewLimiter(rate.Inf, 0)
rateLimiter = rate.NewLimiter(rate.Inf, 0)
klog.V(1).InfoS("Rate limit disabled")
} else {
evictionRateLimiter = rate.NewLimiter(rate.Limit(evictionRateLimit), evictionRateLimitBurst)
rateLimiter = rate.NewLimiter(rate.Limit(rateLimit), rateLimitBurst)
}
return evictionRateLimiter
return rateLimiter
}
// getPodsUpdateOrder returns list of pods that should be updated ordered by update priority
@ -353,11 +363,13 @@ func filterPods(pods []*apiv1.Pod, predicate func(*apiv1.Pod) bool) []*apiv1.Pod
return result
}
func filterNonInPlaceUpdatablePods(pods []*apiv1.Pod, evictionRestriction eviction.PodsEvictionRestriction) []*apiv1.Pod {
return filterPods(pods, evictionRestriction.CanInPlaceUpdate)
func filterNonInPlaceUpdatablePods(pods []*apiv1.Pod, inplaceRestriction restriction.PodsInPlaceRestriction) []*apiv1.Pod {
return filterPods(pods, func(pod *apiv1.Pod) bool {
return inplaceRestriction.CanInPlaceUpdate(pod) != utils.InPlaceDeferred
})
}
func filterNonEvictablePods(pods []*apiv1.Pod, evictionRestriction eviction.PodsEvictionRestriction) []*apiv1.Pod {
func filterNonEvictablePods(pods []*apiv1.Pod, evictionRestriction restriction.PodsEvictionRestriction) []*apiv1.Pod {
return filterPods(pods, evictionRestriction.CanEvict)
}
@ -397,61 +409,3 @@ func newEventRecorder(kubeClient kube_client.Interface) record.EventRecorder {
return eventBroadcaster.NewRecorder(vpascheme, apiv1.EventSource{Component: "vpa-updater"})
}
func (u *updater) AttemptInPlaceUpdate(ctx context.Context, vpa *vpa_types.VerticalPodAutoscaler, pod *apiv1.Pod, evictionLimiter eviction.PodsEvictionRestriction) (fallBackToEviction bool, err error) {
klog.V(4).InfoS("Checking preconditions for attemping in-place update", "pod", klog.KObj(pod))
clock := u.clock
if !evictionLimiter.CanInPlaceUpdate(pod) {
if eviction.IsInPlaceUpdating(pod) {
lastInPlaceUpdateTime, exists := u.lastInPlaceUpdateAttemptTimeMap[eviction.GetPodID(pod)]
if !exists {
klog.V(4).InfoS("In-place update in progress for pod but no lastInPlaceUpdateTime found, setting it to now", "pod", klog.KObj(pod))
lastInPlaceUpdateTime = clock.Now()
u.lastInPlaceUpdateAttemptTimeMap[eviction.GetPodID(pod)] = lastInPlaceUpdateTime
}
// TODO(maxcao13): fix this after 1.33 KEP changes
// if currently inPlaceUpdating, we should only fallback to eviction if the update has failed. i.e: one of the following conditions:
// 1. .status.resize: Infeasible
// 2. .status.resize: Deferred + more than 5 minutes has elapsed since the lastInPlaceUpdateTime
// 3. .status.resize: InProgress + more than 1 hour has elapsed since the lastInPlaceUpdateTime
switch pod.Status.Resize {
case apiv1.PodResizeStatusDeferred:
if clock.Since(lastInPlaceUpdateTime) > DeferredResizeUpdateTimeout {
klog.V(4).InfoS(fmt.Sprintf("In-place update deferred for more than %v, falling back to eviction", DeferredResizeUpdateTimeout), "pod", klog.KObj(pod))
fallBackToEviction = true
} else {
klog.V(4).InfoS("In-place update deferred, NOT falling back to eviction yet", "pod", klog.KObj(pod))
}
case apiv1.PodResizeStatusInProgress:
if clock.Since(lastInPlaceUpdateTime) > InProgressResizeUpdateTimeout {
klog.V(4).InfoS(fmt.Sprintf("In-place update in progress for more than %v, falling back to eviction", InProgressResizeUpdateTimeout), "pod", klog.KObj(pod))
fallBackToEviction = true
} else {
klog.V(4).InfoS("In-place update in progress, NOT falling back to eviction yet", "pod", klog.KObj(pod))
}
case apiv1.PodResizeStatusInfeasible:
klog.V(4).InfoS("In-place update infeasible, falling back to eviction", "pod", klog.KObj(pod))
fallBackToEviction = true
default:
klog.V(4).InfoS("In-place update status unknown, falling back to eviction", "pod", klog.KObj(pod))
fallBackToEviction = true
}
return fallBackToEviction, nil
}
klog.V(4).InfoS("Can't in-place update pod, but not falling back to eviction. Waiting for next loop", "pod", klog.KObj(pod))
return false, nil
}
// TODO(jkyros): need our own rate limiter or can we freeload off the eviction one?
err = u.evictionRateLimiter.Wait(ctx)
if err != nil {
klog.ErrorS(err, "Eviction rate limiter wait failed for in-place resize", "pod", klog.KObj(pod))
return false, err
}
klog.V(2).InfoS("Actuating in-place update", "pod", klog.KObj(pod))
u.lastInPlaceUpdateAttemptTimeMap[eviction.GetPodID(pod)] = u.clock.Now()
err = evictionLimiter.InPlaceUpdate(pod, vpa, u.eventRecorder)
return false, err
}

View File

@ -22,6 +22,9 @@ import (
"testing"
"time"
restriction "k8s.io/autoscaler/vertical-pod-autoscaler/pkg/updater/restriction"
utils "k8s.io/autoscaler/vertical-pod-autoscaler/pkg/updater/utils"
"golang.org/x/time/rate"
v1 "k8s.io/api/autoscaling/v1"
@ -34,14 +37,11 @@ import (
"k8s.io/apimachinery/pkg/runtime"
"k8s.io/client-go/kubernetes/fake"
featuregatetesting "k8s.io/component-base/featuregate/testing"
baseclocktest "k8s.io/utils/clock/testing"
"k8s.io/autoscaler/vertical-pod-autoscaler/pkg/admission-controller/resource/pod/patch"
vpa_types "k8s.io/autoscaler/vertical-pod-autoscaler/pkg/apis/autoscaling.k8s.io/v1"
"k8s.io/autoscaler/vertical-pod-autoscaler/pkg/features"
controllerfetcher "k8s.io/autoscaler/vertical-pod-autoscaler/pkg/target/controller_fetcher"
target_mock "k8s.io/autoscaler/vertical-pod-autoscaler/pkg/target/mock"
"k8s.io/autoscaler/vertical-pod-autoscaler/pkg/updater/eviction"
"k8s.io/autoscaler/vertical-pod-autoscaler/pkg/updater/priority"
"k8s.io/autoscaler/vertical-pod-autoscaler/pkg/utils/status"
"k8s.io/autoscaler/vertical-pod-autoscaler/pkg/utils/test"
@ -60,6 +60,8 @@ func TestRunOnce_Mode(t *testing.T) {
expectFetchCalls bool
expectedEvictionCount int
expectedInPlacedCount int
canEvict bool
canInPlaceUpdate utils.InPlaceDecision
}{
{
name: "with Auto mode",
@ -67,6 +69,8 @@ func TestRunOnce_Mode(t *testing.T) {
expectFetchCalls: true,
expectedEvictionCount: 5,
expectedInPlacedCount: 0,
canEvict: true,
canInPlaceUpdate: utils.InPlaceApproved,
},
{
name: "with Initial mode",
@ -74,6 +78,8 @@ func TestRunOnce_Mode(t *testing.T) {
expectFetchCalls: false,
expectedEvictionCount: 0,
expectedInPlacedCount: 0,
canEvict: true,
canInPlaceUpdate: utils.InPlaceApproved,
},
{
name: "with Off mode",
@ -81,13 +87,35 @@ func TestRunOnce_Mode(t *testing.T) {
expectFetchCalls: false,
expectedEvictionCount: 0,
expectedInPlacedCount: 0,
canEvict: true,
canInPlaceUpdate: utils.InPlaceApproved,
},
{
name: "with InPlaceOrRecreate mode",
name: "with InPlaceOrRecreate mode expecting in-place updates",
updateMode: vpa_types.UpdateModeInPlaceOrRecreate,
expectFetchCalls: true,
expectedEvictionCount: 0,
expectedInPlacedCount: 5,
canEvict: true,
canInPlaceUpdate: utils.InPlaceApproved,
},
{
name: "with InPlaceOrRecreate mode expecting fallback to evictions",
updateMode: vpa_types.UpdateModeInPlaceOrRecreate,
expectFetchCalls: true,
expectedEvictionCount: 5,
expectedInPlacedCount: 0,
canEvict: true,
canInPlaceUpdate: utils.InPlaceEvict,
},
{
name: "with InPlaceOrRecreate mode expecting no evictions or in-place",
updateMode: vpa_types.UpdateModeInPlaceOrRecreate,
expectFetchCalls: true,
expectedEvictionCount: 0,
expectedInPlacedCount: 0,
canEvict: false,
canInPlaceUpdate: utils.InPlaceDeferred,
},
}
for _, tc := range tests {
@ -99,6 +127,7 @@ func TestRunOnce_Mode(t *testing.T) {
tc.expectFetchCalls,
tc.expectedEvictionCount,
tc.expectedInPlacedCount,
tc.canInPlaceUpdate,
)
})
}
@ -136,6 +165,7 @@ func TestRunOnce_Status(t *testing.T) {
tc.expectFetchCalls,
tc.expectedEvictionCount,
tc.expectedInPlacedCount,
utils.InPlaceApproved,
)
})
}
@ -148,6 +178,7 @@ func testRunOnceBase(
expectFetchCalls bool,
expectedEvictionCount int,
expectedInPlacedCount int,
canInPlaceUpdate utils.InPlaceDecision,
) {
featuregatetesting.SetFeatureGateDuringTest(t, features.MutableFeatureGate, features.InPlaceOrRecreate, true)
ctrl := gomock.NewController(t)
@ -173,6 +204,7 @@ func testRunOnceBase(
}
pods := make([]*apiv1.Pod, livePods)
eviction := &test.PodsEvictionRestrictionMock{}
inplace := &test.PodsInPlaceRestrictionMock{}
for i := range pods {
pods[i] = test.Pod().WithName("test_"+strconv.Itoa(i)).
@ -182,15 +214,17 @@ func testRunOnceBase(
pods[i].Labels = labels
eviction.On("CanInPlaceUpdate", pods[i]).Return(updateMode == vpa_types.UpdateModeInPlaceOrRecreate)
eviction.On("IsInPlaceUpdating", pods[i]).Return(false)
eviction.On("InPlaceUpdate", pods[i], nil).Return(nil)
inplace.On("CanInPlaceUpdate", pods[i]).Return(canInPlaceUpdate)
inplace.On("InPlaceUpdate", pods[i], nil).Return(nil)
eviction.On("CanEvict", pods[i]).Return(true)
eviction.On("Evict", pods[i], nil).Return(nil)
}
factory := &fakeEvictFactory{eviction}
factory := &restriction.FakePodsRestrictionFactory{
Eviction: eviction,
InPlace: inplace,
}
vpaLister := &test.VerticalPodAutoscalerListerMock{}
podLister := &test.PodListerMock{}
@ -215,19 +249,18 @@ func testRunOnceBase(
mockSelectorFetcher := target_mock.NewMockVpaTargetSelectorFetcher(ctrl)
updater := &updater{
vpaLister: vpaLister,
podLister: podLister,
evictionFactory: factory,
evictionRateLimiter: rate.NewLimiter(rate.Inf, 0),
evictionAdmission: priority.NewDefaultPodEvictionAdmission(),
recommendationProcessor: &test.FakeRecommendationProcessor{},
selectorFetcher: mockSelectorFetcher,
controllerFetcher: controllerfetcher.FakeControllerFetcher{},
useAdmissionControllerStatus: true,
statusValidator: statusValidator,
priorityProcessor: priority.NewProcessor(),
lastInPlaceUpdateAttemptTimeMap: make(map[string]time.Time),
clock: baseclocktest.NewFakeClock(time.Time{}),
vpaLister: vpaLister,
podLister: podLister,
restrictionFactory: factory,
evictionRateLimiter: rate.NewLimiter(rate.Inf, 0),
inPlaceRateLimiter: rate.NewLimiter(rate.Inf, 0),
evictionAdmission: priority.NewDefaultPodEvictionAdmission(),
recommendationProcessor: &test.FakeRecommendationProcessor{},
selectorFetcher: mockSelectorFetcher,
controllerFetcher: controllerfetcher.FakeControllerFetcher{},
useAdmissionControllerStatus: true,
statusValidator: statusValidator,
priorityProcessor: priority.NewProcessor(),
}
if expectFetchCalls {
@ -235,12 +268,16 @@ func testRunOnceBase(
}
updater.RunOnce(context.Background())
eviction.AssertNumberOfCalls(t, "Evict", expectedEvictionCount)
eviction.AssertNumberOfCalls(t, "InPlaceUpdate", expectedInPlacedCount)
inplace.AssertNumberOfCalls(t, "InPlaceUpdate", expectedInPlacedCount)
}
func TestRunOnceNotingToProcess(t *testing.T) {
eviction := &test.PodsEvictionRestrictionMock{}
factory := &fakeEvictFactory{eviction}
inplace := &test.PodsInPlaceRestrictionMock{}
factory := &restriction.FakePodsRestrictionFactory{
Eviction: eviction,
InPlace: inplace,
}
vpaLister := &test.VerticalPodAutoscalerListerMock{}
podLister := &test.PodListerMock{}
vpaLister.On("List").Return(nil, nil).Once()
@ -248,8 +285,9 @@ func TestRunOnceNotingToProcess(t *testing.T) {
updater := &updater{
vpaLister: vpaLister,
podLister: podLister,
evictionFactory: factory,
restrictionFactory: factory,
evictionRateLimiter: rate.NewLimiter(rate.Inf, 0),
inPlaceRateLimiter: rate.NewLimiter(rate.Inf, 0),
evictionAdmission: priority.NewDefaultPodEvictionAdmission(),
recommendationProcessor: &test.FakeRecommendationProcessor{},
useAdmissionControllerStatus: true,
@ -275,14 +313,6 @@ func TestGetRateLimiter(t *testing.T) {
}
}
type fakeEvictFactory struct {
evict eviction.PodsEvictionRestriction
}
func (f fakeEvictFactory) NewPodsEvictionRestriction(pods []*apiv1.Pod, vpa *vpa_types.VerticalPodAutoscaler, patchCalculators []patch.Calculator) eviction.PodsEvictionRestriction {
return f.evict
}
type fakeValidator struct {
isValid bool
}
@ -320,7 +350,7 @@ func TestRunOnceIgnoreNamespaceMatchingPods(t *testing.T) {
}
pods := make([]*apiv1.Pod, livePods)
eviction := &test.PodsEvictionRestrictionMock{}
inplace := &test.PodsInPlaceRestrictionMock{}
for i := range pods {
pods[i] = test.Pod().WithName("test_"+strconv.Itoa(i)).
AddContainer(test.Container().WithName(containerName).WithCPURequest(resource.MustParse("1")).WithMemRequest(resource.MustParse("100M")).Get()).
@ -332,7 +362,10 @@ func TestRunOnceIgnoreNamespaceMatchingPods(t *testing.T) {
eviction.On("Evict", pods[i], nil).Return(nil)
}
factory := &fakeEvictFactory{eviction}
factory := &restriction.FakePodsRestrictionFactory{
Eviction: eviction,
InPlace: inplace,
}
vpaLister := &test.VerticalPodAutoscalerListerMock{}
podLister := &test.PodListerMock{}
@ -360,8 +393,9 @@ func TestRunOnceIgnoreNamespaceMatchingPods(t *testing.T) {
updater := &updater{
vpaLister: vpaLister,
podLister: podLister,
evictionFactory: factory,
restrictionFactory: factory,
evictionRateLimiter: rate.NewLimiter(rate.Inf, 0),
inPlaceRateLimiter: rate.NewLimiter(rate.Inf, 0),
evictionAdmission: priority.NewDefaultPodEvictionAdmission(),
recommendationProcessor: &test.FakeRecommendationProcessor{},
selectorFetcher: mockSelectorFetcher,
@ -447,143 +481,3 @@ func TestNewEventRecorder(t *testing.T) {
})
}
}
func TestAttempInPlaceUpdate(t *testing.T) {
testCases := []struct {
name string
pod *apiv1.Pod
lastInPlaceUpdateAttempt time.Time
canInPlaceUpdate bool
isInPlaceUpdating bool
expectedFallbackToEviction bool
expectInPlaceUpdated bool
expectError bool
}{
{
name: "CanInPlaceUpdate=true - in-place resize attempt successful",
pod: test.Pod().
WithName("test").
Get(),
lastInPlaceUpdateAttempt: time.Time{},
canInPlaceUpdate: true,
isInPlaceUpdating: false,
expectedFallbackToEviction: false,
expectInPlaceUpdated: true,
expectError: false,
},
{
name: "CanInPlaceUpdate=false - resize Deferred for too long",
pod: test.Pod().
WithName("test").
WithResizeStatus(apiv1.PodResizeStatusDeferred).
Get(),
lastInPlaceUpdateAttempt: time.UnixMilli(0),
canInPlaceUpdate: false,
isInPlaceUpdating: true,
expectedFallbackToEviction: true,
expectInPlaceUpdated: false,
expectError: false,
},
{
name: "CanInPlaceUpdate=false - resize Deferred, conditions not met to fallback",
pod: test.Pod().
WithName("test").
WithResizeStatus(apiv1.PodResizeStatusDeferred).
Get(),
lastInPlaceUpdateAttempt: time.UnixMilli(3600000), // 1 hour from epoch
canInPlaceUpdate: false,
isInPlaceUpdating: true,
expectedFallbackToEviction: false,
expectInPlaceUpdated: false,
expectError: false,
},
{
name: ("CanInPlaceUpdate=false - resize inProgress for more too long"),
pod: test.Pod().
WithName("test").
WithResizeStatus(apiv1.PodResizeStatusInProgress).
Get(),
lastInPlaceUpdateAttempt: time.UnixMilli(0),
canInPlaceUpdate: false,
isInPlaceUpdating: true,
expectedFallbackToEviction: true,
expectInPlaceUpdated: false,
expectError: false,
},
{
name: "CanInPlaceUpdate=false - resize InProgress, conditions not met to fallback",
pod: test.Pod().
WithName("test").
WithResizeStatus(apiv1.PodResizeStatusInProgress).
Get(),
lastInPlaceUpdateAttempt: time.UnixMilli(3600000), // 1 hour from epoch
canInPlaceUpdate: false,
isInPlaceUpdating: true,
expectedFallbackToEviction: false,
expectInPlaceUpdated: false,
expectError: false,
},
{
name: "CanInPlaceUpdate=false - infeasible",
pod: test.Pod().
WithName("test").
WithResizeStatus(apiv1.PodResizeStatusInfeasible).
Get(),
lastInPlaceUpdateAttempt: time.Time{},
canInPlaceUpdate: false,
isInPlaceUpdating: true,
expectedFallbackToEviction: true,
expectInPlaceUpdated: false,
expectError: false,
},
{
name: "CanInPlaceUpdate=false - possibly due to disruption tolerance, retry",
pod: test.Pod().
WithName("test").
Get(),
lastInPlaceUpdateAttempt: time.Time{},
canInPlaceUpdate: false,
isInPlaceUpdating: false,
expectedFallbackToEviction: false,
expectInPlaceUpdated: false,
expectError: false,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
testAttemptInPlaceUpdateBase(t, tc.pod, tc.lastInPlaceUpdateAttempt, tc.canInPlaceUpdate, tc.isInPlaceUpdating, tc.expectedFallbackToEviction, tc.expectInPlaceUpdated, tc.expectError)
})
}
}
func testAttemptInPlaceUpdateBase(t *testing.T, pod *apiv1.Pod, lastInPlace time.Time, canInPlaceUpdate, isInPlaceUpdating, expectedFallBackToEviction, expectInPlaceUpdated, expectError bool) {
podID := eviction.GetPodID(pod)
eviction := &test.PodsEvictionRestrictionMock{}
eviction.On("CanInPlaceUpdate", pod).Return(canInPlaceUpdate)
eviction.On("IsInPlaceUpdating", pod).Return(isInPlaceUpdating)
eviction.On("InPlaceUpdate", pod, nil).Return(nil)
factory := &fakeEvictFactory{eviction}
updater := &updater{
evictionFactory: factory,
evictionRateLimiter: rate.NewLimiter(rate.Inf, 0),
lastInPlaceUpdateAttemptTimeMap: map[string]time.Time{podID: lastInPlace},
clock: baseclocktest.NewFakeClock(time.UnixMilli(3600001)), // 1 hour from epoch + 1 millis
}
fallback, err := updater.AttemptInPlaceUpdate(context.Background(), nil, pod, eviction)
if expectInPlaceUpdated {
eviction.AssertCalled(t, "InPlaceUpdate", pod, nil)
} else {
eviction.AssertNotCalled(t, "InPlaceUpdate", pod, nil)
}
if expectError {
assert.Error(t, err)
} else {
assert.NoError(t, err)
}
assert.Equal(t, expectedFallBackToEviction, fallback)
}

View File

@ -0,0 +1,46 @@
/*
Copyright 2017 The Kubernetes 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 restriction
import (
apiv1 "k8s.io/api/core/v1"
vpa_types "k8s.io/autoscaler/vertical-pod-autoscaler/pkg/apis/autoscaling.k8s.io/v1"
)
// FakePodsRestrictionFactory is a fake implementation of the PodsRestrictionFactory interface.
type FakePodsRestrictionFactory struct {
// Eviction is the fake eviction restriction.
Eviction PodsEvictionRestriction
// InPlace is the fake in-place restriction.
InPlace PodsInPlaceRestriction
}
// NewPodsEvictionRestriction returns the fake eviction restriction.
func (f *FakePodsRestrictionFactory) NewPodsEvictionRestriction(creatorToSingleGroupStatsMap map[podReplicaCreator]singleGroupStats, podToReplicaCreatorMap map[string]podReplicaCreator) PodsEvictionRestriction {
return f.Eviction
}
// NewPodsInPlaceRestriction returns the fake in-place restriction.
func (f *FakePodsRestrictionFactory) NewPodsInPlaceRestriction(creatorToSingleGroupStatsMap map[podReplicaCreator]singleGroupStats, podToReplicaCreatorMap map[string]podReplicaCreator) PodsInPlaceRestriction {
return f.InPlace
}
// GetCreatorMaps returns nil maps.
func (f *FakePodsRestrictionFactory) GetCreatorMaps(pods []*apiv1.Pod, vpa *vpa_types.VerticalPodAutoscaler) (map[podReplicaCreator]singleGroupStats, map[string]podReplicaCreator, error) {
return nil, nil, nil
}

View File

@ -0,0 +1,112 @@
/*
Copyright 2017 The Kubernetes 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 restriction
import (
"context"
"fmt"
"time"
apiv1 "k8s.io/api/core/v1"
policyv1 "k8s.io/api/policy/v1"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
kube_client "k8s.io/client-go/kubernetes"
"k8s.io/client-go/tools/record"
"k8s.io/klog/v2"
"k8s.io/utils/clock"
vpa_types "k8s.io/autoscaler/vertical-pod-autoscaler/pkg/apis/autoscaling.k8s.io/v1"
)
// PodsEvictionRestriction controls pods evictions. It ensures that we will not evict too
// many pods from one replica set. For replica set will allow to evict one pod or more if
// evictionToleranceFraction is configured.
type PodsEvictionRestriction interface {
// Evict sends eviction instruction to the api client.
// Returns error if pod cannot be evicted or if client returned error.
Evict(pod *apiv1.Pod, vpa *vpa_types.VerticalPodAutoscaler, eventRecorder record.EventRecorder) error
// CanEvict checks if pod can be safely evicted
CanEvict(pod *apiv1.Pod) bool
}
// PodsEvictionRestrictionImpl is the implementation of the PodsEvictionRestriction interface.
type PodsEvictionRestrictionImpl struct {
client kube_client.Interface
podToReplicaCreatorMap map[string]podReplicaCreator
creatorToSingleGroupStatsMap map[podReplicaCreator]singleGroupStats
clock clock.Clock
lastInPlaceAttemptTimeMap map[string]time.Time
}
// CanEvict checks if pod can be safely evicted
func (e *PodsEvictionRestrictionImpl) CanEvict(pod *apiv1.Pod) bool {
cr, present := e.podToReplicaCreatorMap[getPodID(pod)]
if present {
singleGroupStats, present := e.creatorToSingleGroupStatsMap[cr]
if pod.Status.Phase == apiv1.PodPending {
return true
}
if present {
if isInPlaceUpdating(pod) {
return CanEvictInPlacingPod(pod, singleGroupStats, e.lastInPlaceAttemptTimeMap, e.clock)
}
return singleGroupStats.isPodDisruptable()
}
}
return false
}
// Evict sends eviction instruction to api client. Returns error if pod cannot be evicted or if client returned error
// Does not check if pod was actually evicted after eviction grace period.
func (e *PodsEvictionRestrictionImpl) Evict(podToEvict *apiv1.Pod, vpa *vpa_types.VerticalPodAutoscaler, eventRecorder record.EventRecorder) error {
cr, present := e.podToReplicaCreatorMap[getPodID(podToEvict)]
if !present {
return fmt.Errorf("pod not suitable for eviction %s/%s: not in replicated pods map", podToEvict.Namespace, podToEvict.Name)
}
if !e.CanEvict(podToEvict) {
return fmt.Errorf("cannot evict pod %s/%s: eviction budget exceeded", podToEvict.Namespace, podToEvict.Name)
}
eviction := &policyv1.Eviction{
ObjectMeta: metav1.ObjectMeta{
Namespace: podToEvict.Namespace,
Name: podToEvict.Name,
},
}
err := e.client.CoreV1().Pods(podToEvict.Namespace).EvictV1(context.TODO(), eviction)
if err != nil {
klog.ErrorS(err, "Failed to evict pod", "pod", klog.KObj(podToEvict))
return err
}
eventRecorder.Event(podToEvict, apiv1.EventTypeNormal, "EvictedByVPA",
"Pod was evicted by VPA Updater to apply resource recommendation.")
eventRecorder.Event(vpa, apiv1.EventTypeNormal, "EvictedPod",
"VPA Updater evicted Pod "+podToEvict.Name+" to apply resource recommendation.")
if podToEvict.Status.Phase != apiv1.PodPending {
singleGroupStats, present := e.creatorToSingleGroupStatsMap[cr]
if !present {
return fmt.Errorf("Internal error - cannot find stats for replication group %v", cr)
}
singleGroupStats.evicted = singleGroupStats.evicted + 1
e.creatorToSingleGroupStatsMap[cr] = singleGroupStats
}
return nil
}

View File

@ -0,0 +1,259 @@
/*
Copyright 2017 The Kubernetes 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 restriction
import (
"fmt"
"testing"
"time"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
apiv1 "k8s.io/api/core/v1"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
baseclocktest "k8s.io/utils/clock/testing"
vpa_types "k8s.io/autoscaler/vertical-pod-autoscaler/pkg/apis/autoscaling.k8s.io/v1"
"k8s.io/autoscaler/vertical-pod-autoscaler/pkg/utils/test"
)
func TestEvictTooFewReplicas(t *testing.T) {
replicas := int32(5)
livePods := 5
rc := apiv1.ReplicationController{
ObjectMeta: metav1.ObjectMeta{
Name: "rc",
Namespace: "default",
},
TypeMeta: metav1.TypeMeta{
Kind: "ReplicationController",
},
Spec: apiv1.ReplicationControllerSpec{
Replicas: &replicas,
},
}
pods := make([]*apiv1.Pod, livePods)
for i := range pods {
pods[i] = test.Pod().WithName(getTestPodName(i)).WithCreator(&rc.ObjectMeta, &rc.TypeMeta).Get()
}
basicVpa := getBasicVpa()
factory, err := getRestrictionFactory(&rc, nil, nil, nil, 10, 0.5, nil, nil, nil)
assert.NoError(t, err)
creatorToSingleGroupStatsMap, podToReplicaCreatorMap, err := factory.GetCreatorMaps(pods, basicVpa)
assert.NoError(t, err)
eviction := factory.NewPodsEvictionRestriction(creatorToSingleGroupStatsMap, podToReplicaCreatorMap)
for _, pod := range pods {
assert.False(t, eviction.CanEvict(pod))
}
for _, pod := range pods {
err := eviction.Evict(pod, basicVpa, test.FakeEventRecorder())
assert.Error(t, err, "Error expected")
}
}
func TestEvictionTolerance(t *testing.T) {
replicas := int32(5)
livePods := 5
tolerance := 0.8
rc := apiv1.ReplicationController{
ObjectMeta: metav1.ObjectMeta{
Name: "rc",
Namespace: "default",
},
TypeMeta: metav1.TypeMeta{
Kind: "ReplicationController",
},
Spec: apiv1.ReplicationControllerSpec{
Replicas: &replicas,
},
}
pods := make([]*apiv1.Pod, livePods)
for i := range pods {
pods[i] = test.Pod().WithName(getTestPodName(i)).WithCreator(&rc.ObjectMeta, &rc.TypeMeta).Get()
}
basicVpa := getBasicVpa()
factory, err := getRestrictionFactory(&rc, nil, nil, nil, 2 /*minReplicas*/, tolerance, nil, nil, nil)
assert.NoError(t, err)
creatorToSingleGroupStatsMap, podToReplicaCreatorMap, err := factory.GetCreatorMaps(pods, basicVpa)
assert.NoError(t, err)
eviction := factory.NewPodsEvictionRestriction(creatorToSingleGroupStatsMap, podToReplicaCreatorMap)
for _, pod := range pods {
assert.True(t, eviction.CanEvict(pod))
}
for _, pod := range pods[:4] {
err := eviction.Evict(pod, basicVpa, test.FakeEventRecorder())
assert.Nil(t, err, "Should evict with no error")
}
for _, pod := range pods[4:] {
err := eviction.Evict(pod, basicVpa, test.FakeEventRecorder())
assert.Error(t, err, "Error expected")
}
}
func TestEvictAtLeastOne(t *testing.T) {
replicas := int32(5)
livePods := 5
tolerance := 0.1
rc := apiv1.ReplicationController{
ObjectMeta: metav1.ObjectMeta{
Name: "rc",
Namespace: "default",
},
TypeMeta: metav1.TypeMeta{
Kind: "ReplicationController",
},
Spec: apiv1.ReplicationControllerSpec{
Replicas: &replicas,
},
}
pods := make([]*apiv1.Pod, livePods)
for i := range pods {
pods[i] = test.Pod().WithName(getTestPodName(i)).WithCreator(&rc.ObjectMeta, &rc.TypeMeta).Get()
}
basicVpa := getBasicVpa()
factory, err := getRestrictionFactory(&rc, nil, nil, nil, 2, tolerance, nil, nil, nil)
assert.NoError(t, err)
creatorToSingleGroupStatsMap, podToReplicaCreatorMap, err := factory.GetCreatorMaps(pods, basicVpa)
assert.NoError(t, err)
eviction := factory.NewPodsEvictionRestriction(creatorToSingleGroupStatsMap, podToReplicaCreatorMap)
for _, pod := range pods {
assert.True(t, eviction.CanEvict(pod))
}
for _, pod := range pods[:1] {
err := eviction.Evict(pod, basicVpa, test.FakeEventRecorder())
assert.Nil(t, err, "Should evict with no error")
}
for _, pod := range pods[1:] {
err := eviction.Evict(pod, basicVpa, test.FakeEventRecorder())
assert.Error(t, err, "Error expected")
}
}
func TestEvictEmitEvent(t *testing.T) {
rc := apiv1.ReplicationController{
ObjectMeta: metav1.ObjectMeta{
Name: "rc",
Namespace: "default",
},
TypeMeta: metav1.TypeMeta{
Kind: "ReplicationController",
},
}
index := 0
generatePod := func() test.PodBuilder {
index++
return test.Pod().WithName(fmt.Sprintf("test-%v", index)).WithCreator(&rc.ObjectMeta, &rc.TypeMeta)
}
basicVpa := getBasicVpa()
testCases := []struct {
name string
replicas int32
evictionTolerance float64
vpa *vpa_types.VerticalPodAutoscaler
pods []podWithExpectations
errorExpected bool
}{
{
name: "Pods that can be evicted",
replicas: 4,
evictionTolerance: 0.5,
vpa: basicVpa,
pods: []podWithExpectations{
{
pod: generatePod().WithPhase(apiv1.PodPending).Get(),
canEvict: true,
evictionSuccess: true,
},
{
pod: generatePod().WithPhase(apiv1.PodPending).Get(),
canEvict: true,
evictionSuccess: true,
},
},
errorExpected: false,
},
{
name: "Pod that can not be evicted",
replicas: 4,
evictionTolerance: 0.5,
vpa: basicVpa,
pods: []podWithExpectations{
{
pod: generatePod().Get(),
canEvict: false,
evictionSuccess: false,
},
},
errorExpected: true,
},
}
for _, testCase := range testCases {
rc.Spec = apiv1.ReplicationControllerSpec{
Replicas: &testCase.replicas,
}
pods := make([]*apiv1.Pod, 0, len(testCase.pods))
for _, p := range testCase.pods {
pods = append(pods, p.pod)
}
clock := baseclocktest.NewFakeClock(time.Time{})
factory, err := getRestrictionFactory(&rc, nil, nil, nil, 2, testCase.evictionTolerance, clock, map[string]time.Time{}, nil)
assert.NoError(t, err)
creatorToSingleGroupStatsMap, podToReplicaCreatorMap, err := factory.GetCreatorMaps(pods, testCase.vpa)
assert.NoError(t, err)
eviction := factory.NewPodsEvictionRestriction(creatorToSingleGroupStatsMap, podToReplicaCreatorMap)
for _, p := range testCase.pods {
mockRecorder := test.MockEventRecorder()
mockRecorder.On("Event", mock.Anything, apiv1.EventTypeNormal, "EvictedByVPA", mock.Anything).Return()
mockRecorder.On("Event", mock.Anything, apiv1.EventTypeNormal, "EvictedPod", mock.Anything).Return()
errGot := eviction.Evict(p.pod, testCase.vpa, mockRecorder)
if testCase.errorExpected {
assert.Error(t, errGot)
} else {
assert.NoError(t, errGot)
}
if p.canEvict {
mockRecorder.AssertNumberOfCalls(t, "Event", 2)
} else {
mockRecorder.AssertNumberOfCalls(t, "Event", 0)
}
}
}
}

View File

@ -0,0 +1,176 @@
/*
Copyright 2015 The Kubernetes 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 restriction
import (
"context"
"fmt"
"time"
apiv1 "k8s.io/api/core/v1"
kube_client "k8s.io/client-go/kubernetes"
"k8s.io/client-go/tools/record"
"k8s.io/klog/v2"
"k8s.io/utils/clock"
vpa_types "k8s.io/autoscaler/vertical-pod-autoscaler/pkg/apis/autoscaling.k8s.io/v1"
"k8s.io/autoscaler/vertical-pod-autoscaler/pkg/features"
"encoding/json"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
k8stypes "k8s.io/apimachinery/pkg/types"
utils "k8s.io/autoscaler/vertical-pod-autoscaler/pkg/updater/utils"
resource_updates "k8s.io/autoscaler/vertical-pod-autoscaler/pkg/admission-controller/resource"
"k8s.io/autoscaler/vertical-pod-autoscaler/pkg/admission-controller/resource/pod/patch"
)
// TODO: Make these configurable by flags
const (
// DeferredResizeUpdateTimeout defines the duration during which an in-place resize request
// is considered deferred. If the resize is not completed within this time, it falls back to eviction.
DeferredResizeUpdateTimeout = 5 * time.Minute
// InProgressResizeUpdateTimeout defines the duration during which an in-place resize request
// is considered in progress. If the resize is not completed within this time, it falls back to eviction.
InProgressResizeUpdateTimeout = 1 * time.Hour
)
// PodsInPlaceRestriction controls pods in-place updates. It ensures that we will not update too
// many pods from one replica set. For replica set will allow to update one pod or more if
// inPlaceToleranceFraction is configured.
type PodsInPlaceRestriction interface {
// InPlaceUpdate attempts to actuate the in-place resize.
// Returns error if client returned error.
InPlaceUpdate(pod *apiv1.Pod, vpa *vpa_types.VerticalPodAutoscaler, eventRecorder record.EventRecorder) error
// CanInPlaceUpdate checks if pod can be safely updated in-place. If not, it will return a decision to potentially evict the pod.
CanInPlaceUpdate(pod *apiv1.Pod) utils.InPlaceDecision
}
// PodsInPlaceRestrictionImpl is the implementation of the PodsInPlaceRestriction interface.
type PodsInPlaceRestrictionImpl struct {
client kube_client.Interface
podToReplicaCreatorMap map[string]podReplicaCreator
creatorToSingleGroupStatsMap map[podReplicaCreator]singleGroupStats
patchCalculators []patch.Calculator
clock clock.Clock
lastInPlaceAttemptTimeMap map[string]time.Time
}
// CanInPlaceUpdate checks if pod can be safely updated
func (ip *PodsInPlaceRestrictionImpl) CanInPlaceUpdate(pod *apiv1.Pod) utils.InPlaceDecision {
if !features.Enabled(features.InPlaceOrRecreate) {
return utils.InPlaceEvict
}
cr, present := ip.podToReplicaCreatorMap[getPodID(pod)]
if present {
singleGroupStats, present := ip.creatorToSingleGroupStatsMap[cr]
if pod.Status.Phase == apiv1.PodPending {
return utils.InPlaceDeferred
}
if present {
if isInPlaceUpdating(pod) {
canEvict := CanEvictInPlacingPod(pod, singleGroupStats, ip.lastInPlaceAttemptTimeMap, ip.clock)
if canEvict {
return utils.InPlaceEvict
}
return utils.InPlaceDeferred
}
if singleGroupStats.isPodDisruptable() {
return utils.InPlaceApproved
}
}
}
klog.V(4).InfoS("Can't in-place update pod, but not falling back to eviction. Waiting for next loop", "pod", klog.KObj(pod))
return utils.InPlaceDeferred
}
// InPlaceUpdate sends calculates patches and sends resize request to api client. Returns error if pod cannot be in-place updated or if client returned error.
// Does not check if pod was actually in-place updated after grace period.
func (ip *PodsInPlaceRestrictionImpl) InPlaceUpdate(podToUpdate *apiv1.Pod, vpa *vpa_types.VerticalPodAutoscaler, eventRecorder record.EventRecorder) error {
cr, present := ip.podToReplicaCreatorMap[getPodID(podToUpdate)]
if !present {
return fmt.Errorf("pod not suitable for in-place update %v: not in replicated pods map", podToUpdate.Name)
}
if ip.CanInPlaceUpdate(podToUpdate) != utils.InPlaceApproved {
return fmt.Errorf("cannot in-place update pod %s", klog.KObj(podToUpdate))
}
// separate patches since we have to patch resize and spec separately
resizePatches := []resource_updates.PatchRecord{}
annotationPatches := []resource_updates.PatchRecord{}
if podToUpdate.Annotations == nil {
annotationPatches = append(annotationPatches, patch.GetAddEmptyAnnotationsPatch())
}
for _, calculator := range ip.patchCalculators {
p, err := calculator.CalculatePatches(podToUpdate, vpa)
if err != nil {
return err
}
klog.V(4).InfoS("Calculated patches for pod", "pod", klog.KObj(podToUpdate), "patches", p)
if calculator.PatchResourceTarget() == patch.Resize {
resizePatches = append(resizePatches, p...)
} else {
annotationPatches = append(annotationPatches, p...)
}
}
if len(resizePatches) > 0 {
patch, err := json.Marshal(resizePatches)
if err != nil {
return err
}
res, err := ip.client.CoreV1().Pods(podToUpdate.Namespace).Patch(context.TODO(), podToUpdate.Name, k8stypes.JSONPatchType, patch, metav1.PatchOptions{}, "resize")
if err != nil {
return err
}
klog.V(4).InfoS("In-place patched pod /resize subresource using patches", "pod", klog.KObj(res), "patches", string(patch))
if len(annotationPatches) > 0 {
patch, err := json.Marshal(annotationPatches)
if err != nil {
return err
}
res, err = ip.client.CoreV1().Pods(podToUpdate.Namespace).Patch(context.TODO(), podToUpdate.Name, k8stypes.JSONPatchType, patch, metav1.PatchOptions{})
if err != nil {
return err
}
klog.V(4).InfoS("Patched pod annotations", "pod", klog.KObj(res), "patches", string(patch))
}
} else {
return fmt.Errorf("no resource patches were calculated to apply")
}
// TODO(maxcao13): If this keeps getting called on the same object with the same reason, it is considered a patch request.
// And we fail to have the corresponding rbac for it. So figure out if we need this later.
// Do we even need to emit an event? The node might reject the resize request. If so, should we rename this to InPlaceResizeAttempted?
// eventRecorder.Event(podToUpdate, apiv1.EventTypeNormal, "InPlaceResizedByVPA", "Pod was resized in place by VPA Updater.")
singleGroupStats, present := ip.creatorToSingleGroupStatsMap[cr]
if !present {
klog.InfoS("Internal error - cannot find stats for replication group", "pod", klog.KObj(podToUpdate), "podReplicaCreator", cr)
} else {
singleGroupStats.inPlaceUpdateInitiated = singleGroupStats.inPlaceUpdateInitiated + 1
ip.creatorToSingleGroupStatsMap[cr] = singleGroupStats
}
return nil
}

View File

@ -0,0 +1,360 @@
/*
Copyright 2025 The Kubernetes 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 restriction
import (
"fmt"
"testing"
"time"
"github.com/stretchr/testify/assert"
apiv1 "k8s.io/api/core/v1"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
featuregatetesting "k8s.io/component-base/featuregate/testing"
baseclocktest "k8s.io/utils/clock/testing"
"k8s.io/autoscaler/vertical-pod-autoscaler/pkg/features"
utils "k8s.io/autoscaler/vertical-pod-autoscaler/pkg/updater/utils"
"k8s.io/autoscaler/vertical-pod-autoscaler/pkg/utils/test"
)
type CanInPlaceUpdateTestParams struct {
name string
pods []*apiv1.Pod
replicas int32
evictionTolerance float64
lastInPlaceAttempt time.Time
expectedInPlaceDecision utils.InPlaceDecision
}
func TestCanInPlaceUpdate(t *testing.T) {
featuregatetesting.SetFeatureGateDuringTest(t, features.MutableFeatureGate, features.InPlaceOrRecreate, true)
rc := apiv1.ReplicationController{
ObjectMeta: metav1.ObjectMeta{
Name: "rc",
Namespace: "default",
},
TypeMeta: metav1.TypeMeta{
Kind: "ReplicationController",
},
}
index := 0
generatePod := func() test.PodBuilder {
index++
return test.Pod().WithName(fmt.Sprintf("test-%v", index)).WithCreator(&rc.ObjectMeta, &rc.TypeMeta)
}
// NOTE: the pod we are checking for CanInPlaceUpdate will always be the first one for these tests
whichPodIdxForCanInPlaceUpdate := 0
testCases := []CanInPlaceUpdateTestParams{
{
name: "CanInPlaceUpdate=InPlaceApproved - (half of 3)",
pods: []*apiv1.Pod{
generatePod().Get(),
generatePod().Get(),
generatePod().Get(),
},
replicas: 3,
evictionTolerance: 0.5,
lastInPlaceAttempt: time.Time{},
expectedInPlaceDecision: utils.InPlaceApproved,
},
{
name: "CanInPlaceUpdate=InPlaceDeferred - no pods can be in-placed, one missing",
pods: []*apiv1.Pod{
generatePod().Get(),
generatePod().Get(),
},
replicas: 3,
evictionTolerance: 0.5,
lastInPlaceAttempt: time.Time{},
expectedInPlaceDecision: utils.InPlaceDeferred,
},
{
name: "CanInPlaceUpdate=InPlaceApproved - small tolerance, all running",
pods: []*apiv1.Pod{
generatePod().Get(),
generatePod().Get(),
generatePod().Get(),
},
replicas: 3,
evictionTolerance: 0.1,
lastInPlaceAttempt: time.Time{},
expectedInPlaceDecision: utils.InPlaceApproved,
},
{
name: "CanInPlaceUpdate=InPlaceApproved - small tolerance, one missing",
pods: []*apiv1.Pod{
generatePod().Get(),
generatePod().Get(),
},
replicas: 3,
evictionTolerance: 0.5,
lastInPlaceAttempt: time.Time{},
expectedInPlaceDecision: utils.InPlaceDeferred,
},
{
name: "CanInPlaceUpdate=InPlaceDeferred - resize Deferred, conditions not met to fallback",
pods: []*apiv1.Pod{
generatePod().WithResizeStatus(apiv1.PodResizeStatusDeferred).Get(),
generatePod().Get(),
generatePod().Get(),
},
replicas: 3,
evictionTolerance: 0.5,
lastInPlaceAttempt: time.UnixMilli(3600000), // 1 hour from epoch
expectedInPlaceDecision: utils.InPlaceDeferred,
},
{
name: ("CanInPlaceUpdate=InPlaceEvict - resize inProgress for more too long"),
pods: []*apiv1.Pod{
generatePod().WithResizeStatus(apiv1.PodResizeStatusInProgress).Get(),
generatePod().Get(),
generatePod().Get(),
},
replicas: 3,
evictionTolerance: 0.5,
lastInPlaceAttempt: time.UnixMilli(0), // epoch (too long ago...)
expectedInPlaceDecision: utils.InPlaceEvict,
},
{
name: "CanInPlaceUpdate=InPlaceDeferred - resize InProgress, conditions not met to fallback",
pods: []*apiv1.Pod{
generatePod().WithResizeStatus(apiv1.PodResizeStatusInProgress).Get(),
generatePod().Get(),
generatePod().Get(),
},
replicas: 3,
evictionTolerance: 0.5,
lastInPlaceAttempt: time.UnixMilli(3600000), // 1 hour from epoch
expectedInPlaceDecision: utils.InPlaceDeferred,
},
{
name: "CanInPlaceUpdate=InPlaceEvict - infeasible",
pods: []*apiv1.Pod{
generatePod().WithResizeStatus(apiv1.PodResizeStatusInfeasible).Get(),
generatePod().Get(),
generatePod().Get(),
},
replicas: 3,
evictionTolerance: 0.5,
lastInPlaceAttempt: time.Time{},
expectedInPlaceDecision: utils.InPlaceEvict,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
rc.Spec = apiv1.ReplicationControllerSpec{
Replicas: &tc.replicas,
}
selectedPod := tc.pods[whichPodIdxForCanInPlaceUpdate]
clock := baseclocktest.NewFakeClock(time.UnixMilli(3600001)) // 1 hour from epoch + 1 millis
lipatm := map[string]time.Time{getPodID(selectedPod): tc.lastInPlaceAttempt}
factory, err := getRestrictionFactory(&rc, nil, nil, nil, 2, tc.evictionTolerance, clock, lipatm, GetFakeCalculatorsWithFakeResourceCalc())
assert.NoError(t, err)
creatorToSingleGroupStatsMap, podToReplicaCreatorMap, err := factory.GetCreatorMaps(tc.pods, getIPORVpa())
assert.NoError(t, err)
inPlace := factory.NewPodsInPlaceRestriction(creatorToSingleGroupStatsMap, podToReplicaCreatorMap)
result := inPlace.CanInPlaceUpdate(selectedPod)
assert.Equal(t, tc.expectedInPlaceDecision, result)
})
}
}
func TestInPlaceDisabledFeatureGate(t *testing.T) {
featuregatetesting.SetFeatureGateDuringTest(t, features.MutableFeatureGate, features.InPlaceOrRecreate, false)
replicas := int32(5)
livePods := 5
tolerance := 1.0
rc := apiv1.ReplicationController{
ObjectMeta: metav1.ObjectMeta{
Name: "rc",
Namespace: "default",
},
TypeMeta: metav1.TypeMeta{
Kind: "ReplicationController",
},
Spec: apiv1.ReplicationControllerSpec{
Replicas: &replicas,
},
}
pods := make([]*apiv1.Pod, livePods)
for i := range pods {
pods[i] = test.Pod().WithName(getTestPodName(i)).WithCreator(&rc.ObjectMeta, &rc.TypeMeta).Get()
}
basicVpa := getBasicVpa()
factory, err := getRestrictionFactory(&rc, nil, nil, nil, 2, tolerance, nil, nil, GetFakeCalculatorsWithFakeResourceCalc())
assert.NoError(t, err)
creatorToSingleGroupStatsMap, podToReplicaCreatorMap, err := factory.GetCreatorMaps(pods, basicVpa)
assert.NoError(t, err)
inplace := factory.NewPodsInPlaceRestriction(creatorToSingleGroupStatsMap, podToReplicaCreatorMap)
for _, pod := range pods {
assert.Equal(t, utils.InPlaceEvict, inplace.CanInPlaceUpdate(pod))
}
}
func TestInPlaceTooFewReplicas(t *testing.T) {
featuregatetesting.SetFeatureGateDuringTest(t, features.MutableFeatureGate, features.InPlaceOrRecreate, true)
replicas := int32(5)
livePods := 5
tolerance := 0.5
rc := apiv1.ReplicationController{
ObjectMeta: metav1.ObjectMeta{
Name: "rc",
Namespace: "default",
},
TypeMeta: metav1.TypeMeta{
Kind: "ReplicationController",
},
Spec: apiv1.ReplicationControllerSpec{
Replicas: &replicas,
},
}
pods := make([]*apiv1.Pod, livePods)
for i := range pods {
pods[i] = test.Pod().WithName(getTestPodName(i)).WithCreator(&rc.ObjectMeta, &rc.TypeMeta).Get()
}
clock := baseclocktest.NewFakeClock(time.Time{})
lipatm := map[string]time.Time{}
basicVpa := getIPORVpa()
factory, err := getRestrictionFactory(&rc, nil, nil, nil, 10 /*minReplicas*/, tolerance, clock, lipatm, GetFakeCalculatorsWithFakeResourceCalc())
assert.NoError(t, err)
creatorToSingleGroupStatsMap, podToReplicaCreatorMap, err := factory.GetCreatorMaps(pods, basicVpa)
assert.NoError(t, err)
inplace := factory.NewPodsInPlaceRestriction(creatorToSingleGroupStatsMap, podToReplicaCreatorMap)
for _, pod := range pods {
assert.Equal(t, utils.InPlaceDeferred, inplace.CanInPlaceUpdate(pod))
}
for _, pod := range pods {
err := inplace.InPlaceUpdate(pod, basicVpa, test.FakeEventRecorder())
assert.Error(t, err, "Error expected")
}
}
func TestEvictionToleranceForInPlace(t *testing.T) {
featuregatetesting.SetFeatureGateDuringTest(t, features.MutableFeatureGate, features.InPlaceOrRecreate, true)
replicas := int32(5)
livePods := 5
tolerance := 0.8
rc := apiv1.ReplicationController{
ObjectMeta: metav1.ObjectMeta{
Name: "rc",
Namespace: "default",
},
TypeMeta: metav1.TypeMeta{
Kind: "ReplicationController",
},
Spec: apiv1.ReplicationControllerSpec{
Replicas: &replicas,
},
}
pods := make([]*apiv1.Pod, livePods)
for i := range pods {
pods[i] = test.Pod().WithName(getTestPodName(i)).WithCreator(&rc.ObjectMeta, &rc.TypeMeta).Get()
}
clock := baseclocktest.NewFakeClock(time.Time{})
lipatm := map[string]time.Time{}
basicVpa := getIPORVpa()
factory, err := getRestrictionFactory(&rc, nil, nil, nil, 2 /*minReplicas*/, tolerance, clock, lipatm, GetFakeCalculatorsWithFakeResourceCalc())
assert.NoError(t, err)
creatorToSingleGroupStatsMap, podToReplicaCreatorMap, err := factory.GetCreatorMaps(pods, basicVpa)
assert.NoError(t, err)
inplace := factory.NewPodsInPlaceRestriction(creatorToSingleGroupStatsMap, podToReplicaCreatorMap)
for _, pod := range pods {
assert.Equal(t, utils.InPlaceApproved, inplace.CanInPlaceUpdate(pod))
}
for _, pod := range pods[:4] {
err := inplace.InPlaceUpdate(pod, basicVpa, test.FakeEventRecorder())
assert.Nil(t, err, "Should evict with no error")
}
for _, pod := range pods[4:] {
err := inplace.InPlaceUpdate(pod, basicVpa, test.FakeEventRecorder())
assert.Error(t, err, "Error expected")
}
}
func TestInPlaceAtLeastOne(t *testing.T) {
featuregatetesting.SetFeatureGateDuringTest(t, features.MutableFeatureGate, features.InPlaceOrRecreate, true)
replicas := int32(5)
livePods := 5
tolerance := 0.1
rc := apiv1.ReplicationController{
ObjectMeta: metav1.ObjectMeta{
Name: "rc",
Namespace: "default",
},
TypeMeta: metav1.TypeMeta{
Kind: "ReplicationController",
},
Spec: apiv1.ReplicationControllerSpec{
Replicas: &replicas,
},
}
pods := make([]*apiv1.Pod, livePods)
for i := range pods {
pods[i] = test.Pod().WithName(getTestPodName(i)).WithCreator(&rc.ObjectMeta, &rc.TypeMeta).Get()
}
basicVpa := getBasicVpa()
factory, err := getRestrictionFactory(&rc, nil, nil, nil, 2, tolerance, nil, nil, GetFakeCalculatorsWithFakeResourceCalc())
assert.NoError(t, err)
creatorToSingleGroupStatsMap, podToReplicaCreatorMap, err := factory.GetCreatorMaps(pods, basicVpa)
assert.NoError(t, err)
inplace := factory.NewPodsInPlaceRestriction(creatorToSingleGroupStatsMap, podToReplicaCreatorMap)
for _, pod := range pods {
assert.Equal(t, utils.InPlaceApproved, inplace.CanInPlaceUpdate(pod))
}
for _, pod := range pods[:1] {
err := inplace.InPlaceUpdate(pod, basicVpa, test.FakeEventRecorder())
assert.Nil(t, err, "Should in-place update with no error")
}
for _, pod := range pods[1:] {
err := inplace.InPlaceUpdate(pod, basicVpa, test.FakeEventRecorder())
assert.Error(t, err, "Error expected")
}
}

View File

@ -0,0 +1,394 @@
/*
Copyright 2025 The Kubernetes 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 restriction
import (
"fmt"
"time"
appsv1 "k8s.io/api/apps/v1"
apiv1 "k8s.io/api/core/v1"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
appsinformer "k8s.io/client-go/informers/apps/v1"
coreinformer "k8s.io/client-go/informers/core/v1"
kube_client "k8s.io/client-go/kubernetes"
"k8s.io/client-go/tools/cache"
"k8s.io/klog/v2"
"k8s.io/utils/clock"
"k8s.io/autoscaler/vertical-pod-autoscaler/pkg/admission-controller/resource/pod/patch"
vpa_types "k8s.io/autoscaler/vertical-pod-autoscaler/pkg/apis/autoscaling.k8s.io/v1"
)
const (
resyncPeriod time.Duration = 1 * time.Minute
)
// ControllerKind is the type of controller that can manage a pod.
type controllerKind string
const (
replicationController controllerKind = "ReplicationController"
statefulSet controllerKind = "StatefulSet"
replicaSet controllerKind = "ReplicaSet"
daemonSet controllerKind = "DaemonSet"
job controllerKind = "Job"
)
type podReplicaCreator struct {
Namespace string
Name string
Kind controllerKind
}
// PodsRestrictionFactory is a factory for creating PodsEvictionRestriction and PodsInPlaceRestriction.
type PodsRestrictionFactory interface {
GetCreatorMaps(pods []*apiv1.Pod, vpa *vpa_types.VerticalPodAutoscaler) (map[podReplicaCreator]singleGroupStats, map[string]podReplicaCreator, error)
NewPodsEvictionRestriction(creatorToSingleGroupStatsMap map[podReplicaCreator]singleGroupStats, podToReplicaCreatorMap map[string]podReplicaCreator) PodsEvictionRestriction
NewPodsInPlaceRestriction(creatorToSingleGroupStatsMap map[podReplicaCreator]singleGroupStats, podToReplicaCreatorMap map[string]podReplicaCreator) PodsInPlaceRestriction
}
// PodsRestrictionFactoryImpl is the implementation of the PodsRestrictionFactory interface.
type PodsRestrictionFactoryImpl struct {
client kube_client.Interface
rcInformer cache.SharedIndexInformer // informer for Replication Controllers
ssInformer cache.SharedIndexInformer // informer for Stateful Sets
rsInformer cache.SharedIndexInformer // informer for Replica Sets
dsInformer cache.SharedIndexInformer // informer for Daemon Sets
minReplicas int
evictionToleranceFraction float64
clock clock.Clock
lastInPlaceAttemptTimeMap map[string]time.Time
patchCalculators []patch.Calculator
}
// NewPodsRestrictionFactory creates a new PodsRestrictionFactory.
func NewPodsRestrictionFactory(client kube_client.Interface, minReplicas int, evictionToleranceFraction float64, patchCalculators []patch.Calculator) (PodsRestrictionFactory, error) {
rcInformer, err := setupInformer(client, replicationController)
if err != nil {
return nil, fmt.Errorf("Failed to create rcInformer: %v", err)
}
ssInformer, err := setupInformer(client, statefulSet)
if err != nil {
return nil, fmt.Errorf("Failed to create ssInformer: %v", err)
}
rsInformer, err := setupInformer(client, replicaSet)
if err != nil {
return nil, fmt.Errorf("Failed to create rsInformer: %v", err)
}
dsInformer, err := setupInformer(client, daemonSet)
if err != nil {
return nil, fmt.Errorf("Failed to create dsInformer: %v", err)
}
return &PodsRestrictionFactoryImpl{
client: client,
rcInformer: rcInformer, // informer for Replication Controllers
ssInformer: ssInformer, // informer for Stateful Sets
rsInformer: rsInformer, // informer for Replica Sets
dsInformer: dsInformer, // informer for Daemon Sets
minReplicas: minReplicas,
evictionToleranceFraction: evictionToleranceFraction,
clock: &clock.RealClock{},
lastInPlaceAttemptTimeMap: make(map[string]time.Time),
patchCalculators: patchCalculators,
}, nil
}
func (f *PodsRestrictionFactoryImpl) getReplicaCount(creator podReplicaCreator) (int, error) {
switch creator.Kind {
case replicationController:
rcObj, exists, err := f.rcInformer.GetStore().GetByKey(creator.Namespace + "/" + creator.Name)
if err != nil {
return 0, fmt.Errorf("replication controller %s/%s is not available, err: %v", creator.Namespace, creator.Name, err)
}
if !exists {
return 0, fmt.Errorf("replication controller %s/%s does not exist", creator.Namespace, creator.Name)
}
rc, ok := rcObj.(*apiv1.ReplicationController)
if !ok {
return 0, fmt.Errorf("Failed to parse Replication Controller")
}
if rc.Spec.Replicas == nil || *rc.Spec.Replicas == 0 {
return 0, fmt.Errorf("replication controller %s/%s has no replicas config", creator.Namespace, creator.Name)
}
return int(*rc.Spec.Replicas), nil
case replicaSet:
rsObj, exists, err := f.rsInformer.GetStore().GetByKey(creator.Namespace + "/" + creator.Name)
if err != nil {
return 0, fmt.Errorf("replica set %s/%s is not available, err: %v", creator.Namespace, creator.Name, err)
}
if !exists {
return 0, fmt.Errorf("replica set %s/%s does not exist", creator.Namespace, creator.Name)
}
rs, ok := rsObj.(*appsv1.ReplicaSet)
if !ok {
return 0, fmt.Errorf("Failed to parse Replicaset")
}
if rs.Spec.Replicas == nil || *rs.Spec.Replicas == 0 {
return 0, fmt.Errorf("replica set %s/%s has no replicas config", creator.Namespace, creator.Name)
}
return int(*rs.Spec.Replicas), nil
case statefulSet:
ssObj, exists, err := f.ssInformer.GetStore().GetByKey(creator.Namespace + "/" + creator.Name)
if err != nil {
return 0, fmt.Errorf("stateful set %s/%s is not available, err: %v", creator.Namespace, creator.Name, err)
}
if !exists {
return 0, fmt.Errorf("stateful set %s/%s does not exist", creator.Namespace, creator.Name)
}
ss, ok := ssObj.(*appsv1.StatefulSet)
if !ok {
return 0, fmt.Errorf("Failed to parse StatefulSet")
}
if ss.Spec.Replicas == nil || *ss.Spec.Replicas == 0 {
return 0, fmt.Errorf("stateful set %s/%s has no replicas config", creator.Namespace, creator.Name)
}
return int(*ss.Spec.Replicas), nil
case daemonSet:
dsObj, exists, err := f.dsInformer.GetStore().GetByKey(creator.Namespace + "/" + creator.Name)
if err != nil {
return 0, fmt.Errorf("daemon set %s/%s is not available, err: %v", creator.Namespace, creator.Name, err)
}
if !exists {
return 0, fmt.Errorf("daemon set %s/%s does not exist", creator.Namespace, creator.Name)
}
ds, ok := dsObj.(*appsv1.DaemonSet)
if !ok {
return 0, fmt.Errorf("Failed to parse DaemonSet")
}
if ds.Status.NumberReady == 0 {
return 0, fmt.Errorf("daemon set %s/%s has no number ready pods", creator.Namespace, creator.Name)
}
return int(ds.Status.NumberReady), nil
}
return 0, nil
}
// GetCreatorMaps is a helper function that returns a map of pod replica creators to their single group stats
// and a map of pod ids to pod replica creator from a list of pods and it's corresponding VPA.
func (f *PodsRestrictionFactoryImpl) GetCreatorMaps(pods []*apiv1.Pod, vpa *vpa_types.VerticalPodAutoscaler) (map[podReplicaCreator]singleGroupStats, map[string]podReplicaCreator, error) {
livePods := make(map[podReplicaCreator][]*apiv1.Pod)
for _, pod := range pods {
creator, err := getPodReplicaCreator(pod)
if err != nil {
klog.ErrorS(err, "Failed to obtain replication info for pod", "pod", klog.KObj(pod))
continue
}
if creator == nil {
klog.V(0).InfoS("Pod is not managed by any controller", "pod", klog.KObj(pod))
continue
}
livePods[*creator] = append(livePods[*creator], pod)
}
podToReplicaCreatorMap := make(map[string]podReplicaCreator)
creatorToSingleGroupStatsMap := make(map[podReplicaCreator]singleGroupStats)
// Use per-VPA minReplicas if present, fall back to the global setting.
required := f.minReplicas
if vpa.Spec.UpdatePolicy != nil && vpa.Spec.UpdatePolicy.MinReplicas != nil {
required = int(*vpa.Spec.UpdatePolicy.MinReplicas)
klog.V(3).InfoS("Overriding minReplicas from global to per-VPA value", "globalMinReplicas", f.minReplicas, "vpaMinReplicas", required, "vpa", klog.KObj(vpa))
}
for creator, replicas := range livePods {
actual := len(replicas)
if actual < required {
klog.V(2).InfoS("Too few replicas", "kind", creator.Kind, "object", klog.KRef(creator.Namespace, creator.Name), "livePods", actual, "requiredPods", required, "globalMinReplicas", f.minReplicas)
continue
}
var configured int
if creator.Kind == job {
// Job has no replicas configuration, so we will use actual number of live pods as replicas count.
configured = actual
} else {
var err error
configured, err = f.getReplicaCount(creator)
if err != nil {
klog.ErrorS(err, "Failed to obtain replication info", "kind", creator.Kind, "object", klog.KRef(creator.Namespace, creator.Name))
continue
}
}
singleGroup := singleGroupStats{}
singleGroup.configured = configured
singleGroup.evictionTolerance = int(float64(configured) * f.evictionToleranceFraction) // truncated
for _, pod := range replicas {
podToReplicaCreatorMap[getPodID(pod)] = creator
if pod.Status.Phase == apiv1.PodPending {
singleGroup.pending = singleGroup.pending + 1
}
if isInPlaceUpdating(pod) {
singleGroup.inPlaceUpdateOngoing = singleGroup.inPlaceUpdateOngoing + 1
}
}
singleGroup.running = len(replicas) - singleGroup.pending
creatorToSingleGroupStatsMap[creator] = singleGroup
}
return creatorToSingleGroupStatsMap, podToReplicaCreatorMap, nil
}
// NewPodsEvictionRestriction creates a new PodsEvictionRestriction.
func (f *PodsRestrictionFactoryImpl) NewPodsEvictionRestriction(creatorToSingleGroupStatsMap map[podReplicaCreator]singleGroupStats, podToReplicaCreatorMap map[string]podReplicaCreator) PodsEvictionRestriction {
return &PodsEvictionRestrictionImpl{
client: f.client,
podToReplicaCreatorMap: podToReplicaCreatorMap,
creatorToSingleGroupStatsMap: creatorToSingleGroupStatsMap,
clock: f.clock,
lastInPlaceAttemptTimeMap: f.lastInPlaceAttemptTimeMap,
}
}
// NewPodsInPlaceRestriction creates a new PodsInPlaceRestriction.
func (f *PodsRestrictionFactoryImpl) NewPodsInPlaceRestriction(creatorToSingleGroupStatsMap map[podReplicaCreator]singleGroupStats, podToReplicaCreatorMap map[string]podReplicaCreator) PodsInPlaceRestriction {
return &PodsInPlaceRestrictionImpl{
client: f.client,
podToReplicaCreatorMap: podToReplicaCreatorMap,
creatorToSingleGroupStatsMap: creatorToSingleGroupStatsMap,
clock: f.clock,
lastInPlaceAttemptTimeMap: f.lastInPlaceAttemptTimeMap,
patchCalculators: f.patchCalculators,
}
}
func getPodID(pod *apiv1.Pod) string {
if pod == nil {
return ""
}
return pod.Namespace + "/" + pod.Name
}
func getPodReplicaCreator(pod *apiv1.Pod) (*podReplicaCreator, error) {
creator := managingControllerRef(pod)
if creator == nil {
return nil, nil
}
podReplicaCreator := &podReplicaCreator{
Namespace: pod.Namespace,
Name: creator.Name,
Kind: controllerKind(creator.Kind),
}
return podReplicaCreator, nil
}
func managingControllerRef(pod *apiv1.Pod) *metav1.OwnerReference {
var managingController metav1.OwnerReference
for _, ownerReference := range pod.ObjectMeta.GetOwnerReferences() {
if *ownerReference.Controller {
managingController = ownerReference
break
}
}
return &managingController
}
func setupInformer(kubeClient kube_client.Interface, kind controllerKind) (cache.SharedIndexInformer, error) {
var informer cache.SharedIndexInformer
switch kind {
case replicationController:
informer = coreinformer.NewReplicationControllerInformer(kubeClient, apiv1.NamespaceAll,
resyncPeriod, cache.Indexers{cache.NamespaceIndex: cache.MetaNamespaceIndexFunc})
case replicaSet:
informer = appsinformer.NewReplicaSetInformer(kubeClient, apiv1.NamespaceAll,
resyncPeriod, cache.Indexers{cache.NamespaceIndex: cache.MetaNamespaceIndexFunc})
case statefulSet:
informer = appsinformer.NewStatefulSetInformer(kubeClient, apiv1.NamespaceAll,
resyncPeriod, cache.Indexers{cache.NamespaceIndex: cache.MetaNamespaceIndexFunc})
case daemonSet:
informer = appsinformer.NewDaemonSetInformer(kubeClient, apiv1.NamespaceAll,
resyncPeriod, cache.Indexers{cache.NamespaceIndex: cache.MetaNamespaceIndexFunc})
default:
return nil, fmt.Errorf("Unknown controller kind: %v", kind)
}
stopCh := make(chan struct{})
go informer.Run(stopCh)
synced := cache.WaitForCacheSync(stopCh, informer.HasSynced)
if !synced {
return nil, fmt.Errorf("Failed to sync %v cache.", kind)
}
return informer, nil
}
type singleGroupStats struct {
configured int
pending int
running int
evictionTolerance int
evicted int
inPlaceUpdateOngoing int // number of pods from last loop that are still in-place updating
inPlaceUpdateInitiated int // number of pods from the current loop that have newly requested in-place resize
}
// isPodDisruptable checks if all pods are running and eviction tolerance is small, we can
// disrupt the current pod.
func (s *singleGroupStats) isPodDisruptable() bool {
shouldBeAlive := s.configured - s.evictionTolerance
actuallyAlive := s.running - (s.evicted + s.inPlaceUpdateInitiated)
return actuallyAlive > shouldBeAlive ||
(s.configured == s.running && s.evictionTolerance == 0 && s.evicted == 0 && s.inPlaceUpdateInitiated == 0)
// we don't want to block pods from being considered for eviction if tolerance is small and some pods are potentially stuck resizing
}
// isInPlaceUpdating checks whether or not the given pod is currently in the middle of an in-place update
func isInPlaceUpdating(podToCheck *apiv1.Pod) bool {
return podToCheck.Status.Resize != ""
}
// CanEvictInPlacingPod checks if the pod can be evicted while it is currently in the middle of an in-place update.
func CanEvictInPlacingPod(pod *apiv1.Pod, singleGroupStats singleGroupStats, lastInPlaceAttemptTimeMap map[string]time.Time, clock clock.Clock) bool {
if !isInPlaceUpdating(pod) {
return false
}
lastUpdate, exists := lastInPlaceAttemptTimeMap[getPodID(pod)]
if !exists {
klog.V(4).InfoS("In-place update in progress for pod but no lastUpdateTime found, setting it to now", "pod", klog.KObj(pod))
lastUpdate = clock.Now()
lastInPlaceAttemptTimeMap[getPodID(pod)] = lastUpdate
}
if singleGroupStats.isPodDisruptable() {
// TODO(maxcao13): fix this after 1.33 KEP changes
// if currently inPlaceUpdating, we should only fallback to eviction if the update has failed. i.e: one of the following conditions:
// 1. .status.resize: Infeasible
// 2. .status.resize: Deferred + more than 5 minutes has elapsed since the lastInPlaceUpdateTime
// 3. .status.resize: InProgress + more than 1 hour has elapsed since the lastInPlaceUpdateTime
switch pod.Status.Resize {
case apiv1.PodResizeStatusDeferred:
if clock.Since(lastUpdate) > DeferredResizeUpdateTimeout {
klog.V(4).InfoS(fmt.Sprintf("In-place update deferred for more than %v, falling back to eviction", DeferredResizeUpdateTimeout), "pod", klog.KObj(pod))
return true
}
case apiv1.PodResizeStatusInProgress:
if clock.Since(lastUpdate) > InProgressResizeUpdateTimeout {
klog.V(4).InfoS(fmt.Sprintf("In-place update in progress for more than %v, falling back to eviction", InProgressResizeUpdateTimeout), "pod", klog.KObj(pod))
return true
}
case apiv1.PodResizeStatusInfeasible:
klog.V(4).InfoS("In-place update infeasible, falling back to eviction", "pod", klog.KObj(pod))
return true
default:
klog.V(4).InfoS("In-place update status unknown, falling back to eviction", "pod", klog.KObj(pod))
return true
}
return false
}
klog.V(4).InfoS("Would be able to evict, but already resizing", "pod", klog.KObj(pod))
return false
}

View File

@ -14,7 +14,7 @@ See the License for the specific language governing permissions and
limitations under the License.
*/
package eviction
package restriction
import (
"fmt"
@ -22,7 +22,8 @@ import (
"time"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
resource_admission "k8s.io/autoscaler/vertical-pod-autoscaler/pkg/admission-controller/resource"
appsv1 "k8s.io/api/apps/v1"
batchv1 "k8s.io/api/batch/v1"
@ -32,28 +33,42 @@ import (
coreinformer "k8s.io/client-go/informers/core/v1"
"k8s.io/client-go/kubernetes/fake"
"k8s.io/client-go/tools/cache"
featuregatetesting "k8s.io/component-base/featuregate/testing"
"k8s.io/utils/clock"
baseclocktest "k8s.io/utils/clock/testing"
"k8s.io/utils/ptr"
"k8s.io/autoscaler/vertical-pod-autoscaler/pkg/admission-controller/resource/pod/patch"
vpa_types "k8s.io/autoscaler/vertical-pod-autoscaler/pkg/apis/autoscaling.k8s.io/v1"
"k8s.io/autoscaler/vertical-pod-autoscaler/pkg/features"
"k8s.io/autoscaler/vertical-pod-autoscaler/pkg/updater/utils"
"k8s.io/autoscaler/vertical-pod-autoscaler/pkg/utils/test"
vpa_api_util "k8s.io/autoscaler/vertical-pod-autoscaler/pkg/utils/vpa"
)
type podWithExpectations struct {
pod *apiv1.Pod
canEvict bool
evictionSuccess bool
pod *apiv1.Pod
canEvict bool
evictionSuccess bool
canInPlaceUpdate utils.InPlaceDecision
inPlaceUpdateSuccess bool
}
func getBasicVpa() *vpa_types.VerticalPodAutoscaler {
return test.VerticalPodAutoscaler().WithContainer("any").Get()
}
func getNoopPatchCalculators() []patch.Calculator {
return []patch.Calculator{}
func getIPORVpa() *vpa_types.VerticalPodAutoscaler {
vpa := getBasicVpa()
vpa.Spec.UpdatePolicy = &vpa_types.PodUpdatePolicy{
UpdateMode: ptr.To(vpa_types.UpdateModeInPlaceOrRecreate),
}
return vpa
}
func TestEvictReplicatedByController(t *testing.T) {
func TestDisruptReplicatedByController(t *testing.T) {
featuregatetesting.SetFeatureGateDuringTest(t, features.MutableFeatureGate, features.InPlaceOrRecreate, true)
rc := apiv1.ReplicationController{
ObjectMeta: metav1.ObjectMeta{
Name: "rc",
@ -79,7 +94,6 @@ func TestEvictReplicatedByController(t *testing.T) {
replicas int32
evictionTolerance float64
vpa *vpa_types.VerticalPodAutoscaler
calculators []patch.Calculator
pods []podWithExpectations
}{
{
@ -87,7 +101,6 @@ func TestEvictReplicatedByController(t *testing.T) {
replicas: 3,
evictionTolerance: 0.5,
vpa: getBasicVpa(),
calculators: getNoopPatchCalculators(),
pods: []podWithExpectations{
{
pod: generatePod().Get(),
@ -111,7 +124,6 @@ func TestEvictReplicatedByController(t *testing.T) {
replicas: 4,
evictionTolerance: 0.5,
vpa: getBasicVpa(),
calculators: getNoopPatchCalculators(),
pods: []podWithExpectations{
{
@ -141,7 +153,6 @@ func TestEvictReplicatedByController(t *testing.T) {
replicas: 4,
evictionTolerance: 0.5,
vpa: getBasicVpa(),
calculators: getNoopPatchCalculators(),
pods: []podWithExpectations{
{
pod: generatePod().Get(),
@ -165,7 +176,6 @@ func TestEvictReplicatedByController(t *testing.T) {
replicas: 3,
evictionTolerance: 0.1,
vpa: getBasicVpa(),
calculators: getNoopPatchCalculators(),
pods: []podWithExpectations{
{
pod: generatePod().Get(),
@ -189,7 +199,6 @@ func TestEvictReplicatedByController(t *testing.T) {
replicas: 3,
evictionTolerance: 0.1,
vpa: getBasicVpa(),
calculators: getNoopPatchCalculators(),
pods: []podWithExpectations{
{
pod: generatePod().Get(),
@ -208,7 +217,6 @@ func TestEvictReplicatedByController(t *testing.T) {
replicas: 3,
evictionTolerance: 0.5,
vpa: getBasicVpa(),
calculators: getNoopPatchCalculators(),
pods: []podWithExpectations{
{
pod: generatePod().Get(),
@ -232,7 +240,6 @@ func TestEvictReplicatedByController(t *testing.T) {
replicas: 4,
evictionTolerance: 0.5,
vpa: getBasicVpa(),
calculators: getNoopPatchCalculators(),
pods: []podWithExpectations{
{
pod: generatePod().Get(),
@ -261,7 +268,6 @@ func TestEvictReplicatedByController(t *testing.T) {
replicas: 1,
evictionTolerance: 0.5,
vpa: getBasicVpa(),
calculators: getNoopPatchCalculators(),
pods: []podWithExpectations{
{
pod: generatePod().Get(),
@ -275,7 +281,6 @@ func TestEvictReplicatedByController(t *testing.T) {
replicas: 1,
evictionTolerance: 0.5,
vpa: vpaSingleReplica,
calculators: getNoopPatchCalculators(),
pods: []podWithExpectations{
{
pod: generatePod().Get(),
@ -284,29 +289,217 @@ func TestEvictReplicatedByController(t *testing.T) {
},
},
},
{
name: "In-place update only first pod (half of 3).",
replicas: 3,
evictionTolerance: 0.5,
vpa: getIPORVpa(),
pods: []podWithExpectations{
{
pod: generatePod().Get(),
canInPlaceUpdate: utils.InPlaceApproved,
inPlaceUpdateSuccess: true,
},
{
pod: generatePod().Get(),
canInPlaceUpdate: utils.InPlaceApproved,
inPlaceUpdateSuccess: false,
},
{
pod: generatePod().Get(),
canInPlaceUpdate: utils.InPlaceApproved,
inPlaceUpdateSuccess: false,
},
},
},
{
name: "For small eviction tolerance at least one pod is in-place resized.",
replicas: 3,
evictionTolerance: 0.1,
vpa: getIPORVpa(),
pods: []podWithExpectations{
{
pod: generatePod().Get(),
canInPlaceUpdate: utils.InPlaceApproved,
inPlaceUpdateSuccess: true,
},
{
pod: generatePod().Get(),
canInPlaceUpdate: utils.InPlaceApproved,
inPlaceUpdateSuccess: false,
},
{
pod: generatePod().Get(),
canInPlaceUpdate: utils.InPlaceApproved,
inPlaceUpdateSuccess: false,
},
},
},
{
name: "Ongoing in-placing pods will not get resized again, but may be considered for eviction or deferred.",
replicas: 3,
evictionTolerance: 0.1,
vpa: getIPORVpa(),
pods: []podWithExpectations{
{
pod: generatePod().WithResizeStatus(apiv1.PodResizeStatusInfeasible).Get(),
canInPlaceUpdate: utils.InPlaceEvict,
inPlaceUpdateSuccess: false,
},
{
pod: generatePod().WithResizeStatus(apiv1.PodResizeStatusInProgress).Get(),
canInPlaceUpdate: utils.InPlaceDeferred,
inPlaceUpdateSuccess: false,
},
{
pod: generatePod().Get(),
canInPlaceUpdate: utils.InPlaceApproved,
inPlaceUpdateSuccess: true,
},
},
},
{
name: "Cannot in-place a single Pod under default settings.",
replicas: 1,
evictionTolerance: 0.5,
vpa: getIPORVpa(),
pods: []podWithExpectations{
{
pod: generatePod().Get(),
canInPlaceUpdate: utils.InPlaceDeferred,
inPlaceUpdateSuccess: false,
},
},
},
{
name: "Can in-place even a single Pod using PodUpdatePolicy.MinReplicas.",
replicas: 1,
evictionTolerance: 0.5,
vpa: func() *vpa_types.VerticalPodAutoscaler {
vpa := getIPORVpa()
vpa.Spec.UpdatePolicy.MinReplicas = ptr.To(int32(1))
return vpa
}(),
pods: []podWithExpectations{
{
pod: generatePod().Get(),
canInPlaceUpdate: utils.InPlaceApproved,
inPlaceUpdateSuccess: true,
},
},
},
{
name: "First pod can be evicted without violation of tolerance, even if other evictable pods have ongoing resizes.",
replicas: 3,
evictionTolerance: 0.5,
vpa: getBasicVpa(),
pods: []podWithExpectations{
{
pod: generatePod().Get(),
canEvict: true,
evictionSuccess: true,
},
{
pod: generatePod().WithResizeStatus(apiv1.PodResizeStatusInfeasible).Get(),
canEvict: true,
evictionSuccess: false,
},
{
pod: generatePod().WithResizeStatus(apiv1.PodResizeStatusInfeasible).Get(),
canEvict: true,
evictionSuccess: false,
},
},
},
{
name: "No pods are evictable even if some pods are stuck resizing, but some are missing and eviction tolerance is small.",
replicas: 4,
evictionTolerance: 0.1,
vpa: getBasicVpa(),
pods: []podWithExpectations{
{
pod: generatePod().Get(),
canEvict: false,
evictionSuccess: false,
},
{
pod: generatePod().WithResizeStatus(apiv1.PodResizeStatusInfeasible).Get(),
canEvict: false,
evictionSuccess: false,
},
{
pod: generatePod().Get(),
canEvict: false,
evictionSuccess: false,
},
},
},
{
name: "All pods, including resizing pods, are evictable due to large tolerance.",
replicas: 3,
evictionTolerance: 1,
vpa: getBasicVpa(),
pods: []podWithExpectations{
{
pod: generatePod().Get(),
canEvict: true,
evictionSuccess: true,
},
{
pod: generatePod().WithResizeStatus(apiv1.PodResizeStatusInfeasible).Get(),
canEvict: true,
evictionSuccess: true,
},
{
pod: generatePod().Get(),
canEvict: true,
evictionSuccess: true,
},
},
},
}
for _, testCase := range testCases {
rc.Spec = apiv1.ReplicationControllerSpec{
Replicas: &testCase.replicas,
}
pods := make([]*apiv1.Pod, 0, len(testCase.pods))
for _, p := range testCase.pods {
pods = append(pods, p.pod)
}
factory, _ := getEvictionRestrictionFactory(&rc, nil, nil, nil, 2, testCase.evictionTolerance)
eviction := factory.NewPodsEvictionRestriction(pods, testCase.vpa, testCase.calculators)
for i, p := range testCase.pods {
assert.Equalf(t, p.canEvict, eviction.CanEvict(p.pod), "TC %v - unexpected CanEvict result for pod-%v %#v", testCase.name, i, p.pod)
}
for i, p := range testCase.pods {
err := eviction.Evict(p.pod, testCase.vpa, test.FakeEventRecorder())
if p.evictionSuccess {
assert.NoErrorf(t, err, "TC %v - unexpected Evict result for pod-%v %#v", testCase.name, i, p.pod)
} else {
assert.Errorf(t, err, "TC %v - unexpected Evict result for pod-%v %#v", testCase.name, i, p.pod)
t.Run(testCase.name, func(t *testing.T) {
rc.Spec = apiv1.ReplicationControllerSpec{
Replicas: &testCase.replicas,
}
}
pods := make([]*apiv1.Pod, 0, len(testCase.pods))
for _, p := range testCase.pods {
pods = append(pods, p.pod)
}
factory, err := getRestrictionFactory(&rc, nil, nil, nil, 2, testCase.evictionTolerance, baseclocktest.NewFakeClock(time.Time{}), make(map[string]time.Time), GetFakeCalculatorsWithFakeResourceCalc())
assert.NoError(t, err)
creatorToSingleGroupStatsMap, podToReplicaCreatorMap, err := factory.GetCreatorMaps(pods, testCase.vpa)
assert.NoError(t, err)
eviction := factory.NewPodsEvictionRestriction(creatorToSingleGroupStatsMap, podToReplicaCreatorMap)
inplace := factory.NewPodsInPlaceRestriction(creatorToSingleGroupStatsMap, podToReplicaCreatorMap)
updateMode := vpa_api_util.GetUpdateMode(testCase.vpa)
for i, p := range testCase.pods {
if updateMode == vpa_types.UpdateModeInPlaceOrRecreate {
assert.Equalf(t, p.canInPlaceUpdate, inplace.CanInPlaceUpdate(p.pod), "TC %v - unexpected CanInPlaceUpdate result for pod-%v %#v", testCase.name, i, p.pod)
} else {
assert.Equalf(t, p.canEvict, eviction.CanEvict(p.pod), "TC %v - unexpected CanEvict result for pod-%v %#v", testCase.name, i, p.pod)
}
}
for i, p := range testCase.pods {
if updateMode == vpa_types.UpdateModeInPlaceOrRecreate {
err := inplace.InPlaceUpdate(p.pod, testCase.vpa, test.FakeEventRecorder())
if p.inPlaceUpdateSuccess {
assert.NoErrorf(t, err, "TC %v - unexpected InPlaceUpdate result for pod-%v %#v", testCase.name, i, p.pod)
} else {
assert.Errorf(t, err, "TC %v - unexpected InPlaceUpdate result for pod-%v %#v", testCase.name, i, p.pod)
}
} else {
err := eviction.Evict(p.pod, testCase.vpa, test.FakeEventRecorder())
if p.evictionSuccess {
assert.NoErrorf(t, err, "TC %v - unexpected Evict result for pod-%v %#v", testCase.name, i, p.pod)
} else {
assert.Errorf(t, err, "TC %v - unexpected Evict result for pod-%v %#v", testCase.name, i, p.pod)
}
}
}
})
}
}
@ -333,8 +526,11 @@ func TestEvictReplicatedByReplicaSet(t *testing.T) {
}
basicVpa := getBasicVpa()
factory, _ := getEvictionRestrictionFactory(nil, &rs, nil, nil, 2, 0.5)
eviction := factory.NewPodsEvictionRestriction(pods, basicVpa, getNoopPatchCalculators())
factory, err := getRestrictionFactory(nil, &rs, nil, nil, 2, 0.5, nil, nil, nil)
assert.NoError(t, err)
creatorToSingleGroupStatsMap, podToReplicaCreatorMap, err := factory.GetCreatorMaps(pods, basicVpa)
assert.NoError(t, err)
eviction := factory.NewPodsEvictionRestriction(creatorToSingleGroupStatsMap, podToReplicaCreatorMap)
for _, pod := range pods {
assert.True(t, eviction.CanEvict(pod))
@ -373,8 +569,11 @@ func TestEvictReplicatedByStatefulSet(t *testing.T) {
}
basicVpa := getBasicVpa()
factory, _ := getEvictionRestrictionFactory(nil, nil, &ss, nil, 2, 0.5)
eviction := factory.NewPodsEvictionRestriction(pods, basicVpa, getNoopPatchCalculators())
factory, err := getRestrictionFactory(nil, nil, &ss, nil, 2, 0.5, nil, nil, nil)
assert.NoError(t, err)
creatorToSingleGroupStatsMap, podToReplicaCreatorMap, err := factory.GetCreatorMaps(pods, basicVpa)
assert.NoError(t, err)
eviction := factory.NewPodsEvictionRestriction(creatorToSingleGroupStatsMap, podToReplicaCreatorMap)
for _, pod := range pods {
assert.True(t, eviction.CanEvict(pod))
@ -412,8 +611,11 @@ func TestEvictReplicatedByDaemonSet(t *testing.T) {
}
basicVpa := getBasicVpa()
factory, _ := getEvictionRestrictionFactory(nil, nil, nil, &ds, 2, 0.5)
eviction := factory.NewPodsEvictionRestriction(pods, basicVpa, getNoopPatchCalculators())
factory, err := getRestrictionFactory(nil, nil, nil, &ds, 2, 0.5, nil, nil, nil)
assert.NoError(t, err)
creatorToSingleGroupStatsMap, podToReplicaCreatorMap, err := factory.GetCreatorMaps(pods, basicVpa)
assert.NoError(t, err)
eviction := factory.NewPodsEvictionRestriction(creatorToSingleGroupStatsMap, podToReplicaCreatorMap)
for _, pod := range pods {
assert.True(t, eviction.CanEvict(pod))
@ -448,8 +650,11 @@ func TestEvictReplicatedByJob(t *testing.T) {
}
basicVpa := getBasicVpa()
factory, _ := getEvictionRestrictionFactory(nil, nil, nil, nil, 2, 0.5)
eviction := factory.NewPodsEvictionRestriction(pods, basicVpa, getNoopPatchCalculators())
factory, err := getRestrictionFactory(nil, nil, nil, nil, 2, 0.5, nil, nil, nil)
assert.NoError(t, err)
creatorToSingleGroupStatsMap, podToReplicaCreatorMap, err := factory.GetCreatorMaps(pods, basicVpa)
assert.NoError(t, err)
eviction := factory.NewPodsEvictionRestriction(creatorToSingleGroupStatsMap, podToReplicaCreatorMap)
for _, pod := range pods {
assert.True(t, eviction.CanEvict(pod))
@ -465,226 +670,9 @@ func TestEvictReplicatedByJob(t *testing.T) {
}
}
func TestEvictTooFewReplicas(t *testing.T) {
replicas := int32(5)
livePods := 5
rc := apiv1.ReplicationController{
ObjectMeta: metav1.ObjectMeta{
Name: "rc",
Namespace: "default",
},
TypeMeta: metav1.TypeMeta{
Kind: "ReplicationController",
},
Spec: apiv1.ReplicationControllerSpec{
Replicas: &replicas,
},
}
pods := make([]*apiv1.Pod, livePods)
for i := range pods {
pods[i] = test.Pod().WithName(getTestPodName(i)).WithCreator(&rc.ObjectMeta, &rc.TypeMeta).Get()
}
basicVpa := getBasicVpa()
factory, _ := getEvictionRestrictionFactory(&rc, nil, nil, nil, 10, 0.5)
eviction := factory.NewPodsEvictionRestriction(pods, basicVpa, getNoopPatchCalculators())
for _, pod := range pods {
assert.False(t, eviction.CanEvict(pod))
}
for _, pod := range pods {
err := eviction.Evict(pod, basicVpa, test.FakeEventRecorder())
assert.Error(t, err, "Error expected")
}
}
func TestEvictionTolerance(t *testing.T) {
replicas := int32(5)
livePods := 5
tolerance := 0.8
rc := apiv1.ReplicationController{
ObjectMeta: metav1.ObjectMeta{
Name: "rc",
Namespace: "default",
},
TypeMeta: metav1.TypeMeta{
Kind: "ReplicationController",
},
Spec: apiv1.ReplicationControllerSpec{
Replicas: &replicas,
},
}
pods := make([]*apiv1.Pod, livePods)
for i := range pods {
pods[i] = test.Pod().WithName(getTestPodName(i)).WithCreator(&rc.ObjectMeta, &rc.TypeMeta).Get()
}
basicVpa := getBasicVpa()
factory, _ := getEvictionRestrictionFactory(&rc, nil, nil, nil, 2 /*minReplicas*/, tolerance)
eviction := factory.NewPodsEvictionRestriction(pods, basicVpa, getNoopPatchCalculators())
for _, pod := range pods {
assert.True(t, eviction.CanEvict(pod))
}
for _, pod := range pods[:4] {
err := eviction.Evict(pod, basicVpa, test.FakeEventRecorder())
assert.Nil(t, err, "Should evict with no error")
}
for _, pod := range pods[4:] {
err := eviction.Evict(pod, basicVpa, test.FakeEventRecorder())
assert.Error(t, err, "Error expected")
}
}
func TestEvictAtLeastOne(t *testing.T) {
replicas := int32(5)
livePods := 5
tolerance := 0.1
rc := apiv1.ReplicationController{
ObjectMeta: metav1.ObjectMeta{
Name: "rc",
Namespace: "default",
},
TypeMeta: metav1.TypeMeta{
Kind: "ReplicationController",
},
Spec: apiv1.ReplicationControllerSpec{
Replicas: &replicas,
},
}
pods := make([]*apiv1.Pod, livePods)
for i := range pods {
pods[i] = test.Pod().WithName(getTestPodName(i)).WithCreator(&rc.ObjectMeta, &rc.TypeMeta).Get()
}
basicVpa := getBasicVpa()
factory, _ := getEvictionRestrictionFactory(&rc, nil, nil, nil, 2, tolerance)
eviction := factory.NewPodsEvictionRestriction(pods, basicVpa, getNoopPatchCalculators())
for _, pod := range pods {
assert.True(t, eviction.CanEvict(pod))
}
for _, pod := range pods[:1] {
err := eviction.Evict(pod, basicVpa, test.FakeEventRecorder())
assert.Nil(t, err, "Should evict with no error")
}
for _, pod := range pods[1:] {
err := eviction.Evict(pod, basicVpa, test.FakeEventRecorder())
assert.Error(t, err, "Error expected")
}
}
func TestEvictEmitEvent(t *testing.T) {
rc := apiv1.ReplicationController{
ObjectMeta: metav1.ObjectMeta{
Name: "rc",
Namespace: "default",
},
TypeMeta: metav1.TypeMeta{
Kind: "ReplicationController",
},
}
index := 0
generatePod := func() test.PodBuilder {
index++
return test.Pod().WithName(fmt.Sprintf("test-%v", index)).WithCreator(&rc.ObjectMeta, &rc.TypeMeta)
}
basicVpa := getBasicVpa()
testCases := []struct {
name string
replicas int32
evictionTolerance float64
vpa *vpa_types.VerticalPodAutoscaler
calculators []patch.Calculator
pods []podWithExpectations
errorExpected bool
}{
{
name: "Pods that can be evicted",
replicas: 4,
evictionTolerance: 0.5,
vpa: basicVpa,
calculators: getNoopPatchCalculators(),
pods: []podWithExpectations{
{
pod: generatePod().WithPhase(apiv1.PodPending).Get(),
canEvict: true,
evictionSuccess: true,
},
{
pod: generatePod().WithPhase(apiv1.PodPending).Get(),
canEvict: true,
evictionSuccess: true,
},
},
errorExpected: false,
},
{
name: "Pod that can not be evicted",
replicas: 4,
evictionTolerance: 0.5,
vpa: basicVpa,
calculators: getNoopPatchCalculators(),
pods: []podWithExpectations{
{
pod: generatePod().Get(),
canEvict: false,
evictionSuccess: false,
},
},
errorExpected: true,
},
}
for _, testCase := range testCases {
rc.Spec = apiv1.ReplicationControllerSpec{
Replicas: &testCase.replicas,
}
pods := make([]*apiv1.Pod, 0, len(testCase.pods))
for _, p := range testCase.pods {
pods = append(pods, p.pod)
}
factory, _ := getEvictionRestrictionFactory(&rc, nil, nil, nil, 2, testCase.evictionTolerance)
eviction := factory.NewPodsEvictionRestriction(pods, testCase.vpa, testCase.calculators)
for _, p := range testCase.pods {
mockRecorder := test.MockEventRecorder()
mockRecorder.On("Event", mock.Anything, apiv1.EventTypeNormal, "EvictedByVPA", mock.Anything).Return()
mockRecorder.On("Event", mock.Anything, apiv1.EventTypeNormal, "EvictedPod", mock.Anything).Return()
errGot := eviction.Evict(p.pod, testCase.vpa, mockRecorder)
if testCase.errorExpected {
assert.Error(t, errGot)
} else {
assert.NoError(t, errGot)
}
if p.canEvict {
mockRecorder.AssertNumberOfCalls(t, "Event", 2)
} else {
mockRecorder.AssertNumberOfCalls(t, "Event", 0)
}
}
}
}
func getEvictionRestrictionFactory(rc *apiv1.ReplicationController, rs *appsv1.ReplicaSet,
func getRestrictionFactory(rc *apiv1.ReplicationController, rs *appsv1.ReplicaSet,
ss *appsv1.StatefulSet, ds *appsv1.DaemonSet, minReplicas int,
evictionToleranceFraction float64) (PodsEvictionRestrictionFactory, error) {
evictionToleranceFraction float64, clock clock.Clock, lipuatm map[string]time.Time, patchCalculators []patch.Calculator) (PodsRestrictionFactory, error) {
kubeClient := &fake.Clientset{}
rcInformer := coreinformer.NewReplicationControllerInformer(kubeClient, apiv1.NamespaceAll,
0*time.Second, cache.Indexers{cache.NamespaceIndex: cache.MetaNamespaceIndexFunc})
@ -719,17 +707,52 @@ func getEvictionRestrictionFactory(rc *apiv1.ReplicationController, rs *appsv1.R
}
}
return &podsEvictionRestrictionFactoryImpl{
return &PodsRestrictionFactoryImpl{
client: kubeClient,
rsInformer: rsInformer,
rcInformer: rcInformer,
ssInformer: ssInformer,
rsInformer: rsInformer,
dsInformer: dsInformer,
minReplicas: minReplicas,
evictionToleranceFraction: evictionToleranceFraction,
clock: clock,
lastInPlaceAttemptTimeMap: lipuatm,
patchCalculators: patchCalculators,
}, nil
}
func getTestPodName(index int) string {
return fmt.Sprintf("test-%v", index)
}
type fakeResizePatchCalculator struct {
patches []resource_admission.PatchRecord
err error
}
func (c *fakeResizePatchCalculator) CalculatePatches(_ *apiv1.Pod, _ *vpa_types.VerticalPodAutoscaler) (
[]resource_admission.PatchRecord, error) {
return c.patches, c.err
}
func (c *fakeResizePatchCalculator) PatchResourceTarget() patch.PatchResourceTarget {
return patch.Resize
}
func NewFakeCalculatorWithInPlacePatches() patch.Calculator {
return &fakeResizePatchCalculator{
patches: []resource_admission.PatchRecord{
{
Op: "fakeop",
Path: "fakepath",
Value: apiv1.ResourceList{},
},
},
}
}
func GetFakeCalculatorsWithFakeResourceCalc() []patch.Calculator {
return []patch.Calculator{
NewFakeCalculatorWithInPlacePatches(),
}
}

View File

@ -0,0 +1,29 @@
/*
Copyright 2025 The Kubernetes 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 utils
// InPlaceDecision is the type of decision that can be made for a pod.
type InPlaceDecision string
const (
// InPlaceApproved means we can in-place update the pod.
InPlaceApproved InPlaceDecision = "InPlaceApproved"
// InPlaceDeferred means we can't in-place update the pod right now, but we will wait for the next loop to check for in-placeability again
InPlaceDeferred InPlaceDecision = "InPlaceDeferred"
// InPlaceEvict means we will attempt to evict the pod.
InPlaceEvict InPlaceDecision = "InPlaceEvict"
)

View File

@ -23,5 +23,5 @@ const (
// GetVpaInPlaceUpdatedValue creates an annotation value for a given pod.
func GetVpaInPlaceUpdatedValue() string {
return "vpaInPlaceUpdated"
return "true"
}

View File

@ -33,6 +33,7 @@ import (
vpa_types_v1beta1 "k8s.io/autoscaler/vertical-pod-autoscaler/pkg/apis/autoscaling.k8s.io/v1beta1"
vpa_lister "k8s.io/autoscaler/vertical-pod-autoscaler/pkg/client/listers/autoscaling.k8s.io/v1"
vpa_lister_v1beta1 "k8s.io/autoscaler/vertical-pod-autoscaler/pkg/client/listers/autoscaling.k8s.io/v1beta1"
utils "k8s.io/autoscaler/vertical-pod-autoscaler/pkg/updater/utils"
)
var (
@ -121,22 +122,21 @@ func (m *PodsEvictionRestrictionMock) CanEvict(pod *apiv1.Pod) bool {
return args.Bool(0)
}
// InPlaceUpdate is a mock implementation of PodsEvictionRestriction.InPlaceUpdate
func (m *PodsEvictionRestrictionMock) InPlaceUpdate(pod *apiv1.Pod, vpa *vpa_types.VerticalPodAutoscaler, eventRecorder record.EventRecorder) error {
// PodsInPlaceRestrictionMock is a mock of PodsInPlaceRestriction
type PodsInPlaceRestrictionMock struct {
mock.Mock
}
// InPlaceUpdate is a mock implementation of PodsInPlaceRestriction.InPlaceUpdate
func (m *PodsInPlaceRestrictionMock) InPlaceUpdate(pod *apiv1.Pod, vpa *vpa_types.VerticalPodAutoscaler, eventRecorder record.EventRecorder) error {
args := m.Called(pod, eventRecorder)
return args.Error(0)
}
// CanInPlaceUpdate is a mock implementation of PodsEvictionRestriction.CanInPlaceUpdate
func (m *PodsEvictionRestrictionMock) CanInPlaceUpdate(pod *apiv1.Pod) bool {
// CanInPlaceUpdate is a mock implementation of PodsInPlaceRestriction.CanInPlaceUpdate
func (m *PodsInPlaceRestrictionMock) CanInPlaceUpdate(pod *apiv1.Pod) utils.InPlaceDecision {
args := m.Called(pod)
return args.Bool(0)
}
// IsInPlaceUpdating is a mock implementation of PodsEvictionRestriction.IsInPlaceUpdating
func (m *PodsEvictionRestrictionMock) IsInPlaceUpdating(pod *apiv1.Pod) bool {
args := m.Called(pod)
return args.Bool(0)
return args.Get(0).(utils.InPlaceDecision)
}
// PodListerMock is a mock of PodLister