Skip to content

Commit 23ce228

Browse files
authored
[FLINK-29501] Add option to override job vertex parallelisms during job submission (#21023)
This allows to change the job vertex parallelism of a JobGraph during job submission time without having to modify the JobGraph upfront. This is intended to be used for (auto)scaling Flink jobs based on metric observations. The initial idea was to add a new field to the payload of the job submit Rest endpoint. However, it is probably more practical to handle the overrides in the same way as other PipelineOptions already do it, i.e. via the configuration. The implementation is deliberately lenient with respect to the presence of job vertices. If vertices have been removed or new ones have been added, only the ones found will have their parallelism overrides. The verification should be performed by the caller, not by Flink. In particular, we want to support scenarios where users modify the deployment and we might not yet have overrides for all vertices.
1 parent a5667e8 commit 23ce228

File tree

4 files changed

+75
-0
lines changed

4 files changed

+75
-0
lines changed

docs/layouts/shortcodes/generated/pipeline_configuration.html

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -80,6 +80,12 @@
8080
<td>List&lt;String&gt;</td>
8181
<td>A semicolon-separated list of the jars to package with the job jars to be sent to the cluster. These have to be valid paths.</td>
8282
</tr>
83+
<tr>
84+
<td><h5>pipeline.jobvertex-parallelism-overrides</h5></td>
85+
<td style="word-wrap: break-word;"></td>
86+
<td>Map</td>
87+
<td>A parallelism override map (jobVertexId -&gt; parallelism) which will be used to update the parallelism of the corresponding job vertices of submitted JobGraphs.</td>
88+
</tr>
8389
<tr>
8490
<td><h5>pipeline.max-parallelism</h5></td>
8591
<td style="word-wrap: break-word;">-1</td>

flink-core/src/main/java/org/apache/flink/configuration/PipelineOptions.java

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@
2424
import org.apache.flink.configuration.description.TextElement;
2525

2626
import java.time.Duration;
27+
import java.util.Collections;
2728
import java.util.List;
2829
import java.util.Map;
2930

@@ -165,6 +166,14 @@ public class PipelineOptions {
165166
"Register a custom, serializable user configuration object. The configuration can be "
166167
+ " accessed in operators");
167168

169+
public static final ConfigOption<Map<String, String>> PARALLELISM_OVERRIDES =
170+
key("pipeline.jobvertex-parallelism-overrides")
171+
.mapType()
172+
.defaultValue(Collections.emptyMap())
173+
.withDescription(
174+
"A parallelism override map (jobVertexId -> parallelism) which will be used to update"
175+
+ " the parallelism of the corresponding job vertices of submitted JobGraphs.");
176+
168177
public static final ConfigOption<Integer> MAX_PARALLELISM =
169178
key("pipeline.max-parallelism")
170179
.intType()

flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java

Lines changed: 18 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -29,6 +29,7 @@
2929
import org.apache.flink.configuration.ClusterOptions;
3030
import org.apache.flink.configuration.Configuration;
3131
import org.apache.flink.configuration.HighAvailabilityOptions;
32+
import org.apache.flink.configuration.PipelineOptions;
3233
import org.apache.flink.core.execution.CheckpointType;
3334
import org.apache.flink.core.execution.SavepointFormatType;
3435
import org.apache.flink.metrics.MetricGroup;
@@ -531,6 +532,7 @@ private boolean isPartialResourceConfigured(JobGraph jobGraph) {
531532
}
532533

533534
private CompletableFuture<Acknowledge> internalSubmitJob(JobGraph jobGraph) {
535+
applyParallelismOverrides(jobGraph);
534536
log.info("Submitting job '{}' ({}).", jobGraph.getName(), jobGraph.getJobID());
535537
return waitForTerminatingJob(jobGraph.getJobID(), jobGraph, this::persistAndRunJob)
536538
.handle((ignored, throwable) -> handleTermination(jobGraph.getJobID(), throwable))
@@ -1334,4 +1336,20 @@ private void registerDispatcherMetrics(MetricGroup jobManagerMetricGroup) {
13341336
public CompletableFuture<Void> onRemovedJobGraph(JobID jobId) {
13351337
return CompletableFuture.runAsync(() -> terminateJob(jobId), getMainThreadExecutor());
13361338
}
1339+
1340+
private void applyParallelismOverrides(JobGraph jobGraph) {
1341+
Map<String, String> overrides = configuration.get(PipelineOptions.PARALLELISM_OVERRIDES);
1342+
for (JobVertex vertex : jobGraph.getVertices()) {
1343+
String override = overrides.get(vertex.getID().toHexString());
1344+
if (override != null) {
1345+
int overrideParallelism = Integer.parseInt(override);
1346+
log.info(
1347+
"Changing job vertex {} parallelism from {} to {}",
1348+
vertex.getID(),
1349+
vertex.getParallelism(),
1350+
overrideParallelism);
1351+
vertex.setParallelism(overrideParallelism);
1352+
}
1353+
}
1354+
}
13371355
}

flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java

Lines changed: 42 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,7 @@
2323
import org.apache.flink.api.common.operators.ResourceSpec;
2424
import org.apache.flink.api.java.tuple.Tuple2;
2525
import org.apache.flink.configuration.Configuration;
26+
import org.apache.flink.configuration.PipelineOptions;
2627
import org.apache.flink.core.execution.SavepointFormatType;
2728
import org.apache.flink.core.testutils.FlinkMatchers;
2829
import org.apache.flink.core.testutils.OneShotLatch;
@@ -46,6 +47,7 @@
4647
import org.apache.flink.runtime.jobgraph.JobGraphBuilder;
4748
import org.apache.flink.runtime.jobgraph.JobGraphTestUtils;
4849
import org.apache.flink.runtime.jobgraph.JobVertex;
50+
import org.apache.flink.runtime.jobgraph.JobVertexID;
4951
import org.apache.flink.runtime.jobmaster.JobManagerRunner;
5052
import org.apache.flink.runtime.jobmaster.JobManagerRunnerResult;
5153
import org.apache.flink.runtime.jobmaster.JobManagerSharedServices;
@@ -88,6 +90,8 @@
8890
import org.apache.flink.util.Preconditions;
8991
import org.apache.flink.util.concurrent.FutureUtils;
9092

93+
import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableMap;
94+
9195
import org.assertj.core.api.Assertions;
9296
import org.hamcrest.Matchers;
9397
import org.junit.After;
@@ -1031,6 +1035,44 @@ public void testRequestMultipleJobDetails_isSerializable() throws Exception {
10311035
InstantiationUtil.serializeObject(multipleJobsDetails);
10321036
}
10331037

1038+
@Test
1039+
public void testOverridingJobVertexParallelisms() throws Exception {
1040+
JobVertex v1 = new JobVertex("v1");
1041+
v1.setParallelism(1);
1042+
JobVertex v2 = new JobVertex("v2");
1043+
v2.setParallelism(2);
1044+
JobVertex v3 = new JobVertex("v3");
1045+
v3.setParallelism(3);
1046+
jobGraph = new JobGraph(jobGraph.getJobID(), "job", v1, v2, v3);
1047+
1048+
configuration.set(
1049+
PipelineOptions.PARALLELISM_OVERRIDES,
1050+
ImmutableMap.of(
1051+
v1.getID().toHexString(), "10",
1052+
// v2 is omitted
1053+
v3.getID().toHexString(), "42",
1054+
// unknown vertex added
1055+
new JobVertexID().toHexString(), "23"));
1056+
1057+
dispatcher =
1058+
createAndStartDispatcher(
1059+
heartbeatServices,
1060+
haServices,
1061+
new ExpectedJobIdJobManagerRunnerFactory(
1062+
jobId, createdJobManagerRunnerLatch));
1063+
DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class);
1064+
1065+
Assert.assertEquals(jobGraph.findVertexByID(v1.getID()).getParallelism(), 1);
1066+
Assert.assertEquals(jobGraph.findVertexByID(v2.getID()).getParallelism(), 2);
1067+
Assert.assertEquals(jobGraph.findVertexByID(v3.getID()).getParallelism(), 3);
1068+
1069+
dispatcherGateway.submitJob(jobGraph, TIMEOUT).get();
1070+
1071+
Assert.assertEquals(jobGraph.findVertexByID(v1.getID()).getParallelism(), 10);
1072+
Assert.assertEquals(jobGraph.findVertexByID(v2.getID()).getParallelism(), 2);
1073+
Assert.assertEquals(jobGraph.findVertexByID(v3.getID()).getParallelism(), 42);
1074+
}
1075+
10341076
private JobManagerRunner runningJobManagerRunnerWithJobStatus(
10351077
final JobStatus currentJobStatus) {
10361078
Preconditions.checkArgument(!currentJobStatus.isTerminalState());

0 commit comments

Comments
 (0)