Flink Source / Sink Exactly-Once 边界分析

1. 这份文档解决什么问题

路线图里已经把 checkpointstatetimerShuffle / Backpressure / Network 先打通了。接下来真正需要闭环的是:

  • source 位点越过后,为什么 record 还能防丢
  • sink 为什么不能只靠 operator state 就保证外部不重
  • notifyCheckpointComplete() 到底是不是 exactly-once 的真正提交边界
  • Flink 内部 exactly-once 和外部 WAL / ActionStateStore 的职责边界到底怎么切

这份文档不直接翻译源码,而是按"演进式代码深度解析"来推导:

  1. 最朴素的 source/sink 一致性模型是什么
  2. 它为什么会在真实流系统里立刻失效
  3. Flink 为什么必须拆成 SourceReader / SplitEnumerator / SourceCoordinator / SinkWriter / Committer / GlobalCommitter
  4. 真实源码里,这些抽象分别落在哪些类上
  5. 这套设计的收益、代价和边界是什么

2. 先说结论

  • Flink 内部 exactly-once 的核心不是"绝不重放",而是"失败后从最近一次完成的 checkpoint 恢复,并让 source 位点、operator state、sink pre-commit 状态对齐到同一条边界"。
  • source 侧真正进入 checkpoint 的不只是 reader 当前读到哪里,还包括 split 分配与 enumerator 状态。否则恢复时不知道哪些 split 已经分给谁、哪些需要回收重分配。
  • sink 侧不能把"写外部系统"直接等同于 checkpoint。更准确地说,prepareSnapshotPreBarrier() 是 pre-commit 边界,notifyCheckpointComplete() 才是可以把副作用正式确认出去的 commit 边界。
  • 新 Sink API 里,writer 负责把"待提交结果"编码成 committables;committer / global committer 负责在 checkpoint 真正确认后提交。旧的 TwoPhaseCommitSinkFunction 也是同一个思想,只是接口形态更直接。
  • Flink 只能把边界推进到"connector 已经把外部副作用变成可提交、可恢复、可去重的协议对象"为止。如果外部系统没有事务、没有幂等键、没有 WAL,那么 Flink 的 exactly-once 不能自动跨过去。

3. 步骤 1:第一性原理

先忘掉 Flink。最原始的问题其实只有一句话:

  • source 从外部系统读一条
  • operator 处理一条
  • sink 把结果写出去
  • 失败后不要丢,也不要重复对外可见

最朴素的写法大概是这样:

java 复制代码
while (true) {
    Record r = source.read();
    Result out = process(r);
    sink.write(out);
    source.commitOffset(r.offset);
}

这个版本看起来很自然,但它其实把三件不同的事情混在了一起:

  1. 读取进度
  2. 计算进度
  3. 外部副作用可见性

只要在下面任意一点失败,就会立刻出问题:

  • sink.write(out) 成功了,但 source.commitOffset(r.offset) 还没做
  • source.commitOffset(r.offset) 成功了,但 sink.write(out) 还没成功
  • process(r) 已经改了本地状态,但这份状态还没被 durable snapshot 固化

所以最朴素模型的致命问题是:

  • 它没有定义"一次一致性切分"的系统边界
  • 它把"处理过"与"外部已经正式确认"混成一个动作

如果把它抽象成一个更泛化的状态机模型,也就是不管你用什么语言、什么框架,只要流处理是"来一条、算一条、写一条、认一条"的串行逻辑:

java 复制代码
for (Record r : externalSource) {
    State s = updateState(r);       // 1. 改本地状态
    ExternalWrite w = sideEffect(s); // 2. 写外部系统(产生副作用)
    ackSource(r.position);           // 3. 确认偏移量
}

这三个动作在物理上是分散在三个不同的系统里的。如果像这样"来一条就顺次执行一遍",中间一旦断掉,三者就会永远处于"进度不一致"的状态。

可以用一个具象化的小规模数字例子来看:

假设正在处理一条"用户充值 100 元"的记录(r.position = 42)。

  1. updateState:内存里的总余额从 900 变成 1000。
  2. sideEffect:调用外部银行接口,真正扣款 100 元。
  3. ackSource:准备告诉 Kafka 第 42 条数据处理完了。

如果刚执行完 sideEffect(银行已经扣了 100 元),正准备执行 ackSource 时,机器突然断电宕机了。当机器重启,灾难开始了,这里没有任何地方能回答下面三个问题:

  • 失败后从哪条记录重新读?
    宕机前 ackSource(42) 没执行,系统重启后会重新读取第 42 条数据。
  • 哪些状态更新已经算数?
    宕机前内存总余额已是 1000,断电就没了。重启后再读第 42 条数据,总余额可能又加了 100。
  • 哪些外部写入已经对世界正式生效?
    宕机前外部银行接口已经成功扣款 100 元,重启后重新处理第 42 条数据,导致银行又扣了 100 元(发生重复的外部副作用)。

这就是为什么仅有"重试"不够,系统必须演进出"一致性边界"。

4. 步骤 2:第一次危机与第一次演进

4.1 第一个致命痛点

一旦系统进入分布式和长链路场景,最先爆炸的不是性能,而是边界定义。

典型故障窗口是:

  1. source 已经把位点推进了
  2. 数据也可能已经进入下游 operator
  3. sink 甚至可能已经把结果写到外部系统
  4. 但这几个动作并没有固定在同一条 checkpoint 边界上

这时系统至少需要一个第一层抽象:

  • 用 checkpoint 把 source progress、operator state、sink pending state 固定到同一条恢复边界

第一次演进后的最小伪代码应该变成:

java 复制代码
onRecord(r):
    bufferOrProcess(r)
    updateManagedState(r)

onCheckpoint(cp):
    snapshot.sourcePosition = currentPosition
    snapshot.operatorState = currentState
    snapshot.sinkPending = pendingWrites

onRestore(snapshot):
    restore(snapshot.sourcePosition,
            snapshot.operatorState,
            snapshot.sinkPending)

这个版本已经比最朴素写法强很多,因为它回答了"失败后从哪里继续"。

但它仍然不够,原因有两个:

  1. source 不只是一个 position,还涉及 split 分配、reader ownership、enumerator 状态
  2. sink 不只是"记住 pendingWrites",还要定义"什么时候允许真正 commit 到外部系统"

5. 步骤 3:识别新痛点与第 N 次演进

5.1 第二次演进:source 不能只有 reader 位点,还要有分配控制面

如果你以前用过旧版的 Kafka Source(FlinkKafkaConsumer),或者从单机 Kafka 消费者的视角来看,你可能会觉得很奇怪:
"Kafka 的消费进度不就是记录一个 Partition 的 Offset 吗?为什么还需要什么 Enumerator、Coordinator 这么多角色?"

这个直觉只在**"静态数据源"(比如分区永远不变的 Kafka Topic)下成立。但在真实的流计算世界里,数据源是会动态生长**的:

  • Kafka Topic 为了扩容,Partition 数量从 10 增加到了 20。
  • HDFS/S3 目录每隔 5 分钟就会生成一个新的文件。
  • Paimon/Hive 表新增了一个分区。

如果只有 Reader(干活的人),谁来发现这些新长出来的数据(Split)?谁来决定把新文件 B 分配给哪个 Reader?

如果让每个 Reader 自己去扫目录、拉元数据,不仅对外部系统压力极大,Reader 之间还会产生严重的抢占和协调问题。

为了解决这个"动态发现与分配"的难题,Flink 引入了控制面(Control Plane):

  • SplitEnumerator(包工头):运行在 JobManager 侧,负责去外部系统发现新分片(Split),并派发给下游。
  • SourceReader(干活的):运行在 TaskManager 侧,只负责无脑读取分给自己的 Split
  • SourceCoordinator(账房先生):负责把"包工头派活的账本"纳入 Checkpoint。

如果只快照 reader 位点,不快照分配状态,会有一个致命的恢复漏洞。

用一个具体的文件读取(或动态 Kafka 分区)场景来看:

  1. T1:Reader 正在读文件 A 的第 100 行,此时 Checkpoint 1 成功。Reader 快照记下:{读 A 到 100 行}
  2. T2:包工头(Enumerator)发现了新文件 B,派发给了 Reader。Reader 把文件 B 放入本地排队队列,还没开始读。
  3. T3:系统突然宕机!

重启后,Reader 从 Checkpoint 1 恢复,只知道自己要继续读文件 A。

那文件 B 呢?因为它是在 Checkpoint 1 之后分配的,Reader 的快照里根本没有文件 B 的影子。如果派发任务的包工头(Coordinator)不记得"我曾把 B 派发出去过",文件 B 就会被彻底遗忘,数据永久丢失。

这个漏洞的本质是:一个数据分片(Split)在"已派发给 Reader"但"还未被下一次 Checkpoint 确认"的这个中间态(In-flight Assignment)里,是最危险的。

所以 source 的第二次演进,必须加入"派活账本"(也就是 SourceCoordinator):

java 复制代码
// 1. 派发时,不仅记下谁拿了任务,还要记下这个任务是属于"下一次 Checkpoint"的
onAssign(split, reader):
    nextCheckpointId = currentCheckpointId + 1
    assignmentTracker.markAssigned(split, reader, nextCheckpointId)

// 2. Reader 干活,并在本地快照记下进度
reader.onCheckpoint(cp):
    snapshot.readerState = ownedSplitsWithPosition

// 3. Coordinator 把派活账本做进快照
coordinator.onCheckpoint(cp):
    snapshot.enumeratorState = enumeratorState
    snapshot.assignments = assignmentTrackerState

// 4. 重点:正常运行时,不需要通信!
// 只要 JobManager 广播说 cp 成功了,Coordinator 就知道 Reader 肯定也存好了,直接清账
coordinator.notifyCheckpointComplete(cp):
    assignmentTracker.removeAssignmentsUpTo(cp)

// 5. 局部失败恢复时的"对账"
onReaderFailure(reader, restoredCp):
    // 如果 Reader 退回到了 restoredCp,那账本上记录的 > restoredCp 的任务,Reader 肯定全忘了
    splits = assignmentTracker.getAssignmentsGreaterThan(reader, restoredCp)
    // 把这些被遗忘的任务拿回来,塞回给 Enumerator 重新派发!
    enumerator.addSplitsBack(splits)

这里第一次真正出现了"source exactly-once 的边界不是单点 offset,而是 reader data-plane(干活进度) + enumerator control-plane(派活账本) 的合成边界"。

这个设计最精妙的地方在于免通信对账:Reader 存好快照后,不需要专门发消息告诉 Coordinator"我存好了"。Coordinator 完全利用 Checkpoint 机制的全局屏障和单调递增的 ID,就能精确算出哪些任务在失败时掉进了"真空期"。

5.2 第三次演进:sink 不能直接 commit,而要拆成 pre-commit 和 commit

如果 sink 每处理一条就直接写外部系统,那么 checkpoint 最多只能保证"失败后我知道自己处理到哪",却不能保证"外部世界只看见一次"。

要跨过这个边界,sink 必须拆成两段:

  1. barrier 到达时,先把当前批次 flush 成 committables
  2. 等 checkpoint 被 JobManager 真正确认完成,再 commit 这些 committables

最小伪代码:

java 复制代码
onRecord(r):
    writer.write(r)

prepareSnapshotPreBarrier(cp):
    writer.flush()
    pending[cp] = writer.prepareCommit()

notifyCheckpointComplete(cp):
    commit(pending[<= cp])
    removeCommitted(pending[<= cp])

这一步非常关键,因为它把两个经常被混淆的边界拆开了:

  • checkpoint snapshot 边界
  • external visibility 边界

前者回答"恢复后从哪里接着跑",后者回答"外部系统从哪一刻开始认为这批结果已经正式生效"。

5.3 第四次演进:为什么需要 Global Commit?有时"收齐输入"本身就是隐式确认

如果你只写过 Kafka,你可能会觉得每个并发节点各自去提交事务(Local Commit)就够了。

但假设你正在写 Iceberg、Hive 或 Paimon,并发度是 3。如果让 3 个并发各自去提交自己写出的小文件(file_1, file_2, file_3):

  • 外部系统会产生 3 个独立的数据快照,下游读到的数据是"一截一截"冒出来的,失去了全局一致性视图。
  • 外部的元数据系统(如 Hive MetaStore)会被这种高频碎片的并发提交直接打挂。

这时候就必须引入一个"总指挥"------Global Committer(全局提交器)

它的并行度通常是 1。上游多个并发 Writer 把自己写好的"待提交文件路径(Committables)"发送给它;它等收齐了所有并发的文件路径后,再把它们打包成一个大事务,向 Iceberg 发起唯一一次原子提交 。这就是 "Global" 的含义:把局部的副作用汇总成全局单次提交。

在包含 Global Committer 的拓扑里,有一处非常精妙的"隐式确认"设计:

当 sink 拓扑是 Writer (并发N) -> Local Committer (并发N) -> Global Committer (并发1) 时,Global Committer 可能不需要再傻等属于自己的 notifyCheckpointComplete() 回调了。

原因不是它更强,而是它利用了一个更高层事实:

  • Local Committer 是严格遵守 2PC 的,它必须等到 JobManager 广播了 completed 确认后,才会把手里的 Committables 往下游发送。
  • 那么,站在 Global Committer 的视角:只要我从上游收齐了这个 Checkpoint 的所有 Committables,就隐式证明了上游已经收到过确认,也就是说这个 Checkpoint 必然已经成功了!

所以 global committer 在某些拓扑里可以这么写:

java 复制代码
onCommittable(msg):
    collector.add(msg)
    if (collector.hasAllFromAllSubtasks(checkpointId)) {
        globalCommit(checkpointId)
    }

而在没有 upstream committer、需要它自己承担 2PC 最终提交职责时,它又必须回到:

java 复制代码
notifyCheckpointComplete(cp):
    globalCommit(cp)

这一步说明 Flink 不是机械地"所有东西都等回调",而是根据拓扑上游是否已经承接过 checkpoint 完成语义,决定最终提交边界放在哪里。

6. 步骤 4:映射到真实源码

下面把上面的演进结果,对应到 Flink 真实实现。

6.1 source reader:本地数据面边界

SourceOperatorsnapshotState() 里直接把 sourceReader.snapshotState(checkpointId) 写入 operator state;恢复时再把这些 split state 重新喂回 reader。这说明 reader 负责的是"我当前持有哪些 split,以及每个 split 消费到了哪里"。

这部分解决的是:

  • source data-plane 的恢复位置

它还没有单独解决:

  • split 的全局分配和回收

6.2 source coordinator:控制面边界

SourceCoordinator.checkpointCoordinator() 会把 assignment tracker 和 enumerator checkpoint 一起序列化;resetToCheckpoint() 会恢复 enumerator;subtaskReset() 会把未被该 checkpoint 确认的 split 通过 addSplitsBack() 放回去。

这部分解决的是:

  • 哪些 split 已经稳定分配
  • 哪些 split 在失败后要回收重派
  • source control-plane 如何进入 checkpoint

所以 source 的 exactly-once 边界不是"reader offset",而是:

  • reader state
  • enumerator state
  • assignment tracker state

三者共同组成的边界。

6.3 checkpoint 完成通知:谁有资格把边界从内部推进到外部

CheckpointPlan 把 checkpoint 生命周期明确拆成:

  • getTasksToTrigger()
  • getTasksToWaitFor()
  • getTasksToCommitTo()

其中 tasksToCommitTo 的语义就是"checkpoint 被确认成功后,谁还需要收到完成通知"。

这说明 notifyCheckpointComplete() 不是一个顺手回调,而是 checkpoint 协议里专门承接"对外确认"语义的一段生命周期。

6.4 source 在 completed checkpoint 之后还能做什么

SourceOperator.notifyCheckpointComplete() 会把 completed checkpoint 事件下传给 sourceReaderSourceCoordinator.notifyCheckpointComplete() 会进一步通知 enumerator。

这意味着 source connector 如果接外部消息系统,可以把"已完成 checkpoint"作为一种确认点,例如:

  • 向外部系统确认某些 split/offset 已经稳定
  • 推进自身的外部对账或流控状态

但这件事是 connector 自己决定的,不是 Flink 自动帮它跨出去。

6.5 sink writer:pre-commit 边界

SinkWriterOperator 的关键动作有三步:

  1. snapshotState() 持久化 writer state
  2. prepareSnapshotPreBarrier()sinkWriter.flush(false),再把 prepareCommit() 产出的 committables 发下去
  3. endInput() 时再做一次 flush 并产出终态 committables

这部分解决的是:

  • 把"已经写到一半的外部意图"编码成 checkpoint 可恢复的 committables

它还没有真正把结果设为外部最终可见。

6.6 committer:completed checkpoint 之后才允许真正提交

CommitterOperatorsnapshotState() 时保存 CommittableCollector,在 notifyCheckpointComplete() 时调用 commitAndEmitCheckpoints(...),把 <= checkpointId 的 committables 真正提交出去。

这部分说明:

  • checkpoint 之前,committables 只是"待提交"
  • checkpoint 完成之后,才进入"允许对外生效"

6.7 global committer:有时靠输入收齐,有时靠 completed callback

GlobalCommitterOperator 的类注释把边界说得很清楚:

这说明 sink 最终提交边界不是死板的一层,而是:

  • 由拓扑里谁承担"最后一跳确认"来决定

6.8 旧 2PC sink:本质和新 Sink API 一样

TwoPhaseCommitSinkFunction 的逻辑更直白:

新旧 API 的本质差异主要在封装层次,不在协议本质:

  • 都是在 checkpoint 时做 pre-commit
  • 都是在 completed checkpoint 之后做 final commit
  • 都把"可恢复的待提交对象"作为跨失败边界的核心载体

6.9 coordinator state 先进入 checkpoint,再去触发 task

Flink 在 checkpoint 流程里,会先触发并确认 coordinator checkpoint,再把 coordinator state 放进 PendingCheckpoint

这对 source 特别重要,因为 SourceCoordinator 本身就是 operator coordinator 的一个具体实现。也就是说,source 的控制面状态从协议层面就不是边角逻辑,而是 checkpoint 的正式组成部分。

7. 一个最小例子:为什么这条链能同时解释防丢和防重

假设有这样一条链路:

  • source 是 Kafka
  • 中间 operator 做聚合并更新 keyed state
  • sink 是 Paimon

下面直接按时间顺序看一次 cp42,这样更容易把 EnumeratorCoordinatorWriterCommitter 放到一条线上理解。

7.1 cp42 的时间线

T0cp42 开始之前

  • KafkaSourceReader 正在消费各个 partition 的 offset。
  • SplitEnumerator / SourceCoordinator 维护 source 控制面状态:哪些 split 已分配、哪些还没分配、哪些是在途分配。
  • Paimon SinkWriter 正在把数据写成新的 data file / changelog file,但这些文件此时还只是"本次 checkpoint 准备提交的候选物",还没有生成对外可见的 table snapshot。

T1:barrier 到达 source 和中间算子

  • SourceReader 先把自己当前持有的 split 以及每个 split 读到的 offset 做进本地快照。
  • 中间 operator 把 keyed state 做进 cp42
  • 这一步固定的是"我已经处理到了哪里"。

T2:source 控制面进入快照

  • SplitEnumerator 把自己当前知道的 split 发现结果做进快照。
  • SourceCoordinator 把 assignment tracker 做进快照,也就是:
    • 哪些 split 已经稳定分配;
    • 哪些 split 是在 cp42 前分配的;
    • 哪些 split 还在"派出去了,但还没被更晚的 completed checkpoint 确认"的真空期里。
  • 这一步固定的是"活是谁领走的,哪些活还可能丢在路上"。

T3:barrier 到达 sink,writer 做 pre-commit

  • Paimon SinkWriter flush 当前批次,把属于 cp42 的写出结果整理成一组 committables
  • 这些 committables 的本质不是"数据已经对外可见",而是"我已经准备好,如果 cp42 真 completed,就可以拿这批文件去提交一个新的 Paimon snapshot"。
  • 这一步固定的是"我准备提交什么",不是"我已经提交成功"。

T4:JobManager 宣告 cp42 completed

  • JobManager 收齐 task side 和 coordinator side 的 ack,正式把 cp42 标记为 completed。
  • 到这一刻,Flink 内部才确认:
    • source 位点边界稳定了;
    • operator state 稳定了;
    • sink 的 pending committables 也稳定了。

T5:completed 通知向下游传播

  • SourceCoordinator 收到 notifyCheckpointComplete(42) 后,可以清理 <= 42 的派活账本,因为这些分配已经被稳定确认。
  • Local Committer 收到 notifyCheckpointComplete(42) 后,才允许把 cp42committables 往后推。
  • 如果 sink 拓扑里有 Global Committer,它会开始等待 cp42 的全部上游 committables 到齐。

T6:Paimon 真正 commit

  • Global Committer 收齐 cp42 的全部 committables 后,向 Paimon 发起一次真正的 snapshot commit。
  • 到这一步,cp42 对应的数据才从"待提交文件"变成"Paimon 表上的一个新 snapshot",开始对读取这个 snapshot 的下游可见。

这里最重要的一句是:

  • checkpoint completed 不等于 Paimon 已经可见
  • 它只表示 "Flink 现在允许你去 commit"

7.2 三个最关键的失败窗口

窗口 A:T2/T3 之后失败,cp42 还没 completed

  • source 恢复到上一个 completed checkpoint 的 reader offset。
  • SourceCoordinator 也恢复到上一个 completed checkpoint 的 assignment 账本;那些掉进真空期的 split 会被 Enumerator 重新派发。
  • Paimon SinkWritercp42 准备的 committables 会跟着恢复出来,但因为 cp42 没有 completed,所以它们还不允许对外 commit。

这时的结果是:

  • 可能会重放数据;
  • 但 Paimon 还没有看到 cp42 的正式 snapshot;
  • 所以不会出现"外部已经可见,但 Flink 内部又回滚了"的撕裂。

窗口 B:cp42 已 completed,但还没真正执行 Paimon commit

  • JobManager 已经认定 cp42 成功;
  • Local Committer / Global Committer 还没来得及把 cp42committables 真正提交到 Paimon。

如果这时宕机:

  • source 和 operator 会从 cp42 恢复;
  • sink 侧也会恢复出 cp42 的 pending committables;
  • 因为 Paimon 那边还没有出现 cp42 对应的新 snapshot,所以恢复后再次执行 commit 是安全的。

这里回答的是一个很关键的问题:

  • checkpoint 成功了,为什么还可能需要"补交"一次 commit?

因为 Flink 的 completed 先保证的是"内部一致性边界已经固定",而不是"外部系统已经瞬时完成提交"。

窗口 C:Paimon commit 已经执行成功,但 Flink 还没来得及清理本地 pending 状态就宕机

这是最危险、也是最容易被忽略的窗口:

  • Global Committer 可能已经把 cp42 对应的 snapshot 成功提交给了 Paimon;
  • 但 Flink 进程还没来得及把本地 pending committables 清掉,或者没来得及把"cp42 已经提交过"这个事实推进到更后续的状态。

如果恢复后又按旧状态再提交一次 cp42,那就要求:

  • Paimon commit 必须对同一个 checkpoint / commit identifier 幂等
  • 或者 Committer 自己必须能识别"这批 committables 对应的 snapshot 已经提交过了",然后跳过重复提交

否则就会出现:

  • 同一批文件被重复纳入 snapshot
  • 或者重复生成逻辑等价但语义重复的提交

所以更准确地说,Flink 的 exactly-once 不是"不要重试 commit",而是:

  • 允许恢复后重试 commit
  • 但要求 commit 对重复执行是安全的

7.3 这条链路到底怎么同时做到防丢和防重

从 Paimon 这个例子里,可以把职责切得很清楚:

  • 防丢 :靠 Kafka source 的 offset 回滚、Enumerator / Coordinator 的分配账本恢复、operator state 恢复,以及 sink pending committables 恢复。
  • 防重 :靠 notifyCheckpointComplete() 之后才允许 commit,再加上 Paimon commit 对同一 checkpoint 的幂等处理。

最后压成一句话:

  • cp42 completed 解决的是"Flink 内部边界已经固定"
  • Paimon snapshot commit 解决的是"外部世界从哪一刻开始真正可见"

这两个动作不是一回事,中间天然存在失败窗口;exactly-once 能跨过这个窗口,依赖的正是 pending committables 可恢复 加上 commit 幂等

到这里就能把一个经常混淆的问题切开了。

Flink 内部 exactly-once 负责的是:

  • source 位点不要越过一条没有被 durable 固定的处理边界
  • operator state 在失败后能回到某个 completed checkpoint
  • sink 预提交对象能跟着 checkpoint 一起恢复

它更偏:

  • 防丢
  • 有界重放
  • 内部状态与处理进度对齐

Flink 不会自动替外部系统发明下面这些能力:

  • 幂等主键
  • 外部事务日志
  • 去重索引
  • 业务侧"这次动作是否已经真正执行过"的 durable ledger

所以一旦副作用不是标准事务 sink,而是:

  • 调外部 HTTP API
  • 发支付请求
  • 调用非事务型业务系统

你就需要额外一层 durable 去重协议,例如:

  • WAL
  • outbox
  • ActionStateStore

这层职责更偏:

  • 防重
  • 外部副作用幂等确认
  • 业务语义级别的"已经做过"

可以把职责压成一句话:

  • Flink checkpoint 负责把"我准备怎么做、做到哪了"固定下来
  • 外部 WAL / ActionStateStore 负责把"这件事对外是否已经生效"固定下来

9. 步骤 5:批判性总结

9.1 优势

这套设计的最终收益有四个:

  1. 把 source progress、operator state、sink pre-commit state 统一进同一条 checkpoint 边界
  2. 把"处理完成"与"对外正式提交"拆成两个阶段,避免把外部副作用直接暴露在失败窗口里
  3. 允许 source 和 sink connector 在统一协议下自己接入事务、幂等、split 管理、外部确认
  4. 在恢复、重叠 checkpoint、subsumed checkpoint、rescale 这些复杂场景下仍然有清晰语义

9.2 代价与局限

它的代价也很明确:

  1. 概念层次变多,需要同时跟住 readerenumeratorcoordinatorwritercommitterglobal committer
  2. 调试困难
    • 一个"重复写"问题,可能卡在 writer、committer、global committer、外部系统四个位置中的任意一个
  3. connector 质量要求高
    • Flink 只给协议骨架,真正能不能做到 exactly-once,要看 connector 是否正确实现 checkpoint、restore、commit、abort
  4. 外部系统能力仍是硬边界
    • 没有事务或幂等协议的外部系统,Flink 不能凭空提供端到端 exactly-once

9.3 有没有更现代、更接近 Unix 哲学的替代思路

有,但它们不是"更强",而是把边界放在别处。

一种常见替代是:

  • Flink 只负责把结果写到 append-only log / outbox
  • 由下游独立提交器再把 log 推到最终业务系统

最小伪代码:

java 复制代码
onCheckpoint(cp):
    persist(outboxRecordsUpToCp)

externalCommitter:
    readOutbox()
    deduplicateByBusinessKey()
    applyToTarget()

这种方式更接近 Unix 风格,因为它把职责拆得更硬:

  • Flink 负责稳定产生日志
  • 外部提交器负责副作用投递与去重

它的好处是:

  • connector 压力小
  • 副作用边界更清晰

它的代价是:

  • 系统链路更长
  • 延迟更高
  • 最终一致性窗口更大

所以更准确的结论不是"谁绝对更好",而是:

  • 如果外部系统本身支持事务或强幂等,Flink 内嵌 2PC / committer 链更直接
  • 如果副作用系统非常异构、协议很弱,outbox / WAL / ActionStateStore 往往更稳

10. 最后压成一句话

Source / Sink Exactly-Once 的本质,不是"source 不重、sink 不重"这么简单,而是:

  • source 用 checkpoint 固定读取边界
  • operator 用 state 固定处理边界
  • sink 用 pre-commit + completed callback 固定外部提交边界
  • 外部副作用再用事务、幂等键或 WAL 固定业务可见边界

只有这四层一起闭环,防丢和防重才算真正闭环。

相关推荐
xyyaihxl2 小时前
将 vue3 项目打包后部署在 springboot 项目运行
java·spring boot·后端
AC赳赳老秦2 小时前
测试工程师:OpenClaw自动化测试脚本生成,批量执行测试用例
大数据·linux·人工智能·python·django·测试用例·openclaw
Simon_lca2 小时前
验厂不翻车!Acushnet 11 项核心政策 + 自查要点,一文搞定
大数据·人工智能·经验分享·算法·制造
0xDevNull2 小时前
Spring Boot 3.x 整合 Nacos 全栈实战教程
java·spring boot·nacos
落木萧萧8252 小时前
MyBatisGX 示例工程:CRUD + 关联查询完整演示
java·后端
两年半的个人练习生^_^3 小时前
每日一学:设计模式之原型模式
java·开发语言·设计模式·原型模式
biubiubiu07063 小时前
Maven 父子工程 SpringBoot 多模块
java·spring boot·maven
tumeng07113 小时前
Spring详解
java·后端·spring
深邃-3 小时前
【Web安全】-基础环境安装:虚拟机安装,JDK环境安装(1)
java·开发语言·计算机网络·安全·web安全·网络安全·安全架构