Per NodeGroup config for scale-down options
This is the implementation of https://github.com/kubernetes/autoscaler/issues/3583#issuecomment-743215343.
This commit is contained in:
parent
08d18a7bd0
commit
3e42b26a22
|
|
@ -33,7 +33,17 @@ type GpuLimits struct {
|
|||
// NodeGroupAutoscalingOptions contain various options to customize how autoscaling of
|
||||
// a given NodeGroup works. Different options can be used for each NodeGroup.
|
||||
type NodeGroupAutoscalingOptions struct {
|
||||
// ScaleDownUtilizationThreshold sets threshold for nodes to be considered for scale down if cpu or memory utilization is over threshold.
|
||||
// Well-utilized nodes are not touched.
|
||||
ScaleDownUtilizationThreshold float64
|
||||
// ScaleDownGpuUtilizationThreshold sets threshold for gpu nodes to be considered for scale down if gpu utilization is over threshold.
|
||||
// Well-utilized nodes are not touched.
|
||||
ScaleDownGpuUtilizationThreshold float64
|
||||
// ScaleDownUnneededTime sets the duration CA expects a node to be unneeded/eligible for removal
|
||||
// before scaling down the node.
|
||||
ScaleDownUnneededTime time.Duration
|
||||
// ScaleDownUnreadyTime represents how long an unready node should be unneeded before it is eligible for scale down
|
||||
ScaleDownUnreadyTime time.Duration
|
||||
}
|
||||
|
||||
// AutoscalingOptions contain various options to customize how autoscaling works
|
||||
|
|
@ -43,17 +53,6 @@ type AutoscalingOptions struct {
|
|||
NodeGroupAutoscalingOptions
|
||||
// MaxEmptyBulkDelete is a number of empty nodes that can be removed at the same time.
|
||||
MaxEmptyBulkDelete int
|
||||
// ScaleDownUtilizationThreshold sets threshold for nodes to be considered for scale down if cpu or memory utilization is over threshold.
|
||||
// Well-utilized nodes are not touched.
|
||||
ScaleDownUtilizationThreshold float64
|
||||
// ScaleDownGpuUtilizationThreshold sets threshold for gpu nodes to be considered for scale down if gpu utilization is over threshold.
|
||||
// Well-utilized nodes are not touched.
|
||||
ScaleDownGpuUtilizationThreshold float64
|
||||
// ScaleDownUnneededTime sets the duration CA expects a node to be unneeded/eligible for removal
|
||||
// before scaling down the node.
|
||||
// ScaleDownUnneededTime time.Duration
|
||||
// ScaleDownUnreadyTime represents how long an unready node should be unneeded before it is eligible for scale down
|
||||
ScaleDownUnreadyTime time.Duration
|
||||
// MaxNodesTotal sets the maximum number of nodes in the whole cluster
|
||||
MaxNodesTotal int
|
||||
// MaxCoresTotal sets the maximum number of cores in the whole cluster
|
||||
|
|
|
|||
|
|
@ -30,6 +30,7 @@ import (
|
|||
"k8s.io/autoscaler/cluster-autoscaler/context"
|
||||
core_utils "k8s.io/autoscaler/cluster-autoscaler/core/utils"
|
||||
"k8s.io/autoscaler/cluster-autoscaler/metrics"
|
||||
"k8s.io/autoscaler/cluster-autoscaler/processors"
|
||||
"k8s.io/autoscaler/cluster-autoscaler/simulator"
|
||||
"k8s.io/autoscaler/cluster-autoscaler/utils"
|
||||
"k8s.io/autoscaler/cluster-autoscaler/utils/deletetaint"
|
||||
|
|
@ -359,6 +360,7 @@ func (limits *scaleDownResourcesLimits) tryDecrementLimitsByDelta(delta scaleDow
|
|||
// ScaleDown is responsible for maintaining the state needed to perform unneeded node removals.
|
||||
type ScaleDown struct {
|
||||
context *context.AutoscalingContext
|
||||
processors *processors.AutoscalingProcessors
|
||||
clusterStateRegistry *clusterstate.ClusterStateRegistry
|
||||
unneededNodes map[string]time.Time
|
||||
unneededNodesList []*apiv1.Node
|
||||
|
|
@ -371,9 +373,10 @@ type ScaleDown struct {
|
|||
}
|
||||
|
||||
// NewScaleDown builds new ScaleDown object.
|
||||
func NewScaleDown(context *context.AutoscalingContext, clusterStateRegistry *clusterstate.ClusterStateRegistry) *ScaleDown {
|
||||
func NewScaleDown(context *context.AutoscalingContext, processors *processors.AutoscalingProcessors, clusterStateRegistry *clusterstate.ClusterStateRegistry) *ScaleDown {
|
||||
return &ScaleDown{
|
||||
context: context,
|
||||
processors: processors,
|
||||
clusterStateRegistry: clusterStateRegistry,
|
||||
unneededNodes: make(map[string]time.Time),
|
||||
unremovableNodes: make(map[string]time.Time),
|
||||
|
|
@ -388,6 +391,8 @@ func NewScaleDown(context *context.AutoscalingContext, clusterStateRegistry *clu
|
|||
|
||||
// CleanUp cleans up the internal ScaleDown state.
|
||||
func (sd *ScaleDown) CleanUp(timestamp time.Time) {
|
||||
// Use default ScaleDownUnneededTime as in this context the value
|
||||
// doesn't apply to any specific NodeGroup.
|
||||
sd.usageTracker.CleanUp(timestamp.Add(-sd.context.ScaleDownUnneededTime))
|
||||
sd.clearUnremovableNodeReasons()
|
||||
}
|
||||
|
|
@ -423,7 +428,23 @@ func (sd *ScaleDown) checkNodeUtilization(timestamp time.Time, node *apiv1.Node,
|
|||
klog.Warningf("Failed to calculate utilization for %s: %v", node.Name, err)
|
||||
}
|
||||
|
||||
if !sd.isNodeBelowUtilizationThreshold(node, utilInfo) {
|
||||
nodeGroup, err := sd.context.CloudProvider.NodeGroupForNode(node)
|
||||
if err != nil {
|
||||
return simulator.UnexpectedError, nil
|
||||
}
|
||||
if nodeGroup == nil || reflect.ValueOf(nodeGroup).IsNil() {
|
||||
// We should never get here as non-autoscaled nodes should not be included in scaleDownCandidates list
|
||||
// (and the default PreFilteringScaleDownNodeProcessor would indeed filter them out).
|
||||
klog.V(4).Infof("Skipped %s from delete considered - the node is not autoscaled", node.Name)
|
||||
return simulator.NotAutoscaled, nil
|
||||
}
|
||||
|
||||
underutilized, err := sd.isNodeBelowUtilizationThreshold(node, nodeGroup, utilInfo)
|
||||
if err != nil {
|
||||
klog.Warningf("Failed to check utilization thresholds for %s: %v", node.Name, err)
|
||||
return simulator.UnexpectedError, nil
|
||||
}
|
||||
if !underutilized {
|
||||
klog.V(4).Infof("Node %s is not suitable for removal - %s utilization too big (%f)", node.Name, utilInfo.ResourceName, utilInfo.Utilization)
|
||||
return simulator.NotUnderutilized, &utilInfo
|
||||
}
|
||||
|
|
@ -616,17 +637,24 @@ func (sd *ScaleDown) UpdateUnneededNodes(
|
|||
}
|
||||
|
||||
// isNodeBelowUtilizationThreshold determines if a given node utilization is below threshold.
|
||||
func (sd *ScaleDown) isNodeBelowUtilizationThreshold(node *apiv1.Node, utilInfo simulator.UtilizationInfo) bool {
|
||||
func (sd *ScaleDown) isNodeBelowUtilizationThreshold(node *apiv1.Node, nodeGroup cloudprovider.NodeGroup, utilInfo simulator.UtilizationInfo) (bool, error) {
|
||||
var threshold float64
|
||||
var err error
|
||||
if gpu.NodeHasGpu(sd.context.CloudProvider.GPULabel(), node) {
|
||||
if utilInfo.Utilization >= sd.context.ScaleDownGpuUtilizationThreshold {
|
||||
return false
|
||||
threshold, err = sd.processors.NodeGroupConfigProcessor.GetScaleDownGpuUtilizationThreshold(sd.context, nodeGroup)
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
} else {
|
||||
if utilInfo.Utilization >= sd.context.ScaleDownUtilizationThreshold {
|
||||
return false
|
||||
threshold, err = sd.processors.NodeGroupConfigProcessor.GetScaleDownUtilizationThreshold(sd.context, nodeGroup)
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
}
|
||||
return true
|
||||
if utilInfo.Utilization >= threshold {
|
||||
return false, nil
|
||||
}
|
||||
return true, nil
|
||||
}
|
||||
|
||||
// updateUnremovableNodes updates unremovableNodes map according to current
|
||||
|
|
@ -812,18 +840,6 @@ func (sd *ScaleDown) TryToScaleDown(
|
|||
ready, _, _ := kube_util.GetReadinessState(node)
|
||||
readinessMap[node.Name] = ready
|
||||
|
||||
// Check how long a ready node was underutilized.
|
||||
if ready && !unneededSince.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 && !unneededSince.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)
|
||||
|
|
@ -836,6 +852,30 @@ func (sd *ScaleDown) TryToScaleDown(
|
|||
continue
|
||||
}
|
||||
|
||||
if ready {
|
||||
// Check how long a ready node was underutilized.
|
||||
unneededTime, err := sd.processors.NodeGroupConfigProcessor.GetScaleDownUnneededTime(sd.context, nodeGroup)
|
||||
if err != nil {
|
||||
klog.Errorf("Error trying to get ScaleDownUnneededTime for node %s (in group: %s)", node.Name, nodeGroup.Id())
|
||||
continue
|
||||
}
|
||||
if !unneededSince.Add(unneededTime).Before(currentTime) {
|
||||
sd.addUnremovableNodeReason(node, simulator.NotUnneededLongEnough)
|
||||
continue
|
||||
}
|
||||
} else {
|
||||
// Unready nodes may be deleted after a different time than underutilized nodes.
|
||||
unreadyTime, err := sd.processors.NodeGroupConfigProcessor.GetScaleDownUnreadyTime(sd.context, nodeGroup)
|
||||
if err != nil {
|
||||
klog.Errorf("Error trying to get ScaleDownUnnreadyTime for node %s (in group: %s)", node.Name, nodeGroup.Id())
|
||||
continue
|
||||
}
|
||||
if !unneededSince.Add(unreadyTime).Before(currentTime) {
|
||||
sd.addUnremovableNodeReason(node, simulator.NotUnreadyLongEnough)
|
||||
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())
|
||||
|
|
|
|||
|
|
@ -130,14 +130,16 @@ func TestFindUnneededNodes(t *testing.T) {
|
|||
provider.AddNode("ng1", n9)
|
||||
|
||||
options := config.AutoscalingOptions{
|
||||
ScaleDownUtilizationThreshold: 0.35,
|
||||
NodeGroupAutoscalingOptions: config.NodeGroupAutoscalingOptions{
|
||||
ScaleDownUtilizationThreshold: 0.35,
|
||||
},
|
||||
UnremovableNodeRecheckTimeout: 5 * time.Minute,
|
||||
}
|
||||
context, err := NewScaleTestAutoscalingContext(options, &fake.Clientset{}, nil, provider, nil)
|
||||
assert.NoError(t, err)
|
||||
|
||||
clusterStateRegistry := clusterstate.NewClusterStateRegistry(provider, clusterstate.ClusterStateRegistryConfig{}, context.LogRecorder, newBackoff())
|
||||
sd := NewScaleDown(&context, clusterStateRegistry)
|
||||
sd := NewScaleDown(&context, NewTestProcessors(), clusterStateRegistry)
|
||||
allNodes := []*apiv1.Node{n1, n2, n3, n4, n5, n7, n8, n9}
|
||||
|
||||
simulator.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, allNodes, []*apiv1.Pod{p1, p2, p3, p4, p5, p6})
|
||||
|
|
@ -243,15 +245,17 @@ func TestFindUnneededGPUNodes(t *testing.T) {
|
|||
provider.AddNode("ng1", n3)
|
||||
|
||||
options := config.AutoscalingOptions{
|
||||
ScaleDownUtilizationThreshold: 0.35,
|
||||
ScaleDownGpuUtilizationThreshold: 0.3,
|
||||
UnremovableNodeRecheckTimeout: 5 * time.Minute,
|
||||
NodeGroupAutoscalingOptions: config.NodeGroupAutoscalingOptions{
|
||||
ScaleDownUtilizationThreshold: 0.35,
|
||||
ScaleDownGpuUtilizationThreshold: 0.3,
|
||||
},
|
||||
UnremovableNodeRecheckTimeout: 5 * time.Minute,
|
||||
}
|
||||
context, err := NewScaleTestAutoscalingContext(options, &fake.Clientset{}, nil, provider, nil)
|
||||
assert.NoError(t, err)
|
||||
|
||||
clusterStateRegistry := clusterstate.NewClusterStateRegistry(provider, clusterstate.ClusterStateRegistryConfig{}, context.LogRecorder, newBackoff())
|
||||
sd := NewScaleDown(&context, clusterStateRegistry)
|
||||
sd := NewScaleDown(&context, NewTestProcessors(), clusterStateRegistry)
|
||||
allNodes := []*apiv1.Node{n1, n2, n3}
|
||||
|
||||
simulator.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, allNodes, []*apiv1.Pod{p1, p2, p3})
|
||||
|
|
@ -314,13 +318,15 @@ func TestPodsWithPreemptionsFindUnneededNodes(t *testing.T) {
|
|||
provider.AddNode("ng1", n4)
|
||||
|
||||
options := config.AutoscalingOptions{
|
||||
ScaleDownUtilizationThreshold: 0.35,
|
||||
NodeGroupAutoscalingOptions: config.NodeGroupAutoscalingOptions{
|
||||
ScaleDownUtilizationThreshold: 0.35,
|
||||
},
|
||||
}
|
||||
context, err := NewScaleTestAutoscalingContext(options, &fake.Clientset{}, nil, provider, nil)
|
||||
assert.NoError(t, err)
|
||||
|
||||
clusterStateRegistry := clusterstate.NewClusterStateRegistry(provider, clusterstate.ClusterStateRegistryConfig{}, context.LogRecorder, newBackoff())
|
||||
sd := NewScaleDown(&context, clusterStateRegistry)
|
||||
sd := NewScaleDown(&context, NewTestProcessors(), clusterStateRegistry)
|
||||
|
||||
allNodes := []*apiv1.Node{n1, n2, n3, n4}
|
||||
simulator.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, allNodes, []*apiv1.Pod{p1, p2, p3, p4})
|
||||
|
|
@ -366,7 +372,9 @@ func TestFindUnneededMaxCandidates(t *testing.T) {
|
|||
numCandidates := 30
|
||||
|
||||
options := config.AutoscalingOptions{
|
||||
ScaleDownUtilizationThreshold: 0.35,
|
||||
NodeGroupAutoscalingOptions: config.NodeGroupAutoscalingOptions{
|
||||
ScaleDownUtilizationThreshold: 0.35,
|
||||
},
|
||||
ScaleDownNonEmptyCandidatesCount: numCandidates,
|
||||
ScaleDownCandidatesPoolRatio: 1,
|
||||
ScaleDownCandidatesPoolMinCount: 1000,
|
||||
|
|
@ -375,7 +383,7 @@ func TestFindUnneededMaxCandidates(t *testing.T) {
|
|||
assert.NoError(t, err)
|
||||
|
||||
clusterStateRegistry := clusterstate.NewClusterStateRegistry(provider, clusterstate.ClusterStateRegistryConfig{}, context.LogRecorder, newBackoff())
|
||||
sd := NewScaleDown(&context, clusterStateRegistry)
|
||||
sd := NewScaleDown(&context, NewTestProcessors(), clusterStateRegistry)
|
||||
|
||||
simulator.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, nodes, pods)
|
||||
autoscalererr = sd.UpdateUnneededNodes(nodes, nodes, time.Now(), nil)
|
||||
|
|
@ -439,7 +447,9 @@ func TestFindUnneededEmptyNodes(t *testing.T) {
|
|||
numCandidates := 30
|
||||
|
||||
options := config.AutoscalingOptions{
|
||||
ScaleDownUtilizationThreshold: 0.35,
|
||||
NodeGroupAutoscalingOptions: config.NodeGroupAutoscalingOptions{
|
||||
ScaleDownUtilizationThreshold: 0.35,
|
||||
},
|
||||
ScaleDownNonEmptyCandidatesCount: numCandidates,
|
||||
ScaleDownCandidatesPoolRatio: 1.0,
|
||||
ScaleDownCandidatesPoolMinCount: 1000,
|
||||
|
|
@ -448,7 +458,7 @@ func TestFindUnneededEmptyNodes(t *testing.T) {
|
|||
assert.NoError(t, err)
|
||||
|
||||
clusterStateRegistry := clusterstate.NewClusterStateRegistry(provider, clusterstate.ClusterStateRegistryConfig{}, context.LogRecorder, newBackoff())
|
||||
sd := NewScaleDown(&context, clusterStateRegistry)
|
||||
sd := NewScaleDown(&context, NewTestProcessors(), clusterStateRegistry)
|
||||
simulator.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, nodes, pods)
|
||||
autoscalererr = sd.UpdateUnneededNodes(nodes, nodes, time.Now(), nil)
|
||||
assert.NoError(t, autoscalererr)
|
||||
|
|
@ -487,7 +497,9 @@ func TestFindUnneededNodePool(t *testing.T) {
|
|||
numCandidates := 30
|
||||
|
||||
options := config.AutoscalingOptions{
|
||||
ScaleDownUtilizationThreshold: 0.35,
|
||||
NodeGroupAutoscalingOptions: config.NodeGroupAutoscalingOptions{
|
||||
ScaleDownUtilizationThreshold: 0.35,
|
||||
},
|
||||
ScaleDownNonEmptyCandidatesCount: numCandidates,
|
||||
ScaleDownCandidatesPoolRatio: 0.1,
|
||||
ScaleDownCandidatesPoolMinCount: 10,
|
||||
|
|
@ -496,7 +508,7 @@ func TestFindUnneededNodePool(t *testing.T) {
|
|||
assert.NoError(t, err)
|
||||
|
||||
clusterStateRegistry := clusterstate.NewClusterStateRegistry(provider, clusterstate.ClusterStateRegistryConfig{}, context.LogRecorder, newBackoff())
|
||||
sd := NewScaleDown(&context, clusterStateRegistry)
|
||||
sd := NewScaleDown(&context, NewTestProcessors(), clusterStateRegistry)
|
||||
simulator.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, nodes, pods)
|
||||
autoscalererr = sd.UpdateUnneededNodes(nodes, nodes, time.Now(), nil)
|
||||
assert.NoError(t, autoscalererr)
|
||||
|
|
@ -640,7 +652,7 @@ func TestDeleteNode(t *testing.T) {
|
|||
assert.NoError(t, err)
|
||||
|
||||
clusterStateRegistry := clusterstate.NewClusterStateRegistry(provider, clusterstate.ClusterStateRegistryConfig{}, context.LogRecorder, newBackoff())
|
||||
sd := NewScaleDown(&context, clusterStateRegistry)
|
||||
sd := NewScaleDown(&context, NewTestProcessors(), clusterStateRegistry)
|
||||
|
||||
// attempt delete
|
||||
result := sd.deleteNode(n1, pods, []*apiv1.Pod{}, provider.GetNodeGroup("ng1"))
|
||||
|
|
@ -1013,10 +1025,10 @@ func TestScaleDown(t *testing.T) {
|
|||
|
||||
options := config.AutoscalingOptions{
|
||||
NodeGroupAutoscalingOptions: config.NodeGroupAutoscalingOptions{
|
||||
ScaleDownUnneededTime: time.Minute,
|
||||
ScaleDownUnneededTime: time.Minute,
|
||||
ScaleDownUtilizationThreshold: 0.5,
|
||||
},
|
||||
ScaleDownUtilizationThreshold: 0.5,
|
||||
MaxGracefulTerminationSec: 60,
|
||||
MaxGracefulTerminationSec: 60,
|
||||
}
|
||||
jobLister, err := kube_util.NewTestJobLister([]*batchv1.Job{&job})
|
||||
assert.NoError(t, err)
|
||||
|
|
@ -1027,7 +1039,7 @@ func TestScaleDown(t *testing.T) {
|
|||
nodes := []*apiv1.Node{n1, n2}
|
||||
|
||||
clusterStateRegistry := clusterstate.NewClusterStateRegistry(provider, clusterstate.ClusterStateRegistryConfig{}, context.LogRecorder, newBackoff())
|
||||
scaleDown := NewScaleDown(&context, clusterStateRegistry)
|
||||
scaleDown := NewScaleDown(&context, NewTestProcessors(), clusterStateRegistry)
|
||||
simulator.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, nodes, []*apiv1.Pod{p1, p2})
|
||||
autoscalererr = scaleDown.UpdateUnneededNodes(nodes, nodes, time.Now().Add(-5*time.Minute), nil)
|
||||
assert.NoError(t, autoscalererr)
|
||||
|
|
@ -1072,16 +1084,16 @@ func assertSubset(t *testing.T, a []string, b []string) {
|
|||
|
||||
var defaultScaleDownOptions = config.AutoscalingOptions{
|
||||
NodeGroupAutoscalingOptions: config.NodeGroupAutoscalingOptions{
|
||||
ScaleDownUnneededTime: time.Minute,
|
||||
ScaleDownUnneededTime: time.Minute,
|
||||
ScaleDownUtilizationThreshold: 0.5,
|
||||
ScaleDownGpuUtilizationThreshold: 0.5,
|
||||
},
|
||||
ScaleDownUtilizationThreshold: 0.5,
|
||||
ScaleDownGpuUtilizationThreshold: 0.5,
|
||||
MaxGracefulTerminationSec: 60,
|
||||
MaxEmptyBulkDelete: 10,
|
||||
MinCoresTotal: 0,
|
||||
MinMemoryTotal: 0,
|
||||
MaxCoresTotal: config.DefaultMaxClusterCores,
|
||||
MaxMemoryTotal: config.DefaultMaxClusterMemory * units.GiB,
|
||||
MaxGracefulTerminationSec: 60,
|
||||
MaxEmptyBulkDelete: 10,
|
||||
MinCoresTotal: 0,
|
||||
MinMemoryTotal: 0,
|
||||
MaxCoresTotal: config.DefaultMaxClusterCores,
|
||||
MaxMemoryTotal: config.DefaultMaxClusterMemory * units.GiB,
|
||||
}
|
||||
|
||||
func TestDaemonSetEvictionForEmptyNodes(t *testing.T) {
|
||||
|
|
@ -1399,7 +1411,7 @@ func simpleScaleDownEmpty(t *testing.T, config *scaleTestConfig) {
|
|||
assert.NoError(t, err)
|
||||
|
||||
clusterStateRegistry := clusterstate.NewClusterStateRegistry(provider, clusterstate.ClusterStateRegistryConfig{}, context.LogRecorder, newBackoff())
|
||||
scaleDown := NewScaleDown(&context, clusterStateRegistry)
|
||||
scaleDown := NewScaleDown(&context, NewTestProcessors(), clusterStateRegistry)
|
||||
if config.nodeDeletionTracker != nil {
|
||||
scaleDown.nodeDeletionTracker = config.nodeDeletionTracker
|
||||
}
|
||||
|
|
@ -1477,11 +1489,11 @@ func TestNoScaleDownUnready(t *testing.T) {
|
|||
|
||||
options := config.AutoscalingOptions{
|
||||
NodeGroupAutoscalingOptions: config.NodeGroupAutoscalingOptions{
|
||||
ScaleDownUnneededTime: time.Minute,
|
||||
ScaleDownUnneededTime: time.Minute,
|
||||
ScaleDownUtilizationThreshold: 0.5,
|
||||
ScaleDownUnreadyTime: time.Hour,
|
||||
},
|
||||
ScaleDownUtilizationThreshold: 0.5,
|
||||
ScaleDownUnreadyTime: time.Hour,
|
||||
MaxGracefulTerminationSec: 60,
|
||||
MaxGracefulTerminationSec: 60,
|
||||
}
|
||||
registry := kube_util.NewListerRegistry(nil, nil, nil, nil, nil, nil, nil, nil, nil, nil)
|
||||
context, err := NewScaleTestAutoscalingContext(options, fakeClient, registry, provider, nil)
|
||||
|
|
@ -1491,7 +1503,7 @@ func TestNoScaleDownUnready(t *testing.T) {
|
|||
|
||||
// N1 is unready so it requires a bigger unneeded time.
|
||||
clusterStateRegistry := clusterstate.NewClusterStateRegistry(provider, clusterstate.ClusterStateRegistryConfig{}, context.LogRecorder, newBackoff())
|
||||
scaleDown := NewScaleDown(&context, clusterStateRegistry)
|
||||
scaleDown := NewScaleDown(&context, NewTestProcessors(), clusterStateRegistry)
|
||||
simulator.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, nodes, []*apiv1.Pod{p2})
|
||||
autoscalererr = scaleDown.UpdateUnneededNodes(nodes, nodes, time.Now().Add(-5*time.Minute), nil)
|
||||
assert.NoError(t, autoscalererr)
|
||||
|
|
@ -1514,7 +1526,7 @@ func TestNoScaleDownUnready(t *testing.T) {
|
|||
|
||||
// N1 has been unready for 2 hours, ok to delete.
|
||||
context.CloudProvider = provider
|
||||
scaleDown = NewScaleDown(&context, clusterStateRegistry)
|
||||
scaleDown = NewScaleDown(&context, NewTestProcessors(), clusterStateRegistry)
|
||||
simulator.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, nodes, []*apiv1.Pod{p2})
|
||||
autoscalererr = scaleDown.UpdateUnneededNodes(nodes, nodes, time.Now().Add(-2*time.Hour), nil)
|
||||
assert.NoError(t, autoscalererr)
|
||||
|
|
@ -1587,11 +1599,11 @@ func TestScaleDownNoMove(t *testing.T) {
|
|||
|
||||
options := config.AutoscalingOptions{
|
||||
NodeGroupAutoscalingOptions: config.NodeGroupAutoscalingOptions{
|
||||
ScaleDownUnneededTime: time.Minute,
|
||||
ScaleDownUnneededTime: time.Minute,
|
||||
ScaleDownUnreadyTime: time.Hour,
|
||||
ScaleDownUtilizationThreshold: 0.5,
|
||||
},
|
||||
ScaleDownUtilizationThreshold: 0.5,
|
||||
ScaleDownUnreadyTime: time.Hour,
|
||||
MaxGracefulTerminationSec: 60,
|
||||
MaxGracefulTerminationSec: 60,
|
||||
}
|
||||
jobLister, err := kube_util.NewTestJobLister([]*batchv1.Job{&job})
|
||||
assert.NoError(t, err)
|
||||
|
|
@ -1603,7 +1615,7 @@ func TestScaleDownNoMove(t *testing.T) {
|
|||
nodes := []*apiv1.Node{n1, n2}
|
||||
|
||||
clusterStateRegistry := clusterstate.NewClusterStateRegistry(provider, clusterstate.ClusterStateRegistryConfig{}, context.LogRecorder, newBackoff())
|
||||
scaleDown := NewScaleDown(&context, clusterStateRegistry)
|
||||
scaleDown := NewScaleDown(&context, NewTestProcessors(), clusterStateRegistry)
|
||||
simulator.InitializeClusterSnapshotOrDie(t, context.ClusterSnapshot, nodes, []*apiv1.Pod{p1, p2})
|
||||
autoscalererr = scaleDown.UpdateUnneededNodes(nodes, nodes, time.Now().Add(-5*time.Minute), nil)
|
||||
assert.NoError(t, autoscalererr)
|
||||
|
|
@ -1837,12 +1849,12 @@ func TestSoftTaint(t *testing.T) {
|
|||
|
||||
options := config.AutoscalingOptions{
|
||||
NodeGroupAutoscalingOptions: config.NodeGroupAutoscalingOptions{
|
||||
ScaleDownUnneededTime: 10 * time.Minute,
|
||||
ScaleDownUnneededTime: 10 * time.Minute,
|
||||
ScaleDownUtilizationThreshold: 0.5,
|
||||
},
|
||||
ScaleDownUtilizationThreshold: 0.5,
|
||||
MaxGracefulTerminationSec: 60,
|
||||
MaxBulkSoftTaintCount: 1,
|
||||
MaxBulkSoftTaintTime: 3 * time.Second,
|
||||
MaxGracefulTerminationSec: 60,
|
||||
MaxBulkSoftTaintCount: 1,
|
||||
MaxBulkSoftTaintTime: 3 * time.Second,
|
||||
}
|
||||
jobLister, err := kube_util.NewTestJobLister([]*batchv1.Job{&job})
|
||||
assert.NoError(t, err)
|
||||
|
|
@ -1852,7 +1864,7 @@ func TestSoftTaint(t *testing.T) {
|
|||
assert.NoError(t, err)
|
||||
|
||||
clusterStateRegistry := clusterstate.NewClusterStateRegistry(provider, clusterstate.ClusterStateRegistryConfig{}, context.LogRecorder, newBackoff())
|
||||
scaleDown := NewScaleDown(&context, clusterStateRegistry)
|
||||
scaleDown := NewScaleDown(&context, NewTestProcessors(), clusterStateRegistry)
|
||||
|
||||
// Test no superfluous nodes
|
||||
nodes := []*apiv1.Node{n1000, n2000}
|
||||
|
|
@ -1958,12 +1970,12 @@ func TestSoftTaintTimeLimit(t *testing.T) {
|
|||
|
||||
options := config.AutoscalingOptions{
|
||||
NodeGroupAutoscalingOptions: config.NodeGroupAutoscalingOptions{
|
||||
ScaleDownUnneededTime: 10 * time.Minute,
|
||||
ScaleDownUnneededTime: 10 * time.Minute,
|
||||
ScaleDownUtilizationThreshold: 0.5,
|
||||
},
|
||||
ScaleDownUtilizationThreshold: 0.5,
|
||||
MaxGracefulTerminationSec: 60,
|
||||
MaxBulkSoftTaintCount: 10,
|
||||
MaxBulkSoftTaintTime: maxSoftTaintDuration,
|
||||
MaxGracefulTerminationSec: 60,
|
||||
MaxBulkSoftTaintCount: 10,
|
||||
MaxBulkSoftTaintTime: maxSoftTaintDuration,
|
||||
}
|
||||
jobLister, err := kube_util.NewTestJobLister([]*batchv1.Job{&job})
|
||||
assert.NoError(t, err)
|
||||
|
|
@ -1973,7 +1985,7 @@ func TestSoftTaintTimeLimit(t *testing.T) {
|
|||
assert.NoError(t, err)
|
||||
|
||||
clusterStateRegistry := clusterstate.NewClusterStateRegistry(provider, clusterstate.ClusterStateRegistryConfig{}, context.LogRecorder, newBackoff())
|
||||
scaleDown := NewScaleDown(&context, clusterStateRegistry)
|
||||
scaleDown := NewScaleDown(&context, NewTestProcessors(), clusterStateRegistry)
|
||||
|
||||
// Test bulk taint
|
||||
nodes := []*apiv1.Node{n1, n2}
|
||||
|
|
|
|||
|
|
@ -32,6 +32,7 @@ import (
|
|||
"k8s.io/autoscaler/cluster-autoscaler/metrics"
|
||||
"k8s.io/autoscaler/cluster-autoscaler/processors"
|
||||
processor_callbacks "k8s.io/autoscaler/cluster-autoscaler/processors/callbacks"
|
||||
"k8s.io/autoscaler/cluster-autoscaler/processors/nodegroupconfig"
|
||||
"k8s.io/autoscaler/cluster-autoscaler/processors/nodegroups"
|
||||
"k8s.io/autoscaler/cluster-autoscaler/processors/nodegroupset"
|
||||
"k8s.io/autoscaler/cluster-autoscaler/processors/nodeinfos"
|
||||
|
|
@ -140,6 +141,7 @@ func NewTestProcessors() *processors.AutoscalingProcessors {
|
|||
AutoscalingStatusProcessor: &status.NoOpAutoscalingStatusProcessor{},
|
||||
NodeGroupManager: nodegroups.NewDefaultNodeGroupManager(),
|
||||
NodeInfoProcessor: nodeinfos.NewDefaultNodeInfoProcessor(),
|
||||
NodeGroupConfigProcessor: nodegroupconfig.NewDefaultNodeGroupConfigProcessor(),
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -147,7 +147,7 @@ func NewStaticAutoscaler(
|
|||
|
||||
clusterStateRegistry := clusterstate.NewClusterStateRegistry(autoscalingContext.CloudProvider, clusterStateConfig, autoscalingContext.LogRecorder, backoff)
|
||||
|
||||
scaleDown := NewScaleDown(autoscalingContext, clusterStateRegistry)
|
||||
scaleDown := NewScaleDown(autoscalingContext, processors, clusterStateRegistry)
|
||||
|
||||
return &StaticAutoscaler{
|
||||
AutoscalingContext: autoscalingContext,
|
||||
|
|
|
|||
|
|
@ -173,15 +173,15 @@ func TestStaticAutoscalerRunOnce(t *testing.T) {
|
|||
// Create context with mocked lister registry.
|
||||
options := config.AutoscalingOptions{
|
||||
NodeGroupAutoscalingOptions: config.NodeGroupAutoscalingOptions{
|
||||
ScaleDownUnneededTime: time.Minute,
|
||||
ScaleDownUnneededTime: time.Minute,
|
||||
ScaleDownUnreadyTime: time.Minute,
|
||||
ScaleDownUtilizationThreshold: 0.5,
|
||||
},
|
||||
EstimatorName: estimator.BinpackingEstimatorName,
|
||||
ScaleDownEnabled: true,
|
||||
ScaleDownUtilizationThreshold: 0.5,
|
||||
MaxNodesTotal: 1,
|
||||
MaxCoresTotal: 10,
|
||||
MaxMemoryTotal: 100000,
|
||||
ScaleDownUnreadyTime: time.Minute,
|
||||
EstimatorName: estimator.BinpackingEstimatorName,
|
||||
ScaleDownEnabled: true,
|
||||
MaxNodesTotal: 1,
|
||||
MaxCoresTotal: 10,
|
||||
MaxMemoryTotal: 100000,
|
||||
}
|
||||
processorCallbacks := newStaticAutoscalerProcessorCallbacks()
|
||||
|
||||
|
|
@ -198,8 +198,9 @@ func TestStaticAutoscalerRunOnce(t *testing.T) {
|
|||
MaxNodeProvisionTime: 10 * time.Second,
|
||||
}
|
||||
|
||||
processors := NewTestProcessors()
|
||||
clusterState := clusterstate.NewClusterStateRegistry(provider, clusterStateConfig, context.LogRecorder, newBackoff())
|
||||
sd := NewScaleDown(&context, clusterState)
|
||||
sd := NewScaleDown(&context, processors, clusterState)
|
||||
|
||||
autoscaler := &StaticAutoscaler{
|
||||
AutoscalingContext: &context,
|
||||
|
|
@ -207,7 +208,7 @@ func TestStaticAutoscalerRunOnce(t *testing.T) {
|
|||
lastScaleUpTime: time.Now(),
|
||||
lastScaleDownFailTime: time.Now(),
|
||||
scaleDown: sd,
|
||||
processors: NewTestProcessors(),
|
||||
processors: processors,
|
||||
processorCallbacks: processorCallbacks,
|
||||
initialized: true,
|
||||
}
|
||||
|
|
@ -361,15 +362,15 @@ func TestStaticAutoscalerRunOnceWithAutoprovisionedEnabled(t *testing.T) {
|
|||
// Create context with mocked lister registry.
|
||||
options := config.AutoscalingOptions{
|
||||
NodeGroupAutoscalingOptions: config.NodeGroupAutoscalingOptions{
|
||||
ScaleDownUnneededTime: time.Minute,
|
||||
ScaleDownUnneededTime: time.Minute,
|
||||
ScaleDownUnreadyTime: time.Minute,
|
||||
ScaleDownUtilizationThreshold: 0.5,
|
||||
},
|
||||
EstimatorName: estimator.BinpackingEstimatorName,
|
||||
ScaleDownEnabled: true,
|
||||
ScaleDownUtilizationThreshold: 0.5,
|
||||
MaxNodesTotal: 100,
|
||||
MaxCoresTotal: 100,
|
||||
MaxMemoryTotal: 100000,
|
||||
ScaleDownUnreadyTime: time.Minute,
|
||||
NodeAutoprovisioningEnabled: true,
|
||||
MaxAutoprovisionedNodeGroupCount: 10,
|
||||
}
|
||||
|
|
@ -389,7 +390,7 @@ func TestStaticAutoscalerRunOnceWithAutoprovisionedEnabled(t *testing.T) {
|
|||
}
|
||||
clusterState := clusterstate.NewClusterStateRegistry(provider, clusterStateConfig, context.LogRecorder, newBackoff())
|
||||
|
||||
sd := NewScaleDown(&context, clusterState)
|
||||
sd := NewScaleDown(&context, processors, clusterState)
|
||||
|
||||
autoscaler := &StaticAutoscaler{
|
||||
AutoscalingContext: &context,
|
||||
|
|
@ -497,16 +498,16 @@ func TestStaticAutoscalerRunOnceWithALongUnregisteredNode(t *testing.T) {
|
|||
// Create context with mocked lister registry.
|
||||
options := config.AutoscalingOptions{
|
||||
NodeGroupAutoscalingOptions: config.NodeGroupAutoscalingOptions{
|
||||
ScaleDownUnneededTime: time.Minute,
|
||||
ScaleDownUnneededTime: time.Minute,
|
||||
ScaleDownUnreadyTime: time.Minute,
|
||||
ScaleDownUtilizationThreshold: 0.5,
|
||||
},
|
||||
EstimatorName: estimator.BinpackingEstimatorName,
|
||||
ScaleDownEnabled: true,
|
||||
ScaleDownUtilizationThreshold: 0.5,
|
||||
MaxNodesTotal: 10,
|
||||
MaxCoresTotal: 10,
|
||||
MaxMemoryTotal: 100000,
|
||||
ScaleDownUnreadyTime: time.Minute,
|
||||
MaxNodeProvisionTime: 10 * time.Second,
|
||||
EstimatorName: estimator.BinpackingEstimatorName,
|
||||
ScaleDownEnabled: true,
|
||||
MaxNodesTotal: 10,
|
||||
MaxCoresTotal: 10,
|
||||
MaxMemoryTotal: 100000,
|
||||
MaxNodeProvisionTime: 10 * time.Second,
|
||||
}
|
||||
processorCallbacks := newStaticAutoscalerProcessorCallbacks()
|
||||
|
||||
|
|
@ -532,7 +533,9 @@ func TestStaticAutoscalerRunOnceWithALongUnregisteredNode(t *testing.T) {
|
|||
// broken node failed to register in time
|
||||
clusterState.UpdateNodes(nodes, nil, later)
|
||||
|
||||
sd := NewScaleDown(&context, clusterState)
|
||||
processors := NewTestProcessors()
|
||||
|
||||
sd := NewScaleDown(&context, processors, clusterState)
|
||||
|
||||
autoscaler := &StaticAutoscaler{
|
||||
AutoscalingContext: &context,
|
||||
|
|
@ -540,7 +543,7 @@ func TestStaticAutoscalerRunOnceWithALongUnregisteredNode(t *testing.T) {
|
|||
lastScaleUpTime: time.Now(),
|
||||
lastScaleDownFailTime: time.Now(),
|
||||
scaleDown: sd,
|
||||
processors: NewTestProcessors(),
|
||||
processors: processors,
|
||||
processorCallbacks: processorCallbacks,
|
||||
}
|
||||
|
||||
|
|
@ -642,16 +645,16 @@ func TestStaticAutoscalerRunOncePodsWithPriorities(t *testing.T) {
|
|||
// Create context with mocked lister registry.
|
||||
options := config.AutoscalingOptions{
|
||||
NodeGroupAutoscalingOptions: config.NodeGroupAutoscalingOptions{
|
||||
ScaleDownUnneededTime: time.Minute,
|
||||
ScaleDownUnneededTime: time.Minute,
|
||||
ScaleDownUtilizationThreshold: 0.5,
|
||||
ScaleDownUnreadyTime: time.Minute,
|
||||
},
|
||||
EstimatorName: estimator.BinpackingEstimatorName,
|
||||
ScaleDownEnabled: true,
|
||||
ScaleDownUtilizationThreshold: 0.5,
|
||||
MaxNodesTotal: 10,
|
||||
MaxCoresTotal: 10,
|
||||
MaxMemoryTotal: 100000,
|
||||
ScaleDownUnreadyTime: time.Minute,
|
||||
ExpendablePodsPriorityCutoff: 10,
|
||||
EstimatorName: estimator.BinpackingEstimatorName,
|
||||
ScaleDownEnabled: true,
|
||||
MaxNodesTotal: 10,
|
||||
MaxCoresTotal: 10,
|
||||
MaxMemoryTotal: 100000,
|
||||
ExpendablePodsPriorityCutoff: 10,
|
||||
}
|
||||
processorCallbacks := newStaticAutoscalerProcessorCallbacks()
|
||||
|
||||
|
|
@ -668,8 +671,9 @@ func TestStaticAutoscalerRunOncePodsWithPriorities(t *testing.T) {
|
|||
MaxNodeProvisionTime: 10 * time.Second,
|
||||
}
|
||||
|
||||
processors := NewTestProcessors()
|
||||
clusterState := clusterstate.NewClusterStateRegistry(provider, clusterStateConfig, context.LogRecorder, newBackoff())
|
||||
sd := NewScaleDown(&context, clusterState)
|
||||
sd := NewScaleDown(&context, processors, clusterState)
|
||||
|
||||
autoscaler := &StaticAutoscaler{
|
||||
AutoscalingContext: &context,
|
||||
|
|
@ -677,7 +681,7 @@ func TestStaticAutoscalerRunOncePodsWithPriorities(t *testing.T) {
|
|||
lastScaleUpTime: time.Now(),
|
||||
lastScaleDownFailTime: time.Now(),
|
||||
scaleDown: sd,
|
||||
processors: NewTestProcessors(),
|
||||
processors: processors,
|
||||
processorCallbacks: processorCallbacks,
|
||||
}
|
||||
|
||||
|
|
@ -770,13 +774,15 @@ func TestStaticAutoscalerRunOnceWithFilteringOnBinPackingEstimator(t *testing.T)
|
|||
|
||||
// Create context with mocked lister registry.
|
||||
options := config.AutoscalingOptions{
|
||||
EstimatorName: estimator.BinpackingEstimatorName,
|
||||
ScaleDownEnabled: false,
|
||||
ScaleDownUtilizationThreshold: 0.5,
|
||||
MaxNodesTotal: 10,
|
||||
MaxCoresTotal: 10,
|
||||
MaxMemoryTotal: 100000,
|
||||
ExpendablePodsPriorityCutoff: 10,
|
||||
NodeGroupAutoscalingOptions: config.NodeGroupAutoscalingOptions{
|
||||
ScaleDownUtilizationThreshold: 0.5,
|
||||
},
|
||||
EstimatorName: estimator.BinpackingEstimatorName,
|
||||
ScaleDownEnabled: false,
|
||||
MaxNodesTotal: 10,
|
||||
MaxCoresTotal: 10,
|
||||
MaxMemoryTotal: 100000,
|
||||
ExpendablePodsPriorityCutoff: 10,
|
||||
}
|
||||
processorCallbacks := newStaticAutoscalerProcessorCallbacks()
|
||||
|
||||
|
|
@ -793,8 +799,9 @@ func TestStaticAutoscalerRunOnceWithFilteringOnBinPackingEstimator(t *testing.T)
|
|||
MaxNodeProvisionTime: 10 * time.Second,
|
||||
}
|
||||
|
||||
processors := NewTestProcessors()
|
||||
clusterState := clusterstate.NewClusterStateRegistry(provider, clusterStateConfig, context.LogRecorder, newBackoff())
|
||||
sd := NewScaleDown(&context, clusterState)
|
||||
sd := NewScaleDown(&context, processors, clusterState)
|
||||
|
||||
autoscaler := &StaticAutoscaler{
|
||||
AutoscalingContext: &context,
|
||||
|
|
@ -802,7 +809,7 @@ func TestStaticAutoscalerRunOnceWithFilteringOnBinPackingEstimator(t *testing.T)
|
|||
lastScaleUpTime: time.Now(),
|
||||
lastScaleDownFailTime: time.Now(),
|
||||
scaleDown: sd,
|
||||
processors: NewTestProcessors(),
|
||||
processors: processors,
|
||||
processorCallbacks: processorCallbacks,
|
||||
}
|
||||
|
||||
|
|
@ -863,13 +870,15 @@ func TestStaticAutoscalerRunOnceWithFilteringOnUpcomingNodesEnabledNoScaleUp(t *
|
|||
|
||||
// Create context with mocked lister registry.
|
||||
options := config.AutoscalingOptions{
|
||||
EstimatorName: estimator.BinpackingEstimatorName,
|
||||
ScaleDownEnabled: false,
|
||||
ScaleDownUtilizationThreshold: 0.5,
|
||||
MaxNodesTotal: 10,
|
||||
MaxCoresTotal: 10,
|
||||
MaxMemoryTotal: 100000,
|
||||
ExpendablePodsPriorityCutoff: 10,
|
||||
NodeGroupAutoscalingOptions: config.NodeGroupAutoscalingOptions{
|
||||
ScaleDownUtilizationThreshold: 0.5,
|
||||
},
|
||||
EstimatorName: estimator.BinpackingEstimatorName,
|
||||
ScaleDownEnabled: false,
|
||||
MaxNodesTotal: 10,
|
||||
MaxCoresTotal: 10,
|
||||
MaxMemoryTotal: 100000,
|
||||
ExpendablePodsPriorityCutoff: 10,
|
||||
}
|
||||
processorCallbacks := newStaticAutoscalerProcessorCallbacks()
|
||||
|
||||
|
|
@ -886,8 +895,9 @@ func TestStaticAutoscalerRunOnceWithFilteringOnUpcomingNodesEnabledNoScaleUp(t *
|
|||
MaxNodeProvisionTime: 10 * time.Second,
|
||||
}
|
||||
|
||||
processors := NewTestProcessors()
|
||||
clusterState := clusterstate.NewClusterStateRegistry(provider, clusterStateConfig, context.LogRecorder, newBackoff())
|
||||
sd := NewScaleDown(&context, clusterState)
|
||||
sd := NewScaleDown(&context, processors, clusterState)
|
||||
|
||||
autoscaler := &StaticAutoscaler{
|
||||
AutoscalingContext: &context,
|
||||
|
|
@ -895,7 +905,7 @@ func TestStaticAutoscalerRunOnceWithFilteringOnUpcomingNodesEnabledNoScaleUp(t *
|
|||
lastScaleUpTime: time.Now(),
|
||||
lastScaleDownFailTime: time.Now(),
|
||||
scaleDown: sd,
|
||||
processors: NewTestProcessors(),
|
||||
processors: processors,
|
||||
processorCallbacks: processorCallbacks,
|
||||
}
|
||||
|
||||
|
|
@ -921,16 +931,16 @@ func TestStaticAutoscalerInstaceCreationErrors(t *testing.T) {
|
|||
// Create context with mocked lister registry.
|
||||
options := config.AutoscalingOptions{
|
||||
NodeGroupAutoscalingOptions: config.NodeGroupAutoscalingOptions{
|
||||
ScaleDownUnneededTime: time.Minute,
|
||||
ScaleDownUnneededTime: time.Minute,
|
||||
ScaleDownUnreadyTime: time.Minute,
|
||||
ScaleDownUtilizationThreshold: 0.5,
|
||||
},
|
||||
EstimatorName: estimator.BinpackingEstimatorName,
|
||||
ScaleDownEnabled: true,
|
||||
ScaleDownUtilizationThreshold: 0.5,
|
||||
MaxNodesTotal: 10,
|
||||
MaxCoresTotal: 10,
|
||||
MaxMemoryTotal: 100000,
|
||||
ScaleDownUnreadyTime: time.Minute,
|
||||
ExpendablePodsPriorityCutoff: 10,
|
||||
EstimatorName: estimator.BinpackingEstimatorName,
|
||||
ScaleDownEnabled: true,
|
||||
MaxNodesTotal: 10,
|
||||
MaxCoresTotal: 10,
|
||||
MaxMemoryTotal: 100000,
|
||||
ExpendablePodsPriorityCutoff: 10,
|
||||
}
|
||||
processorCallbacks := newStaticAutoscalerProcessorCallbacks()
|
||||
|
||||
|
|
|
|||
|
|
@ -198,7 +198,10 @@ func createAutoscalingOptions() config.AutoscalingOptions {
|
|||
}
|
||||
return config.AutoscalingOptions{
|
||||
NodeGroupAutoscalingOptions: config.NodeGroupAutoscalingOptions{
|
||||
ScaleDownUnneededTime: *scaleDownUnneededTime,
|
||||
ScaleDownUtilizationThreshold: *scaleDownUtilizationThreshold,
|
||||
ScaleDownGpuUtilizationThreshold: *scaleDownGpuUtilizationThreshold,
|
||||
ScaleDownUnneededTime: *scaleDownUnneededTime,
|
||||
ScaleDownUnreadyTime: *scaleDownUnreadyTime,
|
||||
},
|
||||
CloudConfig: *cloudConfig,
|
||||
CloudProviderName: *cloudProviderFlag,
|
||||
|
|
@ -226,9 +229,6 @@ func createAutoscalingOptions() config.AutoscalingOptions {
|
|||
ScaleDownDelayAfterDelete: *scaleDownDelayAfterDelete,
|
||||
ScaleDownDelayAfterFailure: *scaleDownDelayAfterFailure,
|
||||
ScaleDownEnabled: *scaleDownEnabled,
|
||||
ScaleDownUnreadyTime: *scaleDownUnreadyTime,
|
||||
ScaleDownUtilizationThreshold: *scaleDownUtilizationThreshold,
|
||||
ScaleDownGpuUtilizationThreshold: *scaleDownGpuUtilizationThreshold,
|
||||
ScaleDownNonEmptyCandidatesCount: *scaleDownNonEmptyCandidatesCount,
|
||||
ScaleDownCandidatesPoolRatio: *scaleDownCandidatesPoolRatio,
|
||||
ScaleDownCandidatesPoolMinCount: *scaleDownCandidatesPoolMinCount,
|
||||
|
|
|
|||
|
|
@ -25,8 +25,14 @@ import (
|
|||
|
||||
// NodeGroupConfigProcessor provides config values for a particular NodeGroup.
|
||||
type NodeGroupConfigProcessor interface {
|
||||
// Process processes a map of nodeInfos for node groups.
|
||||
// GetScaleDownUnneededTime returns ScaleDownUnneededTime value that should be used for a given NodeGroup.
|
||||
GetScaleDownUnneededTime(context *context.AutoscalingContext, nodeGroup cloudprovider.NodeGroup) (time.Duration, error)
|
||||
// GetScaleDownUnreadyTime returns ScaleDownUnreadyTime value that should be used for a given NodeGroup.
|
||||
GetScaleDownUnreadyTime(context *context.AutoscalingContext, nodeGroup cloudprovider.NodeGroup) (time.Duration, error)
|
||||
// GetScaleDownUtilizationThreshold returns ScaleDownUtilizationThreshold value that should be used for a given NodeGroup.
|
||||
GetScaleDownUtilizationThreshold(context *context.AutoscalingContext, nodeGroup cloudprovider.NodeGroup) (float64, error)
|
||||
// GetScaleDownGpuUtilizationThreshold returns ScaleDownGpuUtilizationThreshold value that should be used for a given NodeGroup.
|
||||
GetScaleDownGpuUtilizationThreshold(context *context.AutoscalingContext, nodeGroup cloudprovider.NodeGroup) (float64, error)
|
||||
// CleanUp cleans up processor's internal structures.
|
||||
CleanUp()
|
||||
}
|
||||
|
|
@ -49,6 +55,42 @@ func (p *DelegatingNodeGroupConfigProcessor) GetScaleDownUnneededTime(context *c
|
|||
return ngConfig.ScaleDownUnneededTime, nil
|
||||
}
|
||||
|
||||
// GetScaleDownUnreadyTime returns ScaleDownUnreadyTime value that should be used for a given NodeGroup.
|
||||
func (p *DelegatingNodeGroupConfigProcessor) GetScaleDownUnreadyTime(context *context.AutoscalingContext, nodeGroup cloudprovider.NodeGroup) (time.Duration, error) {
|
||||
ngConfig, err := nodeGroup.GetOptions(context.NodeGroupAutoscalingOptions)
|
||||
if err != nil && err != cloudprovider.ErrNotImplemented {
|
||||
return time.Duration(0), err
|
||||
}
|
||||
if ngConfig == nil || err == cloudprovider.ErrNotImplemented {
|
||||
return context.ScaleDownUnreadyTime, nil
|
||||
}
|
||||
return ngConfig.ScaleDownUnreadyTime, nil
|
||||
}
|
||||
|
||||
// GetScaleDownUtilizationThreshold returns ScaleDownUtilizationThreshold value that should be used for a given NodeGroup.
|
||||
func (p *DelegatingNodeGroupConfigProcessor) GetScaleDownUtilizationThreshold(context *context.AutoscalingContext, nodeGroup cloudprovider.NodeGroup) (float64, error) {
|
||||
ngConfig, err := nodeGroup.GetOptions(context.NodeGroupAutoscalingOptions)
|
||||
if err != nil && err != cloudprovider.ErrNotImplemented {
|
||||
return 0.0, err
|
||||
}
|
||||
if ngConfig == nil || err == cloudprovider.ErrNotImplemented {
|
||||
return context.ScaleDownUtilizationThreshold, nil
|
||||
}
|
||||
return ngConfig.ScaleDownUtilizationThreshold, nil
|
||||
}
|
||||
|
||||
// GetScaleDownGpuUtilizationThreshold returns ScaleDownGpuUtilizationThreshold value that should be used for a given NodeGroup.
|
||||
func (p *DelegatingNodeGroupConfigProcessor) GetScaleDownGpuUtilizationThreshold(context *context.AutoscalingContext, nodeGroup cloudprovider.NodeGroup) (float64, error) {
|
||||
ngConfig, err := nodeGroup.GetOptions(context.NodeGroupAutoscalingOptions)
|
||||
if err != nil && err != cloudprovider.ErrNotImplemented {
|
||||
return 0.0, err
|
||||
}
|
||||
if ngConfig == nil || err == cloudprovider.ErrNotImplemented {
|
||||
return context.ScaleDownGpuUtilizationThreshold, nil
|
||||
}
|
||||
return ngConfig.ScaleDownGpuUtilizationThreshold, nil
|
||||
}
|
||||
|
||||
// CleanUp cleans up processor's internal structures.
|
||||
func (p *DelegatingNodeGroupConfigProcessor) CleanUp() {
|
||||
}
|
||||
|
|
|
|||
|
|
@ -18,6 +18,7 @@ package nodegroupconfig
|
|||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
|
|
@ -29,53 +30,165 @@ import (
|
|||
"github.com/stretchr/testify/assert"
|
||||
)
|
||||
|
||||
func TestApplyingDefaults(t *testing.T) {
|
||||
defaultOptions := config.NodeGroupAutoscalingOptions{
|
||||
ScaleDownUnneededTime: 3 * time.Minute,
|
||||
// This test covers all Get* methods implemented by
|
||||
// DelegatingNodeGroupConfigProcessor. The implementation and expectations are
|
||||
// identical hence a single test for the whole bunch.
|
||||
func TestDelegatingNodeGroupConfigProcessor(t *testing.T) {
|
||||
// Different methods covered by this test have identical implementation,
|
||||
// but return values of different types.
|
||||
// This enum is a generic way to specify test expectations without
|
||||
// some reflection magic.
|
||||
type Want int
|
||||
var NIL Want = 0
|
||||
var GLOBAL Want = 1
|
||||
var NG Want = 2
|
||||
|
||||
testUnneededTime := func(t *testing.T, p DelegatingNodeGroupConfigProcessor, c *context.AutoscalingContext, ng cloudprovider.NodeGroup, w Want, we error) {
|
||||
res, err := p.GetScaleDownUnneededTime(c, ng)
|
||||
assert.Equal(t, err, we)
|
||||
results := map[Want]time.Duration{
|
||||
NIL: time.Duration(0),
|
||||
GLOBAL: 3 * time.Minute,
|
||||
NG: 10 * time.Minute,
|
||||
}
|
||||
assert.Equal(t, res, results[w])
|
||||
}
|
||||
testUnreadyTime := func(t *testing.T, p DelegatingNodeGroupConfigProcessor, c *context.AutoscalingContext, ng cloudprovider.NodeGroup, w Want, we error) {
|
||||
res, err := p.GetScaleDownUnreadyTime(c, ng)
|
||||
assert.Equal(t, err, we)
|
||||
results := map[Want]time.Duration{
|
||||
NIL: time.Duration(0),
|
||||
GLOBAL: 4 * time.Minute,
|
||||
NG: 11 * time.Minute,
|
||||
}
|
||||
assert.Equal(t, res, results[w])
|
||||
}
|
||||
testUtilizationThreshold := func(t *testing.T, p DelegatingNodeGroupConfigProcessor, c *context.AutoscalingContext, ng cloudprovider.NodeGroup, w Want, we error) {
|
||||
res, err := p.GetScaleDownUtilizationThreshold(c, ng)
|
||||
assert.Equal(t, err, we)
|
||||
results := map[Want]float64{
|
||||
NIL: 0.0,
|
||||
GLOBAL: 0.5,
|
||||
NG: 0.75,
|
||||
}
|
||||
assert.Equal(t, res, results[w])
|
||||
}
|
||||
testGpuThreshold := func(t *testing.T, p DelegatingNodeGroupConfigProcessor, c *context.AutoscalingContext, ng cloudprovider.NodeGroup, w Want, we error) {
|
||||
res, err := p.GetScaleDownGpuUtilizationThreshold(c, ng)
|
||||
assert.Equal(t, err, we)
|
||||
results := map[Want]float64{
|
||||
NIL: 0.0,
|
||||
GLOBAL: 0.6,
|
||||
NG: 0.85,
|
||||
}
|
||||
assert.Equal(t, res, results[w])
|
||||
}
|
||||
|
||||
cases := map[string]struct {
|
||||
globalOptions config.NodeGroupAutoscalingOptions
|
||||
ngOptions *config.NodeGroupAutoscalingOptions
|
||||
ngError error
|
||||
wantScaleDownUnneeded time.Duration
|
||||
wantError error
|
||||
funcs := map[string]struct {
|
||||
testFn func(*testing.T, DelegatingNodeGroupConfigProcessor, *context.AutoscalingContext, cloudprovider.NodeGroup, Want, error)
|
||||
globalOpts config.NodeGroupAutoscalingOptions
|
||||
ngOpts *config.NodeGroupAutoscalingOptions
|
||||
}{
|
||||
"NodeGroup.GetOptions not implemented": {
|
||||
globalOptions: defaultOptions,
|
||||
ngError: cloudprovider.ErrNotImplemented,
|
||||
wantScaleDownUnneeded: 3 * time.Minute,
|
||||
},
|
||||
"NodeGroup returns error leads to error": {
|
||||
globalOptions: defaultOptions,
|
||||
ngError: errors.New("This sentence is false."),
|
||||
wantError: errors.New("This sentence is false."),
|
||||
},
|
||||
"NodeGroup returns no value fallbacks to default": {
|
||||
globalOptions: defaultOptions,
|
||||
wantScaleDownUnneeded: 3 * time.Minute,
|
||||
},
|
||||
"NodeGroup option overrides global default": {
|
||||
globalOptions: defaultOptions,
|
||||
ngOptions: &config.NodeGroupAutoscalingOptions{
|
||||
"ScaleDownUnneededTime": {
|
||||
testFn: testUnneededTime,
|
||||
globalOpts: config.NodeGroupAutoscalingOptions{
|
||||
ScaleDownUnneededTime: 3 * time.Minute,
|
||||
},
|
||||
ngOpts: &config.NodeGroupAutoscalingOptions{
|
||||
ScaleDownUnneededTime: 10 * time.Minute,
|
||||
},
|
||||
wantScaleDownUnneeded: 10 * time.Minute,
|
||||
},
|
||||
"ScaleDownUnreadyTime": {
|
||||
testFn: testUnreadyTime,
|
||||
globalOpts: config.NodeGroupAutoscalingOptions{
|
||||
ScaleDownUnreadyTime: 4 * time.Minute,
|
||||
},
|
||||
ngOpts: &config.NodeGroupAutoscalingOptions{
|
||||
ScaleDownUnreadyTime: 11 * time.Minute,
|
||||
},
|
||||
},
|
||||
"ScaleDownUtilizationThreshold": {
|
||||
testFn: testUtilizationThreshold,
|
||||
globalOpts: config.NodeGroupAutoscalingOptions{
|
||||
ScaleDownUtilizationThreshold: 0.5,
|
||||
},
|
||||
ngOpts: &config.NodeGroupAutoscalingOptions{
|
||||
ScaleDownUtilizationThreshold: 0.75,
|
||||
},
|
||||
},
|
||||
"ScaleDownGpuUtilizationThreshold": {
|
||||
testFn: testGpuThreshold,
|
||||
globalOpts: config.NodeGroupAutoscalingOptions{
|
||||
ScaleDownGpuUtilizationThreshold: 0.6,
|
||||
},
|
||||
ngOpts: &config.NodeGroupAutoscalingOptions{
|
||||
ScaleDownGpuUtilizationThreshold: 0.85,
|
||||
},
|
||||
},
|
||||
"MultipleOptions": {
|
||||
testFn: func(t *testing.T, p DelegatingNodeGroupConfigProcessor, c *context.AutoscalingContext, ng cloudprovider.NodeGroup, w Want, we error) {
|
||||
testUnneededTime(t, p, c, ng, w, we)
|
||||
testUnreadyTime(t, p, c, ng, w, we)
|
||||
testUtilizationThreshold(t, p, c, ng, w, we)
|
||||
testUnneededTime(t, p, c, ng, w, we)
|
||||
testUnneededTime(t, p, c, ng, w, we)
|
||||
testGpuThreshold(t, p, c, ng, w, we)
|
||||
},
|
||||
globalOpts: config.NodeGroupAutoscalingOptions{
|
||||
ScaleDownUnneededTime: 3 * time.Minute,
|
||||
ScaleDownUnreadyTime: 4 * time.Minute,
|
||||
ScaleDownGpuUtilizationThreshold: 0.6,
|
||||
ScaleDownUtilizationThreshold: 0.5,
|
||||
},
|
||||
ngOpts: &config.NodeGroupAutoscalingOptions{
|
||||
ScaleDownUnneededTime: 10 * time.Minute,
|
||||
ScaleDownUnreadyTime: 11 * time.Minute,
|
||||
ScaleDownGpuUtilizationThreshold: 0.85,
|
||||
ScaleDownUtilizationThreshold: 0.75,
|
||||
},
|
||||
},
|
||||
}
|
||||
for tn, tc := range cases {
|
||||
t.Run(tn, func(t *testing.T) {
|
||||
context := &context.AutoscalingContext{
|
||||
AutoscalingOptions: config.AutoscalingOptions{
|
||||
NodeGroupAutoscalingOptions: tc.globalOptions,
|
||||
},
|
||||
}
|
||||
ng := &mocks.NodeGroup{}
|
||||
ng.On("GetOptions", tc.globalOptions).Return(tc.ngOptions, tc.ngError).Once()
|
||||
p := NewDefaultNodeGroupConfigProcessor()
|
||||
res, err := p.GetScaleDownUnneededTime(context, ng)
|
||||
assert.Equal(t, res, tc.wantScaleDownUnneeded)
|
||||
assert.Equal(t, err, tc.wantError)
|
||||
})
|
||||
|
||||
for fname, fn := range funcs {
|
||||
cases := map[string]struct {
|
||||
globalOptions config.NodeGroupAutoscalingOptions
|
||||
ngOptions *config.NodeGroupAutoscalingOptions
|
||||
ngError error
|
||||
want Want
|
||||
wantError error
|
||||
}{
|
||||
"NodeGroup.GetOptions not implemented": {
|
||||
globalOptions: fn.globalOpts,
|
||||
ngError: cloudprovider.ErrNotImplemented,
|
||||
want: GLOBAL,
|
||||
},
|
||||
"NodeGroup returns error leads to error": {
|
||||
globalOptions: fn.globalOpts,
|
||||
ngError: errors.New("This sentence is false."),
|
||||
wantError: errors.New("This sentence is false."),
|
||||
},
|
||||
"NodeGroup returns no value fallbacks to default": {
|
||||
globalOptions: fn.globalOpts,
|
||||
want: GLOBAL,
|
||||
},
|
||||
"NodeGroup option overrides global default": {
|
||||
globalOptions: fn.globalOpts,
|
||||
ngOptions: fn.ngOpts,
|
||||
want: NG,
|
||||
},
|
||||
}
|
||||
for tn, tc := range cases {
|
||||
t.Run(fmt.Sprintf("[%s] %s", fname, tn), func(t *testing.T) {
|
||||
context := &context.AutoscalingContext{
|
||||
AutoscalingOptions: config.AutoscalingOptions{
|
||||
NodeGroupAutoscalingOptions: tc.globalOptions,
|
||||
},
|
||||
}
|
||||
ng := &mocks.NodeGroup{}
|
||||
ng.On("GetOptions", tc.globalOptions).Return(tc.ngOptions, tc.ngError)
|
||||
p := DelegatingNodeGroupConfigProcessor{}
|
||||
fn.testFn(t, p, context, ng, tc.want, tc.wantError)
|
||||
})
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
Loading…
Reference in New Issue