Provide ScaleDownStatusProcessor with info about unremovable nodes
This commit is contained in:
parent
66f6328433
commit
7a188ab50d
|
|
@ -363,6 +363,7 @@ type ScaleDown struct {
|
|||
nodeUtilizationMap map[string]simulator.UtilizationInfo
|
||||
usageTracker *simulator.UsageTracker
|
||||
nodeDeletionTracker *NodeDeletionTracker
|
||||
unremovableNodeReasons map[string]*simulator.UnremovableNode
|
||||
}
|
||||
|
||||
// NewScaleDown builds new ScaleDown object.
|
||||
|
|
@ -377,12 +378,14 @@ func NewScaleDown(context *context.AutoscalingContext, clusterStateRegistry *clu
|
|||
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.
|
||||
|
|
|
|||
|
|
@ -196,6 +196,7 @@ func TestDrain(t *testing.T) {
|
|||
ObjectMeta: metav1.ObjectMeta{
|
||||
Name: "bar",
|
||||
Namespace: "default",
|
||||
OwnerReferences: GenerateOwnerReferences(rc.Name, "ReplicationController", "core/v1", ""),
|
||||
},
|
||||
Spec: apiv1.PodSpec{
|
||||
NodeName: "node",
|
||||
|
|
@ -369,6 +370,7 @@ func TestDrain(t *testing.T) {
|
|||
replicaSets []*appsv1.ReplicaSet
|
||||
expectFatal bool
|
||||
expectPods []*apiv1.Pod
|
||||
expectBlockingPod *BlockingPod
|
||||
}{
|
||||
{
|
||||
description: "RC-managed pod",
|
||||
|
|
@ -430,13 +432,16 @@ func TestDrain(t *testing.T) {
|
|||
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{},
|
||||
rcs: []*apiv1.ReplicationController{&rc},
|
||||
expectFatal: true,
|
||||
expectPods: []*apiv1.Pod{},
|
||||
expectBlockingPod: &BlockingPod{Pod: emptydirPod, Reason: LocalStorageRequested},
|
||||
},
|
||||
{
|
||||
description: "failed pod",
|
||||
|
|
@ -487,6 +492,7 @@ func TestDrain(t *testing.T) {
|
|||
pdbs: []*policyv1.PodDisruptionBudget{},
|
||||
expectFatal: true,
|
||||
expectPods: []*apiv1.Pod{},
|
||||
expectBlockingPod: &BlockingPod{Pod: unsafeRcPod, Reason: NotSafeToEvictAnnotation},
|
||||
},
|
||||
{
|
||||
description: "Job-managed pod with PodSafeToEvict=false annotation",
|
||||
|
|
@ -495,6 +501,7 @@ func TestDrain(t *testing.T) {
|
|||
rcs: []*apiv1.ReplicationController{&rc},
|
||||
expectFatal: true,
|
||||
expectPods: []*apiv1.Pod{},
|
||||
expectBlockingPod: &BlockingPod{Pod: unsafeJobPod, Reason: NotSafeToEvictAnnotation},
|
||||
},
|
||||
{
|
||||
description: "empty PDB with RC-managed pod",
|
||||
|
|
@ -519,6 +526,7 @@ func TestDrain(t *testing.T) {
|
|||
rcs: []*apiv1.ReplicationController{&kubeSystemRc},
|
||||
expectFatal: true,
|
||||
expectPods: []*apiv1.Pod{},
|
||||
expectBlockingPod: &BlockingPod{Pod: kubeSystemRcPod, Reason: UnmovableKubeSystemPod},
|
||||
},
|
||||
{
|
||||
description: "kube-system PDB with default namespace pod",
|
||||
|
|
@ -535,6 +543,7 @@ func TestDrain(t *testing.T) {
|
|||
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