排查过程中最让人脑溢血的,往往不是什么惊天动地的内核 Bug,而是对基础架构的运行机制一知半解,最后被默认配置反噬。近期处理了一起极度典型的 Pulsar 跨机房同步(Geo-Replication)雪崩事故。故障现象是主可用区所有 Producer 突然大面积报 TimeoutException 和 NotEnoughBookiesException,消息写入 QPS 直接跌零。
最终结论先行: 远端机房专线网络抖动,导致 Geo-Replication 的内置同步游标(pulsar.repl.xxx)阻塞不动。由于该 Namespace 未配置 Backlog Quota(积压配额),本地机房数据无法被垃圾回收(GC),直接将 Bookie 磁盘撑到 95% 触发 ReadOnly 模式。更致命的是,前人部署时将 BookKeeper 的 Journal 和 Ledger 目录混布在同一块磁盘上,磁盘高水位触发的底层 Compaction 动作彻底打爆了 IOPS,导致 Journal fsync 延迟飙升至 5000ms+,最终引发全局雪崩。
不要以为“存算分离”就能包治百病,一旦连最基本的 IO 隔离和边界防御机制都不做,架构越高级,死得越难看。
案发现场:全线熔断与诡异的 p99 延迟
某次大促压测期间,监控大屏突然全线飘红。Pulsar 生产端的写入 P99 延迟从平时的 5ms 瞬间飙升到 5000ms 以上,紧接着大量 Producer 直接断开连接。
去 Broker 节点抓取日志,满屏都是创建 Ledger 失败的报错:
[pulsar-io-1-8] ERROR org.apache.pulsar.broker.service.ServerCnx - Failed to create ledger for topic persistent://tenant-a/ns-core/order-events
org.apache.pulsar.client.api.PulsarClientException$NotEnoughBookiesException: Not enough non-faulty bookies available
at org.apache.pulsar.client.impl.ConnectionHandler.handleConnectionError(ConnectionHandler.java:123)
看到 NotEnoughBookiesException,直觉告诉我底层存储层 BookKeeper 已经大规模掉线或进入了防御状态。立马切到 Bookie 节点查看状态,果然抓到了罪魁祸首:
[bookie-io-1-1] WARN org.apache.bookkeeper.bookie.LedgerDirsManager - Disk usage on /data/bookkeeper/ledgers is 0.96, which is above the threshold 0.95. Transitioning to ReadOnly mode.
[bookie-io-1-1] INFO org.apache.bookkeeper.bookie.Bookie - Bookie is running in ReadOnly mode.
Bookie 的 diskUsageThreshold 默认是 0.95。一旦磁盘使用率超过 95%,Bookie 会强行把自己设置为 ReadOnly 模式,拒绝所有 addEntry(写入)请求。当集群中处于 ReadOnly 的 Bookie 数量导致无法满足 Topic 的 EnsembleSize 和 WriteQuorum 时,Broker 就会抛出 NotEnoughBookies。
抽丝剥茧:游标为什么不走?IO 为什么夯死?
磁盘写满了?不应该。这批机器配了 2TB 的 SSD,按理说以当时的业务吞吐量和 TTL(配置的 3 天过期),磁盘利用率常年徘徊在 40% 左右。
通过命令行查看涉事 Topic 的状态:
bin/pulsar-admin topics stats persistent://tenant-a/ns-core/order-events
输出结果直接揭示了问题核心:
"replication": {
"us-west": {
"msgRateIn": 0.0,
"msgRateOut": 0.0,
"replicationBacklog": 45000000,
"connected": false,
"replicationDelayInSeconds": 86400
}
}
Pulsar 的 Geo-Replication 底层其实是非常朴素的机制:它本质上是一个跨机房的特殊 Cursor(订阅游标),名字通常叫 pulsar.repl.。
只要这个游标不往前走(比如远端机房失联、专线抖动),这部分数据就会被永远 Retain(保留)在 BookKeeper 中,无论你设置的 TTL 是多少。TTL 只能清理所有游标都已经消费过的数据。
这就解释了磁盘为什么会满。但这里还有一个极度不合理的现象:在磁盘逼近 95% 的过程中,为什么集群的读写延迟会出现剧烈抖动?
我调出了底层系统的 iostat -x 1,发现 await 和 svctm 指标高得离谱,util 稳定 100%。再看 BookKeeper 的配置文件 bookkeeper.conf:
journalDirectory=/data/bookkeeper/shared-disk
ledgerDirectories=/data/bookkeeper/shared-disk
看到这两行配置,我血压直接上来了。把 Journal(WAL 日志,要求极低延迟的顺序写)和 Ledger(数据文件,包含大量随机读写和 Compaction 动作)放在同一个物理挂载点下,这是教科书级别的反面教材。
当磁盘空间吃紧时,BookKeeper 的 Garbage Collector 和 Compaction 线程会疯狂启动,试图合并碎片、清理数据来腾出空间。大量的后台 IO 瞬间榨干了这块 SSD 的带宽,导致主干流程中处理 addEntry 的 Journal fsync 动作被强行阻塞。Journal 刷盘慢了,Pulsar Producer 端的延迟自然就飙升到了 5000ms,甚至超时熔断。
止血与防御:如何彻底根除这类隐患
处理这种烂摊子,第一步永远是先恢复服务,第二步是填补架构上的防御漏洞。
1. 紧急止血:强行干掉阻塞的游标并清理磁盘 既然跨机房同步已经断了,且本地写盘都成问题了,果断舍弃远端同步进度。通过强制取消订阅或卸载数据,释放 Bookie 空间:
# 临时绕过限制,先让本地可用
bin/pulsar-admin namespaces set-retention tenant-a/ns-core --size 10G --time 1h
# 如果确认远端数据可以直接丢弃同步,清理 replication backlog
bin/pulsar-admin persistent unsubscribe persistent://tenant-a/ns-core/order-events -s "pulsar.repl.us-west"
随着游标被强制推进,BookKeeper 的 GC 终于开始回收空间,磁盘使用率跌回 50%,Bookie 退出 ReadOnly 模式,集群恢复写入。
2. 核心防御:配置 Backlog Quota(防御性编程思想在运维端的体现) 永远不要信任下游和网络。必须强制设置 Namespace 级别的积压配额。当跨机房同步或本地消费阻塞导致积压达到阈值时,直接干掉旧数据,保集群可用性。
# 设置最大积压 50G,超过则按 producer_request_hold (阻塞生产) 或 producer_exception (抛出异常),推荐直接丢弃旧数据 consumer_backlog_eviction 保核心链路
bin/pulsar-admin namespaces set-backlog-quota tenant-a/ns-core \
--limit 50G \
--policy consumer_backlog_eviction
注:对于 Geo-Replication,如果网络断开,consumer_backlog_eviction 会强行推进 replication cursor,牺牲远端数据完整性来保住本地存储不被撑爆。
3. 物理隔离:存储层 I/O 隔离 把机器停机维护,强制将 Journal 目录迁移到独立的高性能 NVMe 盘,Ledger 目录放到容量更大的普通 SSD 上。
# bookkeeper.conf 正确姿势
journalDirectory=/data/nvme/bookkeeper/journal
ledgerDirectories=/data/ssd/bookkeeper/ledgers
排查清单与同类问题速查
- Bookie ReadOnly 状态检查
- 现象:Broker 报
NotEnoughBookiesException。 -
动作:检查 Bookie 日志中是否有
Transitioning to ReadOnly mode。排查diskUsageThreshold(默认 0.95) 与diskUsageWarnThreshold(默认 0.90) 的触发情况。 -
隐藏的积压游标(Cursor)排查
- 现象:磁盘满但实际业务消费已经最新。
-
动作:执行
pulsar-admin topics stats,重点检查subscriptions下是否有未消费完的游标,特别注意pulsar.repl.xxx(跨机房复制游标)和pulsar.dedup(去重游标,如果开启了消息去重)。 -
I/O 争用与 Journal 延迟检查
- 现象:
pulsar_storage_write_latency_le_*指标异常,或 P99 延迟极高。 -
动作:通过
bookkeeper_server_ADD_ENTRY_latency监控确认。务必检查journalDirectory和ledgerDirectories是否挂载在不同的物理磁盘上,防止 Compaction 冲爆 Journal 的顺序写 Fsync。 -
防御性 Quota 配置审核
- 动作:所有生产环境 Namespace 必须配置
BacklogQuota。不要裸奔,没有配额限制的集群,被上游乱写或下游阻塞打爆只是时间问题。