Skip to content
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
5 changes: 5 additions & 0 deletions cluster-autoscaler/config/autoscaling_options.go
Original file line number Diff line number Diff line change
Expand Up @@ -349,6 +349,11 @@ type AutoscalingOptions struct {
CapacitybufferControllerEnabled bool
// CapacitybufferPodInjectionEnabled tells if CA should injects fake pods for capacity buffers that are ready for provisioning
CapacitybufferPodInjectionEnabled bool
// LongestNodeScaleDownTimeTrackerEnabled is used to enabled/disable the tracking of longest node ScaleDown evaluation time.
// We want to track all the nodes that were marked as unneeded, but were unprocessed and not deleted during the ScaleDown.
// If a node was unneeded, but unprocessed 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 time
LongestNodeScaleDownEvalTimeTrackerEnabled bool
}

// KubeClientOptions specify options for kube client
Expand Down
2 changes: 2 additions & 0 deletions cluster-autoscaler/config/flags/flags.go
Original file line number Diff line number Diff line change
Expand Up @@ -230,6 +230,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-timetracker-enabled", false, "Whether to track the eval time of longestNodeScaleDown")

// 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)")
Expand Down Expand Up @@ -414,6 +415,7 @@ func createAutoscalingOptions() config.AutoscalingOptions {
NodeDeletionCandidateTTL: *nodeDeletionCandidateTTL,
CapacitybufferControllerEnabled: *capacitybufferControllerEnabled,
CapacitybufferPodInjectionEnabled: *capacitybufferPodInjectionEnabled,
LongestNodeScaleDownEvalTimeTrackerEnabled: *longestNodeScaleDownEvalTimeTrackerEnabled,
}
}

Expand Down
73 changes: 73 additions & 0 deletions cluster-autoscaler/core/scaledown/planner/planner.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import (
"k8s.io/autoscaler/cluster-autoscaler/core/scaledown/resource"
"k8s.io/autoscaler/cluster-autoscaler/core/scaledown/unneeded"
"k8s.io/autoscaler/cluster-autoscaler/core/scaledown/unremovable"
"k8s.io/autoscaler/cluster-autoscaler/metrics"
"k8s.io/autoscaler/cluster-autoscaler/processors"
"k8s.io/autoscaler/cluster-autoscaler/processors/nodes"
"k8s.io/autoscaler/cluster-autoscaler/simulator"
Expand Down Expand Up @@ -76,6 +77,7 @@ type Planner struct {
cc controllerReplicasCalculator
scaleDownSetProcessor nodes.ScaleDownSetProcessor
scaleDownContext *nodes.ScaleDownContext
longestNodeScaleDownT *longestNodeScaleDownEvalTime
}

// New creates a new Planner object.
Expand All @@ -91,6 +93,11 @@ func New(autoscalingCtx *ca_context.AutoscalingContext, processors *processors.A
unneededNodes.LoadFromExistingTaints(autoscalingCtx.ListerRegistry, time.Now(), autoscalingCtx.AutoscalingOptions.NodeDeletionCandidateTTL)
}

var longestNodeScaleDownTime *longestNodeScaleDownEvalTime
if autoscalingCtx.AutoscalingOptions.LongestNodeScaleDownEvalTimeTrackerEnabled {
longestNodeScaleDownTime = newLongestNodeScaleDownEvalTime(time.Now())
}

return &Planner{
autoscalingCtx: autoscalingCtx,
unremovableNodes: unremovable.NewNodes(),
Expand All @@ -104,6 +111,7 @@ func New(autoscalingCtx *ca_context.AutoscalingContext, processors *processors.A
scaleDownSetProcessor: processors.ScaleDownSetProcessor,
scaleDownContext: nodes.NewDefaultScaleDownContext(),
minUpdateInterval: minUpdateInterval,
longestNodeScaleDownT: longestNodeScaleDownTime,
}
}

Expand Down Expand Up @@ -277,13 +285,16 @@ func (p *Planner) categorizeNodes(podDestinations map[string]bool, scaleDownCand
}
p.nodeUtilizationMap = utilizationMap
timer := time.NewTimer(p.autoscalingCtx.ScaleDownSimulationTimeout)
endedPrematurely := false

for i, node := range currentlyUnneededNodeNames {
if timedOut(timer) {
p.handleUnprocessedNodes(currentlyUnneededNodeNames[i:], time.Now(), &endedPrematurely)
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() {
p.handleUnprocessedNodes(currentlyUnneededNodeNames[i:], time.Now(), &endedPrematurely)
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
}
Expand All @@ -306,6 +317,7 @@ func (p *Planner) categorizeNodes(podDestinations map[string]bool, scaleDownCand
p.unremovableNodes.AddTimeout(unremovable, unremovableTimeout)
}
}
p.handleUnprocessedNodes(nil, time.Now(), &endedPrematurely)
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)
Expand Down Expand Up @@ -435,3 +447,64 @@ func timedOut(timer *time.Timer) bool {
return false
}
}

func (p *Planner) handleUnprocessedNodes(unprocessedNodeNames []string, currentTime time.Time, endedPrematurely *bool) {
// if p.longestNodeScaleDownT is not set (flag is disabled) or endedPrematurely is already true (ndoes were already reported in this iteration) do not do anything
if p.longestNodeScaleDownT == nil || *endedPrematurely {
return
}
*endedPrematurely = true
p.longestNodeScaleDownT.update(unprocessedNodeNames, currentTime)
}

type longestNodeScaleDownEvalTime struct {
// lastEvalTime is the time of previous currentlyUnneededNodeNames parsing
lastEvalTime time.Time
nodeNamesWithTimeStamps map[string]time.Time
// minimumTime is the earliest time stored in nodeNamesWithTimeStamps
minimumTime time.Time
}

func newLongestNodeScaleDownEvalTime(currentTime time.Time) *longestNodeScaleDownEvalTime {
return &longestNodeScaleDownEvalTime{lastEvalTime: currentTime}
}

func (l *longestNodeScaleDownEvalTime) get(nodeName string) time.Time {
if _, ok := l.nodeNamesWithTimeStamps[nodeName]; ok {
return l.nodeNamesWithTimeStamps[nodeName]
}
return l.lastEvalTime
}

func (l *longestNodeScaleDownEvalTime) update(nodeNames []string, currentTime time.Time) time.Duration {
var longestTime time.Duration
// if nodeNames is nil it means that all nodes were processed
if nodeNames == nil {
// if l.minimumTime is 0, 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 l.minimumTime.IsZero() {
longestTime = 0
} else {
longestTime = currentTime.Sub(l.minimumTime)
l.minimumTime = time.Time{}
}
l.nodeNamesWithTimeStamps = make(map[string]time.Time)
Comment on lines +483 to +491

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Seems like this will never be called, because you check for null in handleUnprocessedNodes and return early in that case. I would also add unit test covering this path.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

in handleUnprocessedNodes() I do check the p.longestNodeScaleDownT for being null and not nodeNames. It is needed for the case when flag is disabled and longestNodeScaleDownT is not instantiated.
We will end up in this part of code if we have processed all the nodes.
About the unit tests I already check this behavior in "Test the functionality of longestNodeScaleDownT with all nodes processed in the first iteration" and "LongestLastScaleDownEvalDuration flag is disabled"

} else {
newNodes := make(map[string]time.Time, len(nodeNames))
l.minimumTime = l.lastEvalTime
for _, nodeName := range nodeNames {
// if a node is not in nodeNamesWithTimeStamps use the lastEvalTime
// if a node is already in nodeNamesWithTimeStamps copy the last value
valueFromPrevIter := l.get(nodeName)
newNodes[nodeName] = valueFromPrevIter
if l.minimumTime.After(valueFromPrevIter) {
l.minimumTime = valueFromPrevIter
}
}
l.nodeNamesWithTimeStamps = newNodes
longestTime = currentTime.Sub(l.minimumTime)
}
l.lastEvalTime = currentTime
metrics.ObserveLongestNodeScaleDownEvalTime(longestTime)
return longestTime
}
132 changes: 132 additions & 0 deletions cluster-autoscaler/core/scaledown/planner/planner_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1035,6 +1035,138 @@ func TestNodesToDelete(t *testing.T) {
}
}

func TestLongestUnprocessedNodeScaleDownTime(t *testing.T) {
type testCase struct {
name string
unprocessedNodes [][]string
}
start := time.Now()
testCases := []testCase{
{
name: "Test the functionality of longestNodeScaleDownT with all nodes processed in the first iteration",
unprocessedNodes: [][]string{nil, {"n1", "n2"}, {"n2", "n3"}},
},
{
name: "Test the functionality of longestNodeScaleDownT with not all nodes processed in the first iteration",
unprocessedNodes: [][]string{{"n1", "n2"}, {"n1", "n2"}, {"n2", "n3"}},
},
}
for _, tc := range testCases {
tc := tc
t.Run(tc.name, func(t *testing.T) {
t.Parallel()
timestamp := start
longestScaleDownEvalT := newLongestNodeScaleDownEvalTime(start)
timestamp = timestamp.Add(1 * time.Second)
if tc.unprocessedNodes[0] == nil {
assert.Equal(t, longestScaleDownEvalT.update(tc.unprocessedNodes[0], timestamp), time.Duration(0))
start = timestamp
} else {
assert.Equal(t, longestScaleDownEvalT.update(tc.unprocessedNodes[0], timestamp), timestamp.Sub(start))
}
for range 2 {
timestamp = timestamp.Add(1 * time.Second)
longestScaleDownEvalT.update(tc.unprocessedNodes[1], timestamp)
fmt.Println(len(longestScaleDownEvalT.nodeNamesWithTimeStamps))
assert.Equal(t, len(longestScaleDownEvalT.nodeNamesWithTimeStamps), len(tc.unprocessedNodes[1]))
for _, val := range longestScaleDownEvalT.nodeNamesWithTimeStamps {
assert.Equal(t, val, start)
}
}
timestamp = timestamp.Add(1 * time.Second)
currentLastEvalTime := longestScaleDownEvalT.lastEvalTime
assert.Equal(t, longestScaleDownEvalT.update(tc.unprocessedNodes[2], timestamp), timestamp.Sub(start)) // longestTime is for node n2
assert.Equal(t, longestScaleDownEvalT.get("n1"), longestScaleDownEvalT.lastEvalTime)
assert.Equal(t, longestScaleDownEvalT.get("n2"), start)
assert.Equal(t, longestScaleDownEvalT.get("n3"), currentLastEvalTime) // timestamp for new nodes is the default time before update
timestamp = timestamp.Add(1 * time.Second)
assert.Equal(t, longestScaleDownEvalT.update(nil, timestamp), timestamp.Sub(start)) // leftover from the previous iteration is time for node n2
timestamp = timestamp.Add(1 * time.Second)
assert.Equal(t, longestScaleDownEvalT.update(nil, timestamp), time.Duration(0)) // no leftover, so the longestTime will be 0
})
}
}

func TestLongestUnprocessedNodeScaleDownTimeWithTimeout(t *testing.T) {
type testCase struct {
name string
maxParallel int
isSimulationTimeout bool
unprocessedNodes int
isFlagEnabled bool
}
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,
// maxParallel=0 forces p.unneededNodesLimit() to be 0, so we will break in the second check inside p.categorizeNodes() right away
unprocessedNodes: 2,
isFlagEnabled: true,
},
{
name: "Simulation timeout is hit",
maxParallel: 1,
isSimulationTimeout: true,
// first node will be deleted and for the second timeout will be triggered
unprocessedNodes: 1,
isFlagEnabled: true,
},
{
name: "LongestLastScaleDownEvalDuration 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.longestNodeScaleDownT is not initialized
assert.Nil(t, p.longestNodeScaleDownT)
} else {
assert.Equal(t, len(p.longestNodeScaleDownT.nodeNamesWithTimeStamps), tc.unprocessedNodes)
}
})
}
}

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{
Expand Down
17 changes: 17 additions & 0 deletions cluster-autoscaler/metrics/metrics.go
Original file line number Diff line number Diff line change
Expand Up @@ -425,6 +425,14 @@ var (
Buckets: k8smetrics.ExponentialBuckets(1, 2, 6), // 1, 2, 4, ..., 32
}, []string{"instance_type", "cpu_count", "namespace_count"},
)

longestLastScaleDownEvalDuration = k8smetrics.NewGauge(
&k8smetrics.GaugeOpts{
Namespace: caNamespace,
Name: "longest_unneeded_node_scale_down_eval_duration_seconds",
Help: "Longest node evaluation time during ScaleDown.",
},
)
)

// RegisterAll registers all metrics.
Expand Down Expand Up @@ -461,6 +469,7 @@ func RegisterAll(emitPerNodeGroupMetrics bool) {
legacyregistry.MustRegister(nodeTaintsCount)
legacyregistry.MustRegister(inconsistentInstancesMigsCount)
legacyregistry.MustRegister(binpackingHeterogeneity)
legacyregistry.MustRegister(longestLastScaleDownEvalDuration)

if emitPerNodeGroupMetrics {
legacyregistry.MustRegister(nodesGroupMinNodes)
Expand Down Expand Up @@ -748,3 +757,11 @@ func UpdateInconsistentInstancesMigsCount(migCount int) {
func ObserveBinpackingHeterogeneity(instanceType, cpuCount, namespaceCount string, pegCount int) {
binpackingHeterogeneity.WithLabelValues(instanceType, cpuCount, namespaceCount).Observe(float64(pegCount))
}

// ObserveLongestNodeScaleDownEvalTime 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 ObserveLongestNodeScaleDownEvalTime(duration time.Duration) {
longestLastScaleDownEvalDuration.Set(float64(duration))
}
Loading