乐趣区

关于pulsar:BIGO-的-Pulsar-性能调优实战上

作者:陈航 @BIGO,内容由 StreamNative 邀请发动。

1. 背景

大家在应用 / 压测 Pulsar 时,可能会遇到如下问题:

  1. Pulsar broker 节点之间出入流量不平衡
  2. Pulsar broker 端 Cache 命中率低,导致大量读申请进入 bookie,且读性能比拟差
  3. 压测的时候经常出现 broker direct memory OOM(内存溢出)
  4. bookie 呈现 direct memory OOM 导致过程挂掉
  5. bookie 节点之间出入流量不平衡,且常常抖动
  6. 当 Journal 盘为 HDD 时,尽管敞开了 fsync,然而 bookie add entry 99th latency 仍旧很高,写入性能很差
  7. 当大量读申请进入 bookie 时,呈现写被反压,add entry latency 回升
  8. Pulsar client 经常出现“lookup timeout exception”
  9. ZooKeeper 读写提早过高导致整个 Pulsar 集群不稳固
  10. 应用 reader API(eg. flink) 生产 Pulsar topic 时,生产速度很慢(Pulsar 2.5.2 之前版本)

当 Journal/Ledger 盘为机械硬盘(HDD)时,问题 4、5、6、7 体现得尤为重大。这些问题直观来看,是磁盘不够快造成的,如果 Journal/Ledger 盘读写速度足够快,就不会呈现音讯在 Direct Memory 中沉积,也就不会有一系列 OOM 的产生。因为大部分音讯队列生产零碎中,Journal 盘和 Ledger 盘都是 SSD 并不是太广泛,那么有没有可能在 Pulsar / BookKeeper 上做一些参数 / 策略的优化,让 HDD 也能达到近似于 SSD 的性能呢?

本文接下来的局部次要介绍 Pulsar / BookKeeper 中和性能相干的局部,并提出一些性能调优的倡议(这些性能调优曾经在 BIGO 生产零碎中稳固运行,并取得了不错的收益)。因为篇幅起因,本次性能调优系列分为两局部,上半局部次要介绍 Pulsar broker 的性能调优,下半局部次要介绍 BookKeeper 与 Pulsar 联合过程中的性能调优。

2. 环境部署与监控

2.1 环境部署

因为 BookKeeper 和 Pulsar Broker 重度依赖 ZooKeeper,为了保障 Pulsar 的稳固,须要保障 ZooKeeper Read/Write 低提早。此外,BookKeeper 是 IO 密集型工作,为了防止 IO 之间相互烦扰,Journal/Ledger 请放在独立磁盘上。总结如下:

1. Bookie Journal/Ledger 目录放在独立磁盘上;
2. 当 Journal/Ledger 目录的磁盘为 HDD 时,ZooKeeper dataDir/dataLogDir 不要和 Journal/Ledger 目录放在同一块磁盘上。

BookKeeper 和 Pulsar Broker 均依赖 direct memory,而且 BookKeeper 还依赖 PageCache 进行数据读写减速,所以正当的内存调配策略也是至关重要的。Pulsar 社区的 sijie 举荐的内存调配策略如下:

  • OS: 1 ~ 2GB
  • JVM: 1/2

    • heap: 1/3
    • direct memory: 2/3
  • PageCache: 1/2

假如机器物理内存为 128G,bookie 和 broker 混布,内存调配如下:

  • OS: 2GB
  • broker: 31GB

    • heap: 10GB
    • direct memory: 21GB
  • bookie: 32GB

    • heap: 10GB
    • direct memory: 22GB
  • PageCache: 63GB

2.2 Monitor:性能调优,监控后行

为了更加直观地发现零碎性能瓶颈,咱们须要为 Pulsar/BookKeeper 搭建一套欠缺的监控体系,确保每一个环节都有相干指标上报,当出现异常(包含但不限于性能问题)时,可能通过相干监控指标疾速定位性能瓶颈,并制订相应解决方案。

Pulsar / BookKeeper 都提供了 Prometheus 接口,相干统计指标能够间接应用 Http 形式获取并间接对接 Prometheus / Grafana。感兴趣的同学能够间接依照 Pulsar Manager 的领导进行装置。

须要重点关注的 Metric 如下:

  1. Pulsar Broker

    • jvm heap/gc
    • bytes in per broker
    • message in per broker
    • loadbalance
    • broker 端 Cache 命中率
    • bookie client quarantine ratio
    • bookie client request queue
  2. BookKeeper

    • bookie request queue size
    • bookie request queue wait time
    • add entry 99th latency
    • read entry 99th latency
    • journal create log latency
    • ledger write cache flush latency
    • entry read throttle
  3. ZooKeeper

    • local/global ZooKeeper read/write request latency

有一些 Metric 在下面 repo 中没有提供相应 Grafana 模板,大家能够本人增加 PromQL 进行配置。

3. Pulsar broker 端性能调优

对于 Pulsar Broker 的性能调优,次要分为如下六个方面:

  1. 流量平衡
  • broker 之间流量平衡
  • bookie 节点之间的入流量平衡
  1. 限流
  • broker 接管音讯须要做流控,避免突发洪峰流量导致 broker direct memory OOM
  • broker 发送音讯给 consumer/reader 时须要做流控,避免一次发送太多音讯造成 consumer/reader 频繁 GC
  1. 进步 Cache 命中率
  2. 保障 ZooKeeper 读写低提早
  3. 敞开 auto bundle split,保证系统稳固

3.1 流量平衡

3.1.1 broker 之间入流量平衡

broker 之间入流量平衡,可能进步 broker 节点的利用率,进步 Broker Cache 命中率,升高 broker OOM 概率。这一部分内容次要波及到 Pulsar bundle rebalance 相干常识。

Namespace Bundle 构造如下,每个 namespace(命名空间)由肯定数量的 bundle 组成,该 namespace 下的所有 topic 均通过 hash 形式映射到惟一 bundle 上,而后 bundle 通过 load/unload 形式加载 / 卸载到提供服务的 broker 上。如果某个 broker 上没有 bundle 或者 bundle 数量比其余 broker 少,那么这台 broker 的流量就会比其余 broker 低。

现有的 / 默认的 bundle rebalance 策略(OverloadShedder)为:每隔一分钟统计集群中所有 broker 的 CPU、Memory、Direct Memory、BindWith In、BindWith Out 占用率的最大值是否超过阈值(默认为 85%),如果超过阈值,则将肯定数量大入流量的 bundle 从该 broker 中卸载掉,而后由 leader 决定将被卸载掉的 bundle 从新加载到负载最低的 broker 上。

这个策略存在的问题是:

  1. 默认阈值在比拟难达到,很容易导致集群中大部分流量都集中在几个 broker 上;
  2. 阈值调整规范难以确定,受其余因素影响较大,特地是这个节点上部署有其余服务的状况下;
  3. broker 重启后,长时间没有流量平衡到该 broker 上,因为其余 broker 节点均没有达到 bundle unload 阈值。

为此,咱们开发了一个基于均值的从新均衡(rebalance)策略,并反对 CPU、Memory、Direct Memory、BindWith In、BindWith Out 权重配置,相干策略请参见 PR-6772。

该策略在 Pulsar 2.6.0 版本开始反对,默认敞开,能够在 broker.conf 中批改如下参数开启:

loadBalancerLoadSheddingStrategy=org.apache.pulsar.broker.loadbalance.impl.ThresholdShedder

咱们能够通过如下参数来准确管制不同采集指标的权重:

# The broker resource usage threshold.
# When the broker resource usage is greater than the pulsar cluster average resource usage,
# the threshold shedder will be triggered to offload bundles from the broker.
# It only takes effect in ThresholdSheddler strategy.
loadBalancerBrokerThresholdShedderPercentage=10

# When calculating new resource usage, the history usage accounts for.
# It only takes effect in ThresholdSheddler strategy.
loadBalancerHistoryResourcePercentage=0.9

# The BandWithIn usage weight when calculating new resource usage.
# It only takes effect in ThresholdShedder strategy.
loadBalancerBandwithInResourceWeight=1.0

# The BandWithOut usage weight when calculating new resource usage.
# It only takes effect in ThresholdShedder strategy.
loadBalancerBandwithOutResourceWeight=1.0

# The CPU usage weight when calculating new resource usage.
# It only takes effect in ThresholdShedder strategy.
loadBalancerCPUResourceWeight=1.0

# The heap memory usage weight when calculating new resource usage.
# It only takes effect in ThresholdShedder strategy.
loadBalancerMemoryResourceWeight=1.0

# The direct memory usage weight when calculating new resource usage.
# It only takes effect in ThresholdShedder strategy.
loadBalancerDirectMemoryResourceWeight=1.0

# Bundle unload minimum throughput threshold (MB), avoiding bundle unload frequently.
# It only takes effect in ThresholdShedder strategy.
loadBalancerBundleUnloadMinThroughputThreshold=10

3.1.2 平衡 bookie 节点之间的入流量

bookie 节点入流量监控如下图所示,咱们会发现:

  1. bookie 节点之间入流量并不是平均的,最高流量节点和最低流量节点可能相差几百 MB/s
  2. 在高入流量状况下,某写节点入流量可能会呈现周期性上涨和降落,降落周期为 30 分钟

这些问题的影响是:bookie 入流量不平衡,导致 BookKeeper 集群利用率降落,且容易呈现抖动。

呈现这个问题的起因在于:bookie client 对 bookie 写申请的熔断策略粒度太大。

咱们先来回顾一下 Pulsar broker 写入 bookie 的策略:

当 broker 接管到 producer 发送的 message 时,首先会将音讯寄存在 broker 的 direct memory 中,而后调用 bookie client 依据配置的 (EnsembleSize,WriteQuorum,AckQuorum) 策略将 message 以 pipeline 形式发送给 bookies。bookie client 每分钟会统计各 bookie 写入的失败率(包含写超时等各类异样)。默认状况下,当失败率超过 5 次 / 分钟时,就将这台 bookie 关入小黑屋 30 分钟,防止继续向出现异常的 bookie 写入数据,从而保障 message 写入成功率。

这个熔断策略存在的问题是:某台 bookie 负载(流量)很高时,所有写入到该 bookie 的音讯有可能同时会变慢,所有 bookie client 可能同时收到写入异样,如写入超时等,那么所有 bookie client 会同时把这台 bookie 关入小黑屋 30 分钟,等到 30 分钟之后又同时退出可写入列表中。这就导致了 bookie 节点入流量周期性上涨和降落。

为了解决该问题,咱们引入了基于概率的 qurantee 机制,当 bookie client 写入音讯出现异常时,并不是间接将这台 bookie 关入小黑屋,而是基于概率决定是否 qurantee。这一 qurantee 策略能够防止所有 bookie client 同时将同一台 bookie 关入小黑屋,防止 bookie 入流量抖动。相干 PR 请参见:BookKeeper PR-2327,因为代码没有合并和公布到 bookie 主版本,大家如果想应用该性能,须要本人独立编译代码。

从 BIGO 实际测试来看,该性能将 bookie 节点之间入流量标准差从 75MB/s 升高到 40MB/s。

3.2 限流

3.2.1 broker direct memory OOM(内存溢出)

在生产环境中,在高吞吐场景下,咱们常常遇到broker direct memory OOM,导致 broker 过程挂掉。这里的起因可能是底层 bookie 写入变慢,导致大量数据积压在 broker direct memory 中。producer 发送的音讯在 broker 中的处理过程如下图所示:

在生产环境中,咱们不能保障底层 bookie 始终保持非常低的写提早,所以须要在 broker 层做更细粒度的限流。Pulsar 社区的鹏辉开发了一个限流的性能,限流逻辑如下图所示:

在 Pulsar 2.5.1 版本中已公布,请参见 PR-6178。

3.2.2 consumer 耗费大量内存

当 producer 端以 batch 模式发送音讯时,consumer 端往往会占用过多内存导致频繁 GC,监控上的体现是:这个 topic 的出流量在 consumer 启动时飙升,而后逐步回归到失常程度。

这个问题的起因须要联合 consumer 端的生产模式来看。

当 consumer 调用 receive 接口生产一条音讯时,它会间接从本地的 receiverQueue 中申请一条音讯,如果 receiverQueue 中还有音讯能够获取,则间接将音讯返回给 consumer 端,并更新 availablePermit,当 availablePermit < receiverQueueSize/2 时,Pulsar client 会将 availablePermit 发送给 broker,通知 broker 须要 push 多少条音讯过去;如果 receiverQueue 中没有音讯能够获取,则期待 / 返回失败,直到 receiverQueue 收到 broker 推送的音讯才将 consumer 唤醒。

broker 收到 availablePermit 之后,会从 broker Cache/bookie 中读取 max(availablePermit, batchSize) 条 entry,并发送给 consumer 端。解决逻辑如下图所示:

这里的问题是:当 producer 开启 batch 模式发送,一个 entry 蕴含多条音讯,然而 broker 解决 availablePermit 申请依然是依照一条音讯一个 entry 来解决,从而导致 broker 一次性将远超于 availiablePermit 数量的音讯(availiablePermit vs. availiablePermit * batchSize)发送给 consumer,引起 consumer 占用内存暴涨,引发频繁 GC,升高生产性能。

为了解决 consumer 端内存暴涨问题,咱们在 broker 端统计每个 topic 均匀 entry 蕴含的音讯数(avgMessageSizePerEntry),当接管到 consumer 申请的 availablePermit 时,将其换算成须要发送的 entry 大小,而后从 broker Cache/bookie 中拉取相应数量的 entry,而后发送给 consumer。解决逻辑如下图所示:

这个性能在 Pulsar 2.6.0 中已公布,默认是敞开的,大家能够通过如下开关启用该性能:

# Precise dispatcher flow control according to history message number of each entry
preciseDispatcherFlowControl=true

3.3 进步 Cache 命中率

Pulsar 中有多层 Cache 减速 message 的读性能,次要包含:

  • broker Cache
  • bookie write Cache(Memtable)
  • bookie read Cache
  • OS PageCache

本章次要介绍 broker Cache 的运行机制和调优计划,bookie 侧的 Cache 调优放在下一章介绍。

当 broker 收到 producer 发送给某个 topic 的音讯时,首先会判断该 topic 是否有 active cursor,如果有,则将收到的音讯写入该 topic 对应的 Cache 中;否则,不写入 Cache。解决流程如下图所示:

判断是否有 Active Cursor 须要同时满足以下两个条件:

1. 有 durable cursor
2. cursor 的 lag 在 `managedLedgerCursorBackloggedThreshold` 范畴内

因为 reader 应用 non-durable cursor 进行生产,所以 producer 写入的音讯不会进入 broker Cache,从而导致大量申请落到 bookie 上,性能有所损耗。streamnative/pulsar-flink-connector 应用的是 reader API 进行生产,所以同样存在生产性能低的问题。

这个问题由咱们 BIGO 音讯队列团队的赵荣生同学修复,将 durable cursor 从 Active Cursor 判断条件中删除,详情请见 PR-6769,这个 feature 在 Pulsar 2.5.2 公布,有遇到相干性能问题的同学请降级 Pulsar 版本到 2.5.2 以上。

此外,咱们针对 topic 的每个 subscription 增加了 Cache 命中率监控,不便进行生产性能问题定位,后续会奉献到社区。

3.3.1 Tailing Read

那么对于曾经在 broker Cache 中的数据,在 tailing read 场景下,咱们怎么进步 Cache 命中率,升高从 bookie 读取数据的概率呢?咱们的一个思路是尽可能让数据从 broker Cache 中读取,为了保障这一点,咱们能够从两个中央着手优化:

  1. 管制断定为 active cursor 的最大 lag 范畴,默认是 1000 个 entry,由如下参数控:
# Configure the threshold (in number of entries) from where a cursor should be considered 'backlogged'
# and thus should be set as inactive.
managedLedgerCursorBackloggedThreshold=1000

Active Cursor 的断定如下图所示。

  1. 管制 broker Cache 的 eviction 策略,目前 Pulsar 中只反对默认 evict 策略,有需要的同学能够自行扩大。默认 evict 策略由如下参数管制:
# Amount of memory to use for caching data payload in managed ledger. This memory
# is allocated from JVM direct memory and it's shared across all the topics
# running  in the same broker. By default, uses 1/5th of available direct memory
managedLedgerCacheSizeMB=

# Whether we should make a copy of the entry payloads when inserting in cache
managedLedgerCacheCopyEntries=false

# Threshold to which bring down the cache level when eviction is triggered
managedLedgerCacheEvictionWatermark=0.9

# Configure the cache eviction frequency for the managed ledger cache (evictions/sec)
managedLedgerCacheEvictionFrequency=100.0

# All entries that have stayed in cache for more than the configured time, will be evicted
managedLedgerCacheEvictionTimeThresholdMillis=1000

3.3.2 Catchup Read

对于 Catchup Read 场景,broker Cache 大概率会失落,所有的 read 申请都会落到 bookie 上,那么有没有方法进步读 bookie 的性能呢?

broker 向 bookie 发送读取申请是批量发送的,最大 batch 大小有 dispatcherMaxReadBatchSize 管制,默认是 100 个 entry。

# Max number of entries to read from bookkeeper. By default it is 100 entries.
dispatcherMaxReadBatchSize=100

一次读取的 batchSize 越大,底层 bookie 从磁盘读取的效率越高,均摊到单个 entry 的 read latency 就越低。然而如果过大也会造成 batch 读取提早减少,因为底层 bookie 读取操作时每次读一条 entry,而且是同步读取。这一部分的读取调优放在下一章 《BookKeeper 性能调优实战(下)》 介绍。

3.4 保障 ZooKeeper 读写低提早

因为 Pulsar 和 BookKeeper 都是重大依赖 ZooKeeper 的,如果 ZooKeeper 读写提早减少,就会导致 Pulsar 服务不稳固。所以须要优先保障 ZooKeeper 读写低提早。倡议如下:

  1. 在磁盘为 HDD 状况下,ZooKeeper dataDir/dataLogDir 不要和其余耗费 IO 的服务 (如 bookie Journal/Ledger 目录) 放在同一块盘上(SSD 除外);
  2. ZooKeeper dataDir 和 dataLogDir 最好可能放在两块独立磁盘上(SSD 除外);
  3. 监控 broker/bookie 网卡利用率,防止因为网卡打满而造成和 ZooKeeper 失联。

3.5 敞开 bundle autosplit,保证系统稳固

pulsar bundle split 是一个比拟重的操作,会造成连贯到这个 bundle 上的所有 producer/consumer/reader 连贯断开并重连。个别状况下,触发 bundle auto split 的起因是这个 bundle 的压力比拟大,须要切分成两个 bundle,将流量摊派到其余 broker,来升高这个 bundle 的压力。管制 bundle auto split 的参数如下:

# enable/disable namespace bundle auto split
loadBalancerAutoBundleSplitEnabled=true

# enable/disable automatic unloading of split bundles
loadBalancerAutoUnloadSplitBundlesEnabled=true

# maximum topics in a bundle, otherwise bundle split will be triggered
loadBalancerNamespaceBundleMaxTopics=1000

# maximum sessions (producers + consumers) in a bundle, otherwise bundle split will be triggered
loadBalancerNamespaceBundleMaxSessions=1000

# maximum msgRate (in + out) in a bundle, otherwise bundle split will be triggered
loadBalancerNamespaceBundleMaxMsgRate=30000

# maximum bandwidth (in + out) in a bundle, otherwise bundle split will be triggered
loadBalancerNamespaceBundleMaxBandwidthMbytes=100

当触发 bundle auto split 时 broker 的负载比拟高,敞开这个 bundle 上的 producer/consumer/reader 的连贯就会变慢,并且 bundle split 的耗时也会变长,就很容易造成 client 端(producer/consumer/reader)连贯超时而失败,触发 client 端主动重连,造成 Pulsar/Pulsar client 不稳固。

对于生产环境,咱们的倡议是:事后为每个 namespace 调配好 bundle 数,并敞开 bundle auto split 性能。如果在运行过程中发现某个 bundle 压力过大,能够在流量低峰期进行手动 bundle split,升高对 client 端的影响。对于事后调配的 bundle 数量不宜太大,bundle 数太多会给 ZooKeeper 造成比拟大的压力,因为每一个 bundle 都要定期向 ZooKeeper 汇报本身的统计数据。

4. 总结

本篇从性能调优角度介绍了 Pulsar 在 BIGO 实际中的一些优化计划,次要分为环境部署、流量平衡、限流措施、进步 Cache 命中率、保障 Pulsar 稳定性等 5 个方面,并深刻介绍了 BIGO 音讯队列团队在进行 Pulsar 生产落地过程中的一些教训。

本篇次要解决了开篇提到了问题 1、2、5、7、8、9 等 6 个问题,对于问题 3,咱们提出了一个缓解计划,但并没有指出 Pulsar broker OOM 的根本原因,这个问题须要从 BookKeeper 角度来解决,剩下的问题都和 BookKeeper 相干。

因为 Pulsar 应用分层存储架构,底层的 BookKeeper 仍须要进行一些列调优来配合下层 Pulsar 以充分发挥高吞吐、低提早性能,下一篇将从 BookKeeper 性能调优角度介绍 BIGO 的一些实践经验。

对于作者

陈航,BIGO 音讯平台团队负责人,负责承载大规模服务与利用的集中公布 - 订阅音讯平台的创立与开发。他将 Apache Pulsar 引入到 BIGO 音讯平台,并买通上下游零碎,如 Flink、ClickHouse 和其余实时举荐与剖析零碎。他目前聚焦 Pulsar 性能调优、新性能开发及 Pulsar 生态集成方向。

退出移动版