From 4e4804487440434a50e520bd494117ecd83d946d Mon Sep 17 00:00:00 2001 From: Olzhas Shaikenov Date: Fri, 3 Oct 2025 12:04:07 +0000 Subject: [PATCH] feat(nodeScaleDownTimeTracker): add a new metric to track unprocessed nodes during scaleDown --- .../config/autoscaling_options.go | 5 ++ cluster-autoscaler/config/flags/flags.go | 2 + .../longest_node_evaluation_tracker.go | 82 +++++++++++++++++++ .../longest_node_evaluation_tracker_test.go | 65 +++++++++++++++ .../core/scaledown/planner/planner.go | 71 ++++++++++------ .../core/scaledown/planner/planner_test.go | 80 ++++++++++++++++++ cluster-autoscaler/metrics/metrics.go | 17 ++++ 7 files changed, 297 insertions(+), 25 deletions(-) create mode 100644 cluster-autoscaler/core/scaledown/longestevaluationtracker/longest_node_evaluation_tracker.go create mode 100644 cluster-autoscaler/core/scaledown/longestevaluationtracker/longest_node_evaluation_tracker_test.go diff --git a/cluster-autoscaler/config/autoscaling_options.go b/cluster-autoscaler/config/autoscaling_options.go index d910447eb12e..1b75b7f1ae41 100644 --- a/cluster-autoscaler/config/autoscaling_options.go +++ b/cluster-autoscaler/config/autoscaling_options.go @@ -352,6 +352,11 @@ type AutoscalingOptions struct { CapacitybufferControllerEnabled bool // CapacitybufferPodInjectionEnabled tells if CA should injects fake pods for capacity buffers that are ready for provisioning CapacitybufferPodInjectionEnabled bool + // LongestNodeScaleDownEvalTimeTrackerEnabled is used to enabled/disable the tracking of longest node ScaleDown evaluation time. + // We want to track all the nodes that were present in currentlyUnneededNodeNames, but were neither processed nor deleted during the ScaleDown. + // If it happened to a node multiple times consecutively, we store only the earliest time it happened. + // The difference between the current time and the earliest time among all unprocessed nodes will give the longest evaluation time + LongestNodeScaleDownEvalTimeTrackerEnabled bool } // KubeClientOptions specify options for kube client diff --git a/cluster-autoscaler/config/flags/flags.go b/cluster-autoscaler/config/flags/flags.go index ffb79bab4f05..1ee36cf60429 100644 --- a/cluster-autoscaler/config/flags/flags.go +++ b/cluster-autoscaler/config/flags/flags.go @@ -231,6 +231,7 @@ var ( nodeDeletionCandidateTTL = flag.Duration("node-deletion-candidate-ttl", time.Duration(0), "Maximum time a node can be marked as removable before the marking becomes stale. This sets the TTL of Cluster-Autoscaler's state if the Cluste-Autoscaler deployment becomes inactive") capacitybufferControllerEnabled = flag.Bool("capacity-buffer-controller-enabled", false, "Whether to enable the default controller for capacity buffers or not") capacitybufferPodInjectionEnabled = flag.Bool("capacity-buffer-pod-injection-enabled", false, "Whether to enable pod list processor that processes ready capacity buffers and injects fake pods accordingly") + longestNodeScaleDownEvalTimeTrackerEnabled = flag.Bool("longest-node-scaledown-eval-timetracker-enabled", false, "Whether to enable the tracking of the longest node ScaleDown evaluation time") // Deprecated flags ignoreTaintsFlag = multiStringFlag("ignore-taint", "Specifies a taint to ignore in node templates when considering to scale a node group (Deprecated, use startup-taints instead)") @@ -416,6 +417,7 @@ func createAutoscalingOptions() config.AutoscalingOptions { NodeDeletionCandidateTTL: *nodeDeletionCandidateTTL, CapacitybufferControllerEnabled: *capacitybufferControllerEnabled, CapacitybufferPodInjectionEnabled: *capacitybufferPodInjectionEnabled, + LongestNodeScaleDownEvalTimeTrackerEnabled: *longestNodeScaleDownEvalTimeTrackerEnabled, } } diff --git a/cluster-autoscaler/core/scaledown/longestevaluationtracker/longest_node_evaluation_tracker.go b/cluster-autoscaler/core/scaledown/longestevaluationtracker/longest_node_evaluation_tracker.go new file mode 100644 index 000000000000..3eb26cfe1000 --- /dev/null +++ b/cluster-autoscaler/core/scaledown/longestevaluationtracker/longest_node_evaluation_tracker.go @@ -0,0 +1,82 @@ +/* +Copyright 2025 The Kubernetes Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package longestevaluationtracker + +import ( + "time" + + "k8s.io/autoscaler/cluster-autoscaler/metrics" +) + +// LongestNodeScaleDownEvalTime is a time tracker for the longest evaluation time of a node during ScaleDown +type LongestNodeScaleDownEvalTime struct { + // lastEvalTime is the time of previous currentlyUnneededNodeNames parsing + lastEvalTime time.Time + // NodeNamesWithTimeStamps is maps of nodeNames with their time of last successful evaluation + NodeNamesWithTimeStamps map[string]time.Time +} + +// NewLongestNodeScaleDownEvalTime returns LongestNodeScaleDownEvalTime with lastEvalTime set to currentTime +func NewLongestNodeScaleDownEvalTime(currentTime time.Time) *LongestNodeScaleDownEvalTime { + return &LongestNodeScaleDownEvalTime{lastEvalTime: currentTime} +} + +// Retrieves the time of the last evaluation of a node. +func (l *LongestNodeScaleDownEvalTime) get(nodeName string) time.Time { + if _, ok := l.NodeNamesWithTimeStamps[nodeName]; ok { + return l.NodeNamesWithTimeStamps[nodeName] + } + return l.lastEvalTime +} + +// getMin() returns the minimum time in NodeNamesWithTimeStamps or time of last evaluation +func (l *LongestNodeScaleDownEvalTime) getMin() time.Time { + minimumTime := l.lastEvalTime + for _, val := range l.NodeNamesWithTimeStamps { + if minimumTime.After(val) { + minimumTime = val + } + } + return minimumTime +} + +// Update returns the longest evaluation time for the nodes in NodeNamesWithTimeStamps +// and changes NodeNamesWithTimeStamps for nodeNames. +func (l *LongestNodeScaleDownEvalTime) Update(nodeNames []string, currentTime time.Time) time.Duration { + var longestTime time.Duration + minimumTime := l.getMin() + if len(nodeNames) == 0 { + // if l.minimumTime is lastEvalTime, then in previous iteration we also processed all the nodes, so the longest time is 0 + // otherwise -> report the longest time from previous iteration and reset the minimumTime + if minimumTime.Equal(l.lastEvalTime) { + longestTime = 0 + } else { + longestTime = currentTime.Sub(minimumTime) + } + l.NodeNamesWithTimeStamps = make(map[string]time.Time) + } else { + newNodes := make(map[string]time.Time, len(nodeNames)) + for _, nodeName := range nodeNames { + newNodes[nodeName] = l.get(nodeName) + } + l.NodeNamesWithTimeStamps = newNodes + longestTime = currentTime.Sub(minimumTime) + } + l.lastEvalTime = currentTime + metrics.ObserveLongestUnneededNodeScaleDownEvalDurationSeconds(longestTime) + return longestTime +} diff --git a/cluster-autoscaler/core/scaledown/longestevaluationtracker/longest_node_evaluation_tracker_test.go b/cluster-autoscaler/core/scaledown/longestevaluationtracker/longest_node_evaluation_tracker_test.go new file mode 100644 index 000000000000..dcee6f3b9aad --- /dev/null +++ b/cluster-autoscaler/core/scaledown/longestevaluationtracker/longest_node_evaluation_tracker_test.go @@ -0,0 +1,65 @@ +/* +Copyright 2025 The Kubernetes Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package longestevaluationtracker + +import ( + "testing" + "time" + + "github.com/stretchr/testify/assert" +) + +func TestLongestUnprocessedNodeScaleDownTime(t *testing.T) { + type testCase struct { + name string + unprocessedNodes [][]string + wantLongestScaleDownEvalTime []time.Duration + } + start := time.Now() + testCases := []testCase{ + { + name: "All nodes processed in the first iteration", + unprocessedNodes: [][]string{nil, {"n1", "n2"}, {"n2", "n3"}, {}, {}}, + wantLongestScaleDownEvalTime: []time.Duration{time.Duration(0), time.Duration(1 * time.Second), time.Duration(2 * time.Second), + time.Duration(3 * time.Second), time.Duration(0)}, + }, + { + name: "Not all nodes processed in the first iteration", + unprocessedNodes: [][]string{{"n1", "n2"}, {"n1", "n2"}, {"n2", "n3"}, {}, {}}, + wantLongestScaleDownEvalTime: []time.Duration{time.Duration(1 * time.Second), time.Duration(2 * time.Second), time.Duration(3 * time.Second), + time.Duration(4 * time.Second), time.Duration(0)}, + }, + { + name: "Different nodes processed in each iteration", + unprocessedNodes: [][]string{{"n1"}, {"n2"}, {"n3"}, {"n4"}, {}, {}}, + wantLongestScaleDownEvalTime: []time.Duration{time.Duration(1 * time.Second), time.Duration(2 * time.Second), time.Duration(2 * time.Second), + time.Duration(2 * time.Second), time.Duration(2 * time.Second), time.Duration(0)}, + }, + } + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + t.Parallel() + timestamp := start + longestScaleDownEvalT := NewLongestNodeScaleDownEvalTime(start) + for i := 0; i < len(tc.unprocessedNodes); i++ { + timestamp = timestamp.Add(1 * time.Second) + assert.Equal(t, tc.wantLongestScaleDownEvalTime[i], longestScaleDownEvalT.Update(tc.unprocessedNodes[i], timestamp)) + assert.Equal(t, len(tc.unprocessedNodes[i]), len(longestScaleDownEvalT.NodeNamesWithTimeStamps)) + } + }) + } +} diff --git a/cluster-autoscaler/core/scaledown/planner/planner.go b/cluster-autoscaler/core/scaledown/planner/planner.go index 6767d9ddfb4b..253c381eee86 100644 --- a/cluster-autoscaler/core/scaledown/planner/planner.go +++ b/cluster-autoscaler/core/scaledown/planner/planner.go @@ -26,6 +26,7 @@ import ( ca_context "k8s.io/autoscaler/cluster-autoscaler/context" "k8s.io/autoscaler/cluster-autoscaler/core/scaledown" "k8s.io/autoscaler/cluster-autoscaler/core/scaledown/eligibility" + "k8s.io/autoscaler/cluster-autoscaler/core/scaledown/longestevaluationtracker" "k8s.io/autoscaler/cluster-autoscaler/core/scaledown/pdb" "k8s.io/autoscaler/cluster-autoscaler/core/scaledown/resource" "k8s.io/autoscaler/cluster-autoscaler/core/scaledown/unneeded" @@ -63,19 +64,20 @@ type replicasInfo struct { // Planner is responsible for deciding which nodes should be deleted during scale down. type Planner struct { - autoscalingCtx *ca_context.AutoscalingContext - unremovableNodes *unremovable.Nodes - unneededNodes *unneeded.Nodes - rs removalSimulator - actuationInjector *scheduling.HintingSimulator - latestUpdate time.Time - minUpdateInterval time.Duration - eligibilityChecker eligibilityChecker - nodeUtilizationMap map[string]utilization.Info - resourceLimitsFinder *resource.LimitsFinder - cc controllerReplicasCalculator - scaleDownSetProcessor nodes.ScaleDownSetProcessor - scaleDownContext *nodes.ScaleDownContext + autoscalingCtx *ca_context.AutoscalingContext + unremovableNodes *unremovable.Nodes + unneededNodes *unneeded.Nodes + rs removalSimulator + actuationInjector *scheduling.HintingSimulator + latestUpdate time.Time + minUpdateInterval time.Duration + eligibilityChecker eligibilityChecker + nodeUtilizationMap map[string]utilization.Info + resourceLimitsFinder *resource.LimitsFinder + cc controllerReplicasCalculator + scaleDownSetProcessor nodes.ScaleDownSetProcessor + scaleDownContext *nodes.ScaleDownContext + longestNodeScaleDownEvalTime *longestevaluationtracker.LongestNodeScaleDownEvalTime } // New creates a new Planner object. @@ -91,19 +93,25 @@ func New(autoscalingCtx *ca_context.AutoscalingContext, processors *processors.A unneededNodes.LoadFromExistingTaints(autoscalingCtx.ListerRegistry, time.Now(), autoscalingCtx.AutoscalingOptions.NodeDeletionCandidateTTL) } + var longestNodeScaleDownEvalTime *longestevaluationtracker.LongestNodeScaleDownEvalTime + if autoscalingCtx.AutoscalingOptions.LongestNodeScaleDownEvalTimeTrackerEnabled { + longestNodeScaleDownEvalTime = longestevaluationtracker.NewLongestNodeScaleDownEvalTime(time.Now()) + } + return &Planner{ - autoscalingCtx: autoscalingCtx, - unremovableNodes: unremovable.NewNodes(), - unneededNodes: unneededNodes, - rs: simulator.NewRemovalSimulator(autoscalingCtx.ListerRegistry, autoscalingCtx.ClusterSnapshot, deleteOptions, drainabilityRules, true), - actuationInjector: scheduling.NewHintingSimulator(), - eligibilityChecker: eligibility.NewChecker(processors.NodeGroupConfigProcessor), - nodeUtilizationMap: make(map[string]utilization.Info), - resourceLimitsFinder: resourceLimitsFinder, - cc: newControllerReplicasCalculator(autoscalingCtx.ListerRegistry), - scaleDownSetProcessor: processors.ScaleDownSetProcessor, - scaleDownContext: nodes.NewDefaultScaleDownContext(), - minUpdateInterval: minUpdateInterval, + autoscalingCtx: autoscalingCtx, + unremovableNodes: unremovable.NewNodes(), + unneededNodes: unneededNodes, + rs: simulator.NewRemovalSimulator(autoscalingCtx.ListerRegistry, autoscalingCtx.ClusterSnapshot, deleteOptions, drainabilityRules, true), + actuationInjector: scheduling.NewHintingSimulator(), + eligibilityChecker: eligibility.NewChecker(processors.NodeGroupConfigProcessor), + nodeUtilizationMap: make(map[string]utilization.Info), + resourceLimitsFinder: resourceLimitsFinder, + cc: newControllerReplicasCalculator(autoscalingCtx.ListerRegistry), + scaleDownSetProcessor: processors.ScaleDownSetProcessor, + scaleDownContext: nodes.NewDefaultScaleDownContext(), + minUpdateInterval: minUpdateInterval, + longestNodeScaleDownEvalTime: longestNodeScaleDownEvalTime, } } @@ -277,13 +285,16 @@ func (p *Planner) categorizeNodes(podDestinations map[string]bool, scaleDownCand } p.nodeUtilizationMap = utilizationMap timer := time.NewTimer(p.autoscalingCtx.ScaleDownSimulationTimeout) + var skippedNodes []string for i, node := range currentlyUnneededNodeNames { if timedOut(timer) { + skippedNodes = currentlyUnneededNodeNames[i:] klog.Warningf("%d out of %d nodes skipped in scale down simulation due to timeout.", len(currentlyUnneededNodeNames)-i, len(currentlyUnneededNodeNames)) break } if len(removableList)-atomicScaleDownNodesCount >= p.unneededNodesLimit() { + skippedNodes = currentlyUnneededNodeNames[i:] klog.V(4).Infof("%d out of %d nodes skipped in scale down simulation: there are already %d unneeded nodes so no point in looking for more. Total atomic scale down nodes: %d", len(currentlyUnneededNodeNames)-i, len(currentlyUnneededNodeNames), len(removableList), atomicScaleDownNodesCount) break } @@ -306,6 +317,7 @@ func (p *Planner) categorizeNodes(podDestinations map[string]bool, scaleDownCand p.unremovableNodes.AddTimeout(unremovable, unremovableTimeout) } } + p.handleUnprocessedNodes(skippedNodes) p.unneededNodes.Update(removableList, p.latestUpdate) if unremovableCount > 0 { klog.V(1).Infof("%v nodes found to be unremovable in simulation, will re-check them at %v", unremovableCount, unremovableTimeout) @@ -372,6 +384,15 @@ func (p *Planner) unneededNodesLimit() int { return limit } +// handleUnprocessedNodes is used to track the longest time it take for a node to be evaluated as removable or not +func (p *Planner) handleUnprocessedNodes(unprocessedNodeNames []string) { + // if p.longestNodeScaleDownEvalTime is not set (flag is disabled) or endedPrematurely is already true (nodes were already reported in this iteration) do not do anything + if p.longestNodeScaleDownEvalTime == nil { + return + } + p.longestNodeScaleDownEvalTime.Update(unprocessedNodeNames, time.Now()) +} + // getKnownOwnerRef returns ownerRef that is known by CA and CA knows the logic of how this controller recreates pods. func getKnownOwnerRef(ownerRefs []metav1.OwnerReference) *metav1.OwnerReference { for _, ownerRef := range ownerRefs { diff --git a/cluster-autoscaler/core/scaledown/planner/planner_test.go b/cluster-autoscaler/core/scaledown/planner/planner_test.go index 7e3036ae1763..150bd5c4bdfc 100644 --- a/cluster-autoscaler/core/scaledown/planner/planner_test.go +++ b/cluster-autoscaler/core/scaledown/planner/planner_test.go @@ -1035,6 +1035,86 @@ func TestNodesToDelete(t *testing.T) { } } +func TestLongestUnprocessedNodeScaleDown(t *testing.T) { + type testCase struct { + name string + maxParallel int + isSimulationTimeout bool + isFlagEnabled bool + wantUnprocessedNodes int + } + nodes := []*apiv1.Node{ + BuildTestNode("n1", 1000, 10), + BuildTestNode("n2", 1000, 10), + BuildTestNode("n3", 1000, 10), + } + eligible := []string{"n1", "n2"} + registry := kube_util.NewListerRegistry(nil, nil, nil, nil, nil, nil, nil, nil, nil) + provider := testprovider.NewTestCloudProviderBuilder().Build() + provider.AddNodeGroup("ng1", 0, 0, 0) + for _, node := range nodes { + provider.AddNode("ng1", node) + } + testCases := []testCase{ + { + name: "Unneeded node limit is exceeded", + maxParallel: 0, + isSimulationTimeout: false, + isFlagEnabled: true, + // maxParallel=0 forces p.unneededNodesLimit() to be 0, so we will break in the second check inside p.categorizeNodes() right away + wantUnprocessedNodes: 2, + }, + { + name: "Simulation timeout is hit", + maxParallel: 1, + isSimulationTimeout: true, + isFlagEnabled: true, + // first node will be deleted and for the second timeout will be triggered + wantUnprocessedNodes: 1, + }, + { + name: "LongestNodeScaleDownEvalTimeTrackerEnabled flag is disabled", + maxParallel: 1, + isSimulationTimeout: false, + isFlagEnabled: false, + }, + } + for _, tc := range testCases { + tc := tc + t.Run(tc.name, func(t *testing.T) { + t.Parallel() + autoscalingCtx, err := NewScaleTestAutoscalingContext(config.AutoscalingOptions{ + NodeGroupDefaults: config.NodeGroupAutoscalingOptions{ + ScaleDownUnneededTime: 10 * time.Minute, + }, + ScaleDownSimulationTimeout: 1 * time.Second, + MaxScaleDownParallelism: tc.maxParallel, + LongestNodeScaleDownEvalTimeTrackerEnabled: tc.isFlagEnabled, + }, &fake.Clientset{}, registry, provider, nil, nil) + assert.NoError(t, err) + clustersnapshot.InitializeClusterSnapshotOrDie(t, autoscalingCtx.ClusterSnapshot, nodes, nil) + deleteOptions := options.NodeDeleteOptions{} + p := New(&autoscalingCtx, processorstest.NewTestProcessors(&autoscalingCtx), deleteOptions, nil) + p.eligibilityChecker = &fakeEligibilityChecker{eligible: asMap(eligible)} + if tc.isSimulationTimeout { + autoscalingCtx.AutoscalingOptions.ScaleDownSimulationTimeout = 1 * time.Second + rs := &fakeRemovalSimulator{ + nodes: nodes, + sleep: 2 * time.Second, + } + p.rs = rs + } + assert.NoError(t, p.UpdateClusterState(nodes, nodes, &fakeActuationStatus{}, time.Now())) + if !tc.isFlagEnabled { + // if flag is disabled p.longestNodeScaleDownEvalTime is not initialized + assert.Nil(t, p.longestNodeScaleDownEvalTime) + } else { + assert.Equal(t, tc.wantUnprocessedNodes, len(p.longestNodeScaleDownEvalTime.NodeNamesWithTimeStamps)) + } + }) + } +} + func sizedNodeGroup(id string, size int, atomic bool) cloudprovider.NodeGroup { ng := testprovider.NewTestNodeGroup(id, 10000, 0, size, true, false, "n1-standard-2", nil, nil) ng.SetOptions(&config.NodeGroupAutoscalingOptions{ diff --git a/cluster-autoscaler/metrics/metrics.go b/cluster-autoscaler/metrics/metrics.go index 6739ebb4d2b7..803db129678d 100644 --- a/cluster-autoscaler/metrics/metrics.go +++ b/cluster-autoscaler/metrics/metrics.go @@ -425,6 +425,14 @@ var ( Buckets: k8smetrics.ExponentialBuckets(1, 2, 6), // 1, 2, 4, ..., 32 }, []string{"instance_type", "cpu_count", "namespace_count"}, ) + + longestUnneededNodeScaleDownEvalDurationSeconds = k8smetrics.NewGauge( + &k8smetrics.GaugeOpts{ + Namespace: caNamespace, + Name: "longest_unneeded_node_scale_down_eval_duration_seconds", + Help: "Longest unneeded node evaluation time during ScaleDown.", + }, + ) ) // RegisterAll registers all metrics. @@ -461,6 +469,7 @@ func RegisterAll(emitPerNodeGroupMetrics bool) { legacyregistry.MustRegister(nodeTaintsCount) legacyregistry.MustRegister(inconsistentInstancesMigsCount) legacyregistry.MustRegister(binpackingHeterogeneity) + legacyregistry.MustRegister(longestUnneededNodeScaleDownEvalDurationSeconds) if emitPerNodeGroupMetrics { legacyregistry.MustRegister(nodesGroupMinNodes) @@ -748,3 +757,11 @@ func UpdateInconsistentInstancesMigsCount(migCount int) { func ObserveBinpackingHeterogeneity(instanceType, cpuCount, namespaceCount string, pegCount int) { binpackingHeterogeneity.WithLabelValues(instanceType, cpuCount, namespaceCount).Observe(float64(pegCount)) } + +// ObserveLongestUnneededNodeScaleDownEvalDurationSeconds records the longest time during which node was not processed during ScaleDown. +// If a node is not processed multiple times consecutively, we store only the earliest timestamp. +// Here we report the difference between current time and the earliest time among all unprocessed nodes in current ScaleDown iteration +// If we never timedOut in categorizeNodes() or never exceeded p.unneededNodesLimit(), this value will be 0 +func ObserveLongestUnneededNodeScaleDownEvalDurationSeconds(duration time.Duration) { + longestUnneededNodeScaleDownEvalDurationSeconds.Set(duration.Seconds()) +}