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 4, 2024
1 parent 07a3f73 commit ee111e1
Show file tree
Hide file tree
Showing 6 changed files with 76 additions and 20 deletions.
18 changes: 12 additions & 6 deletions docs/layouts/shortcodes/generated/auto_scaler_configuration.html
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,18 @@
<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.bottleneck-propagation.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.catch-up.duration</h5></td>
<td style="word-wrap: break-word;">30 min</td>
Expand Down Expand Up @@ -116,12 +128,6 @@
<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
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@
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 Down Expand Up @@ -85,7 +86,7 @@ public VertexScalingResult computeScaleTargetParallelism(
conf.get(AutoScalerOptions.VERTEX_EXCLUDE_IDS).contains(vertex.toHexString());
if (excluded) {
LOG.debug(
"Vertex {} is part of `vertex.exclude.ids` config, Check for bottleneck but not scale",
"Vertex {} is part of `vertex.exclude.ids` config. Check for being a bottleneck but not scale",
vertex);
}

Expand All @@ -112,8 +113,16 @@ public VertexScalingResult computeScaleTargetParallelism(

LOG.debug("Target processing capacity for {} is {}", vertex, targetCapacity);
double scaleFactor = targetCapacity / averageTrueProcessingRate;
double minScaleFactor = 1 - conf.get(MAX_SCALE_DOWN_FACTOR);
double maxScaleFactor = 1 + conf.get(MAX_SCALE_UP_FACTOR);
double minScaleFactor = 1 - conf.get(MAX_SCALE_DOWN_FACTOR);

// if bottleneck propagation is applied and scaling down is forbidden, limit minScaleFactor
// with 1
if (backpropagationScaleFactor < 1.0
&& !conf.getBoolean(BOTTLENECK_PROPAGATION_SCALE_DOWN_ENABLED)) {
minScaleFactor = 1.0;
}

if (scaleFactor < minScaleFactor) {
LOG.debug(
"Computed scale factor of {} for {} is capped by maximum scale down factor to {}",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,8 +48,8 @@
import java.util.Map;
import java.util.SortedMap;

import static org.apache.flink.autoscaler.config.AutoScalerOptions.BOTTLENECK_PROPAGATION_ENABLED;
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 Down Expand Up @@ -226,8 +226,7 @@ Map<JobVertexID, ScalingSummary> computeScalingSummary(
context, evaluatedMetrics, scalingHistory, restartTime, jobTopology, 1.0);

if (scalingResult.getBottlenecks().isEmpty()
|| !context.getConfiguration()
.getBoolean(PROCESSING_RATE_BACKPROPAGATION_ENABLED)) {
|| !context.getConfiguration().getBoolean(BOTTLENECK_PROPAGATION_ENABLED)) {
return scalingResult.getScalingSummaries();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,15 +58,22 @@ 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")
public static final ConfigOption<Boolean> BOTTLENECK_PROPAGATION_ENABLED =
autoScalerConfig("bottleneck-propagation.enabled")
.booleanType()
.defaultValue(false)
.withFallbackKeys(
oldOperatorConfigKey("processing.rate.backpropagation.enabled"))
.withFallbackKeys(oldOperatorConfigKey("bottleneck-propagation.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
Original file line number Diff line number Diff line change
Expand Up @@ -517,6 +517,41 @@ public void testBackPropagationScaleFactorAffectsScaling() {
assertFalse(result.isBottleneck());
}

@Test
public void testBottleneckPropagationScaleDown() {
var op = new JobVertexID();
conf.set(AutoScalerOptions.TARGET_UTILIZATION, 1.);
conf.set(AutoScalerOptions.VERTEX_MAX_PARALLELISM, 20);
conf.set(AutoScalerOptions.BOTTLENECK_PROPAGATION_SCALE_DOWN_ENABLED, false);

var result =
vertexScaler.computeScaleTargetParallelism(
context,
op,
NOT_ADJUST_INPUTS,
evaluated(4, 200, 200),
Collections.emptySortedMap(),
restartTime,
0.5);

assertEquals(4, result.getParallelism());
assertFalse(result.isBottleneck());

conf.set(AutoScalerOptions.BOTTLENECK_PROPAGATION_SCALE_DOWN_ENABLED, true);
result =
vertexScaler.computeScaleTargetParallelism(
context,
op,
NOT_ADJUST_INPUTS,
evaluated(4, 200, 200),
Collections.emptySortedMap(),
restartTime,
0.5);

assertEquals(2, result.getParallelism());
assertFalse(result.isBottleneck());
}

@Test
public void testScaleDownAfterScaleUpDetection() {
var op = new JobVertexID();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -661,7 +661,7 @@ public void testScalingWithBackPropEnabledSimpleGraph() throws Exception {

var conf = context.getConfiguration();
conf.set(AutoScalerOptions.TARGET_UTILIZATION, 1.d);
conf.set(AutoScalerOptions.PROCESSING_RATE_BACKPROPAGATION_ENABLED, true);
conf.set(AutoScalerOptions.BOTTLENECK_PROPAGATION_ENABLED, true);

// If back propagation is enabled, then parallelism of all vertices is 5
var metrics =
Expand Down Expand Up @@ -698,7 +698,7 @@ public void testScalingWithBackPropEnabledSimpleGraph() throws Exception {
"a6b7102b8d3e3a9564998c1ffeb5e2b7",
"5"));

conf.set(AutoScalerOptions.PROCESSING_RATE_BACKPROPAGATION_ENABLED, false);
conf.set(AutoScalerOptions.BOTTLENECK_PROPAGATION_ENABLED, false);
now = Instant.now();
assertThat(
scalingExecutor.scaleResource(
Expand All @@ -722,7 +722,7 @@ public void testScalingWithBackPropEnabledSimpleGraph() throws Exception {
"a6b7102b8d3e3a9564998c1ffeb5e2b7",
"10"));

conf.set(AutoScalerOptions.PROCESSING_RATE_BACKPROPAGATION_ENABLED, true);
conf.set(AutoScalerOptions.BOTTLENECK_PROPAGATION_ENABLED, true);

jobTopology =
new JobTopology(
Expand Down Expand Up @@ -804,7 +804,7 @@ public void testScalingWithBackPropEnabledComplexGraph() throws Exception {

var conf = context.getConfiguration();
conf.set(AutoScalerOptions.TARGET_UTILIZATION, 1.d);
conf.set(AutoScalerOptions.PROCESSING_RATE_BACKPROPAGATION_ENABLED, true);
conf.set(AutoScalerOptions.BOTTLENECK_PROPAGATION_ENABLED, true);

// The expected new parallelism is 7 without adjustment by max parallelism.
var metrics =
Expand Down Expand Up @@ -875,7 +875,7 @@ public void testScalingWithBackPropEnabledAndExcludedVerticesSimpleGraph() throw

var conf = context.getConfiguration();
conf.set(AutoScalerOptions.TARGET_UTILIZATION, 1.d);
conf.set(AutoScalerOptions.PROCESSING_RATE_BACKPROPAGATION_ENABLED, true);
conf.set(AutoScalerOptions.BOTTLENECK_PROPAGATION_ENABLED, true);

// If back propagation is enabled, then parallelism of all vertices is 5
var metrics =
Expand Down

0 comments on commit ee111e1

Please sign in to comment.