前面的文章我们介绍了 StreamGraph 的生成,这个实际上只对应 Flink 作业在逻辑上的执行计划图。Flink 会进一步对 StreamGraph 进行转换,得到另一个执行计划图,即 JobGraph。

JobVertex

在 StreamGraph 中,每一个算子(Operator) 对应了图中的一个节点(StreamNode)。StreamGraph 会被进一步优化,将多个符合条件的节点串联(Chain) 在一起形成一个节点,从而减少数据在不同节点之间流动所产生的序列化、反序列化、网络传输的开销。多个算子被 chain 在一起的形成的节点在 JobGraph 中对应的就是 JobVertex。

每个 JobVertex 中包含一个或多个 Operators。 JobVertex 的主要成员变量包括

    /** The ID of the vertex. */
    private final JobVertexID 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<>();

    /** 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>();

    /** Number of subtasks to split this task into at runtime.*/
    private int parallelism = ExecutionConfig.PARALLELISM_DEFAULT;

其输入是 JobEdge 列表, 输出是 IntermediateDataSet 列表。

JobEdge

在 StramGraph 中,StreamNode 之间是通过 StreamEdge 建立连接的。在 JobEdge 中,对应的是 JobEdge 。
和 StreamEdge 中同时保留了源节点和目标节点 (sourceId 和 targetId)不同,在 JobEdge 中只有源节点的信息。由于 JobVertex 中保存了所有输入的 JobEdge 的信息,因而同样可以在两个节点之间建立连接。更确切地说,JobEdge 是和节点的输出结果相关联的,我们看下 JobEdge 主要的成员变量:

    /** The vertex connected to this edge. */
    private final JobVertex target;

    /** The distribution pattern that should be used for this job edge. */
  // DistributionPattern 决定了在上游节点(生产者)的子任务和下游节点(消费者)之间的连接模式
    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;

    /** The id of the source intermediate data set */
    private IntermediateDataSetID sourceId;

    /** Optional name for the data shipping strategy (forward, partition hash, rebalance, ...),
     * to be displayed in the JSON plan */
    private String shipStrategyName;

IntermediateDataSet

JobVertex 产生的数据被抽象为 IntermediateDataSet, 字面意思为中间数据集,这个很容易理解。前面提到,JobEdge 是和节点的输出结果相关联的,其实就是指可以把 JobEdge 看作是 IntermediateDataSet 的消费者,那么 JobVertex 自然就是生产者了。

private final IntermediateDataSetID id;         // the identifier

    private final JobVertex producer;            // the operation that produced this data set

    private final List<JobEdge> consumers = new ArrayList<JobEdge>();

    // The type of partition to use at runtime
    private final ResultPartitionType resultType;

其中 ResultPartitionType 表示中间结果的类型,说起来有点抽象,我们看下属性就明白了:

    /** Can the partition be consumed while being produced? */
    private final boolean isPipelined;

    /** Does the partition produce back pressure when not consumed? */
    private final boolean hasBackPressure;

    /** Does this partition use a limited number of (network) buffers? */
    private final boolean isBounded;

这个要结合 Flink 任务运行时的内存管理机制来看,在后面的文章再进行分析。目前在 Stream 模式下使用的类型是 PIPELINED_BOUNDED(true, true, true),上面的三个属性都是 true。

StreamConfig

对于每一个 StreamOperator, 也就是 StreamGraph 中的每一个 StreamGraph, 在生成 JobGraph 的过程中 StreamingJobGraphGenerator 都会创建一个对应的 StreamConfig。

StreamConfig 中保存了这个算子(operator) 在运行是需要的所有配置信息,这些信息都是通过 key/value 的形式存储在 Configuration 中的。例如:

    //保存StreamOperator信息
    public void setStreamOperator(StreamOperator<?> operator) {
        if (operator != null) {
            config.setClass(USER_FUNCTION, operator.getClass());

            try {
                InstantiationUtil.writeObjectToConfig(operator, this.config, SERIALIZEDUDF);
            } catch (IOException e) {
                throw new StreamTaskException("Cannot serialize operator object "
                        + operator.getClass() + ".", e);
            }
        }
    }


  public void setChainedOutputs(List<StreamEdge> chainedOutputs) {
        try {
            InstantiationUtil.writeObjectToConfig(chainedOutputs, this.config, CHAINED_OUTPUTS);
        } catch (IOException e) {
            throw new StreamTaskException("Cannot serialize chained outputs.", e);
        }
    }

  public void setNonChainedOutputs(List<StreamEdge> outputvertexIDs) {
        try {
            InstantiationUtil.writeObjectToConfig(outputvertexIDs, this.config, NONCHAINED_OUTPUTS);
        } catch (IOException e) {
            throw new StreamTaskException("Cannot serialize non chained outputs.", e);
        }
    }

  public void setInPhysicalEdges(List<StreamEdge> inEdges) {
        try {
            InstantiationUtil.writeObjectToConfig(inEdges, this.config, IN_STREAM_EDGES);
        } catch (IOException e) {
            throw new StreamTaskException("Cannot serialize inward edges.", e);
        }
    }

  //......

从 StreamGraph 到 JobGraph
从 StreamGraph 到 JobGraph 的转换入口在 StreamingJobGraphGenerator 中。

首先来看下 StreamingJobGraphGenerator 的成员变量和入口函数:

    //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);

        // 广度优先遍历 StreamGraph 并且为每个SteamNode生成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<>();

        // 主要的转换逻辑,生成 JobVetex, JobEdge 等
        setChaining(hashes, legacyHashes, chainedOperatorHashes);

        // 将每个JobVertex的输入边集合也序列化到该JobVertex的StreamConfig中
    // (出边集合已经在setChaining的时候写入了)
        setPhysicalEdges();

        // 根据group name,为每个 JobVertex 指定所属的 SlotSharingGroup
        // 以及针对 Iteration的头尾设置  CoLocationGroup
        setSlotSharingAndCoLocation();

        // 配置 checkpoint
        configureCheckpointing();

        // 添加用户提供的自定义的文件信息
        JobGraphGenerator.addUserArtifactEntries(streamGraph.getEnvironment().getCachedFiles(), jobGraph);

        // 将 StreamGraph 的 ExecutionConfig 序列化到 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,如果节点在多次提交中没有改变(包括并发度、上下游等),那么这个id就不会改变,这主要用于故障恢复。这里我们不能用 StreamNode.id 来代替,因为这是一个从 1 开始的静态计数变量,同样的 Job 可能会得到不一样的 id。然后就是最关键的 chaining 处理,和生成JobVetex、JobEdge等。之后就是写入各种配置相关的信息。

我们先来看一下,Flink 是如何确定两个 Operator 是否能够被 chain 到同一个节点的:

//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();
    }

只要一条边两端的节点满足上面的条件,那么这两个节点就可以被串联在同一个 JobVertex 中。接着来就来看最为关键的函数 setChaining 的逻辑:

/**
     * 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()) {
            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)
                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<>();
        }
    }

上述过程实际上就是通过 DFS 遍历所有的 StreamNode, 并按照 chainable 的条件不停地将可以串联的呃 operator 放在同一个的 operator chain 中。每一个 StreamNode 的配置信息都会被序列化到对应的 StreamConfig 中。只有 operator chain 的头部节点会生成对应的 JobVertex ,一个 operator chain 的所有内部节点都会以序列化的形式写入头部节点的 CHAINED_TASK_CONFIG 配置项中。

每一个 operator chain 都会为所有的实际输出边创建对应的 JobEdge,并和 JobVertex 连接:

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(
                    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);
        }
    }

小结
本文分析了从 StreamGraph 到 JobGraph 之间的转换过程。 JobGraph 的关键在于将多个 StreamNode 优化为一个 JobVertex, 对应的 StreamEdge 则转化为 JobEdge, 并且 JobVertex 和 JobEdge 之间通过 IntermediateDataSet 形成一个生产者和消费者的连接关系。

转发自: https://blog.jrwang.me/2019/flink-source-code-jobgraph/