Provide ScaleDownStatusProcessor with info about unremovable nodes
This commit is contained in:
		
							parent
							
								
									66f6328433
								
							
						
					
					
						commit
						7a188ab50d
					
				| 
						 | 
				
			
			@ -354,35 +354,38 @@ func (limits *scaleDownResourcesLimits) tryDecrementLimitsByDelta(delta scaleDow
 | 
			
		|||
 | 
			
		||||
// ScaleDown is responsible for maintaining the state needed to perform unneeded node removals.
 | 
			
		||||
type ScaleDown struct {
 | 
			
		||||
	context              *context.AutoscalingContext
 | 
			
		||||
	clusterStateRegistry *clusterstate.ClusterStateRegistry
 | 
			
		||||
	unneededNodes        map[string]time.Time
 | 
			
		||||
	unneededNodesList    []*apiv1.Node
 | 
			
		||||
	unremovableNodes     map[string]time.Time
 | 
			
		||||
	podLocationHints     map[string]string
 | 
			
		||||
	nodeUtilizationMap   map[string]simulator.UtilizationInfo
 | 
			
		||||
	usageTracker         *simulator.UsageTracker
 | 
			
		||||
	nodeDeletionTracker  *NodeDeletionTracker
 | 
			
		||||
	context                *context.AutoscalingContext
 | 
			
		||||
	clusterStateRegistry   *clusterstate.ClusterStateRegistry
 | 
			
		||||
	unneededNodes          map[string]time.Time
 | 
			
		||||
	unneededNodesList      []*apiv1.Node
 | 
			
		||||
	unremovableNodes       map[string]time.Time
 | 
			
		||||
	podLocationHints       map[string]string
 | 
			
		||||
	nodeUtilizationMap     map[string]simulator.UtilizationInfo
 | 
			
		||||
	usageTracker           *simulator.UsageTracker
 | 
			
		||||
	nodeDeletionTracker    *NodeDeletionTracker
 | 
			
		||||
	unremovableNodeReasons map[string]*simulator.UnremovableNode
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// NewScaleDown builds new ScaleDown object.
 | 
			
		||||
func NewScaleDown(context *context.AutoscalingContext, clusterStateRegistry *clusterstate.ClusterStateRegistry) *ScaleDown {
 | 
			
		||||
	return &ScaleDown{
 | 
			
		||||
		context:              context,
 | 
			
		||||
		clusterStateRegistry: clusterStateRegistry,
 | 
			
		||||
		unneededNodes:        make(map[string]time.Time),
 | 
			
		||||
		unremovableNodes:     make(map[string]time.Time),
 | 
			
		||||
		podLocationHints:     make(map[string]string),
 | 
			
		||||
		nodeUtilizationMap:   make(map[string]simulator.UtilizationInfo),
 | 
			
		||||
		usageTracker:         simulator.NewUsageTracker(),
 | 
			
		||||
		unneededNodesList:    make([]*apiv1.Node, 0),
 | 
			
		||||
		nodeDeletionTracker:  NewNodeDeletionTracker(),
 | 
			
		||||
		context:                context,
 | 
			
		||||
		clusterStateRegistry:   clusterStateRegistry,
 | 
			
		||||
		unneededNodes:          make(map[string]time.Time),
 | 
			
		||||
		unremovableNodes:       make(map[string]time.Time),
 | 
			
		||||
		podLocationHints:       make(map[string]string),
 | 
			
		||||
		nodeUtilizationMap:     make(map[string]simulator.UtilizationInfo),
 | 
			
		||||
		usageTracker:           simulator.NewUsageTracker(),
 | 
			
		||||
		unneededNodesList:      make([]*apiv1.Node, 0),
 | 
			
		||||
		nodeDeletionTracker:    NewNodeDeletionTracker(),
 | 
			
		||||
		unremovableNodeReasons: make(map[string]*simulator.UnremovableNode),
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// CleanUp cleans up the internal ScaleDown state.
 | 
			
		||||
func (sd *ScaleDown) CleanUp(timestamp time.Time) {
 | 
			
		||||
	sd.usageTracker.CleanUp(timestamp.Add(-sd.context.ScaleDownUnneededTime))
 | 
			
		||||
	sd.clearUnremovableNodeReasons()
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// CleanUpUnneededNodes clears the list of unneeded nodes.
 | 
			
		||||
| 
						 | 
				
			
			@ -453,6 +456,7 @@ func (sd *ScaleDown) UpdateUnneededNodes(
 | 
			
		|||
	for _, node := range scaleDownCandidates {
 | 
			
		||||
		if unremovableTimestamp, found := sd.unremovableNodes[node.Name]; found {
 | 
			
		||||
			if unremovableTimestamp.After(timestamp) {
 | 
			
		||||
				sd.addUnremovableNodeReason(node, simulator.RecentlyUnremovable)
 | 
			
		||||
				continue
 | 
			
		||||
			}
 | 
			
		||||
			delete(sd.unremovableNodes, node.Name)
 | 
			
		||||
| 
						 | 
				
			
			@ -473,18 +477,21 @@ func (sd *ScaleDown) UpdateUnneededNodes(
 | 
			
		|||
		// and they have not been deleted.
 | 
			
		||||
		if isNodeBeingDeleted(node, timestamp) {
 | 
			
		||||
			klog.V(1).Infof("Skipping %s from delete consideration - the node is currently being deleted", node.Name)
 | 
			
		||||
			sd.addUnremovableNodeReason(node, simulator.CurrentlyBeingDeleted)
 | 
			
		||||
			continue
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		// Skip nodes marked with no scale down annotation
 | 
			
		||||
		if hasNoScaleDownAnnotation(node) {
 | 
			
		||||
			klog.V(1).Infof("Skipping %s from delete consideration - the node is marked as no scale down", node.Name)
 | 
			
		||||
			sd.addUnremovableNodeReason(node, simulator.ScaleDownDisabledAnnotation)
 | 
			
		||||
			continue
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		nodeInfo, found := nodeNameToNodeInfo[node.Name]
 | 
			
		||||
		if !found {
 | 
			
		||||
			klog.Errorf("Node info for %s not found", node.Name)
 | 
			
		||||
			sd.addUnremovableNodeReason(node, simulator.UnexpectedError)
 | 
			
		||||
			continue
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			@ -497,6 +504,7 @@ func (sd *ScaleDown) UpdateUnneededNodes(
 | 
			
		|||
 | 
			
		||||
		if !sd.isNodeBelowUtilzationThreshold(node, utilInfo) {
 | 
			
		||||
			klog.V(4).Infof("Node %s is not suitable for removal - %s utilization too big (%f)", node.Name, utilInfo.ResourceName, utilInfo.Utilization)
 | 
			
		||||
			sd.addUnremovableNodeReason(node, simulator.NotUnderutilized)
 | 
			
		||||
			continue
 | 
			
		||||
		}
 | 
			
		||||
		currentlyUnneededNodes = append(currentlyUnneededNodes, node)
 | 
			
		||||
| 
						 | 
				
			
			@ -595,12 +603,22 @@ func (sd *ScaleDown) UpdateUnneededNodes(
 | 
			
		|||
	// Add nodes to unremovable map
 | 
			
		||||
	if len(unremovable) > 0 {
 | 
			
		||||
		unremovableTimeout := timestamp.Add(sd.context.AutoscalingOptions.UnremovableNodeRecheckTimeout)
 | 
			
		||||
		for _, node := range unremovable {
 | 
			
		||||
			sd.unremovableNodes[node.Name] = unremovableTimeout
 | 
			
		||||
		for _, unremovableNode := range unremovable {
 | 
			
		||||
			sd.unremovableNodes[unremovableNode.Node.Name] = unremovableTimeout
 | 
			
		||||
			sd.addUnremovableNode(unremovableNode)
 | 
			
		||||
		}
 | 
			
		||||
		klog.V(1).Infof("%v nodes found to be unremovable in simulation, will re-check them at %v", len(unremovable), unremovableTimeout)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// This method won't always check all nodes, so let's give a generic reason for all nodes that weren't checked.
 | 
			
		||||
	for _, node := range scaleDownCandidates {
 | 
			
		||||
		_, unremovableReasonProvided := sd.unremovableNodeReasons[node.Name]
 | 
			
		||||
		_, unneeded := result[node.Name]
 | 
			
		||||
		if !unneeded && !unremovableReasonProvided {
 | 
			
		||||
			sd.addUnremovableNodeReason(node, simulator.NotUnneededOtherReason)
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	// Update state and metrics
 | 
			
		||||
	sd.unneededNodesList = unneededNodesList
 | 
			
		||||
	sd.unneededNodes = result
 | 
			
		||||
| 
						 | 
				
			
			@ -648,6 +666,18 @@ func (sd *ScaleDown) updateUnremovableNodes(nodes []*apiv1.Node) {
 | 
			
		|||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (sd *ScaleDown) clearUnremovableNodeReasons() {
 | 
			
		||||
	sd.unremovableNodeReasons = make(map[string]*simulator.UnremovableNode)
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (sd *ScaleDown) addUnremovableNodeReason(node *apiv1.Node, reason simulator.UnremovableReason) {
 | 
			
		||||
	sd.unremovableNodeReasons[node.Name] = &simulator.UnremovableNode{Node: node, Reason: reason, BlockingPod: nil}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func (sd *ScaleDown) addUnremovableNode(unremovableNode *simulator.UnremovableNode) {
 | 
			
		||||
	sd.unremovableNodeReasons[unremovableNode.Node.Name] = unremovableNode
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// markSimulationError indicates a simulation error by clearing  relevant scale
 | 
			
		||||
// down state and returning an appropriate error.
 | 
			
		||||
func (sd *ScaleDown) markSimulationError(simulatorErr errors.AutoscalerError,
 | 
			
		||||
| 
						 | 
				
			
			@ -771,6 +801,7 @@ func (sd *ScaleDown) TryToScaleDown(allNodes []*apiv1.Node, pods []*apiv1.Pod, p
 | 
			
		|||
			// Check if node is marked with no scale down annotation.
 | 
			
		||||
			if hasNoScaleDownAnnotation(node) {
 | 
			
		||||
				klog.V(4).Infof("Skipping %s - scale down disabled annotation found", node.Name)
 | 
			
		||||
				sd.addUnremovableNodeReason(node, simulator.ScaleDownDisabledAnnotation)
 | 
			
		||||
				continue
 | 
			
		||||
			}
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			@ -779,45 +810,53 @@ func (sd *ScaleDown) TryToScaleDown(allNodes []*apiv1.Node, pods []*apiv1.Pod, p
 | 
			
		|||
 | 
			
		||||
			// Check how long the node was underutilized.
 | 
			
		||||
			if ready && !val.Add(sd.context.ScaleDownUnneededTime).Before(currentTime) {
 | 
			
		||||
				sd.addUnremovableNodeReason(node, simulator.NotUnneededLongEnough)
 | 
			
		||||
				continue
 | 
			
		||||
			}
 | 
			
		||||
 | 
			
		||||
			// Unready nodes may be deleted after a different time than underutilized nodes.
 | 
			
		||||
			if !ready && !val.Add(sd.context.ScaleDownUnreadyTime).Before(currentTime) {
 | 
			
		||||
				sd.addUnremovableNodeReason(node, simulator.NotUnreadyLongEnough)
 | 
			
		||||
				continue
 | 
			
		||||
			}
 | 
			
		||||
 | 
			
		||||
			nodeGroup, err := sd.context.CloudProvider.NodeGroupForNode(node)
 | 
			
		||||
			if err != nil {
 | 
			
		||||
				klog.Errorf("Error while checking node group for %s: %v", node.Name, err)
 | 
			
		||||
				sd.addUnremovableNodeReason(node, simulator.UnexpectedError)
 | 
			
		||||
				continue
 | 
			
		||||
			}
 | 
			
		||||
			if nodeGroup == nil || reflect.ValueOf(nodeGroup).IsNil() {
 | 
			
		||||
				klog.V(4).Infof("Skipping %s - no node group config", node.Name)
 | 
			
		||||
				sd.addUnremovableNodeReason(node, simulator.NotAutoscaled)
 | 
			
		||||
				continue
 | 
			
		||||
			}
 | 
			
		||||
 | 
			
		||||
			size, found := nodeGroupSize[nodeGroup.Id()]
 | 
			
		||||
			if !found {
 | 
			
		||||
				klog.Errorf("Error while checking node group size %s: group size not found in cache", nodeGroup.Id())
 | 
			
		||||
				sd.addUnremovableNodeReason(node, simulator.UnexpectedError)
 | 
			
		||||
				continue
 | 
			
		||||
			}
 | 
			
		||||
 | 
			
		||||
			deletionsInProgress := sd.nodeDeletionTracker.GetDeletionsInProgress(nodeGroup.Id())
 | 
			
		||||
			if size-deletionsInProgress <= nodeGroup.MinSize() {
 | 
			
		||||
				klog.V(1).Infof("Skipping %s - node group min size reached", node.Name)
 | 
			
		||||
				sd.addUnremovableNodeReason(node, simulator.NodeGroupMinSizeReached)
 | 
			
		||||
				continue
 | 
			
		||||
			}
 | 
			
		||||
 | 
			
		||||
			scaleDownResourcesDelta, err := computeScaleDownResourcesDelta(sd.context.CloudProvider, node, nodeGroup, resourcesWithLimits)
 | 
			
		||||
			if err != nil {
 | 
			
		||||
				klog.Errorf("Error getting node resources: %v", err)
 | 
			
		||||
				sd.addUnremovableNodeReason(node, simulator.UnexpectedError)
 | 
			
		||||
				continue
 | 
			
		||||
			}
 | 
			
		||||
 | 
			
		||||
			checkResult := scaleDownResourcesLeft.checkScaleDownDeltaWithinLimits(scaleDownResourcesDelta)
 | 
			
		||||
			if checkResult.exceeded {
 | 
			
		||||
				klog.V(4).Infof("Skipping %s - minimal limit exceeded for %v", node.Name, checkResult.exceededResources)
 | 
			
		||||
				sd.addUnremovableNodeReason(node, simulator.MinimalResourceLimitExceeded)
 | 
			
		||||
				continue
 | 
			
		||||
			}
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			@ -857,7 +896,7 @@ func (sd *ScaleDown) TryToScaleDown(allNodes []*apiv1.Node, pods []*apiv1.Pod, p
 | 
			
		|||
	// Only scheduled non expendable pods are taken into account and have to be moved.
 | 
			
		||||
	nonExpendablePods := core_utils.FilterOutExpendablePods(pods, sd.context.ExpendablePodsPriorityCutoff)
 | 
			
		||||
	// We look for only 1 node so new hints may be incomplete.
 | 
			
		||||
	nodesToRemove, _, _, err := simulator.FindNodesToRemove(
 | 
			
		||||
	nodesToRemove, unremovable, _, err := simulator.FindNodesToRemove(
 | 
			
		||||
		candidates,
 | 
			
		||||
		nodesWithoutMaster,
 | 
			
		||||
		nonExpendablePods,
 | 
			
		||||
| 
						 | 
				
			
			@ -872,6 +911,10 @@ func (sd *ScaleDown) TryToScaleDown(allNodes []*apiv1.Node, pods []*apiv1.Pod, p
 | 
			
		|||
		pdbs)
 | 
			
		||||
	findNodesToRemoveDuration = time.Now().Sub(findNodesToRemoveStart)
 | 
			
		||||
 | 
			
		||||
	for _, unremovableNode := range unremovable {
 | 
			
		||||
		sd.addUnremovableNode(unremovableNode)
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		scaleDownStatus.Result = status.ScaleDownError
 | 
			
		||||
		return scaleDownStatus, err.AddPrefix("Find node to remove failed: ")
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -301,6 +301,7 @@ func (a *StaticAutoscaler) RunOnce(currentTime time.Time) errors.AutoscalerError
 | 
			
		|||
			a.processors.ScaleUpStatusProcessor.Process(a.AutoscalingContext, scaleUpStatus)
 | 
			
		||||
		}
 | 
			
		||||
		if !scaleDownStatusProcessorAlreadyCalled && a.processors != nil && a.processors.ScaleDownStatusProcessor != nil {
 | 
			
		||||
			scaleDownStatus.SetUnremovableNodesInfo(scaleDown.unremovableNodeReasons, scaleDown.nodeUtilizationMap, scaleDown.context.CloudProvider)
 | 
			
		||||
			a.processors.ScaleDownStatusProcessor.Process(a.AutoscalingContext, scaleDownStatus)
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			@ -534,6 +535,7 @@ func (a *StaticAutoscaler) RunOnce(currentTime time.Time) errors.AutoscalerError
 | 
			
		|||
			}
 | 
			
		||||
 | 
			
		||||
			if a.processors != nil && a.processors.ScaleDownStatusProcessor != nil {
 | 
			
		||||
				scaleDownStatus.SetUnremovableNodesInfo(scaleDown.unremovableNodeReasons, scaleDown.nodeUtilizationMap, scaleDown.context.CloudProvider)
 | 
			
		||||
				a.processors.ScaleDownStatusProcessor.Process(autoscalingContext, scaleDownStatus)
 | 
			
		||||
				scaleDownStatusProcessorAlreadyCalled = true
 | 
			
		||||
			}
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -21,16 +21,55 @@ import (
 | 
			
		|||
	"k8s.io/autoscaler/cluster-autoscaler/cloudprovider"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/context"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/simulator"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/utils/drain"
 | 
			
		||||
	"k8s.io/klog"
 | 
			
		||||
)
 | 
			
		||||
 | 
			
		||||
// ScaleDownStatus represents the state of scale down.
 | 
			
		||||
type ScaleDownStatus struct {
 | 
			
		||||
	Result            ScaleDownResult
 | 
			
		||||
	ScaledDownNodes   []*ScaleDownNode
 | 
			
		||||
	UnremovableNodes  []*UnremovableNode
 | 
			
		||||
	RemovedNodeGroups []cloudprovider.NodeGroup
 | 
			
		||||
	NodeDeleteResults map[string]NodeDeleteResult
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// SetUnremovableNodesInfo sets the status of nodes that were found to be unremovable.
 | 
			
		||||
func (s *ScaleDownStatus) SetUnremovableNodesInfo(unremovableNodesMap map[string]*simulator.UnremovableNode, nodeUtilizationMap map[string]simulator.UtilizationInfo, cp cloudprovider.CloudProvider) {
 | 
			
		||||
	s.UnremovableNodes = make([]*UnremovableNode, 0, len(unremovableNodesMap))
 | 
			
		||||
 | 
			
		||||
	for _, unremovableNode := range unremovableNodesMap {
 | 
			
		||||
		nodeGroup, err := cp.NodeGroupForNode(unremovableNode.Node)
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			klog.Errorf("Couldn't find node group for unremovable node in cloud provider %s", unremovableNode.Node.Name)
 | 
			
		||||
			continue
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		var utilInfoPtr *simulator.UtilizationInfo
 | 
			
		||||
		if utilInfo, found := nodeUtilizationMap[unremovableNode.Node.Name]; found {
 | 
			
		||||
			utilInfoPtr = &utilInfo
 | 
			
		||||
			// It's okay if we don't find the util info, it's not computed for some unremovable nodes that are skipped early in the loop.
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		s.UnremovableNodes = append(s.UnremovableNodes, &UnremovableNode{
 | 
			
		||||
			Node:        unremovableNode.Node,
 | 
			
		||||
			NodeGroup:   nodeGroup,
 | 
			
		||||
			UtilInfo:    utilInfoPtr,
 | 
			
		||||
			Reason:      unremovableNode.Reason,
 | 
			
		||||
			BlockingPod: unremovableNode.BlockingPod,
 | 
			
		||||
		})
 | 
			
		||||
	}
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// UnremovableNode represents the state of a node that couldn't be removed.
 | 
			
		||||
type UnremovableNode struct {
 | 
			
		||||
	Node        *apiv1.Node
 | 
			
		||||
	NodeGroup   cloudprovider.NodeGroup
 | 
			
		||||
	UtilInfo    *simulator.UtilizationInfo
 | 
			
		||||
	Reason      simulator.UnremovableReason
 | 
			
		||||
	BlockingPod *drain.BlockingPod
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// ScaleDownNode represents the state of a node that's being scaled down.
 | 
			
		||||
type ScaleDownNode struct {
 | 
			
		||||
	Node        *apiv1.Node
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -22,6 +22,7 @@ import (
 | 
			
		|||
	"math/rand"
 | 
			
		||||
	"time"
 | 
			
		||||
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/utils/drain"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/utils/errors"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/utils/glogx"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/utils/gpu"
 | 
			
		||||
| 
						 | 
				
			
			@ -57,6 +58,48 @@ type NodeToBeRemoved struct {
 | 
			
		|||
	PodsToReschedule []*apiv1.Pod
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// UnremovableNode represents a node that can't be removed by CA.
 | 
			
		||||
type UnremovableNode struct {
 | 
			
		||||
	Node        *apiv1.Node
 | 
			
		||||
	Reason      UnremovableReason
 | 
			
		||||
	BlockingPod *drain.BlockingPod
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// UnremovableReason represents a reason why a node can't be removed by CA.
 | 
			
		||||
type UnremovableReason int
 | 
			
		||||
 | 
			
		||||
const (
 | 
			
		||||
	// NoReason - sanity check, this should never be set explicitly. If this is found in the wild, it means that it was
 | 
			
		||||
	// implicitly initialized and might indicate a bug.
 | 
			
		||||
	NoReason UnremovableReason = iota
 | 
			
		||||
	// ScaleDownDisabledAnnotation - node can't be removed because it has a "scale down disabled" annotation.
 | 
			
		||||
	ScaleDownDisabledAnnotation
 | 
			
		||||
	// NotAutoscaled - node can't be removed because it doesn't belong to an autoscaled node group.
 | 
			
		||||
	NotAutoscaled
 | 
			
		||||
	// NotUnneededLongEnough - node can't be removed because it wasn't unneeded for long enough.
 | 
			
		||||
	NotUnneededLongEnough
 | 
			
		||||
	// NotUnreadyLongEnough - node can't be removed because it wasn't unready for long enough.
 | 
			
		||||
	NotUnreadyLongEnough
 | 
			
		||||
	// NodeGroupMinSizeReached - node can't be removed because its node group is at its minimal size already.
 | 
			
		||||
	NodeGroupMinSizeReached
 | 
			
		||||
	// MinimalResourceLimitExceeded - node can't be removed because it would violate cluster-wide minimal resource limits.
 | 
			
		||||
	MinimalResourceLimitExceeded
 | 
			
		||||
	// CurrentlyBeingDeleted - node can't be removed because it's already in the process of being deleted.
 | 
			
		||||
	CurrentlyBeingDeleted
 | 
			
		||||
	// NotUnderutilized - node can't be removed because it's not underutilized.
 | 
			
		||||
	NotUnderutilized
 | 
			
		||||
	// NotUnneededOtherReason - node can't be removed because it's not marked as unneeded for other reasons (e.g. it wasn't inspected at all in a given autoscaler loop).
 | 
			
		||||
	NotUnneededOtherReason
 | 
			
		||||
	// RecentlyUnremovable - node can't be removed because it was recently found to be unremovable.
 | 
			
		||||
	RecentlyUnremovable
 | 
			
		||||
	// NoPlaceToMovePods - node can't be removed because there's no place to move its pods to.
 | 
			
		||||
	NoPlaceToMovePods
 | 
			
		||||
	// BlockedByPod - node can't be removed because a pod running on it can't be moved. The reason why should be in BlockingPod.
 | 
			
		||||
	BlockedByPod
 | 
			
		||||
	// UnexpectedError - node can't be removed because of an unexpected error.
 | 
			
		||||
	UnexpectedError
 | 
			
		||||
)
 | 
			
		||||
 | 
			
		||||
// UtilizationInfo contains utilization information for a node.
 | 
			
		||||
type UtilizationInfo struct {
 | 
			
		||||
	CpuUtil float64
 | 
			
		||||
| 
						 | 
				
			
			@ -75,11 +118,11 @@ func FindNodesToRemove(candidates []*apiv1.Node, destinationNodes []*apiv1.Node,
 | 
			
		|||
	fastCheck bool, oldHints map[string]string, usageTracker *UsageTracker,
 | 
			
		||||
	timestamp time.Time,
 | 
			
		||||
	podDisruptionBudgets []*policyv1.PodDisruptionBudget,
 | 
			
		||||
) (nodesToRemove []NodeToBeRemoved, unremovableNodes []*apiv1.Node, podReschedulingHints map[string]string, finalError errors.AutoscalerError) {
 | 
			
		||||
) (nodesToRemove []NodeToBeRemoved, unremovableNodes []*UnremovableNode, podReschedulingHints map[string]string, finalError errors.AutoscalerError) {
 | 
			
		||||
 | 
			
		||||
	nodeNameToNodeInfo := scheduler_util.CreateNodeNameToInfoMap(pods, destinationNodes)
 | 
			
		||||
	result := make([]NodeToBeRemoved, 0)
 | 
			
		||||
	unremovable := make([]*apiv1.Node, 0)
 | 
			
		||||
	unremovable := make([]*UnremovableNode, 0)
 | 
			
		||||
 | 
			
		||||
	evaluationType := "Detailed evaluation"
 | 
			
		||||
	if fastCheck {
 | 
			
		||||
| 
						 | 
				
			
			@ -92,24 +135,29 @@ candidateloop:
 | 
			
		|||
		klog.V(2).Infof("%s: %s for removal", evaluationType, node.Name)
 | 
			
		||||
 | 
			
		||||
		var podsToRemove []*apiv1.Pod
 | 
			
		||||
		var blockingPod *drain.BlockingPod
 | 
			
		||||
		var err error
 | 
			
		||||
 | 
			
		||||
		if nodeInfo, found := nodeNameToNodeInfo[node.Name]; found {
 | 
			
		||||
			if fastCheck {
 | 
			
		||||
				podsToRemove, err = FastGetPodsToMove(nodeInfo, *skipNodesWithSystemPods, *skipNodesWithLocalStorage,
 | 
			
		||||
				podsToRemove, blockingPod, err = FastGetPodsToMove(nodeInfo, *skipNodesWithSystemPods, *skipNodesWithLocalStorage,
 | 
			
		||||
					podDisruptionBudgets)
 | 
			
		||||
			} else {
 | 
			
		||||
				podsToRemove, err = DetailedGetPodsForMove(nodeInfo, *skipNodesWithSystemPods, *skipNodesWithLocalStorage, listers, int32(*minReplicaCount),
 | 
			
		||||
				podsToRemove, blockingPod, err = DetailedGetPodsForMove(nodeInfo, *skipNodesWithSystemPods, *skipNodesWithLocalStorage, listers, int32(*minReplicaCount),
 | 
			
		||||
					podDisruptionBudgets)
 | 
			
		||||
			}
 | 
			
		||||
			if err != nil {
 | 
			
		||||
				klog.V(2).Infof("%s: node %s cannot be removed: %v", evaluationType, node.Name, err)
 | 
			
		||||
				unremovable = append(unremovable, node)
 | 
			
		||||
				if blockingPod != nil {
 | 
			
		||||
					unremovable = append(unremovable, &UnremovableNode{Node: node, Reason: BlockedByPod, BlockingPod: blockingPod})
 | 
			
		||||
				} else {
 | 
			
		||||
					unremovable = append(unremovable, &UnremovableNode{Node: node, Reason: UnexpectedError})
 | 
			
		||||
				}
 | 
			
		||||
				continue candidateloop
 | 
			
		||||
			}
 | 
			
		||||
		} else {
 | 
			
		||||
			klog.V(2).Infof("%s: nodeInfo for %s not found", evaluationType, node.Name)
 | 
			
		||||
			unremovable = append(unremovable, node)
 | 
			
		||||
			unremovable = append(unremovable, &UnremovableNode{Node: node, Reason: UnexpectedError})
 | 
			
		||||
			continue candidateloop
 | 
			
		||||
		}
 | 
			
		||||
		findProblems := findPlaceFor(node.Name, podsToRemove, destinationNodes, clusterSnapshot,
 | 
			
		||||
| 
						 | 
				
			
			@ -126,7 +174,7 @@ candidateloop:
 | 
			
		|||
			}
 | 
			
		||||
		} else {
 | 
			
		||||
			klog.V(2).Infof("%s: node %s is not suitable for removal: %v", evaluationType, node.Name, findProblems)
 | 
			
		||||
			unremovable = append(unremovable, node)
 | 
			
		||||
			unremovable = append(unremovable, &UnremovableNode{Node: node, Reason: NoPlaceToMovePods})
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
	return result, unremovable, newHints, nil
 | 
			
		||||
| 
						 | 
				
			
			@ -139,7 +187,7 @@ func FindEmptyNodesToRemove(candidates []*apiv1.Node, pods []*apiv1.Pod) []*apiv
 | 
			
		|||
	for _, node := range candidates {
 | 
			
		||||
		if nodeInfo, found := nodeNameToNodeInfo[node.Name]; found {
 | 
			
		||||
			// Should block on all pods.
 | 
			
		||||
			podsToRemove, err := FastGetPodsToMove(nodeInfo, true, true, nil)
 | 
			
		||||
			podsToRemove, _, err := FastGetPodsToMove(nodeInfo, true, true, nil)
 | 
			
		||||
			if err == nil && len(podsToRemove) == 0 {
 | 
			
		||||
				result = append(result, node)
 | 
			
		||||
			}
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -23,6 +23,7 @@ import (
 | 
			
		|||
 | 
			
		||||
	apiv1 "k8s.io/api/core/v1"
 | 
			
		||||
	policyv1 "k8s.io/api/policy/v1beta1"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/utils/drain"
 | 
			
		||||
	. "k8s.io/autoscaler/cluster-autoscaler/utils/test"
 | 
			
		||||
	"k8s.io/kubernetes/pkg/kubelet/types"
 | 
			
		||||
	schedulernodeinfo "k8s.io/kubernetes/pkg/scheduler/nodeinfo"
 | 
			
		||||
| 
						 | 
				
			
			@ -244,7 +245,7 @@ type findNodesToRemoveTestConfig struct {
 | 
			
		|||
	candidates  []*apiv1.Node
 | 
			
		||||
	allNodes    []*apiv1.Node
 | 
			
		||||
	toRemove    []NodeToBeRemoved
 | 
			
		||||
	unremovable []*apiv1.Node
 | 
			
		||||
	unremovable []*UnremovableNode
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func TestFindNodesToRemove(t *testing.T) {
 | 
			
		||||
| 
						 | 
				
			
			@ -299,7 +300,7 @@ func TestFindNodesToRemove(t *testing.T) {
 | 
			
		|||
			candidates:  []*apiv1.Node{emptyNode},
 | 
			
		||||
			allNodes:    []*apiv1.Node{emptyNode},
 | 
			
		||||
			toRemove:    []NodeToBeRemoved{emptyNodeToRemove},
 | 
			
		||||
			unremovable: []*apiv1.Node{},
 | 
			
		||||
			unremovable: []*UnremovableNode{},
 | 
			
		||||
		},
 | 
			
		||||
		// just a drainable node, but nowhere for pods to go to
 | 
			
		||||
		{
 | 
			
		||||
| 
						 | 
				
			
			@ -308,7 +309,7 @@ func TestFindNodesToRemove(t *testing.T) {
 | 
			
		|||
			candidates:  []*apiv1.Node{drainableNode},
 | 
			
		||||
			allNodes:    []*apiv1.Node{drainableNode},
 | 
			
		||||
			toRemove:    []NodeToBeRemoved{},
 | 
			
		||||
			unremovable: []*apiv1.Node{drainableNode},
 | 
			
		||||
			unremovable: []*UnremovableNode{{Node: drainableNode, Reason: NoPlaceToMovePods}},
 | 
			
		||||
		},
 | 
			
		||||
		// drainable node, and a mostly empty node that can take its pods
 | 
			
		||||
		{
 | 
			
		||||
| 
						 | 
				
			
			@ -317,7 +318,7 @@ func TestFindNodesToRemove(t *testing.T) {
 | 
			
		|||
			candidates:  []*apiv1.Node{drainableNode, nonDrainableNode},
 | 
			
		||||
			allNodes:    []*apiv1.Node{drainableNode, nonDrainableNode},
 | 
			
		||||
			toRemove:    []NodeToBeRemoved{drainableNodeToRemove},
 | 
			
		||||
			unremovable: []*apiv1.Node{nonDrainableNode},
 | 
			
		||||
			unremovable: []*UnremovableNode{{Node: nonDrainableNode, Reason: BlockedByPod, BlockingPod: &drain.BlockingPod{Pod: pod3, Reason: drain.NotReplicated}}},
 | 
			
		||||
		},
 | 
			
		||||
		// drainable node, and a full node that cannot fit anymore pods
 | 
			
		||||
		{
 | 
			
		||||
| 
						 | 
				
			
			@ -326,7 +327,7 @@ func TestFindNodesToRemove(t *testing.T) {
 | 
			
		|||
			candidates:  []*apiv1.Node{drainableNode},
 | 
			
		||||
			allNodes:    []*apiv1.Node{drainableNode, fullNode},
 | 
			
		||||
			toRemove:    []NodeToBeRemoved{},
 | 
			
		||||
			unremovable: []*apiv1.Node{drainableNode},
 | 
			
		||||
			unremovable: []*UnremovableNode{{Node: drainableNode, Reason: NoPlaceToMovePods}},
 | 
			
		||||
		},
 | 
			
		||||
		// 4 nodes, 1 empty, 1 drainable
 | 
			
		||||
		{
 | 
			
		||||
| 
						 | 
				
			
			@ -335,7 +336,7 @@ func TestFindNodesToRemove(t *testing.T) {
 | 
			
		|||
			candidates:  []*apiv1.Node{emptyNode, drainableNode},
 | 
			
		||||
			allNodes:    []*apiv1.Node{emptyNode, drainableNode, fullNode, nonDrainableNode},
 | 
			
		||||
			toRemove:    []NodeToBeRemoved{emptyNodeToRemove, drainableNodeToRemove},
 | 
			
		||||
			unremovable: []*apiv1.Node{},
 | 
			
		||||
			unremovable: []*UnremovableNode{},
 | 
			
		||||
		},
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -35,8 +35,8 @@ import (
 | 
			
		|||
// along with their pods (no abandoned pods with dangling created-by annotation). Useful for fast
 | 
			
		||||
// checks.
 | 
			
		||||
func FastGetPodsToMove(nodeInfo *schedulernodeinfo.NodeInfo, skipNodesWithSystemPods bool, skipNodesWithLocalStorage bool,
 | 
			
		||||
	pdbs []*policyv1.PodDisruptionBudget) ([]*apiv1.Pod, error) {
 | 
			
		||||
	pods, err := drain.GetPodsForDeletionOnNodeDrain(
 | 
			
		||||
	pdbs []*policyv1.PodDisruptionBudget) ([]*apiv1.Pod, *drain.BlockingPod, error) {
 | 
			
		||||
	pods, blockingPod, err := drain.GetPodsForDeletionOnNodeDrain(
 | 
			
		||||
		nodeInfo.Pods(),
 | 
			
		||||
		pdbs,
 | 
			
		||||
		skipNodesWithSystemPods,
 | 
			
		||||
| 
						 | 
				
			
			@ -47,13 +47,13 @@ func FastGetPodsToMove(nodeInfo *schedulernodeinfo.NodeInfo, skipNodesWithSystem
 | 
			
		|||
		time.Now())
 | 
			
		||||
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		return pods, err
 | 
			
		||||
		return pods, blockingPod, err
 | 
			
		||||
	}
 | 
			
		||||
	if err := checkPdbs(pods, pdbs); err != nil {
 | 
			
		||||
		return []*apiv1.Pod{}, err
 | 
			
		||||
	if pdbBlockingPod, err := checkPdbs(pods, pdbs); err != nil {
 | 
			
		||||
		return []*apiv1.Pod{}, pdbBlockingPod, err
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	return pods, nil
 | 
			
		||||
	return pods, nil, nil
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// DetailedGetPodsForMove returns a list of pods that should be moved elsewhere if the node
 | 
			
		||||
| 
						 | 
				
			
			@ -62,8 +62,8 @@ func FastGetPodsToMove(nodeInfo *schedulernodeinfo.NodeInfo, skipNodesWithSystem
 | 
			
		|||
// still exist.
 | 
			
		||||
func DetailedGetPodsForMove(nodeInfo *schedulernodeinfo.NodeInfo, skipNodesWithSystemPods bool,
 | 
			
		||||
	skipNodesWithLocalStorage bool, listers kube_util.ListerRegistry, minReplicaCount int32,
 | 
			
		||||
	pdbs []*policyv1.PodDisruptionBudget) ([]*apiv1.Pod, error) {
 | 
			
		||||
	pods, err := drain.GetPodsForDeletionOnNodeDrain(
 | 
			
		||||
	pdbs []*policyv1.PodDisruptionBudget) ([]*apiv1.Pod, *drain.BlockingPod, error) {
 | 
			
		||||
	pods, blockingPod, err := drain.GetPodsForDeletionOnNodeDrain(
 | 
			
		||||
		nodeInfo.Pods(),
 | 
			
		||||
		pdbs,
 | 
			
		||||
		skipNodesWithSystemPods,
 | 
			
		||||
| 
						 | 
				
			
			@ -73,29 +73,29 @@ func DetailedGetPodsForMove(nodeInfo *schedulernodeinfo.NodeInfo, skipNodesWithS
 | 
			
		|||
		minReplicaCount,
 | 
			
		||||
		time.Now())
 | 
			
		||||
	if err != nil {
 | 
			
		||||
		return pods, err
 | 
			
		||||
		return pods, blockingPod, err
 | 
			
		||||
	}
 | 
			
		||||
	if err := checkPdbs(pods, pdbs); err != nil {
 | 
			
		||||
		return []*apiv1.Pod{}, err
 | 
			
		||||
	if pdbBlockingPod, err := checkPdbs(pods, pdbs); err != nil {
 | 
			
		||||
		return []*apiv1.Pod{}, pdbBlockingPod, err
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	return pods, nil
 | 
			
		||||
	return pods, nil, nil
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
func checkPdbs(pods []*apiv1.Pod, pdbs []*policyv1.PodDisruptionBudget) error {
 | 
			
		||||
func checkPdbs(pods []*apiv1.Pod, pdbs []*policyv1.PodDisruptionBudget) (*drain.BlockingPod, error) {
 | 
			
		||||
	// TODO: make it more efficient.
 | 
			
		||||
	for _, pdb := range pdbs {
 | 
			
		||||
		selector, err := metav1.LabelSelectorAsSelector(pdb.Spec.Selector)
 | 
			
		||||
		if err != nil {
 | 
			
		||||
			return err
 | 
			
		||||
			return nil, err
 | 
			
		||||
		}
 | 
			
		||||
		for _, pod := range pods {
 | 
			
		||||
			if pod.Namespace == pdb.Namespace && selector.Matches(labels.Set(pod.Labels)) {
 | 
			
		||||
				if pdb.Status.DisruptionsAllowed < 1 {
 | 
			
		||||
					return fmt.Errorf("not enough pod disruption budget to move %s/%s", pod.Namespace, pod.Name)
 | 
			
		||||
					return &drain.BlockingPod{Pod: pod, Reason: drain.NotEnoughPdb}, fmt.Errorf("not enough pod disruption budget to move %s/%s", pod.Namespace, pod.Name)
 | 
			
		||||
				}
 | 
			
		||||
			}
 | 
			
		||||
		}
 | 
			
		||||
	}
 | 
			
		||||
	return nil
 | 
			
		||||
	return nil, nil
 | 
			
		||||
}
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -23,6 +23,7 @@ import (
 | 
			
		|||
	policyv1 "k8s.io/api/policy/v1beta1"
 | 
			
		||||
	metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
 | 
			
		||||
	"k8s.io/apimachinery/pkg/util/intstr"
 | 
			
		||||
	"k8s.io/autoscaler/cluster-autoscaler/utils/drain"
 | 
			
		||||
	. "k8s.io/autoscaler/cluster-autoscaler/utils/test"
 | 
			
		||||
	"k8s.io/kubernetes/pkg/kubelet/types"
 | 
			
		||||
	schedulernodeinfo "k8s.io/kubernetes/pkg/scheduler/nodeinfo"
 | 
			
		||||
| 
						 | 
				
			
			@ -39,8 +40,9 @@ func TestFastGetPodsToMove(t *testing.T) {
 | 
			
		|||
			Namespace: "ns",
 | 
			
		||||
		},
 | 
			
		||||
	}
 | 
			
		||||
	_, err := FastGetPodsToMove(schedulernodeinfo.NewNodeInfo(pod1), true, true, nil)
 | 
			
		||||
	_, blockingPod, err := FastGetPodsToMove(schedulernodeinfo.NewNodeInfo(pod1), true, true, nil)
 | 
			
		||||
	assert.Error(t, err)
 | 
			
		||||
	assert.Equal(t, &drain.BlockingPod{Pod: pod1, Reason: drain.NotReplicated}, blockingPod)
 | 
			
		||||
 | 
			
		||||
	// Replicated pod
 | 
			
		||||
	pod2 := &apiv1.Pod{
 | 
			
		||||
| 
						 | 
				
			
			@ -50,8 +52,9 @@ func TestFastGetPodsToMove(t *testing.T) {
 | 
			
		|||
			OwnerReferences: GenerateOwnerReferences("rs", "ReplicaSet", "extensions/v1beta1", ""),
 | 
			
		||||
		},
 | 
			
		||||
	}
 | 
			
		||||
	r2, err := FastGetPodsToMove(schedulernodeinfo.NewNodeInfo(pod2), true, true, nil)
 | 
			
		||||
	r2, blockingPod, err := FastGetPodsToMove(schedulernodeinfo.NewNodeInfo(pod2), true, true, nil)
 | 
			
		||||
	assert.NoError(t, err)
 | 
			
		||||
	assert.Nil(t, blockingPod)
 | 
			
		||||
	assert.Equal(t, 1, len(r2))
 | 
			
		||||
	assert.Equal(t, pod2, r2[0])
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			@ -65,8 +68,9 @@ func TestFastGetPodsToMove(t *testing.T) {
 | 
			
		|||
			},
 | 
			
		||||
		},
 | 
			
		||||
	}
 | 
			
		||||
	r3, err := FastGetPodsToMove(schedulernodeinfo.NewNodeInfo(pod3), true, true, nil)
 | 
			
		||||
	r3, blockingPod, err := FastGetPodsToMove(schedulernodeinfo.NewNodeInfo(pod3), true, true, nil)
 | 
			
		||||
	assert.NoError(t, err)
 | 
			
		||||
	assert.Nil(t, blockingPod)
 | 
			
		||||
	assert.Equal(t, 0, len(r3))
 | 
			
		||||
 | 
			
		||||
	// DaemonSet pod
 | 
			
		||||
| 
						 | 
				
			
			@ -77,8 +81,9 @@ func TestFastGetPodsToMove(t *testing.T) {
 | 
			
		|||
			OwnerReferences: GenerateOwnerReferences("ds", "DaemonSet", "extensions/v1beta1", ""),
 | 
			
		||||
		},
 | 
			
		||||
	}
 | 
			
		||||
	r4, err := FastGetPodsToMove(schedulernodeinfo.NewNodeInfo(pod2, pod3, pod4), true, true, nil)
 | 
			
		||||
	r4, blockingPod, err := FastGetPodsToMove(schedulernodeinfo.NewNodeInfo(pod2, pod3, pod4), true, true, nil)
 | 
			
		||||
	assert.NoError(t, err)
 | 
			
		||||
	assert.Nil(t, blockingPod)
 | 
			
		||||
	assert.Equal(t, 1, len(r4))
 | 
			
		||||
	assert.Equal(t, pod2, r4[0])
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			@ -90,8 +95,9 @@ func TestFastGetPodsToMove(t *testing.T) {
 | 
			
		|||
			OwnerReferences: GenerateOwnerReferences("rs", "ReplicaSet", "extensions/v1beta1", ""),
 | 
			
		||||
		},
 | 
			
		||||
	}
 | 
			
		||||
	_, err = FastGetPodsToMove(schedulernodeinfo.NewNodeInfo(pod5), true, true, nil)
 | 
			
		||||
	_, blockingPod, err = FastGetPodsToMove(schedulernodeinfo.NewNodeInfo(pod5), true, true, nil)
 | 
			
		||||
	assert.Error(t, err)
 | 
			
		||||
	assert.Equal(t, &drain.BlockingPod{Pod: pod5, Reason: drain.UnmovableKubeSystemPod}, blockingPod)
 | 
			
		||||
 | 
			
		||||
	// Local storage
 | 
			
		||||
	pod6 := &apiv1.Pod{
 | 
			
		||||
| 
						 | 
				
			
			@ -110,8 +116,9 @@ func TestFastGetPodsToMove(t *testing.T) {
 | 
			
		|||
			},
 | 
			
		||||
		},
 | 
			
		||||
	}
 | 
			
		||||
	_, err = FastGetPodsToMove(schedulernodeinfo.NewNodeInfo(pod6), true, true, nil)
 | 
			
		||||
	_, blockingPod, err = FastGetPodsToMove(schedulernodeinfo.NewNodeInfo(pod6), true, true, nil)
 | 
			
		||||
	assert.Error(t, err)
 | 
			
		||||
	assert.Equal(t, &drain.BlockingPod{Pod: pod6, Reason: drain.LocalStorageRequested}, blockingPod)
 | 
			
		||||
 | 
			
		||||
	// Non-local storage
 | 
			
		||||
	pod7 := &apiv1.Pod{
 | 
			
		||||
| 
						 | 
				
			
			@ -132,8 +139,9 @@ func TestFastGetPodsToMove(t *testing.T) {
 | 
			
		|||
			},
 | 
			
		||||
		},
 | 
			
		||||
	}
 | 
			
		||||
	r7, err := FastGetPodsToMove(schedulernodeinfo.NewNodeInfo(pod7), true, true, nil)
 | 
			
		||||
	r7, blockingPod, err := FastGetPodsToMove(schedulernodeinfo.NewNodeInfo(pod7), true, true, nil)
 | 
			
		||||
	assert.NoError(t, err)
 | 
			
		||||
	assert.Nil(t, blockingPod)
 | 
			
		||||
	assert.Equal(t, 1, len(r7))
 | 
			
		||||
 | 
			
		||||
	// Pdb blocking
 | 
			
		||||
| 
						 | 
				
			
			@ -167,8 +175,9 @@ func TestFastGetPodsToMove(t *testing.T) {
 | 
			
		|||
		},
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	_, err = FastGetPodsToMove(schedulernodeinfo.NewNodeInfo(pod8), true, true, []*policyv1.PodDisruptionBudget{pdb8})
 | 
			
		||||
	_, blockingPod, err = FastGetPodsToMove(schedulernodeinfo.NewNodeInfo(pod8), true, true, []*policyv1.PodDisruptionBudget{pdb8})
 | 
			
		||||
	assert.Error(t, err)
 | 
			
		||||
	assert.Equal(t, &drain.BlockingPod{Pod: pod8, Reason: drain.NotEnoughPdb}, blockingPod)
 | 
			
		||||
 | 
			
		||||
	// Pdb allowing
 | 
			
		||||
	pod9 := &apiv1.Pod{
 | 
			
		||||
| 
						 | 
				
			
			@ -200,7 +209,8 @@ func TestFastGetPodsToMove(t *testing.T) {
 | 
			
		|||
		},
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
	r9, err := FastGetPodsToMove(schedulernodeinfo.NewNodeInfo(pod9), true, true, []*policyv1.PodDisruptionBudget{pdb9})
 | 
			
		||||
	r9, blockingPod, err := FastGetPodsToMove(schedulernodeinfo.NewNodeInfo(pod9), true, true, []*policyv1.PodDisruptionBudget{pdb9})
 | 
			
		||||
	assert.NoError(t, err)
 | 
			
		||||
	assert.Nil(t, blockingPod)
 | 
			
		||||
	assert.Equal(t, 1, len(r9))
 | 
			
		||||
}
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -25,7 +25,6 @@ import (
 | 
			
		|||
	apierrors "k8s.io/apimachinery/pkg/api/errors"
 | 
			
		||||
	metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
 | 
			
		||||
	"k8s.io/apimachinery/pkg/labels"
 | 
			
		||||
 | 
			
		||||
	kube_util "k8s.io/autoscaler/cluster-autoscaler/utils/kubernetes"
 | 
			
		||||
	pod_util "k8s.io/autoscaler/cluster-autoscaler/utils/pod"
 | 
			
		||||
)
 | 
			
		||||
| 
						 | 
				
			
			@ -41,6 +40,37 @@ const (
 | 
			
		|||
	PodSafeToEvictKey = "cluster-autoscaler.kubernetes.io/safe-to-evict"
 | 
			
		||||
)
 | 
			
		||||
 | 
			
		||||
// BlockingPod represents a pod which is blocking the scale down of a node.
 | 
			
		||||
type BlockingPod struct {
 | 
			
		||||
	Pod    *apiv1.Pod
 | 
			
		||||
	Reason BlockingPodReason
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// BlockingPodReason represents a reason why a pod is blocking the scale down of a node.
 | 
			
		||||
type BlockingPodReason int
 | 
			
		||||
 | 
			
		||||
const (
 | 
			
		||||
	// NoReason - sanity check, this should never be set explicitly. If this is found in the wild, it means that it was
 | 
			
		||||
	// implicitly initialized and might indicate a bug.
 | 
			
		||||
	NoReason BlockingPodReason = iota
 | 
			
		||||
	// ControllerNotFound - pod is blocking scale down because its controller can't be found.
 | 
			
		||||
	ControllerNotFound
 | 
			
		||||
	// MinReplicasReached - pod is blocking scale down because its controller already has the minimum number of replicas.
 | 
			
		||||
	MinReplicasReached
 | 
			
		||||
	// NotReplicated - pod is blocking scale down because it's not replicated.
 | 
			
		||||
	NotReplicated
 | 
			
		||||
	// LocalStorageRequested - pod is blocking scale down because it requests local storage.
 | 
			
		||||
	LocalStorageRequested
 | 
			
		||||
	// NotSafeToEvictAnnotation - pod is blocking scale down because it has a "not safe to evict" annotation.
 | 
			
		||||
	NotSafeToEvictAnnotation
 | 
			
		||||
	// UnmovableKubeSystemPod - pod is blocking scale down because it's a non-daemonset, non-mirrored, non-pdb-assigned kube-system pod.
 | 
			
		||||
	UnmovableKubeSystemPod
 | 
			
		||||
	// NotEnoughPdb - pod is blocking scale down because it doesn't have enough PDB left.
 | 
			
		||||
	NotEnoughPdb
 | 
			
		||||
	// UnexpectedError - pod is blocking scale down because of an unexpected error.
 | 
			
		||||
	UnexpectedError
 | 
			
		||||
)
 | 
			
		||||
 | 
			
		||||
// GetPodsForDeletionOnNodeDrain returns pods that should be deleted on node drain as well as some extra information
 | 
			
		||||
// about possibly problematic pods (unreplicated and daemonsets).
 | 
			
		||||
func GetPodsForDeletionOnNodeDrain(
 | 
			
		||||
| 
						 | 
				
			
			@ -51,7 +81,7 @@ func GetPodsForDeletionOnNodeDrain(
 | 
			
		|||
	checkReferences bool, // Setting this to true requires client to be not-null.
 | 
			
		||||
	listers kube_util.ListerRegistry,
 | 
			
		||||
	minReplica int32,
 | 
			
		||||
	currentTime time.Time) ([]*apiv1.Pod, error) {
 | 
			
		||||
	currentTime time.Time) ([]*apiv1.Pod, *BlockingPod, error) {
 | 
			
		||||
 | 
			
		||||
	pods := []*apiv1.Pod{}
 | 
			
		||||
	// filter kube-system PDBs to avoid doing it for every kube-system pod
 | 
			
		||||
| 
						 | 
				
			
			@ -98,12 +128,12 @@ func GetPodsForDeletionOnNodeDrain(
 | 
			
		|||
				// TODO: replace the minReplica check with pod disruption budget.
 | 
			
		||||
				if err == nil && rc != nil {
 | 
			
		||||
					if rc.Spec.Replicas != nil && *rc.Spec.Replicas < minReplica {
 | 
			
		||||
						return []*apiv1.Pod{}, fmt.Errorf("replication controller for %s/%s has too few replicas spec: %d min: %d",
 | 
			
		||||
						return []*apiv1.Pod{}, &BlockingPod{Pod: pod, Reason: MinReplicasReached}, fmt.Errorf("replication controller for %s/%s has too few replicas spec: %d min: %d",
 | 
			
		||||
							pod.Namespace, pod.Name, rc.Spec.Replicas, minReplica)
 | 
			
		||||
					}
 | 
			
		||||
					replicated = true
 | 
			
		||||
				} else {
 | 
			
		||||
					return []*apiv1.Pod{}, fmt.Errorf("replication controller for %s/%s is not available, err: %v", pod.Namespace, pod.Name, err)
 | 
			
		||||
					return []*apiv1.Pod{}, &BlockingPod{Pod: pod, Reason: ControllerNotFound}, fmt.Errorf("replication controller for %s/%s is not available, err: %v", pod.Namespace, pod.Name, err)
 | 
			
		||||
				}
 | 
			
		||||
			} else {
 | 
			
		||||
				replicated = true
 | 
			
		||||
| 
						 | 
				
			
			@ -115,9 +145,9 @@ func GetPodsForDeletionOnNodeDrain(
 | 
			
		|||
			if checkReferences && refKind == "DaemonSet" {
 | 
			
		||||
				_, err := listers.DaemonSetLister().DaemonSets(controllerNamespace).Get(controllerRef.Name)
 | 
			
		||||
				if apierrors.IsNotFound(err) {
 | 
			
		||||
					return []*apiv1.Pod{}, fmt.Errorf("daemonset for %s/%s is not present, err: %v", pod.Namespace, pod.Name, err)
 | 
			
		||||
					return []*apiv1.Pod{}, &BlockingPod{Pod: pod, Reason: ControllerNotFound}, fmt.Errorf("daemonset for %s/%s is not present, err: %v", pod.Namespace, pod.Name, err)
 | 
			
		||||
				} else if err != nil {
 | 
			
		||||
					return []*apiv1.Pod{}, fmt.Errorf("error when trying to get daemonset for %s/%s , err: %v", pod.Namespace, pod.Name, err)
 | 
			
		||||
					return []*apiv1.Pod{}, &BlockingPod{Pod: pod, Reason: UnexpectedError}, fmt.Errorf("error when trying to get daemonset for %s/%s , err: %v", pod.Namespace, pod.Name, err)
 | 
			
		||||
				}
 | 
			
		||||
			}
 | 
			
		||||
		} else if refKind == "Job" {
 | 
			
		||||
| 
						 | 
				
			
			@ -130,7 +160,7 @@ func GetPodsForDeletionOnNodeDrain(
 | 
			
		|||
				if err == nil && job != nil {
 | 
			
		||||
					replicated = true
 | 
			
		||||
				} else {
 | 
			
		||||
					return []*apiv1.Pod{}, fmt.Errorf("job for %s/%s is not available: err: %v", pod.Namespace, pod.Name, err)
 | 
			
		||||
					return []*apiv1.Pod{}, &BlockingPod{Pod: pod, Reason: ControllerNotFound}, fmt.Errorf("job for %s/%s is not available: err: %v", pod.Namespace, pod.Name, err)
 | 
			
		||||
				}
 | 
			
		||||
			} else {
 | 
			
		||||
				replicated = true
 | 
			
		||||
| 
						 | 
				
			
			@ -144,12 +174,12 @@ func GetPodsForDeletionOnNodeDrain(
 | 
			
		|||
				// sophisticated than this
 | 
			
		||||
				if err == nil && rs != nil {
 | 
			
		||||
					if rs.Spec.Replicas != nil && *rs.Spec.Replicas < minReplica {
 | 
			
		||||
						return []*apiv1.Pod{}, fmt.Errorf("replication controller for %s/%s has too few replicas spec: %d min: %d",
 | 
			
		||||
						return []*apiv1.Pod{}, &BlockingPod{Pod: pod, Reason: MinReplicasReached}, fmt.Errorf("replication controller for %s/%s has too few replicas spec: %d min: %d",
 | 
			
		||||
							pod.Namespace, pod.Name, rs.Spec.Replicas, minReplica)
 | 
			
		||||
					}
 | 
			
		||||
					replicated = true
 | 
			
		||||
				} else {
 | 
			
		||||
					return []*apiv1.Pod{}, fmt.Errorf("replication controller for %s/%s is not available, err: %v", pod.Namespace, pod.Name, err)
 | 
			
		||||
					return []*apiv1.Pod{}, &BlockingPod{Pod: pod, Reason: ControllerNotFound}, fmt.Errorf("replication controller for %s/%s is not available, err: %v", pod.Namespace, pod.Name, err)
 | 
			
		||||
				}
 | 
			
		||||
			} else {
 | 
			
		||||
				replicated = true
 | 
			
		||||
| 
						 | 
				
			
			@ -164,7 +194,7 @@ func GetPodsForDeletionOnNodeDrain(
 | 
			
		|||
				if err == nil && ss != nil {
 | 
			
		||||
					replicated = true
 | 
			
		||||
				} else {
 | 
			
		||||
					return []*apiv1.Pod{}, fmt.Errorf("statefulset for %s/%s is not available: err: %v", pod.Namespace, pod.Name, err)
 | 
			
		||||
					return []*apiv1.Pod{}, &BlockingPod{Pod: pod, Reason: ControllerNotFound}, fmt.Errorf("statefulset for %s/%s is not available: err: %v", pod.Namespace, pod.Name, err)
 | 
			
		||||
				}
 | 
			
		||||
			} else {
 | 
			
		||||
				replicated = true
 | 
			
		||||
| 
						 | 
				
			
			@ -176,27 +206,27 @@ func GetPodsForDeletionOnNodeDrain(
 | 
			
		|||
 | 
			
		||||
		if !safeToEvict && !terminal {
 | 
			
		||||
			if !replicated {
 | 
			
		||||
				return []*apiv1.Pod{}, fmt.Errorf("%s/%s is not replicated", pod.Namespace, pod.Name)
 | 
			
		||||
				return []*apiv1.Pod{}, &BlockingPod{Pod: pod, Reason: NotReplicated}, fmt.Errorf("%s/%s is not replicated", pod.Namespace, pod.Name)
 | 
			
		||||
			}
 | 
			
		||||
			if pod.Namespace == "kube-system" && skipNodesWithSystemPods {
 | 
			
		||||
				hasPDB, err := checkKubeSystemPDBs(pod, kubeSystemPDBs)
 | 
			
		||||
				if err != nil {
 | 
			
		||||
					return []*apiv1.Pod{}, fmt.Errorf("error matching pods to pdbs: %v", err)
 | 
			
		||||
					return []*apiv1.Pod{}, &BlockingPod{Pod: pod, Reason: UnexpectedError}, fmt.Errorf("error matching pods to pdbs: %v", err)
 | 
			
		||||
				}
 | 
			
		||||
				if !hasPDB {
 | 
			
		||||
					return []*apiv1.Pod{}, fmt.Errorf("non-daemonset, non-mirrored, non-pdb-assigned kube-system pod present: %s", pod.Name)
 | 
			
		||||
					return []*apiv1.Pod{}, &BlockingPod{Pod: pod, Reason: UnmovableKubeSystemPod}, fmt.Errorf("non-daemonset, non-mirrored, non-pdb-assigned kube-system pod present: %s", pod.Name)
 | 
			
		||||
				}
 | 
			
		||||
			}
 | 
			
		||||
			if HasLocalStorage(pod) && skipNodesWithLocalStorage {
 | 
			
		||||
				return []*apiv1.Pod{}, fmt.Errorf("pod with local storage present: %s", pod.Name)
 | 
			
		||||
				return []*apiv1.Pod{}, &BlockingPod{Pod: pod, Reason: LocalStorageRequested}, fmt.Errorf("pod with local storage present: %s", pod.Name)
 | 
			
		||||
			}
 | 
			
		||||
			if hasNotSafeToEvictAnnotation(pod) {
 | 
			
		||||
				return []*apiv1.Pod{}, fmt.Errorf("pod annotated as not safe to evict present: %s", pod.Name)
 | 
			
		||||
				return []*apiv1.Pod{}, &BlockingPod{Pod: pod, Reason: NotSafeToEvictAnnotation}, fmt.Errorf("pod annotated as not safe to evict present: %s", pod.Name)
 | 
			
		||||
			}
 | 
			
		||||
		}
 | 
			
		||||
		pods = append(pods, pod)
 | 
			
		||||
	}
 | 
			
		||||
	return pods, nil
 | 
			
		||||
	return pods, nil, nil
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
// ControllerRef returns the OwnerReference to pod's controller.
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -194,8 +194,9 @@ func TestDrain(t *testing.T) {
 | 
			
		|||
 | 
			
		||||
	emptydirPod := &apiv1.Pod{
 | 
			
		||||
		ObjectMeta: metav1.ObjectMeta{
 | 
			
		||||
			Name:      "bar",
 | 
			
		||||
			Namespace: "default",
 | 
			
		||||
			Name:            "bar",
 | 
			
		||||
			Namespace:       "default",
 | 
			
		||||
			OwnerReferences: GenerateOwnerReferences(rc.Name, "ReplicationController", "core/v1", ""),
 | 
			
		||||
		},
 | 
			
		||||
		Spec: apiv1.PodSpec{
 | 
			
		||||
			NodeName: "node",
 | 
			
		||||
| 
						 | 
				
			
			@ -362,13 +363,14 @@ func TestDrain(t *testing.T) {
 | 
			
		|||
	}
 | 
			
		||||
 | 
			
		||||
	tests := []struct {
 | 
			
		||||
		description string
 | 
			
		||||
		pods        []*apiv1.Pod
 | 
			
		||||
		pdbs        []*policyv1.PodDisruptionBudget
 | 
			
		||||
		rcs         []*apiv1.ReplicationController
 | 
			
		||||
		replicaSets []*appsv1.ReplicaSet
 | 
			
		||||
		expectFatal bool
 | 
			
		||||
		expectPods  []*apiv1.Pod
 | 
			
		||||
		description       string
 | 
			
		||||
		pods              []*apiv1.Pod
 | 
			
		||||
		pdbs              []*policyv1.PodDisruptionBudget
 | 
			
		||||
		rcs               []*apiv1.ReplicationController
 | 
			
		||||
		replicaSets       []*appsv1.ReplicaSet
 | 
			
		||||
		expectFatal       bool
 | 
			
		||||
		expectPods        []*apiv1.Pod
 | 
			
		||||
		expectBlockingPod *BlockingPod
 | 
			
		||||
	}{
 | 
			
		||||
		{
 | 
			
		||||
			description: "RC-managed pod",
 | 
			
		||||
| 
						 | 
				
			
			@ -425,18 +427,21 @@ func TestDrain(t *testing.T) {
 | 
			
		|||
			expectPods:  []*apiv1.Pod{},
 | 
			
		||||
		},
 | 
			
		||||
		{
 | 
			
		||||
			description: "naked pod",
 | 
			
		||||
			pods:        []*apiv1.Pod{nakedPod},
 | 
			
		||||
			pdbs:        []*policyv1.PodDisruptionBudget{},
 | 
			
		||||
			expectFatal: true,
 | 
			
		||||
			expectPods:  []*apiv1.Pod{},
 | 
			
		||||
			description:       "naked pod",
 | 
			
		||||
			pods:              []*apiv1.Pod{nakedPod},
 | 
			
		||||
			pdbs:              []*policyv1.PodDisruptionBudget{},
 | 
			
		||||
			expectFatal:       true,
 | 
			
		||||
			expectPods:        []*apiv1.Pod{},
 | 
			
		||||
			expectBlockingPod: &BlockingPod{Pod: nakedPod, Reason: NotReplicated},
 | 
			
		||||
		},
 | 
			
		||||
		{
 | 
			
		||||
			description: "pod with EmptyDir",
 | 
			
		||||
			pods:        []*apiv1.Pod{emptydirPod},
 | 
			
		||||
			pdbs:        []*policyv1.PodDisruptionBudget{},
 | 
			
		||||
			expectFatal: true,
 | 
			
		||||
			expectPods:  []*apiv1.Pod{},
 | 
			
		||||
			description:       "pod with EmptyDir",
 | 
			
		||||
			pods:              []*apiv1.Pod{emptydirPod},
 | 
			
		||||
			pdbs:              []*policyv1.PodDisruptionBudget{},
 | 
			
		||||
			rcs:               []*apiv1.ReplicationController{&rc},
 | 
			
		||||
			expectFatal:       true,
 | 
			
		||||
			expectPods:        []*apiv1.Pod{},
 | 
			
		||||
			expectBlockingPod: &BlockingPod{Pod: emptydirPod, Reason: LocalStorageRequested},
 | 
			
		||||
		},
 | 
			
		||||
		{
 | 
			
		||||
			description: "failed pod",
 | 
			
		||||
| 
						 | 
				
			
			@ -481,20 +486,22 @@ func TestDrain(t *testing.T) {
 | 
			
		|||
			expectPods:  []*apiv1.Pod{emptydirSafePod},
 | 
			
		||||
		},
 | 
			
		||||
		{
 | 
			
		||||
			description: "RC-managed pod with PodSafeToEvict=false annotation",
 | 
			
		||||
			pods:        []*apiv1.Pod{unsafeRcPod},
 | 
			
		||||
			rcs:         []*apiv1.ReplicationController{&rc},
 | 
			
		||||
			pdbs:        []*policyv1.PodDisruptionBudget{},
 | 
			
		||||
			expectFatal: true,
 | 
			
		||||
			expectPods:  []*apiv1.Pod{},
 | 
			
		||||
			description:       "RC-managed pod with PodSafeToEvict=false annotation",
 | 
			
		||||
			pods:              []*apiv1.Pod{unsafeRcPod},
 | 
			
		||||
			rcs:               []*apiv1.ReplicationController{&rc},
 | 
			
		||||
			pdbs:              []*policyv1.PodDisruptionBudget{},
 | 
			
		||||
			expectFatal:       true,
 | 
			
		||||
			expectPods:        []*apiv1.Pod{},
 | 
			
		||||
			expectBlockingPod: &BlockingPod{Pod: unsafeRcPod, Reason: NotSafeToEvictAnnotation},
 | 
			
		||||
		},
 | 
			
		||||
		{
 | 
			
		||||
			description: "Job-managed pod with PodSafeToEvict=false annotation",
 | 
			
		||||
			pods:        []*apiv1.Pod{unsafeJobPod},
 | 
			
		||||
			pdbs:        []*policyv1.PodDisruptionBudget{},
 | 
			
		||||
			rcs:         []*apiv1.ReplicationController{&rc},
 | 
			
		||||
			expectFatal: true,
 | 
			
		||||
			expectPods:  []*apiv1.Pod{},
 | 
			
		||||
			description:       "Job-managed pod with PodSafeToEvict=false annotation",
 | 
			
		||||
			pods:              []*apiv1.Pod{unsafeJobPod},
 | 
			
		||||
			pdbs:              []*policyv1.PodDisruptionBudget{},
 | 
			
		||||
			rcs:               []*apiv1.ReplicationController{&rc},
 | 
			
		||||
			expectFatal:       true,
 | 
			
		||||
			expectPods:        []*apiv1.Pod{},
 | 
			
		||||
			expectBlockingPod: &BlockingPod{Pod: unsafeJobPod, Reason: NotSafeToEvictAnnotation},
 | 
			
		||||
		},
 | 
			
		||||
		{
 | 
			
		||||
			description: "empty PDB with RC-managed pod",
 | 
			
		||||
| 
						 | 
				
			
			@ -513,12 +520,13 @@ func TestDrain(t *testing.T) {
 | 
			
		|||
			expectPods:  []*apiv1.Pod{kubeSystemRcPod},
 | 
			
		||||
		},
 | 
			
		||||
		{
 | 
			
		||||
			description: "kube-system PDB with non-matching kube-system pod",
 | 
			
		||||
			pods:        []*apiv1.Pod{kubeSystemRcPod},
 | 
			
		||||
			pdbs:        []*policyv1.PodDisruptionBudget{kubeSystemFakePDB},
 | 
			
		||||
			rcs:         []*apiv1.ReplicationController{&kubeSystemRc},
 | 
			
		||||
			expectFatal: true,
 | 
			
		||||
			expectPods:  []*apiv1.Pod{},
 | 
			
		||||
			description:       "kube-system PDB with non-matching kube-system pod",
 | 
			
		||||
			pods:              []*apiv1.Pod{kubeSystemRcPod},
 | 
			
		||||
			pdbs:              []*policyv1.PodDisruptionBudget{kubeSystemFakePDB},
 | 
			
		||||
			rcs:               []*apiv1.ReplicationController{&kubeSystemRc},
 | 
			
		||||
			expectFatal:       true,
 | 
			
		||||
			expectPods:        []*apiv1.Pod{},
 | 
			
		||||
			expectBlockingPod: &BlockingPod{Pod: kubeSystemRcPod, Reason: UnmovableKubeSystemPod},
 | 
			
		||||
		},
 | 
			
		||||
		{
 | 
			
		||||
			description: "kube-system PDB with default namespace pod",
 | 
			
		||||
| 
						 | 
				
			
			@ -529,12 +537,13 @@ func TestDrain(t *testing.T) {
 | 
			
		|||
			expectPods:  []*apiv1.Pod{rcPod},
 | 
			
		||||
		},
 | 
			
		||||
		{
 | 
			
		||||
			description: "default namespace PDB with matching labels kube-system pod",
 | 
			
		||||
			pods:        []*apiv1.Pod{kubeSystemRcPod},
 | 
			
		||||
			pdbs:        []*policyv1.PodDisruptionBudget{defaultNamespacePDB},
 | 
			
		||||
			rcs:         []*apiv1.ReplicationController{&kubeSystemRc},
 | 
			
		||||
			expectFatal: true,
 | 
			
		||||
			expectPods:  []*apiv1.Pod{},
 | 
			
		||||
			description:       "default namespace PDB with matching labels kube-system pod",
 | 
			
		||||
			pods:              []*apiv1.Pod{kubeSystemRcPod},
 | 
			
		||||
			pdbs:              []*policyv1.PodDisruptionBudget{defaultNamespacePDB},
 | 
			
		||||
			rcs:               []*apiv1.ReplicationController{&kubeSystemRc},
 | 
			
		||||
			expectFatal:       true,
 | 
			
		||||
			expectPods:        []*apiv1.Pod{},
 | 
			
		||||
			expectBlockingPod: &BlockingPod{Pod: kubeSystemRcPod, Reason: UnmovableKubeSystemPod},
 | 
			
		||||
		},
 | 
			
		||||
	}
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			@ -560,15 +569,17 @@ func TestDrain(t *testing.T) {
 | 
			
		|||
 | 
			
		||||
		registry := kube_util.NewListerRegistry(nil, nil, nil, nil, nil, dsLister, rcLister, jobLister, rsLister, ssLister)
 | 
			
		||||
 | 
			
		||||
		pods, err := GetPodsForDeletionOnNodeDrain(test.pods, test.pdbs, true, true, true, registry, 0, time.Now())
 | 
			
		||||
		pods, blockingPod, err := GetPodsForDeletionOnNodeDrain(test.pods, test.pdbs, true, true, true, registry, 0, time.Now())
 | 
			
		||||
 | 
			
		||||
		if test.expectFatal {
 | 
			
		||||
			assert.Equal(t, test.expectBlockingPod, blockingPod)
 | 
			
		||||
			if err == nil {
 | 
			
		||||
				t.Fatalf("%s: unexpected non-error", test.description)
 | 
			
		||||
			}
 | 
			
		||||
		}
 | 
			
		||||
 | 
			
		||||
		if !test.expectFatal {
 | 
			
		||||
			assert.Nil(t, blockingPod)
 | 
			
		||||
			if err != nil {
 | 
			
		||||
				t.Fatalf("%s: error occurred: %v", test.description, err)
 | 
			
		||||
			}
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
		Loading…
	
		Reference in New Issue