Skip to content
Merged
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
18 changes: 18 additions & 0 deletions docs/layouts/shortcodes/generated/auto_scaler_configuration.html
Original file line number Diff line number Diff line change
Expand Up @@ -98,6 +98,24 @@
<td>Duration</td>
<td>Scaling metrics aggregation window size.</td>
</tr>
<tr>
<td><h5>job.autoscaler.observed-scalability.coefficient-min</h5></td>
<td style="word-wrap: break-word;">0.5</td>
<td>Double</td>
<td>Minimum allowed value for the observed scalability coefficient. Prevents aggressive scaling by clamping low coefficient estimates. If the estimated coefficient falls below this value, it is capped at the configured minimum.</td>
</tr>
<tr>
<td><h5>job.autoscaler.observed-scalability.enabled</h5></td>
<td style="word-wrap: break-word;">false</td>
<td>Boolean</td>
<td>Enables the use of an observed scalability coefficient when computing target parallelism. If enabled, the system will estimate the scalability coefficient based on historical scaling data instead of assuming perfect linear scaling. This helps account for real-world inefficiencies such as network overhead and coordination costs.</td>
</tr>
<tr>
<td><h5>job.autoscaler.observed-scalability.min-observations</h5></td>
<td style="word-wrap: break-word;">3</td>
<td>Integer</td>
<td>Defines the minimum number of historical scaling observations required to estimate the scalability coefficient. If the number of available observations is below this threshold, the system falls back to assuming linear scaling. Note: To effectively use a higher minimum observation count, you need to increase job.autoscaler.history.max.count. Avoid setting job.autoscaler.history.max.count to a very high value, as the number of retained data points is limited by the size of the state store—particularly when using Kubernetes-based state store.</td>
</tr>
<tr>
<td><h5>job.autoscaler.observed-true-processing-rate.lag-threshold</h5></td>
<td style="word-wrap: break-word;">30 s</td>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,18 +34,24 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.math.BigDecimal;
import java.math.RoundingMode;
import java.time.Clock;
import java.time.Duration;
import java.time.Instant;
import java.time.ZoneId;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.SortedMap;

import static org.apache.flink.autoscaler.JobVertexScaler.KeyGroupOrPartitionsAdjustMode.MAXIMIZE_UTILISATION;
import static org.apache.flink.autoscaler.config.AutoScalerOptions.MAX_SCALE_DOWN_FACTOR;
import static org.apache.flink.autoscaler.config.AutoScalerOptions.MAX_SCALE_UP_FACTOR;
import static org.apache.flink.autoscaler.config.AutoScalerOptions.OBSERVED_SCALABILITY_ENABLED;
import static org.apache.flink.autoscaler.config.AutoScalerOptions.OBSERVED_SCALABILITY_MIN_OBSERVATIONS;
import static org.apache.flink.autoscaler.config.AutoScalerOptions.SCALE_DOWN_INTERVAL;
import static org.apache.flink.autoscaler.config.AutoScalerOptions.SCALING_EVENT_INTERVAL;
import static org.apache.flink.autoscaler.config.AutoScalerOptions.SCALING_KEY_GROUP_PARTITIONS_ADJUST_MODE;
Expand Down Expand Up @@ -178,6 +184,13 @@ public ParallelismChange computeScaleTargetParallelism(

LOG.debug("Target processing capacity for {} is {}", vertex, targetCapacity);
double scaleFactor = targetCapacity / averageTrueProcessingRate;
if (conf.get(OBSERVED_SCALABILITY_ENABLED)) {

double scalingCoefficient =
JobVertexScaler.calculateObservedScalingCoefficient(history, conf);

scaleFactor = scaleFactor / scalingCoefficient;
}
double minScaleFactor = 1 - conf.get(MAX_SCALE_DOWN_FACTOR);
double maxScaleFactor = 1 + conf.get(MAX_SCALE_UP_FACTOR);
if (scaleFactor < minScaleFactor) {
Expand Down Expand Up @@ -236,6 +249,83 @@ public ParallelismChange computeScaleTargetParallelism(
delayedScaleDown);
}

/**
* Calculates the scaling coefficient based on historical scaling data.
*
* <p>The scaling coefficient is computed using the least squares approach. If there are not
* enough observations, or if the computed coefficient is invalid, a default value of {@code
* 1.0} is returned, assuming linear scaling.
*
* @param history A {@code SortedMap} of {@code Instant} timestamps to {@code ScalingSummary}
* @param conf Deployment configuration.
* @return The computed scaling coefficient.
*/
@VisibleForTesting
protected static double calculateObservedScalingCoefficient(
SortedMap<Instant, ScalingSummary> history, Configuration conf) {
/*
* The scaling coefficient is computed using the least squares approach
* to fit a linear model:
*
* R_i = β * P_i * α
*
* where:
* - R_i = observed processing rate
* - P_i = parallelism
* - β = baseline processing rate
* - α = scaling coefficient to optimize
*
* The optimization minimizes the **sum of squared errors**:
*
* Loss = ∑ (R_i - β * α * P_i)^2
*
* Differentiating w.r.t. α and solving for α:
*
* α = ∑ (P_i * R_i) / (∑ (P_i^2) * β)
*
* We keep the system conservative for higher returns scenario by clamping computed α to an upper bound of 1.0.
*/

var minObservations = conf.get(OBSERVED_SCALABILITY_MIN_OBSERVATIONS);

// not enough data to compute scaling coefficient; we assume linear scaling.
if (history.isEmpty() || history.size() < minObservations) {
return 1.0;
}

var baselineProcessingRate = AutoScalerUtils.computeBaselineProcessingRate(history);

if (Double.isNaN(baselineProcessingRate)) {
return 1.0;
}

List<Double> parallelismList = new ArrayList<>();
List<Double> processingRateList = new ArrayList<>();

for (Map.Entry<Instant, ScalingSummary> entry : history.entrySet()) {
ScalingSummary summary = entry.getValue();
double parallelism = summary.getCurrentParallelism();
double processingRate = summary.getMetrics().get(TRUE_PROCESSING_RATE).getAverage();

if (Double.isNaN(processingRate)) {
LOG.warn(
"True processing rate is not available in scaling history. Cannot compute scaling coefficient.");
return 1.0;
}

parallelismList.add(parallelism);
processingRateList.add(processingRate);
}

double lowerBound = conf.get(AutoScalerOptions.OBSERVED_SCALABILITY_COEFFICIENT_MIN);

var coefficient =
AutoScalerUtils.optimizeLinearScalingCoefficient(
parallelismList, processingRateList, baselineProcessingRate, 1, lowerBound);

return BigDecimal.valueOf(coefficient).setScale(2, RoundingMode.CEILING).doubleValue();
}

private ParallelismChange detectBlockScaling(
Context context,
JobVertexID vertex,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -382,4 +382,40 @@ private static ConfigOptions.OptionBuilder autoScalerConfig(String key) {
"scaling.key-group.partitions.adjust.mode"))
.withDescription(
"How to adjust the parallelism of Source vertex or upstream shuffle is keyBy");

public static final ConfigOption<Boolean> OBSERVED_SCALABILITY_ENABLED =
autoScalerConfig("observed-scalability.enabled")
.booleanType()
.defaultValue(false)
.withFallbackKeys(oldOperatorConfigKey("observed-scalability.enabled"))
.withDescription(
"Enables the use of an observed scalability coefficient when computing target parallelism. "
+ "If enabled, the system will estimate the scalability coefficient based on historical scaling data "
+ "instead of assuming perfect linear scaling. "
+ "This helps account for real-world inefficiencies such as network overhead and coordination costs.");

public static final ConfigOption<Integer> OBSERVED_SCALABILITY_MIN_OBSERVATIONS =
autoScalerConfig("observed-scalability.min-observations")
.intType()
.defaultValue(3)
.withFallbackKeys(oldOperatorConfigKey("observed-scalability.min-observations"))
.withDescription(
"Defines the minimum number of historical scaling observations required to estimate the scalability coefficient. "
+ "If the number of available observations is below this threshold, the system falls back to assuming linear scaling. "
+ "Note: To effectively use a higher minimum observation count, you need to increase "
+ VERTEX_SCALING_HISTORY_COUNT.key()
+ ". Avoid setting "
+ VERTEX_SCALING_HISTORY_COUNT.key()
+ " to a very high value, as the number of retained data points is limited by the size of the state store—"
+ "particularly when using Kubernetes-based state store.");

public static final ConfigOption<Double> OBSERVED_SCALABILITY_COEFFICIENT_MIN =
autoScalerConfig("observed-scalability.coefficient-min")
.doubleType()
.defaultValue(0.5)
.withFallbackKeys(oldOperatorConfigKey("observed-scalability.coefficient-min"))
.withDescription(
"Minimum allowed value for the observed scalability coefficient. "
+ "Prevents aggressive scaling by clamping low coefficient estimates. "
+ "If the estimated coefficient falls below this value, it is capped at the configured minimum.");
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,22 +17,27 @@

package org.apache.flink.autoscaler.utils;

import org.apache.flink.autoscaler.ScalingSummary;
import org.apache.flink.autoscaler.config.AutoScalerOptions;
import org.apache.flink.autoscaler.metrics.EvaluatedScalingMetric;
import org.apache.flink.autoscaler.metrics.ScalingMetric;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.jobgraph.JobVertexID;

import java.time.Duration;
import java.time.Instant;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.NavigableMap;
import java.util.Set;
import java.util.SortedMap;

import static org.apache.flink.autoscaler.metrics.ScalingMetric.CATCH_UP_DATA_RATE;
import static org.apache.flink.autoscaler.metrics.ScalingMetric.TARGET_DATA_RATE;
import static org.apache.flink.autoscaler.metrics.ScalingMetric.TRUE_PROCESSING_RATE;

/** AutoScaler utilities. */
public class AutoScalerUtils {
Expand Down Expand Up @@ -94,4 +99,89 @@ public static boolean excludeVerticesFromScaling(
conf.set(AutoScalerOptions.VERTEX_EXCLUDE_IDS, new ArrayList<>(excludedIds));
return anyAdded;
}

/**
* Computes the optimized linear scaling coefficient (α) by minimizing the least squares error.
*
* <p>This method estimates the scaling coefficient in a linear scaling model by fitting
* observed processing rates and parallelism levels.
*
* <p>The computed coefficient is clamped within the specified lower and upper bounds to ensure
* stability and prevent extreme scaling adjustments.
*
* @param parallelismLevels List of parallelism levels.
* @param processingRates List of observed processing rates.
* @param baselineProcessingRate Baseline processing rate.
* @param upperBound Maximum allowable value for the scaling coefficient.
* @param lowerBound Minimum allowable value for the scaling coefficient.
* @return The optimized scaling coefficient (α), constrained within {@code [lowerBound,
* upperBound]}.
*/
public static double optimizeLinearScalingCoefficient(
List<Double> parallelismLevels,
List<Double> processingRates,
double baselineProcessingRate,
double upperBound,
double lowerBound) {

double sum = 0.0;
double squaredSum = 0.0;

for (int i = 0; i < parallelismLevels.size(); i++) {
double parallelism = parallelismLevels.get(i);
double processingRate = processingRates.get(i);

sum += parallelism * processingRate;
squaredSum += parallelism * parallelism;
}

if (squaredSum == 0.0) {
return 1.0; // Fallback to linear scaling if denominator is zero
}

double alpha = sum / (squaredSum * baselineProcessingRate);

return Math.max(lowerBound, Math.min(upperBound, alpha));
}

/**
* Computes the baseline processing rate from historical scaling data.
*
* <p>The baseline processing rate represents the **processing rate per unit of parallelism**.
* It is determined using the smallest observed parallelism in the history.
*
* @param history A {@code SortedMap} where keys are timestamps ({@code Instant}), and values
* are {@code ScalingSummary} objects.
* @return The computed baseline processing rate (processing rate per unit of parallelism).
*/
public static double computeBaselineProcessingRate(SortedMap<Instant, ScalingSummary> history) {
ScalingSummary latestSmallestParallelismSummary = null;

for (Map.Entry<Instant, ScalingSummary> entry :
((NavigableMap<Instant, ScalingSummary>) history).descendingMap().entrySet()) {
ScalingSummary summary = entry.getValue();
double parallelism = summary.getCurrentParallelism();

if (parallelism == 1) {
return summary.getMetrics().get(TRUE_PROCESSING_RATE).getAverage();
}

if (latestSmallestParallelismSummary == null
|| parallelism < latestSmallestParallelismSummary.getCurrentParallelism()) {
latestSmallestParallelismSummary = entry.getValue();
}
}

if (latestSmallestParallelismSummary == null) {
return Double.NaN;
}

double parallelism = latestSmallestParallelismSummary.getCurrentParallelism();
double processingRate =
latestSmallestParallelismSummary
.getMetrics()
.get(TRUE_PROCESSING_RATE)
.getAverage();
return processingRate / parallelism;
}
}
Loading