Provide ScaleDownStatusProcessor with info about unremovable nodes

This commit is contained in:
Jakub Tużnik 2020-02-06 17:40:43 +01:00
parent 66f6328433
commit 7a188ab50d
9 changed files with 304 additions and 120 deletions

View File

@ -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: ")

View File

@ -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
}

View File

@ -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

View File

@ -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)
}

View File

@ -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{},
},
}

View File

@ -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
}

View File

@ -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))
}

View File

@ -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.

View File

@ -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)
}