9.Dispatcher 启动流程解析

Dispatcher 启动流程解析(DefaultDispatcherRunner / LeaderProcess)

本文聚焦 Session 集群下 Dispatcher 的启动链路:从 Entrypoint 侧创建 DispatcherRunner,到参与 HA 选举成为 leader,再到 leader process 内部完成作业恢复并拉起真正的 DispatcherDispatcherGateway 对外可用)。

0. 主题:Dispatcher 做什么?外部请求怎么进来?

这篇文章主要回答"Dispatcher 是怎么被启动起来的"。但在读启动流程之前,先把职责边界说清楚:

  • Dispatcher(org.apache.flink.runtime.dispatcher.Dispatcher)是 JobManager 侧的"作业调度/提交中枢":对外提供 DispatcherGateway,接收作业提交/触发执行,并编排 JobMaster 生命周期。
  • 外部用户的入口通常不是直接调用 Dispatcher,而是通过 Web REST(也常被口语化叫 webRestEndpoint):org.apache.flink.runtime.webmonitor.WebMonitorEndpoint 提供 HTTP/REST API(例如上传 jar、提交 jar 运行等)。
  • WebMonitorEndpoint 收到请求后会通过 dispatcherGatewayRetriever 连接到当前 leader 的 DispatcherGateway,再把"提交 jar / 提交作业"等动作转交给 Dispatcher 执行。

1. 核心组件职责(先把边界讲清楚)

  • org.apache.flink.runtime.entrypoint.component.DefaultDispatcherResourceManagerComponentFactory:在集群启动阶段组装并启动 Dispatcher/RM/Web 相关组件,包含"创建 DispatcherRunner 并接入 HA 选举"的关键一步。
  • org.apache.flink.runtime.webmonitor.WebMonitorEndpoint:对外提供 REST API(接收 jar 上传/作业提交等),并通过 gateway retriever 把请求转发到当前 leader 的 DispatcherGateway
  • org.apache.flink.runtime.dispatcher.runner.DispatcherRunnerFactory:工厂抽象,负责把"HA 的 LeaderElection + Dispatcher 所需依赖"拼成可运行的 DispatcherRunner
  • org.apache.flink.runtime.dispatcher.runner.DefaultDispatcherRunnerDispatcherRunner 的默认实现,同时是 LeaderContender;负责对接 LeaderElection,在 grant/revoke leadership 时启动/停止 DispatcherLeaderProcess
  • org.apache.flink.runtime.dispatcher.runner.DispatcherLeaderProcess:一次 leader 任期内的 Dispatcher 生命周期"进程",承载恢复逻辑与 Dispatcher 创建/关闭。
  • org.apache.flink.runtime.dispatcher.runner.AbstractDispatcherLeaderProcess:LeaderProcess 的通用基类,把 "DispatcherGateway 何时可用 / leaderAddress 如何产生 / 终止与 shutdown 如何传递"收敛成一套 Future 与状态机。
  • org.apache.flink.runtime.dispatcher.runner.SessionDispatcherLeaderProcess:Session 模式下的 leader process:启动 JobGraphStore,恢复历史 jobGraphs/dirty jobResults,并创建 Dispatcher。

2. 启动入口:ComponentFactory 创建 DispatcherRunner

DispatcherRunner 的创建发生在 DefaultDispatcherResourceManagerComponentFactory#create 过程中;它把 HA 提供的 DispatcherLeaderElection 注入进去,从而把 Dispatcher 的生命周期绑定到 leader 任期。

路径:<flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DefaultDispatcherResourceManagerComponentFactory.java>

FQCN:org.apache.flink.runtime.entrypoint.component.DefaultDispatcherResourceManagerComponentFactory#create

java 复制代码
dispatcherRunner =
        dispatcherRunnerFactory.createDispatcherRunner(
                highAvailabilityServices.getDispatcherLeaderElection(),
                fatalErrorHandler,
                new HaServicesJobPersistenceComponentFactory(highAvailabilityServices),
                ioExecutor,
                rpcService,
                partialDispatcherServices);
  • highAvailabilityServices.getDispatcherLeaderElection():把 HA 层的选举句柄交给 DispatcherRunner;后续是否真正启动 Dispatcher,取决于是否被授予 leadership。
  • HaServicesJobPersistenceComponentFactory(highAvailabilityServices):把"JobGraph/JobResult 持久化相关组件"的创建能力注入到 leader process(恢复逻辑依赖它)。
  • rpcService/partialDispatcherServices/ioExecutor:为后续创建 Dispatcher RPC Endpoint、异步恢复等提供地基能力。

3. RunnerFactory:构造 LeaderProcessFactory 并创建 DefaultDispatcherRunner

DefaultDispatcherRunnerFactory 的职责是把"如何创建某种 DispatcherLeaderProcess"的策略(Session / Per-Job)封装为 DispatcherLeaderProcessFactory,再交给 DefaultDispatcherRunner 管理。

路径:<flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerFactory.java>

FQCN:org.apache.flink.runtime.dispatcher.runner.DefaultDispatcherRunnerFactory#createDispatcherRunner

java 复制代码
@Override
public DispatcherRunner createDispatcherRunner(
        LeaderElection leaderElection,
        FatalErrorHandler fatalErrorHandler,
        JobPersistenceComponentFactory jobPersistenceComponentFactory,
        Executor ioExecutor,
        RpcService rpcService,
        PartialDispatcherServices partialDispatcherServices)
        throws Exception {

    final DispatcherLeaderProcessFactory dispatcherLeaderProcessFactory =
            dispatcherLeaderProcessFactoryFactory.createFactory(
                    jobPersistenceComponentFactory,
                    ioExecutor,
                    rpcService,
                    partialDispatcherServices,
                    fatalErrorHandler);

    return DefaultDispatcherRunner.create(
            leaderElection, fatalErrorHandler, dispatcherLeaderProcessFactory);
}
  • dispatcherLeaderProcessFactoryFactory.createFactory(...):把"恢复所需的存储(JobGraphStore/JobResultStore)+ Dispatcher 创建所需依赖"封装成一个可反复创建 leader process 的工厂。
  • DefaultDispatcherRunner.create(...):真正把 runner 启动起来(内部会 startLeaderElection)。

4. DefaultDispatcherRunner:leader 任期驱动的"单实例 Dispatcher 管理器"

核心点:DefaultDispatcherRunnerLeaderContender,对 LeaderElection 来说它就是参与选举并接收回调的业务方;一旦 grant leadership,就创建一个新的 DispatcherLeaderProcess 并启动;revoke leadership 就停止当前 leader process。

路径:<flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunner.java>

FQCN:org.apache.flink.runtime.dispatcher.runner.DefaultDispatcherRunner#create

java 复制代码
public static DispatcherRunner create(
        LeaderElection leaderElection,
        FatalErrorHandler fatalErrorHandler,
        DispatcherLeaderProcessFactory dispatcherLeaderProcessFactory)
        throws Exception {
    final DefaultDispatcherRunner dispatcherRunner =
            new DefaultDispatcherRunner(
                    leaderElection, fatalErrorHandler, dispatcherLeaderProcessFactory);
    dispatcherRunner.start();
    return dispatcherRunner;
}

路径:<flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunner.java>

FQCN:org.apache.flink.runtime.dispatcher.runner.DefaultDispatcherRunner#start

java 复制代码
void start() throws Exception {
    leaderElection.startLeaderElection(this);
}
  • startLeaderElection(this):把自己作为 contender 注册给 HA;后续 grant/revoke 将以回调方式进入 runner。

4.1 grantLeadership:创建新的 LeaderProcess,并确保"串行切换"

路径:<flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunner.java>

FQCN:org.apache.flink.runtime.dispatcher.runner.DefaultDispatcherRunner#grantLeadership

java 复制代码
@Override
public void grantLeadership(UUID leaderSessionID) {
    runActionIfRunning(
            () -> {
                startNewDispatcherLeaderProcess(leaderSessionID);
            });
}

路径:<flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunner.java>

FQCN:org.apache.flink.runtime.dispatcher.runner.DefaultDispatcherRunner#startNewDispatcherLeaderProcess

java 复制代码
private void startNewDispatcherLeaderProcess(UUID leaderSessionID) {
    stopDispatcherLeaderProcess();

    dispatcherLeaderProcess = createNewDispatcherLeaderProcess(leaderSessionID);

    final DispatcherLeaderProcess newDispatcherLeaderProcess = dispatcherLeaderProcess;
    FutureUtils.assertNoException(
            previousDispatcherLeaderProcessTerminationFuture.thenRun(
                    newDispatcherLeaderProcess::start));
}

路径:<flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunner.java>

FQCN:org.apache.flink.runtime.dispatcher.runner.DefaultDispatcherRunner#createNewDispatcherLeaderProcess

java 复制代码
private DispatcherLeaderProcess createNewDispatcherLeaderProcess(UUID leaderSessionID) {
    final DispatcherLeaderProcess newDispatcherLeaderProcess =
            dispatcherLeaderProcessFactory.create(leaderSessionID);

    forwardShutDownFuture(newDispatcherLeaderProcess);
    forwardConfirmLeaderSessionFuture(leaderSessionID, newDispatcherLeaderProcess);

    return newDispatcherLeaderProcess;
}
  • stopDispatcherLeaderProcess():先停旧的 leader process,避免出现两个 Dispatcher 同时存活。
  • previousDispatcherLeaderProcessTerminationFuture.thenRun(newDispatcherLeaderProcess::start):强制串行切换,等旧的完全终止后再启动新的。
  • forwardConfirmLeaderSessionFuture(...):把"leaderSessionID 与 leaderAddress"确认回 HA(非常关键,决定外部能否发现 leader)。

4.2 confirmLeadership:leaderAddress 从 LeaderProcess 的 DispatcherGateway 推导出来

路径:<flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunner.java>

FQCN:org.apache.flink.runtime.dispatcher.runner.DefaultDispatcherRunner#forwardConfirmLeaderSessionFuture

java 复制代码
private void forwardConfirmLeaderSessionFuture(
        UUID leaderSessionID, DispatcherLeaderProcess newDispatcherLeaderProcess) {
    FutureUtils.assertNoException(
            newDispatcherLeaderProcess
                    .getLeaderAddressFuture()
                    .thenCompose(
                            leaderAddress ->
                                    leaderElection.confirmLeadershipAsync(
                                            leaderSessionID, leaderAddress)));
}
  • getLeaderAddressFuture():来自 leader process;它只有在 DispatcherGateway 可用之后才能完成(见第 6 节)。
  • leaderElection.confirmLeadershipAsync(leaderSessionID, leaderAddress):把 leader 地址写回 HA 的选举/存储系统,为 retrieval(发现)提供数据源。

4.3 revokeLeadership:停止当前 LeaderProcess

路径:<flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunner.java>

FQCN:org.apache.flink.runtime.dispatcher.runner.DefaultDispatcherRunner#revokeLeadership

java 复制代码
@Override
public void revokeLeadership() {
    runActionIfRunning(
            () -> {
                this.stopDispatcherLeaderProcess();
            });
}

5. SessionDispatcherLeaderProcess:恢复 + 创建 Dispatcher 的"任期内主流程"

Session 模式下,SessionDispatcherLeaderProcessFactory 负责把持久化组件(JobGraphStore/JobResultStore)与 Dispatcher 创建能力塞进 leader process。

路径:<flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactory.java>

FQCN:org.apache.flink.runtime.dispatcher.runner.SessionDispatcherLeaderProcessFactory#create

java 复制代码
@Override
public DispatcherLeaderProcess create(UUID leaderSessionID) {
    return SessionDispatcherLeaderProcess.create(
            leaderSessionID,
            dispatcherGatewayServiceFactory,
            jobPersistenceComponentFactory.createJobGraphStore(),
            jobPersistenceComponentFactory.createJobResultStore(),
            ioExecutor,
            fatalErrorHandler);
}

SessionDispatcherLeaderProcess 继承 AbstractDispatcherLeaderProcess,在 onStart() 中启动恢复与创建 Dispatcher。

路径:<flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcess.java>

FQCN:org.apache.flink.runtime.dispatcher.runner.SessionDispatcherLeaderProcess#onStart

java 复制代码
@Override
protected void onStart() {
    startServices();

    onGoingRecoveryOperation =
            createDispatcherBasedOnRecoveredJobGraphsAndRecoveredDirtyJobResults();
}

路径:<flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcess.java>

FQCN:org.apache.flink.runtime.dispatcher.runner.SessionDispatcherLeaderProcess#createDispatcher

java 复制代码
private void createDispatcher(
        Collection<JobGraph> jobGraphs, Collection<JobResult> recoveredDirtyJobResults) {

    final DispatcherGatewayService dispatcherService =
            dispatcherGatewayServiceFactory.create(
                    DispatcherId.fromUuid(getLeaderSessionId()),
                    jobGraphs,
                    recoveredDirtyJobResults,
                    jobGraphStore,
                    jobResultStore);

    completeDispatcherSetup(dispatcherService);
}
  • jobGraphStore.start(this):启动持久化 jobGraph 的监听/服务,为恢复提供数据源。
  • dispatcherGatewayServiceFactory.create(...):创建真正的 Dispatcher 服务(内部会启动 Dispatcher 的 RPC Endpoint,并产出 DispatcherGateway)。
  • completeDispatcherSetup(dispatcherService):把 DispatcherGateway 写入基类的 dispatcherGatewayFuture,从而间接完成 leaderAddressFuture,并把 shutdown/termination 链路串起来(见第 6 节)。

6. AbstractDispatcherLeaderProcess:把 DispatcherGateway/leaderAddress/termination 串成一条闭环

这个基类的关键设计是:LeaderProcess 对外暴露的 "leaderAddressFuture / shutdownFuture / terminationFuture",都由内部的 DispatcherGatewayService 统一驱动完成,从而把 leader election 的 confirm、外部服务发现等动作延迟到 Dispatcher 真正 ready 之后。

路径:<flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/AbstractDispatcherLeaderProcess.java>

FQCN:org.apache.flink.runtime.dispatcher.runner.AbstractDispatcherLeaderProcess#AbstractDispatcherLeaderProcess

java 复制代码
AbstractDispatcherLeaderProcess(UUID leaderSessionId, FatalErrorHandler fatalErrorHandler) {
    this.leaderSessionId = leaderSessionId;
    this.fatalErrorHandler = fatalErrorHandler;

    this.dispatcherGatewayFuture = new CompletableFuture<>();
    this.leaderAddressFuture = dispatcherGatewayFuture.thenApply(RestfulGateway::getAddress);
    this.terminationFuture = new CompletableFuture<>();
    this.shutDownFuture = new CompletableFuture<>();

    this.state = State.CREATED;
}

路径:<flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/AbstractDispatcherLeaderProcess.java>

FQCN:org.apache.flink.runtime.dispatcher.runner.AbstractDispatcherLeaderProcess#completeDispatcherSetup

java 复制代码
final void completeDispatcherSetup(DispatcherGatewayService dispatcherService) {
    runIfStateIs(State.RUNNING, () -> completeDispatcherSetupInternal(dispatcherService));
}

private void completeDispatcherSetupInternal(
        DispatcherGatewayService createdDispatcherService) {
    dispatcherService = createdDispatcherService;
    dispatcherGatewayFuture.complete(createdDispatcherService.getGateway());
    FutureUtils.forward(createdDispatcherService.getShutDownFuture(), shutDownFuture);
    handleUnexpectedDispatcherServiceTermination(createdDispatcherService);
}
  • leaderAddressFuture = dispatcherGatewayFuture.thenApply(RestfulGateway::getAddress):leader 地址不是"拍脑袋拼出来",而是严格依赖 DispatcherGateway ready 之后的 address。
  • dispatcherGatewayFuture.complete(createdDispatcherService.getGateway()):一旦 DispatcherGateway 可用,DefaultDispatcherRunner#forwardConfirmLeaderSessionFuture 才有机会 confirm leadership。
  • FutureUtils.forward(createdDispatcherService.getShutDownFuture(), shutDownFuture):leader process 的 shutdown 状态由 DispatcherService 的 shutdown 直接驱动。

7. UML:类关系与端到端流程

7.1 类关系(谁创建谁、谁管理谁)

createDispatcherRunner(...)
create(...)
start/confirm/close
implements
implements
create(leaderSessionId)
implements
extends
create(...)
DefaultDispatcherResourceManagerComponentFactory
DefaultDispatcherRunnerFactory
DefaultDispatcherRunner
<<interface>>
LeaderElection
<<interface>>
LeaderContender
<<interface>>
DispatcherRunner
<<interface>>
DispatcherLeaderProcessFactory
<<interface>>
DispatcherLeaderProcess
AbstractDispatcherLeaderProcess
SessionDispatcherLeaderProcess
SessionDispatcherLeaderProcessFactory

7.2 主链路(从创建 runner 到确认 leader)

DefaultDispatcherResourceManagerComponentFactory#create
DispatcherRunnerFactory#createDispatcherRunner
DefaultDispatcherRunner.create → startLeaderElection(this)
LeaderElection.grantLeadership(leaderSessionId) 回调
DefaultDispatcherRunner.startNewDispatcherLeaderProcess
SessionDispatcherLeaderProcess.start → onStart
恢复 JobGraphs/DirtyJobResults
dispatcherGatewayServiceFactory.create → DispatcherGateway ready
AbstractDispatcherLeaderProcess.leaderAddressFuture 完成
LeaderElection.confirmLeadershipAsync(leaderSessionId, leaderAddress)

7.3 一次 grantLeadership 的时序图(把 Future 依赖关系画清楚)

AbstractDispatcherLeaderProcess SessionDispatcherLeaderProcess DispatcherLeaderProcessFactory DefaultDispatcherRunner LeaderElection AbstractDispatcherLeaderProcess SessionDispatcherLeaderProcess DispatcherLeaderProcessFactory DefaultDispatcherRunner LeaderElection grantLeadership(leaderSessionId) stopDispatcherLeaderProcess() create(leaderSessionId) LeaderProcess start() onStart() 恢复 + createDispatcher() completeDispatcherSetup(dispatcherService) leaderAddressFuture 完成 confirmLeadershipAsync(leaderSessionId, leaderAddress)

8. 回到主题:Dispatcher 启动的"关键拐点"在哪里

  • DispatcherRunner 的存在意义:把 Dispatcher 的真实启动绑定到 HA 的 leader 任期;只有 leader 才会创建 Dispatcher(非 leader 不会对外提供 DispatcherGateway)。
  • leaderAddress 的来源:不是配置推导,而是 AbstractDispatcherLeaderProcessDispatcherGateway 就绪后通过 RestfulGateway#getAddress 推导出来,再由 runner confirm 给 HA。
  • SessionLeaderProcess 的核心价值:把作业恢复(JobGraphStore/JobResultStore)与 Dispatcher 创建封装在一个任期内流程中,任期切换时能整体 stop/start。
  • 串行切换的保障:previousDispatcherLeaderProcessTerminationFuture 确保旧进程完全退出后才启动新进程,避免"双 Dispatcher"并存。
相关推荐
狂奔蜗牛飙车1 小时前
大数据赛项(中职组)-三个节点的创建及名字网络配置
大数据·大数据入门·大数据竞赛指南(中职组)·centos三节点创建·主机名及ip地址配置
小王毕业啦1 小时前
1990-2024年 省级-绿色金融指数(+文献)
大数据·人工智能·数据挖掘·数据分析·社科数据·实证分析·经管数据
AI 编程助手GPT1 小时前
GPT-5.5与Claude Opus 4.7编程能力深度对比:2026年4月主流AI编程模型选型指南
大数据·人工智能·gpt·ai·ai编程
小王毕业啦1 小时前
1985.1-2026.1 世界各国经济政策不确定性指数(xlsx)
大数据·人工智能·数据挖掘·数据分析·社科数据·实证数据·经管数据
于先生吖2 小时前
家政派单小程序定制厂家
大数据·小程序
摘星编程2 小时前
AI Agent 觉醒时刻:从单点工具到多Agent协作系统的范式革命
大数据·人工智能·自动化
Francek Chen2 小时前
【大数据存储与管理】NoSQL数据库:05 NoSQL的三大基石
大数据·数据库·分布式·nosql
老王谈企服2 小时前
流程型制造业生产优化,未来将如何被大模型技术重构?2026智造深研:实在Agent驱动端到端生产闭环
大数据·网络·人工智能·ai·重构
大大大大晴天️2 小时前
Flink技术实践——Flink资源扩缩容方案演进
大数据·flink