背景
在人工智能技術(shù)的支持下察署,BIGO 基于視頻的產(chǎn)品和服務(wù)受到廣泛歡迎,在 150 多個(gè)國家/地區(qū)擁有用戶峻汉,其中包括 Bigo Live(直播)和 Likee(短視頻)箕母。Bigo Live 在 150 多個(gè)國家/地區(qū)興起储藐,Likee 有 1 億多用戶,并在 Z 世代中很受歡迎嘶是。
隨著業(yè)務(wù)的迅速增長钙勃,BIGO 消息隊(duì)列平臺(tái)承載的數(shù)據(jù)規(guī)模出現(xiàn)了成倍增長,下游的在線模型訓(xùn)練聂喇、在線推薦辖源、實(shí)時(shí)數(shù)據(jù)分析、實(shí)時(shí)數(shù)倉等業(yè)務(wù)對(duì)消息的實(shí)時(shí)性和穩(wěn)定性提出了更高的要求希太。
BIGO 消息隊(duì)列平臺(tái)使用的是開源 Kafka克饶,然而隨著業(yè)務(wù)數(shù)據(jù)量的成倍增長、消息實(shí)時(shí)性和系統(tǒng)穩(wěn)定性要求不斷提高誊辉,多個(gè) Kafka 集群的維護(hù)成本越來越高矾湃,主要體現(xiàn)在:
- 數(shù)據(jù)存儲(chǔ)和消息隊(duì)列服務(wù)綁定,集群擴(kuò)縮容/分區(qū)均衡需要大量拷貝數(shù)據(jù)堕澄,造成集群性能下降
- 當(dāng)分區(qū)副本不處于 ISR(同步)狀態(tài)時(shí)邀跃,一旦有 broker 發(fā)生故障,可能會(huì)造成丟數(shù)或該分區(qū)無法提供讀寫服務(wù)
- 當(dāng) Kafka broker 磁盤故障/使用率過高時(shí)蛙紫,需要進(jìn)行人工干預(yù)
- 集群跨區(qū)域同步使用 KMM(Kafka Mirror Maker)拍屑,性能和穩(wěn)定性難以達(dá)到預(yù)期
- 在 catch-up 讀場景下,容易出現(xiàn) PageCache 污染坑傅,造成讀寫性能下降
- 雖然 Kafka 的 topic partition 是順序?qū)懭虢┏郏钱?dāng) broker上有成百上千個(gè)topic partition 時(shí),從磁盤角度看就變成了隨機(jī)寫入唁毒,此時(shí)磁盤讀寫性能會(huì)隨著 topic partition 數(shù)量的增加而降低蒜茴,因此 Kafka broker 上存儲(chǔ)的 topic partition 數(shù)量是有限制的
- 隨著 Kafka 集群規(guī)模的增長,Kakfa 集群的運(yùn)維成本急劇增長浆西,需要投入大量的人力進(jìn)行日常運(yùn)維粉私。在 BIGO,擴(kuò)容一臺(tái)機(jī)器到 Kafka 集群并進(jìn)行分區(qū)均衡室谚,需要 0.5人/天毡鉴;縮容一臺(tái)機(jī)器需要 1 人/天
為了提高消息隊(duì)列實(shí)時(shí)性崔泵、穩(wěn)定性和可靠性秒赤,降低運(yùn)維成本,我們重新考慮了 Kafka 架構(gòu)設(shè)計(jì)上的不足憎瘸,調(diào)研能否從架構(gòu)設(shè)計(jì)上解決這些問題入篮,滿足當(dāng)前的業(yè)務(wù)要求。
下一代消息流平臺(tái):Pulsar
Apache Pulsar 是 Apache 軟件基金會(huì)頂級(jí)項(xiàng)目幌甘,是下一代云原生分布式消息流平臺(tái)潮售,集消息痊项、存儲(chǔ)、輕量化函數(shù)式計(jì)算為一體酥诽。Pulsar 于 2016 年由 Yahoo 開源并捐贈(zèng)給 Apache 軟件基金會(huì)進(jìn)行孵化鞍泉,2018 年成為Apache 軟件基金會(huì)頂級(jí)項(xiàng)目。
Pulsar 采用計(jì)算與存儲(chǔ)分離的分層架構(gòu)設(shè)計(jì)肮帐,支持多租戶咖驮、持久化存儲(chǔ)、多機(jī)房跨區(qū)域數(shù)據(jù)復(fù)制训枢,具有強(qiáng)一致性托修、高吞吐以及低延時(shí)的高可擴(kuò)展流數(shù)據(jù)存儲(chǔ)特性。
Pulsar 吸引我們的主要特性如下:
- 線性擴(kuò)展:能夠無縫擴(kuò)容到成百上千個(gè)節(jié)點(diǎn)
- 高吞吐:已經(jīng)在 Yahoo 的生產(chǎn)環(huán)境中經(jīng)受了考驗(yàn)恒界,支持每秒數(shù)百萬消息的 發(fā)布-訂閱(Pub-Sub)
- 低延遲:在大規(guī)模的消息量下依然能夠保持低延遲(小于 5 ms)
- 持久化機(jī)制:Plusar 的持久化機(jī)制構(gòu)建在 Apache BookKeeper 上睦刃,提供了讀寫分離
- 讀寫分離:BookKeeper 的讀寫分離 IO 模型極大發(fā)揮了磁盤順序?qū)懶阅埽瑢?duì)機(jī)械硬盤相對(duì)比較友好十酣,單臺(tái) bookie 節(jié)點(diǎn)支撐的 topic 數(shù)不受限制
Apache Pulsar 的架構(gòu)設(shè)計(jì)解決了我們使用 Kafka 過程中遇到的各種問題涩拙,并且提供了很多非常棒的特性,如多租戶婆誓、消息隊(duì)列和批流融合的消費(fèi)模型吃环、強(qiáng)一致性等。
為了進(jìn)一步加深對(duì) Apache Pulsar 的理解洋幻,衡量 Pulsar 能否真正滿足我們生產(chǎn)環(huán)境大規(guī)模消息 Pub-Sub 的需求郁轻,我們從 2019 年 12 月份開始進(jìn)行了一系列壓測工作。由于我們使用的是機(jī)械硬盤文留,沒有 SSD好唯,在壓測過程中遇到了一些列性能問題,非常感謝 StreamNative 同學(xué)的幫助燥翅,感謝斯杰骑篙、翟佳、鵬輝的耐心指導(dǎo)和探討森书,經(jīng)過一系列的性能調(diào)優(yōu)靶端,不斷提高 Pulsar 的吞吐和穩(wěn)定性。
經(jīng)過 3~4 個(gè)月的壓測和調(diào)優(yōu)凛膏,2020 年 4 月份我們正式在生產(chǎn)環(huán)境中使用 Pulsar 集群杨名。我們采用 bookie 和 broker 在同一個(gè)節(jié)點(diǎn)的混部模式,逐步替換生產(chǎn)環(huán)境的 Kafka 集群猖毫。截止到目前為止台谍,生產(chǎn)環(huán)境中 Pulsar 集群規(guī)模為十幾臺(tái),日處理消息量為百億級(jí)別吁断,并且正在逐步擴(kuò)容和遷移 Kafka 流量到 Pulsar 集群趁蕊。
壓測/使用 Pulsar 遇到的問題
大家在使用/壓測 Pulsar 時(shí)坞生,可能會(huì)遇到如下問題:
- Pulsar broker 節(jié)點(diǎn)負(fù)載不均衡。
- Pulsar broker 端 Cache 命中率低掷伙,導(dǎo)致大量讀請(qǐng)求進(jìn)入 bookie是己,且讀性能比較差。
- 壓測時(shí)經(jīng)常出現(xiàn) broker 內(nèi)存溢出現(xiàn)象(OOM)任柜。
- Bookie 出現(xiàn) direct memory OOM 導(dǎo)致進(jìn)程掛掉赃泡。
- Bookie 節(jié)點(diǎn)負(fù)載不均衡,且經(jīng)常抖動(dòng)乘盼。
- 當(dāng) Journal 盤為 HDD 時(shí)升熊,雖然關(guān)閉了 fsync,但是 bookie add entry 99th latency 依舊很高绸栅,寫入性能較差级野。
- 當(dāng) bookie 中有大量讀請(qǐng)求時(shí),出現(xiàn)寫被反壓粹胯,add entry latency 上升蓖柔。
- Pulsar client 經(jīng)常出現(xiàn)“Lookup Timeout Exception”。
- ZooKeeper 讀寫延遲過高導(dǎo)致整個(gè) Pulsar 集群不穩(wěn)定风纠。
- 使用 reader API(eg. pulsar flink connector) 消費(fèi) Pulsar topic 時(shí)况鸣,消費(fèi)速度較慢(Pulsar 2.5.2 之前版本)。
當(dāng) Journal/Ledger 盤為機(jī)械硬盤(HDD)時(shí)竹观,問題 4镐捧、5、6臭增、7 表現(xiàn)得尤為嚴(yán)重懂酱。這些問題直觀來看,是磁盤不夠快造成的誊抛,如果 Journal/Ledger 盤讀寫速度足夠快列牺,就不會(huì)出現(xiàn)消息在 direct memory 中堆積,也就不會(huì)有一系列 OOM 的發(fā)生拗窃。
由于在我們消息隊(duì)列生產(chǎn)系統(tǒng)中瞎领,需要存儲(chǔ)的數(shù)據(jù)量比較大(TB ~ PB 級(jí)別),Journal 盤和 Ledger 盤都是 SSD 需要較高的成本随夸,那么有沒有可能在 Pulsar / BookKeeper 上做一些參數(shù)/策略的優(yōu)化九默,讓 HDD 也能發(fā)揮出較好的性能呢?
在壓測和使用 Pulsar 過程中,我們遇到了一系列性能問題逃魄,主要分為 Pulsar Broker 層面和 BookKeeper 層面荤西。為此澜搅,本系列性能調(diào)優(yōu)文章分為兩篇伍俘,分別介紹 BIGO 在使用 Pulsar 過程中對(duì) Pulsar Broker 和 Bookkeeper 進(jìn)行性能調(diào)優(yōu)的解決方案邪锌,以使得 Pulsar 無論在磁盤為 SSD 還是 HDD 場景下,都能獲得比較好的性能癌瘾。
由于篇幅原因觅丰,本次性能調(diào)優(yōu)系列分為兩部分,上半部分主要介紹 Pulsar broker 的性能調(diào)優(yōu)妨退,下半部分主要介紹 BookKeeper 與 Pulsar 結(jié)合過程中的性能調(diào)優(yōu)妇萄。
本文接下來主要介紹 Pulsar / BookKeeper 中和性能相關(guān)的部分,并提出一些性能調(diào)優(yōu)的建議(這些性能調(diào)優(yōu)方案已經(jīng)在 BIGO 生產(chǎn)系統(tǒng)中穩(wěn)定運(yùn)行咬荷,并獲得了不錯(cuò)的收益)冠句。
環(huán)境部署與監(jiān)控
環(huán)境部署與監(jiān)控
由于 BookKeeper 和 Pulsar Broker 重度依賴 ZooKeeper,為了保證 Pulsar 的穩(wěn)定幸乒,需要保證 ZooKeeper Read/Write 低延遲懦底。此外,BookKeeper 是 IO 密集型任務(wù)罕扎,為了避免 IO 之間互相干擾聚唐,Journal/Ledger 放在獨(dú)立磁盤上∏徽伲總結(jié)如下:
- Bookie Journal/Ledger 目錄放在獨(dú)立磁盤上
- 當(dāng) Journal/Ledger 目錄的磁盤為 HDD 時(shí)杆查,ZooKeeper dataDir/dataLogDir 不要和 Journal/Ledger 目錄放在同一塊磁盤上
BookKeeper 和 Pulsar Broker 均依賴 direct memory,而且 BookKeeper 還依賴 PageCache 進(jìn)行數(shù)據(jù)讀寫加速臀蛛,所以合理的內(nèi)存分配策略也是至關(guān)重要的亲桦。Pulsar 社區(qū)的 sijie 推薦的內(nèi)存分配策略如下:
- OS: 1 ~ 2 GB
- JVM: 1/2
- heap: 1/3
- direct memory: 2/3
- PageCache: 1/2
假設(shè)機(jī)器物理內(nèi)存為 128G,bookie 和 broker 混部浊仆,內(nèi)存分配如下:
- OS: 2GB
- Broker: 31GB
- heap: 10GB
- direct memory: 21GB
- Bookie: 32GB
- heap: 10GB
- direct memory: 22GB
- PageCache: 63GB
Monitor:性能調(diào)優(yōu)烙肺,監(jiān)控先行
為了更加直觀地發(fā)現(xiàn)系統(tǒng)性能瓶頸,我們需要為 Pulsar/BookKeeper 搭建一套完善的監(jiān)控體系氧卧,確保每一個(gè)環(huán)節(jié)都有相關(guān)指標(biāo)上報(bào)桃笙,當(dāng)出現(xiàn)異常(包括但不限于性能問題)時(shí),能夠通過相關(guān)監(jiān)控指標(biāo)快速定位性能瓶頸沙绝,并制定相應(yīng)解決方案搏明。
Pulsar/BookKeeper 都提供了 Prometheus 接口,相關(guān)統(tǒng)計(jì)指標(biāo)可以直接使用 Http 方式獲取并直接對(duì)接 Prometheus/Grafana闪檬。感興趣的同學(xué)可以直接按照 Pulsar Manager 的指導(dǎo)進(jìn)行安裝: https://github.com/streamnative/pulsar-manager星著。
需要重點(diǎn)關(guān)注的指標(biāo)如下:
-
Pulsar Broker
- jvm heap/gc
- bytes in per broker
- message in per broker
- loadbalance
- broker 端 Cache 命中率
- bookie client quarantine ratio
- bookie client request queue
-
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
-
ZooKeeper
- local/global ZooKeeper read/write request latency
有一些指標(biāo)在上面 repo 中沒有提供相應(yīng) Grafana 模板,大家可以自己添加 PromQL 進(jìn)行配置粗悯。
Pulsar broker 端性能調(diào)優(yōu)
對(duì) Pulsar broker 的性能調(diào)優(yōu)虚循,主要分為如下幾個(gè)方面:
- 負(fù)載均衡
- Broker 之間負(fù)載均衡
- Bookie 節(jié)點(diǎn)之間的負(fù)載均衡
- 限流
- Broker 接收消息需要做流控,防止突發(fā)洪峰流量導(dǎo)致 broker direct memory OOM。
- Broker 發(fā)送消息給 consumer/reader 時(shí)需要做流控横缔,防止一次發(fā)送太多消息造成 consumer/reader 頻繁 GC铺遂。
- 提高 Cache 命中率
- 保證 ZooKeeper 讀寫低延遲
- 關(guān)閉 auto bundle split,保證系統(tǒng)穩(wěn)定
負(fù)載均衡
Broker 之間負(fù)載均衡
Broker 之間負(fù)載均衡茎刚,能夠提高 broker 節(jié)點(diǎn)的利用率襟锐,提高 Broker Cache 命中率,降低 broker OOM 概率膛锭。這一部分內(nèi)容主要涉及到 Pulsar bundle rebalance 相關(guān)知識(shí)粮坞。
Namespace Bundle 結(jié)構(gòu)如下,每個(gè) namespace(命名空間)由一定數(shù)量的 bundle 組成初狰,該 namespace 下的所有 topic 均通過 hash 方式映射到唯一 bundle 上莫杈,然后 bundle 通過 load/unload 方式加載/卸載到提供服務(wù)的 broker 上。
如果某個(gè) broker 上沒有 bundle 或者 bundle 數(shù)量比其他 broker 少奢入,那么這臺(tái) broker 的流量就會(huì)比其他 broker 低姓迅。
現(xiàn)有的/默認(rèn)的 bundle rebalance 策略(OverloadShedder)為:每隔一分鐘統(tǒng)計(jì)集群中所有 broker 的 CPU、Memory俊马、Direct Memory丁存、BindWith In奈揍、BindWith Out 占用率的最大值是否超過閾值(默認(rèn)為85%)账磺;如果超過閾值纯蛾,則將一定數(shù)量大入流量的 bundle 從該 broker 中卸載掉守伸,然后由 leader 決定將被卸載掉的 bundle 重新加載到負(fù)載最低的 broker 上鸭栖。
這個(gè)策略存在的問題是:
- 默認(rèn)閾值比較難達(dá)到赞赖,很容易導(dǎo)致集群中大部分流量都集中在幾個(gè) broker 上绪爸;
- 閾值調(diào)整標(biāo)準(zhǔn)難以確定蛉拙,受其他因素影響較大界睁,特別是這個(gè)節(jié)點(diǎn)上部署有其他服務(wù)的情況下觉增;
- broker 重啟后,長時(shí)間沒有流量均衡到該 broker 上翻斟,因?yàn)槠渌?broker 節(jié)點(diǎn)均沒有達(dá)到 bundle unload 閾值逾礁。
為此,我們開發(fā)了一個(gè)基于均值的負(fù)載均衡策略访惜,并支持 CPU嘹履、Memory、Direct Memory债热、BindWith In砾嫉、BindWith Out 權(quán)重配置,相關(guān)策略請(qǐng)參見 PR-6772窒篱。
該策略在 Pulsar 2.6.0 版本開始支持焕刮,默認(rèn)關(guān)閉舶沿,可以在 broker.conf 中修改如下參數(shù)開啟:
loadBalancerLoadSheddingStrategy=org.apache.pulsar.broker.loadbalance.impl.ThresholdShedder
我們可以通過如下參數(shù)來精確控制不同采集指標(biāo)的權(quán)重:
# The broker resource usage threshold.
# When the broker resource usage is greater than the pulsar cluster average resource usage,
# the threshold shredder 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
均衡 bookie 節(jié)點(diǎn)之間的負(fù)載
Bookie 節(jié)點(diǎn)負(fù)載監(jiān)控如下圖所示,我們會(huì)發(fā)現(xiàn):
- Bookie 節(jié)點(diǎn)之間負(fù)載并不是均勻的配并,最高流量節(jié)點(diǎn)和最低流量節(jié)點(diǎn)可能相差幾百 MB/s
- 在高負(fù)載情況下括荡,某些節(jié)點(diǎn)的負(fù)載可能會(huì)出現(xiàn)周期性上漲和下降,周期為 30 分鐘
這些問題的影響是:bookie 負(fù)載不均衡荐绝,導(dǎo)致 BookKeeper 集群利用率下降,且容易出現(xiàn)抖動(dòng)避消。
出現(xiàn)這個(gè)問題的原因在于:bookie client 對(duì) bookie 寫請(qǐng)求的熔斷策略粒度太大低滩。
先來回顧一下 Pulsar broker 寫入 bookie 的策略:
當(dāng) broker 接收到 producer 發(fā)送的 message 時(shí),首先會(huì)將消息存放在 broker 的 direct memory 中岩喷,然后調(diào)用 bookie client 根據(jù)配置的(EnsembleSize,WriteQuorum,AckQuorum)策略將 message 以 pipeline 方式發(fā)送給 bookies恕沫。
Bookie client 每分鐘會(huì)統(tǒng)計(jì)各 bookie 寫入的失敗率(包括寫超時(shí)等各類異常)。默認(rèn)情況下纱意,當(dāng)失敗率超過 5 次/分鐘時(shí)婶溯,這臺(tái) bookie 將會(huì)被關(guān)入小黑屋 30 分鐘,避免持續(xù)向出現(xiàn)異常的 bookie 寫入數(shù)據(jù)偷霉,從而保證 message 寫入成功率迄委。
這個(gè)熔斷策略存在的問題是:某臺(tái) bookie 負(fù)載(流量)很高時(shí),所有寫入到該 bookie 的消息有可能同時(shí)會(huì)變慢类少,所有 bookie client 可能同時(shí)收到寫入異常叙身,如寫入超時(shí)等,那么所有 bookie client 會(huì)同時(shí)把這臺(tái) bookie 關(guān)入小黑屋 30 分鐘硫狞,等到 30 分鐘之后又同時(shí)加入可寫入列表中信轿。這就導(dǎo)致了這臺(tái) bookie 的負(fù)載周期性上漲和下降。
為了解決該問題残吩,我們引入了基于概率的 quarantine 機(jī)制财忽,當(dāng) bookie client 寫入消息出現(xiàn)異常時(shí),并不是直接將這臺(tái) bookie 關(guān)入小黑屋泣侮,而是基于概率決定是否 quarantine即彪。
這一 quarantine 策略可以避免所有 bookie client 同時(shí)將同一臺(tái) bookie 關(guān)入小黑屋,避免 bookie 入流量抖動(dòng)活尊。相關(guān) PR 請(qǐng)參見:BookKeeper PR-2327 祖凫,由于代碼沒有合并和發(fā)布到 bookie 主版本,大家如果想使用該功能酬凳,需要自己獨(dú)立編譯代碼:https://github.com/apache/bookkeeper/pull/2327惠况。
從 BIGO 實(shí)踐測試來看,該功能將 bookie 節(jié)點(diǎn)之間入流量標(biāo)準(zhǔn)差從 75 MB/s 降低到 40 MB/s宁仔。
限流
>>Broker direct memory OOM(內(nèi)存溢出)
在生產(chǎn)環(huán)境中稠屠,在高吞吐場景下,我們經(jīng)常遇到 broker direct memory OOM,導(dǎo)致 broker 進(jìn)程掛掉权埠。這里的原因可能是底層 bookie 寫入變慢榨了,導(dǎo)致大量數(shù)據(jù)積壓在 broker direct memory 中。Producer 發(fā)送的消息在 broker 中的處理過程如下圖所示:
在生產(chǎn)環(huán)境中攘蔽,我們不能保證底層 bookie 始終保持非常低的寫延遲龙屉,所以需要在 broker 層做限流。Pulsar 社區(qū)的鵬輝開發(fā)了限流功能满俗,限流邏輯如下圖所示:
在 Pulsar 2.5.1 版本中已發(fā)布转捕,請(qǐng)參見 PR-6178。
Consumer 消耗大量內(nèi)存
當(dāng) producer 端以 batch 模式發(fā)送消息時(shí)唆垃,consumer 端往往會(huì)占用過多內(nèi)存導(dǎo)致頻繁 GC五芝,監(jiān)控上的表現(xiàn)是:這個(gè) topic 的負(fù)載在 consumer 啟動(dòng)時(shí)飆升,然后逐漸回歸到正常水平辕万。
這個(gè)問題的原因需要結(jié)合 consumer 端的消費(fèi)模式來看枢步。
當(dāng) consumer 調(diào)用 receive 接口消費(fèi)一條消息時(shí),它會(huì)直接從本地的 receiverQueue 中請(qǐng)求一條消息渐尿,如果 receiverQueue 中還有消息可以獲取醉途,則直接將消息返回給 consumer 端,并更新 availablePermit砖茸,當(dāng) availablePermit < receiverQueueSize/2 時(shí)结蟋,Pulsar client 會(huì)將 availablePermit 發(fā)送給 broker,告訴 broker 需要 push 多少條消息過來渔彰;如果 receiverQueue 中沒有消息可以獲取嵌屎,則等待/返回失敗,直到 receiverQueue 收到 broker 推送的消息才將 consumer 喚醒恍涂。
Broker 收到 availablePermit 之后宝惰,會(huì)從 broker Cache/bookie 中讀取 max(availablePermit, batchSize)
條 entry,并發(fā)送給 consumer 端再沧。處理邏輯如下圖所示:
這里的問題是:當(dāng) producer 開啟 batch 模式發(fā)送尼夺,一個(gè) entry 包含多條消息,但是 broker 處理 availablePermit 請(qǐng)求仍然把一條消息作為一個(gè) entry 來處理炒瘸,從而導(dǎo)致 broker 一次性將大量信息發(fā)送給 consumer淤堵,這些消息數(shù)量遠(yuǎn)遠(yuǎn)超過 availiablePermit(availiablePermit vs. availiablePermit * batchSize)的接受能力,引起 consumer 占用內(nèi)存暴漲顷扩,引發(fā)頻繁 GC拐邪,降低消費(fèi)性能。
為了解決 consumer 端內(nèi)存暴漲問題隘截,我們?cè)?broker 端統(tǒng)計(jì)每個(gè) topic 平均 entry 包含的消息數(shù)(avgMessageSizePerEntry)扎阶, 當(dāng)接收到 consumer 請(qǐng)求的 availablePermit 時(shí)汹胃,將其換算成需要發(fā)送的 entry 大小,然后從 broker Cache/bookie 中拉取相應(yīng)數(shù)量的 entry东臀,然后發(fā)送給 consumer着饥。處理邏輯如下圖所示:
這個(gè)功能在 Pulsar 2.6.0 中已發(fā)布,默認(rèn)是關(guān)閉的惰赋,大家可以通過如下開關(guān)啟用該功能:
# Precise dispatcher flow control according to history message number of each entry
preciseDispatcherFlowControl=true
提高 Cache 命中率
Pulsar 中有多層 Cache 提升 message 的讀性能宰掉,主要包括:
- Broker Cache
- Bookie write Cache(Memtable)
- Bookie read Cache
- OS PageCache
本章主要介紹 broker Cache 的運(yùn)行機(jī)制和調(diào)優(yōu)方案,bookie 側(cè)的 Cache 調(diào)優(yōu)放在下篇介紹赁濒。
當(dāng) broker 收到 producer 發(fā)送給某個(gè) topic 的消息時(shí)轨奄,首先會(huì)判斷該 topic 是否有 Active Cursor,如果有流部,則將收到的消息寫入該 topic 對(duì)應(yīng)的 Cache 中戚绕;否則纹坐,不寫入 Cache枝冀。處理流程如下圖所示:
判斷是否有 Active Cursor 需要同時(shí)滿足以下兩個(gè)條件:
- 有 durable cursor
- Cursor 的 lag 在 managedLedgerCursorBackloggedThreshold 范圍內(nèi)
由于 reader 使用 non-durable cursor 進(jìn)行消費(fèi),所以 producer 寫入的消息不會(huì)進(jìn)入 broker Cache耘子,從而導(dǎo)致大量請(qǐng)求落到 bookie 上果漾,性能有所損耗。
streamnative/pulsar-flink-connector 使用的是 reader API 進(jìn)行消費(fèi)谷誓,所以同樣存在消費(fèi)性能低的問題绒障。
我們 BIGO 消息隊(duì)列團(tuán)隊(duì)的趙榮生同學(xué)修復(fù)了這個(gè)問題,將 durable cursor 從 Active Cursor 判斷條件中刪除捍歪,詳情請(qǐng)見 PR-6769 户辱,這個(gè) feature 在 Pulsar 2.5.2 發(fā)布,有遇到相關(guān)性能問題的同學(xué)請(qǐng)升級(jí) Pulsar 版本到 2.5.2 以上糙臼。
此外庐镐,我們針對(duì) topic 的每個(gè) subscription 添加了 Cache 命中率監(jiān)控,方便進(jìn)行消費(fèi)性能問題定位变逃,后續(xù)會(huì)貢獻(xiàn)到社區(qū)必逆。
Tailing Read
對(duì)于已經(jīng)在 broker Cache 中的數(shù)據(jù),在 tailing read 場景下揽乱,我們?cè)鯓犹岣?Cache 命中率名眉,降低從 bookie 讀取數(shù)據(jù)的概率呢?我們的思路是盡可能讓數(shù)據(jù)從 broker Cache 中讀取凰棉,為了保證這一點(diǎn)损拢,我們從兩個(gè)地方著手優(yōu)化:
- 控制判定為 Active Cursor 的最大 lag 范圍,默認(rèn)是 1000 個(gè) entry 撒犀,由如下參數(shù)控:
# 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 的判定如下圖所示探橱。
- 控制 broker Cache 的 eviction 策略申屹,目前 Pulsar 中只支持默認(rèn) eviction 策略隧膏,有需求的同學(xué)可以自行擴(kuò)展。默認(rèn) eviction 策略由如下參數(shù)控制:
# 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
Catchup Read
對(duì)于 Catchup Read 場景,broker Cache 大概率會(huì)丟失构诚,所有的 read 請(qǐng)求都會(huì)落到 bookie 上,那么有沒有辦法提高讀 bookie 的性能呢?
Broker 向 bookie 批量發(fā)送讀取請(qǐng)求受裹,最大 batch 由 dispatcherMaxReadBatchSize 控制,默認(rèn)是 100 個(gè) entry。
# Max number of entries to read from bookkeeper. By default it is 100 entries.
dispatcherMaxReadBatchSize=100
一次讀取的 batchSize 越大,底層 bookie 從磁盤讀取的效率越高冀值,均攤到單個(gè) entry 的 read latency 就越低浪蹂。但是如果過大也會(huì)造成 batch 讀取延遲增加古劲,因?yàn)榈讓?bookie 讀取操作時(shí)每次讀一條 entry疤剑,而且是同步讀取。
這一部分的讀取調(diào)優(yōu)放在《Apache Pulsar 在 BIGO 的性能調(diào)優(yōu)實(shí)戰(zhàn)(下)》中介紹。
保證 ZooKeeper 讀寫低延遲
由于 Pulsar 和 BookKeeper 都是嚴(yán)重依賴 ZooKeeper 的,如果 ZooKeeper 讀寫延遲增加蹋嵌,就會(huì)導(dǎo)致 Pulsar 服務(wù)不穩(wěn)定。所以需要優(yōu)先保證 ZooKeeper 讀寫低延遲。建議如下:
- 在磁盤為 HDD 情況下糟描,ZooKeeper dataDir/dataLogDir 不要和其他消耗 IO 的服務(wù)(如 bookie Journal/Ledger 目錄)放在同一塊盤上(SSD 除外)躲履;
- ZooKeeper dataDir 和 dataLogDir 最好能夠放在兩塊獨(dú)立磁盤上(SSD 除外)米诉;
- 監(jiān)控 broker/bookie 網(wǎng)卡利用率拴泌,避免由于網(wǎng)卡打滿而造成和 ZooKeeper 失聯(lián)税朴。
關(guān)閉 auto bundle split茧跋,保證系統(tǒng)穩(wěn)定
Pulsar bundle split 是一個(gè)比較耗費(fèi)資源的操作,會(huì)造成連接到這個(gè) bundle 上的所有 producer/consumer/reader 連接斷開并重連蝇棉。一般情況下板辽,觸發(fā) auto bundle split
的原因是這個(gè) bundle 的壓力比較大耳标,需要切分成兩個(gè) bundle,將流量分?jǐn)偟狡渌?broker邑跪,來降低這個(gè) bundle 的壓力次坡。控制 auto bundle split 的參數(shù)如下:
# 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
當(dāng)觸發(fā) auto bundle split 時(shí) broker 負(fù)載比較高画畅,關(guān)閉這個(gè) bundle 上的 producer/consumer/reader砸琅,連接就會(huì)變慢,并且 bundle split 的耗時(shí)也會(huì)變長夜赵,就很容易造成 client 端(producer/consumer/reader)連接超時(shí)而失敗明棍,觸發(fā) client 端自動(dòng)重連,造成 Pulsar/Pulsar client 不穩(wěn)定寇僧。
對(duì)于生產(chǎn)環(huán)境摊腋,我們的建議是:預(yù)先為每個(gè) namespace 分配好 bundle 數(shù)沸版,并關(guān)閉 auto bundle split 功能。如果在運(yùn)行過程中發(fā)現(xiàn)某個(gè) bundle 壓力過大兴蒸,可以在流量低峰期進(jìn)行手動(dòng) bundle split视粮,降低對(duì) client 端的影響。
關(guān)于預(yù)先分配的 bundle 數(shù)量不宜太大橙凳,bundle 數(shù)太多會(huì)給 ZooKeeper 造成比較大的壓力蕾殴,因?yàn)槊恳粋€(gè) bundle 都要定期向 ZooKeeper 匯報(bào)自身的統(tǒng)計(jì)數(shù)據(jù)。
總結(jié)
本篇從性能調(diào)優(yōu)角度介紹了 Pulsar 在 BIGO 實(shí)踐中的優(yōu)化方案岛啸,主要分為環(huán)境部署钓觉、流量均衡、限流措施坚踩、提高 Cache 命中率荡灾、保證 Pulsar 穩(wěn)定性等 5 個(gè)方面,并深入介紹了 BIGO 消息隊(duì)列團(tuán)隊(duì)在進(jìn)行 Pulsar 生產(chǎn)落地過程中的一些經(jīng)驗(yàn)瞬铸。
本篇主要解決了開篇提到的這幾個(gè)問題(1批幌、2、5嗓节、7荧缘、8、9 )拦宣。對(duì)于問題 3截粗,我們提出了一個(gè)緩解方案,但并沒有指出 Pulsar broker OOM 的根本原因恢着,這個(gè)問題需要從 BookKeeper 角度來解決桐愉,剩下的問題都和 BookKeeper 相關(guān)财破。
由于 Pulsar 使用分層存儲(chǔ)架構(gòu)掰派,底層的 BookKeeper 仍需要進(jìn)行一系列調(diào)優(yōu)來配合上層 Pulsar,充分發(fā)揮高吞吐左痢、低延遲性能靡羡;下篇將從 BookKeeper 性能調(diào)優(yōu)角度介紹 BIGO 的實(shí)踐經(jīng)驗(yàn)。
非常感謝 StreamNative 同學(xué)的悉心指導(dǎo)和無私幫助俊性,讓 Pulsar 在 BIGO 落地邁出了堅(jiān)實(shí)的一步略步。Apache Pulsar 提供的高吞吐、低延遲定页、高可靠性等特性極大提高了 BIGO 消息處理能力趟薄,降低了消息隊(duì)列運(yùn)維成本,節(jié)約了近一半的硬件成本典徊。
同時(shí)杭煎,我們也積極融入 Pulsar 社區(qū)恩够,并將相關(guān)成果貢獻(xiàn)回社區(qū)。我們?cè)?Pulsar Broker 負(fù)載均衡羡铲、Broker Cache 命中率優(yōu)化蜂桶、Broker 相關(guān)監(jiān)控、Bookkeeper 讀寫性能優(yōu)也切、Bookkeeper 磁盤 IO 性能優(yōu)化扑媚、Pulsar 與 Flink & Flink SQL 結(jié)合等方面做了大量工作,幫助社區(qū)進(jìn)一步優(yōu)化雷恃、完善 Pulsar 功能疆股。
關(guān)于作者
陳航,BIGO 大數(shù)據(jù)消息平臺(tái)團(tuán)隊(duì)負(fù)責(zé)人倒槐,負(fù)責(zé)承載大規(guī)模服務(wù)與應(yīng)用的集中發(fā)布-訂閱消息平臺(tái)的創(chuàng)建與開發(fā)押桃。他將 Apache Pulsar 引入到 BIGO 消息平臺(tái),并打通上下游系統(tǒng)导犹,如 Flink唱凯、ClickHouse 和其他實(shí)時(shí)推薦與分析系統(tǒng)。他目前聚焦 Pulsar 性能調(diào)優(yōu)谎痢、新功能開發(fā)及 Pulsar 生態(tài)集成方向磕昼。