StreamGraph 转变成 JobGraph 也是在 Client 完成,主要做了三件事:
书接上回,execute 方法中通过 getJobGraph 将 Pipeline 转换为 JobGraph,下面来看一下 getJobGraph 源码
我们从继承 Pipeline 接口的 AbstractJobClusterExecutor 类开始
AbstractJobClusterExecutor.java
public CompletableFuture<JobClient> execute(
@Nonnull final Pipeline pipeline,
@Nonnull final Configuration configuration,
@Nonnull final ClassLoader userCodeClassloader)
throws Exception {
final JobGraph jobGraph = PipelineExecutorUtils.getJobGraph(pipeline, configuration);
try (final ClusterDescriptor<ClusterID> clusterDescriptor =
clusterClientFactory.createClusterDescriptor(configuration)) {
final ExecutionConfigAccessor configAccessor =
ExecutionConfigAccessor.fromConfiguration(configuration);
final ClusterSpecification clusterSpecification =
clusterClientFactory.getClusterSpecification(configuration);
final ClusterClientProvider<ClusterID> clusterClientProvider =
clusterDescriptor.deployJobCluster(
clusterSpecification, jobGraph, configAccessor.getDetachedMode());
LOG.info("Job has been submitted with JobID " + jobGraph.getJobID());
return CompletableFuture.completedFuture(
new ClusterClientJobClientAdapter<>(
clusterClientProvider, jobGraph.getJobID(), userCodeClassloader));
}
}
PipelineExecutorUtils.java
public static JobGraph getJobGraph(
@Nonnull final Pipeline pipeline, @Nonnull final Configuration configuration)
throws MalformedURLException {
checkNotNull(pipeline);
checkNotNull(configuration);
final ExecutionConfigAccessor executionConfigAccessor =
ExecutionConfigAccessor.fromConfiguration(configuration);
final JobGraph jobGraph =
FlinkPipelineTranslationUtil.getJobGraph(
pipeline, configuration, executionConfigAccessor.getParallelism());
configuration
.getOptional(PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID)
.ifPresent(strJobID -> jobGraph.setJobID(JobID.fromHexString(strJobID)));
jobGraph.addJars(executionConfigAccessor.getJars());
jobGraph.setClasspaths(executionConfigAccessor.getClasspaths());
jobGraph.setSavepointRestoreSettings(executionConfigAccessor.getSavepointRestoreSettings());
return jobGraph;
}
继续往下,找到 FlinkPipelineTranslationUtil.getJobGraph
FlinkPipelineTranslationUtil.java
public static JobGraph getJobGraph(
Pipeline pipeline, Configuration optimizerConfiguration, int defaultParallelism) {
FlinkPipelineTranslator pipelineTranslator = getPipelineTranslator(pipeline);
return pipelineTranslator.translateToJobGraph(
pipeline, optimizerConfiguration, defaultParallelism);
}
看到 pipelineTranslator.translateToJobGraph 我们继续往下,发现 FlinkPipelineTranslator 是一个接口,找到其实现类 StreamGraphTranslator
StreamGraphTranslator.java
public JobGraph translateToJobGraph(
Pipeline pipeline, Configuration optimizerConfiguration, int defaultParallelism) {
checkArgument(
pipeline instanceof StreamGraph, "Given pipeline is not a DataStream StreamGraph.");
StreamGraph streamGraph = (StreamGraph) pipeline;
return streamGraph.getJobGraph(null);
}
兜了一圈最后还是回到 StreamGraph 的 getJobGraph 方法
StreamGraph.java
public JobGraph getJobGraph(@Nullable JobID jobID) {
return StreamingJobGraphGenerator.createJobGraph(this, jobID);
}
StreamingJobGraphGenerator.java
public static JobGraph createJobGraph(StreamGraph streamGraph) {
return new StreamingJobGraphGenerator(streamGraph, null, Runnable::run).createJobGraph();
}
这一行代码先是构造了 StreamingJobGraphGenerator 匿名类再调用 createJobGraph 方法。我们先来看一下属性和构造方法
private final StreamGraph streamGraph;
// id -> JobVertex
private final Map<Integer, JobVertex> jobVertices;
private final JobGraph jobGraph;
// 已经构建的 JobVertex 的 id 集合
private final Collection<Integer> builtVertices;
// 物理边集合(排除了 chain 内部的边),按创建顺序排序
private final List<StreamEdge> physicalEdgesInOrder;
// 保存 chain 信息,部署时用来创建 OperatorChain,startNodeId -> (currentNodeId -> StreamConfig)
private final Map<Integer, Map<Integer, StreamConfig>> chainedConfigs;
// 所有节点的配置信息,id -> StreamConfig
private final Map<Integer, StreamConfig> vertexConfigs;
// 保存每个节点的名字,id -> chainedName
private final Map<Integer, String> chainedNames;
private final Map<Integer, ResourceSpec> chainedMinResources;
private final Map<Integer, ResourceSpec> chainedPreferredResources;
private final Map<Integer, InputOutputFormatContainer> chainedInputOutputFormats;
private final StreamGraphHasher defaultStreamGraphHasher;
private final List<StreamGraphHasher> legacyStreamGraphHashers;
private boolean hasHybridResultPartition = false;
private final Executor serializationExecutor;
// Futures for the serialization of operator coordinators
private final List<CompletableFuture<Void>> coordinatorSerializationFutures = new ArrayList<>();
private final Map<Integer, Map<StreamEdge, NonChainedOutput>> opIntermediateOutputs;
// 构造函数
private StreamingJobGraphGenerator(
StreamGraph streamGraph, @Nullable JobID jobID, Executor serializationExecutor) {
this.streamGraph = streamGraph;
this.defaultStreamGraphHasher = new StreamGraphHasherV2();
this.legacyStreamGraphHashers = Arrays.asList(new StreamGraphUserHashHasher());
this.jobVertices = new HashMap<>();
this.builtVertices = new HashSet<>();
this.chainedConfigs = new HashMap<>();
this.vertexConfigs = new HashMap<>();
this.chainedNames = new HashMap<>();
this.chainedMinResources = new HashMap<>();
this.chainedPreferredResources = new HashMap<>();
this.chainedInputOutputFormats = new HashMap<>();
this.physicalEdgesInOrder = new ArrayList<>();
this.serializationExecutor = Preconditions.checkNotNull(serializationExecutor);
this.opIntermediateOutputs = new HashMap<>();
jobGraph = new JobGraph(jobID, streamGraph.getJobName());
}
核心逻辑:根据 StreamGraph,生成 JobGraph
private JobGraph createJobGraph() {
preValidate();
jobGraph.setJobType(streamGraph.getJobType());
jobGraph.enableApproximateLocalRecovery(
streamGraph.getCheckpointConfig().isApproximateLocalRecoveryEnabled());
// Generate deterministic hashes for the nodes in order to identify them across
// submission iff they didn't change.
// 广度优先遍历 StreamGraph,并且为每个 StreamNode 生成 hash id,
// 保证如果提交的拓扑没有改变,则每次生成的 hash 都是一样的
Map<Integer, byte[]> hashes =
defaultStreamGraphHasher.traverseStreamGraphAndGenerateHashes(streamGraph);
// Generate legacy version hashes for backwards compatibility
List<Map<Integer, byte[]>> legacyHashes = new ArrayList<>(legacyStreamGraphHashers.size());
for (StreamGraphHasher hasher : legacyStreamGraphHashers) {
legacyHashes.add(hasher.traverseStreamGraphAndGenerateHashes(streamGraph));
}
// 最重要的函数,生成 JobVertex,JobEdge 等,并尽可能地将多个节点 chain 在一起
setChaining(hashes, legacyHashes);
// 将每个 JobVertex 的入边集合也序列化到该 JobVertex 的 StreamConfig 中
// (出边集合已经在 setChaining 的时候写入了)
setPhysicalEdges();
markContainsSourcesOrSinks();
// 根据 group name,为每个 JobVertex 指定所属的 SlotSharingGroup
// 以及针对 Iteration 的头尾设置 CoLocationGroup
setSlotSharingAndCoLocation();
setManagedMemoryFraction(
Collections.unmodifiableMap(jobVertices),
Collections.unmodifiableMap(vertexConfigs),
Collections.unmodifiableMap(chainedConfigs),
id -> streamGraph.getStreamNode(id).getManagedMemoryOperatorScopeUseCaseWeights(),
id -> streamGraph.getStreamNode(id).getManagedMemorySlotScopeUseCases());
// 配置 checkpoint
configureCheckpointing();
jobGraph.setSavepointRestoreSettings(streamGraph.getSavepointRestoreSettings());
final Map<String, DistributedCache.DistributedCacheEntry> distributedCacheEntries =
JobGraphUtils.prepareUserArtifactEntries(
streamGraph.getUserArtifacts().stream()
.collect(Collectors.toMap(e -> e.f0, e -> e.f1)),
jobGraph.getJobID());
for (Map.Entry<String, DistributedCache.DistributedCacheEntry> entry :
distributedCacheEntries.entrySet()) {
jobGraph.addUserArtifact(entry.getKey(), entry.getValue());
}
// set the ExecutionConfig last when it has been finalized
try {
// 将 StreamGraph 的 ExecutionConfig 序列化到 JobGraph 的配置中
jobGraph.setExecutionConfig(streamGraph.getExecutionConfig());
} catch (IOException e) {
throw new IllegalConfigurationException(
"Could not serialize the ExecutionConfig."
+ "This indicates that non-serializable types (like custom serializers) were registered");
}
jobGraph.setChangelogStateBackendEnabled(streamGraph.isChangelogStateBackendEnabled());
addVertexIndexPrefixInVertexName();
setVertexDescription();
// Wait for the serialization of operator coordinators and stream config.
try {
FutureUtils.combineAll(
vertexConfigs.values().stream()
.map(
config ->
config.triggerSerializationAndReturnFuture(
serializationExecutor))
.collect(Collectors.toList()))
.get();
FutureUtils.combineAll(coordinatorSerializationFutures).get();
} catch (Exception e) {
throw new FlinkRuntimeException("Error in serialization.", e);
}
if (!streamGraph.getJobStatusHooks().isEmpty()) {
jobGraph.setJobStatusHooks(streamGraph.getJobStatusHooks());
}
return jobGraph;
}
StreamingJobGraphGenerator 的成员变量都是为了辅助生成最终的 JobGraph
为所有节点生成一个唯一的 hash id,如果节点在多次提交中没有改变(包括并发度、上下游等),那么这个 id 就不会改变,这主要用于故障恢复
这里不能用 StreamNode.id 来代替,因为这是一个从 1 开始的静态计数变量,同样的 Job 可能会得到不一样的 id,如下示例的两个 job 是完全一样的,但是 source 的 id 却不一样了。
// 范例1:A.id=1 B.id=2
DataStream<String> A = ...
DataStream<String> B = ...
A.union(B).print();
// 范例2:A.id=2 B.id=1
DataStream<String> B = ...
DataStream<String> A = ...
A.union(B).print();
接下来看一下最关键的 chaining 处理:
private void setChaining(Map<Integer, byte[]> hashes, List<Map<Integer, byte[]>> legacyHashes) {
// we separate out the sources that run as inputs to another operator (chained inputs)
// from the sources that needs to run as the main (head) operator.
final Map<Integer, OperatorChainInfo> chainEntryPoints =
buildChainedInputsAndGetHeadInputs(hashes, legacyHashes);
final Collection<OperatorChainInfo> initialEntryPoints =
chainEntryPoints.entrySet().stream()
.sorted(Comparator.comparing(Map.Entry::getKey))
.map(Map.Entry::getValue)
.collect(Collectors.toList());
// iterate over a copy of the values, because this map gets concurrently modified
// 从 source 开始建立 node chains
for (OperatorChainInfo info : initialEntryPoints) {
createChain(
info.getStartNodeId(),
1, // operators start at position 1 because 0 is for chained source inputs
info,
chainEntryPoints);
}
}
/**
* 构建 node chains,返回当前节点的物理出边
* startNodeId != currentNodeId 时,说明 currentNode 是 chain 中的子节点
* @param currentNodeId
* @param chainIndex
* @param chainInfo
* @param chainEntryPoints
* @return
*/
private List<StreamEdge> createChain(
final Integer currentNodeId,
final int chainIndex,
final OperatorChainInfo chainInfo,
final Map<Integer, OperatorChainInfo> chainEntryPoints) {
Integer startNodeId = chainInfo.getStartNodeId();
if (!builtVertices.contains(startNodeId)) {
// 过渡用的出边集合,用来生成最终的 JobEdge,注意不包括 chain 内部的边
List<StreamEdge> transitiveOutEdges = new ArrayList<StreamEdge>();
List<StreamEdge> chainableOutputs = new ArrayList<StreamEdge>();
List<StreamEdge> nonChainableOutputs = new ArrayList<StreamEdge>();
StreamNode currentNode = streamGraph.getStreamNode(currentNodeId);
// 将当前节点的出边分成 chainable 和 nonChainable 两类
for (StreamEdge outEdge : currentNode.getOutEdges()) {
if (isChainable(outEdge, streamGraph)) {
chainableOutputs.add(outEdge);
} else {
nonChainableOutputs.add(outEdge);
}
}
// 递归调用
for (StreamEdge chainable : chainableOutputs) {
transitiveOutEdges.addAll(
createChain(
chainable.getTargetId(),
chainIndex + 1,
chainInfo,
chainEntryPoints));
}
for (StreamEdge nonChainable : nonChainableOutputs) {
transitiveOutEdges.add(nonChainable);
createChain(
nonChainable.getTargetId(),
1, // operators start at position 1 because 0 is for chained source inputs
chainEntryPoints.computeIfAbsent(
nonChainable.getTargetId(),
(k) -> chainInfo.newChain(nonChainable.getTargetId())),
chainEntryPoints);
}
// 生成当前节点的显示名
chainedNames.put(
currentNodeId,
createChainedName(
currentNodeId,
chainableOutputs,
Optional.ofNullable(chainEntryPoints.get(currentNodeId))));
chainedMinResources.put(
currentNodeId, createChainedMinResources(currentNodeId, chainableOutputs));
chainedPreferredResources.put(
currentNodeId,
createChainedPreferredResources(currentNodeId, chainableOutputs));
OperatorID currentOperatorId =
chainInfo.addNodeToChain(
currentNodeId,
streamGraph.getStreamNode(currentNodeId).getOperatorName());
if (currentNode.getInputFormat() != null) {
getOrCreateFormatContainer(startNodeId)
.addInputFormat(currentOperatorId, currentNode.getInputFormat());
}
if (currentNode.getOutputFormat() != null) {
getOrCreateFormatContainer(startNodeId)
.addOutputFormat(currentOperatorId, currentNode.getOutputFormat());
}
// 如果当前节点是起始节点,则直接创建 JobVertex 并返回 StreamConfig,否则先创建一个空的 StreamConfig
// createJobVertex 方法就是根据 StreamNode 创建对应的 JobVertex,并返回了空的 StreamConfig
StreamConfig config =
currentNodeId.equals(startNodeId)
? createJobVertex(startNodeId, chainInfo)
: new StreamConfig(new Configuration());
// 设置 JobVertex 的 StreamConfig,基本上是序列化 StreamNode 中的配置到 StreamConfig 中
// 其中包括 序列化器,StreamOperator,Checkpoint 等相关配置
setVertexConfig(
currentNodeId,
config,
chainableOutputs,
nonChainableOutputs,
chainInfo.getChainedSources());
if (currentNodeId.equals(startNodeId)) {
// 如果 chain 是起始节点(不是 chain 中的节点,也会被标记成 chain start)
config.setChainStart();
config.setChainIndex(chainIndex);
config.setOperatorName(streamGraph.getStreamNode(currentNodeId).getOperatorName());
LinkedHashSet<NonChainedOutput> transitiveOutputs = new LinkedHashSet<>();
// 将当前节点(headOfChain)与所有出边相连
for (StreamEdge edge : transitiveOutEdges) {
NonChainedOutput output =
opIntermediateOutputs.get(edge.getSourceId()).get(edge);
transitiveOutputs.add(output);
// 通过 StreamEdge 构建出 JobEdge,创建 IntermediateDataSet
// 用来将 JobVertex 和 JobEdge 相连
connect(startNodeId, edge, output);
}
// 把物理出边写入配置,部署时会用到
config.setVertexNonChainedOutputs(new ArrayList<>(transitiveOutputs));
// 将 chain 中所有子节点的 StreamConfig
// 写入到 headOfChain 节点的 CHAINED_TASK_CONFIG 配置中
config.setTransitiveChainedTaskConfigs(chainedConfigs.get(startNodeId));
} else {
// 如果是 chain 中的子节点
chainedConfigs.computeIfAbsent(
startNodeId, k -> new HashMap<Integer, StreamConfig>());
config.setChainIndex(chainIndex);
StreamNode node = streamGraph.getStreamNode(currentNodeId);
config.setOperatorName(node.getOperatorName());
// 将当前节点的 StreamConfig 添加到该 chain 的 config 集合中
chainedConfigs.get(startNodeId).put(currentNodeId, config);
}
config.setOperatorID(currentOperatorId);
if (chainableOutputs.isEmpty()) {
config.setChainEnd();
}
// 返回连往 chain 外部的出边集合
return transitiveOutEdges;
} else {
return new ArrayList<>();
}
}