本文首發於泊浮目的語雀:https://www.yuque.com/17sing
版本 | 日期 | 備註 |
---|---|---|
1.0 | 2022.1.26 | 文章首發 |
0.前言
前陣子組裡的小夥伴問我“為什麼Flink從我們的程式碼到真正可執行的狀態,要經過這麼多個graph轉換?這樣做有什麼好處嘛?”我早期看到這裡的設計時的確有過相同的疑惑,當時由於手裡還在看別的東西,查閱過一些資料後就翻頁了。如今又碰到了這樣的問題,不妨就在這篇文章中好好搞清楚。
本文的原始碼基於Flink1.14.0
。
1. 分層設計
該圖來自Jark大佬的部落格:http://wuchong.me/blog/2016/0...
以上是Flink的Graph層次圖,在接下來的內容我們會逐一揭開它們的面紗,得知它們存在的意義。
1.1 BatchAPI的OptimizedPlan
在這個小節中,我們會看到DataSet從Plan轉換到OptimizedPlan的過程中。為了方便讀者有個概念,我們在這裡解釋一下幾個名詞:
- DataSet:面向使用者的批處理API。
- Plan:描述DataSource以及DataSink以及Operation如何互動的計劃。
- OptimizedPlan:優化過的執行計劃。
程式碼入口:
|--ClientFrontend#main
\-- parseAndRun
\-- runApplication
\-- getPackagedProgram
\-- buildProgram
\-- executeProgram
|-- ClientUtils#executeProgram
|-- PackagedProgram#invokeInteractiveModeForExecution
\-- callMainMethod //呼叫使用者編寫的程式入口
|-- ExecutionEnvironment#execute
\-- executeAsync // 建立Plan
|-- PipelineExecutorFactory#execute
|-- EmbeddedExecutor#execute
\-- submitAndGetJobClientFuture
|-- PipelineExecutorUtils#getJobGraph
|-- FlinkPipelineTranslationUtil#getJobGraph
|-- FlinkPipelineTranslator#translateToJobGraph //如果傳入的是Plan,則會在內部實現中先轉換出OptimizedPlan,再轉換到JobGraph;如果是StreamGraph,則會直接轉換出JobGraph
|-- PlanTranslator#translateToJobGraph
\-- compilePlan
我們看一下這段程式碼:
private JobGraph compilePlan(Plan plan, Configuration optimizerConfiguration) {
Optimizer optimizer = new Optimizer(new DataStatistics(), optimizerConfiguration);
OptimizedPlan optimizedPlan = optimizer.compile(plan);
JobGraphGenerator jobGraphGenerator = new JobGraphGenerator(optimizerConfiguration);
return jobGraphGenerator.compileJobGraph(optimizedPlan, plan.getJobId());
}
非常的清晰。就是從OptimizedPlan
到JobGraph
。OptimizedPlan的轉換過程我們看Optimizer#compile
方法。先看方法簽名上的註釋:
/**
* Translates the given program to an OptimizedPlan. The optimized plan describes for each
* operator which strategy to use (such as hash join versus sort-merge join), what data exchange
* method to use (local pipe forward, shuffle, broadcast), what exchange mode to use (pipelined,
* batch), where to cache intermediate results, etc,
*
* <p>The optimization happens in multiple phases:
*
* <ol>
* <li>Create optimizer dag implementation of the program.
* <p><tt>OptimizerNode</tt> representations of the PACTs, assign parallelism and compute
* size estimates.
* <li>Compute interesting properties and auxiliary structures.
* <li>Enumerate plan alternatives. This cannot be done in the same step as the interesting
* property computation (as opposed to the Database approaches), because we support plans
* that are not trees.
* </ol>
*
* @param program The program to be translated.
* @param postPasser The function to be used for post passing the optimizer's plan and setting
* the data type specific serialization routines.
* @return The optimized plan.
* @throws CompilerException Thrown, if the plan is invalid or the optimizer encountered an
* inconsistent situation during the compilation process.
*/
private OptimizedPlan compile(Plan program, OptimizerPostPass postPasser)
這裡提到了會有好幾步來做優化:
- 建立優化過的DAG,為其生成的OptimizerNode遵循PACT模型,併為其分配了併發度以及計算資源。
- 生成一些重要的屬性以及輔助性資料結構。
- 列舉所有的代替方案。
在方法的實現中,會建立大量的Visitor來對程式做遍歷優化。
1.1.1GraphCreatingVisitor
首先是建立GraphCreatingVisitor,對原始的Plan進行優化,將每個operator優化成OptimizerNode,OptimizerNode之間通過DagConnection相連,DagConnection相當於一個邊模型,有source和target,可以表示OptimizerNode的輸入和輸出。在這個過程中會做這些事:
- 為每個運算元建立一個OptimizerNode——更加接近執行描述的Node(估算出資料的大小、data flow在哪裡進行拆分和合並等)
- 用Channel將它們連線起來
- 根據建議生成相應的策略:Operator用什麼策略執行:比如Hash Join or Sort Merge Join;Operator間的資料交換策略,是Local Pipe Forward、Shuffle,還是Broadcast;Operator間的資料交換模式,是Pipelined還是Batch。
1.1.2 IdAndEstimatesVisitor
顧名思義,為每個運算元生成id,並估算其資料量。估算的實現見OptimizerNode#computeOutputEstimates
——這是一個抽象函式,我們可以關注一下DataSourceNode裡的實現,它會根據上游資料來源的一系列屬性(比如行數、大小)得出估算值。**但這段程式碼放在這裡並不合適
,作者的原意似乎是關注file型別的上游,註釋這麼說道:see, if we have a statistics object that can tell us a bit about the file
**。
1.1.3 UnionParallelismAndForwardEnforcer
這裡會保證UnionNode的併發度與下游對其,避免資料分佈有誤而導致資料不精準(見https://github.com/apache/fli...)。
1.1.4 BranchesVisitor
計算不會閉合的下游子DAG圖。見其定義:
/**
* Description of an unclosed branch. An unclosed branch is when the data flow branched (one
* operator's result is consumed by multiple targets), but these different branches (targets)
* have not been joined together.
*/
public static final class UnclosedBranchDescriptor {
1.1.5 InterestingPropertyVisitor
根據Node的屬性估算成本。
估算演算法見:node.computeInterestingPropertiesForInputs
- WorksetIterationNode
- TwoInputNode
- SingleInputNode
- BulkIterationNode
之後便會根據成本算出一系列的執行計劃:
// the final step is now to generate the actual plan alternatives
List<PlanNode> bestPlan = rootNode.getAlternativePlans(this.costEstimator);
在這裡,OptimizerNode優化成了PlanNode,PlanNode是最終的優化節點型別,它包含了節點的更多屬性,節點之間通過Channel進行連線,Channel也是一種邊模型,同時確定了節點之間的資料交換方式ShipStrategyType和DataExchangeMode,ShipStrategyType表示的兩個節點之間資料的傳輸策略,比如是否進行資料分割槽,進行hash分割槽,範圍分割槽等; DataExchangeMode表示的是兩個節點間資料交換的模式,有PIPELINED和BATCH,和ExecutionMode是一樣的,ExecutionMode決定了DataExchangeMode——直接發下去還是先落盤。
1.1.6 PlanFinalizer.createFinalPlan
PlanFinalizer.createFinalPlan()。其大致的實現就是將節點新增到sources、sinks、allNodes中,還可能會為每個節點設定任務佔用的記憶體等。
1.1.7 BinaryUnionReplacer
顧名思義,針對上游同樣是Union的操作做去重替換,合併到一起。筆者認為,這在輸出等價的情況下,減少了Node的生成。
1.1.8 RangePartitionRewriter
在使用範圍分割槽這一特性時,需要儘可能保證各分割槽所處理的資料集均衡性以最大化利用計算資源並減少作業的執行時間。為此,優化器提供了範圍分割槽重寫器(RangePartitionRewriter)來對範圍分割槽的分割槽策略進行優化,使其儘可能平均地分配資料,避免資料傾斜。
如果要儘可能的平均分配資料,肯定要對資料來源進行估算。但顯然是沒法讀取所有的資料進行估算的,這裡Flink採用了ReservoirSampling
演算法的改良版——可以參考論文Optimal Random Sampling from Distributed Streams Revisited
,在程式碼中由org.apache.flink.api.java.sampling.ReservoirSamplerWithReplacement
和org.apache.flink.api.java.sampling.ReservoirSamplerWithoutReplacement
實現。
值得一提的是,無論是Plan
還是OptimizerNode
都實現了Visitable
介面,這是典型的策略模式使用,這讓程式碼變得非常靈活,正如註釋所說的——遍歷方式是可以自由編寫的。
package org.apache.flink.util;
import org.apache.flink.annotation.Internal;
/**
* This interface marks types as visitable during a traversal. The central method <i>accept(...)</i>
* contains the logic about how to invoke the supplied {@link Visitor} on the visitable object, and
* how to traverse further.
*
* <p>This concept makes it easy to implement for example a depth-first traversal of a tree or DAG
* with different types of logic during the traversal. The <i>accept(...)</i> method calls the
* visitor and then send the visitor to its children (or predecessors). Using different types of
* visitors, different operations can be performed during the traversal, while writing the actual
* traversal code only once.
*
* @see Visitor
*/
@Internal
public interface Visitable<T extends Visitable<T>> {
/**
* Contains the logic to invoke the visitor and continue the traversal. Typically invokes the
* pre-visit method of the visitor, then sends the visitor to the children (or predecessors) and
* then invokes the post-visit method.
*
* <p>A typical code example is the following:
*
* <pre>{@code
* public void accept(Visitor<Operator> visitor) {
* boolean descend = visitor.preVisit(this);
* if (descend) {
* if (this.input != null) {
* this.input.accept(visitor);
* }
* visitor.postVisit(this);
* }
* }
* }</pre>
*
* @param visitor The visitor to be called with this object as the parameter.
* @see Visitor#preVisit(Visitable)
* @see Visitor#postVisit(Visitable)
*/
void accept(Visitor<T> visitor);
}
1.2 StreamAPI的StreamGraph
構造StreamGraph的入口函式是 StreamGraphGenerator.generate()
。該函式會由觸發程式執行的方法StreamExecutionEnvironment.execute()
呼叫到。就像OptimizedPla,StreamGraph 也是在 Client 端構造的。
在這個過程中,流水線首先被轉換為Transformation流水線,然後被對映為StreamGraph,該圖與具體的執行無關,核心是表達計算過程的邏輯。
關於Transformation
的引入,可以看社群的issue:https://issues.apache.org/jir...。本質是為了避免DataStream這一層對StreamGraph的耦合,因此引入這一層做解耦。
Transformation
關注的屬性偏向框架內部,如:name(運算元名)、uid(job重啟時分配之前相同的狀態,持久儲存狀態)、bufferTimeout、parallelism、outputType、soltSharingGroup等。另外,Transformation分為物理Transformation和虛擬Transformation,這於下一層的StreamGraph實現是有關聯的。
StreamGraph的核心物件有兩個:
- StreamNode:它可以有多個輸出,也可以有多個輸入。由Transformation轉換而來——實體的StreamNode會最終變成物運算元,虛擬的StreamNode會附著在StreamEdge上。
- StreamEdge:StreamGraph的邊,用於連線兩個StreamNode。就像上面說的——一個StreamNode可以有多個出邊、入邊。StreamEdge中包含了旁路輸出、分割槽器、欄位篩選輸出(與SQL Select中選擇欄位的邏輯一樣)等的資訊。
具體的轉換程式碼在org.apache.flink.streaming.api.graph.StreamGraphGenerator
中,每個Transformation都有對應的轉換邏輯:
static {
@SuppressWarnings("rawtypes")
Map<Class<? extends Transformation>, TransformationTranslator<?, ? extends Transformation>>
tmp = new HashMap<>();
tmp.put(OneInputTransformation.class, new OneInputTransformationTranslator<>());
tmp.put(TwoInputTransformation.class, new TwoInputTransformationTranslator<>());
tmp.put(MultipleInputTransformation.class, new MultiInputTransformationTranslator<>());
tmp.put(KeyedMultipleInputTransformation.class, new MultiInputTransformationTranslator<>());
tmp.put(SourceTransformation.class, new SourceTransformationTranslator<>());
tmp.put(SinkTransformation.class, new SinkTransformationTranslator<>());
tmp.put(LegacySinkTransformation.class, new LegacySinkTransformationTranslator<>());
tmp.put(LegacySourceTransformation.class, new LegacySourceTransformationTranslator<>());
tmp.put(UnionTransformation.class, new UnionTransformationTranslator<>());
tmp.put(PartitionTransformation.class, new PartitionTransformationTranslator<>());
tmp.put(SideOutputTransformation.class, new SideOutputTransformationTranslator<>());
tmp.put(ReduceTransformation.class, new ReduceTransformationTranslator<>());
tmp.put(
TimestampsAndWatermarksTransformation.class,
new TimestampsAndWatermarksTransformationTranslator<>());
tmp.put(BroadcastStateTransformation.class, new BroadcastStateTransformationTranslator<>());
tmp.put(
KeyedBroadcastStateTransformation.class,
new KeyedBroadcastStateTransformationTranslator<>());
translatorMap = Collections.unmodifiableMap(tmp);
}
1.3 流批一體的JobGraph
程式碼入口和1.1小節幾乎一摸一樣,DataSet的入口類是ExecutionEnvironment
,而DataStream的入口是StreamExecutionEnvironment
。PlanTranslator
變成了StreamGraphTranslator
。所以,StreamGraph到JobGraph的轉化也是在Client端進行的,主要工作做優化。其中非常重要的一個優化就是Operator Chain,它會將條件允許的運算元合併到一起,避免跨執行緒、跨網路的傳遞。
是否開啟OperationChain可以在程式中顯示的調整。
接下來,我們來看下JobGraph到底是什麼。先看註釋:
/**
* The JobGraph represents a Flink dataflow program, at the low level that the JobManager accepts.
* All programs from higher level APIs are transformed into JobGraphs.
*
* <p>The JobGraph is a graph of vertices and intermediate results that are connected together to
* form a DAG. Note that iterations (feedback edges) are currently not encoded inside the JobGraph
* but inside certain special vertices that establish the feedback channel amongst themselves.
*
* <p>The JobGraph defines the job-wide configuration settings, while each vertex and intermediate
* result define the characteristics of the concrete operation and intermediate data.
*/
public class JobGraph implements Serializable {
它是一張圖,由vertices
和intermediate
組成。並且它是一個低等級的API,為JobMaster而生——所有高等級的API都會被轉換成JobGraph。接下來我們需要關注的物件分別是JobVertex
、JobEdge
、IntermediateDataSet
。其中,JobVertex的輸入是JobEdge,輸出是IntermediateDataSet。
1.3.1 JoBVertex
經過符合條件的多個StreamNode經過優化後的可能會融合在一起生成一個JobVertex,即一個JobVertex包含一個或多個運算元(有興趣的同學可以看StreamingJobGraphGenerator#buildChainedInputsAndGetHeadInputs
或者閱讀相關的Issue:https://issues.apache.org/jir...)。
1.3.2 JobEdge
JobEdge是連線IntermediateDatSet和JobVertex的邊,代表著JobGraph中的一個資料流轉通道,其上游是IntermediateDataSet,下游是JobVertex——資料通過JobEdge由IntermediateDataSet傳遞給目標JobVertex。
在這裡,我們要關注它的一個成員變數:
/**
* A distribution pattern determines, which sub tasks of a producing task are connected to which
* consuming sub tasks.
*
* <p>It affects how {@link ExecutionVertex} and {@link IntermediateResultPartition} are connected
* in {@link EdgeManagerBuildUtil}
*/
public enum DistributionPattern {
/** Each producing sub task is connected to each sub task of the consuming task. */
ALL_TO_ALL,
/** Each producing sub task is connected to one or more subtask(s) of the consuming task. */
POINTWISE
}
該分發模式會直接影響執行時Task之間的資料連線關係:點對點連線or全連線(或者叫廣播)。
1.3.3 IntermediateDataSet
中間資料集IntermediateDataSet是一種邏輯結構,用來表示JobVertex的輸出,即該JobVertex中包含的運算元會產生的資料集。在這裡我們需要關注ResultPartitionType:
- Blocking:顧名思義。都上游處理完資料後,再交給下游處理。這個資料分割槽可以被消費多次,也可以併發消費。這個分割槽並不會被自動銷燬,而是交給排程器判斷。
- BlokingPersistent:類似於Blocking,但是其生命週期由使用者端指定。呼叫JobMaster或者ResourceManager的API來銷燬,而不是由排程器控制。
- Pipelined:流交換模式。可以用於有界和無界流。這種分割槽型別的資料只能被每個消費者消費一次。且這種分割槽可以保留任意資料。
- PipelinedBounded:與Pipelined有些不同,這種分割槽保留的資料是有限的,這不會使資料和檢查點延遲太久。因此適用於流計算場景(需注意,批處理模式下沒有CheckpointBarrier)。
- Pipelined_Approximate:1.12引入的策略,用於針對單個task做fast failover的分割槽策略。有興趣的同學可以閱讀相關issue:https://issues.apache.org/jir...。
不同的執行模式下,其對應的結果分割槽型別不同,決定了在執行時刻資料交換的模式。
IntermediateDataSet的個數與該JobVertext對應的StreamNode的出邊數量相同,可以是一個或者多個。
1.4 ExecutionGraph
JobManager接收到Client端提交的JobGraph及其依賴Jar之後就要開始排程執行該任務了,但JobGraph還是一個邏輯上的圖,需要再進一步轉化為並行化、可排程的執行圖。這個動作是JobMaster做的——通過SchedulerBase觸發,實際動作交給DefaultExecutionGraphBuilder#buildGraph
來做。在這些動作中,會生成與JobVertex對應的ExecutionJobVertex(邏輯概念)和ExecutionVertex,與IntermediateDataSet對應的IntermediateResult(邏輯概念)和IntermediateResultPartition等,所謂的並行度也將通過上述類實現。
接下來要聊聊ExecutionGraph的一些細節,會涉及一些邏輯概念,因此在這裡筆者畫了一張圖,便於參考。
1.4.1 ExecutionJobVertex與ExecutionVertex
ExecutionJobVertex和JobGraph中的JobVertex一一對應。該物件還包含一組ExecutionVertex,數量與該JobVertex中所包含的StreamNode的並行度一致,如上圖所示,如果並行度為N,那麼就會有N個ExecutionVertex。所以每一個並行執行的例項就是ExecutionVertex。同時也會構建ExecutionVertex的輸出IntermediateResult。
因此ExecutionJobVertex更像是一個邏輯概念。
1.4.2 IntermediaResult與IntermediaResuktParitition
IntermediateResult表示ExecutionJobVertex的輸出,和JobGraph中的IntermediateDataSet一一對應,該物件也是一個邏輯概念。同理,一個ExecutionJobVertex可以有多箇中間結果,取決於當前JobVertex有幾個出邊(JobEdge)。
一箇中間結果集包含多箇中間結果分割槽IntermediateResultPartition,其個數等於該Job Vertext的併發度,或者叫作運算元的並行度。每個IntermediateResultPartition表示1個ExecutionVertex輸出結果。
1.4.3 Execution
ExecutionVertex在Runtime對應了一個Task。在真正執行的時會將ExecutionVerterx包裝為一個Execution。
關於JobGraph如何提交到JobMaster不是本文的重點,有興趣的同學可以自行檢視org.apache.flink.runtime.dispatcher.DispatcherGateway#submitJob
的相關呼叫棧。
1.4.5 從JobGraph到ExecutionGraph
上面介紹了幾個重要概念。接下來看一下ExecutionGraph的構建過程。主要參考方法為org.apache.flink.runtime.executiongraph.DefaultExecutionGraph#attachJobGraph
。
首先是構建ExecutionJobVertex(參考其構造方法),設定其並行度、共享Solt、CoLocationGroup,並構建IntermediaResult與IntermediaResuktParitition,根據併發度建立ExecutionVertex,並檢查IntermediateResults是否有重複引用。最後,會對可切分的資料來源進行切分。
其次便是構建Edge(參考 org.apache.flink.runtime.executiongraph.EdgeManagerBuildUtil#connectVertexToResult)。根據DistributionPattern來建立EdgeManager,並將ExecutionVertex和IntermediateResult關聯起來,為執行時建立Task之間的資料交換就是以此為基礎建立資料的物理傳輸通道的。
1.4.6 開胃菜:從ExecutionGraph到真正的執行
當JobMaster生成ExecutionGraph後,便進入了作業排程階段。這裡面涉及到了不同的排程策略、資源申請、任務分發以及Failover的管理。涉及的內容極多,因此會在另外的文章中討論。對此好奇的同學,可以先看DefaultExecutionGraphDeploymentTest#setupScheduler
,裡面的程式碼較為簡單,可以觀察ExecutionGraph到Scheduling的過程。
private SchedulerBase setupScheduler(JobVertex v1, int dop1, JobVertex v2, int dop2)
throws Exception {
v1.setParallelism(dop1);
v2.setParallelism(dop2);
v1.setInvokableClass(BatchTask.class);
v2.setInvokableClass(BatchTask.class);
DirectScheduledExecutorService executorService = new DirectScheduledExecutorService();
// execution graph that executes actions synchronously
final SchedulerBase scheduler =
SchedulerTestingUtils.newSchedulerBuilder(
JobGraphTestUtils.streamingJobGraph(v1, v2),
ComponentMainThreadExecutorServiceAdapter.forMainThread())
.setExecutionSlotAllocatorFactory(
SchedulerTestingUtils.newSlotSharingExecutionSlotAllocatorFactory())
.setFutureExecutor(executorService)
.setBlobWriter(blobWriter)
.build();
final ExecutionGraph eg = scheduler.getExecutionGraph();
checkJobOffloaded((DefaultExecutionGraph) eg);
// schedule, this triggers mock deployment
scheduler.startScheduling();
Map<ExecutionAttemptID, Execution> executions = eg.getRegisteredExecutions();
assertEquals(dop1 + dop2, executions.size());
return scheduler;
}
2.小結
通過本文,我們瞭解各層圖存在的意義:
- StreamGraph與OptimizedPlan:從外部API轉向內部API,生成Graph的基本屬性。如果是批處理,則會進行一系列的優化。
- JobGraph:流批統一的Graph。在這裡做一些通用的優化,比如OperatorChain。
- ExecutionGraph:可執行級別的圖,構建時關注大量的執行細節:如併發、Checkpoint配置有效性、監控打點設定、重複引用檢查、可切分的資料來源進行切分等等。
通過圖的分層,Flink將不同的優化項、檢查項放到了合適它們的層次,這也是單一職責原則的體現。