本文整理自螞蟻集團實時計算組技術專家閔文俊在 FFA 2023 核心技術(一)中 的分享,內容關於螞蟻流場景狀態演進和最佳化的研究,主要分為以下三部分:
- 狀態後端的演進
- 最佳化與適配
- 未來展望
一. 狀態後端的演進
1. 螞蟻狀態後端的演進
在螞蟻的生產環境中,最早有一套流計算系統內部的專案,代號為 Kepler。在 Kepler 系統實現了一套可插拔的狀態後端儲存系統。在我們生產環境中使用最廣泛的是基於 Hbase 和 OceanBase 這種遠端儲存的狀態後端,它的優勢在於 checkpoint 非常的快,因為流任務在計算的過程中,它的資料已經寫到遠端儲存中,並且它的 failover 也會比較快,因為在 failover 的過程中,它不涉及到資料的下載以及恢復的過程。它也不依賴於本地磁碟,對於雲化環境會比較友好。
並且在分散式儲存系統中,一般都提供了訪問這些資料的 API,所以對於這些已經儲存在狀態後端裡面的資料是可以透過這些 API 直接訪問、查詢出來,比如我們線上會透過 Hbase 的 API 直接將我們儲存在這些系統中的流任務的狀態的 offset 直接暴露出來,這就類似於 Flink 裡面的 qureyable state 這種能力。
它的劣勢也比較明顯。首先,它的效能會比較差。因為對於這種分散式儲存系統,它所能提供的訪問延遲,基本都是在毫秒級,而本地儲存一般能夠到達微秒級,所以這個時延就比較高,它的整體的訪問 TPS 就比較受限,一般不太適合於大狀態的任務。第二,它的穩定性會不足,因為它會受限於第三方服務的穩定性影響,它整體的 SLA 就不如本地儲存。
2. 基於 RocksDB 本地狀態後端
在我們陸續從螞蟻內部資源的流計算,逐步切到社群的 Flink 的過程中,狀態後端也就自然切換到 Flink 自帶的基於 RocksDB 的本地狀態儲存中。
RocksDB 狀態儲存也是現在使用的非常多的狀態儲存,它的優勢主要是,它是基於 RocksDB 這種高效能的本地 KVstore,所以它的狀態訪問的延時非常低,它能滿足時間敏感比較高的任務吞吐的需求。穩定性更好。
它的劣勢在於,第一因為 RocksDB 本身是基於 LSMtree 這樣的結構的,犧牲了一定的讀的效能,來滿足它的寫的效能。但是當我們寫入的前臺流量比較大的時候,它的後臺的 compaction 會去逐步佔用更多的機器的 IO 和 CPU 資源,而影響到前臺的寫入以及訪問的需求,表現的比較明顯. 線上上的任務中會看觀察到的現象,任務在高峰期的時候,會看到它的吞吐反而會有下降的表現。
第二,這種基於 RocksDB 的本地狀態後端,它的 Rescale 和 Failover 也會比較慢。
第三,它的 Checkpoint 的代價會比較高,因為它涉及到本地的狀態檔案的上傳. 為了更好的解決這些問題,螞蟻內部自研了一套基於 RocksDB 的本地的 KV 儲存去嘗試最佳化以上的幾個問題。
3. RocksDB compaction 過程
在介紹本地 KV 儲存之前,我們先看一下 RocksDB 的 compaction 的過程。它在 compaction 過程中,會將某一層級校驗出它所需要進行 compaction 的檔案,然後去進行 compact。在 compact 的過程中,它會挑選出下一層級中與這一層的檔案,將 key 存在 overlap 的檔案進行Merge。在Merge的過程中,它的目的是去降低空間放大,以及透過檔案的合併來減少後續查詢中的檔案的開銷。但是在合併的過程中,因為我們本身 KV 的格式,並且一般來說 value 的是比較大的。但是在合併的過程中,其實這個 value 是不需要隨著 KV 合併的過程而合併的。所以,基於這樣的一個思想,提出了 KV 分離的 RocksDB 的增強的實現。
4. AntKV: 基於 RocksDB KV 分離的本地儲存引擎
社群或者工業界有非常多的 KV 分離的思路,比較典型的是論文 whiskey 裡面的一個 KV 分離的實踐。我們的 KV 整體的思路也是接近於這種 KV 分離的思想,這個圖裡面顯示的是一個 KV value 的寫入的過程。
首先,它的 value 部分會被寫入到 value log 裡,同時會將 value log 所對應的檔案 offset 替換成原始的 KV 資料裡面 value 部分,然後再寫到 LSM tree 裡。在後續的 compaction 的過程中,只有 SST file 裡面的資料才會參與 compaction,value log 這個資料不會參與 compaction。
注意這裡面, 並不是所有的 KV 都參與 KV 分離的實現。因為從觀察到的現象來看,流計算裡也並不是所有的資料都滿足 KV 分離的 pattern。因為從 KV 分離的 pattern 的主要受益場景來說, value 比較大的時候,透過減少 value 的 compaction 的 IO 操作來提升整體吞吐。如果 value 並不大,就會導致後續查詢的時候,需要先透過 LSM 查到對應的 key 所對應的 value 的 address ,然後再跳轉到 value log 裡去查詢這個 value 所對應的真實的資料,這樣就增加了查詢的開銷。所以,為了解決這樣的問題,在 KV 分離的時候,能夠讓它自適應的選擇 KV 分離的策略,簡單來說就是會透過 value 的閾值是否達到配置的大小,來選擇它是否 KV 分離。在此之外,還支援了其他的一些功能,包括 value log 的空間回收 GC、TTL 和 check point 、Merge Op、AsyncRestore 還有 Table API 以及其他的一些最佳化。
這裡主要講一下它在 GC 上面的支援。因為在非 KV 分離的模式下,它的 compaction 的過程中,就已經會將各個 SST 檔案中重複的 key 所對應的資料只保留最新的那一份,所以在 compaction 過程中,這個空間就已經被自然而然的回收。
但是在分離的模式下,在參與 compaction 過程中,它其實在 LSM tree 只有 key 和 value Address 的部分去參與 compaction。所以,當這個 value 不再被引用的時候,它就已經成為垃圾被放在 value log 裡面,而且需要一個清理機制將其清理。
這裡的思路有很多種,我們現在的方案裡面採用的是一種被動回收的策略,就是在 compaction 的過程中,我們會去記錄每一個 value log 所對應的 value 佔用的垃圾比例。因為我們在 compaction 的過程中就可以知道哪些 key 被 compact 掉,所以我們知道這個 key 所對應的 value log 的檔案,因此我們就能完全統計出這樣的資料。當 value log 垃圾佔比達到百分之百的時候,就可以安全的將這個 value log 直接清理掉。
但是還會存在另外一個問題, value log 裡面有可能某一些 key 遲遲的沒有參與 compaction 過程,就會造成 value log 殘留下來,導致一定程度的空間放大。對於這個問題,我們主要是透過設定一個清理的閾值解決,比如當達到百分之九十的時候,我們就需要透過這些垃圾佔比過高的檔案去反向查詢 SST 檔案,然後將這些 SST 檔案作為下一次 compaction 的候選,然後將它進行 compaction 。在這個過程中,它就能將這些 value log 的空間回收掉。
從以上來看,這套 AntKV 的儲存系統的主要的優勢在於,首先降低了 compaction 的讀寫放大,對於線上的效果來看,整體的 get put 的耗時更加穩定,也相比於之前更低。第二點是更高效的快取,因為它降低的 LSM tree 本身有大小,所以它更多的 key 可以被快取在 block cache 裡面,以及自適應的 KV 分離, 適用多種場景。
它的主要劣勢在於 scan 的效率會受影響,因為 scan 在 KV 分離模式中,可能會變成一個隨機 IO,以及它的資料清理會延後,於是帶來額外的磁碟的開銷。
從 AntKV 這個角度來去測試整體的吞吐讀寫測試的話,相比於 RocksDB 來說,達到了兩到四倍左右的提升,比較可觀。
二.最佳化與適配
從引擎側去和 AntKV 適配,我們主要實現了 AntKV 的 statebackend,Backend 實現了外掛化改造,防止和 RocksDB 的 JNI 的命名衝突,並且在 checkpoint 階段去對 AntKV 所產生的額外的 value log 的資料檔案進行持久化管理。其他的最佳化包括 scan 的 prefetch 最佳化,為了解決在它 scan 產生隨機 IO 的情況下如何提升它 scan 的效能的問題、以及基於 learned index 的這種 O(1) 的查詢最佳化能夠極大的提升它的 get 的時效性、以及在 Statebackend 裡面內建支援 TTL 的功能。
1. No Unique Key Join 最佳化
以上是從 AntKV 的角度來看儲存引擎。從引擎的角度來看,以上是一個非常典型的 No Unique Key Join 的一個 case。我們知道在 Flink 裡面,在它 Join 的場景中,它會根據你 Join Input 流是否包含 Unique Key 以及 Join Key 是否包含 Unique Key 去選擇合適的狀態儲存的方式。
以圖為例,Input1 是一個沒有 Unique Key 的流, Input2 是一個 Join key 包含了 Unique Key 的流。當 Input1 進來的時候,如果它去查詢 Input2 的資料,它的查詢的模式是以 Join Key 去查詢這種模式。所以它的模式是比較符合於 AntKV 這樣的最佳化,它是一個小 Key 大 Value 的 Pattern,它的查詢過程也只需要 get 點查。而當我 Input2 的資料來的時候,它的儲存模式是按照 Join Key 加 Record 拼接成一個大 Key,而它的 Value 是一個表示這個 Record 的行數,所以在整體儲存的 Pattern 下,它看起來就會比較奇怪。然後它在查詢的過程中,就需要按照 Join key 先去 seek 到某一個字首,然後在 iterator 去遍歷出這個 Pattern 下所有的 Record。首先,這個查詢效率在非 KV 分離模式或者在 RocksDB 的場景中,它的效果就不好。這是因為,首先這種大 Key 在查詢的過程中就涉及比較大的一個字串的比較以及它的快取效率會比較低。
那麼有沒有辦法將這種 Pattern 和 KV 分離的最佳化匹配上去呢?我們就想辦法去將儲存格式改成一個比較合適的方式。如圖的 Join Key 拼對應的一個 List Record ,就是一種設計方式。然後,我們線上上也採用了多種方式去測試這種場景。
方案如圖中顯示的,它使用的是一個 Value State,然後 Join Key 就對應到一個 List Record,表示這個 Join Key 下所有出現的 Record。方案二是 Map State ,它也是以 Join Key 作為字首,再加上 Record 的 MD5 值拼接成一個 Key,然後它的 Value 是 Record 以及它出現的次數。方案三是使用的一個 List State 的 Value ,就是在我寫入的過程中,有一條 Record,Append 寫進去的這樣的效果。從我們線上測試的效果來看,就是這三種方案,都看到它的 Join Key 的 Value Pattern 都是小 Key 帶 Value 的這種模式。
所以,方案一、方案二、方案三的 TPS 就是優於原始方案的。而方案一的整體吞吐應該能達到原始方案的四倍以上,所以我們最終選擇了方案一的實現方案。以上就是螞蟻在狀態後端上的演進。
2. 狀態問題最佳化
2.1 小檔案問題
小檔案的問題在社群也是一個比較有名的問題,它主要在我們內部可能會導致兩個問題。第一個問題是會導致我們線上出現一些 Task 持續 deploying,它沒有辦法恢復。第二個問題是小檔案導致我們的檔案數擴大持續不足,持續影響 DFS 的穩定性。
我們先來看一下為什麼會有小檔案的問題。小檔案的問題可能主要和增量 Checkpoint 的方案有關,因為增量 Checkpoint 本身就是問題。
2.2 增量 Checkpoint 問題
增量 Checkpoint 的過程中要上傳全量檔案的最佳化方案,但是它在增量 Checkpoint 的過程中,會將每一個 Task 在 Checkpoint 之前,將 RocksDB 先 Flush 生成本次 Checkpoint 之間的新增的 SST 檔案,然後再將這些本次和上次之間 Diff 出來的 SST 檔案上傳到遠端,但是這些 SST 檔案通常不會很大。在持續的過程中,這些 SST 檔案就會導致產生很多的小檔案。
3. 對狀態恢復的影響
然後我們再來看為什麼這種小檔案會導致 Task deploy 的問題。首先,我們可以看到在 Checkpoint 的過程中,它是一個增量的過程,比如我們圖中的 Subtask Manager 在做 Checkpoint 的過程中,它可能在 value logs 裡有一個小於 20KB 的小檔案,那麼它在現有的機制裡就會直接將這個資料傳到 Job Manager 中,由 Job Manager 寫入到 Metadata 檔案裡面,這也是一種最佳化機制,為了無需單獨生成一個檔案,將那些特別小的檔案直接放到 Job Manager 的 MataData 。而在這個持續的過程中,它可能會產生比較多的小檔案,並記錄在 Matadata Data 裡面。因為做 Checkpoint 是一個增量的過程,但是在恢復的過程中,是一個 FullRestore 的過程,就可能會存在提交 Task 到 Task Executor 上,部署的這個 Descriptor 包含了 ByteStream Statehandle 以及 File Stream Handle。這些 ByteStream Statehandle 在持續的增量、Checkpoint 的過程中,可能累加超過 RPC 限制的大小,最終導致任務在跑的過程中,因為掛一次而導致再也恢復不了。因為要傳輸的 State Handle 已經超過了 RPC 的限制。這個問題可能會和我們的這套 KV 分離的架構有一定的關係。因為在非 KV 分離的模式下,它的 SST 會持續的做 compaction。但是在 KV 分離模式下的 Value Log 的 Compaction 的頻率會低於 SST 的 Compaction 的頻率,因為本身的目的是為了減少 Value Log 的 Compaction 的頻率。
為了解決這樣的問題,我們參考了社群之前分享的一套基於 Segment 增量的 Checkpoint 方案。這套方案的主要思路就在於每一個 Task 在做 Checkpoint 過程中,並不會為每一個 SST 檔案都生成一個遠端的 DFS 檔案,而是讓每一個 Task 上 Checkpoint 過程中的要上傳的檔案,儘可能複用一個遠端的檔案,並且記錄每一個 SST 檔案。在這個檔案中所佔用的 Offset 和對應的 Length,去找到它所對應的 Segment。然後透過這種方式,就已經能夠非常好的緩解小檔案的問題。
我們上線之後發現小檔案的問題雖然解決了,但是它還是會有空間放大的問題。如上面的任務裡面,它從任務側彙報上來,這個任務只是佔用了 38 G 左右的狀態大小。但實際上,從 DFS 測試, Checkpoint 目錄所佔用的空間可能達到了它原始大小的七百多倍左右,並且它的檔案數也比較多。所以,這種 case 並沒有達到我們所預期的效果。
4. 基於 Segment 增量 Checkpoint 方案
有一個問題在於,可能在某一些 DFS 的檔案裡面,它的檔案持續在被引用。比如我們這個檔案裡面的第一個檔案的四號 SST,它持續被後面的 checkpoint 引用,並沒有被 GC 或者被合併成另外一個 SST 檔案。這樣的話就導致整個檔案的大小無法釋放,最終導致它的檔案空間放大和檔案數無法消減的問題。
現在要解決這個問題,我的思路非常簡單,類似於我們之前在 AntKV 裡面 Value Log 釋放的思路,就是在儲存空間和 IO 之間,儘可能尋找到一個平衡,比如在上傳的時候檢測這個 DFS 檔案的有效比例,當檔案有效比例低於多少的時候,我們就會將這個原始檔案中的 Segment 作為本次 Checkpoint 中新增的 SST 上傳到遠端。這樣的話,原始的 DFS 的檔案就可以直接被清理掉。
5. 小檔案合併儲存空間放大問題
在上線之後,我們可以看到它的效果,實際儲存量下降到了 Full State 的1.4倍左右,檔案數也大大減少了。
6. 大任務 Union State restore 效能瓶頸
第二個問題是大任務的 Union State restore 的問題。 Union State restore 的過程其實是需要每一個 TM 去 Union State,它本質的意思就是每一個 TM 在做 Checkpoint 的過程中,每一個 sub task 上傳自己的這部分狀態,但是在恢復過程中,每一個 subtask 都能恢復到原始的狀態。當我們以這種狀態恢復的時候,就可以看到它是一種 All-to-all 模式的恢復。當我們的併發比較大的時候,比如我們現在這種是基於 File StateHandle 的模式恢復,它的 DFS 就會成為瓶頸。從我們線上的任務來看,比如有兩千多左右的併發,它整個恢復時長可能會達到半個小時左右的長度,這個在業務上是無法容忍的。
我們也嘗試另一種方式,比如去調低閾值,降低檔案數。如果我們讓它直接寫到 Job Manager 的 Matter Data 裡面,這樣就只有一個檔案去恢復,但是它的 Job Manager 也會成為瓶頸,它會頻繁的在恢復的過程中去 OOM 。OOM 的原因也很簡單,因為我們放在了它的 ByteStreamStateHandle ,那麼在提交任務的時候,TDD 的大小會佔用 Job Manager 的記憶體。
從我們提交的實際情況來看,它在恢復的過程中,這個 StreamStateHandle 可能會佔到百萬級別的大小,最終 JM 根本無法恢復。
解決這個問題的思路,第一點是在 JobManager 側,我們只去建立一份原始的 Statehandle,就不用給每一個 Task 都複製一份,第二點是利用 Blob Server 去傳輸 TDD 中的大狀態物件。
這個 Blob Server 在 Job Manager 裡面本身就在用,所以我們也是利用了這樣的一個機制去傳輸大物件。因為 Blob Server 本身也是在做這樣的事情。
第三點是在同一個 TaskManager 上面,我們只去拉取一次 Union State 的狀態,資料 Task 之間的初始化是共享的。經過以上的一系列的最佳化,最終它原來半小時的重啟任務,現在可以沒有壓力地重啟,而且 JM 記憶體也不再是它的瓶頸。
7. 任務 Rescale 問題
最後一個問題是任務 Rescale 的問題,社群也在持續最佳化這個問題。
我們在很早的時候就已經在嘗試社群的 DeleteRange API 的最佳化。我們發現在 DeleteRange API 的模式下,InitialDB 的選擇可以有更寬鬆的限制。
因為在早期的版本中,它的 Rescale 的過程就是每個 Task 先去下載屬於我的 Handle 資料,下載回來之後,需要將資料匯入進去,然後再將不屬於我的 DB 資料刪除掉。原來實現這種刪除是透過遍歷來刪除,所以它需要一個高比例的 DB 作為它的 base,否則它的刪除代價也是很高的。但是現在在 DeleteRange 這種 API 的模式下,從右圖可以看到,在我們的測試中,它 Rescale 的耗時分佈裡面 DeleteRange 所佔用的時間可以忽略不計,基本都是毫秒級別。所以在 DeleteRange 模式下,我們應該儘可能的選擇一個 DB 作為它的 baseDB。
第二點是,我們發現在 Rescale 的過程中,它磁碟的開銷可能會達到原始的兩到三倍。因為我們跑的是一個雲化環境,對於磁碟有比較高的限制,所以我們對於每一個 Worker 都有磁碟監控,當它磁碟超過某個值的時候,就可能會觸發 Kill 的機制,讓它釋放掉,否則穩定性就無法得到保障。但是在 Rescale 的過程中,以上圖為例,它在恢復的過程中,四擴到八個場景中,每一個原始的 Handle 比如1到10部分的狀態會被1到5和5到10這兩部分都下載。這樣的話,狀態在恢復的過程中它是 Double 的,並且可能由於 InitialDB 選擇的並不合適導致我在 Open 時建立一個全新的 DB 將原始的資料全部匯入進去,這樣就可能會達到兩到三倍的儲存開銷。
要解決這個問題,第一點就是剛剛我們說到的,DeleteRange InitialDB 的選擇。第二個點是我們在恢復的過程中會去做磁碟的一定彈性,提供了磁碟一定的彈效能力,不讓它在 Rescale 的過程中因為恢復的過程直接將一個 Worker 幹掉。第三個是嘗試使用 SST ingest API 去實現常量時間的 Rescale,對於這個功能,社群也在做這樣的嘗試。在我們內部的實現過程中,第一點最開始實現的也是類似於社群之前的方案,就是在恢復的過程中,將原始的 DB 去遍歷,然後透過 SST writer 的 API 去寫出圖裡的測試結果。可以看到,它的主要耗時還是在於原始的 DB 遍歷和 Put 的開銷上,所以透過 SST writer 這樣的方式,還是不能解決問題。所以我們也在看社群現有的方案,以及我們內部的 AntKV 同學努力去將整體的耗時儘可能的降下來。
三.未來展望
未來展望主要分為兩方面。
一方面是期望能夠在現有的狀態後端上,實現更快的 Failover和 Rescale 的能力。第二塊是基於我們現在的 AntKV 的這套儲存引擎的特性,去做出更多的運算元級別的效能最佳化,包括 Join 場景的最佳化,以及在 Windows 上的場景也會去做這樣的探索。
另一方面是降低本地儲存的依賴,因為主流的還是在遷往雲化環境,所以對於本地儲存的依賴也將會成為一個問題。
其次是部分計算下沉儲存引擎,我們現在也在探索 Paimon 資料湖產品,它也是基於 LSM tree 的湖儲存。從圖可以看到,它所支援的是比較靈活的去定義兩個 SST 檔案合併,它的 key 合併的 Merge engine,以及現在已經支援的 Aggregation 的 Merge engine,在生產業務中比較簡單的聚合場景就已經可以透過這種模式玩轉起來。
它的好處在於,因為有很多業務在 Flink 裡面做聚合的時候,需要 Flink 狀態去幫它儲存全量的資料。但是我們不推薦它將 State作為一個持久化的儲存。然而在資料湖裡面,它完全可以透過這種方式,因為在計算儲存端的 Merge 過程中,就已經被完成了。
另外一個好處在於,這種 Merge 的過程是大批次的去做的,它會比原始的透過 Flink State 的 Record 級別去 Get Put 的這種方式的效能會更好。但是它的代價也是會用一定的時延去換整體的吞吐,對於那些可以接受這樣的時延開銷的業務場景,我們已經在嘗試使用這種方式去接入這種業務,來滿足他們的業務需求。
Flink Forward Asia 2023
本屆 Flink Forward Asia 更多精彩內容,可微信掃描圖片二維碼觀看全部議題的影片回放及 FFA 2023 峰會資料!
更多內容
活動推薦
阿里雲基於 Apache Flink 構建的企業級產品-實時計算 Flink 版現開啟活動:
59 元試用 實時計算 Flink 版(3000CU*小時,3 個月內)
瞭解活動詳情:https://free.aliyun.com/?pipCode=sc