Flink Checkpoint 流程、Barrier 流动与 RocksDB 排障

1. 这份文档看什么

  • 目标:先从 JM / TM / task 三层推导一次 checkpoint 到底包含什么,再把 JM 触发barrier 流动Task 执行快照RocksDB 落盘/上传JM finalize 这几段完整串起来。
  • 范围:重点结合 state / timer / channel stateRocksDB State Backend增量 checkpointaligned/unaligned barrier,最后落到 checkpoint timeout 的排查。
  • 阅读顺序:先看流程,再看设计决策,最后看排障。

2. 一次 checkpoint 的主流程

2.1 总览

一次 checkpoint 在源码里,不是一个"存一下状态"的单点动作,而是一条跨越 JobManagerTask网络栈状态后端checkpoint 存储 的链路:

2.1.1 先问:一次 checkpoint 到底包含什么

先看压缩版结论:

  • JM 保存全局元数据,以及 MasterHook / OperatorCoordinator 的控制面状态
  • task 真正保存的是每个 operator subtask 的快照,而不是一份"TM 级业务快照"
  • 一个 subtask 的快照不只有业务 state,还可能包含 timer、raw state、channel state

OperatorSubtaskState 基本就是这个问题的结构化答案:

它包含:

  • managedOperatorState
  • rawOperatorState
  • managedKeyedState
  • rawKeyedState
  • inputChannelState
  • upstreamOutputBufferState
  • resultSubpartitionState

从本质上看:

  1. JM 侧
    • 保存 checkpoint 全局元数据、MasterState、coordinator state
    • MasterState 本质就是 name + bytes + version 的控制面快照: MasterState.java#L25-L38
  2. task 侧
  3. TM 侧
    • 负责承载执行、网络、状态后端和物化过程
    • 不单独形成一份"TM 级业务快照"

三类最容易混的内容可以这样记:

channel state 再单独记一条:

  • 它不是业务 state,而是 unaligned 或 timeout 转 unaligned 时,为了固定边界而额外持久化的网络在途数据

一个最小例子:

  • keyed state:countState[user=42] = 3
  • timer:ts=1000
  • operator state:一条 sink writer 元信息
  • unaligned 时再加两个未消费完的 buffer

那这次 checkpoint 里真正固定下来的,就是:

  • JM 的全局 metadata / master / coordinator state
  • 这个 subtask 的 state、timer,以及必要时的 channel state

一句话总结:

  • JM 保存全局控制面,task 按 operator subtask 保存 state/timer/raw state/channel state;TM 负责承载和物化
  1. CheckpointCoordinator 在 JM 侧创建 PendingCheckpoint,并从这一刻开始计时 timeout。
  2. JM 先触发 OperatorCoordinator 的 checkpoint,再触发 MasterHook,把这些"控制面状态"纳入同一次 checkpoint。
  3. JM 向 checkpoint plan 里的 tasksToTrigger 下发 triggerCheckpoint(...)。这通常是 source,但严格说不是"所有非 source task"。
  4. Source 任务不需要等待上游 barrier;它们会在本地 checkpoint 开始时主动向下游广播 barrier。非 source task 则是在从上游收到 barrier 后进入 checkpoint。
  5. barrier 随数据流向下游传播;多输入算子可能发生 barrier 对齐,必要时会转成 unaligned。
  6. 各个 Task 在本地执行 checkpoint:
    • 先做 prepareSnapshotPreBarrier
    • 再广播 barrier 给下游
    • 再执行同步阶段 snapshot
    • 再执行异步阶段 materialize / upload
  7. Task 异步完成后向 JM 发送 ack 和统计信息。
  8. JM 收齐 task/coordinator/master state 的 ack 后,写 checkpoint metadata,生成 CompletedCheckpoint
  9. 如果任何一步在 timeout 内没有完成,JM 直接把这次 checkpoint 标记为过期失败。

关键入口代码:

2.2 JM 侧:先创建 PendingCheckpoint,再开始计时

CheckpointCoordinator.createPendingCheckpoint(...) 做了两件很关键的事:

  1. 创建 PendingCheckpoint
  2. timer.schedule(new CheckpointCanceller(...), checkpointTimeout, ...) 注册超时任务

对应代码: CheckpointCoordinator.java#L922-L960

这意味着:

  • timeout 不是从某个 Task 真正开始上传 RocksDB 文件时才算。
  • timeout 是从 JM 认为"这次 checkpoint 已经开始"就开始算。
  • 所以 barrier 传播慢对齐慢同步阶段慢异步上传慢JM finalize 慢,都会一起吃掉同一个 timeout 预算。

这一点是后面分析 几百 GB 为什么也会 timeout 的前提。

2.3 JM 侧:先做 OperatorCoordinator 和 MasterHook,再触发 tasks

严格来说,JM 不是"创建完 PendingCheckpoint 就立刻通知所有 task"。

CheckpointCoordinator 里,顺序是:

  1. 先触发所有 OperatorCoordinator 的 checkpoint
  2. 再触发所有 MasterHook
  3. 两边都完成后,才进入 triggerCheckpointRequest(...)
  4. 最后再由 JM 把 trigger 发给各个 task

对应代码:

这样设计的直接含义是:

  • checkpoint 不只包含 operator/task 的运行时状态
  • 还包含一部分"控制面状态"
  • 所以某些 checkpoint 慢,根因也可能不在 task 或 RocksDB,而在 coordinator/master hook
2.3.1 OperatorCoordinator 做什么

OperatorCoordinatorCheckpoints 负责触发各个 OperatorCoordinator 的 checkpoint,并把结果回填到 PendingCheckpoint

核心流程:

  1. checkpointCoordinator(checkpointId, checkpointFuture)
  2. 得到一份 byte[]
  3. 包成 ByteStreamStateHandle
  4. 回写到 PendingCheckpoint.acknowledgeCoordinatorState(...)

对应代码:

它解决的是:

  • source enumerator、协调器、控制面组件不运行在普通 operator 数据路径里
  • 但它们同样要参与一致性恢复
2.3.2 MasterHook 做什么

MasterHook 是另一类运行在 JM 侧的 checkpoint 钩子。它通过 MasterHooks.triggerHook(...) 返回一份 hook state,最终作为 MasterState 写进 checkpoint metadata。

对应代码:

它解决的是:

  • 有些状态天然属于"作业级别"或"JM 侧"
  • 不适合挂在某个 subtask 的 operator state 上

2.4 JM 侧:把 trigger 发给 tasks

CheckpointCoordinator.triggerTasks(...) 会为这次 checkpoint 构造 CheckpointOptions,然后给要触发的 tasks 发送 execution.triggerCheckpoint(...)

对应代码: CheckpointCoordinator.java#L838-L870

这里已经把几项关键策略打包进去了:

  • checkpoint 类型:普通 checkpoint / full checkpoint / savepoint
  • checkpoint 存储位置引用
  • 是否 exactly-once
  • 是否启用 unaligned checkpoint
  • aligned checkpoint timeout

也就是说,后续 barrier 的行为,其实在这里就已经决定了大方向。

这里还要特别澄清一件事:

  • tasksToTrigger 表示"这次 checkpoint 开始时,需要由 JM 主动发起 trigger 的那批 task"

CheckpointPlan 对三类 task 做了区分:

  • getTasksToTrigger():checkpoint 开始时要接收 trigger 消息的 task
  • getTasksToWaitFor():checkpoint 成功前必须等待 ack 的 task
  • getTasksToCommitTo():checkpoint 完成后要收到 notify complete 的 task

对应接口定义: CheckpointPlan.java#L29-L56

在默认计算逻辑里,DefaultCheckpointPlanCalculator 会遍历拓扑,把"没有运行中上游前驱的运行中 task"放进 tasksToTrigger。如果某个 task 仍有运行中的上游,它通常不会被 JM 直接 trigger,而是等上游 barrier 传播过来。

对应代码: DefaultCheckpointPlanCalculator.java#L187-L241

所以更准确的说法是:

  • 在常见流式拓扑里,tasksToTrigger 大多等于 source subtasks
  • 但在"允许部分任务已完成"的场景下,它的定义是"本次需要注入 checkpoint 起点的任务",不是简单按"是不是 source"划分
2.4.1 为什么 tasksToWaitFortasksToCommitTo 还要分开

如果所有 task 都一直处于运行中,看起来这两个集合几乎是一样的:

  • checkpoint 成功前要等所有 task ack
  • checkpoint 成功后也要通知所有 task notifyCheckpointComplete

源码里"所有 task 都在运行"的默认计划,确实就是这么做的:

  • tasksToTrigger = sourceTasks
  • tasksToWaitFor = allTasks
  • tasksToCommitTo = allTasks

对应代码: DefaultCheckpointPlanCalculator.java#L152-L173

但 Flink 允许在某些场景下"部分 task 已经 finished,剩余 task 继续跑 checkpoint"。一旦进入这个场景,waitcommit 的语义就必须拆开。

这里说的不是很少见的内部边角逻辑,而是几类明确存在的运行场景:

  • bounded source 或混合作业
    • 上游 source 先读完,先结束
    • 下游算子还在消费尾部数据、flush 缓冲、完成最终提交
    • 官方文档对这一类场景有专门说明: checkpointing.md#L205-L229
  • stop-with-savepoint --drain
    • task 收到 END_OF_DATA 后会 endData(StopMode.DRAIN),finish operators,并把 end-of-data 往下游传播: StreamTask.java#L660-L717
    • 但 finish 后不会立刻完全退出;如果允许 finished tasks 之后继续 checkpoint,task 还会等待下游把数据处理完,并等待最终 checkpoint 完成: StreamTask.java#L997-L1016
  • finished on restore
    • 恢复后某些 task/subtask 已经是 finished 状态,不再参与正常数据处理
    • 但 checkpoint 元数据里仍要显式记录"它已 finished": FinishedTaskStateProvider.java#L27-L38

另外,这个能力本身就是一个显式配置项:

先看三者各自的职责:

  • tasksToTrigger:谁是这次 checkpoint 的起点,需要由 JM 主动发出 trigger
  • tasksToWaitFor:谁必须 ack,这次 checkpoint 才能成功
  • tasksToCommitTo:这次 checkpoint 完成或中止后,谁还需要收到完成/中止通知

对应接口定义: CheckpointPlan.java#L29-L56

2.4.2 tasksToWaitFor 决定"成功条件"

PendingCheckpoint 构造时,会直接把 checkpointPlan.getTasksToWaitFor() 填进 notYetAcknowledgedTasks,后续是否 isFullyAcknowledged(),就是看这批 task 是否全部 ack 完。

对应代码: PendingCheckpoint.java#L140-L185

也就是说:

  • tasksToWaitFor 是 checkpoint 成功判定集合
  • 不在这个集合里的 task,不会影响这次 checkpoint 能否完成

这很好理解,因为已经 finished 的 task 不会再发新的 task-side ack;如果还把它们算进"必须等待 ack 的集合",checkpoint 将永远无法完成。

2.4.3 tasksToCommitTo 决定"完成后还要通知谁"

checkpoint 一旦完成,JM 会调用:

  • sendAcknowledgeMessages(tasksToCommitTo, ...)

它会对这批仍需要被通知的运行中 task 发送 notifyCheckpointOnComplete(...),同时也会通知 coordinator:

如果 checkpoint 失败或被中止,则会调用:

  • sendAbortedMessages(tasksToCommitTo, ...)

对应代码: CheckpointCoordinator.java#L1598-L1628

所以:

  • tasksToCommitTo 不是"成功判定集合"
  • 它是"checkpoint 结果广播集合"

这还不是一个"只是多发一次通知"的小细节。tasksToCommitTo 背后承接的是一整条 notifyCheckpointComplete/Aborted 回调链:

  1. JM 完成 checkpoint 后,对 tasksToCommitToee.notifyCheckpointOnComplete(...)CheckpointCoordinator.java#L1578-L1596
  2. Task 侧进入 StreamTask.notifyCheckpointCompleteAsync(...),并通过 mailbox 执行通知: StreamTask.java#L1528-L1577
  3. 再进入 SubtaskCheckpointCoordinatorImpl.notifyCheckpointComplete(...)SubtaskCheckpointCoordinatorImpl.java#L434-L503
  4. 最终分发给:
    • operatorChain.notifyCheckpointComplete(...)
    • TaskStateManager.notifyCheckpointComplete(...)
    • fileMergingSnapshotManager.notifyCheckpointComplete(...)

也就是说,tasksToCommitTo 的用途是:

  • 把"这次 checkpoint 已经被 JM 认定成功了"这个事实,传回仍然活着的 task
  • 让这些 task 推进那些必须"等 checkpoint 真正确认成功后"才能执行的逻辑

最典型的用途不是 snapshot,而是 commit / finalize / cleanup:

这两个集合分开后,Flink 才能表达下面这种情况:

  • 某个 task 在本次 checkpoint 时已经 finished,不需要再等它 ack
  • 但仍在运行的其他 task,在 checkpoint 完成后必须收到 notify complete / abort,才能做事务提交、清理本地状态、推进内部状态机
2.4.4 一个小例子

假设有一个简单拓扑:

  • bounded Source -> Map -> exactly-once Sink

现在 Source 已经读完并 finished,MapSink 还在运行:

  • Map 还在处理 source 留下的尾部数据
  • Sink 还在等待最终 checkpoint 完成后提交事务或 committables

那么一次 checkpoint 里,三类集合可能是:

  1. tasksToTrigger
    • 不一定再包含已经 finished 的 Source
    • 而是包含当前仍在运行、又没有运行中上游前驱的 task
  2. tasksToWaitFor
    • 只包含仍会真正 ack 的运行中 task,比如 MapSink
  3. tasksToCommitTo
    • 也只包含仍需接收 complete/abort 通知的运行中 task,比如 MapSink

这里 Source 已经 finished:

  • 它不该再被等 ack
  • 也不该再接收新的 checkpoint complete 通知

Sink 仍然需要 tasksToCommitTo 这条通知链。原因不是它"还活着"这么简单,而是它要在 notifyCheckpointComplete(...) 之后才能做真正的外部提交。

如果没有 tasksToCommitTo,只靠 tasksToWaitFor,会有一个问题:

  • JM 知道 checkpoint 成功了
  • 但 Task 侧的 sink/operator/source reader/state backend 并不知道"现在可以把 checkpoint N 当成已确认成功"
  • 那些依赖 notifyCheckpointComplete(...) 的提交、清理、状态推进逻辑就无从发生

但它之前留下的"finished task 状态"仍然要被纳入 checkpoint 元数据,这就是 finishedTasksfulfillFinishedTaskStatus(...) 要做的事情。

对应代码:

2.4.5 为什么不能只保留一个集合

如果只保留一个"参与 checkpoint 的 task 集合",会出现两类问题:

  1. 语义不清
    • 你没法表达"某 task 不需要再 ack,但这次 checkpoint 仍要考虑它已经 finished 的事实"
  2. 生命周期冲突
    • "谁决定 checkpoint 成功" 和 "谁要接收 checkpoint 完成通知" 其实是两个不同阶段的问题

从时序上看更清楚:

  1. tasksToTrigger
    • 解决"从哪里开始注入这次 checkpoint"
  2. tasksToWaitFor
    • 解决"谁 ack 完,这次 checkpoint 才算成功"
  3. tasksToCommitTo
    • 解决"谁在 checkpoint 成功/失败后还需要收到通知并继续推进运行时逻辑"

所以三个集合不是为了"多设计一层抽象",而是为了把 checkpoint 的三个阶段明确拆开:

  • 起点注入
  • 成功判定
  • 完成通知

2.5 Source 怎样注入 barrier

这里很容易有一个误解:好像 barrier 永远都是"从上游漂下来"。其实从源码看,source task 本身就是 barrier 的注入点。

普通 source 或无上游输入的 task,在收到 JM 的 triggerCheckpointAsync(...) 后,会直接进入 mailbox 执行 triggerCheckpointAsyncInMailbox(...)。这条路径里有一句很关键的注释:

  • No alignment if we inject a checkpoint (如果我们是注入 checkpoint 的源头,就不需要进行对齐)

为什么不需要对齐?

  • 对齐(Alignment)是针对下游多输入通道的算子而言的。下游算子必须等待所有输入通道的 barrier 都到达,才能保证状态快照的一致性。
  • 而 Source 没有上游 Flink 任务,不从 Flink 网络栈接收数据,它是读取外部系统(如 Kafka)的源头。
  • 因此,当 Source 收到 JM 的触发指令时,它只需立刻 记录自己的当前读取位置(比如 Kafka Offset),并把 Barrier 凭空创造出来并广播给下游。整个过程不存在"等待其他通道 barrier"的动作,也就是"No alignment"。

对应代码: StreamTask.java#L1312-L1335

随后它会:

  1. 初始化本地 checkpoint 输入状态
  2. 进入 performCheckpoint(...)
  3. SubtaskCheckpointCoordinatorImpl.checkpointState(...) 里构造 CheckpointBarrier
  4. operatorChain.broadcastEvent(checkpointBarrier, ...) 把 barrier 广播给下游

对应代码:

所以可以把"barrier 流动"的起点理解为:

  • JM 决定 checkpoint 开始
  • Source task 把 barrier 插入数据流
  • 下游 task 再依据 barrier 建立一致性边界
2.5.1 ExternallyInducedSource 为什么更特殊

ExternallyInducedSource 不是收到 JM RPC 就立刻 checkpoint,而是要和外部系统自己的切分点配合。

先给一个简单场景:

  • 假设某个外部日志系统会周期性产出一个"切分标记"或"epoch 边界"
  • 这个标记之前的数据属于上一段,这个标记之后的数据属于下一段
  • Flink 如果在两个标记中间随便截一刀做 checkpoint,恢复时虽然可能还能从 offset 继续读,但这个 checkpoint 边界并不是外部系统自己认可的边界

这时 externally induced source 想解决的问题就是:

  • JM 可以先说"我要做 checkpoint 42"
  • 但 source 不立刻切
  • 它要等外部系统真正出现那个合法的"切分标记"
  • 然后再在这个点触发本地 checkpoint,并把 barrier 注入下游

所以它不是普通业务常用能力,而更像一种"连接器和外部系统做一致性对齐"的特殊机制。

老 Source API 的 SourceStreamTask 会给 source 注册一个 CheckpointTrigger,由 source 在合适的时刻主动回调,再转回 SourceStreamTask.super.triggerCheckpointAsync(...)

FLIP-27 Source 的 SourceOperatorStreamTask 则做了一个双向握手:

  1. JM 的 checkpoint RPC 先到,先把 metadata 和 options 暂存
  2. 外部 source 真正发出"现在可以切分"的信号后,再触发本地 checkpoint
  3. 如果顺序反过来,也会先记住 checkpoint id,等 RPC 到来后再继续

对应代码:

这就是源码里那句注释的背景:

  • This is to ensure the tasks are checkpointed after the OperatorCoordinators in case ExternallyInducedSource is used.

对应位置: CheckpointCoordinator.java#L720-L723

也就是说,Flink 要先把 coordinator/control-plane 这部分状态固定住,再允许 externally induced source 把 barrier 真正注入数据流,避免边界顺序错乱。

2.6 Task 侧:真正执行 checkpoint

Task 侧入口是 StreamTask.triggerCheckpointOnBarrier(...) -> performCheckpoint(...) -> SubtaskCheckpointCoordinatorImpl.checkpointState(...)

这段链路里,源码注释已经把顺序写得比较清楚:

  1. 记录 checkpoint id,检查是否已经被 abort。
  2. prepareSnapshotPreBarrier(),给算子一个"barrier 前的最小准备窗口"。
  3. 向下游广播 checkpoint barrier。
  4. 如果需要,启动 alignment timeout 计时器。
    • 如果在超时时间内还没等到所有输入通道的 barrier,对齐会超时,并从 aligned checkpoint 转成 unaligned checkpoint,不再继续等齐,而是直接开始把当时网络里尚未消费完的数据也纳入 checkpoint: SingleCheckpointBarrierHandler.java#L310-L333
  5. 如果需要 channel state,准备把 in-flight buffers 也纳入 checkpoint。
    • 这里的 channel state 不是算子自己的 keyed state / operator state,而是"网络通道上的状态",也就是 input channel 和 result subpartition 里那些还在路上、还没被完全消费或确认发送完成的数据页。ChannelStateWriter 最终写出的也是 InputChannelStateHandleResultSubpartitionStateHandleChannelStateWriter.java#L34-L69
    • channel state 不是 RocksDB 里的 state,也不是 operator 的 state,它是 网络通道里"还在路上"的数据状态 。纯 aligned 不要 channel state;unaligned 一定要 channel state;aligned + timeout,可转 unaligned 也要提前准备 channel state。
    • 为什么要提前准备? 因为 needsChannelState 的判断是:如果当前是 unaligned 或者 可能超时转成 unalignedaligned checkpoint,都要开启这条路径。也就是说,Flink 会"先挂上保险丝",万一对齐超时了,能立刻把通道状态落盘。
    • 如果没有超时会怎么样? 如果对齐顺利完成,说明一致性边界已经靠 barrier 完美对齐,此时网络通道里不需要额外保存跨越边界的数据。这套预先开启的 channel state 机制虽然走了,但最终生成的 handle 通常是空的或极小,恢复时也基本不依赖它。
  6. 执行同步阶段 snapshot。
  7. 把异步阶段提交到异步线程池执行。

其中同步阶段的核心代码在:

异步阶段提交在:

2.7 Task 侧:异步完成后再 ack 给 JM

异步阶段由 AsyncCheckpointRunnable 执行。它会:

  1. finalize 各个 operator 的 OperatorSnapshotFutures
  2. 统计 asyncDurationMillis
  3. 通过 TaskStateManager.reportTaskStateSnapshots(...) 把状态和 metrics 一起上报给 JM

关键代码:

2.8 JM 侧:收齐 ack 后 finalize

JM 收到 task ack 后,会在 CheckpointCoordinator.receiveAcknowledgeMessage(...) 里累积每个 subtask 的状态。

对应代码: CheckpointCoordinator.java#L1212-L1357

一旦 PendingCheckpoint.isFullyAcknowledged() 成立,JM 会:

  1. completePendingCheckpoint(...)
  2. 再调 PendingCheckpoint.finalizeCheckpoint(...)
  3. 把完整 checkpoint metadata 写入目标存储
  4. 生成 CompletedCheckpoint

对应代码:

注意这里还有一个经常被忽略的点:

  • 即使所有 Task 都已经 ack,checkpoint 也还没算"真正成功"。
  • 只有 JM 把 metadata finalize 成功,才算完成。

3. Barrier 是怎么流动的

3.1 先抓住对象:barrier 不是状态,它是"切分点"

可以把 barrier 理解成一张"拍照线":

  • 数据流像传送带
  • barrier 像插在传送带上的一张标记卡
  • 当一个算子确认所有输入都已经越过同一张标记卡,它就知道"这张卡之前的数据都属于 checkpoint N"

所以 checkpoint 的关键不是"存状态"这么简单,而是"定义一条一致性的切分边界"。

3.1.1 barrier 视角下,window state / namespace 到底影响什么

这一点很容易混淆,因为 window operator 同时牵涉:

  • barrier 边界
  • keyed state
  • window namespace

先说结论:

  • barrier 决定"这次 checkpoint 截到哪里"
  • namespace 决定"边界之前已经产生的状态,落在哪个槽位里"
  • window / namespace 不会改变 barrier 本身的边界定义

也就是说,从 checkpoint 视角看,window 并不是又引入了一条新的边界线。真正的一致性边界仍然只有一条:

  • barrier 到达并被当前 task 确认的那个位置

window operator 做的事情,是把边界之前已经处理过的记录,按 (key, window) 这组坐标写入不同 namespace。

对应源码可以串成一条链:

  1. WindowOperator 处理记录时,会先对命中的 window 调 windowState.setCurrentNamespace(window),然后再 add/get/clearWindowOperator.java#L405-L432
  2. backend 侧 getPartitionedState(...) 命中同一个 state handle 时,通常只是切换 currentNamespace,不是新建一份 state 对象: AbstractKeyedStateBackend.java#L426-L455
  3. 如果不是按 window 分区的 state,而是 globalState() 这类 keyed state,则默认走 VoidNamespaceAbstractStreamOperator.java#L541-L565

所以 barrier 视角下,更准确的理解是:

  • checkpoint 不会去问"这条记录属于哪个 window"
  • checkpoint 只关心"barrier 之前,哪些状态更新已经真正生效"
  • 一旦这些更新已经生效,它们自然就已经落在各自的 namespace 里了

可以用一个最小例子理解:

  • 当前 key 是 userId=42
  • 一条事件 e1 命中窗口 [0,10),并已执行到 windowState.setCurrentNamespace([0,10))add(e1)
  • 下一条事件 e2 命中窗口 [10,20),但 barrier 先到了,e2 还没真正更新 state

那么这次 checkpoint 保存的结果是:

  • (42, [0,10)) 这个 namespace 里已经包含 e1
  • (42, [10,20)) 这个 namespace 里还没有 e2

这里决定结果的,不是"哪个 window 比较早",而是:

  • 在 barrier 建立边界之前,哪次 state 更新已经执行完成

这也解释了一个常见误解:

  • window 很多,不会让 checkpoint 产生很多条边界
  • 它只会让"边界之前的状态"分散落到很多 namespace 槽位

再往下走一步,对齐和非对齐的差别也可以顺手说清:

  • aligned checkpoint
    • barrier 边界靠"所有输入对齐"建立
    • state snapshot 时,直接把当前已经生效的各 namespace 状态拍下来
  • unaligned checkpoint
    • barrier 边界更早建立
    • 但额外把边界前仍在网络里的 in-flight 数据写成 channel state
    • 这些 in-flight 数据还没有变成任何 window namespace 里的 state,所以它们不在 window state 里,而在 channel state 里

所以这里有个很重要的区分:

  • namespace 处理的是"已经进入 operator 并生效的状态"
  • channel state 处理的是"barrier 边界前、但还没来得及进入 operator 状态机的数据"

这两者共同组成恢复语义:

  1. 先恢复各个 keyed/window namespace 里的已生效状态
  2. 如果是 unaligned 或 timeout 转 unaligned,再回放 channel state 里的 in-flight 数据
  3. 这些数据重新进入 operator 后,才会再按各自 window 写入对应 namespace

所以如果从排障角度问:

  • "某条数据为什么没出现在这次 checkpoint 的某个 window state 里?"

最该先问的不是:

  • "它属于哪个 window?"

而是:

  • "在 barrier 固定边界前,它到底已经进入了 operator state,还是还躺在网络 / 反序列化 / input channel 里?"

3.2 从 Source 到下游算子,barrier 怎么穿过网络层

把路径拆开看,一次 barrier 从 source 到下游多输入算子,大致会经过下面几层:

  1. Source task 在本地 checkpoint 开始时构造 CheckpointBarrier
  2. operatorChain.broadcastEvent(...) 把 barrier 写到下游 result partition
  3. 下游 InputGate 从网络层拉到一个 BufferOrEvent
  4. CheckpointedInputGate.pollNext() 发现它是 CheckpointBarrier 事件
  5. CheckpointBarrierHandler.processBarrier(...) 更新对齐状态
  6. StreamTaskNetworkInput.emitNext() 看到这是事件而不是数据 buffer,会尽快返回 mailbox
  7. 当这次 barrier 对当前 task 已经形成可执行的 checkpoint 边界时,CheckpointBarrierHandler.notifyCheckpoint(...) 回调 triggerCheckpointOnBarrier(...)

可以先对应几段核心代码:

这个过程里,barrier 不是普通记录,不会经过 operator 的用户逻辑;它先在网络输入和 barrier handler 这一层被拦截、解释,再决定何时触发 checkpoint。

这一小节先只回答两件事:

  • barrier 为什么能比普通数据更早进入 checkpoint 控制路径
  • 边界固定后,哪些"尚未处理完的数据"会被算进当前 checkpoint

aligned、unaligned、timeout 转换各自的代价和行为,放到 3.43.53.6 再展开。

3.2.1 网络层先把 barrier 当成控制事件

下游 task 从 InputGate 拉到的不是"已经反序列化好的业务记录",而是 BufferOrEvent

  • buffer:普通数据页
  • event:checkpoint barrier、cancel marker、end of partition 等控制事件

CheckpointedInputGate.pollNext() 会先分流 event:

  • CheckpointBarrier -> processBarrier(...)
  • CancelCheckpointMarker -> processCancellationBarrier(...)
  • EventAnnouncement -> processBarrierAnnouncement(...)

对应代码: CheckpointedInputGate.java#L178-L206

所以 barrier 在进入 operator 用户逻辑之前,就已经先被 checkpoint 子系统截获了。

3.2.2 barrier 为什么能尽快被看到

这部分的主线其实只有一句话:Flink 用"网络优先队列 + announcement + urgent mailbox"三层机制,让 barrier 尽快进入 checkpoint 控制路径。

可以按顺序看:

  1. 网络层优先队列
  2. barrier announcement
  3. urgent mailbox
    • CheckpointedInputGate 监听 priority event 的 future,一旦可用,就用 urgent mail 把处理逻辑塞进 mailbox,让 task 线程优先回到 checkpoint 路径: CheckpointedInputGate.java#L125-L142
  4. 公告转真正优先 barrier

这一段最容易误解的地方是:

  • priority event / urgent mail 的"优先"不等于"立刻打断一条正在执行中的 record 处理"
  • Flink 这里是协作式切换,不是抢占式中断
  • 它通常只能在"当前 record 处理完成""当前一次 emitNext() 完成""代码主动 yield / continuation 切回 mailbox"这些安全切换点生效

对应代码:

所以更准确的说法是:

  • barrier 会在下一个可切换点被尽快处理,而不是在任意时刻硬插入当前调用栈中
3.2.3 用一个最小例子理解边界

先看一个单 channel 例子:

  • r1 r2 barrier r3 r4

这个例子只用来帮助理解"checkpoint 边界在什么地方"。要先记住一个前提:

  • 同一个 channel 内,barrier 不会物理越过已经排在它前面的完整 record

因此,这个例子里最重要的结论只有两个:

  1. r1 r2 在逻辑上属于 barrier 之前,r3 r4 属于 barrier 之后
  2. mailbox/priority 的作用是"尽快固定这个边界",不是把 r3 r4 提前算进当前 checkpoint

把这个边界映射到三种模式:

  • aligned checkpoint
    • 靠"等待对齐"建立边界,不靠 channel state
    • 在单 channel 例子里,r3 r4 不属于这次 checkpoint
  • aligned checkpoint 超时转 unaligned
    • 先按 aligned 等齐;超时后切成 unaligned
    • 切换时会把"边界前但尚未处理完"的数据纳入 channel state
  • unaligned checkpoint
    • 不再要求先等齐
    • r2 这类 barrier 前但当前还没消费完的数据,会进入当前 checkpoint 的 in-flight
    • r3 r4 不属于当前 checkpoint 的 in-flight

这里故意只保留最小边界语义,不把"为什么要等齐""什么时候会转成 unaligned""代价为什么会转移"混进来。后面 3.43.53.6 再分别展开。

3.2.4 continuation 场景为什么更容易把人绕晕

上面讨论的是 Flink runtime 自带的 record / barrier 路径。真正容易让人误判的,是像 flink-agents 这类把"一条输入拆成多段执行"的 continuation 场景。

可以参考:

这种场景里要把两件事分开:

  1. checkpoint 保存什么
  2. checkpoint 不保存什么

所以 continuation 算子里还有一条额外判断:

  • 如果 source 的 checkpoint 位置已经越过某条输入,而这条输入的"未完成进度"又没有被写进 operator state,那恢复后它就真的会丢

这也是为什么 continuation 场景里必须区分两条保障线:

  • Flink managed state 负责防丢
  • ActionStateStore 负责防重

后者的细节见:

3.2.5 多输入与 in-flight 数据放到哪里

真正把问题变复杂的,通常不是单 channel,而是多输入和在途数据积压。

多输入算子通常通过 StreamMultipleInputProcessor.processInput() 在多个输入之间轮询读取:

所以多输入场景里,先抓住一个判断就够了:

  • barrier handler 是否已经为这次 checkpoint 建立了合法的一致性边界

对应代码:

一旦进入 unaligned 或 timeout 转 unaligned,边界前但尚未消费完的数据就会被写成 channel state。输入侧最直接的实现是:

  1. 遍历每个 channel 的 record deserializer
  2. 取出 getUnconsumedBuffer()
  3. channelStateWriter.addInputData(...)
  4. 再等待 checkpointedInputGate.getAllBarriersReceivedFuture(checkpointId)

对应代码: StreamTaskNetworkInput.java#L118-L138

对 remote input channel,是否属于当前 checkpoint 的 in-flight 还有更精确的边界判定:

  1. runtime 记住当前 barrier 的 lastBarrierSequenceNumber
  2. 只收集 sequenceNumber < lastBarrierSequenceNumber 的 data buffer
  3. event 自身不算 inflight

对应代码: RemoteInputChannel.java#L777-L820 RemoteInputChannel.java#L834-L849

这一小节先到"边界前未消费完的数据最后写去哪里"为止。为什么 aligned 要等待、unaligned 为什么会变大,后面再分别看。

3.3 Barrier 到达 Task 后,谁处理

核心入口是 CheckpointBarrierHandler.notifyCheckpoint(...)SingleCheckpointBarrierHandler.processBarrier(...)

CheckpointBarrierHandler 会在这里记录几个很重要的统计:

  • checkpointStartDelayNanos
  • alignmentDurationNanos
  • bytesProcessedDuringAlignment

对应代码: CheckpointBarrierHandler.java#L167-L197

这些统计值后来会进入 CheckpointMetrics,也是排查 timeout 最直接的线索:

3.4 Aligned checkpoint:等所有输入都到齐

多输入场景下,最朴素的一致性方案就是:

  1. 第一个 barrier 到达某个输入通道。
  2. 先把这个通道"记账"下来。
  3. 继续等其他输入通道的同一 checkpoint barrier。
  4. 全部到齐后,才触发 checkpoint。

这个逻辑在 SingleCheckpointBarrierHandler 里可以直接看到:

一个最小例子:

  • 有两个上游通道 AB
  • A 的 barrier 先到,B 还在传旧数据
  • 下游算子如果立刻 snapshot,就会把 A 已经跨过边界的数据和 B 还没跨过边界的数据混到一起
  • 所以 aligned checkpoint 必须等 B 的 barrier 也到

优点:

  • 语义直观
  • 不需要把网络里尚未消费的数据额外持久化

代价:

  • 一旦背压严重,慢通道会拖住所有快通道
  • alignment 时间可能非常长

3.5 Unaligned checkpoint:不再等齐,而是把网络里的在途数据也存下来

如果 aligned 太慢,Flink 可以转成 unaligned。配置入口是:

  • execution.checkpointing.unaligned.enabled
  • execution.checkpointing.aligned-checkpoint-timeout

对应代码: CheckpointingOptions.java#L530-L584

切换逻辑可以从两段代码看到:

更早超时、还没看到任何 barrier 时,会转入 AlternatingWaitingForFirstBarrierUnaligned;真正收到 barrier 后:

  1. checkpointBarrier.asUnaligned()
  2. controller.initInputsCheckpoint(unalignedBarrier)
  3. input.checkpointStarted(unalignedBarrier)
  4. controller.triggerGlobalCheckpoint(unalignedBarrier)

对应代码: AlternatingWaitingForFirstBarrierUnaligned.java#L58-L86

这就是 unaligned 的核心语义:

  • 不再强求所有输入先对齐
  • 直接把当时网络栈里还没消费完的 buffer 作为 channel state 一起保存

3.6 为什么 unaligned 不能简单理解成"更快"

它更准确的含义是:

  • barrier 边界更快建立
  • 不等于 整次 checkpoint 更轻

因为它把原来"等一等"的问题,改成了"多写一份 in-flight 数据"的问题。

相关代码:

所以:

  • aligned 慢时,unaligned 往往能显著降低 checkpointStartDelay / alignmentDuration
  • 但如果网络里堆了很多 buffer,bytesPersistedDuringAlignment 和异步上传压力会升高

4. RocksDB checkpoint 是怎么做的

4.1 同步阶段:先把本地 RocksDB 视图固定下来

RocksDB keyed state 的快照入口在:

这里先做了两件事:

  1. writeBatchWrapper.flush()
  2. 进入 SnapshotStrategyRunner.snapshot(...)

SnapshotStrategyRunner 会先执行 syncPrepareResources(checkpointId),再把异步部分包装成 FutureTask

RocksDB 的 syncPrepareResources(...) 在:

它会:

  1. 准备本地 snapshot 目录
  2. 收集 state meta info
  3. Checkpoint.create(db).createCheckpoint(...) 做 RocksDB native checkpoint

这个设计很关键:

  • 同步阶段不直接把所有状态上传到远端
  • 而是先在本地得到一个一致的 RocksDB 文件视图
  • 然后把"真正耗时的远端 materialization"放到异步阶段

这里先把"谁负责什么"说清楚:

  • RocksDB 负责:在本地生成一套可恢复的 checkpoint 目录,也就是 RocksDB 自己的原生文件视图
  • Flink 负责:把这些文件上传到 checkpoint 存储,并额外组织 Flink 自己的 checkpoint 元数据

也就是说:

  • 本地 snapshot 目录里的 .sstMANIFESTCURRENTOPTIONS-* 这些文件格式,是 RocksDB 自己定义的
  • 但"哪些文件属于 shared state、哪些属于 private state、这个 keyed state handle 对应哪个 key-group range、backend meta 是什么",这些是 Flink 自己组织的

这一层分工很重要,因为后面"全量"和"增量"的差别,不在于 RocksDB 生成了两种不同的本地格式,而在于:

  • Flink 扫描这套本地目录后,决定全部上传 还是复用一部分旧 SST handle

上传文件的通用实现是 RocksDBStateUploader

它的做法很直接:

  1. 对每个本地文件创建上传 future
  2. 逐文件读取本地 InputStream
  3. 通过 CheckpointStreamFactory 写到 checkpoint FS
  4. 所有 future 完成后返回 state handles

这个实现透露出两个现实问题:

  • 不是只看字节总量,还要看文件数
  • 对对象存储来说,大量小文件时,元数据和 PUT 开销会很重

对应代码直接能看到,它就是一个"逐文件 copy"的过程:

  • 打开本地 InputStream
  • 通过 CheckpointStreamFactory 创建输出流
  • 把字节写到 checkpoint 存储
  • 返回 HandleAndLocalPath

RocksDBStateUploader.java#L130-L169

这里有个常见误解需要提前澄清:

  • 不是 RocksDB 自己负责把 checkpoint 远程上传
  • 也不是 Flink 重新理解 SST 内容后再按 key/value 重新编码上传

而是:

  • RocksDB 先在本地产出原生 checkpoint 目录
  • Flink 再按"文件"这个粒度把目录里的内容上传

在上传文件之外,Flink 还会生成一份自己的 meta handle。无论全量还是增量,在真正上传 RocksDB 文件前,都会先调用 materializeMetaData(...)

这份 meta handle 不是 RocksDB 的 MANIFEST,而是 Flink 自己的 backend meta,用来描述:

  • state meta info
  • key-group range
  • 以及后面 state handle 里要引用哪些文件

最终,Flink 会把"文件句柄 + meta handle"组装成自己的 KeyedStateHandle。对 RocksDB native snapshot,核心句柄类型是:

  • IncrementalRemoteKeyedStateHandle

它本身就显式区分了:

  • shared state
  • private state
  • meta state

IncrementalRemoteKeyedStateHandle.java#L37-L57

可以先用一个最小例子理解"RocksDB 保存什么、Flink 又额外保存什么"。

假设某个 subtask 当前 RocksDB 本地 checkpoint 目录里只有下面几类文件:

  • 000123.sst
  • 000124.sst
  • MANIFEST-000009
  • CURRENT
  • OPTIONS-000007

这时:

  1. RocksDB 自己负责的内容
    • 上面这几个文件本身,以及它们内部的格式
    • 比如哪个 SST 包含哪段 LSM 层级信息、MANIFEST 如何描述版本切换,这些都属于 RocksDB 自己的语义
  2. Flink 额外负责的内容
    • 这几个文件上传后分别对应哪个 StreamStateHandle
    • 哪些文件属于 sharedState
    • 哪些文件属于 privateState
    • 这份 state handle 覆盖哪个 KeyGroupRange
    • backend 的 metaStateHandle 是什么

如果只停在上面这几个名字,还是容易觉得"为什么要记这么多层"。更自然的理解方式,是做一次演进式分析:Flink 为了恢复 RocksDB,到底最少需要记住什么

可以从一个最朴素的问题开始:

  • 如果机器挂了,Flink 想把这份 RocksDB 状态恢复回来,最少需要知道什么?

第一层,只从"文件恢复"出发,最少要知道:

  1. 有哪些文件
  2. 这些文件去哪里拿

所以第一批必须保存的信息就是:

  • 每个上传后文件对应哪个 StreamStateHandle

否则恢复时连 000123.sstMANIFEST-000009CURRENT 分别在 checkpoint 存储里的哪个对象/文件都不知道。

第二层,只从"增量复用"出发,还必须知道:

  1. 这份文件是这次 checkpoint 新上传的
  2. 还是前面 checkpoint 共享过来的

所以第二批必须保存的信息就是:

  • 哪些文件属于 sharedState
  • 哪些文件属于 privateState

如果不区分这两类,Flink 就没法表达:

  • "000123.sst 这次没重传,直接复用老 checkpoint 的 handle"
  • "MANIFEST / CURRENT / OPTIONS-* 这些是这次新生成并上传的"

第三层,只从"逻辑归属"出发,还必须知道:

  1. 这份 RocksDB handle 在 Flink 语义上属于哪个 keyed operator subtask
  2. 它覆盖哪些 key-group

所以第三批必须保存的信息就是:

  • KeyGroupRange

注意这里记录的不是:

  • 每个 key 在 SST 里的具体位置

而是:

  • 这份 handle 在 Flink 逻辑上负责哪一段 key-group

因为恢复时,Flink真正关心的是:

  • 这份状态应不应该交给当前 subtask
  • rescale 后这份状态要不要导入、裁剪或校验范围

而 key 在某个 SST 的 block offset、页内位置这些事情,交给 RocksDB 自己的索引去解决就够了。

第四层,只从"能不能把文件重新打开成一个可用 backend"出发,还必须知道:

  1. 有哪些 logical state
  2. 它们对应哪些 meta info / serializers / column family

所以第四批必须保存的信息就是:

  • metaStateHandle

这份信息不是 RocksDB 的 MANIFEST,而是 Flink 自己的 backend meta。没有它,Flink 只拿到一堆 SST 文件,也不知道:

  • 有哪些 state 名称
  • 对应什么 state type / serializer
  • 恢复时该如何重新注册 column family 和 state meta info

所以最终,Flink 需要保存的不是"每个 key 到底躺在哪个 SST 的哪里",而是下面这四层:

  1. 文件引用层
    • 每个文件对应哪个 StreamStateHandle
  2. 共享关系层
    • sharedState / privateState
  3. 逻辑归属层
    • KeyGroupRange
  4. 状态语义层
    • metaStateHandle

可以用一个很小的例子理解:

  • 假设某个 subtask 负责 keyGroupRange = [0, 3]
  • 它有两个 logical state:countStatetimerState
  • 本地目录里最终有:
    • 000123.sst
    • 000124.sst
    • MANIFEST-000009
    • CURRENT

这时 Flink 真正需要记的是:

  1. 000123.sst -> handle_A
  2. 000124.sst -> handle_B
  3. MANIFEST-000009 -> handle_C
  4. 其中 handle_A 可能是 sharedStatehandle_B/C 可能是 privateState
  5. 这整份 handle 覆盖 [0,3]
  6. 这份 backend meta 里包含 countStatetimerState 的 meta 信息

而不需要记的是:

  • user:42000123.sst 第几个 block、第几字节

因为那是 RocksDB 自己恢复后去查的。

换句话说:

  • RocksDB 负责"文件内部如何定位 key"
  • Flink 负责"恢复时应该拿哪些文件、这些文件在逻辑上属于谁、以及它们代表哪些 state"

这里再补两个经常会问到的"存储粒度"问题。

问题 1:这个 RocksDB 是 TM 级别共享,还是 task / subtask 级别?

  • RocksDBKeyedStateBackendBuilder 的构造参数就能看出来,它绑定了:
  • 真正 build() 时,也是为这一份 keyed state backend 创建一套自己的 RocksDB 实例、自己的 checkpoint strategy、自己的恢复结果: RocksDBKeyedStateBackendBuilder.java#L336-L431
  • 所以更准确地说:
    • RocksDB 实例是 keyed operator subtask 级别的
    • 不是"整个 TM 上所有 task 共用一个 RocksDB"

同一个 operator subtask 内部,不同 keyed state 也不是各开一个 DB,而是:

  • 共用同一个 RocksDB 实例
  • 但写到各自的 column family

这点在 RocksDBKeyedStateBackend 里直接写了:

问题 2:两个 key 的数据会不会落进同一个 SST?

  • 会,而且很常见。
  • Flink 在 RocksDB key 里会写入 key-group prefix,也就是 key 的前缀里带上 key-group: RocksDBKeyedStateBackendBuilder.java#L359-L363
  • 但 RocksDB 的 SST 是按排序后的 key range 落盘,不是"一 key-group 一个 SST"
  • 所以:
    • 同一个 column family 里,多个 key-group 的数据完全可能落到同一个 SST
    • 同一个 key-group 里,多个 key 的数据当然也可能落到同一个 SST

从本质上看,RocksDB 这里的落盘单位是:

  • 按 column family 分库
  • 在 column family 内按 composite key 的字节序排序
  • 然后由 LSM compaction / flush 形成 SST 文件

所以"一个 SST 里到底有什么"更像是:

  • 某个 column family 某一段 key range 的数据片段

而不是:

  • "只保存一个 key-group"
  • 或者"只保存一个业务 key"

因此要再细分:

  • 不同 logical state:通常通过不同 column family 隔离,所以通常不会混在同一个 SST
  • 同一个 logical state 下的不同 key / 不同 key-group:完全可能混在同一个 SST

这也是为什么 Flink 的 checkpoint handle 需要额外记 KeyGroupRange

  • 不是因为每个 SST 天然只属于一个 key-group
  • 而是因为恢复时要知道"这一份 keyed state handle 在逻辑上覆盖哪些 key-group",必要时还要做裁剪、导入或范围校验: RocksDBIncrementalRestoreOperation.java#L388-L402

换句话说,Flink 不解析 SST 内部结构来"重造一份数据库",而是保存一种更高一层的引用关系:

  • "这个 checkpoint 的 RocksDB 文件集合有哪些"
  • "这些文件里哪些是老 checkpoint 共享过来的,哪些是本次新上传的"
  • "恢复时应该把这些句柄重新还原成一个本地 RocksDB 目录,再让 RocksDB 打开"

到这里先把"异步阶段究竟在产出什么"说清楚。下面再只看一个问题:

  • 这套目录在异步物化时,什么时候会全传,什么时候会复用老 SST

4.3 增量 checkpoint:复用老 SST,只上传新增部分

增量策略在:

核心逻辑:

  1. 扫描本地 checkpoint 目录里的文件
  2. .sst 文件,尝试从 previousSnapshot 查找是否可复用
  3. 可复用则直接引用旧 handle
  4. 不可复用则重新上传
  5. misc 文件照样上传

对应判断在:

这说明"增量 checkpoint"并不是按 key 做逻辑增量,而是按 RocksDB 文件做物理复用。也就是说,增量发生在 Flink 异步 materialization 阶段,不是发生在 RocksDB 本地导出阶段。

从结果句柄看,增量 checkpoint 会把:

  • 可复用或新上传的 .sst 放进 sharedState
  • 其他文件放进 privateState
  • Flink 自己的 backend meta 放进 metaStateHandle

对应组装逻辑: RocksIncrementalSnapshotStrategy.java#L300-L308

所以你可以把"增量 checkpoint 上传"理解成:

  1. Flink 拿到这次本地 checkpoint 目录
  2. 扫描目录
  3. .sst
    • 命中 previousSnapshot 且可复用,就只记录旧 handle
    • 否则重新上传
  4. misc
    • 直接上传
  5. Flink 最后生成一份"shared/private/meta"三段式的 IncrementalRemoteKeyedStateHandle

这里还要补一个判断:为什么它不是"零副作用、零开销,所以最好无脑默认开"

先说结论:

  • 对 RocksDB 大状态作业,增量 checkpoint 通常值得优先考虑,官方文档甚至明确说它应当是"减少 checkpoint 时间时首先考虑的选项之一": large_state_tuning.md#L121-L126
  • 但它不是"纯赚不赔",副作用主要不在 checkpoint 触发代码多几行,而在存储形态、恢复路径和观测口径都发生了变化

可以拆成四点看:

  1. checkpoint 不再是自包含快照,而会依赖以前的 checkpoint

  2. 恢复不一定更快,取决于瓶颈在哪里

    • 官方文档写得很直白:恢复时间可能更长,也可能更短。
      • 如果瓶颈是网络,增量恢复可能更慢,因为要抓取更多 delta / 更多引用链
      • 如果瓶颈是 CPU 或 IOPS,增量恢复可能更快,因为不需要从 canonical key/value 快照重建本地 RocksDB 表: state_backends.md#L233-L240
    • 所以它优化的重点是checkpoint 上传成本,不是保证所有场景下恢复也一定更优
  3. Web UI 上的大小含义会变

    • 一旦启用增量 checkpoint,UI / REST API 里看到的 Checkpointed Data Size 只代表这次 delta 的大小,不再代表完整状态量: CheckpointingOptions.java#L126-L142 state_backends.md#L239-L240
    • 这很容易造成误判:看起来这次只写了几十 MB,不代表完整恢复代价真的只有几十 MB
  4. 收益取决于 SST 复用率,不是打开就天然很小

    • 只有 .sst 能命中 previousSnapshot.getUploaded(fileName)couldReuseStateHandle(...) 时,才会真正复用;否则还是重新上传: RocksIncrementalSnapshotStrategy.java#L421-L428
    • 如果 compaction 很频繁、SST churn 很高、misc 文件很多,增量收益会明显打折

所以更准确的建议是:

  • RocksDB + 大状态 + checkpoint 时间是核心瓶颈:优先打开
  • 小状态或恢复链路比 checkpoint 更敏感:不要因为"看起来没什么额外开销"就想当然,还是要结合恢复时延、SST 复用率和对象存储表现来看

如果把这一节压成一句话,可以记成:

  • 增量 checkpoint 优化的是"这次要上传哪些文件",不是改变 RocksDB 本地快照的生成方式

配置用法也很直接:

  1. 配置文件
yaml 复制代码
execution.checkpointing.incremental: true

对应配置项定义: CheckpointingOptions.java#L122-L142

注意默认值是:

  • false

也就是说,Flink 不会自动替你打开它。

  1. 代码里直接指定 RocksDB backend
java 复制代码
EmbeddedRocksDBStateBackend backend = new EmbeddedRocksDBStateBackend(true);
env.setStateBackend(backend);

对应 RocksDB backend 构造入口: EmbeddedRocksDBStateBackend.java#L205-L224

如果你只想记一句最实用的话,可以记成:

  • 增量 checkpoint 不是"默认一定更好"
  • 但对 RocksDB 大状态场景,它通常是默认推荐开启的选项
  • 它的副作用不在"额外多写一点代码",而在"恢复链路、观测口径和对 SST 复用率的依赖都变了"

4.4 Full checkpoint:当前视图下所有文件都要上传

full checkpoint 的实现是:

这一节只看和 4.3 的真正分界:

  • full 不复用 SST
  • full 会把当前视图下需要的文件全部上传

所以 full checkpoint 的异步物化可以直接理解成:

  1. 先生成 Flink 自己的 meta handle
  2. 扫描本地目录里的所有文件
  3. 全部上传
  4. 组装成一个 IncrementalRemoteKeyedStateHandle

这里容易意外的一点是:

  • 即使是 full checkpoint,Flink 这里仍然用 IncrementalRemoteKeyedStateHandle
  • 只是它的 sharedState 为空,所有上传文件都进 privateFiles

对应代码:

所以在排障时必须先分清:

  • 这次是普通增量 checkpoint
  • 还是 full checkpoint / savepoint

两者对时延的影响可能完全不是一个数量级。

如果把 full 和 incremental 并排压成一句对比:

  • full:当前视图需要的文件全上传,这次 checkpoint 基本自包含
  • incremental:扫描当前视图,但尽量把可复用 SST 变成旧 handle 引用,只上传新增文件

4.5 如果是内存 / Heap backend,为什么也能把 checkpoint 落到 HDFS

前面一直在说 RocksDB。这里单独补这一节,是为了避免把"运行时状态放哪"和"checkpoint 最终写哪"混成一件事。

可以,而且这恰恰说明 state backendcheckpoint storage 是两层不同的职责。

先说现在更准确的概念:

  • "内存 backend" 如果按新版本术语说,通常对应 HashMapStateBackend
  • 它把工作状态 放在 TaskManager 的 JVM heap 里: HashMapStateBackend.java#L40-L48
  • 但 checkpoint 落到哪里,不由它单独决定,而是由配置的 CheckpointStorage 决定

也就是说:

  • state backend 决定"运行时状态放在哪里、怎么组织"
  • checkpoint storage 决定"做 checkpoint 时,快照最终写到哪里"

HashMapStateBackend 自己的注释已经写得很明确:

如果配置的是:

  • FileSystemCheckpointStorage("hdfs://...")

那么即使运行时状态在 JVM heap,checkpoint 也照样会写到 HDFS。这个 storage 的职责本来就是"把 checkpoint state 作为文件写到文件系统",类注释写得很直接: FileSystemCheckpointStorage.java#L50-L68

实现上,heap backend 的快照不是生成 SST 文件,而是:

  1. 遍历 heap 里的 keyed state / priority queue
  2. 通过 CheckpointStreamFactory 打开 checkpoint 输出流
  3. 把 key-group 数据序列化写进去
  4. 最终生成一个 KeyGroupsStateHandle

入口在:

它和 RocksDB 的区别只是:

  • RocksDB:先得到本地数据库文件目录,再逐文件上传
  • Heap backend:直接把内存中的 key-group 数据序列化写进 checkpoint 输出流

最终得到的核心句柄也不同:

可以再用一个最小例子理解:

假设某个 key-group 里只有两条状态:

  • user:1 -> count=3
  • user:2 -> count=7

如果是 HashMapStateBackend + FileSystemCheckpointStorage(hdfs://...)

  1. 运行时,这两条数据在 JVM heap 的 hash map / state table 里
  2. checkpoint 时,HeapSnapshotStrategy 把它们按 key-group 顺序序列化到 CheckpointStreamFactory 打开的输出流
  3. 这个输出流背后如果是 HDFS,就会落到 HDFS 上
  4. 最终 Flink 用一个 KeyGroupsStateHandle 记住:
    • 这份字节流对应哪个文件句柄
    • 哪些 offset 对应哪些 key-group

所以"内存 backend 的 checkpoint 能不能落到 HDFS"这个问题,本质答案是:

  • 因为 checkpoint 落地位置取决于 CheckpointStorage
  • 而不是取决于运行时状态是不是在内存里

4.6 Checkpoint storage 自身为什么也可能成为瓶颈

除了 RocksDB 本地 flush 和上传,checkpoint storage 本身也可能卡住 asyncDuration

过程很简单:

  1. JM 初始化 checkpoint 目录
  2. task 通过 CheckpointStreamFactory 往存储写数据
  3. JM 最后 finalize metadata

入口:

排障时把 storage 瓶颈拆成三类就够了:

  • 吞吐慢:远端文件系统 / 对象存储写得慢
  • 文件太多:目录、创建、PUT、元数据操作过重
  • JM finalize 慢:task 已 ack,但元数据收口还没完成

一个最小例子:

  • 20 GB 状态如果拆成 20 万个小文件,瓶颈可能不是字节量,而是 mkdir/create/open/close 和对象存储 PUT 次数

所以 asyncDuration 高时,不只看 "SST 是否复用",还要看:

  • 写了多少文件
  • 元数据操作是否过重
  • task ack 之后是否还卡在 JM finalize

对应优化方向是 file-merging checkpoint:

它优化的是小文件洪泛,但代价是空间放大。

下面不只回答"现在怎么做",还回答"为什么不是更简单的做法"。

5.1 最简单的设计:所有状态都放内存,每次全量复制

一种最直观的方案是:

  1. 所有 operator state 都在 JVM heap
  2. checkpoint 时停一下业务处理
  3. 把整份状态序列化后写到远端存储

这个方案在状态很小时可行,但一旦状态上到 GB 级、几十 GB、上百 GB,就会出现几个问题:

  • 同步停顿太长
  • JVM heap 压力和 GC 压力过大
  • 同一份数据每次都全量重写,写放大非常严重

所以 Flink 后面必须把"在线读写状态"和"持久化 checkpoint"解耦。

早期经验是:

  • 在线状态高频随机读写,纯远端同步访问通常太慢

所以 Flink 长期主流模型一直是:

  • 本地在线状态
  • 远端异步 checkpoint

近年又重新探索存算分离,不是因为这个判断错了,而是背景变了:

  • 云原生更普遍
  • 超大状态更多
  • 更关注轻量 checkpoint、快速恢复和 rescale

Flink 2.0 已把 disaggregated state 作为正式方向:

现在更准确的结论是:

  • 传统同步状态访问模型下,本地在线状态仍是默认自然选择
  • 超大状态和异步状态访问场景下,带本地 cache 的存算分离后端已成为正式演进方向

典型实现是 ForStStateBackend

  • 远端主存储
  • 本地 cache / buffer
  • 异步 I/O 访问

对应配置与设计可看:

5.3 为什么选 RocksDB / LSM tree

从 checkpoint 角度看,RocksDB 的价值不只是在"能把状态放磁盘",更在于:

  • LSM tree 天然以 SST 文件组织稳定数据
  • native checkpoint 可以快速固定一个一致文件视图
  • 增量 checkpoint 可以按 SST 复用,而不是按 key 逐条扫描

这三点合起来,才让"本地在线状态 + 异步物化 + 增量复用"成为可能。

如果底层不是 LSM tree,而是纯内存哈希表:

  • 你很难低成本拿到一个稳定、可复用、按文件共享的快照形态
  • 最后大概率仍然要回到"大对象序列化 + 全量上传"

5.4 为什么 timeout 从 JM 创建 PendingCheckpoint 就开始算

从实现上看,这个决定很合理。

如果 timeout 只从"某个 Task 开始上传状态"那一刻起算,会有几个漏洞:

  • barrier 长时间传播/对齐不算超时
  • 某个 Task 一直没真正开始本地 snapshot,也不算超时
  • JM 很难定义这次 checkpoint 到底"开始了没有"

而现在的定义很统一:

  • 既然 JM 已经发起了这次全局一致性切分
  • 那就从全局发起时刻开始度量它是否按时完成

这也是为什么排查时不能只盯 RocksDB 上传速度。

5.5 为什么 checkpoint 不一定按你以为的节奏触发

checkpoint 调度不是"每隔 interval 机械触发一次",JM 还会同时看:

  • minPauseBetweenCheckpoints
  • maxConcurrentCheckpoints
  • 当前 pending / queue / cleaning 状态

核心入口:

实际过程是:

  1. trigger request 先进入 decider
  2. decider 决定立刻执行、排队、重调度或拒绝

最容易误判的是 minPauseBetweenCheckpoints

maxConcurrentCheckpoints 则限制同时存在多少个 pending checkpoint;达到上限后,普通 request 不会继续触发: CheckpointRequestDecider.java#L160-L164

最实用的排障结论是:

  • 还没真正 trigger 出去:先查 JM 调度层
  • 已经 trigger 出去,但 task 迟迟没收到:再查 barrier / 背压 / 传播

5.6 为什么 checkpoint 失败了,不一定马上 fail job

checkpoint 失败和 job failover 之间还有一层 CheckpointFailureManager

它做三件事:

  • 给失败分类
  • 更新 stats
  • 决定是否累计到连续失败并最终 fail job

5.6.1 先分"失败发生在哪一层"

实现里先区分:

  • JM failure / pre-flight failure
  • TM failure

判断入口在 CheckpointFailureManager.java#L148-L153

然后再看这次是不是:

  • checkpoint
  • savepoint
  • sync savepoint

这会直接影响是否立刻 fail job:

  • JM 上的 savepoint 失败,通常不计入 failover
  • TM 上的 sync savepoint 失败,会直接 fail job,避免 stop-with-savepoint 卡死

CheckpointFailureManager.java#L164-L195

5.6.2 再分"这类失败算不算连续失败"

并不是每个 failure reason 都会推高连续失败计数。

实现里明确忽略了一大批原因,例如:

  • TOO_MANY_CHECKPOINT_REQUESTS
  • MINIMUM_TIME_BETWEEN_CHECKPOINTS
  • CHECKPOINT_SUBSUMED
    • 含义:这个 checkpoint 还没完成,就已经被一个更晚且允许覆盖它的 checkpoint 顶掉了
    • 典型触发:CheckpointCoordinator.dropSubsumedCheckpoints(...) 会把 checkpointId < 新 checkpointIdcanBeSubsumed() 的 pending checkpoint 统一 abort 成这个原因: CheckpointCoordinator.java#L1638-L1642
    • 排障解释:这通常不是"checkpoint 自己坏了",而是系统在重叠 checkpoint 语义下主动淘汰了一个旧 checkpoint;所以它不应当算进"连续失败"
  • CHECKPOINT_DECLINED_TASK_NOT_READY
    • 含义:checkpoint 请求已经到 task,但这个 task 当前还没到能安全参与 checkpoint 的状态
    • 典型触发:task 不是 running,或者 StreamTask 当前明确选择 decline;比如 Task 对非 running task 会直接回这个原因: Task.java#L1440-L1447StreamTask.declineCheckpoint(...) 也会回同一个 reason: StreamTask.java#L1514-L1520
    • 排障解释:这更像"task 当前状态不适合做 checkpoint",常见于任务启动/切换/恢复中的窗口,而不是 checkpoint 运行到一半真的失败了
  • TASK_FAILURE

CheckpointFailureManager.java#L222-L247

真正会累计连续失败计数的,更接近"这次 checkpoint 自己真的没做成"的那批,比如:

  • IO_EXCEPTION
  • CHECKPOINT_ASYNC_EXCEPTION
  • CHECKPOINT_DECLINED
  • CHECKPOINT_EXPIRED
  • FINALIZE_CHECKPOINT_FAILURE

CheckpointFailureManager.java#L249-L258

这就是为什么现场里经常会出现两个看似矛盾的现象:

  • checkpoint 明明一直失败
  • 但 job 一段时间内并没有马上挂

因为系统不是按"失败次数总数"机械计数,而是按:

  • 失败原因是否可计数
  • checkpoint id 是否应该只算一次
  • 是否超过 tolerableCpFailureNumber

来决定是否最终 fail job。

排障时把两个问题分开就够了:

  • "为什么 checkpoint fail 了"
  • "为什么这次 failure 没有直接把 job 打挂"

前者看 root cause,后者看 CheckpointFailureManager 的分类和阈值策略。

5.7 为什么 complete / abort / subsumed 的顺序看起来会反直觉

重叠 checkpoint 存在时,通知顺序不一定是"先 complete 再 abort 同代 checkpoint"。入口看:

关键逻辑是 notifyCheckpointAbortAsync(...)

  • 如果已有 latestCompletedCheckpointId,会先补发一次 complete,再通知当前 checkpoint abort: StreamTask.java#L1535-L1548

所以日志里完全可能先看到:

  • 更晚 checkpoint 的 complete
  • 再看到更早 checkpoint 的 abort / subsumed

这不是乱序 bug,而是 runtime 在维持"最新已完成边界优先可见"的语义。

5.7.1 为什么 2PC sink 尤其要理解这点

TwoPhaseCommitSinkFunction 的注释把这个语义写得很直白:

  • 可能存在多个 pending transaction
  • checkpoint complete 通知不一定对应"最新触发的那个 checkpoint"
  • 更晚的 completed checkpoint 可能需要把更早 checkpoint 关联的 pending transaction 一并提交

TwoPhaseCommitSinkFunction.java#L273-L324

它在 notifyCheckpointComplete(checkpointId) 里会遍历所有 pendingTransactionCheckpointId <= checkpointId 的事务并统一 commit:

这背后的本质是:

  • 外部事务的提交语义绑定的是"到某个 completed checkpoint 为止,哪些事务已经被一个成功完成的全局边界覆盖"

而不是:

  • "只提交那个 checkpoint 自己当时新开的唯一事务"

所以排障里如果你看到"先 complete 一个更晚 checkpoint,再 abort 一个更早 checkpoint",先回到 overlap + subsumed 语义,不要先怀疑乱序 bug。

5.8 savepoint 和 checkpoint 为什么不能简单混看

两者都是一致性快照,但目标不同:

  • checkpoint:自动恢复、持续容错、运行时性能
  • savepoint:运维操作、升级迁移、显式恢复边界

三个最实用的区别:

  1. 恢复目标
    • savepoint 更强调可迁移、可升级、显式恢复点
    • 格式可选 canonical / nativesavepoints.md#L138-L168
  2. side effect 语义
  3. --drain 行为
    • 会先发 MAX_WATERMARK,尽量把窗口和 timer 冲刷掉: cli.md#L237-L245

排查 "savepoint 为什么比 checkpoint 慢" 时,先分清:

  • savepoint 格式更重
  • 是否用了 --drain
  • 是否在等待 source 停机、timer 触发、窗口出清

5.9 近年的补充演进:local recovery、file-merging、changelog

对排障最有用的三个补充方向是:local recoveryfile-merging checkpointstate changelog

5.9.1 local recovery 优化的是恢复,不是 checkpoint 触发本身

task-local recovery 默认是关闭的:

它优化的核心不是:

  • "让这次 checkpoint 更快完成"

而是:

  • "失败恢复时,尽量直接复用本地已有副本,减少远端拉取"

但它也有明确边界:

  • 目前主要覆盖 keyed state
  • unaligned checkpoint 目前不支持 task-local recovery

large_state_tuning.md#L300-L318

所以 local recovery 更影响:

  • recovery time

而不是直接解释:

  • asyncDuration 为什么高

5.9.2 file-merging 优化的是"小文件洪泛"

  • 如果瓶颈是状态字节太大,它不是万能药
  • 如果瓶颈是小文件太多、元数据操作太重,它就可能很有效

5.9.3 changelog 优化的是"把写放大分摊到平时"

它的思路是持续写变更日志,再周期性 materialize:

核心取舍只有一句:

  • checkpoint 可能更轻,但恢复路径和监控复杂度会上升,恢复时间也不一定单调变短: state_backends.md#L430-L434

6. Barrier 流动如何映射到 timeout 排查

6.1 先看这些指标在 UI / metrics 里怎么出现

CheckpointMetrics 里几项字段的含义:

  • checkpointStartDelayNanos:从 checkpoint 创建到当前 Task 真正开始参与这次 checkpoint,前面已经消耗了多久
  • alignmentDurationNanos:这次 barrier 对齐花了多久
  • syncDurationMillis:Task 本地同步阶段花了多久
  • asyncDurationMillis:Task 异步 materialize / upload 到完成 ack 花了多久
  • bytesProcessedDuringAlignment:aligned 期间实际又处理了多少字节
  • bytesPersistedDuringAlignment:unaligned 或 channel state 场景下,因对齐/在途数据而写出的额外字节
  • bytesPersistedOfThisCheckpoint:这次 checkpoint 实际持久化写出的总字节数

对应定义: CheckpointMetrics.java#L34-L90

常见可见性分两层:

  1. Web UI / REST
  2. metrics system

所以排查时,先不要一上来就问"状态是不是太大",而是先问:

  • 慢在哪一段

下面 6.26.8 就按这个索引展开;到 8.2 时只需要回到这张索引,不需要重新定义一遍指标。

  • checkpointStartDelay -> barrier 传播 / 上游背压
  • alignmentDurationbytesProcessedDuringAlignmentbytesPersistedDuringAlignment -> 对齐 / unaligned / channel state
  • syncDuration -> 本地同步阶段
  • asyncDurationCheckpointed Data Size -> 异步上传 / 增量复用 / 远端存储
6.1.1 这些统计口径最容易误导人的地方

最容易误导人的有四点:

  1. request 失败真正 in-progress 后失败 不是一回事
    只有后者才会留下完整的 per-checkpoint 明细;前者可能只体现在 failedCheckpointsWithoutInProgress。实现可看 DefaultCheckpointStatsTracker.java#L305-L341 DefaultCheckpointStatsTracker.java#L521-L535
  2. Checkpointed Data Size 不等于完整状态量
    fullSizecheckpointedSize 在 stats tracker 里是分开的: DefaultCheckpointStatsTracker.java#L379-L393
  3. task breakdown 不等于最终 completed 时间
    task 只覆盖自己参与到 ack 的局部阶段;JM 汇总和 finalize 还在后面,所以单 task 不高也可能最终很慢
  4. latest successful 会掩盖中间失败
    只看最后一次成功,很容易漏掉中间连续 expired

更稳妥的顺序是:

  1. 先看 failed / completed 的历史交替
  2. 再看单次 breakdown
  3. 最后用 metrics 或日志核对 root cause

6.2 如果 checkpointStartDelay

优先怀疑:

  • barrier 从 source 传播到这个 task 很慢
  • 上游背压严重
  • 网络 buffer 堵塞
  • 某些通道特别慢,或者某个 subtask 成为热点

对应源码依据:

这意味着:

  • 当你看到某个 subtask 的 start delay 明显高于其他 subtask,问题通常在它之前的链路,而不是它本地的 RocksDB。

6.3 如果 alignmentDuration

优先怀疑:

  • 多输入对齐时有慢通道
  • 下游背压导致快通道先到、慢通道迟迟不到
  • aligned checkpoint 在当前流量形态下不适合

对应代码:

排查动作:

  1. 先看是否开启了 unaligned
  2. 再看 aligned-checkpoint-timeout 是否合理
  3. 再看是不是存在明显慢分区、热点 key、下游吞吐不足

如果同时看到:

  • bytesProcessedDuringAlignment 很高

那说明问题更像是:

  • barrier 对齐期间,task 还在继续处理大量边界前数据
  • 这通常意味着慢通道、背压或大规模 in-flight 数据比较明显

6.4 如果 syncDuration

优先怀疑:

  • prepareSnapshotPreBarrier() 里做了重操作
  • RocksDB writeBatchWrapper.flush() 成本高
  • 本地盘慢
  • compaction 正在和 checkpoint 抢 IO
  • 本地 native checkpoint 目录创建/硬链接/文件元数据操作慢

关键代码:

6.5 如果 asyncDuration

优先怀疑:

  • 这次增量其实并不小
  • SST 复用率低
  • compaction 让很多 SST 失去复用机会
  • 文件数过多
  • 远端 checkpoint 存储吞吐低
  • 上传线程数偏小
  • 异步线程池排队

源码上的依据:

这里有一个很容易误判的点:

  • asyncDurationMillis 不是纯上传耗时
  • 它从 async runnable 构造时开始算
  • 如果前面的异步任务还没跑完、线程池里排队了,这个排队时间也会被算进去

6.6 如果 bytesPersistedDuringAlignment

这个指标是理解 unaligned / timeout 转 unaligned 时非常关键的补充项。

优先怀疑:

  • 这次 checkpoint 已经切到了 unaligned
  • 网络里"边界前但尚未消费完"的 in-flight 数据很多
  • channel state 写出量已经明显上升

换句话说:

  • 这时 checkpoint 变慢,不一定是 RocksDB 本体突然变慢
  • 而可能是 barrier 建得更快了,但代价转成了"多写一份 channel state"

如果这个指标高,而 alignmentDuration 反而不高,往往说明:

  • unaligned 的确帮你避开了长时间等齐
  • 但现在瓶颈转成了 channel state 的持久化和后续上传

6.7 如果 Checkpointed Data Size 很大

这个指标更适合和 asyncDuration 一起看。

优先怀疑:

  • 这次增量其实不小
  • SST 复用率低
  • misc 文件很多
  • channel state 写出量大
  • 远端存储对大量文件 / 大量字节的写入吞吐不足

这里再提醒一次:

  • 启用增量 checkpoint 后,这个值不再代表完整状态量
  • 它只代表"这次真的持久化写出了多少"

所以更准确的解读是:

  • 它适合回答"这次 checkpoint 为什么 async 很慢"
  • 不适合单独回答"整个作业总状态到底有多大"

6.8 如果所有 Task 都差不多结束了,但最终还是 timeout

优先怀疑:

  • JM finalize metadata 慢
  • master hooks / coordinator state 慢
  • 已经 ack 的 task 不代表 checkpoint 已经完成

对应代码:

7. 为什么"几百 GB 看起来不大",却还是可能 timeout

这个问题要拆成两个量:

  • 总状态量
  • 本次 checkpoint 真正要新增写出的量

对 RocksDB 增量 checkpoint 来说,第二个量才更关键。

一个小例子:

  • 总状态量:300 GB
  • 并行度:200
  • 平均每个 subtask 本地状态:1.5 GB
  • 本次由于 compaction、新写入、SST 失去复用,最终有 10% 文件需要重传

那么这次 checkpoint 不是"只传一点 metadata",而是大约要重传:

  • 300 GB * 10% = 30 GB

如果:

  • 又开启了 unaligned
  • 并且网络里堆了 5 GB channel state

那本次真实要写出的量就可能接近:

  • 30 GB + 5 GB = 35 GB

这时决定是否 timeout 的就不再是"总状态 300 GB 大不大",而是:

  • 本地盘有没有被 compaction 抢占
  • 远端存储是否能稳定写 35 GB
  • 文件数是否过多
  • barrier 前面对齐是否先花掉了几分钟
  • JM finalize 是否又花了一段时间

8. 一份按阶段排查的最短路径

8.1 第一步:先确认失败是不是超时

看失败原因是否是:

  • CHECKPOINT_EXPIRED

对应定义: CheckpointFailureReason.java#L39-L47

对应超时取消代码:

8.2 第二步:按 metrics 定位阶段

优先按下面顺序看:

  1. checkpointStartDelay
  2. alignmentDuration
  3. syncDuration
  4. asyncDuration
  5. bytesPersistedDuringAlignment
  6. bytesPersistedOfThisCheckpoint

这里不把 6 章已经讲过的内容再抄一遍,直接按索引跳转就够了:

  • checkpointStartDelay 高:回到 6.2
  • alignmentDuration 高:回到 6.3
  • syncDuration 高:回到 6.4
  • asyncDuration 高:回到 6.5
  • bytesPersistedDuringAlignment 高:回到 6.6
  • Checkpointed Data SizebytesPersistedOfThisCheckpoint 高:回到 6.7
  • 单个 Task 看起来都不高,但总时长还是超时:回到 6.8

8.3 第三步:把根因映射回设计层

很多现象其实是设计取舍的自然结果:

  • aligned checkpoint 慢:因为它用"等待"换取"少写 channel state"
  • unaligned checkpoint 变大:因为它用"多写在途数据"换取"更快建立一致性边界"
  • RocksDB 增量 checkpoint 失效:因为它是"按 SST 复用",不是"按 key 逻辑增量"
  • full checkpoint 很重:因为它本来就是"当前快照目录全部物化"

也就是说,排障不要只停在"哪个参数要调大"。

更有用的问法通常是:

  • 我现在慢的是"边界建立"还是"状态物化"?
  • 我当前工作负载更怕"等待"还是更怕"多写"?
  • 我当前瓶颈在"本地磁盘"还是"远端 checkpoint 存储"?

9. 结论

  • Checkpoint 不是一个单机状态保存动作,而是一次全局一致性切分。
  • Barrier 的职责是定义边界;状态后端的职责是把这个边界对应的本地状态物化出来。
  • RocksDB + LSM tree + native checkpoint + SST 复用,使 Flink 有机会把大状态 checkpoint 做成"同步阶段短、异步阶段增量化"的形态。
  • 但 timeout 是否发生,取决于整条链路,而不只取决于"总状态量"。
  • 真正排查时,要先把时间花在哪一段搞清楚,再决定是调 barrier 策略、RocksDB 参数、上传并发,还是 checkpoint 存储。
相关推荐
小红的布丁2 小时前
BIO、NIO、AIO 与 IO 多路复用:select、poll、epoll 详解
java·数据库·nio
Elastic 中国社区官方博客2 小时前
在 Elastic 中使用 OpenTelemetry 内容包可视化 OpenTelemetry 数据
大数据·开发语言·数据库·elasticsearch·搜索引擎
疯狂打码的少年2 小时前
【Day12 Java转Python】Python工程的“骨架”——模块、包与__name__
java·开发语言·python
Mike117.2 小时前
GBase 8a UNION 和 UNION ALL 的使用边界
大数据·数据库
希望永不加班2 小时前
SpringBoot 自定义 Starter:从零开发一个私有 Starter
java·spring boot·后端·spring·mybatis
悟空码字2 小时前
别再System.out了!这份SpringBoot日志优雅指南,让你告别日志混乱
java·spring boot·后端
一 乐2 小时前
工会管理|基于springboot + vue工会管理系统(源码+数据库+文档)
java·数据库·vue.js·spring boot·论文·毕设·工会管理系统
callJJ2 小时前
Spring AI ETL 数据处理管道实战指南:从原始文档到向量索引
java·人工智能·spring·ai·etl·spring ai
暗暗别做白日梦2 小时前
Maven 内部 Jar 包私服部署 + 多模块父工程核心配置
java·maven·jar