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
|
nodeUtilizationMap map[string]simulator.UtilizationInfo
|
||||||
usageTracker *simulator.UsageTracker
|
usageTracker *simulator.UsageTracker
|
||||||
nodeDeletionTracker *NodeDeletionTracker
|
nodeDeletionTracker *NodeDeletionTracker
|
||||||
|
unremovableNodeReasons map[string]*simulator.UnremovableNode
|
||||||
}
|
}
|
||||||
|
|
||||||
// NewScaleDown builds new ScaleDown object.
|
// NewScaleDown builds new ScaleDown object.
|
||||||
|
|
@ -377,12 +378,14 @@ func NewScaleDown(context *context.AutoscalingContext, clusterStateRegistry *clu
|
||||||
usageTracker: simulator.NewUsageTracker(),
|
usageTracker: simulator.NewUsageTracker(),
|
||||||
unneededNodesList: make([]*apiv1.Node, 0),
|
unneededNodesList: make([]*apiv1.Node, 0),
|
||||||
nodeDeletionTracker: NewNodeDeletionTracker(),
|
nodeDeletionTracker: NewNodeDeletionTracker(),
|
||||||
|
unremovableNodeReasons: make(map[string]*simulator.UnremovableNode),
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// CleanUp cleans up the internal ScaleDown state.
|
// CleanUp cleans up the internal ScaleDown state.
|
||||||
func (sd *ScaleDown) CleanUp(timestamp time.Time) {
|
func (sd *ScaleDown) CleanUp(timestamp time.Time) {
|
||||||
sd.usageTracker.CleanUp(timestamp.Add(-sd.context.ScaleDownUnneededTime))
|
sd.usageTracker.CleanUp(timestamp.Add(-sd.context.ScaleDownUnneededTime))
|
||||||
|
sd.clearUnremovableNodeReasons()
|
||||||
}
|
}
|
||||||
|
|
||||||
// CleanUpUnneededNodes clears the list of unneeded nodes.
|
// CleanUpUnneededNodes clears the list of unneeded nodes.
|
||||||
|
|
@ -453,6 +456,7 @@ func (sd *ScaleDown) UpdateUnneededNodes(
|
||||||
for _, node := range scaleDownCandidates {
|
for _, node := range scaleDownCandidates {
|
||||||
if unremovableTimestamp, found := sd.unremovableNodes[node.Name]; found {
|
if unremovableTimestamp, found := sd.unremovableNodes[node.Name]; found {
|
||||||
if unremovableTimestamp.After(timestamp) {
|
if unremovableTimestamp.After(timestamp) {
|
||||||
|
sd.addUnremovableNodeReason(node, simulator.RecentlyUnremovable)
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
delete(sd.unremovableNodes, node.Name)
|
delete(sd.unremovableNodes, node.Name)
|
||||||
|
|
@ -473,18 +477,21 @@ func (sd *ScaleDown) UpdateUnneededNodes(
|
||||||
// and they have not been deleted.
|
// and they have not been deleted.
|
||||||
if isNodeBeingDeleted(node, timestamp) {
|
if isNodeBeingDeleted(node, timestamp) {
|
||||||
klog.V(1).Infof("Skipping %s from delete consideration - the node is currently being deleted", node.Name)
|
klog.V(1).Infof("Skipping %s from delete consideration - the node is currently being deleted", node.Name)
|
||||||
|
sd.addUnremovableNodeReason(node, simulator.CurrentlyBeingDeleted)
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
|
|
||||||
// Skip nodes marked with no scale down annotation
|
// Skip nodes marked with no scale down annotation
|
||||||
if hasNoScaleDownAnnotation(node) {
|
if hasNoScaleDownAnnotation(node) {
|
||||||
klog.V(1).Infof("Skipping %s from delete consideration - the node is marked as no scale down", node.Name)
|
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
|
continue
|
||||||
}
|
}
|
||||||
|
|
||||||
nodeInfo, found := nodeNameToNodeInfo[node.Name]
|
nodeInfo, found := nodeNameToNodeInfo[node.Name]
|
||||||
if !found {
|
if !found {
|
||||||
klog.Errorf("Node info for %s not found", node.Name)
|
klog.Errorf("Node info for %s not found", node.Name)
|
||||||
|
sd.addUnremovableNodeReason(node, simulator.UnexpectedError)
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -497,6 +504,7 @@ func (sd *ScaleDown) UpdateUnneededNodes(
|
||||||
|
|
||||||
if !sd.isNodeBelowUtilzationThreshold(node, utilInfo) {
|
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)
|
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
|
continue
|
||||||
}
|
}
|
||||||
currentlyUnneededNodes = append(currentlyUnneededNodes, node)
|
currentlyUnneededNodes = append(currentlyUnneededNodes, node)
|
||||||
|
|
@ -595,12 +603,22 @@ func (sd *ScaleDown) UpdateUnneededNodes(
|
||||||
// Add nodes to unremovable map
|
// Add nodes to unremovable map
|
||||||
if len(unremovable) > 0 {
|
if len(unremovable) > 0 {
|
||||||
unremovableTimeout := timestamp.Add(sd.context.AutoscalingOptions.UnremovableNodeRecheckTimeout)
|
unremovableTimeout := timestamp.Add(sd.context.AutoscalingOptions.UnremovableNodeRecheckTimeout)
|
||||||
for _, node := range unremovable {
|
for _, unremovableNode := range unremovable {
|
||||||
sd.unremovableNodes[node.Name] = unremovableTimeout
|
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)
|
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
|
// Update state and metrics
|
||||||
sd.unneededNodesList = unneededNodesList
|
sd.unneededNodesList = unneededNodesList
|
||||||
sd.unneededNodes = result
|
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
|
// markSimulationError indicates a simulation error by clearing relevant scale
|
||||||
// down state and returning an appropriate error.
|
// down state and returning an appropriate error.
|
||||||
func (sd *ScaleDown) markSimulationError(simulatorErr errors.AutoscalerError,
|
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.
|
// Check if node is marked with no scale down annotation.
|
||||||
if hasNoScaleDownAnnotation(node) {
|
if hasNoScaleDownAnnotation(node) {
|
||||||
klog.V(4).Infof("Skipping %s - scale down disabled annotation found", node.Name)
|
klog.V(4).Infof("Skipping %s - scale down disabled annotation found", node.Name)
|
||||||
|
sd.addUnremovableNodeReason(node, simulator.ScaleDownDisabledAnnotation)
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -779,45 +810,53 @@ func (sd *ScaleDown) TryToScaleDown(allNodes []*apiv1.Node, pods []*apiv1.Pod, p
|
||||||
|
|
||||||
// Check how long the node was underutilized.
|
// Check how long the node was underutilized.
|
||||||
if ready && !val.Add(sd.context.ScaleDownUnneededTime).Before(currentTime) {
|
if ready && !val.Add(sd.context.ScaleDownUnneededTime).Before(currentTime) {
|
||||||
|
sd.addUnremovableNodeReason(node, simulator.NotUnneededLongEnough)
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
|
|
||||||
// Unready nodes may be deleted after a different time than underutilized nodes.
|
// Unready nodes may be deleted after a different time than underutilized nodes.
|
||||||
if !ready && !val.Add(sd.context.ScaleDownUnreadyTime).Before(currentTime) {
|
if !ready && !val.Add(sd.context.ScaleDownUnreadyTime).Before(currentTime) {
|
||||||
|
sd.addUnremovableNodeReason(node, simulator.NotUnreadyLongEnough)
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
|
|
||||||
nodeGroup, err := sd.context.CloudProvider.NodeGroupForNode(node)
|
nodeGroup, err := sd.context.CloudProvider.NodeGroupForNode(node)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
klog.Errorf("Error while checking node group for %s: %v", node.Name, err)
|
klog.Errorf("Error while checking node group for %s: %v", node.Name, err)
|
||||||
|
sd.addUnremovableNodeReason(node, simulator.UnexpectedError)
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
if nodeGroup == nil || reflect.ValueOf(nodeGroup).IsNil() {
|
if nodeGroup == nil || reflect.ValueOf(nodeGroup).IsNil() {
|
||||||
klog.V(4).Infof("Skipping %s - no node group config", node.Name)
|
klog.V(4).Infof("Skipping %s - no node group config", node.Name)
|
||||||
|
sd.addUnremovableNodeReason(node, simulator.NotAutoscaled)
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
|
|
||||||
size, found := nodeGroupSize[nodeGroup.Id()]
|
size, found := nodeGroupSize[nodeGroup.Id()]
|
||||||
if !found {
|
if !found {
|
||||||
klog.Errorf("Error while checking node group size %s: group size not found in cache", nodeGroup.Id())
|
klog.Errorf("Error while checking node group size %s: group size not found in cache", nodeGroup.Id())
|
||||||
|
sd.addUnremovableNodeReason(node, simulator.UnexpectedError)
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
|
|
||||||
deletionsInProgress := sd.nodeDeletionTracker.GetDeletionsInProgress(nodeGroup.Id())
|
deletionsInProgress := sd.nodeDeletionTracker.GetDeletionsInProgress(nodeGroup.Id())
|
||||||
if size-deletionsInProgress <= nodeGroup.MinSize() {
|
if size-deletionsInProgress <= nodeGroup.MinSize() {
|
||||||
klog.V(1).Infof("Skipping %s - node group min size reached", node.Name)
|
klog.V(1).Infof("Skipping %s - node group min size reached", node.Name)
|
||||||
|
sd.addUnremovableNodeReason(node, simulator.NodeGroupMinSizeReached)
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
|
|
||||||
scaleDownResourcesDelta, err := computeScaleDownResourcesDelta(sd.context.CloudProvider, node, nodeGroup, resourcesWithLimits)
|
scaleDownResourcesDelta, err := computeScaleDownResourcesDelta(sd.context.CloudProvider, node, nodeGroup, resourcesWithLimits)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
klog.Errorf("Error getting node resources: %v", err)
|
klog.Errorf("Error getting node resources: %v", err)
|
||||||
|
sd.addUnremovableNodeReason(node, simulator.UnexpectedError)
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
|
|
||||||
checkResult := scaleDownResourcesLeft.checkScaleDownDeltaWithinLimits(scaleDownResourcesDelta)
|
checkResult := scaleDownResourcesLeft.checkScaleDownDeltaWithinLimits(scaleDownResourcesDelta)
|
||||||
if checkResult.exceeded {
|
if checkResult.exceeded {
|
||||||
klog.V(4).Infof("Skipping %s - minimal limit exceeded for %v", node.Name, checkResult.exceededResources)
|
klog.V(4).Infof("Skipping %s - minimal limit exceeded for %v", node.Name, checkResult.exceededResources)
|
||||||
|
sd.addUnremovableNodeReason(node, simulator.MinimalResourceLimitExceeded)
|
||||||
continue
|
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.
|
// Only scheduled non expendable pods are taken into account and have to be moved.
|
||||||
nonExpendablePods := core_utils.FilterOutExpendablePods(pods, sd.context.ExpendablePodsPriorityCutoff)
|
nonExpendablePods := core_utils.FilterOutExpendablePods(pods, sd.context.ExpendablePodsPriorityCutoff)
|
||||||
// We look for only 1 node so new hints may be incomplete.
|
// We look for only 1 node so new hints may be incomplete.
|
||||||
nodesToRemove, _, _, err := simulator.FindNodesToRemove(
|
nodesToRemove, unremovable, _, err := simulator.FindNodesToRemove(
|
||||||
candidates,
|
candidates,
|
||||||
nodesWithoutMaster,
|
nodesWithoutMaster,
|
||||||
nonExpendablePods,
|
nonExpendablePods,
|
||||||
|
|
@ -872,6 +911,10 @@ func (sd *ScaleDown) TryToScaleDown(allNodes []*apiv1.Node, pods []*apiv1.Pod, p
|
||||||
pdbs)
|
pdbs)
|
||||||
findNodesToRemoveDuration = time.Now().Sub(findNodesToRemoveStart)
|
findNodesToRemoveDuration = time.Now().Sub(findNodesToRemoveStart)
|
||||||
|
|
||||||
|
for _, unremovableNode := range unremovable {
|
||||||
|
sd.addUnremovableNode(unremovableNode)
|
||||||
|
}
|
||||||
|
|
||||||
if err != nil {
|
if err != nil {
|
||||||
scaleDownStatus.Result = status.ScaleDownError
|
scaleDownStatus.Result = status.ScaleDownError
|
||||||
return scaleDownStatus, err.AddPrefix("Find node to remove failed: ")
|
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)
|
a.processors.ScaleUpStatusProcessor.Process(a.AutoscalingContext, scaleUpStatus)
|
||||||
}
|
}
|
||||||
if !scaleDownStatusProcessorAlreadyCalled && a.processors != nil && a.processors.ScaleDownStatusProcessor != nil {
|
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)
|
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 {
|
if a.processors != nil && a.processors.ScaleDownStatusProcessor != nil {
|
||||||
|
scaleDownStatus.SetUnremovableNodesInfo(scaleDown.unremovableNodeReasons, scaleDown.nodeUtilizationMap, scaleDown.context.CloudProvider)
|
||||||
a.processors.ScaleDownStatusProcessor.Process(autoscalingContext, scaleDownStatus)
|
a.processors.ScaleDownStatusProcessor.Process(autoscalingContext, scaleDownStatus)
|
||||||
scaleDownStatusProcessorAlreadyCalled = true
|
scaleDownStatusProcessorAlreadyCalled = true
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -21,16 +21,55 @@ import (
|
||||||
"k8s.io/autoscaler/cluster-autoscaler/cloudprovider"
|
"k8s.io/autoscaler/cluster-autoscaler/cloudprovider"
|
||||||
"k8s.io/autoscaler/cluster-autoscaler/context"
|
"k8s.io/autoscaler/cluster-autoscaler/context"
|
||||||
"k8s.io/autoscaler/cluster-autoscaler/simulator"
|
"k8s.io/autoscaler/cluster-autoscaler/simulator"
|
||||||
|
"k8s.io/autoscaler/cluster-autoscaler/utils/drain"
|
||||||
|
"k8s.io/klog"
|
||||||
)
|
)
|
||||||
|
|
||||||
// ScaleDownStatus represents the state of scale down.
|
// ScaleDownStatus represents the state of scale down.
|
||||||
type ScaleDownStatus struct {
|
type ScaleDownStatus struct {
|
||||||
Result ScaleDownResult
|
Result ScaleDownResult
|
||||||
ScaledDownNodes []*ScaleDownNode
|
ScaledDownNodes []*ScaleDownNode
|
||||||
|
UnremovableNodes []*UnremovableNode
|
||||||
RemovedNodeGroups []cloudprovider.NodeGroup
|
RemovedNodeGroups []cloudprovider.NodeGroup
|
||||||
NodeDeleteResults map[string]NodeDeleteResult
|
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.
|
// ScaleDownNode represents the state of a node that's being scaled down.
|
||||||
type ScaleDownNode struct {
|
type ScaleDownNode struct {
|
||||||
Node *apiv1.Node
|
Node *apiv1.Node
|
||||||
|
|
|
||||||
|
|
@ -22,6 +22,7 @@ import (
|
||||||
"math/rand"
|
"math/rand"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"k8s.io/autoscaler/cluster-autoscaler/utils/drain"
|
||||||
"k8s.io/autoscaler/cluster-autoscaler/utils/errors"
|
"k8s.io/autoscaler/cluster-autoscaler/utils/errors"
|
||||||
"k8s.io/autoscaler/cluster-autoscaler/utils/glogx"
|
"k8s.io/autoscaler/cluster-autoscaler/utils/glogx"
|
||||||
"k8s.io/autoscaler/cluster-autoscaler/utils/gpu"
|
"k8s.io/autoscaler/cluster-autoscaler/utils/gpu"
|
||||||
|
|
@ -57,6 +58,48 @@ type NodeToBeRemoved struct {
|
||||||
PodsToReschedule []*apiv1.Pod
|
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.
|
// UtilizationInfo contains utilization information for a node.
|
||||||
type UtilizationInfo struct {
|
type UtilizationInfo struct {
|
||||||
CpuUtil float64
|
CpuUtil float64
|
||||||
|
|
@ -75,11 +118,11 @@ func FindNodesToRemove(candidates []*apiv1.Node, destinationNodes []*apiv1.Node,
|
||||||
fastCheck bool, oldHints map[string]string, usageTracker *UsageTracker,
|
fastCheck bool, oldHints map[string]string, usageTracker *UsageTracker,
|
||||||
timestamp time.Time,
|
timestamp time.Time,
|
||||||
podDisruptionBudgets []*policyv1.PodDisruptionBudget,
|
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)
|
nodeNameToNodeInfo := scheduler_util.CreateNodeNameToInfoMap(pods, destinationNodes)
|
||||||
result := make([]NodeToBeRemoved, 0)
|
result := make([]NodeToBeRemoved, 0)
|
||||||
unremovable := make([]*apiv1.Node, 0)
|
unremovable := make([]*UnremovableNode, 0)
|
||||||
|
|
||||||
evaluationType := "Detailed evaluation"
|
evaluationType := "Detailed evaluation"
|
||||||
if fastCheck {
|
if fastCheck {
|
||||||
|
|
@ -92,24 +135,29 @@ candidateloop:
|
||||||
klog.V(2).Infof("%s: %s for removal", evaluationType, node.Name)
|
klog.V(2).Infof("%s: %s for removal", evaluationType, node.Name)
|
||||||
|
|
||||||
var podsToRemove []*apiv1.Pod
|
var podsToRemove []*apiv1.Pod
|
||||||
|
var blockingPod *drain.BlockingPod
|
||||||
var err error
|
var err error
|
||||||
|
|
||||||
if nodeInfo, found := nodeNameToNodeInfo[node.Name]; found {
|
if nodeInfo, found := nodeNameToNodeInfo[node.Name]; found {
|
||||||
if fastCheck {
|
if fastCheck {
|
||||||
podsToRemove, err = FastGetPodsToMove(nodeInfo, *skipNodesWithSystemPods, *skipNodesWithLocalStorage,
|
podsToRemove, blockingPod, err = FastGetPodsToMove(nodeInfo, *skipNodesWithSystemPods, *skipNodesWithLocalStorage,
|
||||||
podDisruptionBudgets)
|
podDisruptionBudgets)
|
||||||
} else {
|
} else {
|
||||||
podsToRemove, err = DetailedGetPodsForMove(nodeInfo, *skipNodesWithSystemPods, *skipNodesWithLocalStorage, listers, int32(*minReplicaCount),
|
podsToRemove, blockingPod, err = DetailedGetPodsForMove(nodeInfo, *skipNodesWithSystemPods, *skipNodesWithLocalStorage, listers, int32(*minReplicaCount),
|
||||||
podDisruptionBudgets)
|
podDisruptionBudgets)
|
||||||
}
|
}
|
||||||
if err != nil {
|
if err != nil {
|
||||||
klog.V(2).Infof("%s: node %s cannot be removed: %v", evaluationType, node.Name, err)
|
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
|
continue candidateloop
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
klog.V(2).Infof("%s: nodeInfo for %s not found", evaluationType, node.Name)
|
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
|
continue candidateloop
|
||||||
}
|
}
|
||||||
findProblems := findPlaceFor(node.Name, podsToRemove, destinationNodes, clusterSnapshot,
|
findProblems := findPlaceFor(node.Name, podsToRemove, destinationNodes, clusterSnapshot,
|
||||||
|
|
@ -126,7 +174,7 @@ candidateloop:
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
klog.V(2).Infof("%s: node %s is not suitable for removal: %v", evaluationType, node.Name, findProblems)
|
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
|
return result, unremovable, newHints, nil
|
||||||
|
|
@ -139,7 +187,7 @@ func FindEmptyNodesToRemove(candidates []*apiv1.Node, pods []*apiv1.Pod) []*apiv
|
||||||
for _, node := range candidates {
|
for _, node := range candidates {
|
||||||
if nodeInfo, found := nodeNameToNodeInfo[node.Name]; found {
|
if nodeInfo, found := nodeNameToNodeInfo[node.Name]; found {
|
||||||
// Should block on all pods.
|
// 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 {
|
if err == nil && len(podsToRemove) == 0 {
|
||||||
result = append(result, node)
|
result = append(result, node)
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -23,6 +23,7 @@ import (
|
||||||
|
|
||||||
apiv1 "k8s.io/api/core/v1"
|
apiv1 "k8s.io/api/core/v1"
|
||||||
policyv1 "k8s.io/api/policy/v1beta1"
|
policyv1 "k8s.io/api/policy/v1beta1"
|
||||||
|
"k8s.io/autoscaler/cluster-autoscaler/utils/drain"
|
||||||
. "k8s.io/autoscaler/cluster-autoscaler/utils/test"
|
. "k8s.io/autoscaler/cluster-autoscaler/utils/test"
|
||||||
"k8s.io/kubernetes/pkg/kubelet/types"
|
"k8s.io/kubernetes/pkg/kubelet/types"
|
||||||
schedulernodeinfo "k8s.io/kubernetes/pkg/scheduler/nodeinfo"
|
schedulernodeinfo "k8s.io/kubernetes/pkg/scheduler/nodeinfo"
|
||||||
|
|
@ -244,7 +245,7 @@ type findNodesToRemoveTestConfig struct {
|
||||||
candidates []*apiv1.Node
|
candidates []*apiv1.Node
|
||||||
allNodes []*apiv1.Node
|
allNodes []*apiv1.Node
|
||||||
toRemove []NodeToBeRemoved
|
toRemove []NodeToBeRemoved
|
||||||
unremovable []*apiv1.Node
|
unremovable []*UnremovableNode
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestFindNodesToRemove(t *testing.T) {
|
func TestFindNodesToRemove(t *testing.T) {
|
||||||
|
|
@ -299,7 +300,7 @@ func TestFindNodesToRemove(t *testing.T) {
|
||||||
candidates: []*apiv1.Node{emptyNode},
|
candidates: []*apiv1.Node{emptyNode},
|
||||||
allNodes: []*apiv1.Node{emptyNode},
|
allNodes: []*apiv1.Node{emptyNode},
|
||||||
toRemove: []NodeToBeRemoved{emptyNodeToRemove},
|
toRemove: []NodeToBeRemoved{emptyNodeToRemove},
|
||||||
unremovable: []*apiv1.Node{},
|
unremovable: []*UnremovableNode{},
|
||||||
},
|
},
|
||||||
// just a drainable node, but nowhere for pods to go to
|
// just a drainable node, but nowhere for pods to go to
|
||||||
{
|
{
|
||||||
|
|
@ -308,7 +309,7 @@ func TestFindNodesToRemove(t *testing.T) {
|
||||||
candidates: []*apiv1.Node{drainableNode},
|
candidates: []*apiv1.Node{drainableNode},
|
||||||
allNodes: []*apiv1.Node{drainableNode},
|
allNodes: []*apiv1.Node{drainableNode},
|
||||||
toRemove: []NodeToBeRemoved{},
|
toRemove: []NodeToBeRemoved{},
|
||||||
unremovable: []*apiv1.Node{drainableNode},
|
unremovable: []*UnremovableNode{{Node: drainableNode, Reason: NoPlaceToMovePods}},
|
||||||
},
|
},
|
||||||
// drainable node, and a mostly empty node that can take its pods
|
// 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},
|
candidates: []*apiv1.Node{drainableNode, nonDrainableNode},
|
||||||
allNodes: []*apiv1.Node{drainableNode, nonDrainableNode},
|
allNodes: []*apiv1.Node{drainableNode, nonDrainableNode},
|
||||||
toRemove: []NodeToBeRemoved{drainableNodeToRemove},
|
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
|
// drainable node, and a full node that cannot fit anymore pods
|
||||||
{
|
{
|
||||||
|
|
@ -326,7 +327,7 @@ func TestFindNodesToRemove(t *testing.T) {
|
||||||
candidates: []*apiv1.Node{drainableNode},
|
candidates: []*apiv1.Node{drainableNode},
|
||||||
allNodes: []*apiv1.Node{drainableNode, fullNode},
|
allNodes: []*apiv1.Node{drainableNode, fullNode},
|
||||||
toRemove: []NodeToBeRemoved{},
|
toRemove: []NodeToBeRemoved{},
|
||||||
unremovable: []*apiv1.Node{drainableNode},
|
unremovable: []*UnremovableNode{{Node: drainableNode, Reason: NoPlaceToMovePods}},
|
||||||
},
|
},
|
||||||
// 4 nodes, 1 empty, 1 drainable
|
// 4 nodes, 1 empty, 1 drainable
|
||||||
{
|
{
|
||||||
|
|
@ -335,7 +336,7 @@ func TestFindNodesToRemove(t *testing.T) {
|
||||||
candidates: []*apiv1.Node{emptyNode, drainableNode},
|
candidates: []*apiv1.Node{emptyNode, drainableNode},
|
||||||
allNodes: []*apiv1.Node{emptyNode, drainableNode, fullNode, nonDrainableNode},
|
allNodes: []*apiv1.Node{emptyNode, drainableNode, fullNode, nonDrainableNode},
|
||||||
toRemove: []NodeToBeRemoved{emptyNodeToRemove, drainableNodeToRemove},
|
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
|
// along with their pods (no abandoned pods with dangling created-by annotation). Useful for fast
|
||||||
// checks.
|
// checks.
|
||||||
func FastGetPodsToMove(nodeInfo *schedulernodeinfo.NodeInfo, skipNodesWithSystemPods bool, skipNodesWithLocalStorage bool,
|
func FastGetPodsToMove(nodeInfo *schedulernodeinfo.NodeInfo, skipNodesWithSystemPods bool, skipNodesWithLocalStorage bool,
|
||||||
pdbs []*policyv1.PodDisruptionBudget) ([]*apiv1.Pod, error) {
|
pdbs []*policyv1.PodDisruptionBudget) ([]*apiv1.Pod, *drain.BlockingPod, error) {
|
||||||
pods, err := drain.GetPodsForDeletionOnNodeDrain(
|
pods, blockingPod, err := drain.GetPodsForDeletionOnNodeDrain(
|
||||||
nodeInfo.Pods(),
|
nodeInfo.Pods(),
|
||||||
pdbs,
|
pdbs,
|
||||||
skipNodesWithSystemPods,
|
skipNodesWithSystemPods,
|
||||||
|
|
@ -47,13 +47,13 @@ func FastGetPodsToMove(nodeInfo *schedulernodeinfo.NodeInfo, skipNodesWithSystem
|
||||||
time.Now())
|
time.Now())
|
||||||
|
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return pods, err
|
return pods, blockingPod, err
|
||||||
}
|
}
|
||||||
if err := checkPdbs(pods, pdbs); err != nil {
|
if pdbBlockingPod, err := checkPdbs(pods, pdbs); err != nil {
|
||||||
return []*apiv1.Pod{}, err
|
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
|
// 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.
|
// still exist.
|
||||||
func DetailedGetPodsForMove(nodeInfo *schedulernodeinfo.NodeInfo, skipNodesWithSystemPods bool,
|
func DetailedGetPodsForMove(nodeInfo *schedulernodeinfo.NodeInfo, skipNodesWithSystemPods bool,
|
||||||
skipNodesWithLocalStorage bool, listers kube_util.ListerRegistry, minReplicaCount int32,
|
skipNodesWithLocalStorage bool, listers kube_util.ListerRegistry, minReplicaCount int32,
|
||||||
pdbs []*policyv1.PodDisruptionBudget) ([]*apiv1.Pod, error) {
|
pdbs []*policyv1.PodDisruptionBudget) ([]*apiv1.Pod, *drain.BlockingPod, error) {
|
||||||
pods, err := drain.GetPodsForDeletionOnNodeDrain(
|
pods, blockingPod, err := drain.GetPodsForDeletionOnNodeDrain(
|
||||||
nodeInfo.Pods(),
|
nodeInfo.Pods(),
|
||||||
pdbs,
|
pdbs,
|
||||||
skipNodesWithSystemPods,
|
skipNodesWithSystemPods,
|
||||||
|
|
@ -73,29 +73,29 @@ func DetailedGetPodsForMove(nodeInfo *schedulernodeinfo.NodeInfo, skipNodesWithS
|
||||||
minReplicaCount,
|
minReplicaCount,
|
||||||
time.Now())
|
time.Now())
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return pods, err
|
return pods, blockingPod, err
|
||||||
}
|
}
|
||||||
if err := checkPdbs(pods, pdbs); err != nil {
|
if pdbBlockingPod, err := checkPdbs(pods, pdbs); err != nil {
|
||||||
return []*apiv1.Pod{}, err
|
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.
|
// TODO: make it more efficient.
|
||||||
for _, pdb := range pdbs {
|
for _, pdb := range pdbs {
|
||||||
selector, err := metav1.LabelSelectorAsSelector(pdb.Spec.Selector)
|
selector, err := metav1.LabelSelectorAsSelector(pdb.Spec.Selector)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return nil, err
|
||||||
}
|
}
|
||||||
for _, pod := range pods {
|
for _, pod := range pods {
|
||||||
if pod.Namespace == pdb.Namespace && selector.Matches(labels.Set(pod.Labels)) {
|
if pod.Namespace == pdb.Namespace && selector.Matches(labels.Set(pod.Labels)) {
|
||||||
if pdb.Status.DisruptionsAllowed < 1 {
|
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"
|
policyv1 "k8s.io/api/policy/v1beta1"
|
||||||
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
||||||
"k8s.io/apimachinery/pkg/util/intstr"
|
"k8s.io/apimachinery/pkg/util/intstr"
|
||||||
|
"k8s.io/autoscaler/cluster-autoscaler/utils/drain"
|
||||||
. "k8s.io/autoscaler/cluster-autoscaler/utils/test"
|
. "k8s.io/autoscaler/cluster-autoscaler/utils/test"
|
||||||
"k8s.io/kubernetes/pkg/kubelet/types"
|
"k8s.io/kubernetes/pkg/kubelet/types"
|
||||||
schedulernodeinfo "k8s.io/kubernetes/pkg/scheduler/nodeinfo"
|
schedulernodeinfo "k8s.io/kubernetes/pkg/scheduler/nodeinfo"
|
||||||
|
|
@ -39,8 +40,9 @@ func TestFastGetPodsToMove(t *testing.T) {
|
||||||
Namespace: "ns",
|
Namespace: "ns",
|
||||||
},
|
},
|
||||||
}
|
}
|
||||||
_, err := FastGetPodsToMove(schedulernodeinfo.NewNodeInfo(pod1), true, true, nil)
|
_, blockingPod, err := FastGetPodsToMove(schedulernodeinfo.NewNodeInfo(pod1), true, true, nil)
|
||||||
assert.Error(t, err)
|
assert.Error(t, err)
|
||||||
|
assert.Equal(t, &drain.BlockingPod{Pod: pod1, Reason: drain.NotReplicated}, blockingPod)
|
||||||
|
|
||||||
// Replicated pod
|
// Replicated pod
|
||||||
pod2 := &apiv1.Pod{
|
pod2 := &apiv1.Pod{
|
||||||
|
|
@ -50,8 +52,9 @@ func TestFastGetPodsToMove(t *testing.T) {
|
||||||
OwnerReferences: GenerateOwnerReferences("rs", "ReplicaSet", "extensions/v1beta1", ""),
|
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.NoError(t, err)
|
||||||
|
assert.Nil(t, blockingPod)
|
||||||
assert.Equal(t, 1, len(r2))
|
assert.Equal(t, 1, len(r2))
|
||||||
assert.Equal(t, pod2, r2[0])
|
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.NoError(t, err)
|
||||||
|
assert.Nil(t, blockingPod)
|
||||||
assert.Equal(t, 0, len(r3))
|
assert.Equal(t, 0, len(r3))
|
||||||
|
|
||||||
// DaemonSet pod
|
// DaemonSet pod
|
||||||
|
|
@ -77,8 +81,9 @@ func TestFastGetPodsToMove(t *testing.T) {
|
||||||
OwnerReferences: GenerateOwnerReferences("ds", "DaemonSet", "extensions/v1beta1", ""),
|
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.NoError(t, err)
|
||||||
|
assert.Nil(t, blockingPod)
|
||||||
assert.Equal(t, 1, len(r4))
|
assert.Equal(t, 1, len(r4))
|
||||||
assert.Equal(t, pod2, r4[0])
|
assert.Equal(t, pod2, r4[0])
|
||||||
|
|
||||||
|
|
@ -90,8 +95,9 @@ func TestFastGetPodsToMove(t *testing.T) {
|
||||||
OwnerReferences: GenerateOwnerReferences("rs", "ReplicaSet", "extensions/v1beta1", ""),
|
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.Error(t, err)
|
||||||
|
assert.Equal(t, &drain.BlockingPod{Pod: pod5, Reason: drain.UnmovableKubeSystemPod}, blockingPod)
|
||||||
|
|
||||||
// Local storage
|
// Local storage
|
||||||
pod6 := &apiv1.Pod{
|
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.Error(t, err)
|
||||||
|
assert.Equal(t, &drain.BlockingPod{Pod: pod6, Reason: drain.LocalStorageRequested}, blockingPod)
|
||||||
|
|
||||||
// Non-local storage
|
// Non-local storage
|
||||||
pod7 := &apiv1.Pod{
|
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.NoError(t, err)
|
||||||
|
assert.Nil(t, blockingPod)
|
||||||
assert.Equal(t, 1, len(r7))
|
assert.Equal(t, 1, len(r7))
|
||||||
|
|
||||||
// Pdb blocking
|
// 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.Error(t, err)
|
||||||
|
assert.Equal(t, &drain.BlockingPod{Pod: pod8, Reason: drain.NotEnoughPdb}, blockingPod)
|
||||||
|
|
||||||
// Pdb allowing
|
// Pdb allowing
|
||||||
pod9 := &apiv1.Pod{
|
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.NoError(t, err)
|
||||||
|
assert.Nil(t, blockingPod)
|
||||||
assert.Equal(t, 1, len(r9))
|
assert.Equal(t, 1, len(r9))
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -25,7 +25,6 @@ import (
|
||||||
apierrors "k8s.io/apimachinery/pkg/api/errors"
|
apierrors "k8s.io/apimachinery/pkg/api/errors"
|
||||||
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
|
||||||
"k8s.io/apimachinery/pkg/labels"
|
"k8s.io/apimachinery/pkg/labels"
|
||||||
|
|
||||||
kube_util "k8s.io/autoscaler/cluster-autoscaler/utils/kubernetes"
|
kube_util "k8s.io/autoscaler/cluster-autoscaler/utils/kubernetes"
|
||||||
pod_util "k8s.io/autoscaler/cluster-autoscaler/utils/pod"
|
pod_util "k8s.io/autoscaler/cluster-autoscaler/utils/pod"
|
||||||
)
|
)
|
||||||
|
|
@ -41,6 +40,37 @@ const (
|
||||||
PodSafeToEvictKey = "cluster-autoscaler.kubernetes.io/safe-to-evict"
|
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
|
// GetPodsForDeletionOnNodeDrain returns pods that should be deleted on node drain as well as some extra information
|
||||||
// about possibly problematic pods (unreplicated and daemonsets).
|
// about possibly problematic pods (unreplicated and daemonsets).
|
||||||
func GetPodsForDeletionOnNodeDrain(
|
func GetPodsForDeletionOnNodeDrain(
|
||||||
|
|
@ -51,7 +81,7 @@ func GetPodsForDeletionOnNodeDrain(
|
||||||
checkReferences bool, // Setting this to true requires client to be not-null.
|
checkReferences bool, // Setting this to true requires client to be not-null.
|
||||||
listers kube_util.ListerRegistry,
|
listers kube_util.ListerRegistry,
|
||||||
minReplica int32,
|
minReplica int32,
|
||||||
currentTime time.Time) ([]*apiv1.Pod, error) {
|
currentTime time.Time) ([]*apiv1.Pod, *BlockingPod, error) {
|
||||||
|
|
||||||
pods := []*apiv1.Pod{}
|
pods := []*apiv1.Pod{}
|
||||||
// filter kube-system PDBs to avoid doing it for every kube-system 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.
|
// TODO: replace the minReplica check with pod disruption budget.
|
||||||
if err == nil && rc != nil {
|
if err == nil && rc != nil {
|
||||||
if rc.Spec.Replicas != nil && *rc.Spec.Replicas < minReplica {
|
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)
|
pod.Namespace, pod.Name, rc.Spec.Replicas, minReplica)
|
||||||
}
|
}
|
||||||
replicated = true
|
replicated = true
|
||||||
} else {
|
} 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 {
|
} else {
|
||||||
replicated = true
|
replicated = true
|
||||||
|
|
@ -115,9 +145,9 @@ func GetPodsForDeletionOnNodeDrain(
|
||||||
if checkReferences && refKind == "DaemonSet" {
|
if checkReferences && refKind == "DaemonSet" {
|
||||||
_, err := listers.DaemonSetLister().DaemonSets(controllerNamespace).Get(controllerRef.Name)
|
_, err := listers.DaemonSetLister().DaemonSets(controllerNamespace).Get(controllerRef.Name)
|
||||||
if apierrors.IsNotFound(err) {
|
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 {
|
} 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" {
|
} else if refKind == "Job" {
|
||||||
|
|
@ -130,7 +160,7 @@ func GetPodsForDeletionOnNodeDrain(
|
||||||
if err == nil && job != nil {
|
if err == nil && job != nil {
|
||||||
replicated = true
|
replicated = true
|
||||||
} else {
|
} 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 {
|
} else {
|
||||||
replicated = true
|
replicated = true
|
||||||
|
|
@ -144,12 +174,12 @@ func GetPodsForDeletionOnNodeDrain(
|
||||||
// sophisticated than this
|
// sophisticated than this
|
||||||
if err == nil && rs != nil {
|
if err == nil && rs != nil {
|
||||||
if rs.Spec.Replicas != nil && *rs.Spec.Replicas < minReplica {
|
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)
|
pod.Namespace, pod.Name, rs.Spec.Replicas, minReplica)
|
||||||
}
|
}
|
||||||
replicated = true
|
replicated = true
|
||||||
} else {
|
} 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 {
|
} else {
|
||||||
replicated = true
|
replicated = true
|
||||||
|
|
@ -164,7 +194,7 @@ func GetPodsForDeletionOnNodeDrain(
|
||||||
if err == nil && ss != nil {
|
if err == nil && ss != nil {
|
||||||
replicated = true
|
replicated = true
|
||||||
} else {
|
} 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 {
|
} else {
|
||||||
replicated = true
|
replicated = true
|
||||||
|
|
@ -176,27 +206,27 @@ func GetPodsForDeletionOnNodeDrain(
|
||||||
|
|
||||||
if !safeToEvict && !terminal {
|
if !safeToEvict && !terminal {
|
||||||
if !replicated {
|
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 {
|
if pod.Namespace == "kube-system" && skipNodesWithSystemPods {
|
||||||
hasPDB, err := checkKubeSystemPDBs(pod, kubeSystemPDBs)
|
hasPDB, err := checkKubeSystemPDBs(pod, kubeSystemPDBs)
|
||||||
if err != nil {
|
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 {
|
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 {
|
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) {
|
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)
|
pods = append(pods, pod)
|
||||||
}
|
}
|
||||||
return pods, nil
|
return pods, nil, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// ControllerRef returns the OwnerReference to pod's controller.
|
// ControllerRef returns the OwnerReference to pod's controller.
|
||||||
|
|
|
||||||
|
|
@ -196,6 +196,7 @@ func TestDrain(t *testing.T) {
|
||||||
ObjectMeta: metav1.ObjectMeta{
|
ObjectMeta: metav1.ObjectMeta{
|
||||||
Name: "bar",
|
Name: "bar",
|
||||||
Namespace: "default",
|
Namespace: "default",
|
||||||
|
OwnerReferences: GenerateOwnerReferences(rc.Name, "ReplicationController", "core/v1", ""),
|
||||||
},
|
},
|
||||||
Spec: apiv1.PodSpec{
|
Spec: apiv1.PodSpec{
|
||||||
NodeName: "node",
|
NodeName: "node",
|
||||||
|
|
@ -369,6 +370,7 @@ func TestDrain(t *testing.T) {
|
||||||
replicaSets []*appsv1.ReplicaSet
|
replicaSets []*appsv1.ReplicaSet
|
||||||
expectFatal bool
|
expectFatal bool
|
||||||
expectPods []*apiv1.Pod
|
expectPods []*apiv1.Pod
|
||||||
|
expectBlockingPod *BlockingPod
|
||||||
}{
|
}{
|
||||||
{
|
{
|
||||||
description: "RC-managed pod",
|
description: "RC-managed pod",
|
||||||
|
|
@ -430,13 +432,16 @@ func TestDrain(t *testing.T) {
|
||||||
pdbs: []*policyv1.PodDisruptionBudget{},
|
pdbs: []*policyv1.PodDisruptionBudget{},
|
||||||
expectFatal: true,
|
expectFatal: true,
|
||||||
expectPods: []*apiv1.Pod{},
|
expectPods: []*apiv1.Pod{},
|
||||||
|
expectBlockingPod: &BlockingPod{Pod: nakedPod, Reason: NotReplicated},
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
description: "pod with EmptyDir",
|
description: "pod with EmptyDir",
|
||||||
pods: []*apiv1.Pod{emptydirPod},
|
pods: []*apiv1.Pod{emptydirPod},
|
||||||
pdbs: []*policyv1.PodDisruptionBudget{},
|
pdbs: []*policyv1.PodDisruptionBudget{},
|
||||||
|
rcs: []*apiv1.ReplicationController{&rc},
|
||||||
expectFatal: true,
|
expectFatal: true,
|
||||||
expectPods: []*apiv1.Pod{},
|
expectPods: []*apiv1.Pod{},
|
||||||
|
expectBlockingPod: &BlockingPod{Pod: emptydirPod, Reason: LocalStorageRequested},
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
description: "failed pod",
|
description: "failed pod",
|
||||||
|
|
@ -487,6 +492,7 @@ func TestDrain(t *testing.T) {
|
||||||
pdbs: []*policyv1.PodDisruptionBudget{},
|
pdbs: []*policyv1.PodDisruptionBudget{},
|
||||||
expectFatal: true,
|
expectFatal: true,
|
||||||
expectPods: []*apiv1.Pod{},
|
expectPods: []*apiv1.Pod{},
|
||||||
|
expectBlockingPod: &BlockingPod{Pod: unsafeRcPod, Reason: NotSafeToEvictAnnotation},
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
description: "Job-managed pod with PodSafeToEvict=false annotation",
|
description: "Job-managed pod with PodSafeToEvict=false annotation",
|
||||||
|
|
@ -495,6 +501,7 @@ func TestDrain(t *testing.T) {
|
||||||
rcs: []*apiv1.ReplicationController{&rc},
|
rcs: []*apiv1.ReplicationController{&rc},
|
||||||
expectFatal: true,
|
expectFatal: true,
|
||||||
expectPods: []*apiv1.Pod{},
|
expectPods: []*apiv1.Pod{},
|
||||||
|
expectBlockingPod: &BlockingPod{Pod: unsafeJobPod, Reason: NotSafeToEvictAnnotation},
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
description: "empty PDB with RC-managed pod",
|
description: "empty PDB with RC-managed pod",
|
||||||
|
|
@ -519,6 +526,7 @@ func TestDrain(t *testing.T) {
|
||||||
rcs: []*apiv1.ReplicationController{&kubeSystemRc},
|
rcs: []*apiv1.ReplicationController{&kubeSystemRc},
|
||||||
expectFatal: true,
|
expectFatal: true,
|
||||||
expectPods: []*apiv1.Pod{},
|
expectPods: []*apiv1.Pod{},
|
||||||
|
expectBlockingPod: &BlockingPod{Pod: kubeSystemRcPod, Reason: UnmovableKubeSystemPod},
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
description: "kube-system PDB with default namespace pod",
|
description: "kube-system PDB with default namespace pod",
|
||||||
|
|
@ -535,6 +543,7 @@ func TestDrain(t *testing.T) {
|
||||||
rcs: []*apiv1.ReplicationController{&kubeSystemRc},
|
rcs: []*apiv1.ReplicationController{&kubeSystemRc},
|
||||||
expectFatal: true,
|
expectFatal: true,
|
||||||
expectPods: []*apiv1.Pod{},
|
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)
|
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 {
|
if test.expectFatal {
|
||||||
|
assert.Equal(t, test.expectBlockingPod, blockingPod)
|
||||||
if err == nil {
|
if err == nil {
|
||||||
t.Fatalf("%s: unexpected non-error", test.description)
|
t.Fatalf("%s: unexpected non-error", test.description)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if !test.expectFatal {
|
if !test.expectFatal {
|
||||||
|
assert.Nil(t, blockingPod)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("%s: error occurred: %v", test.description, err)
|
t.Fatalf("%s: error occurred: %v", test.description, err)
|
||||||
}
|
}
|
||||||
|
|
|
||||||
Loading…
Reference in New Issue