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