上一节介绍了StreamGraph的生成,这个实际上只对应 Flink 作业在逻辑上的执行计划图。Flink 会进一步对 StreamGraph 进行转换,得到另一个执行计划图,即JobGraph。然后将JobGraph发送到server端进行ExecutionGraph的解析。
主要的JobGraph的源码对象如下(其最主要的是Map<JobVertexID, JobVertex> taskVertices任务对象):
public class JobGraph implements Serializable {
// --- job and configuration ---
/** List of task vertices included in this job graph. */
private final Map<JobVertexID, JobVertex> taskVertices = new LinkedHashMap<JobVertexID, JobVertex>(); // JobGraph中包含的所有任务
/** The job configuration attached to this job. */
private final Configuration jobConfiguration = new Configuration();
/** ID of this job. May be set if specific job id is desired (e.g. session management) */
private final JobID jobID;
/** Name of this job. */
private final String jobName;
/** The number of seconds after which the corresponding ExecutionGraph is removed at the
* job manager after it has been executed. */
private long sessionTimeout = 0;
/** flag to enable queued scheduling */
private boolean allowQueuedScheduling;
/** The mode in which the job is scheduled */
private ScheduleMode scheduleMode = ScheduleMode.LAZY_FROM_SOURCES;
// --- checkpointing ---
/** Job specific execution config */
private SerializedValue<ExecutionConfig> serializedExecutionConfig;
/** The settings for the job checkpoints */
private JobCheckpointingSettings snapshotSettings;
/** Savepoint restore settings. */
private SavepointRestoreSettings savepointRestoreSettings = SavepointRestoreSettings.none();
// --- attached resources ---
// 用户相关的一些资源文件(userJars、分布式缓存等等))
/** Set of JAR files required to run this job. */
private final List<Path> userJars = new ArrayList<Path>();
/** Set of custom files required to run this job. */
private final Map<String, DistributedCache.DistributedCacheEntry> userArtifacts = new HashMap<>();
/** Set of blob keys identifying the JAR files required to run this job. */
private final List<PermanentBlobKey> userJarBlobKeys = new ArrayList<>();
/** List of classpaths required to run this job. */
private List<URL> classpaths = Collections.emptyList();
......
}
1、 JobGraph生成源码
与StreamGraph类似,JobGraph的入口方法是StreamingJobGraphGenerator(streamGraph, jobID).createJobGraph()。 在StreamGraph中,每一个算子(Operator)对应了图中的一个节点(StreamNode)。StreamGraph首先会对每一个节点生成对应的hash id,之后StreamGraph会被进一步优化,将多个符合条件的节点串联(Chain)在一起形成一个节点,从而减少数据在不同节点之间流动所产生的序列化、反序列化、网络传输的开销。多个算子被chain在一起的形成的节点在JobGraph中对应的就是JobVertex。每个JobVertex中包含一个或多个Operators。
针对JobGraph的生成,其StreamingJobGraphGenerator源码解析如下:
private final StreamGraph streamGraph; // 需要转换传入的streamGraph对象
private final JobGraph jobGraph; // 最终生成的jobGraph对象
// id -> JobVertex 的对应关系
private final Map<Integer, JobVertex> jobVertices;
// 已经构建的JobVertex的id集合
private final Collection<Integer> builtVertices;
// 物理边集合(不包含chain内部的边), 按创建顺序排序
private List<StreamEdge> physicalEdgesInOrder;
// 保存 operataor chain 的信息,部署时用来构建 OperatorChain,startNodeId -> (currentNodeId -> StreamConfig)
private Map<Integer, Map<Integer, StreamConfig>> chainedConfigs;
// 所有节点的配置信息,id -> StreamConfig
private Map<Integer, StreamConfig> vertexConfigs;
// 保存每个节点的名字,id -> chainedName
private Map<Integer, String> chainedNames;
// 用于计算hash值的算法
private final StreamGraphHasher defaultStreamGraphHasher;
private final List<StreamGraphHasher> legacyStreamGraphHashers;
//.....
private JobGraph createJobGraph() {
// 设置启动模式为所有节点均在一开始就启动
jobGraph.setScheduleMode(ScheduleMode.EAGER);
// 为每个节点生成hash id
// 广度优先遍历 StreamGraph 中的每个SteamNode节点,并为每个Node生成hash,hash值将被用于 JobVertexId 中
// 保证如果提交的拓扑没有改变,则每次生成的hash都是一样的
Map<Integer, byte[]> hashes = defaultStreamGraphHasher.traverseStreamGraphAndGenerateHashes(streamGraph);
List<Map<Integer, byte[]>> legacyHashes = new ArrayList<>(legacyStreamGraphHashers.size());
for (StreamGraphHasher hasher : legacyStreamGraphHashers) {
legacyHashes.add(hasher.traverseStreamGraphAndGenerateHashes(streamGraph));
}
Map<Integer, List<Tuple2<byte[], byte[]>>> chainedOperatorHashes = new HashMap<>();
// 生成jobvertex,串成chain等
// 这里的逻辑大致可以理解为,挨个遍历节点,如果该节点是一个chain的头节点,就生成一个JobVertex,
// 如果不是头节点,就要把自身配置并入头节点,然后把头节点和自己的出边相连;对于不能chain的节点,当作只有头节点处理即可
setChaining(hashes, legacyHashes, chainedOperatorHashes);
// 设置输入边edge
setPhysicalEdges();
// 设置slot共享group
setSlotSharing();
// 配置检查点
configureCheckpointing();
// 添加用户提供的自定义的文件信息
JobGraphGenerator.addUserArtifactEntries(streamGraph.getEnvironment().getCachedFiles(), jobGraph);
// 传递执行环境配置
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;
}
StreamingJobGraphGenerator#createJobGraph函数的逻辑也很清晰,首先为所有节点生成一个唯一的hash id,(此处主要使用广度优先遍历每个节点,并对连接边StreamEdge进行上下游节点的是否可链式进行判断)如果节点在多次提交中没有改变(包括并发度、上下游等),那么这个id就不会改变,这主要用于故障恢复。这里我们不能用 StreamNode.id 来代替,因为这是一个从 1 开始的静态计数变量,同样的 Job 可能会得到不一样的 id。然后就是最关键的 chaining 处理,和生成JobVetex、JobEdge等。之后就是写入各种配置相关的信息。
Flink在JobGraph转化过程中是如何确定两个Operator是否能够被chain到同一个节点的;其主要判断是否能被chain在同一个节点的代码如下StreamingJobGraphGenerator#isChainable,其将会对数据流节点的连接边(StreamEdge)进行上下游节点的(并行度、slot组、传输方式FORWARD)等条件的判断 :
// StreamEdge 两端的节点是否能够被 chain 到同一个 JobVertex 中
public static boolean isChainable(StreamEdge edge, StreamGraph streamGraph) {
// 获取到上游和下游节点
StreamNode upStreamVertex = streamGraph.getSourceVertex(edge);
StreamNode downStreamVertex = streamGraph.getTargetVertex(edge);
// 获取到上游和下游节点具体的算子 StreamOperator
StreamOperator<?> headOperator = upStreamVertex.getOperator();
StreamOperator<?> outOperator = downStreamVertex.getOperator();
return downStreamVertex.getInEdges().size() == 1 // 下游节点只有一个输入
&& outOperator != null
&& headOperator != null
&& upStreamVertex.isSameSlotSharingGroup(downStreamVertex) // 在同一个slot共享组中
// 上下游算子的 chainning 策略,要允许chainning
// 默认的是 ALWAYS
// 在添加算子时,也可以强制使用 disableChain 设置为 NEVER
&& outOperator.getChainingStrategy() == ChainingStrategy.ALWAYS
&& (headOperator.getChainingStrategy() == ChainingStrategy.HEAD ||
headOperator.getChainingStrategy() == ChainingStrategy.ALWAYS)
// 上下游节点之间的数据传输方式必须是FORWARD,而不能是REBALANCE等其它模式
&& (edge.getPartitioner() instanceof ForwardPartitioner)
// 上下游节点的并行度要一致
&& upStreamVertex.getParallelism() == downStreamVertex.getParallelism()
&& streamGraph.isChainingEnabled();
}
只要数据连接边StreamEdge两端的节点满足上面的条件,那么这两个节点就可以被串联在同一个JobVertex中。接着来就来看最为关键的函数setChaining的逻辑,其会对StreamGraph的每一个数据源Source节点进行operator chain的创建:
/**
* Sets up task chains from the source {@link StreamNode} instances.
* <p>This will recursively create all {@link JobVertex} instances.
*/
private void setChaining(Map<Integer, byte[]> hashes, List<Map<Integer, byte[]>> legacyHashes, Map<Integer, List<Tuple2<byte[], byte[]>>> chainedOperatorHashes) {
for (Integer sourceNodeId : streamGraph.getSourceIDs()) { // 对从数据源节点进行operator chain的创建
createChain(sourceNodeId, sourceNodeId, hashes, legacyHashes, 0, chainedOperatorHashes);
}
}
// 构建operator chain(可能包含一个或多个StreamNode),返回值是当前的这个operator chain实际的输出边(不包括内部的边)
// 如果currentNodeId != startNodeId, 说明当前节点在operator chain的内部
private List<StreamEdge> createChain(
Integer startNodeId,
Integer currentNodeId,
Map<Integer, byte[]> hashes,
List<Map<Integer, byte[]>> legacyHashes,
int chainIndex,
Map<Integer, List<Tuple2<byte[], byte[]>>> chainedOperatorHashes) {
if (!builtVertices.contains(startNodeId)) {
// 当前operator chain最终的输出边,不包括内部的边
List<StreamEdge> transitiveOutEdges = new ArrayList<StreamEdge>();
List<StreamEdge> chainableOutputs = new ArrayList<StreamEdge>();
List<StreamEdge> nonChainableOutputs = new ArrayList<StreamEdge>();
// 将当前节点的出边分为两组,即chainable和nonChainable
for (StreamEdge outEdge : streamGraph.getStreamNode(currentNodeId).getOutEdges()) {
if (isChainable(outEdge, streamGraph)) { // 判断当前StreamEdge的上下游是否可以串联在一起
chainableOutputs.add(outEdge);
} else {
nonChainableOutputs.add(outEdge);
}
}
// 对于chainable的输出边,递归调用,找到最终的可链在一起的输出边并加入到输出列表中
for (StreamEdge chainable : chainableOutputs) {
transitiveOutEdges.addAll(
createChain(startNodeId, chainable.getTargetId(), hashes, legacyHashes, chainIndex + 1, chainedOperatorHashes));
}
// 对于nonChainable的边
for (StreamEdge nonChainable : nonChainableOutputs) {
// 这个边本身就应该加入到当前节点的输出列表中
transitiveOutEdges.add(nonChainable);
// 递归调用,以下游节点为起始点创建新的operator chain
createChain(nonChainable.getTargetId(), nonChainable.getTargetId(), hashes, legacyHashes, 0, chainedOperatorHashes);
}
// 用于保存一个operator chain所有operator的hash信息
List<Tuple2<byte[], byte[]>> operatorHashes =
chainedOperatorHashes.computeIfAbsent(startNodeId, k -> new ArrayList<>());
byte[] primaryHashBytes = hashes.get(currentNodeId);
for (Map<Integer, byte[]> legacyHash : legacyHashes) {
operatorHashes.add(new Tuple2<>(primaryHashBytes, legacyHash.get(currentNodeId)));
}
// 当前节点的名称,资源要求等信息
chainedNames.put(currentNodeId, createChainedName(currentNodeId, chainableOutputs));
chainedMinResources.put(currentNodeId, createChainedMinResources(currentNodeId, chainableOutputs));
chainedPreferredResources.put(currentNodeId, createChainedPreferredResources(currentNodeId, chainableOutputs));
// 如果当前节点是起始节点, 则直接创建 JobVertex 并返回 StreamConfig, 否则先创建一个空的 StreamConfig
// createJobVertex 函数就是根据 StreamNode 创建对应的 JobVertex, 并返回了空的 StreamConfig
StreamConfig config = currentNodeId.equals(startNodeId)
? createJobVertex(startNodeId, hashes, legacyHashes, chainedOperatorHashes)
: new StreamConfig(new Configuration());
// 设置 JobVertex 的 StreamConfig, 基本上是序列化 StreamNode 中的配置到 StreamConfig 中.
// 其中包括 序列化器, StreamOperator, Checkpoint 等相关配置
setVertexConfig(currentNodeId, config, chainableOutputs, nonChainableOutputs);
if (currentNodeId.equals(startNodeId)) {
// 如果是chain的起始节点。(不是chain中的节点,也会被标记成 chain start)
// 最开始的chainable的source节点 和 nonChainable的起始节点
config.setChainStart();
config.setChainIndex(0);
config.setOperatorName(streamGraph.getStreamNode(currentNodeId).getOperatorName());
//把实际的输出边写入配置, 部署时会用到
config.setOutEdgesInOrder(transitiveOutEdges);
//operator chain 的头部 operator 的输出边,包括内部的边
config.setOutEdges(streamGraph.getStreamNode(currentNodeId).getOutEdges());
// 将当前节点(headOfChain)与所有出边相连
for (StreamEdge edge : transitiveOutEdges) {
// 通过StreamEdge构建出JobEdge,创建IntermediateDataSet,用来将JobVertex和JobEdge相连
connect(startNodeId, edge);
}
// 将operator chain中所有子节点的 StreamConfig 写入到 headOfChain 节点的 CHAINED_TASK_CONFIG 配置中
config.setTransitiveChainedTaskConfigs(chainedConfigs.get(startNodeId));
} else {
//如果是 operator chain 内部的节点
Map<Integer, StreamConfig> chainedConfs = chainedConfigs.get(startNodeId);
if (chainedConfs == null) {
chainedConfigs.put(startNodeId, new HashMap<Integer, StreamConfig>());
}
config.setChainIndex(chainIndex);
StreamNode node = streamGraph.getStreamNode(currentNodeId);
config.setOperatorName(node.getOperatorName());
// 将当前节点的 StreamConfig 添加所在的 operator chain 的 config 集合中
chainedConfigs.get(startNodeId).put(currentNodeId, config);
}
//设置当前 operator 的 OperatorID
config.setOperatorID(new OperatorID(primaryHashBytes));
if (chainableOutputs.isEmpty()) {
config.setChainEnd();
}
return transitiveOutEdges;
} else {
return new ArrayList<>();
}
}
针对上一节中的StreamGraph进行JobGraph遍历生成分析;函数createChain(startNodeId, currentNodeId)会记录能够被chain在一起的 起点--终点 的链路:
- 首先从SourceNode#1出发createChain(1,1),遍历判断其outEdges列表中的下游节点是否可以和SourceNode节点Chainable,其下游节点MapNode#2的slot组、并行度、分区Forward等符合chain条件,将会把SourceNode#1和MapNode#2链接在一起;形成chain(SourceNode#1 --> MapNode#2);之后将会继续递归调用createChain(1,2)判断MapNode#2的下游节点WindowNode#4是否能够加入到当前的chain中;
- 针对MapNode#2的createChain(1,2)判断,遍历判断其outEdges列表中的下游节点WindowNode#4的分区方式不是Forward方式,所以WindowNode#4将不会和当前chain(SourceNode#1 --> MapNode#2)在一起;所以将会以WindowNode#4为起始节点,再开始一条新的chain链路(WindowNode#4 --> ...);
- 针对WindowNode#4的createChain(4,4)判断;遍历判断其outEdges列表中的下游节点FilterNode#5是可以被chain在一起的,所以会形成(WindowNode#4 --> FilterNode#5)这样的一个chain;
- 针对FilterNode#5的createChain(4,5)判断;遍历判断其outEdges列表中的下游节点SinkNode#6是可以被chain在一起的;最终形成chain(WindowNode#4 --> FilterNode#5 --> SinkNode#6);
所以最终会形成chain#1(SourceNode#1 --> MapNode#2),chain#2(WindowNode#4 --> FilterNode#5 --> SinkNode#6)的这样的链路;
在递归调用过程中,会对每次递归调用栈进行startNodeId到currentNodeId这样的一个链路记录,1、并对上诉每一个步骤所涉及到的SteamNode节点(非chain的起始节点)生成对应节点的StreamConfig配置(主要是StreamOperator的配置,包括ckp配置、序列化配置、StreamOperator所涉及的userFunction配置、输出配置等);2、所涉及的SteamNode节点(chain的起始节点)生成对应的JobVertex可执行对象(主要包括资源、并行度、config、可执行invokableClassName、可chain在一起的operator id等),其JobVertex内的运行时Config对象会标识其是chain的起始点,并将当前起始节点的outEdges输出边序列化写入运行时Config对象的outStreamEdges中;
一个operator chain的所有内部节点都会以序列化的形式写入头部起始节点的CHAINED_TASK_CONFIG配置项中。
针对StreamNode节点(chain的起始节点)生成对应的链路JobVertex,其将会把每个链路所包含的操作的节点的hash id、输入输出等都会存储在JobGraph的Map<JobVertexID, JobVertex> taskVertices中,其中JobVertex的主要成员变量包括:
public class JobVertex implements java.io.Serializable {
/** The ID of the vertex. */
private final JobVertexID id; // 唯一性id
/** The alternative IDs of the vertex. */
private final ArrayList<JobVertexID> idAlternatives = new ArrayList<>();
/** The IDs of all operators contained in this vertex. */
private final ArrayList<OperatorID> operatorIDs = new ArrayList<>(); // 保存所有chain在一起的操作StreamOperator id
/** The alternative IDs of all operators contained in this vertex. */
private final ArrayList<OperatorID> operatorIdsAlternatives = new ArrayList<>();
/** List of produced data sets, one per writer */
private final ArrayList<IntermediateDataSet> results = new ArrayList<IntermediateDataSet>(); // 输出数据集
/** List of edges with incoming data. One per Reader. */
private final ArrayList<JobEdge> inputs = new ArrayList<JobEdge>(); // 输入JobEdge列表
/** Custom configuration passed to the assigned task at runtime. */
private Configuration configuration; // 运行时config
/** The class of the invokable. */
private String invokableClassName; // 可执行类名
......
}
其中最主要的是 输入JobEdge列表,输出IntermediateDataSet列表。
public class JobEdge implements java.io.Serializable {
/** The vertex connected to this edge. */
private final JobVertex target; // 输出的JobVertex
/** The distribution pattern that should be used for this job edge. */
private final DistributionPattern distributionPattern;
/** The data set at the source of the edge, may be null if the edge is not yet connected*/
private IntermediateDataSet source; // 输入的dataset
}
public class IntermediateDataSet implements java.io.Serializable
private final IntermediateDataSetID id; // the identifier
private final JobVertex producer; // the operation that produced this data set (生产者operator chain)
private final List<JobEdge> consumers = new ArrayList<JobEdge>(); // (消费者JobEdge)
// The type of partition to use at runtime
private final ResultPartitionType resultType;
}
针对多个StreamNode节点(chain的起始节点)生成对应的链路JobVertex;其会进行chain链路的connect连接,会为所有的实际输出边创建对应的JobEdge,并和JobVertex 连接;其主要方法为connect(startNodeId, edge);
private void connect(Integer headOfChain, StreamEdge edge) { // 头结点 与其对应的 连接边
physicalEdgesInOrder.add(edge);
Integer downStreamvertexID = edge.getTargetId();
//上下游节点
JobVertex headVertex = jobVertices.get(headOfChain);
JobVertex downStreamVertex = jobVertices.get(downStreamvertexID);
StreamConfig downStreamConfig = new StreamConfig(downStreamVertex.getConfiguration());
//下游节点增加一个输入
downStreamConfig.setNumberOfInputs(downStreamConfig.getNumberOfInputs() + 1);
StreamPartitioner<?> partitioner = edge.getPartitioner();
JobEdge jobEdge;
//创建 JobEdge 和 IntermediateDataSet
//根据StreamPartitioner类型决定在上游节点(生产者)的子任务和下游节点(消费者)之间的连接模式
if (partitioner instanceof ForwardPartitioner || partitioner instanceof RescalePartitioner) {
jobEdge = downStreamVertex.connectNewDataSetAsInput(
headVertex,
DistributionPattern.POINTWISE,
ResultPartitionType.PIPELINED_BOUNDED);
} else {
jobEdge = downStreamVertex.connectNewDataSetAsInput( // 下游节点将上游作为DataSet进行连接
headVertex,
DistributionPattern.ALL_TO_ALL,
ResultPartitionType.PIPELINED_BOUNDED);
}
// 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); // 创建dataset
JobEdge edge = new JobEdge(dataSet, this, distPattern);
this.inputs.add(edge);
dataSet.addConsumer(edge);
return edge;
}
针对上述生成的两个operator chain边,chain#1(SourceNode#1 --> MapNode#2),chain#2(WindowNode#4 --> FilterNode#5 --> SinkNode#6);其会调用connect(StreamNode#1, StreamEdge#(2 -> 4)),在上下游operator chain之间生成JobEdge链接边来实际连接JobVertex chain#1和JobVertex chain#2;
- 首先会获取上下游operator chain所对应的JobVertex对象,其次会从StreamEdge中获取到StreamPartitioner分区器,之后会根据分区器类型生成对应的上游节点(生产者)的子任务和下游节点(消费者)之间的连接模式;
- 之后根据downStreamVertex.connectNewDataSetAsInput生成对应的IntermediateDataSet数据集,其里面主要包含当前生产者JobVertex,和多个消费者List<JobEdge>;
- 之后便继续根据该dataSet生成完整的JobEdge 连接边;
JobGraph的关键在于将多个StreamNode优化为一个JobVertex, 对应的StreamEdge则转化为JobEdge, 并且JobVertex和JobEdge之间通过IntermediateDataSet形成一个生产者和消费者的连接关系。最终形成的链路如下:
- JobVertex chain#1(SourceNode#1 --> MapNode#2,inputs=null,results=dataset#1(producer=this(JobVertex#1), consumers=JobEdge#1));
- JobEdge#1(source=dataset#1, target=JobVertex chain#2);
- JobVertex chain#2(WindowNode#4 --> FilterNode#5 --> SinkNode#6,inputs=JobEdge#1,results=null(print));