如何保证 Kafka 的消息顺序性?
先有一张阅读地图
理解 Kafka 消息顺序性时最容易乱,是因为大家经常把"发送顺序、落盘顺序、消费顺序、业务处理顺序"混为一谈。
更稳妥的读法是先把链路拆开,始终带着下面几个问题去审视整个流程:
- 我们在保谁的顺序?(作用域):是要求全局所有消息排队,还是只要"同一个用户/订单"不乱就行?
- 怎么确保发出去时不乱?(生产者):网络抖动、发送失败需要重试时,如果允许并发发送,先发的消息后到怎么办?
- 服务端怎么接住写入顺序?(Broker):面对客户端的重发、并发到达的请求,服务端如何判断这是一条新消息,还是一条重复/乱序的消息;如果写到一半负责写入的 broker 换了,新的接管者又凭什么继续判断顺序?
- 怎么确保处理完不乱?(消费者):把消息拉到本地后,如果丢进线程池并发处理,是不是又乱了?提交进度的时机对不对?
- 多条消息的一致性怎么保?(事务边界):如果涉及多条结果的联动输出,或者要和消费位点绑定,这部分内部一致性怎么收口?
下面正文基本就按这张图展开:
- 先定义顺序作用域
- 再看 producer / broker 这条写入链路
- 然后看 consumer / commit 这条读取链路
- 最后补事务和源码映射
先把问题定义清楚
如果一句话概括 Kafka 的顺序性,那就是:
- Kafka 天然保证的是 单个 partition 内按 offset 的顺序。
- Kafka 不天然保证 同一 topic 的全局顺序。
- 业务上真正想保序,先要明确"哪些消息彼此必须有序",也就是顺序作用域。
一个最朴素的顺序日志系统,至少要先有一份所有消费者都能看到的共享日志:
text
nextOffset = 0
brokerLog = []
send(msg):
brokerLog.append((nextOffset, msg))
nextOffset += 1
consume():
cursor = 0
while cursor < len(brokerLog):
handle(brokerLog[cursor])
cursor += 1
这个模型天然保序,因为"写入顺序 = offset 顺序 = 消费顺序"。
这里虽然还没有出现完整的 Kafka 集群,但已经有了一个关键角色:
- producer 不直接把消息交给 consumer。
- 它先把消息写进一份共享日志。
- consumer 再按日志中的 offset 顺序读取。
Kafka 里的 broker,本质上就是把这份"共享日志"做成了可持久化、可复制、可供多个 consumer 组反复读取的系统。
但它的问题也很直接:只有一条日志,吞吐上不去。
Kafka 的第一步演进,就是把一条日志拆成多条:
text
partitions = [log0, log1, ..., logN-1]
send(msg):
p = partitionBy(msg.key)
partitions[p].append(msg)
consume(p):
for record in partitions[p] by offset:
handle(record)
从这里开始,顺序的作用域就变了:
- 同一个 partition 内可以谈顺序。
- 跨 partition 天然就不能再谈严格先后。
所以顺序问题的第一原则不是"怎么调 Kafka 参数",而是:
- 先定义顺序作用对象。
- 再让这个对象稳定映射到同一个 partition。
常见的顺序作用对象是:
- 同一用户
- 同一订单
- 同一账户
- 同一设备
如果这个作用对象没有稳定 key,后面所有保序配置都只能保证"局部不乱",不能保证"业务不乱"。
顺序成立,需要哪几个条件
对某个业务实体 entityId 来说,消息顺序最终要满足 3 个条件:
- 生产前:同一实体的消息必须进入同一个 partition。
- 生产时:producer 侧的重试和并发,不能把这个 partition 的写入顺序打乱。
- broker 侧:leader 追加日志和故障切主后,仍然要能判断哪些请求是合法顺序、哪些是重复或乱序。
- 消费后:应用必须按这个 partition 内的顺序处理,并在处理完成后再提交位点。
也就是说,真正要看的不是一条"发送 -> 接收"的直线,而是三段:
生产端的本质逻辑:
text
send(msg):
tp = routeByKey(msg.entityId)
queue[tp].append(msg)
senderLoop():
batch = dequeue(tp)
sendToLeader(batch)
waitAckOrRetry(batch)
broker 侧的本质逻辑:
text
onProduce(batch):
validate(batch)
appendToPartitionLog(batch)
replyAckOrError(batch)
消费端的本质逻辑:
text
poll():
records = fetchByOffset(tp)
for record in records_of_tp_in_offset_order:
process(record)
commit(nextOffset(tp))
注意这里 producer、broker、consumer 讨论的是三种不同顺序:
- producer 端讨论的是"请求先后会不会被重试和并发打乱"。
- broker 端讨论的是"哪些请求能被合法落盘并拿到 offset"。
- consumer 端讨论的是"拉到了有序数据后,应用会不会自己处理乱"。
接下来先看生产端。
生产端里最容易把顺序打乱的,不是"分区队列里消息的先后",而是"已经发出去但还没收到响应的请求之间,是否会因为失败、超时、重试而发生前后倒置"。
所以要理解生产端顺序,得先理解 Kafka 在这里到底把什么叫作 in-flight。
先讲生产端:in-flight 到底是什么
max.in.flight.requests.per.connection 的意思不是"内存里最多缓存多少条消息",也不是"每个 partition 同时只能有多少条消息"。
它的准确含义是:
- 同一个 TCP 连接上,已经发出去但还没有收到响应的请求数上限。
这里的关键是"请求",不是"消息":
- 一个请求里可能带一个或多个 batch。
- 一个 batch 里可能有多条消息。
- 同一个 broker 上可能有多个 partition 的请求共用同一条连接。
对顺序性最相关的场景是:
- 某个 partition 的 leader 在 broker
B1 - producer 到
B1有一条连接 - 同一 partition 的两个 batch
A、B都已经发出,但还没收到响应
如果 max.in.flight.requests.per.connection=2,这种情况就是允许的。
如果是 1,那就不允许,必须等 A 的响应回来,才能发 B。
为什么"只有一个 in-flight"是最简单的保序办法
因为它把同一连接上的发送关系强行串行化了:
text
send A
wait ack(A)
send B
wait ack(B)
在这种模式下,即使有重试,也不会出现"后面的请求先成功、前面的请求后重试回来"的倒序竞争。
Kafka 在客户端里就是这么做的:
KafkaProducer把max.in.flight.requests.per.connection == 1转成guaranteeMessageOrder=true,见 KafkaProducer#L525-L547。Sender发送后会暂时mute这个 partition,等 batch 完成再unmute,见 Sender#L418-L425 和 Sender#L736-L738。RecordAccumulator.ready()也明确把"muted partition 不可继续发送"写进了条件里,见 RecordAccumulator#L752-L768。
所以如果你的目标是"先求绝对稳,再谈吞吐",max.in.flight=1 是最直观也最不容易误解的方案。
如果想要更高吞吐,顺序靠什么继续兜住
到这里,生产端其实已经有两条路:
- 一条路是把发送彻底串行化,也就是
max.in.flight=1。 - 另一条路是允许多个请求同时在路上,以换取更高吞吐。
第一条路最简单,但每次都要等前一个请求返回,链路利用率会偏低。
第二条路更高效,但会立刻带来一个新的顺序问题:
- 前一个 batch 失败重试回来时,broker 怎么知道它是"旧 batch 的重发"?
- 后一个 batch 先到时,broker 怎么知道它是"越过前序 batch 的乱序写入"?
也就是说,多个 in-flight 一旦放开,并发本身不是问题,怎么识别"重复"和"乱序"才是问题 。
Kafka 对这个问题的回答,就是幂等 producer。
enable.idempotence 在这里解决什么问题
enable.idempotence=true 的核心含义不是"开启一个抽象的高级可靠性模式",而是:
- producer 给每个 partition 的 batch 编号。
- broker 记住这个 producer 在这个 partition 上已经成功写到了哪个 sequence。
- 重试时,broker 能识别"这是重发的旧 batch"还是"这是越过前序 batch 的乱序 batch"。
所以幂等直接解决的是两类问题:
- 重试导致的重复写入。
- 多个 in-flight 请求下的乱序写入。
但幂等也有边界:
- 它解决的是 producer 到 broker 这段链路 的重复与乱序。
- 它不解决跨 partition 顺序。
- 它也不替你解决消费线程池自己把顺序打乱的问题。
Kafka 自己在 KafkaProducer 注释里也写了:
- 幂等模式下如果碰到
OutOfOrderSequenceException后还继续发送,可能导致乱序。 - 若要严格确保顺序,应关闭 producer 并重建,见 KafkaProducer#L1028-L1034。
多个 in-flight 时,broker 到底怎么判断
如果允许多个请求并发在路上,Kafka 并不是放弃顺序,而是换了一种保序方式:
- 允许多个请求同时发送。
- 但每个 batch 都带上单调递增的 sequence。
- broker 只接受"符合当前 sequence 状态"的 batch。
- 如果发现这是已经成功过的重复 batch,就直接返回之前的结果。
本质伪代码如下:
text
batch.seq = nextSeq[tp]
send(batch)
broker.onAppend(batch):
validate(batch.seq, producerEpoch)
if duplicate(batch):
return previousMetadata
append(batch)
这里"允许并发"不等于"允许乱序",因为"先到达"和"先被合法写入"不是一回事。
一个最小例子:
- 同一个 partition 上,先发送
A(seq=0),再发送B(seq=1)。 - 由于网络抖动,
B先到 broker。
这时有两种情况:
- 没开幂等:broker 不用 sequence 约束,只看谁先来,
B就可能先落盘。 - 开了幂等:broker 会校验 sequence,
B(seq=1)在A(seq=0)之前到达时,不会被当成合法的"下一批"直接写进去。
Kafka 的配置文档也直接说明了这一点:
- 当
enable.idempotence=false且max.in.flight.requests.per.connection>1时,重试会导致重排,见 ProducerConfig#L291-L293。 - 当开启幂等时,要求
acks=all、retries>0、max.in.flight.requests.per.connection<=5,并说明在这个允许范围内保序,见 ProducerConfig#L337-L347 和 ProducerConfig#L269-L276。
客户端和 broker 两边也都做了对应工作:
- 客户端重试时,不是简单把失败 batch 塞回队头,而是按 sequence 重新插回正确位置,见 RecordAccumulator#L542-L592。
- broker 侧会校验 sequence 连续性,不连续就抛出
OutOfOrderSequenceException,见 ProducerAppendInfo#L156-L198。 - broker 还会检测 duplicate batch,见 UnifiedLog#L1397-L1406。
TCP 已经有重传,为什么这里还不够
最容易产生的误解是:
- 如果只是网络里丢了几个 TCP 包,但连接还活着,TCP 会自动重传。
- 在同一条 TCP 连接上,字节流仍然按发送顺序交付。
- TCP 的 ACK 只表示:对端机器的 TCP 协议栈已经收到这些字节了。
这里的关键边界是:
- TCP ACK 不是 Kafka 的成功响应。
- TCP ACK 确认的是"字节到了对端内核"。
- Kafka 真正关心的是"这次 Produce 请求有没有按 Kafka 语义成功处理"。
这个"成功处理"至少可能包含:
- broker 进程已经从 socket 里读到了请求。
- 请求已经被正确解析。
- 目标 partition 的状态允许这次写入。
- 消息已经 append 到日志。
- 如果配置了
acks=all,ISR 副本也已经满足确认条件。
所以如果 broker 在"刚收到字节"时就算成功,那它和 TCP ACK 没本质区别,确认语义太弱。
Kafka 必须等待"处理完成"再返回自己的 ProduceResponse,否则 producer 无法知道"消息到了"还是"消息真的写成了"。
Kafka 真正担心的,通常不是"某个 TCP 包丢了",而是更大的失败:
- 连接断了。
- 请求超时了。
- broker 返回了应用层可重试错误。
- leader 变了,需要把后续请求发到新 leader。
一旦进入这些场景,TCP 能告诉应用的只是"连接还活着还是已经坏了",却不能替 Kafka 回答更关键的问题:
- 刚才那个 Produce 请求,到底有没有在 broker 侧真正写进 partition 日志?
看一个最小例子:
- producer 发送
A - broker 进程其实已经把
A写进日志,并且准备返回ProduceResponse - 但这个应用层响应在返回路上丢了,或者连接在返回前断了
- producer 没收到明确成功响应,只知道这次请求结果不确定
这时 TCP 并不能告诉 producer:
A是根本没到 broker- 还是字节已经到 broker 内核了,但 broker 进程还没真正处理
- 还是已经成功落盘,只是
ProduceResponse没有送回 producer
所以此时如果 producer 再重试 A,问题已经不是"TCP 会不会重发某个包",而是:
- 这个重试的
A,在 Kafka 语义上到底是"应该补写的旧请求" - 还是"已经成功过的重复请求"
- 或者相对于后续请求来说,已经变成了"乱序请求"
所以更准确地说:
- TCP 解决的是"连接里的字节流有没有按序送达"。
- Kafka 幂等解决的是"请求在 broker 侧有没有成功生效,以及重试后该怎么判定重复和乱序"。
顺着这点再往前一步,其实也就能理解 Kafka 自己的重试在做什么:
- 如果只是 TCP 丢包而连接还活着,TCP 自己会重传,应用通常感知不到。
- Kafka producer 真正要处理的,是"请求级结果不确定"后的重试。
也就是说,Kafka 的重试不是单纯补发某个 TCP 包,而是:
- broker 返回了可重试错误,producer 重新发送这个 batch
- 或者 producer 一直没拿到明确成功响应,只能把这次请求当成"可能失败"后再试一次
这也是为什么 Kafka 的重试最终会落到前面说的"重复"和"乱序"判断上。
如果一直失败,producer 最终通常有两种结果:
- 如果遇到的是不可重试错误,会直接失败,不再继续重试。
- 如果一直是可重试错误,也不会无限拖下去,而是一直重试到
delivery.timeout.ms到期,再以失败结束。
配置定义里写得很明确:
retries表示对瞬时错误重试多少次,直到成功、失败为非瞬时错误,或者delivery.timeout.ms到期,见 ProducerConfig#L278-L282。delivery.timeout.ms则限制了一次send()从返回 future 到最终报告成功或失败的总时长,见 ProducerConfig#L167-L176。
校验失败后,broker 会缓存乱序 batch 吗
一个容易继续追问的问题是:
- 如果校验不通过,broker 会不会像 TCP 窗口那样,先把这个 batch 缓存起来,等前面的 batch 来了再接上?
Kafka 这里的选择通常不是"缓存等待",而是"直接拒绝非法写入"。
更准确地说:
- 如果 broker 判断这是一个已经成功过的重复 batch,它不会重写数据,而是直接返回之前的元数据。
- 如果 broker 判断这是一个乱序 batch,它会抛出
OutOfOrderSequenceException,而不是替 producer 暂存 payload 等前序 batch。
源码上可以直接看到这一点:
- sequence 校验不通过时,
ProducerAppendInfo.checkSequence()直接抛异常,见 ProducerAppendInfo#L156-L194。 - duplicate 的情况只返回已有 batch 的元数据,不会重新 append,见 UnifiedLog#L1247-L1255。
- 真正的
localLog.append(...)只发生在校验通过、且不是 duplicate 的分支里,见 UnifiedLog#L1255-L1267。
Kafka 不走"broker 暂存乱序 batch,等待前序 batch 补齐"这条路,主要是因为:
- Kafka 的顺序单位不是 TCP 的字节流,而是带
producerId、epoch、sequence的 batch。 - 如果 broker 要缓存乱序 batch,就要维护按 producer-partition 隔离的重排窗口、超时、淘汰和故障恢复状态,复杂度和内存压力都很高。
- Kafka 的设计里,真正负责重试和重排的是 producer 客户端,而不是 broker,见 RecordAccumulator#L542-L592。
那多个 in-flight 到底值不值得
既然最终还是要按顺序接受,多个 in-flight 的意义就在于:
- Kafka 追求的不是"谁都不能先发",而是"谁都可以先发,但不能非法落盘"。
如果只有一个 in-flight,链路会变成:
text
send A -> wait ack(A) -> send B -> wait ack(B)
这样最简单,但代价也最直接:
- 网络 RTT 期间连接是空等的。
- broker 处理能力和客户端批量能力利用不起来。
- 吞吐会更低,尤其是高延迟链路下更明显。
如果允许多个 in-flight,链路更像:
text
send A
send B
send C
broker 依 sequence 和状态决定谁能被合法写入
这时并发的价值在于:
- producer 不必每发一个请求都停下来等响应。
- 网络链路和 broker 处理流水线可以更饱满。
- 在顺序仍受 sequence 约束的前提下,吞吐通常会更高。
这里的结论可以收成一句话:
max.in.flight=1是"靠串行化保序"。enable.idempotence=true且max.in.flight<=5是"靠 sequence 校验和去重保序"。
为什么幂等要求 acks=all
先回答结论:
- Kafka producer 的
acks不是"多数派阈值可配置"。 - producer 侧只有
0、1、all(-1)三档,见 ProducerConfig#L125-L144。 - 其中
acks=all的意思是:leader 要等 ISR 内所有副本确认后,才算这次写成功。
所以"只要一半以上"或者"只要一主一从",不是 producer 可以直接表达的 ack 语义。
Kafka 里更接近这个安全要求的组合是:
acks=all- 合理设置副本数
replication.factor - 合理设置
min.insync.replicas
例如:
- 副本数是 3
min.insync.replicas=2- producer 用
acks=all
这时 broker 至少要有 2 个 ISR 副本在线,才会接受写入。
幂等为什么必须依赖 acks=all,本质原因是 sequence 状态必须在故障切主后还能延续。
看一个最小例子:
A(seq=10)写到 leader 成功了,但还没复制到 follower。- 如果此时
acks=1,producer 已经收到成功响应。 - 紧接着 leader 宕机,新 leader 从没见过
A(seq=10)。 - producer 之后如果因为超时、重试、继续发送等动作再带着后续 sequence 来,新 leader 看到的 sequence 状态就可能和旧 leader 不一致。
这时你最怕的不是"单次请求失败",而是:
- producer 认为某个 sequence 已经成功过
- 新 leader 却没有那段历史
一旦这件事发生,broker 就无法可靠地区分:
- 这是重复 batch
- 这是丢失后补发的 batch
- 这是跳过前序 batch 的乱序 batch
所以 acks=all 不是为了"让顺序 magically 变强",而是为了让幂等依赖的那份 broker 状态在 leader 切换后仍然尽量一致。
换句话说:
max.in.flight决定"你允许多少并发请求在路上"。enable.idempotence决定"这些并发请求如何靠 sequence 保序"。acks=all决定"broker 返回成功时,这份 sequence 状态是否足够稳,能扛 leader 切换"。
生产端要怎么选
如果只看顺序性,生产端大致有两档策略。
方案一:最保守,靠串行化保序
- 用业务主键做 key,保证同一实体进入同一 partition。
- 设置
enable.idempotence=true。 - 设置
acks=all。 - 设置
max.in.flight.requests.per.connection=1。
优点:
- 最容易理解,也最不容易误配。
代价:
- 吞吐和链路利用率较低。
方案二:更高吞吐,靠 sequence 保序
- 用业务主键做 key,保证同一实体进入同一 partition。
- 设置
enable.idempotence=true。 - 设置
acks=all。 - 设置
max.in.flight.requests.per.connection为2..5。
这套配置在 Kafka 的设计里仍然可以保序,但这里的"保序"是有边界的:
- 保的是同一个 producer 会话、同一个 partition、同一条写入链路上的顺序。
- 靠的是 sequence 校验与去重,不是靠"物理上绝不并发"。
所以"多个 in-flight 也能保序"这句话本身没有错,但必须带上前提:
- 必须开启幂等。
- 必须
acks=all。 max.in.flight不能超过 Kafka 允许的范围。- 应用最终仍然只能在"同一 partition 顺序域"上谈顺序。
再讲消费端:broker 已经有序,不代表应用就有序
消费端最容易混淆的点,是把"poll 返回顺序正确"和"业务处理顺序正确"混为一谈。
Kafka 在拉取时,对单个 partition 的读取逻辑本身是顺序的:
CompletedFetch.nextFetchedRecord()会按 batch、record 依次取数据,并推进nextFetchOffset,见 CompletedFetch#L187-L243。FetchCollector.fetchRecords()只会在抓到的数据正好接着当前位置时才返回,并在返回前推进 position,见 FetchCollector#L163-L205。
但应用层很容易自己把顺序打乱,例如:
- 同一个 partition 的消息丢进线程池并发处理。
- 先完成的先写库,后完成的后写库。
这时 broker 没乱,consumer 也没乱,乱的是应用。
消费端真正该做的是:
text
for partition in records.partitions():
for record in records.records(partition):
process(record) # 必须按 partition 内顺序完成
commit(nextOffset(partition))
这里也要拆开理解两个问题:
records.records(partition)给你的,是这个 partition 的有序记录列表。for (record : records)只是把多个 partition 的记录列表拼起来遍历,不能当成"全局顺序流"。
源码上也能看出来:
ConsumerRecords.records(partition)返回某个 partition 的记录列表,见 ConsumerRecords#L66-L72。ConsumerRecords.iterator()只是把records.values()拼接起来,见 ConsumerRecords#L105-L106。
所以如果你在需要保序的场景里直接写:
text
for (record : records) {
dispatchToThreadPool(record)
}
那你拿到的不是"一个全局有序流",而是"多个 partition 片段的拼接结果"。
提交位点为什么一定要放在处理之后
因为 offset 的含义不是"我拉到了",而应该是"我已经处理完,下一次应该从哪里继续读"。
Kafka 的注释里写得很直接:
- 提交的 offset 应该是"下一条要读的消息"的 offset,见 KafkaConsumer#L285-L289。
- 官方也专门给了"按 partition 处理,然后提交
records.nextOffsets().get(partition)"的例子,见 KafkaConsumer#L263-L289。
还要注意一件事:
commitSync()默认提交的是 consumer 当前 position。- 这个 position 代表"poll 这边已经向前推进到哪了",不是"你异步业务线程真正处理到哪了"。
对应源码:
ClassicKafkaConsumer.commitSync()最终会提交subscriptions.allConsumed(),见 ClassicKafkaConsumer#L743-L759。allConsumed()直接取的是当前 position,见 SubscriptionState#L786-L793。
所以如果你是异步处理模型,最危险的错误就是:
poll()拉到 100 条- 立刻
commitSync() - 真正的业务线程还没处理完这 100 条
一旦进程中途挂掉,就可能"位点已经前进,但业务实际上没做完",这就不是重复消费,而是直接漏处理。
顺序、幂等、事务,到底是什么关系
这三个概念经常被混在一起,但作用并不相同。
幂等
幂等解决的是 producer 写入阶段的问题:
- retry 重复写
- 多个 in-flight 请求下的乱序写
它主要作用在"同一个 producer 会话写某个 partition"的链路上。
事务
事务主要解决的不是"顺序",而是 Kafka 内部的原子可见性和位点一致性。
按容易混淆的 3 个场景看,会更清楚:
-
场景 1:producer 只往 Kafka 写多条消息
-
目标:多个 topic / partition 上的写入,要么一起可见,要么一起不可见。
-
实现流程:
textbeginTransaction() send(record to p0) send(record to p3) send(record to p7) commitTransaction() / abortTransaction() -
本质:事务里的消息不是等到
commitTransaction()才写 broker,而是在send()时就会正常发往各个 partition leader;区别在于,这些写入会被标记为"属于当前事务",并被纳入同一组事务状态管理,见 KafkaProducer#L707-L713、TransactionManager#L457-L464。 -
commitTransaction()做的不是"开始写数据",而是先等待事务内未完成的 batch 全部 flush,再发送EndTxn(COMMIT),把这批已写入日志的事务消息统一标记为已提交,见 KafkaProducer#L825-L841、KafkaProducer#L856-L863、TransactionManager#L393-L413、TransactionManager#L914-L924。 -
abortTransaction()则分两部分处理:本地还没 flush 的消息直接放弃;已经写进 broker 的事务消息不会物理删除,而是通过EndTxn(ABORT)把整笔事务标成 abort,后续由read_committed消费者跳过,见 KafkaProducer#L867-L876、KafkaProducer#L890-L897、TransactionManager#L381-L389。
-
-
场景 2:consume -> process -> produce
-
目标:写到下游 Kafka 的结果消息,和上游消费位点的提交,要么一起成功,要么一起失败。
-
实现流程:
textbeginTransaction() consume upstream records process() send(downstream records) sendOffsetsToTransaction(consumedOffsets) commitTransaction() -
本质:这里不是把"处理结果"和"位点提交"分开做,而是把它们都并入同一个 Kafka 事务。
sendOffsetsToTransaction(...)会把消费位点也作为事务的一部分交给协调器,只有事务最终 commit,这些 offset 才算真正提交,见 KafkaProducer#L717-L735、KafkaProducer#L765-L776、TransactionManager#L424-L425、TransactionManager#L1241-L1266。 -
这里的"位点提交"指的是:这条应用已经告诉 Kafka,"上游这批消息我处理完了,下次不用再从这里重读"。所以它解决的不是 producer 单独一侧的问题,而是
consume -> process -> produce整条链路里,两边动作分离带来的不一致。 -
一个最小例子:
text读到上游 offset 100 处理完成 写下游结果 提交上游 next offset = 101 -
如果没有事务,下面两种故障都会出问题:
- 下游结果已经写到 Kafka 了,但应用还没来得及提交上游 offset 就挂了。重启后会再次消费 offset 100,于是同一份结果可能重复写出。
- 应用先把上游 offset=101 提交了,但下游结果还没成功写出去就挂了。重启后 consumer 会从 101 继续读,等于 offset 100 这条消息被"跳过处理"了。
-
sendOffsetsToTransaction(...)的作用,就是把"提交上游消费位点"也放进当前 producer 的事务里。这样最终要么"结果消息 + offset 提交"一起成功,要么一起失败回滚,而不是各做各的。
-
-
场景 3:consumer 读取事务消息
- 目标:只读到已经提交的事务结果,不读到 aborted 的半成品。
- 本质:broker 日志里可能已经存在那些 aborted 的事务消息,但
read_committed不会把它们暴露给应用。consumer 读取时会结合 abort 标记维护一组已中止事务的 producerId;如果某个 batch 属于已 abort 的事务,就直接跳过,见 CompletedFetch#L208-L223、CompletedFetch#L366-L367、CompletedFetch#L381-L383。
所以事务最擅长的是:
- Kafka -> Kafka
- 或者 consume Kafka -> produce Kafka
它不直接解决这些问题:
- 外部数据库和 Kafka 之间的原子一致性
- Redis、HTTP、RPC、第三方系统这些外部副作用的一致性
- 跨 partition 的顺序本身
所以事务和顺序的关系可以压成 3 句话:
- 幂等解决 producer 写入链路上的重复和乱序。
- 事务解决 Kafka 内部多条写入及位点提交的一致可见性。
- 真正的顺序作用域,仍然要靠"相同 key -> 相同 partition"来保证。
如果你在中间重分区、换 key、或者把同一 partition 的消息分发到多个无序 worker,事务并不能替你恢复顺序。
Kafka 源码里,这套逻辑是怎么落地的
1. 路由到哪个 partition
- 默认分区逻辑写在 ProducerConfig#L312-L320。
- 有 key 时按 key 选 partition。
- 没 key 时走 sticky partition。
sticky partition 的当前选择与切换逻辑在:
所以"无 key 也想保顺序"通常是不成立的,因为 sticky 分区会切换。
2. 生产端先在客户端排队
KafkaProducer.doSend()算出 partition 后,把记录 append 到RecordAccumulator,见 KafkaProducer#L1133-L1149。RecordAccumulator对每个 partition 维护一个Deque<ProducerBatch>,先尝试追加到最后一个 batch,不够再新建 batch 并addLast,见 RecordAccumulator#L313-L324 和 RecordAccumulator#L393-L401。
这说明同一个 partition 在 producer 侧一开始就是有序队列。
3. max.in.flight=1 时怎么强制串行
KafkaProducer把maxInflightRequests == 1转成guaranteeMessageOrder=true,见 KafkaProducer#L525-L547。Sender发送后会mutePartition,完成后再unmutePartition,见 Sender#L418-L425 和 Sender#L736-L738。
4. 多个 in-flight 时怎么靠 sequence 保序
- 配置校验写在 ProducerConfig#L604-L629。
- 客户端重试重排逻辑写在 RecordAccumulator#L542-L592。
- Broker sequence 校验写在 ProducerAppendInfo#L156-L198。
- duplicate 检测写在 UnifiedLog#L1397-L1406。
5. 最终为什么是"partition 内 offset 顺序"
UnifiedLog.append()在锁内分配单调递增 offset,然后写本地日志,见 UnifiedLog#L1143-L1169 和 UnifiedLog#L1256-L1267。
这就是 Kafka 顺序性的物理基础:
- 对一个 partition 来说,它最后就是一条按 offset 追加的日志。
最后收敛成可执行结论
如果你只关心"怎么用 Kafka 尽量稳地保序",可以直接按下面判断。
场景一:同一业务实体必须严格保序
- 用业务主键做 key。
- 让同一实体稳定落到同一 partition。
- 生产端用
enable.idempotence=true、acks=all、max.in.flight.requests.per.connection=1。 - 消费端按 partition 串行处理,处理完再提交位点。
场景二:同一业务实体要保序,同时还想要更高吞吐
- 仍然先保证 key 稳定。
- 生产端用
enable.idempotence=true、acks=all、max.in.flight.requests.per.connection=2..5。 - 接受 Kafka 是"靠 sequence 保序",不是"靠绝不并发保序"。
- 消费端仍然必须按 partition 串行完成业务处理。
场景三:消费后再生产,要端到端一致
- 不要只看 producer,要同时看 consumer 位点和下游输出。
- producer 至少开幂等;需要原子可见性时再用事务。
- consumer 用
read_committed,避免看到 aborted 结果。 - 全链路保持同样的顺序作用域,不要中途换 key 或随意重分区。
场景四:你真正要的是全局顺序
- Kafka 不擅长这个问题。
- 最直接的办法只能是单 partition。
- 代价是吞吐、扩展性、可用性都会明显下降。
批判性总结
这套设计的收益是明确的:
- Kafka 把"顺序"收敛到 partition 内追加日志,所以它能同时拿到局部顺序和整体扩展性。
- 当你愿意牺牲一些吞吐时,可以用
max.in.flight=1换最简单的顺序保证。 - 当你要更高吞吐时,可以用幂等、sequence 和 duplicate 检测继续保住 partition 内顺序。
这套设计的代价也同样明确:
- 顺序作用域被限制在 partition。
- 生产端、Broker、消费端、应用线程模型、位点提交时机,任一层处理不当都会破坏最终顺序。
- 事务、幂等、ack、重试这些机制叠加后,理解成本和排障成本都会升高。
所以更本质的建议只有一句:
- 不要先问"Kafka 能不能全局保序"。
- 要先问"我的业务里,哪些消息必须彼此有序"。
- 然后把这个顺序作用域,尽量精确地对齐到 partition key。