目录

ExectionGraph执行图

ExecutionGraph的核心对象

1、ExecutionJobVertex

2、ExecutionVertex

3、IntermediateResult

4、IntermediateResultPartition

5、ExecutionEdge

6、Execution

ExecutionGraph具体生成流程

小结

1、Flink客户端提交JobGraph给JobManager

2、构建ExecutionGraph对象


ExectionGraph执行图

在Flink中ExecutionGraph执行图是协调数据流的分布式执行的中心数据结构,它保留了每个并行任务、每个中间流以及它们之间的的通信表示。StreamGraph和JobGraph的转化生成都是在Flink客户端,而最终Flink作业运行时调度层的核心执行图ExecutionGraph是在服务器的JobManager中生成的

ExecutionGraph在实际处理转换上只是改动了GobGraph的每个节点,并没有对整个拓扑结构进行改变。主要发生以下转变:

  • 加入了并行度的概念,成为真正可调度的图结构;
  • 生成了与JobVertex对应的ExecutionJobVertex和ExecutionVertex,以及IntermediateDataSet对应的IntermediateResult和IntermediateResultPartition等,并行将通过这些类实现。

flink JobGraph 如何拆分为多个 flink executiongraph_封装

ExecutionGraph的核心对象

1、ExecutionJobVertex

ExecutionJobVertex和JobGraph中的JobVertex一一对应。ExecutionJobVertex表示执行过程中来自JobGraph的一个顶点,它保存所有并行子任务的聚合状态。每个ExecutionJobVertex都有和并行度一样多的ExecutionVertex

2、ExecutionVertex

ExecutionVertex表示ExecutionVertex的其中一个并发子任务,输入是ExecutionEdge,输出是IntermediateResultPartition。ExecutionVertex由ExecutionJobVertex和并行子任务的索引标识

3、IntermediateResult

IntermediateResult和JobGraph中的IntermediateDataSet一一对应。一个IntermediateResult包含多个IntermediateResultPartition,其个数等于该算子的并行度。

4、IntermediateResultPartition

IntermediateResultPartition表示ExecutionVertex的一个输出分区(中间结果),生产者是ExecutionVertex,消费者是若干个ExecutionEdge

5、ExecutionEdge

ExecutionEdge表示ExecutionVertex的输入,输入源是IntermediateResultPartition,目的地是ExecutionVertex。源和目的地都只能有一个。

6、Execution

ExecutionVertex可以被执行多次(用于恢复、重新计算、重新配置),Execution负责跟踪该顶点和资源的一次执行的状态信息

为了防止出现故障,或者在某些数据需要重新计算的情况下,ExecutionVertex可能会有多次执行。因为在以后的操作请求中,它不再可用。执行由ExecutionAttemptID标识。JobMananger和TaskManager之间关于任务部署和任务状态更新的所有消息都是使用ExecutionAttemptID来定位消息接受者的。

ExecutionGraph具体生成流程

ExecutionGraph是一个描述并行化JobGraph执行图,生成并维护CheckpointCordinator,TaskTracker,ExecutionVertex,ExecutionEdge,IntermediateResult等组件的细粒度数据结构。而ExecutionGraph只是Flink核心数据结构的一部分,在JM的SchedulerNG生成ExecutionGraph之前,Flink会执行一系列在RM分配的Container中的操作,跟Yarn集群的交互,基于Akka的RpcServer(Flink封装的actor)通信模式注册,基于ZK的高可用选举等,而其中涉及到的核心管理和调度组件包括ResourceManager,Dispatcher,JobManager,Scheduler等。

本章设计的框架较多,在构造ExecutionGraph过程中会依次解析Flink跟Yarn的交互,基于Akka的通信模式以及基于ZK的HA等服务。

上一章节说到了JobGraph的建立,JobGraph创建完毕后,Flink会创建YarnJobClusterExecutor向集群提交任务。在StreamExecutionEnvironment类中,调用public JobExecutionResult execute(StreamGraph streamGraph) throws Exception中的final JobClient jobClient = executeAsync(streamGraph)方法。

public JobClient executeAsync(StreamGraph streamGraph) throws Exception {
		checkNotNull(streamGraph, "StreamGraph cannot be null.");
		checkNotNull(configuration.get(DeploymentOptions.TARGET), "No execution.target specified in your configuration file.");

		final PipelineExecutorFactory executorFactory =
			executorServiceLoader.getExecutorFactory(configuration);

		checkNotNull(
			executorFactory,
			"Cannot find compatible factory for specified execution.target (=%s)",
			configuration.get(DeploymentOptions.TARGET));
        // 从这里调用execute方法开始创建ExecutionGraph
		CompletableFuture<JobClient> jobClientFuture = executorFactory
			.getExecutor(configuration)
			.execute(streamGraph, configuration, userClassloader);

		try {
			JobClient jobClient = jobClientFuture.get();
			jobListeners.forEach(jobListener -> jobListener.onJobSubmitted(jobClient, null));
			return jobClient;
		} catch (ExecutionException executionException) {
			final Throwable strippedException = ExceptionUtils.stripExecutionException(executionException);
			jobListeners.forEach(jobListener -> jobListener.onJobSubmitted(null, strippedException));

			throw new FlinkException(
				String.format("Failed to execute job '%s'.", streamGraph.getJobName()),
				strippedException);
		}
	}

调用在AbstractJobClusterExecutor类中的execute方法。

public CompletableFuture<JobClient> execute(@Nonnull final Pipeline pipeline, @Nonnull final Configuration configuration, @Nonnull final ClassLoader userCodeClassloader) throws Exception {
        // 生成JobGraph
		final JobGraph jobGraph = PipelineExecutorUtils.getJobGraph(pipeline, configuration);

		try (final ClusterDescriptor<ClusterID> clusterDescriptor = clusterClientFactory.createClusterDescriptor(configuration)) {
			final ExecutionConfigAccessor configAccessor = ExecutionConfigAccessor.fromConfiguration(configuration);

			final ClusterSpecification clusterSpecification = clusterClientFactory.getClusterSpecification(configuration);
            // 开始向集群发布部署任务
			final ClusterClientProvider<ClusterID> clusterClientProvider = clusterDescriptor
					.deployJobCluster(clusterSpecification, jobGraph, configAccessor.getDetachedMode());
			LOG.info("Job has been submitted with JobID " + jobGraph.getJobID());

			return CompletableFuture.completedFuture(
					new ClusterClientJobClientAdapter<>(clusterClientProvider, jobGraph.getJobID(), userCodeClassloader));
		}
	}

大概步骤为创建YarnClient,check环境,比如检查Yarn队列是否存在,注册需要申请的内存,CPU等资源,初始化Jar等文件存放环境(hdfs),kerberos身份认证等。所有的申请资源和执行动作都会put进Yarn的ApplicationSubmissionContext上下文中,最后通过yarnClient.submitApplication(appContext)向Yarn集群提交任务。

private ApplicationReport startAppMaster(
			Configuration configuration,
			String applicationName,
			String yarnClusterEntrypoint,
			JobGraph jobGraph,
			YarnClient yarnClient,
			YarnClientApplication yarnApplication,
			ClusterSpecification clusterSpecification) throws Exception {
 
		// ------------------ Initialize the file systems -------------------------
...
 
LOG.info("Submitting application master " + appId);
		yarnClient.submitApplication(appContext);
...
}

当Yarn集群接收到Client申请后它会做一系列资源验证动作,包括是否有空闲的内存,CPU能给到用户(flink)申请的资源,用户权限检验,是否有空闲的Container,是否有指定的Yarn队列等,若满足以上条件则开始通知RM分配一个Container到空闲的NM上用作启动Flink的AM,在Container分配完毕后会去hdfs(默认)加载用户上传的Jar包,反射加载完成后调用deployInternal指定的入口类的Main方法。这里除了最后一步是Flink的内部代码,其余的都是在Yarn集群内部完成。

这里我们指定的是Per-Job模式:

/**
	 * The class to start the application master with. This class runs the main
	 * method in case of the job cluster.
	 */
	protected String getYarnJobClusterEntrypoint() {
		return YarnJobClusterEntrypoint.class.getName();
	}
 
 
    //当flink client提交的任务到ResourceManager完成后
	//ResourceManager会做资源验证后在集群找到有空闲资源的NodeManager分配一个container作为flink的AppMaster再调用下面的main函数
	public static void main(String[] args) {
		// startup checks and logging
		....
 
		try {
			YarnEntrypointUtils.logYarnEnvironmentInformation(env, LOG);
		} catch (IOException e) {
			LOG.warn("Could not log YARN environment information.", e);
		}
 
		Configuration configuration = YarnEntrypointUtils.loadConfiguration(workingDirectory, env);
 
		YarnJobClusterEntrypoint yarnJobClusterEntrypoint = new YarnJobClusterEntrypoint(configuration);
 
		//准备启动Flink集群
		ClusterEntrypoint.runClusterEntrypoint(yarnJobClusterEntrypoint);
	}

ClusterEntrypoint是FlinkOnYarn在AM的主入口类,它主要做两件事,

第一,启动部分外部依赖服务,列如:RpcService(Akka),HAService,BlobService等

第二,启动自身的RM和Dispatcher,这里RM是Flink自身内部维护的资源控制器,比如Flink的Slot是两阶段提交申请(后面会解析),Dispatcher主要是生成JobManager,

而RM和Dispatcher均是可配置HA的(LeaderContender)
 

private void runCluster(Configuration configuration, PluginManager pluginManager) throws Exception {
		synchronized (lock) {
 
			//创建RPCService(封装的Akka的ActorSystem,用作创建和维护后面flink各服务的actors)
			//跟spark一样,flink的消息通讯基于Akka的actor异步事件通信模式,而数据交互是基于擅长高并发异步IO框架Netty
			//启动HaService(HA模式会使用zk作为主备master元数据的同步保存点),负责大文件传输服务的blobServer等
			//使用连接zk客户端的Curator框架在zk创建flink目录并开启监听该目录数据变化的线程
			//像Dispatcher,JobManager等基于zk的HA模式也是调用Curator的LeaderLatch来选主并且它们的核心逻辑入口也是在选主成功的代码块中
			initializeServices(configuration, pluginManager);
 
			// write host information into configuration
			configuration.setString(JobManagerOptions.ADDRESS, commonRpcService.getAddress());
			configuration.setInteger(JobManagerOptions.PORT, commonRpcService.getPort());
 
			//创建DefaultDispatcherResourceManagerComponentFactory
			final DispatcherResourceManagerComponentFactory dispatcherResourceManagerComponentFactory = createDispatcherResourceManagerComponentFactory(configuration);
 
			clusterComponent = dispatcherResourceManagerComponentFactory.create(
				configuration,
				ioExecutor,
				commonRpcService,
				haServices,
				blobServer,
				heartbeatServices,
				metricRegistry,
				archivedExecutionGraphStore,
				new RpcMetricQueryServiceRetriever(metricRegistry.getMetricQueryServiceRpcService()),
				this);
 
		....
	}

这里主要提下RpcService(Akka),HAService两个服务

数据之间的传输用的Netty(后面会解析),而各角色之间的通信用的Akka,Flink基于Akka主要封装了两种接口,1:RpcServer,2:RpcEndpoint,它们分别封装的是AkkaSystem和AkkaActor。Akka所有的角色都是通过AkkaSystem这个顶级父类创建,各角色之间本身属性不会暴露出去而是通过ActorRef引用发送消息,所以Flink在最开始创建的RpcServer而在后面提及的包括像RM,JobManager,TaskExecutor都继承了封装了AkkaActor的RpcEndpoint(在构造函数中由RpcServer创建)

//初始化RpcEndpoint时,会创建AkkaActor
	protected RpcEndpoint(final RpcService rpcService, final String endpointId) {
		this.rpcService = checkNotNull(rpcService, "rpcService");
		this.endpointId = checkNotNull(endpointId, "endpointId");
 
		//统一由FLinkClusterEntrypoint的ActorSystem创建的子actor,里面封装了ActorRef
		this.rpcServer = rpcService.startServer(this);
 
		this.mainThreadExecutor = new MainThreadExecutor(rpcServer, this::validateRunsInMainThread);
	}
 
public <C extends RpcEndpoint & RpcGateway> RpcServer startServer(C rpcEndpoint) {
		checkNotNull(rpcEndpoint, "rpc endpoint");
 
		//注册Actor并获取ActorRef
		final SupervisorActor.ActorRegistration actorRegistration = registerAkkaRpcActor(rpcEndpoint);
		final ActorRef actorRef = actorRegistration.getActorRef();
..
}

RpcEndpoint本身有自带的生命周期接口,不同的角色实现可能不一样,当然不同的RpcEndpoint角色也封装了不同的消息类容,比如TaskExecutor有offerSlotsToJobManager的消息,JobManager有startScheduling的消息,均是通过ActorRef调用消息tell接口。
 

//启动RpcEndpoint
public final void start() {
		rpcServer.start();
	}
 
//触发启动逻辑
protected void onStart() throws Exception {}
 
//停止RpcEndpoint
protected final void stop() {
		rpcServer.stop();
	}
 
//触发停止逻辑
protected CompletableFuture<Void> onStop() {
		return CompletableFuture.completedFuture(null);
}

这里再说下HAService这个服务组件

public static HighAvailabilityServices createHighAvailabilityServices(
		Configuration configuration,
		Executor executor,
		AddressResolution addressResolution) throws Exception {
 
		HighAvailabilityMode highAvailabilityMode = HighAvailabilityMode.fromConfig(configuration);
 
		switch (highAvailabilityMode) {
			
...
			case ZOOKEEPER:
				BlobStoreService blobStoreService = BlobUtils.createBlobStoreFromConfig(configuration);
 
            //使用zk作为主备master元数据的同步保存点
 
				return new ZooKeeperHaServices(
					ZooKeeperUtils.startCuratorFramework(configuration),
					executor,
					configuration,
					blobStoreService);
 
			...
	}

 Flink借助Curator交互的Zookeeper,并开启监听该目录数据变化的线程,采用的是leaderLatch选主策略,也就是在zk指定目录下创建临时有序节点(LeaderContender),节点号最小的设置为leader,若该节点挂了会重新选举

public static CuratorFramework startCuratorFramework(Configuration configuration) {
		Preconditions.checkNotNull(configuration, "configuration");
		....
 
		CuratorFramework cf = CuratorFrameworkFactory.builder()
				.connectString(zkQuorum)
				.sessionTimeoutMs(sessionTimeout)
				.connectionTimeoutMs(connectionTimeout)
				.retryPolicy(new ExponentialBackoffRetry(retryWait, maxRetryAttempts))
				// Curator prepends a '/' manually and throws an Exception if the
				// namespace starts with a '/'.
				.namespace(rootWithNamespace.startsWith("/") ? rootWithNamespace.substring(1) : rootWithNamespace)
				.aclProvider(aclProvider)
				.build();
 
		cf.start();
 
		return cf;
	}

Flink基于Curator主要封装了几个接口,这里主要说下提供选举服务的ZooKeeperLeaderElectionService和参与竞争的LeaderContender

ZooKeeperLeaderElectionService:主要负责选举服务和数据变化的监听功能

LeaderContender:选举的参与者需继承该接口,抢主成功后会回调它的grantLeadership接口

public ZooKeeperLeaderElectionService(CuratorFramework client, String latchPath, String leaderPath) {
		this.client = Preconditions.checkNotNull(client, "CuratorFramework client");
		this.leaderPath = Preconditions.checkNotNull(leaderPath, "leaderPath");
 
		//LeaderLatch选举策略
		leaderLatch = new LeaderLatch(client, latchPath);
		//监控的node节点变换数据
		cache = new NodeCache(client, leaderPath);
 
		issuedLeaderSessionID = null;
		confirmedLeaderSessionID = null;
		confirmedLeaderAddress = null;
		leaderContender = null;
 
		running = false;
	}
 
public void start(LeaderContender contender) throws Exception {
		Preconditions.checkNotNull(contender, "Contender must not be null.");
		Preconditions.checkState(leaderContender == null, "Contender was already set.");
 
		LOG.info("Starting ZooKeeperLeaderElectionService {}.", this);
 
		synchronized (lock) {
 
			client.getUnhandledErrorListenable().addListener(this);
 
			leaderContender = contender;
 
			//向leaderLatch添加自己,监听leaderLatch的选举结果
			leaderLatch.addListener(this);
			//开始选举
			leaderLatch.start();
 
			//监听该node节点
			cache.getListenable().addListener(this);
			cache.start();
 
			client.getConnectionStateListenable().addListener(listener);
 
			running = true;
		}
	}
public interface LeaderContender {
 
	/**
	 * Callback method which is called by the {@link LeaderElectionService} upon selecting this
	 * instance as the new leader. The method is called with the new leader session ID.
	 *
	 * @param leaderSessionID New leader session ID
	 */
	void grantLeadership(UUID leaderSessionID);
...
}

OK,初始化这几个核心服务后回到最开始的代码,开始创建RM和Dispatcher以及Web监控等服务
RM:Flink自身维护的资源管理组件,主要是内部维护了负责slot分发的SlotManager,
Dispatcher:主要是创建JobManager,JobManager会创建负责生成ExecutionGraph和CheckpointCoordinator的SchedulerNG和负责管理TM的Slot的生命周期的SlotPool。

public DispatcherResourceManagerComponent create(
			Configuration configuration,
			Executor ioExecutor,
			RpcService rpcService,
			HighAvailabilityServices highAvailabilityServices,
			BlobServer blobServer,
			HeartbeatServices heartbeatServices,
			MetricRegistry metricRegistry,
			ArchivedExecutionGraphStore archivedExecutionGraphStore,
			MetricQueryServiceRetriever metricQueryServiceRetriever,
			FatalErrorHandler fatalErrorHandler) throws Exception {
 
		LeaderRetrievalService dispatcherLeaderRetrievalService = null;
		LeaderRetrievalService resourceManagerRetrievalService = null;
		WebMonitorEndpoint<?> webMonitorEndpoint = null;
		ResourceManager<?> resourceManager = null;
		DispatcherRunner dispatcherRunner = null;
 
		try {
			//onyarn模式启动zk的LeaderRetriever,监听zk/dispatcher_lock的数据变化并缓存到NodeCache中.
			dispatcherLeaderRetrievalService = highAvailabilityServices.getDispatcherLeaderRetriever();
 
			//同上只是zk目录路径为resource_manager_lock
			resourceManagerRetrievalService = highAvailabilityServices.getResourceManagerLeaderRetriever();
 
			.......
 
			//Flink用来跟Yarn交互的组件,主要负责向Yarn的RM申请及释放Flink的JM,TM资源
			//创建RMPRCEndpoint,创建slotManager(负责slot的申请和释放,RM选主后主要是启动SlotManager,
			//SlotManager自身也会启动taskManager的2个周期检测线程包括检测TM的连接超时,空闲和slot申请的超时情况。
			resourceManager = resourceManagerFactory.createResourceManager(
				configuration,
				ResourceID.generate(),
				rpcService,
				highAvailabilityServices,
				heartbeatServices,
				fatalErrorHandler,
				new ClusterInformation(hostname, blobServer.getPort()),
				webMonitorEndpoint.getRestBaseUrl(),
				metricRegistry,
				hostname,
				ioExecutor);
 
			final HistoryServerArchivist historyServerArchivist = HistoryServerArchivist.createHistoryServerArchivist(configuration, webMonitorEndpoint, ioExecutor);
 
			final PartialDispatcherServices partialDispatcherServices = new PartialDispatcherServices(
				configuration,
				highAvailabilityServices,
				resourceManagerGatewayRetriever,
				blobServer,
				heartbeatServices,
				() -> MetricUtils.instantiateJobManagerMetricGroup(metricRegistry, hostname),
				archivedExecutionGraphStore,
				fatalErrorHandler,
				historyServerArchivist,
				metricRegistry.getMetricQueryServiceGatewayRpcAddress(),
				ioExecutor);
 
			log.debug("Starting Dispatcher.");
			//创建DefaultDispatcherRunner(LeaderContender),开始选举leader
			//主要负责创建JM
			dispatcherRunner = dispatcherRunnerFactory.createDispatcherRunner(
				highAvailabilityServices.getDispatcherLeaderElectionService(),
				fatalErrorHandler,
				new HaServicesJobGraphStoreFactory(highAvailabilityServices),
				ioExecutor,
				rpcService,
				partialDispatcherServices);
 
			log.debug("Starting ResourceManager.");
			//向RMRpcEndpoint(ActorRef)发送启动指令
			//RM的启动通知(onStart)接收到后创建Yarn的RMClient并注册自己建立心跳后根据用户的Yarn配置信息创建RM,NM客户端
			//最后选主后启动SlotManager
			resourceManager.start();
 
			resourceManagerRetrievalService.start(resourceManagerGatewayRetriever);
			dispatcherLeaderRetrievalService.start(dispatcherGatewayRetriever);
 
			......
	}

开始Dispatcher选举,抢主成功后创建MiniDispatcher后开始创建JM

这里可以看到是调用我们之前分析过的ZooKeeperLeaderElectionService服务来启动Dispatcher(LeaderContender)选举

public static DispatcherRunner create(
			LeaderElectionService leaderElectionService,
			FatalErrorHandler fatalErrorHandler,
			DispatcherLeaderProcessFactory dispatcherLeaderProcessFactory) throws Exception {
 
		final DefaultDispatcherRunner dispatcherRunner = new DefaultDispatcherRunner(
			leaderElectionService,
			fatalErrorHandler,
			dispatcherLeaderProcessFactory);
 
		//开始DefaultDispatcherRunner的选举
		return DispatcherRunnerLeaderElectionLifecycleManager.createFor(dispatcherRunner, leaderElectionService);
	}
private DispatcherRunnerLeaderElectionLifecycleManager(T dispatcherRunner, LeaderElectionService leaderElectionService) throws Exception {
		this.dispatcherRunner = dispatcherRunner;
		this.leaderElectionService = leaderElectionService;
 
		//开始选举
		leaderElectionService.start(dispatcherRunner);
	}

回调我们之前说过的grantLeadership接口

这里会创建MiniDispatcher,它也继承了RpcEndpoint特性并负责生成JM,并启动MiniDispatcherActor

//Dispatcher抢主成功
	@Override
	public void grantLeadership(UUID leaderSessionID) {
		runActionIfRunning(() -> startNewDispatcherLeaderProcess(leaderSessionID));
	}
 
public AbstractDispatcherLeaderProcess.DispatcherGatewayService create(
			DispatcherId fencingToken,
			Collection<JobGraph> recoveredJobs,
			JobGraphWriter jobGraphWriter) {
 
		final Dispatcher dispatcher;
		try {
			//创建MiniDispatcherRpcEndpoint(Actor)
			dispatcher = dispatcherFactory.createDispatcher(
				rpcService,
				fencingToken,
				recoveredJobs,
				(dispatcherGateway, scheduledExecutor, errorHandler) -> new NoOpDispatcherBootstrap(),
				PartialDispatcherServicesWithJobGraphStore.from(partialDispatcherServices, jobGraphWriter));
		} catch (Exception e) {
			throw new FlinkRuntimeException("Could not create the Dispatcher rpc endpoint.", e);
		}
 
		//启动dispatcherRpcEndpoint
		dispatcher.start();
 
		return DefaultDispatcherGatewayService.from(dispatcher);
	}

之前也说过了所有的RpcEndpoint都实现了一套生命周期

这里开始启动该RpcEndpoint ,Start->OnStart

//------------------------------------------------------
	// Lifecycle methods
	//------------------------------------------------------
 
	@Override
	public void onStart() throws Exception {
		try {
			startDispatcherServices();
		} catch (Throwable t) {
			final DispatcherException exception = new DispatcherException(String.format("Could not start the Dispatcher %s", getAddress()), t);
			onFatalError(exception);
			throw exception;
		}
 
		//准备开始创建JobManager
		startRecoveredJobs();
		this.dispatcherBootstrap = this.dispatcherBootstrapFactory.create(
				getSelfGateway(DispatcherGateway.class),
				this.getRpcService().getScheduledExecutor() ,
				this::onFatalError);
	}

创建JM,JM是Flink是最核心的组件之一,JM维护了负责内部的Slot资源调度的SlotPool以及负责维护CheckpointCoordinator和生产ExecutionGraph的SchedulerNG等

CompletableFuture<JobManagerRunner> createJobManagerRunner(JobGraph jobGraph, long initializationTimestamp) {
		//获取clusterEntrypoint的actorSystem用来创建JobManagerRunnerRpcpoint
		final RpcService rpcService = getRpcService();
		//创建JM->生成SchedulerNG->生成ExecutionGraph->生成ExecutionJobVertex
		//->生成ExecutionVertex,IntermediateResult/partiton->生成CheckpointCoordinator
		//->JM选主->启动SchedulerNG分发任务和ck
		return CompletableFuture.supplyAsync(
			() -> {
				try {
					JobManagerRunner runner = jobManagerRunnerFactory.createJobManagerRunner(
						jobGraph,
						configuration,
						rpcService,
						highAvailabilityServices,
						heartbeatServices,
						jobManagerSharedServices,
						new DefaultJobManagerJobMetricGroupFactory(jobManagerMetricGroup),
						fatalErrorHandler,
						initializationTimestamp);
					//开始选举JMLeader
					runner.start();
					return runner;
				} catch (Exception e) {
					throw new CompletionException(new JobInitializationException(jobGraph.getJobID(), "Could not instantiate JobManager.", e));
				}
			},
			ioExecutor); // do not use main thread executor. Otherwise, Dispatcher is blocked on JobManager creation
	}
 
public JobMaster(
			RpcService rpcService,
			JobMasterConfiguration jobMasterConfiguration,
			ResourceID resourceId,
			JobGraph jobGraph,
			HighAvailabilityServices highAvailabilityService,
			SlotPoolFactory slotPoolFactory,
			JobManagerSharedServices jobManagerSharedServices,
			HeartbeatServices heartbeatServices,
			JobManagerJobMetricGroupFactory jobMetricGroupFactory,
			OnCompletionActions jobCompletionActions,
			FatalErrorHandler fatalErrorHandler,
			ClassLoader userCodeLoader,
			SchedulerNGFactory schedulerNGFactory,
			ShuffleMaster<?> shuffleMaster,
			PartitionTrackerFactory partitionTrackerFactory,
			ExecutionDeploymentTracker executionDeploymentTracker,
			ExecutionDeploymentReconciler.Factory executionDeploymentReconcilerFactory,
			long initializationTimestamp) throws Exception {
 
		super(rpcService, AkkaRpcServiceUtils.createRandomName(JOB_MANAGER_NAME), null);
 
		.....
 
		//创建slotPool,主要负责向RM申请和释放slot
		this.slotPool = checkNotNull(slotPoolFactory).createSlotPool(jobGraph.getJobID());
 
		//存放后面创建的TM的Map
		this.registeredTaskManagers = new HashMap<>(4);
		//JM接受各TM信息反馈
		this.partitionTracker = checkNotNull(partitionTrackerFactory)
			.create(resourceID -> {
				Tuple2<TaskManagerLocation, TaskExecutorGateway> taskManagerInfo = registeredTaskManagers.get(resourceID);
				if (taskManagerInfo == null) {
					return Optional.empty();
				}
 
				return Optional.of(taskManagerInfo.f1);
			});
 
		//追踪每个TM任务情况是否pending,反馈属性实例为PendingBackPressureRequest
		this.backPressureStatsTracker = checkNotNull(jobManagerSharedServices.getBackPressureStatsTracker());
 
		//NettyShuffleMaster,负责创建Producer的InetSocketAddress连接信息
		this.shuffleMaster = checkNotNull(shuffleMaster);
 
		//监测指标组
		this.jobManagerJobMetricGroup = jobMetricGroupFactory.create(jobGraph);
		//scheduler会创建ExecutionGraph
		this.schedulerNG = createScheduler(executionDeploymentTracker, jobManagerJobMetricGroup);
		//Job状态监听器,在启动Scheduler时创建
		this.jobStatusListener = null;
 
		//建立RM连接
		this.resourceManagerConnection = null;
		this.establishedResourceManagerConnection = null;
 
		//建立TM和RM心跳
		this.accumulators = new HashMap<>();
		this.taskManagerHeartbeatManager = NoOpHeartbeatManager.getInstance();
		this.resourceManagerHeartbeatManager = NoOpHeartbeatManager.getInstance();
	}

这里主要说下Scheduler

Scheduler是JM最核心的组件之一,它会生成ExecutionGraph并在JM选主后触发ExecutionGraph的Checkpoint调度和Slot的申请以及部署TM

public SchedulerBase(
		final Logger log,
		final JobGraph jobGraph,
		final BackPressureStatsTracker backPressureStatsTracker,
		final Executor ioExecutor,
		final Configuration jobMasterConfiguration,
		final SlotProvider slotProvider,
		final ScheduledExecutorService futureExecutor,
		final ClassLoader userCodeLoader,
		final CheckpointRecoveryFactory checkpointRecoveryFactory,
		final Time rpcTimeout,
		final RestartStrategyFactory restartStrategyFactory,
		final BlobWriter blobWriter,
		final JobManagerJobMetricGroup jobManagerJobMetricGroup,
		final Time slotRequestTimeout,
		final ShuffleMaster<?> shuffleMaster,
		final JobMasterPartitionTracker partitionTracker,
		final ExecutionVertexVersioner executionVertexVersioner,
		final ExecutionDeploymentTracker executionDeploymentTracker,
		final boolean legacyScheduling,
		long initializationTimestamp) throws Exception {
 
		this.log = checkNotNull(log);
		this.jobGraph = checkNotNull(jobGraph);
		this.backPressureStatsTracker = checkNotNull(backPressureStatsTracker);
		this.ioExecutor = checkNotNull(ioExecutor);
		this.jobMasterConfiguration = checkNotNull(jobMasterConfiguration);
		this.slotProvider = checkNotNull(slotProvider);
		this.futureExecutor = checkNotNull(futureExecutor);
		this.userCodeLoader = checkNotNull(userCodeLoader);
		this.checkpointRecoveryFactory = checkNotNull(checkpointRecoveryFactory);
		this.rpcTimeout = checkNotNull(rpcTimeout);
 
		final RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration =
			jobGraph.getSerializedExecutionConfig()
				.deserializeValue(userCodeLoader)
				.getRestartStrategy();
 
		//重试机制,有FixedDelayRestartStrategy,FailureRateRestartStrategy等策略
		this.restartStrategy = RestartStrategyResolving.resolve(restartStrategyConfiguration,
			restartStrategyFactory,
			jobGraph.isCheckpointingEnabled());
 
		if (legacyScheduling) {
			log.info("Using restart strategy {} for {} ({}).", this.restartStrategy, jobGraph.getName(), jobGraph.getJobID());
		}
 
		//大文件BlobWriter服务
		this.blobWriter = checkNotNull(blobWriter);
		//JM检测指标组
		this.jobManagerJobMetricGroup = checkNotNull(jobManagerJobMetricGroup);
		//申请Slot超时时间,启动Scheduler后会去申请Slot
		this.slotRequestTimeout = checkNotNull(slotRequestTimeout);
		//部署和取消都会修改该executionVertexVersioner
		this.executionVertexVersioner = checkNotNull(executionVertexVersioner);
		//是否延迟调度
		this.legacyScheduling = legacyScheduling;
 
		//创建executionGraph
		this.executionGraph = createAndRestoreExecutionGraph(jobManagerJobMetricGroup, checkNotNull(shuffleMaster), checkNotNull(partitionTracker), checkNotNull(executionDeploymentTracker), initializationTimestamp);
 
		....
	}

主要创建了ExecutionGraph,StateBackend,CheckpointCoordinator等

public static ExecutionGraph buildGraph(
		@Nullable ExecutionGraph prior,
		JobGraph jobGraph,
		Configuration jobManagerConfig,
		ScheduledExecutorService futureExecutor,
		Executor ioExecutor,
		SlotProvider slotProvider,
		ClassLoader classLoader,
		CheckpointRecoveryFactory recoveryFactory,
		Time rpcTimeout,
		RestartStrategy restartStrategy,
		MetricGroup metrics,
		BlobWriter blobWriter,
		Time allocationTimeout,
		Logger log,
		ShuffleMaster<?> shuffleMaster,
		JobMasterPartitionTracker partitionTracker,
		FailoverStrategy.Factory failoverStrategyFactory,
		ExecutionDeploymentListener executionDeploymentListener,
		ExecutionStateUpdateListener executionStateUpdateListener,
		long initializationTimestamp) throws JobExecutionException, JobException {
 
		.....
 
		// create a new execution graph, if none exists so far
		//生成ExecutionGraph,主要负责维护ExecutionJobVertex和负责checkpoint的coordinator
		final ExecutionGraph executionGraph;
		try {
			executionGraph = (prior != null) ? prior :
				new ExecutionGraph(
					jobInformation,
					futureExecutor,
					ioExecutor,
					rpcTimeout,
					restartStrategy,
					maxPriorAttemptsHistoryLength,
					failoverStrategyFactory,
					slotProvider,
					classLoader,
					blobWriter,
					allocationTimeout,
					partitionReleaseStrategyFactory,
					shuffleMaster,
					partitionTracker,
					jobGraph.getScheduleMode(),
					executionDeploymentListener,
					executionStateUpdateListener,
					initializationTimestamp);
		} catch (IOException e) {
			throw new JobException("Could not create the ExecutionGraph.", e);
		}
 
		.....
 
		//创建ExecutionJobVertex,创建ExecutionGraph的核心方法
		executionGraph.attachJobGraph(sortedTopology);
 
		.....
 
			final StateBackend rootBackend;
			try {
				//创建StateBackend,根据用户配置生成
				//包括基于Flink自身JVM heap的MemoryStateBackend
				//外部的状态存储RocksDBStateBackend,支持hdfs等的FsStateBackend
				rootBackend = StateBackendLoader.fromApplicationOrConfigOrDefault(
						applicationConfiguredBackend, jobManagerConfig, classLoader, log);
			}
			catch (IllegalConfigurationException | IOException | DynamicCodeLoadingException e) {
				throw new JobExecutionException(jobId, "Could not instantiate configured state backend", e);
			}
 
			....
 
			//如果true,生成CheckpointCoordinator
			//但目前并不会真正执行,而是等到后面JM的SchedulerNG初始化的时候通知coordinator状态改变为running后才执行
			executionGraph.enableCheckpointing(
				chkConfig,
				triggerVertices,
				ackVertices,
				confirmVertices,
				hooks,
				checkpointIdCounter,
				completedCheckpoints,
				rootBackend,
				checkpointStatsTracker);
		}
 
		// create all the metrics for the Execution Graph
 
        //注册检测指标
		metrics.gauge(RestartTimeGauge.METRIC_NAME, new RestartTimeGauge(executionGraph));
		metrics.gauge(DownTimeGauge.METRIC_NAME, new DownTimeGauge(executionGraph));
		metrics.gauge(UpTimeGauge.METRIC_NAME, new UpTimeGauge(executionGraph));
 
		executionGraph.getFailoverStrategy().registerMetrics(metrics);
 
		return executionGraph;
	}
 
public void enableCheckpointing(
			CheckpointCoordinatorConfiguration chkConfig,
			List<ExecutionJobVertex> verticesToTrigger,
			List<ExecutionJobVertex> verticesToWaitFor,
			List<ExecutionJobVertex> verticesToCommitTo,
			List<MasterTriggerRestoreHook<?>> masterHooks,
			CheckpointIDCounter checkpointIDCounter,
			CompletedCheckpointStore checkpointStore,
			StateBackend checkpointStateBackend,
			CheckpointStatsTracker statsTracker) {
 
		....
 
		//调用checkpoint的ExecutorService单线程池,用作ExecutionGraph周期性触发checkpoint
		checkpointCoordinatorTimer = Executors.newSingleThreadScheduledExecutor(
			new DispatcherThreadFactory(
				Thread.currentThread().getThreadGroup(), "Checkpoint Timer"));
 
		//创建CheckpointCoordinator
		checkpointCoordinator = new CheckpointCoordinator(
			jobInformation.getJobId(),
			chkConfig,
			tasksToTrigger,
			tasksToWaitFor,
			tasksToCommitTo,
			operatorCoordinators,
			checkpointIDCounter,
			checkpointStore,
			checkpointStateBackend,
			ioExecutor,
			new CheckpointsCleaner(),
			new ScheduledExecutorServiceAdapter(checkpointCoordinatorTimer),
			SharedStateRegistry.DEFAULT_FACTORY,
			failureManager);
 
		...
 
		if (chkConfig.getCheckpointInterval() != Long.MAX_VALUE) {
			// the periodic checkpoint scheduler is activated and deactivated as a result of
			// job status changes (running -> on, all other states -> off)
 
			// 注册Job状态监听器,在SchedulerNG初始化的时候 会通知coordinator改变为running并真正触发checkpoint
			registerJobStatusListener(checkpointCoordinator.createActivatorDeactivator());
		}
 
		this.stateBackendName = checkpointStateBackend.getClass().getSimpleName();
	}

这里我们重点说下构建ExecutionGraph过程:

1:主要生成了跟上游Producer(JobVertex)个数对应的ExecutionJobVertex

2:(核心)ExecutionJobVertex开始构建以分区细粒度的ExecutionVertex(Flink构建包含JobVertex之前都是以Operator的粗粒度执行图,下一章节会分析到最小单位的SubResultPartition等粒度的数据结构)

public void attachJobGraph(List<JobVertex> topologiallySorted) throws JobException {
 
		....
        // 遍历JobVertex,循环创建ExecutionJobVertex
 
		for (JobVertex jobVertex : topologiallySorted) {
            // 如果JobVertex需要传入数据的JobEdge集合是空的并且该JobVertex不可停止
			if (jobVertex.isInputVertex() && !jobVertex.isStoppable()) {
                // 是否所有源Task都是可停止的
				this.isStoppable = false;
			}
 
			// create the execution job vertex and attach it to the graph
			// 创建个数对应JobVertex的ExecutionVertex,并将其附加到图上
			ExecutionJobVertex ejv = new ExecutionJobVertex(
					this,
					jobVertex,
					1,
					maxPriorAttemptsHistoryLength,
					rpcTimeout,
					globalModVersion,
					createTimestamp);
             // 将创建的ExecutionJobVertex与前置的IntermediateResult连接起来
			ejv.connectToPredecessors(this.intermediateResults);
             // 构建Map<JobVertexID,ExecutionJobVertex>对象
			ExecutionJobVertex previousTask = this.tasks.putIfAbsent(jobVertex.getID(), ejv);
			if (previousTask != null) {
				throw new JobException(String.format("Encountered two job vertices with ID %s : previous=[%s] / new=[%s]",
					jobVertex.getID(), ejv, previousTask));
			}
             // 得到ExecutionJobVertex产生的IntermediateResult集合,并循环该集合
			for (IntermediateResult res : ejv.getProducedDataSets()) {
                // 所有中间结果都是该图的一部分,构建Map<IntermediateDataSetID, IntermediateResult>类型对象
				IntermediateResult previousDataSet = this.intermediateResults.putIfAbsent(res.getId(), res);
				if (previousDataSet != null) {
					throw new JobException(String.format("Encountered two intermediate data set with ID %s : previous=[%s] / new=[%s]",
						res.getId(), res, previousDataSet));
				}
			}
 
			//按顺序保存executionJobVertex用于后面按顺序加入checkpoint的OperatorCoordinators队列中
			this.verticesInCreationOrder.add(ejv);
            // 记录当前执行图中的ExecutionVertex的综述,等于ExecutionJobVertex总并行度
			this.numVerticesTotal += ejv.getParallelism();
			newExecJobVertices.add(ejv);
		}
 
	...
	}

ExecutionJobVertex核心组件分析:

     -->ExecutionVertex:对应当前ExecutionJobVertex每个分区,主要负责创建IntermediateResultPartition,ExecutionEdge,Execution

      --> IntermediateResultPartition:主要是存储上游ExecutionVertex以及下游List<List<ExecutionEdge>>

      --> ExecutionEdge:主要维护Source和Target

      --> Execution:主要负责Flink分区级别的细粒度物理执行触发,向TM提交部署任务,触发TM该Execution的Checkpoint
 

ExecutionJobVertex(
			ExecutionGraph graph,
			JobVertex jobVertex,
			int defaultParallelism,
			int maxPriorAttemptsHistoryLength,
			Time timeout,
			long initialGlobalModVersion,
			long createTimestamp) throws JobException {
 
		....
 
        //当前Vertex并行度
		this.parallelism = numTaskVertices;
		this.resourceProfile = ResourceProfile.fromResourceSpec(jobVertex.getMinResources(), MemorySize.ZERO);
 
		this.taskVertices = new ExecutionVertex[numTaskVertices];
 
		this.inputs = new ArrayList<>(jobVertex.getInputs().size());
 
		// take the sharing group
		this.slotSharingGroup = checkNotNull(jobVertex.getSlotSharingGroup());
		this.coLocationGroup = jobVertex.getCoLocationGroup();
 
		// create the intermediate results
		// 之前在客户端创建JobVertex时创建的IntermediateDataSet(每个ChainableOperator(JobVertex)生成一个IntermediateDataSet)
		this.producedDataSets = new IntermediateResult[jobVertex.getNumberOfProducedIntermediateDataSets()];
 
		//可能存在有多个上游
		for (int i = 0; i < jobVertex.getProducedDataSets().size(); i++) {
			final IntermediateDataSet result = jobVertex.getProducedDataSets().get(i);
 
			// 生成对应IntermediateDataSet(JobVertex)个数的IntermediateResult(ExecutionJobVertex)
			this.producedDataSets[i] = new IntermediateResult(
					result.getId(),
					this,
					numTaskVertices,
					result.getResultType());
		}
 
		// create all task vertices
		// 根据并行度创建ExecutionVertex
		// 生成IntermediateResultPartition注册到producedDataSets
		// 生成inputEdges和execution
		for (int i = 0; i < numTaskVertices; i++) {
			ExecutionVertex vertex = new ExecutionVertex(
					this,
					i,
					producedDataSets,
					timeout,
					initialGlobalModVersion,
					createTimestamp,
					maxPriorAttemptsHistoryLength);
 
			this.taskVertices[i] = vertex;
		}
 
		....
	}
 
ExecutionVertex(
			ExecutionJobVertex jobVertex,
			int subTaskIndex,
			IntermediateResult[] producedDataSets,
			Time timeout,
			long initialGlobalModVersion,
			long createTimestamp,
			int maxPriorExecutionHistoryLength) {
 
		this.jobVertex = jobVertex;
		this.subTaskIndex = subTaskIndex;
		this.executionVertexId = new ExecutionVertexID(jobVertex.getJobVertexId(), subTaskIndex);
		this.taskNameWithSubtask = String.format("%s (%d/%d)",
				jobVertex.getJobVertex().getName(), subTaskIndex + 1, jobVertex.getParallelism());
 
		this.resultPartitions = new LinkedHashMap<>(producedDataSets.length, 1);
 
		//可能有多个上游ChainableOperator(ExecutionJobVertex),生成IntermediateResultPartition注册到IntermediateResult中
		for (IntermediateResult result : producedDataSets) {
			IntermediateResultPartition irp = new IntermediateResultPartition(result, this, subTaskIndex);
			result.setPartition(subTaskIndex, irp);
 
			resultPartitions.put(irp.getPartitionId(), irp);
		}
 
		//ExecutionEdge主要维护Source和Target(可能有多个上游和下游)
		this.inputEdges = new ExecutionEdge[jobVertex.getJobVertex().getInputs().size()][];
 
		this.priorExecutions = new EvictingBoundedList<>(maxPriorExecutionHistoryLength);
 
		//创建负责执行物理逻辑的execution
		//主要负责向taskManager部署TaskDeploymentDescriptor,触发checkpoint,异常处理等
		//后面JM选主后,将通过SchedulerNG调用execution向TaskManager提交TaskDescriptor
		this.currentExecution = new Execution(
			getExecutionGraph().getFutureExecutor(),
			this,
			0,
			initialGlobalModVersion,
			createTimestamp,
			timeout);
 
		// create a co-location scheduling hint, if necessary
		CoLocationGroup clg = jobVertex.getCoLocationGroup();
		if (clg != null) {
			this.locationConstraint = clg.getLocationConstraint(subTaskIndex);
		}
		else {
			this.locationConstraint = null;
		}
 
		getExecutionGraph().registerExecution(currentExecution);
 
		this.timeout = timeout;
		this.inputSplits = new ArrayList<>();
	}

       至此,ExecutionGraph构建完毕,看的出Flink在Client任务的提交到ExecutionGraph生成,整个构造过程开始变得复杂,从本地Flink的构建到集群的构建,包括出现了多种外部依赖框架,像申请AMContainer时跟Yarn的RM的交互,不同RpcEndpoin角色之间基于Akka的通信机制,还有基于ZK的HA选举.... 当然在构建过程中还生成了JM分发的Dispatcher,物理调度的Scheduler,资源管理的RM以及其他细粒度的调度监控组件等...但目前来说还Flink还处于一个待部署的就绪过程。

小结

大概总结一下ExecutionGraph的转换流程。在生成完了JobGraph之后,经过提交Job后,在JobManager进行ExecutionGraph的转换。

1、Flink客户端提交JobGraph给JobManager

一个程序的JobGraph真正被提交始于对JobClient的submitJobAndWait()方法的调用,而且JobClient的submitJobAndWait()方法会触发基于Akka的Actor之间的消息通信。JobClient在这其中起到了一个“桥接”的作用,它连接了同步的方法调用和异步的消息通信

在submitJobAndWait()方法中,首先会创建一个JobClientActor的ActorRef,并向其发送一个包含JobGraph实例的SubmitJobAndWith消息。该SubmitJobAndWait消息被JobClientActor接收后,调用trySubmitJob()方法触发真正的提交动作,即通过jobManager.tell()的方式给JobManagerActor发送封装JobGraph的submitJob消息。随后,JobManagerActor会接收到来自JobClientActor的SubmitJob消息,进而触发submitJob()方法。

2、构建ExecutionGraph对象

代码new ExecutionJobVertex()用来将一个个JobVertex封装成ExecutionJobVertex,并依次创建ExecutionVertex、Execution、IntermediateResult、IntermediateResultPartition,作为ExecutionGraph的核心对象。

在ExecutionJobVertex的构造函数中,首先依据对应的JobVertex的并发度生成对应个数的ExecutionVertex。其中,一种ExecutionVertex代表着一个ExecutionJobVertex的并发子任务。然后将原来JobVertex的中间结果IntermediateResultDataSet转化为ExecutionGraph中的IntermediateResult。IntermediateResult.setPartition()建立IntermediateResult和IntermediateResultPartition之间的关系,然后生成Execution,并配置相关资源。

新创建的ExecutionJobVertex会调用ejv.connectToPredecessor()方法,按照不同的分发策略连接上游,其参数为上游生成的IntermediateResult集合。其中,根据JobEdge中两种不同的DistributionPattern属性分别调用connectPoinWise()或者connectAllToAll()方法,创建ExecutionEdge,将ExecutionVertex和上游的IntermediateResultPartition连接起来。

总的来说,ExecutionGraph的转换过程为:将JobGraph按照拓扑排序后得到一个JobVertex集合,遍历该JobVertex集合,即从Source开始,将JobVertex封装成ExecutionJobVertex,并依次创建ExecutionVertex、Execution、IntermediateResult、IntermediateResultPartition。然后通过ejv.connectToPredecessor()方法创建ExecutionEdge,建立当前节点与其上游节点之间的联系,即连接ExecutionVertex和IntermediateResultPartition。

构建好ExecutionGraph后,接下来会基于ExecutionGraph触发作业的调度,申请Task Slot,部署任务到TaskManager执行。