Apache Flink作为国内最火的大数据计算引擎之一,自身支持高吞吐,低延迟,exactly-once语义,有状态流等特性,阅读源码有助加深对框架的理解和认知。
Flink分为四种执行图,本章主要解析JobGraph的生成计划
本章及后续源码解读环境以生产为主:运行模式:OnYarn,HA模式:ZK,Mode:Streaming,关键逻辑解释我会备注到代码上(灰色字体)勿忽视。
Flink会生成JobGraph去优化StreamGraph,其中包括设置Checkpoint,Slot分组策略,内存占比外,最主要的还是将符合条件的Operator组合成ChainableOperator,生成对应的JobVertex,IntermediateDateSet,JobEdge等核心组件并通过JobEdge连接上IntermediateDateSet和JobVertex,但目前JobGraph生成的JobVertex等还是只是粗粒度的用户代码的逻辑结构,包括像IntermediateDateSet也并不存储数据而仅仅是维护了上下游的生产消费者的数据结构而已,而真正包含物理执行的需要等后面生成最细粒度的Task实例时所构造的ResultSubPartition,InpuGate等才会交互用户的物理数据(后面章节会解析)。
接着上一章节开始,当StreamGraph生成完毕后,Flink会继续在Client端生成JobGraph(ExecutionGraph和物理执行图是生成在集群的AM上)
@Internal
public JobClient executeAsync(StreamGraph streamGraph) throws Exception {
checkNotNull(streamGraph, "StreamGraph cannot be null.");
checkNotNull(configuration.get(DeploymentOptions.TARGET), "No execution.target specified in your configuration file.");
//调用DefaultExecutorServiceLoader生成YarnJobClusterExecutorFactory
final PipelineExecutorFactory executorFactory =
executorServiceLoader.getExecutorFactory(configuration);
checkNotNull(
executorFactory,
"Cannot find compatible factory for specified execution.target (=%s)",
configuration.get(DeploymentOptions.TARGET));
//生成YarnJobClusterExecutor调用生成JobGraph后向集群提交任务资源申请
CompletableFuture<JobClient> jobClientFuture = executorFactory
.getExecutor(configuration) //new YarnJobClusterExecutor
.execute(streamGraph, configuration, userClassloader);
..
}
@Override
public CompletableFuture<JobClient> execute(@Nonnull final Pipeline pipeline, @Nonnull final Configuration configuration, @Nonnull final ClassLoader userCodeClassloader) throws Exception {
//生成JobGraph
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));
}
}
public static JobGraph getJobGraph(
Pipeline pipeline,
Configuration optimizerConfiguration,
int defaultParallelism) {
//根据Pipeline类型生成离线planTranslator或者实时的streamGraphTranslator
FlinkPipelineTranslator pipelineTranslator = getPipelineTranslator(pipeline);
return pipelineTranslator.translateToJobGraph(pipeline,
optimizerConfiguration,
defaultParallelism);
}
//生成StreamingJobGraphGenerator实例并创建JobGraph并
public static JobGraph createJobGraph(StreamGraph streamGraph, @Nullable JobID jobID) {
return new StreamingJobGraphGenerator(streamGraph, jobID).createJobGraph();
}
准备开始创建JobGraph,其中最核心的还是 setChaining(hashes, legacyHashes) 做可链优化。
private JobGraph createJobGraph() {
preValidate();
// make sure that all vertices start immediately
// 非batch模式默认为EAGER
jobGraph.setScheduleMode(streamGraph.getScheduleMode());
// Generate deterministic hashes for the nodes in order to identify them across
// submission iff they didn't change.
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));
}
setChaining(hashes, legacyHashes);
//保存产生所有JobEdge时对应的StreamEdge
setPhysicalEdges();
//设置slot分组策略
setSlotSharingAndCoLocation();
//设置缓存容量
setManagedMemoryFraction(
Collections.unmodifiableMap(jobVertices),
Collections.unmodifiableMap(vertexConfigs),
Collections.unmodifiableMap(chainedConfigs),
id -> streamGraph.getStreamNode(id).getManagedMemoryOperatorScopeUseCaseWeights(),
id -> streamGraph.getStreamNode(id).getManagedMemorySlotScopeUseCases());
//配置checkpoint属性
configureCheckpointing();
//配置savepoint属性
jobGraph.setSavepointRestoreSettings(streamGraph.getSavepointRestoreSettings());
//配置用户自定义文件
JobGraphUtils.addUserArtifactEntries(streamGraph.getUserArtifacts(), jobGraph);
// set the ExecutionConfig last when it has been finalized
try {
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");
}
return jobGraph;
}
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.
// 遍历并缓存所有的sourceNode以及生成后面用作checkpoint的SourceCoordinatorProvider
final Map<Integer, OperatorChainInfo> chainEntryPoints = buildChainedInputsAndGetHeadInputs(hashes, legacyHashes);
final Collection<OperatorChainInfo> initialEntryPoints = new ArrayList<>(chainEntryPoints.values());
// iterate over a copy of the values, because this map gets concurrently modified
for (OperatorChainInfo info : initialEntryPoints) {
createChain(
info.getStartNodeId(),
1, // operators start at position 1 because 0 is for chained source inputs
info,
chainEntryPoints);
}
}
Flink创建ChainableOperator的核心代码如下循环嵌套迭代比较多,大概逻辑为:
一:前向遍历:先把可链和不可链的edge缓存起来用作后面创建JobVertex和JobEdge,直到遍历到最后一个streamNode
二:后向遍历:嵌套代码依次跳出循环,若遇到断点就从第一个operator创建和链接JobVertex-JobEdge-IntermediateDataSet....
备注:注意我代码上的关键代码解释。
private List<StreamEdge> createChain(
final Integer currentNodeId,
final int chainIndex,
final OperatorChainInfo chainInfo,
final Map<Integer, OperatorChainInfo> chainEntryPoints) {
Integer startNodeId = chainInfo.getStartNodeId();
//遍历未解索过的streamNode(createJobVertex后builtVertices会加入当前nodeId避免后续重复调用)
if (!builtVertices.contains(startNodeId)) {
//不可chain的edges
List<StreamEdge> transitiveOutEdges = new ArrayList<StreamEdge>();
List<StreamEdge> chainableOutputs = new ArrayList<StreamEdge>();
List<StreamEdge> nonChainableOutputs = new ArrayList<StreamEdge>();
StreamNode currentNode = streamGraph.getStreamNode(currentNodeId);
//当前streamNode可能存在多个outEdges
for (StreamEdge outEdge : currentNode.getOutEdges()) {
//判断是否能chain
if (isChainable(outEdge, streamGraph)) {
chainableOutputs.add(outEdge);
} else {
nonChainableOutputs.add(outEdge);
}
}
//若上面streamNode可链则继续循环调用createChain
//此时获取下一个streamNodeId重复上述步骤并addAll所有不可链的edge用作后面connect直至当前的streamNode的outEdge不可链
for (StreamEdge chainable : chainableOutputs) {
transitiveOutEdges.addAll(
createChain(chainable.getTargetId(), chainIndex + 1, chainInfo, chainEntryPoints));
}
//走到这里 分为两种情况:
//一:当前streamNode的outEdge不可链 但是下游有streamNode
//二:当前streamNode的outEdge不可链 下游无streamNode,属于end stream node
//三:当前streamNode的outEdge不可链 下游已经创建完JobVertex(transitiveOutEdges.addAll(createChain....)循环调用完毕)
for (StreamEdge nonChainable : nonChainableOutputs) {
//进入这里属于outEdge不可链 但是下游有streamNode,先缓存起来,使用下个streamNode继续调用createChain,并且初始化chainIndex为1(因为下个streamNode是链头
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);
}
//代码第一次走到这里说明createChain遍历到最后一个streamNode,所以从这里开始(反向)创建JobVertex
//首先遍历出chainableOutputs 并merge当前链中所有StreamNode的资源(比如cpuCores,HeapMemory等)
chainedNames.put(currentNodeId, createChainedName(currentNodeId, chainableOutputs));
chainedMinResources.put(currentNodeId, createChainedMinResources(currentNodeId, chainableOutputs));
chainedPreferredResources.put(currentNodeId, createChainedPreferredResources(currentNodeId, chainableOutputs));
OperatorID currentOperatorId = chainInfo.addNodeToChain(currentNodeId, chainedNames.get(currentNodeId));
//缓存所有inputFormat(比如jdbcinputformat,hbaseInputFormat等)
if (currentNode.getInputFormat() != null) {
getOrCreateFormatContainer(startNodeId).addInputFormat(currentOperatorId, currentNode.getInputFormat());
}
//同上
if (currentNode.getOutputFormat() != null) {
getOrCreateFormatContainer(startNodeId).addOutputFormat(currentOperatorId, currentNode.getOutputFormat());
}
//如果属于起始节点则生成JobVertex,主要为以下三步骤
//1:创建JobVertex(如果是inputFormat或者outputFormat类型则是InputOutputFormatVertex)
//2:设置并行度,资源等属性
//3:缓存进StreamingJobGraphGenerator中
StreamConfig config = currentNodeId.equals(startNodeId)
? createJobVertex(startNodeId, chainInfo)
: new StreamConfig(new Configuration());
setVertexConfig(currentNodeId, config, chainableOutputs, nonChainableOutputs, chainInfo.getChainedSources());
//跟上面生成JobVertex一样 只有当前operator是head时候 才生成生成IntermediateDataSet 和 JobEdge 并跟下游相连
if (currentNodeId.equals(startNodeId)) {
config.setChainStart();
config.setChainIndex(chainIndex);
config.setOperatorName(streamGraph.getStreamNode(currentNodeId).getOperatorName());
//遍历出不可链的edge,生成IntermediateDataSet 和 JobEdge 并跟当前currentNodeId相连
for (StreamEdge edge : transitiveOutEdges) {
connect(startNodeId, edge);
}
config.setOutEdgesInOrder(transitiveOutEdges);
config.setTransitiveChainedTaskConfigs(chainedConfigs.get(startNodeId));
} else {
chainedConfigs.computeIfAbsent(startNodeId, k -> new HashMap<Integer, StreamConfig>());
config.setChainIndex(chainIndex);
StreamNode node = streamGraph.getStreamNode(currentNodeId);
config.setOperatorName(node.getOperatorName());
chainedConfigs.get(startNodeId).put(currentNodeId, config);
}
config.setOperatorID(currentOperatorId);
if (chainableOutputs.isEmpty()) {
config.setChainEnd();
}
return transitiveOutEdges;
} else {
return new ArrayList<>();
}
}
其中JobVertex的链接逻辑:
private void connect(Integer headOfChain, StreamEdge edge) {
....
ResultPartitionType resultPartitionType;
//根据outEdge的shuffleMode类型生成不同的ResultPartition
//ResultPartition的类型决定后面的数据交互中的流或批模式以及是否可做反压和基于Credit的通信模式(后面会做分析)
//默认是UNDEFINED,此时将由partitioner类型和GlobalDataExchangeMode决定
switch (edge.getShuffleMode()) {
case PIPELINED:
resultPartitionType = ResultPartitionType.PIPELINED_BOUNDED;
break;
case BATCH:
resultPartitionType = ResultPartitionType.BLOCKING;
break;
case UNDEFINED:
resultPartitionType = determineResultPartitionType(partitioner);
break;
default:
throw new UnsupportedOperationException("Data exchange mode " +
edge.getShuffleMode() + " is not supported yet.");
}
checkAndResetBufferTimeout(resultPartitionType, edge);
//创建JobEdge,上游为IntermediateDataSet,下游为JobVertex
//IntermediateDataSet只是维护了它的producer和consumers的队列
JobEdge jobEdge;
if (isPointwisePartitioner(partitioner)) {
jobEdge = downStreamVertex.connectNewDataSetAsInput(
headVertex,
DistributionPattern.POINTWISE,
resultPartitionType);
} else {
jobEdge = downStreamVertex.connectNewDataSetAsInput(
headVertex,
DistributionPattern.ALL_TO_ALL,
resultPartitionType);
}
// set strategy name so that web interface can show it.
jobEdge.setShipStrategyName(partitioner.toString());
if (LOG.isDebugEnabled()) {
LOG.debug("CONNECTED: {} - {} -> {}", partitioner.getClass().getSimpleName(),
headOfChain, downStreamVertexID);
}
}
public JobEdge connectNewDataSetAsInput(
JobVertex input,
DistributionPattern distPattern,
ResultPartitionType partitionType) {
IntermediateDataSet dataSet = input.createAndAddResultDataSet(partitionType);
//创建JobEdge
JobEdge edge = new JobEdge(dataSet, this, distPattern);
this.inputs.add(edge);
dataSet.addConsumer(edge);
return edge;
}
public IntermediateDataSet createAndAddResultDataSet(
IntermediateDataSetID id,
ResultPartitionType partitionType) {
//只是维护了当前JobVertex所有outEdge和ResultPartitionType
IntermediateDataSet result = new IntermediateDataSet(id, partitionType, this);
//后面在创建JobVertex的时候会根据results.size生成相同数量的IntermediateResult
this.results.add(result);
return result;
}
至此,JobGraph构建完成,整个过程跟生成StreamGraph类似而且都是发生在Client端,而后面构建ExecutionGraph和物理执行图是部署在集群上,也会交互不同的外部依赖,相对前面的步骤更加复杂,具体细节放在后面章分析。Thanks。