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:
commit
31422a8495
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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))
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue