Skip to content

Commit

Permalink
[FLINK-31215] Backpropagate processing rate limits from non-scalable …
Browse files Browse the repository at this point in the history
…bottlenecks to upstream operators
  • Loading branch information
aplyusnin committed Jul 12, 2024
1 parent ffaa3dd commit 0721d34
Show file tree
Hide file tree
Showing 9 changed files with 912 additions and 5 deletions.
14 changes: 13 additions & 1 deletion docs/layouts/shortcodes/generated/auto_scaler_configuration.html
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,12 @@
<td>Duration</td>
<td>Lag threshold which will prevent unnecessary scalings while removing the pending messages responsible for the lag.</td>
</tr>
<tr>
<td><h5>job.autoscaler.bottleneck-propagation.allow-scale-down</h5></td>
<td style="word-wrap: break-word;">false</td>
<td>Boolean</td>
<td>Allow vertices scale down during bottleneck propagation.</td>
</tr>
<tr>
<td><h5>job.autoscaler.catch-up.duration</h5></td>
<td style="word-wrap: break-word;">30 min</td>
Expand Down Expand Up @@ -116,6 +122,12 @@
<td>Double</td>
<td>Percentage threshold for switching to observed from busy time based true processing rate if the measurement is off by at least the configured fraction. For example 0.15 means we switch to observed if the busy time based computation is at least 15% higher during catchup.</td>
</tr>
<tr>
<td><h5>job.autoscaler.processing.rate.backpropagation.enabled</h5></td>
<td style="word-wrap: break-word;">false</td>
<td>Boolean</td>
<td>Enable backpropagation of processing rate during autoscaling to reduce resources usage.</td>
</tr>
<tr>
<td><h5>job.autoscaler.quota.cpu</h5></td>
<td style="word-wrap: break-word;">(none)</td>
Expand Down Expand Up @@ -210,7 +222,7 @@
<td><h5>job.autoscaler.vertex.exclude.ids</h5></td>
<td style="word-wrap: break-word;"></td>
<td>List&lt;String&gt;</td>
<td>A (semicolon-separated) list of vertex ids in hexstring for which to disable scaling. Caution: For non-sink vertices this will still scale their downstream operators until https://issues.apache.org/jira/browse/FLINK-31215 is implemented.</td>
<td>A (semicolon-separated) list of vertex ids in hexstring for which to disable scaling.</td>
</tr>
<tr>
<td><h5>job.autoscaler.vertex.max-parallelism</h5></td>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,10 @@
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.autoscaler.config.AutoScalerOptions;
import org.apache.flink.autoscaler.event.AutoScalerEventHandler;
import org.apache.flink.autoscaler.metrics.EvaluatedMetrics;
import org.apache.flink.autoscaler.metrics.EvaluatedScalingMetric;
import org.apache.flink.autoscaler.metrics.ScalingMetric;
import org.apache.flink.autoscaler.topology.JobTopology;
import org.apache.flink.autoscaler.topology.ShipStrategy;
import org.apache.flink.autoscaler.utils.AutoScalerUtils;
import org.apache.flink.configuration.Configuration;
Expand All @@ -36,9 +38,11 @@
import java.time.Instant;
import java.time.ZoneId;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.SortedMap;

import static org.apache.flink.autoscaler.config.AutoScalerOptions.BOTTLENECK_PROPAGATION_SCALE_DOWN_ENABLED;
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.SCALE_UP_GRACE_PERIOD;
Expand All @@ -49,6 +53,7 @@
import static org.apache.flink.autoscaler.metrics.ScalingMetric.EXPECTED_PROCESSING_RATE;
import static org.apache.flink.autoscaler.metrics.ScalingMetric.MAX_PARALLELISM;
import static org.apache.flink.autoscaler.metrics.ScalingMetric.PARALLELISM;
import static org.apache.flink.autoscaler.metrics.ScalingMetric.TARGET_DATA_RATE;
import static org.apache.flink.autoscaler.metrics.ScalingMetric.TRUE_PROCESSING_RATE;
import static org.apache.flink.autoscaler.topology.ShipStrategy.HASH;

Expand Down Expand Up @@ -150,6 +155,122 @@ public int computeScaleTargetParallelism(
return newParallelism;
}

public boolean propagateBackpropScaleFactor(
Configuration conf,
JobVertexID vertex,
JobTopology topology,
EvaluatedMetrics evaluatedMetrics,
Map<JobVertexID, Double> backpropScaleFactors,
List<String> excludedVertices) {

double averageTrueProcessingRate =
evaluatedMetrics
.getVertexMetrics()
.get(vertex)
.get(TRUE_PROCESSING_RATE)
.getAverage();

// target parallelism is not defined -> cannot propagate the bottle factor to the upstream
if (Double.isNaN(averageTrueProcessingRate)
|| Double.isInfinite(averageTrueProcessingRate)) {
LOG.debug(
"Unable to backpropagate bottleneck scale factor of vertex {}, average true processing rate is {}",
vertex,
averageTrueProcessingRate);
return false;
}

double minScaleFactor = 1 - conf.get(MAX_SCALE_DOWN_FACTOR);
double maxScaleFactor = 1 + conf.get(MAX_SCALE_UP_FACTOR);

double processingRateCapacity =
evaluatedMetrics.getVertexMetrics().get(vertex).get(TARGET_DATA_RATE).getAverage();

if (Double.isNaN(processingRateCapacity)) {
LOG.debug(
"Unable to backpropagate bottleneck scale factor of vertex {}, processing rate capacity is {}",
vertex,
processingRateCapacity);
return false;
}

// if scale down is disabled, the adjusted scale factor cannot be less than the default
// factor
if (!conf.getBoolean(BOTTLENECK_PROPAGATION_SCALE_DOWN_ENABLED)) {
double scaleFactor = processingRateCapacity / averageTrueProcessingRate;
scaleFactor = Math.max(scaleFactor, minScaleFactor);
minScaleFactor = Math.min(1.0, scaleFactor);
}

// we scaled processing rate capacity by upstream
double currentBackPropFactor = backpropScaleFactors.getOrDefault(vertex, 1.0);
processingRateCapacity *= currentBackPropFactor;

double targetScaleFactor = processingRateCapacity / averageTrueProcessingRate;

if (excludedVertices.contains(vertex.toHexString())) {
LOG.debug(
"Vertex {} is excluded from scaling. Target scale factor is 1.0",
vertex.toHexString());
targetScaleFactor = 1.0;
}

if (targetScaleFactor < minScaleFactor) {
LOG.debug(
"Computed scale factor of {} for {} is capped by maximum scale down factor to {}",
targetScaleFactor,
vertex,
minScaleFactor);
targetScaleFactor = minScaleFactor;
}
if (maxScaleFactor < targetScaleFactor) {
LOG.debug(
"Computed scale factor of {} for {} is capped by maximum scale up factor to {}",
targetScaleFactor,
vertex,
maxScaleFactor);
targetScaleFactor = maxScaleFactor;
}

double maxVertexScaleFactor =
evaluatedMetrics.getVertexMetrics().get(vertex).get(MAX_PARALLELISM).getCurrent()
/ evaluatedMetrics
.getVertexMetrics()
.get(vertex)
.get(PARALLELISM)
.getCurrent();

// check if scaling violates max parallelism cap
if (maxVertexScaleFactor < targetScaleFactor) {
targetScaleFactor = maxVertexScaleFactor;
}

double targetProcessingCapacity = targetScaleFactor * averageTrueProcessingRate;
double adjustedProcessingRateCapacity =
AutoScalerUtils.getInPlaceTargetProcessingCapacity(
evaluatedMetrics, topology, vertex, backpropScaleFactors);
if (Double.isNaN(adjustedProcessingRateCapacity)) {
return false;
}

LOG.debug(
"Vertex {} has target capacity of {} and receives capacity {} from the downstream",
vertex,
targetProcessingCapacity,
adjustedProcessingRateCapacity);

// if the capacity from the upstream vertices exceeds target processing rate ->
// backpropagate scale factor
if (targetProcessingCapacity < adjustedProcessingRateCapacity) {
double adjustFactor = targetProcessingCapacity / adjustedProcessingRateCapacity;
for (var input : topology.getVertexInfos().get(vertex).getInputs().keySet()) {
double factor = backpropScaleFactors.getOrDefault(input, 1.0);
backpropScaleFactors.put(input, factor * adjustFactor);
}
}
return true;
}

private boolean blockScalingBasedOnPastActions(
Context context,
JobVertexID vertex,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@
import java.util.SortedMap;

import static org.apache.flink.autoscaler.config.AutoScalerOptions.EXCLUDED_PERIODS;
import static org.apache.flink.autoscaler.config.AutoScalerOptions.PROCESSING_RATE_BACKPROPAGATION_ENABLED;
import static org.apache.flink.autoscaler.config.AutoScalerOptions.SCALING_ENABLED;
import static org.apache.flink.autoscaler.config.AutoScalerOptions.SCALING_EVENT_INTERVAL;
import static org.apache.flink.autoscaler.event.AutoScalerEventHandler.SCALING_EXECUTION_DISABLED_REASON;
Expand All @@ -57,6 +58,7 @@
import static org.apache.flink.autoscaler.metrics.ScalingHistoryUtils.addToScalingHistoryAndStore;
import static org.apache.flink.autoscaler.metrics.ScalingMetric.SCALE_DOWN_RATE_THRESHOLD;
import static org.apache.flink.autoscaler.metrics.ScalingMetric.SCALE_UP_RATE_THRESHOLD;
import static org.apache.flink.autoscaler.metrics.ScalingMetric.TARGET_DATA_RATE;
import static org.apache.flink.autoscaler.metrics.ScalingMetric.TRUE_PROCESSING_RATE;

/** Class responsible for executing scaling decisions. */
Expand Down Expand Up @@ -221,8 +223,14 @@ Map<JobVertexID, ScalingSummary> computeScalingSummary(
}

var out = new HashMap<JobVertexID, ScalingSummary>();

if (context.getConfiguration().getBoolean(PROCESSING_RATE_BACKPROPAGATION_ENABLED)) {
backpropagateProcessingRate(context, evaluatedMetrics, restartTime, jobTopology);
}

var excludeVertexIdList =
context.getConfiguration().get(AutoScalerOptions.VERTEX_EXCLUDE_IDS);

evaluatedMetrics
.getVertexMetrics()
.forEach(
Expand Down Expand Up @@ -255,6 +263,75 @@ Map<JobVertexID, ScalingSummary> computeScalingSummary(
return out;
}

private void backpropagateProcessingRate(
Context context,
EvaluatedMetrics evaluatedMetrics,
Duration restartTime,
JobTopology jobTopology) {
var conf = context.getConfiguration();
var backpropScaleFactors = new HashMap<JobVertexID, Double>();
var excludeVertexIdList =
context.getConfiguration().get(AutoScalerOptions.VERTEX_EXCLUDE_IDS);
var vertexIterator =
jobTopology
.getVerticesInTopologicalOrder()
.listIterator(jobTopology.getVerticesInTopologicalOrder().size());

boolean canPropagate = true;

// backpropagate scale factors
while (canPropagate && vertexIterator.hasPrevious()) {
var vertex = vertexIterator.previous();
canPropagate =
jobVertexScaler.propagateBackpropScaleFactor(
conf,
vertex,
jobTopology,
evaluatedMetrics,
backpropScaleFactors,
excludeVertexIdList);
}

if (!canPropagate) {
LOG.debug("Cannot properly perform backpropagation because metrics are incomplete");
return;
}

// use an extra map to not lose precision
Map<JobVertexID, Double> adjustedDataRate = new HashMap<>();

// re-evaluating vertices capacity
// Target data rate metric is rewritten for parallelism evaluation
for (var vertex : jobTopology.getVerticesInTopologicalOrder()) {
double adjustedCapacity = 0.0;

if (jobTopology.isSource(vertex)) {
adjustedCapacity +=
evaluatedMetrics
.getVertexMetrics()
.get(vertex)
.get(TARGET_DATA_RATE)
.getAverage()
* backpropScaleFactors.getOrDefault(vertex, 1.0);
} else {
for (var input : jobTopology.getVertexInfos().get(vertex).getInputs().keySet()) {
adjustedCapacity +=
adjustedDataRate.get(input)
* jobTopology
.getVertexInfos()
.get(vertex)
.getInputRatios()
.get(input);
}
}
adjustedDataRate.put(vertex, adjustedCapacity);
evaluatedMetrics
.getVertexMetrics()
.get(vertex)
.put(TARGET_DATA_RATE, EvaluatedScalingMetric.avg(adjustedCapacity));
}
}

private boolean isJobUnderMemoryPressure(
Context ctx, Map<ScalingMetric, EvaluatedScalingMetric> evaluatedMetrics) {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -346,6 +346,7 @@ private void computeTargetDataRate(
var inputTargetRate = inputEvaluatedMetrics.get(TARGET_DATA_RATE);
var outputRatio =
computeEdgeOutputRatio(inputVertex, vertex, topology, metricsHistory);
topology.get(vertex).getInputRatios().put(inputVertex, outputRatio);
LOG.debug(
"Computed output ratio for edge ({} -> {}) : {}",
inputVertex,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,23 @@ private static ConfigOptions.OptionBuilder autoScalerConfig(String key) {
.withDescription(
"Enable vertex scaling execution by the autoscaler. If disabled, the autoscaler will only collect metrics and evaluate the suggested parallelism for each vertex but will not upgrade the jobs.");

public static final ConfigOption<Boolean> PROCESSING_RATE_BACKPROPAGATION_ENABLED =
autoScalerConfig("processing.rate.backpropagation.enabled")
.booleanType()
.defaultValue(false)
.withFallbackKeys(
oldOperatorConfigKey("processing.rate.backpropagation.enabled"))
.withDescription(
"Enable backpropagation of processing rate during autoscaling to reduce resources usage.");

public static final ConfigOption<Boolean> BOTTLENECK_PROPAGATION_SCALE_DOWN_ENABLED =
autoScalerConfig("bottleneck-propagation.allow-scale-down")
.booleanType()
.defaultValue(false)
.withFallbackKeys(
oldOperatorConfigKey("bottleneck-propagation.allow-scale-down"))
.withDescription("Allow vertices scale down during bottleneck propagation.");

public static final ConfigOption<Duration> METRICS_WINDOW =
autoScalerConfig("metrics.window")
.durationType()
Expand Down Expand Up @@ -313,7 +330,7 @@ private static ConfigOptions.OptionBuilder autoScalerConfig(String key) {
.defaultValues()
.withFallbackKeys(oldOperatorConfigKey("vertex.exclude.ids"))
.withDescription(
"A (semicolon-separated) list of vertex ids in hexstring for which to disable scaling. Caution: For non-sink vertices this will still scale their downstream operators until https://issues.apache.org/jira/browse/FLINK-31215 is implemented.");
"A (semicolon-separated) list of vertex ids in hexstring for which to disable scaling.");

public static final ConfigOption<Duration> SCALING_EVENT_INTERVAL =
autoScalerConfig("scaling.event.interval")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import lombok.Data;
import lombok.Setter;

import java.util.HashMap;
import java.util.Map;

/** Job vertex information. */
Expand All @@ -36,6 +37,9 @@ public class VertexInfo {
// All input vertices and the ship_strategy
private final Map<JobVertexID, ShipStrategy> inputs;

// Output ratios from input vertices. Used for backpropagation
private final Map<JobVertexID, Double> inputRatios;

private final SlotSharingGroupId slotSharingGroupId;

// All output vertices and the ship_strategy
Expand Down Expand Up @@ -68,6 +72,7 @@ public VertexInfo(
this.originalMaxParallelism = maxParallelism;
this.finished = finished;
this.ioMetrics = ioMetrics;
this.inputRatios = new HashMap<>();
}

@VisibleForTesting
Expand Down
Loading

0 comments on commit 0721d34

Please sign in to comment.