21
21
import org .apache .flink .api .common .JobExecutionResult ;
22
22
import org .apache .flink .api .common .JobStatus ;
23
23
import org .apache .flink .configuration .Configuration ;
24
+ import org .apache .flink .configuration .PipelineOptions ;
24
25
import org .apache .flink .core .execution .JobClient ;
25
26
import org .apache .flink .core .execution .SavepointFormatType ;
27
+ import org .apache .flink .runtime .execution .Environment ;
26
28
import org .apache .flink .runtime .jobgraph .JobGraph ;
27
29
import org .apache .flink .runtime .jobgraph .JobGraphTestUtils ;
28
30
import org .apache .flink .runtime .jobgraph .JobVertex ;
29
31
import org .apache .flink .runtime .minicluster .MiniCluster ;
32
+ import org .apache .flink .runtime .testutils .CancelableInvokable ;
30
33
import org .apache .flink .runtime .testutils .WaitingCancelableInvokable ;
31
34
35
+ import org .apache .flink .shaded .guava30 .com .google .common .collect .ImmutableMap ;
36
+
32
37
import org .junit .jupiter .api .AfterEach ;
33
38
import org .junit .jupiter .api .Test ;
34
39
35
40
import java .time .Duration ;
36
41
import java .util .Map ;
42
+ import java .util .concurrent .CountDownLatch ;
37
43
import java .util .concurrent .ExecutionException ;
38
44
45
+ import static org .apache .flink .util .Preconditions .checkState ;
39
46
import static org .assertj .core .api .Assertions .assertThat ;
40
47
import static org .assertj .core .api .Assertions .assertThatThrownBy ;
41
48
@@ -152,9 +159,44 @@ void testJobClientInteractionAfterShutdown() throws Exception {
152
159
"MiniCluster is not yet running or has already been shut down." );
153
160
}
154
161
162
+ @ Test
163
+ void testTurnUpParallelismByOverwriteParallelism () throws Exception {
164
+ JobVertex jobVertex = getBlockingJobVertex ();
165
+ JobGraph jobGraph = JobGraphTestUtils .streamingJobGraph (jobVertex );
166
+ int overwriteParallelism = jobVertex .getParallelism () + 1 ;
167
+ BlockingInvokable .reset (overwriteParallelism );
168
+
169
+ Configuration configuration = new Configuration ();
170
+ configuration .set (
171
+ PipelineOptions .PARALLELISM_OVERRIDES ,
172
+ ImmutableMap .of (
173
+ jobVertex .getID ().toHexString (), String .valueOf (overwriteParallelism )));
174
+
175
+ PerJobMiniClusterFactory perJobMiniClusterFactory = initializeMiniCluster (configuration );
176
+ JobClient jobClient =
177
+ perJobMiniClusterFactory
178
+ .submitJob (jobGraph , ClassLoader .getSystemClassLoader ())
179
+ .get ();
180
+
181
+ // wait for tasks to be properly running
182
+ BlockingInvokable .latch .await ();
183
+
184
+ jobClient .cancel ().get ();
185
+ assertThat (jobClient .getJobExecutionResult ())
186
+ .failsWithin (Duration .ofSeconds (1 ))
187
+ .withThrowableOfType (ExecutionException .class )
188
+ .withMessageContaining ("Job was cancelled" );
189
+
190
+ assertThatMiniClusterIsShutdown ();
191
+ }
192
+
155
193
private PerJobMiniClusterFactory initializeMiniCluster () {
194
+ return initializeMiniCluster (new Configuration ());
195
+ }
196
+
197
+ private PerJobMiniClusterFactory initializeMiniCluster (Configuration configuration ) {
156
198
return PerJobMiniClusterFactory .createWithFactory (
157
- new Configuration () ,
199
+ configuration ,
158
200
config -> {
159
201
miniCluster = new MiniCluster (config );
160
202
return miniCluster ;
@@ -175,4 +217,32 @@ private static JobGraph getCancellableJobGraph() {
175
217
jobVertex .setParallelism (1 );
176
218
return JobGraphTestUtils .streamingJobGraph (jobVertex );
177
219
}
220
+
221
+ private static JobVertex getBlockingJobVertex () {
222
+ JobVertex jobVertex = new JobVertex ("jobVertex" );
223
+ jobVertex .setInvokableClass (BlockingInvokable .class );
224
+ jobVertex .setParallelism (2 );
225
+ return jobVertex ;
226
+ }
227
+
228
+ /** Test invokable that allows waiting for all subtasks to be running. */
229
+ public static class BlockingInvokable extends CancelableInvokable {
230
+
231
+ private static CountDownLatch latch ;
232
+
233
+ public BlockingInvokable (Environment environment ) {
234
+ super (environment );
235
+ }
236
+
237
+ @ Override
238
+ public void doInvoke () throws Exception {
239
+ checkState (latch != null , "The invokable should be reset first." );
240
+ latch .countDown ();
241
+ waitUntilCancelled ();
242
+ }
243
+
244
+ public static void reset (int count ) {
245
+ latch = new CountDownLatch (count );
246
+ }
247
+ }
178
248
}
0 commit comments