/
ExecutionJobVertex.java
676 lines (574 loc) · 25.8 KB
/
ExecutionJobVertex.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.runtime.executiongraph;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.common.Archiveable;
import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.accumulators.Accumulator;
import org.apache.flink.api.common.accumulators.AccumulatorHelper;
import org.apache.flink.api.common.time.Time;
import org.apache.flink.configuration.MemorySize;
import org.apache.flink.core.io.InputSplit;
import org.apache.flink.core.io.InputSplitAssigner;
import org.apache.flink.core.io.InputSplitSource;
import org.apache.flink.runtime.JobException;
import org.apache.flink.runtime.OperatorIDPair;
import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult;
import org.apache.flink.runtime.blob.BlobWriter;
import org.apache.flink.runtime.blob.PermanentBlobKey;
import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
import org.apache.flink.runtime.execution.ExecutionState;
import org.apache.flink.runtime.jobgraph.IntermediateDataSet;
import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
import org.apache.flink.runtime.jobgraph.JobEdge;
import org.apache.flink.runtime.jobgraph.JobVertex;
import org.apache.flink.runtime.jobgraph.JobVertexID;
import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup;
import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
import org.apache.flink.runtime.metrics.groups.JobManagerJobMetricGroup;
import org.apache.flink.runtime.operators.coordination.CoordinatorStore;
import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
import org.apache.flink.runtime.operators.coordination.OperatorCoordinatorHolder;
import org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator;
import org.apache.flink.runtime.scheduler.VertexParallelismInformation;
import org.apache.flink.runtime.source.coordinator.SourceCoordinator;
import org.apache.flink.types.Either;
import org.apache.flink.util.IOUtils;
import org.apache.flink.util.OptionalFailure;
import org.apache.flink.util.SerializedValue;
import org.apache.flink.util.concurrent.FutureUtils;
import org.slf4j.Logger;
import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CompletableFuture;
import java.util.function.Function;
import java.util.stream.Collectors;
import static org.apache.flink.util.Preconditions.checkNotNull;
import static org.apache.flink.util.Preconditions.checkState;
/**
* An {@code ExecutionJobVertex} is part of the {@link ExecutionGraph}, and the peer to the {@link
* JobVertex}.
*
* <p>The {@code ExecutionJobVertex} corresponds to a parallelized operation. It contains an {@link
* ExecutionVertex} for each parallel instance of that operation.
*/
public class ExecutionJobVertex
implements AccessExecutionJobVertex, Archiveable<ArchivedExecutionJobVertex> {
/** Use the same log for all ExecutionGraph classes. */
private static final Logger LOG = DefaultExecutionGraph.LOG;
private final Object stateMonitor = new Object();
private final InternalExecutionGraphAccessor graph;
private final JobVertex jobVertex;
@Nullable private ExecutionVertex[] taskVertices;
@Nullable private IntermediateResult[] producedDataSets;
@Nullable private List<IntermediateResult> inputs;
private final VertexParallelismInformation parallelismInfo;
private final SlotSharingGroup slotSharingGroup;
@Nullable private final CoLocationGroup coLocationGroup;
@Nullable private InputSplit[] inputSplits;
private final ResourceProfile resourceProfile;
private int numExecutionVertexFinished;
/**
* Either store a serialized task information, which is for all sub tasks the same, or the
* permanent blob key of the offloaded task information BLOB containing the serialized task
* information.
*/
private Either<SerializedValue<TaskInformation>, PermanentBlobKey> taskInformationOrBlobKey =
null;
private final Collection<OperatorCoordinatorHolder> operatorCoordinators;
@Nullable private InputSplitAssigner splitAssigner;
@VisibleForTesting
public ExecutionJobVertex(
InternalExecutionGraphAccessor graph,
JobVertex jobVertex,
VertexParallelismInformation parallelismInfo,
CoordinatorStore coordinatorStore,
JobManagerJobMetricGroup jobManagerJobMetricGroup)
throws JobException {
if (graph == null || jobVertex == null) {
throw new NullPointerException();
}
this.graph = graph;
this.jobVertex = jobVertex;
this.parallelismInfo = parallelismInfo;
// verify that our parallelism is not higher than the maximum parallelism
if (this.parallelismInfo.getParallelism() > this.parallelismInfo.getMaxParallelism()) {
throw new JobException(
String.format(
"Vertex %s's parallelism (%s) is higher than the max parallelism (%s). Please lower the parallelism or increase the max parallelism.",
jobVertex.getName(),
this.parallelismInfo.getParallelism(),
this.parallelismInfo.getMaxParallelism()));
}
this.resourceProfile =
ResourceProfile.fromResourceSpec(jobVertex.getMinResources(), MemorySize.ZERO);
// take the sharing group
this.slotSharingGroup = checkNotNull(jobVertex.getSlotSharingGroup());
this.coLocationGroup = jobVertex.getCoLocationGroup();
final List<SerializedValue<OperatorCoordinator.Provider>> coordinatorProviders =
getJobVertex().getOperatorCoordinators();
if (coordinatorProviders.isEmpty()) {
this.operatorCoordinators = Collections.emptyList();
} else {
final ArrayList<OperatorCoordinatorHolder> coordinators =
new ArrayList<>(coordinatorProviders.size());
try {
for (final SerializedValue<OperatorCoordinator.Provider> provider :
coordinatorProviders) {
coordinators.add(
createOperatorCoordinatorHolder(
provider,
graph.getUserClassLoader(),
coordinatorStore,
jobManagerJobMetricGroup));
}
} catch (Exception | LinkageError e) {
IOUtils.closeAllQuietly(coordinators);
throw new JobException(
"Cannot instantiate the coordinator for operator " + getName(), e);
}
this.operatorCoordinators = Collections.unmodifiableList(coordinators);
}
}
protected void initialize(
int executionHistorySizeLimit,
Time timeout,
long createTimestamp,
SubtaskAttemptNumberStore initialAttemptCounts)
throws JobException {
checkState(parallelismInfo.getParallelism() > 0);
checkState(!isInitialized());
this.taskVertices = new ExecutionVertex[parallelismInfo.getParallelism()];
this.inputs = new ArrayList<>(jobVertex.getInputs().size());
// create the intermediate results
this.producedDataSets =
new IntermediateResult[jobVertex.getNumberOfProducedIntermediateDataSets()];
for (int i = 0; i < jobVertex.getProducedDataSets().size(); i++) {
final IntermediateDataSet result = jobVertex.getProducedDataSets().get(i);
this.producedDataSets[i] =
new IntermediateResult(
result,
this,
this.parallelismInfo.getParallelism(),
result.getResultType());
}
// create all task vertices
for (int i = 0; i < this.parallelismInfo.getParallelism(); i++) {
ExecutionVertex vertex =
createExecutionVertex(
this,
i,
producedDataSets,
timeout,
createTimestamp,
executionHistorySizeLimit,
initialAttemptCounts.getAttemptCount(i));
this.taskVertices[i] = vertex;
}
// sanity check for the double referencing between intermediate result partitions and
// execution vertices
for (IntermediateResult ir : this.producedDataSets) {
if (ir.getNumberOfAssignedPartitions() != this.parallelismInfo.getParallelism()) {
throw new RuntimeException(
"The intermediate result's partitions were not correctly assigned.");
}
}
// set up the input splits, if the vertex has any
try {
@SuppressWarnings("unchecked")
InputSplitSource<InputSplit> splitSource =
(InputSplitSource<InputSplit>) jobVertex.getInputSplitSource();
if (splitSource != null) {
Thread currentThread = Thread.currentThread();
ClassLoader oldContextClassLoader = currentThread.getContextClassLoader();
currentThread.setContextClassLoader(graph.getUserClassLoader());
try {
inputSplits =
splitSource.createInputSplits(this.parallelismInfo.getParallelism());
if (inputSplits != null) {
splitAssigner = splitSource.getInputSplitAssigner(inputSplits);
}
} finally {
currentThread.setContextClassLoader(oldContextClassLoader);
}
} else {
inputSplits = null;
}
} catch (Throwable t) {
throw new JobException(
"Creating the input splits caused an error: " + t.getMessage(), t);
}
}
protected ExecutionVertex createExecutionVertex(
ExecutionJobVertex jobVertex,
int subTaskIndex,
IntermediateResult[] producedDataSets,
Time timeout,
long createTimestamp,
int executionHistorySizeLimit,
int initialAttemptCount) {
return new ExecutionVertex(
jobVertex,
subTaskIndex,
producedDataSets,
timeout,
createTimestamp,
executionHistorySizeLimit,
initialAttemptCount);
}
protected OperatorCoordinatorHolder createOperatorCoordinatorHolder(
SerializedValue<OperatorCoordinator.Provider> provider,
ClassLoader classLoader,
CoordinatorStore coordinatorStore,
JobManagerJobMetricGroup jobManagerJobMetricGroup)
throws Exception {
return OperatorCoordinatorHolder.create(
provider,
this,
classLoader,
coordinatorStore,
false,
getTaskInformation(),
jobManagerJobMetricGroup);
}
public boolean isInitialized() {
return taskVertices != null;
}
public boolean isParallelismDecided() {
return parallelismInfo.getParallelism() > 0;
}
/**
* Returns a list containing the ID pairs of all operators contained in this execution job
* vertex.
*
* @return list containing the ID pairs of all contained operators
*/
public List<OperatorIDPair> getOperatorIDs() {
return jobVertex.getOperatorIDs();
}
public void setMaxParallelism(int maxParallelism) {
parallelismInfo.setMaxParallelism(maxParallelism);
}
public InternalExecutionGraphAccessor getGraph() {
return graph;
}
public void setParallelism(int parallelism) {
parallelismInfo.setParallelism(parallelism);
}
public JobVertex getJobVertex() {
return jobVertex;
}
@Override
public String getName() {
return getJobVertex().getName();
}
@Override
public int getParallelism() {
return parallelismInfo.getParallelism();
}
@Override
public int getMaxParallelism() {
return parallelismInfo.getMaxParallelism();
}
@Override
public ResourceProfile getResourceProfile() {
return resourceProfile;
}
public boolean canRescaleMaxParallelism(int desiredMaxParallelism) {
return parallelismInfo.canRescaleMaxParallelism(desiredMaxParallelism);
}
public JobID getJobId() {
return graph.getJobID();
}
@Override
public JobVertexID getJobVertexId() {
return jobVertex.getID();
}
@Override
public ExecutionVertex[] getTaskVertices() {
if (taskVertices == null) {
// The REST/web may try to get execution vertices of an uninitialized job vertex. Using
// DEBUG log level to avoid flooding logs.
LOG.debug(
"Trying to get execution vertices of an uninitialized job vertex "
+ getJobVertexId());
return new ExecutionVertex[0];
}
return taskVertices;
}
public IntermediateResult[] getProducedDataSets() {
checkState(isInitialized());
return producedDataSets;
}
public InputSplitAssigner getSplitAssigner() {
checkState(isInitialized());
return splitAssigner;
}
@Override
public SlotSharingGroup getSlotSharingGroup() {
return slotSharingGroup;
}
@Nullable
public CoLocationGroup getCoLocationGroup() {
return coLocationGroup;
}
public List<IntermediateResult> getInputs() {
checkState(isInitialized());
return inputs;
}
public Collection<OperatorCoordinatorHolder> getOperatorCoordinators() {
checkState(isInitialized());
return operatorCoordinators;
}
public List<SourceCoordinator<?, ?>> getSourceCoordinators() {
List<SourceCoordinator<?, ?>> sourceCoordinators = new ArrayList<>();
for (OperatorCoordinatorHolder oph : operatorCoordinators) {
if (oph.coordinator() instanceof RecreateOnResetOperatorCoordinator) {
RecreateOnResetOperatorCoordinator opc =
(RecreateOnResetOperatorCoordinator) oph.coordinator();
try {
if (opc.getInternalCoordinator() instanceof SourceCoordinator) {
sourceCoordinators.add(
(SourceCoordinator<?, ?>) opc.getInternalCoordinator());
}
} catch (Throwable e) {
throw new RuntimeException(
"Unexpected error occurred when get sourceCoordinators.", e);
}
}
}
return sourceCoordinators;
}
int getNumExecutionVertexFinished() {
return numExecutionVertexFinished;
}
public Either<SerializedValue<TaskInformation>, PermanentBlobKey> getTaskInformationOrBlobKey()
throws IOException {
// only one thread should offload the task information, so let's also let only one thread
// serialize the task information!
synchronized (stateMonitor) {
if (taskInformationOrBlobKey == null) {
final BlobWriter blobWriter = graph.getBlobWriter();
final TaskInformation taskInformation = getTaskInformation();
taskInformationOrBlobKey =
BlobWriter.serializeAndTryOffload(taskInformation, getJobId(), blobWriter);
}
return taskInformationOrBlobKey;
}
}
public TaskInformation getTaskInformation() {
return new TaskInformation(
jobVertex.getID(),
jobVertex.getName(),
parallelismInfo.getParallelism(),
parallelismInfo.getMaxParallelism(),
jobVertex.getInvokableClassName(),
jobVertex.getConfiguration());
}
@Override
public ExecutionState getAggregateState() {
int[] num = new int[ExecutionState.values().length];
for (ExecutionVertex vertex : getTaskVertices()) {
num[vertex.getExecutionState().ordinal()]++;
}
return getAggregateJobVertexState(num, this.parallelismInfo.getParallelism());
}
// ---------------------------------------------------------------------------------------------
public void connectToPredecessors(
Map<IntermediateDataSetID, IntermediateResult> intermediateDataSets)
throws JobException {
checkState(isInitialized());
List<JobEdge> inputs = jobVertex.getInputs();
if (LOG.isDebugEnabled()) {
LOG.debug(
String.format(
"Connecting ExecutionJobVertex %s (%s) to %d predecessors.",
jobVertex.getID(), jobVertex.getName(), inputs.size()));
}
for (int num = 0; num < inputs.size(); num++) {
JobEdge edge = inputs.get(num);
if (LOG.isDebugEnabled()) {
if (edge.getSource() == null) {
LOG.debug(
String.format(
"Connecting input %d of vertex %s (%s) to intermediate result referenced via ID %s.",
num,
jobVertex.getID(),
jobVertex.getName(),
edge.getSourceId()));
} else {
LOG.debug(
String.format(
"Connecting input %d of vertex %s (%s) to intermediate result referenced via predecessor %s (%s).",
num,
jobVertex.getID(),
jobVertex.getName(),
edge.getSource().getProducer().getID(),
edge.getSource().getProducer().getName()));
}
}
// fetch the intermediate result via ID. if it does not exist, then it either has not
// been created, or the order
// in which this method is called for the job vertices is not a topological order
IntermediateResult ires = intermediateDataSets.get(edge.getSourceId());
if (ires == null) {
throw new JobException(
"Cannot connect this job graph to the previous graph. No previous intermediate result found for ID "
+ edge.getSourceId());
}
this.inputs.add(ires);
EdgeManagerBuildUtil.connectVertexToResult(this, ires);
}
}
// ---------------------------------------------------------------------------------------------
// Actions
// ---------------------------------------------------------------------------------------------
/** Cancels all currently running vertex executions. */
public void cancel() {
for (ExecutionVertex ev : getTaskVertices()) {
ev.cancel();
}
}
/**
* Cancels all currently running vertex executions.
*
* @return A future that is complete once all tasks have canceled.
*/
public CompletableFuture<Void> cancelWithFuture() {
return FutureUtils.waitForAll(mapExecutionVertices(ExecutionVertex::cancel));
}
public CompletableFuture<Void> suspend() {
return FutureUtils.waitForAll(mapExecutionVertices(ExecutionVertex::suspend));
}
@Nonnull
private Collection<CompletableFuture<?>> mapExecutionVertices(
final Function<ExecutionVertex, CompletableFuture<?>> mapFunction) {
return Arrays.stream(getTaskVertices()).map(mapFunction).collect(Collectors.toList());
}
public void fail(Throwable t) {
for (ExecutionVertex ev : getTaskVertices()) {
ev.fail(t);
}
}
void executionVertexFinished() {
checkState(isInitialized());
numExecutionVertexFinished++;
if (numExecutionVertexFinished == parallelismInfo.getParallelism()) {
getGraph().jobVertexFinished();
}
}
void executionVertexUnFinished() {
checkState(isInitialized());
if (numExecutionVertexFinished == parallelismInfo.getParallelism()) {
getGraph().jobVertexUnFinished();
}
numExecutionVertexFinished--;
}
public boolean isFinished() {
return isParallelismDecided()
&& numExecutionVertexFinished == parallelismInfo.getParallelism();
}
// --------------------------------------------------------------------------------------------
// Accumulators / Metrics
// --------------------------------------------------------------------------------------------
public StringifiedAccumulatorResult[] getAggregatedUserAccumulatorsStringified() {
Map<String, OptionalFailure<Accumulator<?, ?>>> userAccumulators = new HashMap<>();
for (ExecutionVertex vertex : getTaskVertices()) {
Map<String, Accumulator<?, ?>> next =
vertex.getCurrentExecutionAttempt().getUserAccumulators();
if (next != null) {
AccumulatorHelper.mergeInto(userAccumulators, next);
}
}
return StringifiedAccumulatorResult.stringifyAccumulatorResults(userAccumulators);
}
// --------------------------------------------------------------------------------------------
// Archiving
// --------------------------------------------------------------------------------------------
@Override
public ArchivedExecutionJobVertex archive() {
return new ArchivedExecutionJobVertex(this);
}
// ------------------------------------------------------------------------
// Static Utilities
// ------------------------------------------------------------------------
/**
* A utility function that computes an "aggregated" state for the vertex.
*
* <p>This state is not used anywhere in the coordination, but can be used for display in
* dashboards to as a summary for how the particular parallel operation represented by this
* ExecutionJobVertex is currently behaving.
*
* <p>For example, if at least one parallel task is failed, the aggregate state is failed. If
* not, and at least one parallel task is cancelling (or cancelled), the aggregate state is
* cancelling (or cancelled). If all tasks are finished, the aggregate state is finished, and so
* on.
*
* @param verticesPerState The number of vertices in each state (indexed by the ordinal of the
* ExecutionState values).
* @param parallelism The parallelism of the ExecutionJobVertex
* @return The aggregate state of this ExecutionJobVertex.
*/
public static ExecutionState getAggregateJobVertexState(
int[] verticesPerState, int parallelism) {
if (verticesPerState == null || verticesPerState.length != ExecutionState.values().length) {
throw new IllegalArgumentException(
"Must provide an array as large as there are execution states.");
}
if (verticesPerState[ExecutionState.FAILED.ordinal()] > 0) {
return ExecutionState.FAILED;
}
if (verticesPerState[ExecutionState.CANCELING.ordinal()] > 0) {
return ExecutionState.CANCELING;
} else if (verticesPerState[ExecutionState.CANCELED.ordinal()] > 0) {
return ExecutionState.CANCELED;
} else if (verticesPerState[ExecutionState.INITIALIZING.ordinal()] > 0) {
return ExecutionState.INITIALIZING;
} else if (verticesPerState[ExecutionState.RUNNING.ordinal()] > 0) {
return ExecutionState.RUNNING;
} else if (verticesPerState[ExecutionState.FINISHED.ordinal()] > 0) {
return verticesPerState[ExecutionState.FINISHED.ordinal()] == parallelism
? ExecutionState.FINISHED
: ExecutionState.RUNNING;
} else {
// all else collapses under created
return ExecutionState.CREATED;
}
}
/** Factory to create {@link ExecutionJobVertex}. */
public static class Factory {
ExecutionJobVertex createExecutionJobVertex(
InternalExecutionGraphAccessor graph,
JobVertex jobVertex,
VertexParallelismInformation parallelismInfo,
CoordinatorStore coordinatorStore,
JobManagerJobMetricGroup jobManagerJobMetricGroup)
throws JobException {
return new ExecutionJobVertex(
graph, jobVertex, parallelismInfo, coordinatorStore, jobManagerJobMetricGroup);
}
}
}