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:
Maciek Pytel 2020-12-30 19:12:31 +01:00
parent 08d18a7bd0
commit 3e42b26a22
9 changed files with 411 additions and 193 deletions

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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