[源码解析] Flink的Slot究竟是什么?(1)

[源码解析] Flink的Slot究竟是什么?(1)

0x00 摘要

Flink的Slot概念大家应该都听说过,但是可能很多朋友还不甚了解其中细节,比如具体Slot究竟代表什么?在代码中如何实现?Slot在生成执行图、调度、分配资源、部署、执行阶段分别起到什么作用?本文和下文将带领大家一起分析源码,为你揭开Slot背后的机理。

0x01 概述 & 问题

1.1 Fllink工作原理

从下图可以大致了解Flink的工作原理,即从提交Job到运行具体Task的过程。我们可以看到在具体运行时候,Task是依附于某一个Slot上的。

                                                                                        +--------------+
                                                                                        | +----------+ |
+--------+          +--------+         +---------+              +---------+             | |Task Slot | |
| Flink  |  Submit  |  Job   | Submit  |  Job    | Submit Task  |  Task   |Execute Task | +----------+ |
|Program +--------->+ Client +-------> | Manager +------------->+ Manager +------------>+              |
+--------+          +--------+         +---------+              +---------+             | +----------+ |
                                                                                        | |Task Slot | |
                                                                                        | +----------+ |
                                                                                        +--------------+

下图是为了手机上观看。

1.2 问题

带着问题学习比较好,我这里整理了几个问题,希望有一定代表性:

  • Slot究竟是什么?
  • Slot在代码中是如何实现的?
  • Slot定义里面究竟有什么?CPU?内存?
  • Slot是怎么实现各种隔离的?
  • TM中需要分成多少个Slot?
  • Slot是怎么分配给Task的?或者说Task是怎么跑在Slot上的?

如果想搞清楚这些问题可不是一件容易的事情,需要把Flink任务执行的流程梳理一遍才行。下面我就带着大家探寻一下。

0x02 示例代码

2.1 示例代码

示例代码就是Flink本身自带的WordCount。只不过添加了一些配置:

  • taskmanager.numberOfTaskSlots 是为了设置有几个taskmanager。
  • 其他是为了调试,加长了心跳时间或者超时时间。
Configuration conf = new Configuration();
conf.setString("heartbeat.timeout","18000000");
conf.setString("resourcemanager.job.timeout","18000000");
conf.setString("resourcemanager.taskmanager-timeout","18000000");
conf.setString("slotmanager.request-timeout","18000000");
conf.setString("slotmanager.taskmanager-timeout","18000000");
conf.setString("slot.request.timeout","18000000");
conf.setString("slot.idle.timeout","18000000");
conf.setString("akka.ask.timeout","18000000");
conf.setString("taskmanager.numberOfTaskSlots","1");
final LocalEnvironment env = ExecutionEnvironment.createLocalEnvironment(conf);

0x03 从Slot角度看系统划分

3.1 Flink组件

其实系统还是那么划分,只是我们从Slot资源分配角度看会更清晰

一个 Flink Cluster 是由一个 Flink Master 和多个 Task Manager 组成的,一个 Flink Master 中有一个 Resource Manager 和多个 Job Manager。

  • Flink Master 中每一个 Job Manager 单独管理一个具体的 Job。
    • Job Manager 中的 Scheduler 组件负责调度执行该 Job 的 DAG 中所有 Task,发出资源请求,即整个资源调度的起点;
    • JobManager 中的 Slot Pool 组件持有分配到该 Job 的所有资源。
  • Flink Master 中唯一的 Resource Manager 负责整个 Flink Cluster 的资源调度以及与外部调度系统对接,这里的外部调度系统指的是 Kubernetes、Mesos、Yarn 等资源管理系统。
  • Task Manager 负责 Task 的执行,其中的 Slot 是 Task Manager 资源的一个子集,也是 Flink 资源管理的基本单位,Slot 的概念贯穿资源调度过程的始终。

Flink Master 和 Task Manager 是进程级组件,其他的组件都是进程内的组件

3.2 Slot的由来

前面我们介绍了 TaskManager 是一个 JVM 进程,并会以独立的线程来执行一个task或多个subtask。

所以在 多线程处理 的 TaskManager 的内部是:在不同的线程上去执行一个或者多个它的子任务。而这个线程到底能执行多少个子任务呢?

为了控制内部线程运行子任务的个数,即为了控制一个 TaskManager 能接受多少个 task,就提出了slots概念。slots就是TaskManager的固定大小资源的一个集合。ResourceManager在做资源分配管理的时候,最小的单位就是slot。

Slot概念的优势就在于,如果JobMaster通过分发而来的作业,可以独立的在不同的Slot中执行。有一点类似于资源的隔离,这样,就可以尽可能的提高整个资源的效率。

在子任务同属一个 job 时,Flink还允许共享Slot。之所以允许共享,主要是因为既可以迅速的执行一些占用资源较小的任务,又可以从逻辑上抽离对并行计算是资源的消耗的多余计算(这点和虚拟内存有异曲同工之妙)。通过Map-reduce的映射来更好的进行作业和任务的执行。

3.3 资源分配

Flink 的资源调度是一个经典的两层模型,其中从 Cluster 到 Job 的分配过程是由 Slot Manager 来完成,Job 内部分配给 Task 资源的过程则是由 Scheduler 来完成。Scheduler 向 Slot Pool 发出 Slot Request(资源请求),Slot Pool 如果不能满足该资源需求则会进一步请求 Resource Manager,Resource Manager中具体来满足该请求的组件是 Slot Manager。

在 Operator 和 Task 中间的 Chaining 是指如何用 Operator 组成 Task 。在 Task 和 Job 之间的 Slot Sharing 是指多个 Task 如何共享一个 Slot 资源,这种情况不会发生在跨作业的情况中。在 Flink Cluster 和 Job 之间的 Slot Allocation 是指 Flink Cluster 中的 Slot 是怎样分配给不同的 Job 。

先使用 http://asciiflow.com/ 画个图总结下。

           +------------------------------------------+
           |              TaskManager                 |
           |   +-----------------------------------+  |
           |   |    TaskManagerServices            |  | 2.Status Report
           |   |    +-------------------------+    |  +--------------------+
           |   |    |     TaskSlotTable       |    |  |                    |
           |   |    |   +------------------+  |    |  | 1.Reqister         |
           |   |    |   |TaskSlot TaskSlot |  |    |  +---------------+    |
           |   |    |   +------------------+  |    |  |               |    |
           |   |    +-------------------------+    |  |               |    |
           |   +-----------------------------------+  | <---------+   |    |
           +------------------------------------------+ 6.Request |   |    |
                        |           ^                             |   |    |
                        | 7.Offer   | 8.submitTask                |   |    |
                        v           |                             |   v    v
+-----------------------+-----------+----------------+         +---+---+----+-----------+ 
|                       JobManager                   |         |                        |
|          +-------------------------------------+   |         |  ResourceManager       |
|          |          Scheduler                  |   |         |                        |
|          | +---------------------------------+ |   |         | +--------------------+ |
|          | | LogicalSlot  PhysicalSlot       | |   |         | |    SlotManager     | |
|          | +---------------------------------+ |   |         | |                    | |
|          +-------------------------------------+   |         | | +----------------+ | |
|                    |         |                     |         | | |                | | |
|3.allocateSharedSlot|         |4.allocateSingleSlot |         | | |TaskManagerSlot | | |
|                    v         |                     |         | | |                | | |
| +------------------+-+       |  +----------------+ |         | | |                | | |
| |  SlotSharingManager|       +->+  SlotPool      | |5.Request| | |                | | |
| | +----------------+ |          | +------------+ +---------> | | +----------------+ | |
| | |MultiTaskSlot   | |          | |AllocatedSlot | |         | |                    | |
| | |SingleTaskSlot  | |          | |            + | |         | +--------------------+ |
| | +----------------+ |          | +------------+ | |         +------------------------+
| +--------------------+          +----------------+ |
+----------------------------------------------------+

图. Flink 资源管理相关组件

下面这个图是为了在手机上观看。

如图,Cluster 到 Job 的资源调度过程中主要包含五个过程。

  • TE注册(就是上图中的 1,2 两项)

    • Reqister : 当 TE 启动之后,会向 RM 注册自己和自己内部的Slot。
    • Status Report:TE启动之后,会定期向 RM 进行心跳汇报,在心跳 payload 中,会携带 Slot 信息,RM 会随即更新自己内部Slot状态。
  • JM内部分配(就是上图中的 3,4 两项)

    • allocateSingleSlot : Scheduler 向 Slot Pool 发送请求,如果 Slot 资源足够则直接分配,如果 Slot 资源不够,则由 Slot Pool 再向 Slot Manager发送请求(此时即为 Job 向 Cluster 请求资源)
    • allocateSharedSlot : Scheduler 向 Slot Sharing Manager 发送请求,Slot Sharing Manager 构建好Slot树之后, 向 Slot Pool 发送请求,如果 Slot 资源足够则直接分配,如果 Slot 资源不够,则由 Slot Pool 再向 Slot Manager发送请求(此时即为 Job 向 Cluster 请求资源)
  • Job 向 Cluster 请求资源(就是上图的 5,6 两项)

    • 如果 Slot Manager 判断集群当中有足够的资源可以满足需求,那么就会向 Task Manager 发送 Request 指令,Slot Pool 再去满足 Scheduler 的资源请求。
    • 在 Active Resource Manager 资源部署模式下,当 Resource Manager 判定 Flink Cluster 中没有足够的资源去满足需求时,它会进一步去底层的资源调度系统请求资源,由调度系统把新的 Task Manager 启动起来,并且 TaskManager 向 Resource Manager 注册,则完成了新 Slot 的补充。
  • TE Offer Slot(就是上图的第 7 项)

    • Offer : Task Manager 就会提供 Slot 给 Slot Pool。
  • JM 会向 TE提交 Task(就是上图的第 8 项)

    • submitTask : JM 会更新内部Slot状态,然后向 TE 提交任务。

这些组件具体阐释如下。

3.4 Task Manager 范畴

Task Manager 内部相应组件为 TaskManagerServices,TaskSlotTableImpl。TaskManagerServices 是提供了 TaskManager 的基础服务,其中就包括了 Slot相关功能 TaskSlotTable

3.4.1 TaskManagerServices

TaskManagerServices里面有TaskSlotTable taskSlotTable; 这里负责Slot的管理和分配。

public class TaskManagerServices {
	/** TaskManager services. */
	private final TaskManagerLocation taskManagerLocation;
	private final long managedMemorySize;
	private final IOManager ioManager;
	private final ShuffleEnvironment<?,?> shuffleEnvironment;
	private final KvStateService kvStateService;
	private final BroadcastVariableManager broadcastVariableManager;
	private final TaskSlotTable<Task> taskSlotTable; // 这里是 Slot 相关
	private final JobManagerTable jobManagerTable;
	private final JobLeaderService jobLeaderService;
	private final TaskExecutorLocalStateStoresManager taskManagerStateStore;
	private final TaskEventDispatcher taskEventDispatcher;
}  

3.4.2 TaskSlotTableImpl

TaskSlotTableImpl 是Slots的容器,可以依据配置文件进行构建。其中重要的成员变量是:

  • taskSlots :本 TE 所有 task slot 的列表
  • allocatedSlots :本 TE 所有已经分配的 task slot 列表
  • slotsPerJob:每个job分配的 Slot 列表
public class TaskSlotTableImpl<T extends TaskSlotPayload> implements TaskSlotTable<T> {

   /**
    * Number of slots in static slot allocation.
    * If slot is requested with an index,the requested index must within the range of [0,numberSlots).
    * When generating slot report,we should always generate slots with index in [0,numberSlots) even the slot does not exist.
    */
   private final int numberSlots;

   /** Slot resource profile for static slot allocation. */
   private final ResourceProfile defaultSlotResourceProfile; // 定义了Slot拥有的资源

   /** Timer service used to time out allocated slots. */
   private final TimerService<AllocationID> timerService;

   /** The list of all task slots. */
   private final Map<Integer,TaskSlot<T>> taskSlots;

   /** Mapping from allocation id to task slot. */
   private final Map<AllocationID,TaskSlot<T>> allocatedSlots;

   /** Mapping from execution attempt id to task and task slot. */
   private final Map<ExecutionAttemptID,TaskSlotMapping<T>> taskSlotMappings;

   /** Mapping from job id to allocated slots for a job. */
   private final Map<JobID,Set<AllocationID>> slotsPerJob;

   /** Interface for slot actions,such as freeing them or timing them out. */
   @Nullable
   private SlotActions slotActions;
}  

3.4.3 ResourceProfile

其中 ResourceProfile 这个类需要特殊说明下,它定义了Slot拥有的资源,包括

  • CPU核数
  • task heap memory 大小
  • task off-heap memory大小
  • managed memory大小
  • network memory大小
  • 扩展资源,比如GPU and FPGA

当用户申请Slot时候,会根据用户需求来计算是否本Slot满足需求。

public class ResourceProfile implements Serializable {
	/** How many cpu cores are needed. Can be null only if it is unknown. */
	@Nullable
	private final Resource cpuCores;

	/** How much task heap memory is needed. */
	@Nullable // can be null only for UNKNOWN
	private final MemorySize taskHeapMemory;

	/** How much task off-heap memory is needed. */
	@Nullable // can be null only for UNKNOWN
	private final MemorySize taskOffHeapMemory;

	/** How much managed memory is needed. */
	@Nullable // can be null only for UNKNOWN
	private final MemorySize managedMemory;

	/** How much network memory is needed. */
	@Nullable // can be null only for UNKNOWN
	private final MemorySize networkMemory;

	/** A extensible field for user specified resources from {@link ResourceSpec}. */
	private final Map<String,Resource> extendedResources = new HashMap<>(1);
}

3.5 Resource Manager范畴

RM 中相关的组件是SlotManager。SlotManager的作用是负责维护一个统一视图,包括:

  • 所有注册的 task manager slots
  • slots的分配情况
  • 所有pending状态的 Slot 请求
  • 当前注册的 Task Manager

其简略版定义如下:

public class SlotManagerImpl implements SlotManager {

   /** Scheduled executor for timeouts. */
   private final ScheduledExecutor scheduledExecutor;

   /** Map for all registered slots. */
   private final HashMap<SlotID,TaskManagerSlot> slots;

   /** Index of all currently free slots. */
   private final LinkedHashMap<SlotID,TaskManagerSlot> freeSlots;

   /** All currently registered task managers. */
   private final HashMap<InstanceID,TaskManagerRegistration> taskManagerRegistrations;

   /** Map of fulfilled and active allocations for request deduplication purposes. */
   private final HashMap<AllocationID,SlotID> fulfilledSlotRequests;

   /** Map of pending/unfulfilled slot allocation requests. */
   private final HashMap<AllocationID,PendingSlotRequest> pendingSlotRequests;

   private final HashMap<TaskManagerSlotId,PendingTaskManagerSlot> pendingSlots;

   private final SlotMatchingStrategy slotMatchingStrategy;

   /** ResourceManager's id. */
   private ResourceManagerId resourceManagerId;

   /** Executor for future callbacks which have to be "synchronized". */
   private Executor mainThreadExecutor;

   /** Callbacks for resource (de-)allocations. */
   private ResourceActions resourceActions;
}    

3.6 Job Master范畴

JM 中关于Slot的主要有以下四个组件:Scheduler,SlotSharingManager,SlotPool,Execution。

3.6.1 Scheduler

Scheduler 组件是整个资源调度的起点,Scheduler主要作用是:

  • 负责调度执行该 Job 的 DAG 中所有 Task。根据 Execution Graph 和 Task 的执行状态,决定接下来要调度的 Task
  • 发起 SlotRequest 资源请求
  • 决定 Task / Slot 之间的分配

其简略版定义如下,可以看到其包括 slotSharingManagers 和 SlotPool。

SchedulerImpl 使用 LogicalSlot 和 PhysicalSlot

public class SchedulerImpl implements Scheduler {
   /** Strategy that selects the best slot for a given slot allocation request. */
   @Nonnull
   private final SlotSelectionStrategy slotSelectionStrategy;

   /** The slot pool from which slots are allocated. */
   @Nonnull
   private final SlotPool slotPool;

   /** Managers for the different slot sharing groups. */
   @Nonnull
   private final Map<SlotSharingGroupId,SlotSharingManager> slotSharingManagers;
}  

3.6.2 SlotSharingManager

SlotSharingManager 负责管理 slot sharing。

所谓的共享Slot,就是指不同operator下面的subTask(一个operator往往会因为并行度的原因,被分解成并行度个数的Task,并行执行)可以在同一个Task Slot中运行,即共享Slot。

在Storm中,supervisor下面是work,work中往往一个Executor执行一个Task。而在Flink中,TaskManager下面是slot,相同的是Slot和Work都是一个JVM进程,不同的是TaskManager会对Slot进行资源分配。

SlotSharingGroup是Flink中用来实现slot共享的类,它尽可能地让subtasks共享一个slot。保证同一个group的并行度相同的sub-tasks 共享同一个slots。算子的默认group为default(即默认一个job下的subtask都可以共享一个slot)

SlotSharingManager 允许建立一颗 TaskSlot hierarchy树。hierarchy树具体由 MultiTaskSlot 和 SingleTaskSlot 来实现,MultiTaskSlot代表树的中间节点,其包括一系列其他的TaskSlot,而SingleTaskSlot代表叶子节点。

在申请时候,SlotSharingManager 会通过 Slot Pool来具体申请物理Slot。

SlotSharingManager其精简版定义如下:

public class SlotSharingManager {
   private final SlotSharingGroupId slotSharingGroupId;

   /** Actions to release allocated slots after a complete multi task slot hierarchy has been released. */
   private final AllocatedSlotActions allocatedSlotActions; //指向SlotPool

   /** Owner of the slots to which to return them when they are released from the outside. */
   private final SlotOwner slotOwner;//指向SlotPool.ProviderAndOwner

   private final Map<SlotRequestId,TaskSlot> allTaskSlots;//存储MultiTaskSlot和SingleTaskSlot

   /** Root nodes which have not been completed because the allocated slot is still pending. */
   private final Map<SlotRequestId,MultiTaskSlot> unresolvedRootSlots;//在申请Slot后并且在确认前的临时存储

   /** Root nodes which have been completed (the underlying allocated slot has been assigned). */
   private final Map<TaskManagerLocation,Map<AllocationID,MultiTaskSlot>> resolvedRootSlots; //在申请Slot并且在确认后,根据TM划分为形如TM1-(MultiTaskSlot,MultiTaskSlot),TM2-(MultiTaskSlot,MultiTaskSlot)
}    

其运行时变量举例如下

this = {SlotSharingManager@5659} Object is being initialized
 slotSharingGroupId = {SlotSharingGroupId@5660} "041826faeb51ada2ba89356613583507"
 allocatedSlotActions = {SlotPoolImpl@5661} 
 slotOwner = {SchedulerImpl@5662} 
 allTaskSlots = {HashMap@6489}  size = 0
 unresolvedRootSlots = {HashMap@6153}  size = 0
 resolvedRootSlots = null

3.6.3 SlotPool

JobManager 中的 Slot Pool 组件持有分配到该 Job 的所有资源。其精简版定义如下,可以看出其拥有 resourceManagerGateway 以向 RM 发出资源请求,pending的请求保持在pendingRequests 或者 waitingForResourceManager之中。

SlotPool 中 管理的是 AllocatedSlots,就是物理Slot

public class SlotPoolImpl implements SlotPool {

   private final JobID jobId;

   /** All registered TaskManagers,slots will be accepted and used only if the resource is registered. */
   private final HashSet<ResourceID> registeredTaskManagers;

   /** The book-keeping of all allocated slots. */
   private final AllocatedSlots allocatedSlots;

   /** The book-keeping of all available slots. */
   private final AvailableSlots availableSlots;

   /** All pending requests waiting for slots. */
   private final DualKeyLinkedMap<SlotRequestId,AllocationID,PendingRequest> pendingRequests;

   /** The requests that are waiting for the resource manager to be connected. */
   private final LinkedHashMap<SlotRequestId,PendingRequest> waitingForResourceManager;

   /** The gateway to communicate with resource manager. */
   private ResourceManagerGateway resourceManagerGateway;

   private String jobManagerAddress;
}     

3.6.4 Execution

JobManager 将 JobGraph 转换为 ExecutionGraph,ExecutionGraph 是 JobGraph 的并行版本:每个 JobVertex 包含并行子任务的 ExecutionVertex。一个并行度为100的算子将拥有一个 JobVertex 和100个 ExecutionVertex。

每个 ExecutionGraph 都有一个与其相关联的作业状态。此作业状态指示作业执行的当前状态。

在执行 ExecutionGraph 期间,每个并行任务经过多个阶段,从创建(created)到完成(finished)或失败(failed),任务可以执行多次(例如故障恢复)。

ExecutionVertex 会跟踪特定子任务的执行状态。来自一个 JobVertex 的所有 ExecutionVertex 都由一个 ExecutionJobVertex 管理保存,ExecutionJobVertex 跟踪算子整体状态。

每个 Execution 表示一个 ExecutionVertex 的执行,每个 ExecutionVertex 都有一个当前 Execution(current execution)和一个前驱 Execution(prior execution)。

而每个 Execution 中间包含一个 private volatile LogicalSlot assignedResource;。这个变量在运行时候就是 SingleLogicalSlot。

Execution 部署时候,是 从 SingleLogicalSlot ---> AllocatedSlot ---> TaskManagerGateway 这个顺序获取了 TaskManager 的 RPC 网关,然后通过 taskManagerGateway.submitTask 才能提交任务的。这样就把 Execution 部署阶段和执行阶段联系起来了

/**
 * A single execution of a vertex. While an {@link ExecutionVertex} can be executed multiple times
 * (for recovery,re-computation,re-configuration),this class tracks the state of a single execution
 * of that vertex and the resources.  */
public class Execution implements AccessExecution,Archiveable<ArchivedExecution>,LogicalSlot.Payload {
	/** The executor which is used to execute futures. */
	private final Executor executor;

	/** The execution vertex whose task this execution executes. */
	private final ExecutionVertex vertex;

	/** The unique ID marking the specific execution instant of the task. */
	private final ExecutionAttemptID attemptId;

	private final Collection<PartitionInfo> partitionInfos;

	private final CompletableFuture<TaskManagerLocation> taskManagerLocationFuture;

	private volatile LogicalSlot assignedResource;
}    

0x04 Slot分配

4.1 Slot隔离原则

Flink 中的计算资源通过 Task Slot 来定义。每个 task slot 代表了 TaskManager 的一个固定大小的资源子集。例如,一个拥有3个slot的 TaskManager,会将其管理的内存平均分成三分分给各个 slot。

+----------------------------------------------+
|     TaskExecutor (Flink Memory)              |
|                                              |
| +-------------------+ +--------+ +--------+  |
| |      Slot         | |  Slot  | |  Slot  |  |
| |                   | |        | |        |  |
| | +---------------+ | | +----+ | | +----+ |  |
| | | Task Heap     | | | |    | | | |    | |  |
| | +---------------+ | | +----+ | | +----+ |  |
| | +---------------+ | | +----+ | | +----+ |  |
| | | Task Off Heap | | | |    | | | |    | |  |
| | +---------------+ | | +----+ | | +----+ |  |
| | +---------------+ | | +----+ | | +----+ |  |
| | |  Network      | | | |    | | | |    | |  |
| | +---------------+ | | +----+ | | +----+ |  |
| | +---------------+ | | +----+ | | +----+ |  |
| | | Managed       | | | |    | | | |    | |  |
| | +---------------+ | | +----+ | | +----+ |  |
| +-----------------------------------------+  |
| +-----------------------------------------+  |
| |      Framework Heap                     |  |
| +-----------------------------------------+  |
| +-----------------------------------------+  |
| |     Framework Off Heap                  |  |
| +-----------------------------------------+  |
+----------------------------------------------+

下面这个图是为了在手机上观看。

任务槽可以实现TaskManager中不同Task的资源隔离,不过是逻辑隔离,并且只隔离内存,亦即在调度层面认为每个任务槽“应该”得到taskmanager.heap.size的N分之一大小的内存,这意味着来自不同job的task不会为了内存而竞争。

slot隔离主要是对内存的隔离,CPU本身是不做隔离的,CPU在不同的slot之间是可以共享的

如果每个 TaskManager 有多个slot的话,也就是说多个task运行在同一个JVM中。而在同一个JVM进程中的task,可以共享TCP连接(基于多路复用)和心跳消息,可以减少数据的网络传输。也能共享一些数据结构,一定程度上减少了每个task的消耗。

每个 slot 都能跑由多个连续 task 组成的一个 pipeline,比如 MapFunction 的第n个并行实例和 ReduceFunction 的第n个并行实例可以组成一个 pipeline。通过动态的对槽的大小和数量的调整,就可以把任务的执行较好的并行起来。

4.2 系统里有多少Slot?

通过调整 task slot 的数量,用户可以定义task之间是如何相互隔离的。

我们按照每一个TaskManager机器的性能,它所含有的资源来配置slot。slot相当于它所有资源的一个子集,这个子集在执行过程中,就是一个隔离开的独立的子任务(线程)。相当于是用slot把不同的子任务之间做了一个隔离。如果机器的内存很大,cpu数量也多,那么就可以让它同时并行执行任务分配更多的slot。

slot内存是平均分配的,比如机器上有16G内存,如果划分4个slot的话,那每个slot就是4G内存了。如果每个slot内存太小的话,任务就执行不下去了,内存直接被撑爆这个是完全有可能的。

所以我们要根据我们执行任务的复杂程度,占用资源的角度和我们本身的机器它本身所有的资源大小做一个整体的分配。确定TaskManager到底分成几个slot。如果我们机器内存充足,为了避免不同的slot之间共享CPU导致我们资源本身负载的程度不高,这时我们往往按照CPU的数量来分配多少个slot。

JobManager拿到任务执行计划后,它如何确定到底需要多少个slot,这时它只要看整个作业里面,并行度最高的那个算子设置的并行度就可以了,只要满足它的需求,别的就都能满足了。

  • 在 Standalone 部署模式下,Task Manager 的数量是固定的,如果是 start-cluster.sh 脚本来启动集群,可以通过修改以下文件中的配置来决定 TM 的数量;也可以通过手动执行 taskmanager.sh 脚本来启动一个 TM 。

  • 在Kubernetes,Yarn,Mesos部署模式下,Task Manager 的数量由 SlotManager / ResourceManager 按需动态决定:

    • 当前 Slot 数量不能满足新的 Slot Request 时,申请并启动新的 TaskManager
    • TaskManager 空闲一段时间后,超时则释放

0x05 Slot分类

Flink中有很多种Slot定义,比较复杂。分布在不同范畴,不同模块之中。

这里先把几个 "Slot相关类" 整理出一张表(这里假定 启用CoLocationConstraint)。

  • 第一行是组件范畴。
  • 第二行是使用该 "Slot相关类" 的具体组件。比如 PhysicalSlot 在 SlotPool / Scheduler 这两个组件中使用。
  • 第三行是该 "Slot相关类" 对应的逻辑概念,比如 TaskSlot 表示 在 Task Executor 上的一个 Slot。
  • 第四行 ~ 最后 是该 "Slot相关类" 的类名,其中在同一行上的类是一一对应的,比如 一个 TaskSlot 对应 一个 AllocatedSlot,也对应一个 TaskManagerSlot。
  • SingleLogicalSlot 的成员变量SlotContext slotContext 实际指向了对应的 AllocatedSlot。这就把逻辑Slot 和 物理Slot联系起来了
  • 每个Execution最后对应一个SingleTaskSlot,SingleTaskSlot 的JobVertex信息从Execution中取。
  • Execution 部署时候,是 从 SingleLogicalSlot ---> AllocatedSlot ---> TaskManagerGateway 这个顺序获取了 TaskManager 的 RPC 网关,然后通过 taskManagerGateway.submitTask 才能提交任务的。这样就把 Execution 部署阶段和执行阶段联系起来了
  • MultiTaskSlot本身子节点只可能是SingleTaskSlot。
  • SlotOffer 只是一个RPC的中间变量。

具体关系如下:

范畴 Task Manager Job Master Job Master Job Master Resource Manager
组件 TaskSlotTable SlotPool / Scheduler SlotSharingManager / Scheduler Scheduler SlotManager
概念 Slot on TE Physical Slot Sharing Slot Logical Slot
TaskSlot AllocatedSlot SingleTaskSlot SingleLogicalSlot TaskManagerSlot
SlotOffer
MultiTaskSlot

其一一对应如下,除了 “taskManagerGateway.submitTask(调用)” 是表示调用关系,其余都是真实物理对应关系或者生成关系

  • TaskSlot 是 TE 中对物理资源的封装
  • SlotOffer是一个中间环节,和TaskSlot一一对应,TE用它来向JM提交Slot
  • SingleTaskSlot 是共享Slot树中的叶子结点,和SlotOffer一一对应。
  • MultiTaskSlot是共享Slot树的中间节点
  • SingleLogicalSlot 是一个Slot的逻辑抽象,由 SingleTaskSlot生成,并一一对应
  • Execution#assignedResource 成员变量指向SingleLogicalSlot,并一一对应
  • TaskManagerSlot 是 RM 中对 TE 的TaskSlot 的状态记录/管理,TaskManagerSlot 和 TaskSlot 一一对应
                              taskManagerGateway.submitTask(调用)
              +---------------------------------------------------------------+
              |                                                               |
              |                                                               |
+-------------+----------+         +------------------------+                 |
|       Execution        |         |    SingleLogicalSlot   |                 |
|                        |         |                        |                 |
| +--------------------+ |         |  +------------------+  |                 |
| |                    | |         |  |                  |  |                 |
| |  assignedResource+-----------> |  |  slotContext  +----------------+      |
| |                    | |         |  |                  |  |          |      |
| |                    | |         |  |  slotOwner       |  |          |      |
| |                    | |         |  |  (SlotPoolImpl)  |  |          |      |
| |                    | |         |  |                  |  |          |      |
| +--------------------+ |         |  +------------------+  |          |      |
|                        |         |                        |          |      |
+------------------------+         +------------------------+          |      |
                                                                       |      |
           +-----------------------------------------------------------+      |
           |                                                                  |
           v                                                                  v
+----------+--------------+        +-------------------------+    +-----------+-------+
|       AllocatedSlot     |        |       SlotOffer         |    | TaskSlot in TM    |
| +---------------------+ |        | +---------------------+ |    | +---------------+ |
| | resourceProfile +------------> | |     resourceProfile+-----> | |resourceProfile| |
| |                     | |        | |                     | |    | |               | |
| |  allocationId  +-------------> | |     allocationId  +------> | |allocationId   | |
| |                     | |        | |                     | |    | |               | |
| | physicalSlotNumber+----------> | |     slotIndex   +--------> | |index          | |
| |                     | |        | +---------------------| |    | +---------------+ |
| |                     | |        +-------------------------+    +-------------------+
| |                     | |        +----------------------------+
| |                     | |        |       JobMaster            |
| | TaskManagerLocation+---------> | +------------------------+ |
| |                     | |        | |                        | |
| | taskManagerGateway+----------> | | registeredTaskManagers | |
| +---------------------| |        | +------------------------+ |
+-------------------------+        +----------------------------+


下面这个为手机观看

我们再给出一个Slot流程(可以认为是从底向上的)。其中一个重点是:

当 JM 收到 SlotOffer时候,就会根据 RPC传递过来的 taskManagerId 参数,构建一个 taskExecutorGateway,然后这个 taskExecutorGateway 被赋予为 AllocatedSlot . taskManagerGateway。这样就把 JM 范畴的 Slot 和 Slot 所在的 taskManager 联系起来

---------- Task Executor ----------
       │ 
       │ 
┌─────────────┐   
│  TaskSlot   │  requestSlot
└─────────────┘     
       │ // TaskSlot 是 TE 中对物理资源的封装
       │  
       │                  
┌──────────────┐   
│  SlotOffer   │  offerSlotsToJobManager
└──────────────┘   
       │ // SlotOffer是一个中间环节,和TaskSlot一一对应,TE用它来向JM提交Slot  
       │ 
       │      
------------- Job Manager -------------
       │ 
       │       
┌──────────────┐   
│  SlotOffer   │  JobMaster#offerSlots(taskManagerId,slots)
└──────────────┘     
       │ //taskManager = registeredTaskManagers.get(taskManagerId);     
       │ //taskManagerLocation = taskManager.f0;     
       │ //taskExecutorGateway = taskManager.f1;     
       │    
       │       
┌──────────────┐   
│  SlotOffer   │  SlotPoolImpl#offerSlots
└──────────────┘     
       │ 
       │      
┌───────────────┐   
│ AllocatedSlot │  SlotPoolImpl#offerSlot
└───────────────┘      
       │ 
       │      
┌───────────────┐   
│ 回调 Future 3  │ SlotSharingManager#createRootSlot
└───────────────┘      
       │ 
       │      
┌───────────────┐   
│ 回调 Future 2  │  SingleTaskSlot#SingleTaskSlot 
└───────────────┘      
       │ // SingleTaskSlot 是共享Slot树中的叶子结点,和SlotOffer一一对应
       │   
       │      
┌───────────────────┐   
│ SingleLogicalSlot │ new SingleLogicalSlot
└───────────────────┘    
       │ // SingleLogicalSlot 是一个Slot的逻辑抽象,由 SingleLogicalSlot 生成,并一一对应
       │     
       │    
┌───────────────────┐   
│ SingleLogicalSlot │  
│ 回调 Future 1      │ allocationResultFuture.complete()
└───────────────────┘   
       │    
       │        
┌───────────────────────────────┐  
│     SingleLogicalSlot         │  
│回调 assignResourceOrHandleError│ 
└───────────────────────────────┘
       │    
       │        
┌────────────────┐   
│ ExecutionVertex│ tryAssignResource
└────────────────┘    
       │    
       │        
┌────────────────┐   
│    Execution   │ tryAssignResource
└────────────────┘       
       │ // Execution#assignedResource 成员变量指向SingleLogicalSlot,并一一对应
       │     
       │        
┌──────────────────┐   
│ SingleLogicalSlot│ tryAssignPayload
└──────────────────┘  
       │    
       │        
┌───────────────────────┐   
│   SingleLogicalSlot   │      
│ 回调deployOrHandleError│ 
└───────────────────────┘   
       │    
       │       
┌────────────────┐   
│ ExecutionVertex│ deploy
└────────────────┘    
       │    
       │        
┌────────────────┐   
│    Execution   │ deploy
└────────────────┘        
       │    
       │        
 ---------- Task Executor ----------
       │    
       │     
┌────────────────┐   
│  TaskExecutor  │ submitTask
└────────────────┘     
       │    
       │        
┌────────────────┐   
│  TaskExecutor  │ startTaskThread
└────────────────┘         

下面这个图是为了在手机上观看。

下面我们一一讲解。

5.1 Task Manager 范畴

在 Task Manager 中有如下几个Slot相关结构。

5.1.1 TaskSlot

org.apache.flink.runtime.taskexecutor.slot.TaskSlot 是 TM 范畴内关于Slot的基本定义。我们可以看出,其主要包括分配给哪个job,资源特征,在此slot中运行的task,资源管理器,Slot状态等等。

其中资源隔离就是通过 MemoryManager 完成的。在MemoryManager中,根据要管理的内存的总量和和每个内存页的大小得到内存页的数量生成相应大小数量的内存页来作为可以使用的内存。

Flink并不能保证TM的资源是严格平分给所有slot的。JVM中不同线程的资源并无严格隔离。所谓的平均划分更多的是调度上的考虑,可以理解为在调度时认为一个slot的资源相当于TM资源的1/n(n为slot数)。

特例就是对于DataSet作业使用到的managed memory,Flink目前是保证了TM的managed memory平均划分给所有slot的。Managed memory由TM上的MemoryManager管理,task在运行期间向MemoryManager申请内存,因此可以控制每个slot中task申请的内存上限

// Container for multiple {@link TaskSlotPayload tasks} belonging to the same slot. 
public class TaskSlot<T extends TaskSlotPayload> implements AutoCloseableAsync {

   /** Index of the task slot. */
   private final int index;

   /** Resource characteristics for this slot. */
   private final ResourceProfile resourceProfile;

   /** Tasks running in this slot. */
   private final Map<ExecutionAttemptID,T> tasks;

   private final MemoryManager memoryManager; // 物理资源隔离

   /** State of this slot. */
   private TaskSlotState state;

   /** Job id to which the slot has been allocated. */
   private final JobID jobId;

   /** Allocation id of this slot. */
   private final AllocationID allocationId; // 这个是在 SlotPoolImpl # requestSlotFromResourceManager 中生成的。

   /** The closing future is completed when the slot is freed and closed. */
   private final CompletableFuture<Void> closingFuture;
    
	public SlotOffer generateSlotOffer() {
		return new SlotOffer(allocationId,index,resourceProfile);
	}    
}  

RM 会 给 TM 发送 分配 Slot 的请求,TM 分配 Slot 之后,会更新 TaskSlotTableImpl 内部变量。分配的 TaskSlot 其runtime时候变量如下。

taskSlot = {TaskSlot@12027}
 index = 0
 resourceProfile = {ResourceProfile@5359} "ResourceProfile{managedMemory=128.000mb (134217728 bytes),networkMemory=64.000mb (67108864 bytes)}"
 tasks = {HashMap@18322}  size = 0
 memoryManager = {MemoryManager@18323} 
 state = {TaskSlotState@18324} "ALLOCATED"
 jobId = {JobID@6259} "c7be7a4944784caac382cdcd9e651863"
 allocationId = {AllocationID@6257} "20d50091f2d16939f79f06edf66494f7"
 closingFuture = {CompletableFuture@18325} "java.util.concurrent.CompletableFuture@1ec6b32[Not completed]"

5.1.2 SlotOffer

当 TM 分配 TaskSlot 之后,会调用 TaskSlot # generateSlotOffer 函数生成一个SlotOffer,发给 JM

SlotOffer 就是 RPC 专用的一个中间变量。

/**
 * Describe the slot offering to job manager provided by task manager.
 */
public class SlotOffer implements Serializable {
	/** Allocation id of this slot,this would be the only identifier for this slot offer */
	private AllocationID allocationId;

	/** Index of the offered slot */
	private final int slotIndex;

	/** The resource profile of the offered slot */
	private final ResourceProfile resourceProfile;
}

5.2 Job Manager 范畴

JM 内部关于 Slot 的类大致如下:

物理Slot

org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot extends SlotContext

org.apache.flink.runtime.jobmaster.slotpool.AllocatedSlot implements PhysicalSlot

分享Slot

org.apache.flink.runtime.jobmaster.slotpool.SlotSharingManager.TaskSlot

org.apache.flink.runtime.jobmaster.slotpool.SlotSharingManager.MultiTaskSlot

org.apache.flink.runtime.jobmaster.slotpool.SlotSharingManager.SingleTaskSlot

逻辑Slot

org.apache.flink.runtime.jobmaster.LogicalSlot

org.apache.flink.runtime.jobmaster.slotpool.SingleLogicalSlot implements LogicalSlot,PhysicalSlot.Payload

5.2.1 物理Slot

一个 TM 的 TaskSlot 对应一个 SlotOffer,通过 RPC offer 这个Slot 给 JM,然后 JM 根据这个 SlotOffer 生成一个 AllocatedSlot,这样 TM 的 TaskSlot 就对应起一个 SlotPool 的 物理 SlotSlotPool 里面管理的都是物理Slot

/* The context of an {@link AllocatedSlot}. This represent an interface to classes outside the slot pool to interact with allocated slots.*/
public interface PhysicalSlot extends SlotContext {

   /**
    * Tries to assign the given payload to this allocated slot. This only works if there has not been another payload assigned to this slot.
    *
    * @param payload to assign to this slot
    * @return true if the payload could be assigned,otherwise false
    */
   boolean tryAssignPayload(Payload payload);

   /**
    * Payload which can be assigned to an {@link AllocatedSlot}.
    */
   interface Payload {

      /**
       * Releases the payload
       *
       * @param cause of the payload release
       */
      void release(Throwable cause);
   }
}

AllocatedSlot 实现了 PhysicalSlot,因为是代表了具体物理Slot,所以里面有 TaskManagerGateway 以便和 TaskManager交互。

TaskManagerGateway接口定义了和TaskManager通信的方法,有两种具体实现,分别基于Actor模式和RPC模式。基于RPC的实现会包含一个TaskExecutorGateway的实现类TaskExecutor来代理提交任务的实际工作。

class AllocatedSlot implements PhysicalSlot {

	/** The ID under which the slot is allocated. Uniquely identifies the slot. */
	private final AllocationID allocationId;

	/** The location information of the TaskManager to which this slot belongs */
	private final TaskManagerLocation taskManagerLocation;

	/** The resource profile of the slot provides */
	private final ResourceProfile resourceProfile;

	/** RPC gateway to call the TaskManager that holds this slot */
	private final TaskManagerGateway taskManagerGateway;

	/** The number of the slot on the TaskManager to which slot belongs. Purely informational. */
	private final int physicalSlotNumber;

	private final AtomicReference<Payload> payloadReference;
} 

5.2.2 分享Slot

以下三个都是 在 SlotSharingManager 内部定义的。在 SlotSharingManager 和 Scheduler 中都有涉及,在某种程度上,可以认为是 物理 Slot 和 Logical 的一个中间状态

  • org.apache.flink.runtime.jobmaster.slotpool.SlotSharingManager.TaskSlot

  • org.apache.flink.runtime.jobmaster.slotpool.SlotSharingManager.MultiTaskSlot

  • org.apache.flink.runtime.jobmaster.slotpool.SlotSharingManager.SingleTaskSlot

正如前面所述,SlotSharingManager 负责管理 slot sharing,slot sharing允许不同task在同一个slot中运行以实现co-location constraints。SlotSharingManager 允许建立一颗 TaskSlot hierarchy树。

hierarchy树具体由 MultiTaskSlot 和 SingleTaskSlot 来实现,MultiTaskSlot代表树的中间节点,其包括一系列其他的TaskSlot,而SingleTaskSlot代表叶子节点。

当 JM 生成一个 物理Slot,即 AllocatedSlot 之后,会 回调 一个 Future 2(代码在 SingleTaskSlot 构造函数,回调函数的输入参数是 SlotContext ),因为有 PhysicalSlot extends SlotContext, 所以 SingleTaskSlot 这里就把 物理Slot 映射成了一个 逻辑Slot : SingleLogicalSlot

/**
 * Base class for all task slots.
 */
public abstract static class TaskSlot {
   // every TaskSlot has an associated slot request id
   private final SlotRequestId slotRequestId;

   // all task slots except for the root slots have a group id assigned
   @Nullable
   private final AbstractID groupId;
}   

MultiTaskSlot继承了TaskSlot

public final class MultiTaskSlot extends TaskSlot implements PhysicalSlot.Payload {

   private final Map<AbstractID,TaskSlot> children;

   // the root node has its parent set to null
   @Nullable
   private final MultiTaskSlot parent;

   // underlying allocated slot
   private final CompletableFuture<? extends SlotContext> slotContextFuture;

   // slot request id of the allocated slot
   @Nullable
   private final SlotRequestId allocatedSlotRequestId;

   // true if we are currently releasing our children
   private boolean releasingChildren;

   // the total resources reserved by all the descendants.
   private ResourceProfile reservedResources;
}    

SingleTaskSlot 也继承了 TaskSlot,但是 每一个 SingleTaskSlot 有一个 MultiTaskSlot 父亲。

/**
 * {@link TaskSlot} implementation which harbours a {@link LogicalSlot}. The {@link SingleTaskSlot}
 * cannot have any children assigned.
 */
public final class SingleTaskSlot extends TaskSlot {
		private final MultiTaskSlot parent;

		// future containing a LogicalSlot which is completed once the underlying SlotContext future is completed
		private final CompletableFuture<SingleLogicalSlot> singleLogicalSlotFuture;

		// the resource profile of this slot.
		private final ResourceProfile resourceProfile;  
}   

5.2.3 逻辑Slot

上节的回调函数会带着 SingleLogicalSlot 继续调用,这时候调用到的回调函数 是 Deploy 阶段设置的回调函数 assignResourceOrHandleError,这就是分配资源阶段了

/**
 * A logical slot represents a resource on a TaskManager into
 * which a single task can be deployed.
 */
public interface LogicalSlot {

   Payload TERMINATED_PAYLOAD = new Payload() {

      private final CompletableFuture<?> completedTerminationFuture = CompletableFuture.completedFuture(null);

      @Override
      public void fail(Throwable cause) {
         // ignore
      }

      @Override
      public CompletableFuture<?> getTerminalStateFuture() {
         return completedTerminationFuture;
      }
   };
 }  

SingleLogicalSlot 实现了 LogicalSlot。其成员变量 SlotContext slotContext; 实际指向了对应的AllocatedSlot

/**
 * Implementation of the {@link LogicalSlot} which is used by the {@link SlotPoolImpl}.
 */
public class SingleLogicalSlot implements LogicalSlot,PhysicalSlot.Payload {

   private final SlotRequestId slotRequestId;

   private final SlotContext slotContext;  // 这里实际指向了对应的AllocatedSlot

   // null if the logical slot does not belong to a slot sharing group,otherwise non-null
   @Nullable
   private final SlotSharingGroupId slotSharingGroupId;

   // locality of this slot wrt the requested preferred locations
   private final Locality locality;

   // owner of this slot to which it is returned upon release
   private final SlotOwner slotOwner;

   private final CompletableFuture<Void> releaseFuture;

   private volatile State state;

   // LogicalSlot.Payload of this slot
   private volatile Payload payload;
}   

5.2.4 Execution

现在我们已经有了 逻辑Slot,而且通过回调函数到了部署阶段。

org.apache.flink.runtime.executiongraph.Execution 里面包含 LogicalSlot。而且要注意,Execution 实现了 LogicalSlot.Payload。这个和 PhysicalSlot.Payload 混用。最后 Slot.Payload就是Execution 自己。

public class Execution implements AccessExecution,LogicalSlot.Payload {
    ......
	private volatile LogicalSlot assignedResource;
    ......
}

回调函数继续做如下操作

  • 深入调用 executionVertex.tryAssignResource,

  • ExecutionVertex # tryAssignResource

  • Execution # tryAssignResource

  • SingleLogicalSlot# tryAssignPayload(this),这里会把 Execution 自己 赋值给Slot.payload。

这样就完成了资源的部署,即Execution 和 LogicalSlot 联系在一起。

5.3 Resource Manager 范畴

RM的组件 SlotManagerImpl 使用 TaskManagerSlot

package org.apache.flink.runtime.resourcemanager.slotmanager.TaskManagerSlot

5.3.1 TaskManagerSlot

这里当 TM 启动或者状态变化时候(利用心跳机制),TM会向 RM 注册Slot,或者发送 Slot报告。

RM(SlotManagerImpl )会 相应 update 自己内部TaskManagerSlot的状态。 TaskManagerSlot 和 TM的 TaskSlot 是一一对应的。

/**
 * A TaskManagerSlot represents a slot located in a TaskManager. It has a unique identification and
 * resource profile associated.
 */
public class TaskManagerSlot implements TaskManagerSlotInformation {

	/** The unique identification of this slot. */
	private final SlotID slotId;

	/** The resource profile of this slot. */
	private final ResourceProfile resourceProfile;

	/** Gateway to the TaskExecutor which owns the slot. */
	private final TaskExecutorConnection taskManagerConnection;

	/** Allocation id for which this slot has been allocated. */
	private AllocationID allocationId;

	/** Allocation id for which this slot has been allocated. */
	@Nullable
	private JobID jobId;
} 	

数据结构我们已经分析完毕,我们看看系统启动时候如何构建这些数据结构。

0x06 系统启动

以下是基于IDEA调试的过程,线上代码大同小异。

Flink启动时,会建立相应的模块,比如 TaskManagerServices,Scheduler,SlotPool等。

我们具体按照大模块的范畴来讲解。

6.1 TM & TE范畴

6.1.1 TaskManagerServices启动

Slot的分配管理就在TaskManagerServices之中,具体如下。

6.1.1.1 调用路径

在建立TaskExecutor的过程中,会根据配置来建立 TaskSlotTable,其代码调用路径如下:

  1. LocalExecutor#execute
  2. LocalExecutor#startMiniCluster
  3. MiniCluster#start
  4. MiniCluster#startTaskManagers
  5. MiniCluster#startTaskExecutor
  6. TaskManagerRunner#startTaskManager
  7. TaskManagerServices#fromConfiguration
  8. TaskManagerServices#createTaskSlotTable,根据配置信息建立TaskSlotTable
  9. TaskSlotTableImpl#TaskSlotTableImpl
6.1.1.2 配置信息

TaskManager的信息是可以通过配置文件进行调整的。在 TaskManagerServices#createTaskSlotTable 中能看到根据配置得到的信息

taskManagerServicesConfiguration = {TaskManagerServicesConfiguration@4263} 
  confData = {HashMap@4319}  size = 13
   "heartbeat.timeout" -> "18000000"
   "taskmanager.memory.network.min" -> {MemorySize@4338} "64 mb"
   "taskmanager.cpu.cores" -> {Double@4340} 1.7976931348623157E308
   "taskmanager.memory.task.off-heap.size" -> {MemorySize@4342} "9223372036854775807 bytes"
   "execution.target" -> "local"
   "rest.bind-port" -> "0"
   "taskmanager.memory.network.max" -> {MemorySize@4338} "64 mb"
   "execution.attached" -> {Boolean@4349} true
   "jobmanager.scheduler" -> "ng"
   "taskmanager.memory.managed.size" -> {MemorySize@4353} "128 mb"
   "taskmanager.numberOfTaskSlots" -> {Integer@4355} 2
   "taskmanager.memory.task.heap.size" -> {MemorySize@4342} "9223372036854775807 bytes"
   "rest.address" -> "localhost"
 resourceID = {ResourceID@4274} "40d390ec-7d52-4f34-af86-d06bb515cc48"
 taskManagerAddress = {Inet4Address@4276} "localhost/127.0.0.1"
 numberOfSlots = 2
6.1.1.3 TaskSlotTableImpl

TaskSlotTableImpl的runtime的信息如下,目前只是建立了数据结构,并没有具体生成Slot资源,因为此时只是知道TM有这些资源,但是具体分配应该RM来处理

this = {TaskSlotTableImpl@4192} 
 numberSlots = 2
 defaultSlotResourceProfile = {ResourceProfile@4194} "ResourceProfile{managedMemory=64.000mb (67108864 bytes),networkMemory=32.000mb (33554432 bytes)}"
 memoryPageSize = 32768
 timerService = {TimerService@4195} 
 taskSlots = {HashMap@4375}  size = 0
 allocatedSlots = {HashMap@4387}  size = 0
 taskSlotMappings = {HashMap@4391}  size = 0
 slotsPerJob = {HashMap@4395}  size = 0
 slotActions = null
 state = {TaskSlotTableImpl$State@4403} "CREATED"
 budgetManager = {ResourceBudgetManager@4383} 
 closingFuture = {CompletableFuture@4408} "java.util.concurrent.CompletableFuture@7aac8884[Not completed]"
 mainThreadExecutor = {ComponentMainThreadExecutor$DummyComponentMainThreadExecutor@4219} 

6.1.2 TaskExecutor启动

Flink会通过如下的代码调用路径来建立Slot。

  1. TaskExecutor#onStart;

  2. TaskExecutor#startTaskExecutorServices;

    • taskSlotTable.start(new SlotActionsImpl(),getMainThreadExecutor());
      
  3. TaskSlotTableImpl#start。此处会把TaskExecutor的 "main thread execution context" 和TaskSlotTable联系起来;

6.2 Resource Manager范畴

RM 这里主要是SlotManager 的启动。

6.2.1 SlotManager启动

SlotManager的调用栈如下:

  • MiniCluster#setupDispatcherResourceManagerComponents
  • MiniCluster#createDispatcherResourceManagerComponents
  • DefaultDispatcherResourceManagerComponentFactory#create
  • StandaloneResourceManagerFactory#createResourceManager
  • ResourceManagerRuntimeServices#fromConfiguration
  • ResourceManagerRuntimeServices#createSlotManager
  • SlotManagerImpl#SlotManagerImpl

6.3 Job Master范畴

这里模块比较多。

6.3.1 Scheduler

Scheduler 的执行路径如下:

  • JobMaster#JobMaster

  • JobMaster#createScheduler

  • DefaultSchedulerFactory#createInstance

  • DefaultScheduler#init,生成 schedulingStrategy = LazyFromSourcesSchedulingStrategy。Flink支持两种执行模式,LAZY_FROM_SOURCE模式只有在一个Operator的输入数据就绪时才初始化该节点,EAGER模式会在一开始就按拓扑顺序加载计算图中的所有节点。

    • this.schedulingStrategy = schedulingStrategyFactory.createInstance(this,getSchedulingTopology());
      
      

6.3.2 SlotSharingManager

这里提前拿出来说明,因为实际上 SlotSharingManager 是在具体使用时候,如果发现没有才会生成。

SlotSharingManager的执行路径如下:

  • SlotProviderStrategy$NormalSlotProviderStrategy#allocateSlot
  • SchedulerImpl#allocateSlot
  • SchedulerImpl#allocateSlotInternal
  • SchedulerImpl#internalAllocateSlot
  • SchedulerImpl#allocateSharedSlot
  • HashMap#computeIfAbsent
  • SchedulerImpl#lambda$allocateSharedSlot
  • SlotSharingManager#init

6.3.3 SlotPool

SlotPool的执行路径如下:

  • DefaultJobManagerRunnerFactory#createJobManagerRunner
  • JobManagerRunnerImpl#JobManagerRunnerImpl
  • DefaultJobMasterServiceFactory#createJobMasterService
  • JobMaster#JobMaster
  • DefaultSlotPoolFactory#createSlotPool
  • SlotPoolImpl#SlotPoolImpl

至此,我们需要的基本模块都已经生成。

下文我们会围绕 Slot 从源码执行流程入手来一一分析,具体包括分配资源,部署等各个角度来讲解,敬请期待。

0xFF 参考

一文了解 Apache Flink 的资源管理机制

Flink5:Flink运行架构(Slot和并行度)

Flink Slot详解与Job Execution Graph优化

聊聊flink的slot.request.timeout配置

Apache Flink 源码解析(三)Flink on Yarn (2) Resource Manager

Flink on Yarn模式下的TaskManager个数

Flink on YARN时,如何确定TaskManager数

Flink】Flink作业调度流程分析

Flink原理与实现:如何生成ExecutionGraph及物理执行图

Flink源码走读(一):Flink工程目录

flink分析使用之七任务的启动

flink源码解析3 ExecutionGraph的形成与物理执行

Flink 内部原理之作业与调度

Flink之用户代码生成调度层图结构

3. Flink Slot申请

Flink 任务和调度

Flink的Slot是如何做到平均划分TM内存的?

Flink-1.10.0中的readTextFile解读

Flink1.7.2 Dataset 文件切片计算方式和切片数据读取源码分析

flink任务提交流程分析

Flink Parallelism和Slot理解

相关文章

文章浏览阅读5.3k次,点赞10次,收藏39次。本章详细写了mysq...
文章浏览阅读1.8k次,点赞50次,收藏31次。本篇文章讲解Spar...
文章浏览阅读7.8k次,点赞9次,收藏34次。ES查询常用语法目录...
文章浏览阅读928次,点赞27次,收藏18次。
文章浏览阅读1.1k次,点赞24次,收藏24次。作用描述分布式协...
文章浏览阅读1.5k次,点赞26次,收藏29次。为贯彻执行集团数...