从提交来一步一步分析,本文源码基于Apache社区 1.8-release 版本
REST提交作业流程:
1.集群启动后 通过 /jars/upload 向集群提交可执行jar文件
2.通过 /jars/:jarid/run 来启动一个job
1.构建并提交JobGraph
我们直接找到WebSubmissionExtension这个类,在StandaloneSession 集群模式下集群初始化DispatcherRestEndpoint的时候会从WebSubmissionExtension里加载所有的Handlers(webSubmissionHandlers)
在WebSubmissionExtension中可以找到 /jars/:jarid/run 对应的Headers是JarRunHeaders,而接受http请求的是jarRunHandler。
Flink的rest服务是基于netty实现的,在jarRunHandler接受http请求后会调用handleRequest()方法来处理请求。
在handleRequest()方法的第一行如下,会从request中构造一个JarHandlerContext对象,而jobId就是JarHandlerContext对象的一个属性。在之后的getJobGraphAsync()传入的第一个参数就是context
在getJobGraphAsync()方法中调用context的toJobGraph()方法获取jobGraph
1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20 21 22 |
protected CompletableFuture<JarRunResponseBody> handleRequest( @Nonnull final HandlerRequest<JarRunRequestBody, JarRunMessageParameters> request, @Nonnull final DispatcherGateway gateway) throws RestHandlerException {
final JarHandlerContext context = JarHandlerContext.fromRequest(request, jarDir, log); ... final CompletableFuture<JobGraph> jobGraphFuture = getJobGraphAsync(context, savepointRestoreSettings, jobName, streamGraphPlan, userLibJars); ... } private CompletableFuture<JobGraph> getJobGraphAsync( JarHandlerContext context, final SavepointRestoreSettings savepointRestoreSettings, final String jobName, final String plan, final List<URL> userLibJars) {
return CompletableFuture.supplyAsync(() -> {
final JobGraph jobGraph = context.toJobGraph(configuration, jobName, plan, userLibJars); jobGraph.setSavepointRestoreSettings(savepointRestoreSettings); return jobGraph; }, executor); } |
内部版本当前判断streamGraphPlan是否存在来执行不同的createJobGraph方法,区别在于是否传入jobId。
社区版调用PackagedProgramUtils的createJobGraph()方法的时候会把JarHandlerContext的jobId属性传过去,随后通过steamPlan(streamGraph)的getJobGraph()方法把jobId传进去,之后调用StreamingJobGraphGenerator.createJobGraph()方法传入this(streamGraph)和jobId,在new jobGraph时传入jobId和jobName。
JobGraph的构造方法判断jobId和jobName是否为空,如果为空新生成一个jobId实例,jobName则使用默认值"(unnamed job)"
JobGraph的构造方法:
1 2 3 4 5 6 7 8 9 10 11 |
public JobGraph(JobID jobId, String jobName) {
this .jobID = jobId == null ? new JobID() : jobId; this .jobName = jobName == null ? "(unnamed job)" : jobName; try {
setExecutionConfig( new ExecutionConfig()); } catch (IOException e) {
// this should never happen, since an empty execution config is always serializable throw new RuntimeException( "bug, empty execution config is not serializable" ); } } |
在拿到jobGraph后进行一些后续处理然后向集群提交job
1 2 3 4 5 |
CompletableFuture<Acknowledge> jobSubmissionFuture = jarUploadFuture.thenCompose(jobGraph -> {
// we have to enable queued scheduling because slots will be allocated lazily jobGraph.setAllowQueuedScheduling( true ); return gateway.submitJob(jobGraph, timeout); }); |
集群在接受jobGraph后,有如下的代码:
1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 |
private CompletableFuture<Acknowledge> internalSubmitJob(JobGraph jobGraph) {
log.info( "Submitting job {} ({})." , jobGraph.getJobID(), jobGraph.getName()); final CompletableFuture<Acknowledge> persistAndRunFuture = waitForTerminatingJobManager(jobGraph.getJobID(), jobGraph, this ::persistAndRunJob) .thenApply(ignored -> Acknowledge.get()); return persistAndRunFuture.handleAsync((acknowledge, throwable) -> {
if (throwable != null ) {
cleanUpJobData(jobGraph.getJobID(), true ); final Throwable strippedThrowable = ExceptionUtils.stripCompletionException(throwable); log.error( "Failed to submit job {}." , jobGraph.getJobID(), strippedThrowable); throw new CompletionException( new JobSubmissionException(jobGraph.getJobID(), "Failed to submit job." , strippedThrowable)); } else {
return acknowledge; } }, getRpcService().getExecutor()); } |
在internalSubmitJob()方法中调用waitForTerminatingJobManager()第一个参数就是jobId,随后在异步执行完成后判断时候有异常,在没有异常即提交成功的情况下,调用cleanUpJobData()清理client在提交过程中的数据,清理的标识也是jobId
接着看waitForTerminatingJobManager()方法
1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20 21 22 23 |
private CompletableFuture<Void> waitForTerminatingJobManager(JobID jobId, JobGraph jobGraph, FunctionWithException<JobGraph, CompletableFuture<Void>, ?> action) {
final CompletableFuture<Void> jobManagerTerminationFuture = getJobTerminationFuture(jobId) .exceptionally((Throwable throwable) -> {
throw new
|