Merge pull request #8112 from jackfrancis/ca-soft-taints-during-scale-down

[cluster-autoscaler-release-1.31] Fix cool down status condition to trigger scale down
This commit is contained in:
Kubernetes Prow Robot 2025-05-13 22:47:17 -07:00 committed by GitHub
commit 31422a8495
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
3 changed files with 197 additions and 73 deletions

View File

@ -100,6 +100,8 @@ const (
ScaleDownInCooldown
// ScaleDownInProgress - the scale down wasn't attempted, because a previous scale-down was still in progress.
ScaleDownInProgress
// ScaleDownNoCandidates - the scale down was skipped because of no scale down candidates.
ScaleDownNoCandidates
)
// NodeDeleteResultType denotes the type of the result of node deletion. It provides deeper

View File

@ -619,7 +619,7 @@ func (a *StaticAutoscaler) RunOnce(currentTime time.Time) caerrors.AutoscalerErr
metrics.UpdateDurationFromStart(metrics.FindUnneeded, unneededStart)
scaleDownInCooldown := a.isScaleDownInCooldown(currentTime, scaleDownCandidates)
scaleDownInCooldown := a.isScaleDownInCooldown(currentTime)
klog.V(4).Infof("Scale down status: lastScaleUpTime=%s lastScaleDownDeleteTime=%v "+
"lastScaleDownFailTime=%s scaleDownForbidden=%v scaleDownInCooldown=%v",
a.lastScaleUpTime, a.lastScaleDownDeleteTime, a.lastScaleDownFailTime,
@ -640,6 +640,7 @@ func (a *StaticAutoscaler) RunOnce(currentTime time.Time) caerrors.AutoscalerErr
if scaleDownInCooldown {
scaleDownStatus.Result = scaledownstatus.ScaleDownInCooldown
a.updateSoftDeletionTaints(allNodes)
} else {
klog.V(4).Infof("Starting scale down")
@ -659,20 +660,7 @@ func (a *StaticAutoscaler) RunOnce(currentTime time.Time) caerrors.AutoscalerErr
a.clusterStateRegistry.Recalculate()
}
if (scaleDownStatus.Result == scaledownstatus.ScaleDownNoNodeDeleted ||
scaleDownStatus.Result == scaledownstatus.ScaleDownNoUnneeded) &&
a.AutoscalingContext.AutoscalingOptions.MaxBulkSoftTaintCount != 0 {
taintableNodes := a.scaleDownPlanner.UnneededNodes()
// Make sure we are only cleaning taints from selected node groups.
selectedNodes := filterNodesFromSelectedGroups(a.CloudProvider, allNodes...)
// This is a sanity check to make sure `taintableNodes` only includes
// nodes from selected nodes.
taintableNodes = intersectNodes(selectedNodes, taintableNodes)
untaintableNodes := subtractNodes(selectedNodes, taintableNodes)
actuation.UpdateSoftDeletionTaints(a.AutoscalingContext, taintableNodes, untaintableNodes)
}
a.updateSoftDeletionTaints(allNodes)
if typedErr != nil {
klog.Errorf("Failed to scale down: %v", typedErr)
@ -693,6 +681,21 @@ func (a *StaticAutoscaler) RunOnce(currentTime time.Time) caerrors.AutoscalerErr
return nil
}
func (a *StaticAutoscaler) updateSoftDeletionTaints(allNodes []*apiv1.Node) {
if a.AutoscalingContext.AutoscalingOptions.MaxBulkSoftTaintCount != 0 {
taintableNodes := a.scaleDownPlanner.UnneededNodes()
// Make sure we are only cleaning taints from selected node groups.
selectedNodes := filterNodesFromSelectedGroups(a.CloudProvider, allNodes...)
// This is a sanity check to make sure `taintableNodes` only includes
// nodes from selected nodes.
taintableNodes = intersectNodes(selectedNodes, taintableNodes)
untaintableNodes := subtractNodes(selectedNodes, taintableNodes)
actuation.UpdateSoftDeletionTaints(a.AutoscalingContext, taintableNodes, untaintableNodes)
}
}
func (a *StaticAutoscaler) addUpcomingNodesToClusterSnapshot(upcomingCounts map[string]int, nodeInfosForGroups map[string]*schedulerframework.NodeInfo) error {
nodeGroups := a.nodeGroupsById()
upcomingNodeGroups := make(map[string]int)
@ -724,8 +727,8 @@ func (a *StaticAutoscaler) addUpcomingNodesToClusterSnapshot(upcomingCounts map[
return nil
}
func (a *StaticAutoscaler) isScaleDownInCooldown(currentTime time.Time, scaleDownCandidates []*apiv1.Node) bool {
scaleDownInCooldown := a.processorCallbacks.disableScaleDownForLoop || len(scaleDownCandidates) == 0
func (a *StaticAutoscaler) isScaleDownInCooldown(currentTime time.Time) bool {
scaleDownInCooldown := a.processorCallbacks.disableScaleDownForLoop
if a.ScaleDownDelayTypeLocal {
return scaleDownInCooldown

View File

@ -38,6 +38,7 @@ import (
"k8s.io/autoscaler/cluster-autoscaler/core/scaledown/actuation"
"k8s.io/autoscaler/cluster-autoscaler/core/scaledown/deletiontracker"
"k8s.io/autoscaler/cluster-autoscaler/core/scaledown/legacy"
"k8s.io/autoscaler/cluster-autoscaler/core/scaledown/planner"
"k8s.io/autoscaler/cluster-autoscaler/core/scaledown/status"
"k8s.io/autoscaler/cluster-autoscaler/core/scaleup/orchestrator"
. "k8s.io/autoscaler/cluster-autoscaler/core/test"
@ -1393,78 +1394,37 @@ func TestStaticAutoscalerRunOnceWithUnselectedNodeGroups(t *testing.T) {
provider.AddNode("ng1", n1)
assert.NotNil(t, provider)
tests := map[string]struct {
tests := []struct {
name string
node *apiv1.Node
pods []*apiv1.Pod
expectedTaints []apiv1.Taint
}{
"Node from selected node groups can get their deletion candidate taints removed": {
{
name: "Node from selected node groups can get their deletion candidate taints removed",
node: n1,
pods: []*apiv1.Pod{p1},
expectedTaints: []apiv1.Taint{},
},
"Node from non-selected node groups should keep their deletion candidate taints": {
{
name: "Node from non-selected node groups should keep their deletion candidate taints",
node: n2,
pods: nil,
expectedTaints: n2.Spec.Taints,
},
}
for name, test := range tests {
// prevent issues with scoping, we should be able to get rid of that with Go 1.22
test := test
t.Run(name, func(t *testing.T) {
for _, test := range tests {
t.Run(test.name, func(t *testing.T) {
t.Parallel()
// Create fake listers for the generated nodes, nothing returned by the rest (but the ones used in the tested path have to be defined).
readyNodeLister := kubernetes.NewTestNodeLister([]*apiv1.Node{test.node})
allNodeLister := kubernetes.NewTestNodeLister([]*apiv1.Node{test.node})
allPodListerMock := kubernetes.NewTestPodLister(test.pods)
daemonSetLister, err := kubernetes.NewTestDaemonSetLister(nil)
assert.NoError(t, err)
listerRegistry := kube_util.NewListerRegistry(allNodeLister, readyNodeLister, allPodListerMock,
kubernetes.NewTestPodDisruptionBudgetLister(nil), daemonSetLister,
nil, nil, nil, nil)
// Create context with minimal autoscalingOptions that guarantee we reach the tested logic.
autoscalingOptions := config.AutoscalingOptions{
ScaleDownEnabled: true,
MaxBulkSoftTaintCount: 10,
MaxBulkSoftTaintTime: 3 * time.Second,
}
processorCallbacks := newStaticAutoscalerProcessorCallbacks()
clientset := fake.NewSimpleClientset(test.node)
context, err := NewScaleTestAutoscalingContext(autoscalingOptions, clientset, listerRegistry, provider, processorCallbacks, nil)
assert.NoError(t, err)
// Create CSR with unhealthy cluster protection effectively disabled, to guarantee we reach the tested logic.
clusterStateConfig := clusterstate.ClusterStateRegistryConfig{
OkTotalUnreadyCount: 1,
}
processors := NewTestProcessors(&context)
clusterState := clusterstate.NewClusterStateRegistry(provider, clusterStateConfig, context.LogRecorder, NewBackoff(), nodegroupconfig.NewDefaultNodeGroupConfigProcessor(autoscalingOptions.NodeGroupDefaults), processors.AsyncNodeGroupStateChecker)
// Setting the Actuator is necessary for testing any scale-down logic, it shouldn't have anything to do in this test.
sdActuator := actuation.NewActuator(&context, clusterState, deletiontracker.NewNodeDeletionTracker(0*time.Second), options.NodeDeleteOptions{}, nil, processors.NodeGroupConfigProcessor)
context.ScaleDownActuator = sdActuator
// Fake planner that keeps track of the scale-down candidates passed to UpdateClusterState.
sdPlanner := &candidateTrackingFakePlanner{}
autoscaler := &StaticAutoscaler{
AutoscalingContext: &context,
clusterStateRegistry: clusterState,
scaleDownPlanner: sdPlanner,
scaleDownActuator: sdActuator,
processors: processors,
loopStartNotifier: loopstart.NewObserversList(nil),
processorCallbacks: processorCallbacks,
}
err = autoscaler.RunOnce(time.Now().Add(5 * time.Hour))
assert.NoError(t, err)
newNode, err := clientset.CoreV1().Nodes().Get(stdcontext.TODO(), test.node.Name, metav1.GetOptions{})
allNodes := []*apiv1.Node{test.node}
fakeClient := buildFakeClient(t, allNodes...)
autoscaler := buildStaticAutoscaler(t, provider, allNodes, allNodes, fakeClient)
runningTime := time.Now()
err := autoscaler.RunOnce(runningTime)
assert.NoError(t, err)
newNode, clientErr := fakeClient.CoreV1().Nodes().Get(stdcontext.TODO(), test.node.Name, metav1.GetOptions{})
assert.NoError(t, clientErr)
assert.Equal(t, test.expectedTaints, newNode.Spec.Taints)
})
}
@ -2677,3 +2637,162 @@ func newEstimatorBuilder() estimator.EstimatorBuilder {
return estimatorBuilder
}
func TestCleaningSoftTaintsInScaleDown(t *testing.T) {
provider := testprovider.NewTestCloudProvider(nil, nil)
minSizeNgName := "ng-min-size"
nodesToHaveNoTaints := createNodeGroupWithSoftTaintedNodes(provider, minSizeNgName, 2, 10, 2)
notSizeNgName := "ng"
nodesToHaveTaints := createNodeGroupWithSoftTaintedNodes(provider, notSizeNgName, 3, 10, 4)
tests := []struct {
name string
testNodes []*apiv1.Node
scaleDownInCoolDown bool
expectedNodesWithSoftTaints []*apiv1.Node
expectedNodesWithNoSoftTaints []*apiv1.Node
}{
{
name: "Soft tainted nodes are cleaned when scale down skipped",
testNodes: nodesToHaveNoTaints,
scaleDownInCoolDown: false,
expectedNodesWithSoftTaints: []*apiv1.Node{},
expectedNodesWithNoSoftTaints: nodesToHaveNoTaints,
},
{
name: "Soft tainted nodes are cleaned when scale down in cooldown",
testNodes: nodesToHaveNoTaints,
scaleDownInCoolDown: true,
expectedNodesWithSoftTaints: []*apiv1.Node{},
expectedNodesWithNoSoftTaints: nodesToHaveNoTaints,
},
{
name: "Soft tainted nodes are not cleaned when scale down requested",
testNodes: nodesToHaveTaints,
scaleDownInCoolDown: false,
expectedNodesWithSoftTaints: nodesToHaveTaints,
expectedNodesWithNoSoftTaints: []*apiv1.Node{},
},
{
name: "Soft tainted nodes are cleaned only from min sized node group when scale down requested",
testNodes: append(nodesToHaveNoTaints, nodesToHaveTaints...),
scaleDownInCoolDown: false,
expectedNodesWithSoftTaints: nodesToHaveTaints,
expectedNodesWithNoSoftTaints: nodesToHaveNoTaints,
},
}
for _, test := range tests {
t.Run(test.name, func(t *testing.T) {
t.Parallel()
fakeClient := buildFakeClient(t, test.testNodes...)
autoscaler := buildStaticAutoscaler(t, provider, test.testNodes, test.testNodes, fakeClient)
autoscaler.processorCallbacks.disableScaleDownForLoop = test.scaleDownInCoolDown
assert.Equal(t, autoscaler.isScaleDownInCooldown(time.Now()), test.scaleDownInCoolDown)
err := autoscaler.RunOnce(time.Now())
assert.NoError(t, err)
assertNodesSoftTaintsStatus(t, fakeClient, test.expectedNodesWithSoftTaints, true)
assertNodesSoftTaintsStatus(t, fakeClient, test.expectedNodesWithNoSoftTaints, false)
})
}
}
func buildStaticAutoscaler(t *testing.T, provider cloudprovider.CloudProvider, allNodes []*apiv1.Node, readyNodes []*apiv1.Node, fakeClient *fake.Clientset) *StaticAutoscaler {
autoscalingOptions := config.AutoscalingOptions{
NodeGroupDefaults: config.NodeGroupAutoscalingOptions{
ScaleDownUnneededTime: time.Minute,
ScaleDownUnreadyTime: time.Minute,
ScaleDownUtilizationThreshold: 0.5,
MaxNodeProvisionTime: 10 * time.Second,
},
MaxScaleDownParallelism: 10,
MaxDrainParallelism: 1,
ScaleDownEnabled: true,
MaxBulkSoftTaintCount: 20,
MaxBulkSoftTaintTime: 5 * time.Second,
NodeDeleteDelayAfterTaint: 5 * time.Minute,
ScaleDownSimulationTimeout: 10 * time.Second,
}
allNodeLister := kubernetes.NewTestNodeLister(allNodes)
readyNodeLister := kubernetes.NewTestNodeLister(readyNodes)
daemonSetLister, err := kubernetes.NewTestDaemonSetLister(nil)
assert.NoError(t, err)
listerRegistry := kube_util.NewListerRegistry(allNodeLister, readyNodeLister,
kubernetes.NewTestPodLister(nil),
kubernetes.NewTestPodDisruptionBudgetLister(nil), daemonSetLister, nil, nil, nil, nil)
processorCallbacks := newStaticAutoscalerProcessorCallbacks()
ctx, err := NewScaleTestAutoscalingContext(autoscalingOptions, fakeClient, listerRegistry, provider, processorCallbacks, nil)
assert.NoError(t, err)
processors := NewTestProcessors(&ctx)
cp := scaledowncandidates.NewCombinedScaleDownCandidatesProcessor()
cp.Register(scaledowncandidates.NewScaleDownCandidatesSortingProcessor([]scaledowncandidates.CandidatesComparer{}))
processors.ScaleDownNodeProcessor = cp
csr := clusterstate.NewClusterStateRegistry(provider, clusterstate.ClusterStateRegistryConfig{OkTotalUnreadyCount: 1}, ctx.LogRecorder, NewBackoff(), nodegroupconfig.NewDefaultNodeGroupConfigProcessor(config.NodeGroupAutoscalingOptions{MaxNodeProvisionTime: 15 * time.Minute}), processors.AsyncNodeGroupStateChecker)
actuator := actuation.NewActuator(&ctx, csr, deletiontracker.NewNodeDeletionTracker(0*time.Second), options.NodeDeleteOptions{}, nil, processors.NodeGroupConfigProcessor)
ctx.ScaleDownActuator = actuator
deleteOptions := options.NewNodeDeleteOptions(ctx.AutoscalingOptions)
drainabilityRules := rules.Default(deleteOptions)
sdPlanner := planner.New(&ctx, processors, deleteOptions, drainabilityRules)
autoscaler := &StaticAutoscaler{
AutoscalingContext: &ctx,
clusterStateRegistry: csr,
scaleDownActuator: actuator,
scaleDownPlanner: sdPlanner,
processors: processors,
loopStartNotifier: loopstart.NewObserversList(nil),
processorCallbacks: processorCallbacks,
}
return autoscaler
}
func buildFakeClient(t *testing.T, nodes ...*apiv1.Node) *fake.Clientset {
fakeClient := fake.NewSimpleClientset()
for _, node := range nodes {
_, err := fakeClient.CoreV1().Nodes().Create(stdcontext.TODO(), node, metav1.CreateOptions{})
assert.NoError(t, err)
}
return fakeClient
}
func createNodeGroupWithSoftTaintedNodes(provider *testprovider.TestCloudProvider, name string, minSize int, maxSize int, size int) []*apiv1.Node {
nodesCreationTime := time.Time{}
var ngNodes []*apiv1.Node
ng := provider.BuildNodeGroup(name, minSize, maxSize, size, true, false, "", nil)
provider.InsertNodeGroup(ng)
for i := range size {
node := BuildTestNode(fmt.Sprintf("%s-node-%d", name, i), 2000, 1000)
node.CreationTimestamp = metav1.NewTime(nodesCreationTime)
node.Spec.Taints = []apiv1.Taint{{
Key: taints.DeletionCandidateTaint,
Value: "1",
Effect: apiv1.TaintEffectNoSchedule,
}}
SetNodeReadyState(node, true, nodesCreationTime)
ngNodes = append(ngNodes, node)
provider.AddNode(ng.Id(), node)
}
return ngNodes
}
func assertNodesSoftTaintsStatus(t *testing.T, fakeClient *fake.Clientset, nodes []*apiv1.Node, tainted bool) {
for _, node := range nodes {
newNode, clientErr := fakeClient.CoreV1().Nodes().Get(stdcontext.TODO(), node.Name, metav1.GetOptions{})
assert.NoError(t, clientErr)
assert.Equal(t, tainted, taints.HasDeletionCandidateTaint(newNode))
}
}