-
Notifications
You must be signed in to change notification settings - Fork 13k
/
TaskDeploymentDescriptorFactory.java
267 lines (237 loc) · 12.3 KB
/
TaskDeploymentDescriptorFactory.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
/*
* 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.deployment;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.common.JobID;
import org.apache.flink.runtime.blob.PermanentBlobKey;
import org.apache.flink.runtime.checkpoint.JobManagerTaskRestore;
import org.apache.flink.runtime.clusterframework.types.AllocationID;
import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor.MaybeOffloaded;
import org.apache.flink.runtime.execution.ExecutionState;
import org.apache.flink.runtime.executiongraph.Execution;
import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
import org.apache.flink.runtime.executiongraph.ExecutionGraph;
import org.apache.flink.runtime.executiongraph.ExecutionVertex;
import org.apache.flink.runtime.executiongraph.IntermediateResult;
import org.apache.flink.runtime.executiongraph.IntermediateResultPartition;
import org.apache.flink.runtime.executiongraph.JobInformation;
import org.apache.flink.runtime.executiongraph.TaskInformation;
import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
import org.apache.flink.runtime.scheduler.strategy.ConsumedPartitionGroup;
import org.apache.flink.runtime.shuffle.ShuffleDescriptor;
import org.apache.flink.runtime.shuffle.UnknownShuffleDescriptor;
import org.apache.flink.types.Either;
import org.apache.flink.util.SerializedValue;
import javax.annotation.Nullable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.Optional;
/**
* Factory of {@link TaskDeploymentDescriptor} to deploy {@link
* org.apache.flink.runtime.taskmanager.Task} from {@link Execution}.
*/
public class TaskDeploymentDescriptorFactory {
private final ExecutionAttemptID executionId;
private final int attemptNumber;
private final MaybeOffloaded<JobInformation> serializedJobInformation;
private final MaybeOffloaded<TaskInformation> taskInfo;
private final JobID jobID;
private final boolean allowUnknownPartitions;
private final int subtaskIndex;
private final List<List<IntermediateResultPartition>> consumedPartitions;
private TaskDeploymentDescriptorFactory(
ExecutionAttemptID executionId,
int attemptNumber,
MaybeOffloaded<JobInformation> serializedJobInformation,
MaybeOffloaded<TaskInformation> taskInfo,
JobID jobID,
boolean allowUnknownPartitions,
int subtaskIndex,
List<List<IntermediateResultPartition>> consumedPartitions) {
this.executionId = executionId;
this.attemptNumber = attemptNumber;
this.serializedJobInformation = serializedJobInformation;
this.taskInfo = taskInfo;
this.jobID = jobID;
this.allowUnknownPartitions = allowUnknownPartitions;
this.subtaskIndex = subtaskIndex;
this.consumedPartitions = consumedPartitions;
}
public TaskDeploymentDescriptor createDeploymentDescriptor(
AllocationID allocationID,
@Nullable JobManagerTaskRestore taskRestore,
Collection<ResultPartitionDeploymentDescriptor> producedPartitions) {
return new TaskDeploymentDescriptor(
jobID,
serializedJobInformation,
taskInfo,
executionId,
allocationID,
subtaskIndex,
attemptNumber,
taskRestore,
new ArrayList<>(producedPartitions),
createInputGateDeploymentDescriptors());
}
private List<InputGateDeploymentDescriptor> createInputGateDeploymentDescriptors() {
List<InputGateDeploymentDescriptor> inputGates = new ArrayList<>(consumedPartitions.size());
for (List<IntermediateResultPartition> partitions : consumedPartitions) {
// If the produced partition has multiple consumers registered, we
// need to request the one matching our sub task index.
// TODO Refactor after removing the consumers from the intermediate result partitions
IntermediateResultPartition resultPartition = partitions.get(0);
int numConsumer = resultPartition.getConsumers().get(0).getVertices().size();
int queueToRequest = subtaskIndex % numConsumer;
IntermediateResult consumedIntermediateResult = resultPartition.getIntermediateResult();
IntermediateDataSetID resultId = consumedIntermediateResult.getId();
ResultPartitionType partitionType = consumedIntermediateResult.getResultType();
inputGates.add(
new InputGateDeploymentDescriptor(
resultId,
partitionType,
queueToRequest,
getConsumedPartitionShuffleDescriptors(partitions)));
}
return inputGates;
}
private ShuffleDescriptor[] getConsumedPartitionShuffleDescriptors(
List<IntermediateResultPartition> partitions) {
ShuffleDescriptor[] shuffleDescriptors = new ShuffleDescriptor[partitions.size()];
// Each edge is connected to a different result partition
for (int i = 0; i < partitions.size(); i++) {
shuffleDescriptors[i] =
getConsumedPartitionShuffleDescriptor(
partitions.get(i), allowUnknownPartitions);
}
return shuffleDescriptors;
}
public static TaskDeploymentDescriptorFactory fromExecutionVertex(
ExecutionVertex executionVertex, int attemptNumber) throws IOException {
final ExecutionGraph executionGraph = executionVertex.getExecutionGraph();
final List<List<IntermediateResultPartition>> consumedPartitions = new ArrayList<>();
for (ConsumedPartitionGroup partitionGroup : executionVertex.getAllConsumedPartitions()) {
List<IntermediateResultPartition> partitions = new ArrayList<>();
for (IntermediateResultPartitionID partitionId : partitionGroup.getResultPartitions()) {
partitions.add(executionGraph.getResultPartitionOrThrow(partitionId));
}
consumedPartitions.add(partitions);
}
return new TaskDeploymentDescriptorFactory(
executionVertex.getCurrentExecutionAttempt().getAttemptId(),
attemptNumber,
getSerializedJobInformation(executionGraph),
getSerializedTaskInformation(
executionVertex.getJobVertex().getTaskInformationOrBlobKey()),
executionGraph.getJobID(),
executionGraph.getScheduleMode().allowLazyDeployment(),
executionVertex.getParallelSubtaskIndex(),
consumedPartitions);
}
private static MaybeOffloaded<JobInformation> getSerializedJobInformation(
ExecutionGraph executionGraph) {
Either<SerializedValue<JobInformation>, PermanentBlobKey> jobInformationOrBlobKey =
executionGraph.getJobInformationOrBlobKey();
if (jobInformationOrBlobKey.isLeft()) {
return new TaskDeploymentDescriptor.NonOffloaded<>(jobInformationOrBlobKey.left());
} else {
return new TaskDeploymentDescriptor.Offloaded<>(jobInformationOrBlobKey.right());
}
}
private static MaybeOffloaded<TaskInformation> getSerializedTaskInformation(
Either<SerializedValue<TaskInformation>, PermanentBlobKey> taskInfo) {
return taskInfo.isLeft()
? new TaskDeploymentDescriptor.NonOffloaded<>(taskInfo.left())
: new TaskDeploymentDescriptor.Offloaded<>(taskInfo.right());
}
public static ShuffleDescriptor getConsumedPartitionShuffleDescriptor(
IntermediateResultPartition consumedPartition, boolean allowUnknownPartitions) {
Execution producer = consumedPartition.getProducer().getCurrentExecutionAttempt();
ExecutionState producerState = producer.getState();
Optional<ResultPartitionDeploymentDescriptor> consumedPartitionDescriptor =
producer.getResultPartitionDeploymentDescriptor(consumedPartition.getPartitionId());
ResultPartitionID consumedPartitionId =
new ResultPartitionID(consumedPartition.getPartitionId(), producer.getAttemptId());
return getConsumedPartitionShuffleDescriptor(
consumedPartitionId,
consumedPartition.getResultType(),
consumedPartition.isConsumable(),
producerState,
allowUnknownPartitions,
consumedPartitionDescriptor.orElse(null));
}
@VisibleForTesting
static ShuffleDescriptor getConsumedPartitionShuffleDescriptor(
ResultPartitionID consumedPartitionId,
ResultPartitionType resultPartitionType,
boolean isConsumable,
ExecutionState producerState,
boolean allowUnknownPartitions,
@Nullable ResultPartitionDeploymentDescriptor consumedPartitionDescriptor) {
// The producing task needs to be RUNNING or already FINISHED
if ((resultPartitionType.isPipelined() || isConsumable)
&& consumedPartitionDescriptor != null
&& isProducerAvailable(producerState)) {
// partition is already registered
return consumedPartitionDescriptor.getShuffleDescriptor();
} else if (allowUnknownPartitions) {
// The producing task might not have registered the partition yet
return new UnknownShuffleDescriptor(consumedPartitionId);
} else {
// throw respective exceptions
throw handleConsumedPartitionShuffleDescriptorErrors(
consumedPartitionId, resultPartitionType, isConsumable, producerState);
}
}
private static RuntimeException handleConsumedPartitionShuffleDescriptorErrors(
ResultPartitionID consumedPartitionId,
ResultPartitionType resultPartitionType,
boolean isConsumable,
ExecutionState producerState) {
String msg;
if (isProducerFailedOrCanceled(producerState)) {
msg =
"Trying to consume an input partition whose producer has been canceled or failed. "
+ "The producer is in state "
+ producerState
+ ".";
} else {
msg =
String.format(
"Trying to consume an input partition whose producer "
+ "is not ready (result type: %s, partition consumable: %s, producer state: %s, partition id: %s).",
resultPartitionType, isConsumable, producerState, consumedPartitionId);
}
return new IllegalStateException(msg);
}
private static boolean isProducerAvailable(ExecutionState producerState) {
return producerState == ExecutionState.RUNNING
|| producerState == ExecutionState.FINISHED
|| producerState == ExecutionState.SCHEDULED
|| producerState == ExecutionState.DEPLOYING;
}
private static boolean isProducerFailedOrCanceled(ExecutionState producerState) {
return producerState == ExecutionState.CANCELING
|| producerState == ExecutionState.CANCELED
|| producerState == ExecutionState.FAILED;
}
}