Storm通信機制
Storm和Flink都是流計算系統中比較成熟的開源實現,在一些流計算平臺選型的文章中,兩者也常常被拿來重點比較,但是兩個平臺都在持續更新,一些選型比較和壓測結果文章都可能已經過時。而一個系統的核心實現往往比較少改動,因此本文嘗試從底層實現的層面來比較兩者的異同。
在上一篇文章中,我們自頂向下地了解了Flink的內部通信機制,本文按照同樣的結構來說明Storm的通信機制。代碼基于tag v1.2.2。
當我們編寫一個Storm topology時,也是在描繪一個有向圖。圖中的邊相當于由一系列數據記錄組成的數據流,圖中的頂點相當于我們對數據流的處理。
先簡單介紹Storm的術語使用,并和Flink作對比,方便熟悉其中任意一個系統的同學更快地理解這些術語。術語介紹部分主要參考來源為Storm官方文檔Concepts和
Understanding the Parallelism of a Storm Topology
- Spout: Spout是有向運行圖中沒有上游的節點,是數據流的起點。相當于Flink中的SourceOperator。
- Bolt: 對數據流進行處理的節點,相當于Flink中的TransformationOperator
- Component:Bolt和Spout的統稱 ,相當于Flink中的Operator
- Executor: 代表了Component的并行度。每個executor都會有一個工作線程,負責處理用戶定義的業務邏輯,一個發送線程,負責把數據發送到下游隊列。一個Executor中可以運行同一個component的多個實例。(相當于Flink中的SubTask)
- Task: task代表一個component實例, 同一個Execotor中的task會被串行執行(要區別于并行度)
- Worker: 代表一個進程,一個Work中可以運行多個Executor,相當于Flink中的TaskManager
術語對比:
Storm | Flink |
---|---|
Worker | TaskManager |
Executor | Task |
Spout | SourceOperator |
Bolt | TransformationOperator |
Component | Operator |
Stream | Stream |
Task | SubTask |
? | Chain |
Storm的Tasks和Flink的Chain并不能等價,在storm中多個Task可以運行在一個Executor上,但這些Task指的是屬于同一個component的不同實例,兩個Task之間是是等價關系而不是上下游關系。Flink中一個Task中可以有多個Subtask , 在Flink中的Chain是指有上下游關系的且滿足一定條件的多個Subtask可以成鏈的方式被當成一個Task處理。
Component間的通信實現
以Storm的WordCount為例,此例用Storm的底層API編寫(相對Trident來說的底層),源碼可以從Storm官方的Storm-starter模塊獲取。
Storm的編程模型由Spout,Bolt和Stream組成。在圖中split和count為Bolt, 灰色箭頭為stream。圖中總共有三種不同的component, 其中spout的實現類為RandomSentenceSpout,負責隨機地從一個字符串數組中選擇句子。split的功能是對句子進行分詞。count功能是計算單詞的出現次數。功能上與Flink的WordCount例子大同小異。
為了圖例簡潔和簡化模型,在圖中這三種executor的并行度分別為2、2、1,和代碼中并行度不一致。這里沒有開啟acker和metric功能, 因此本文沒有畫出__acker和__metric兩種系統實現的bolt,這acker部分會在分布式事務的對比中分析。
Storm沒有實現Flink那樣的Chain功能,上下游component不會位于同一個線程中,因此Storm的上下游component通信只有兩種方式,本地線程通信或遠程線程通信。
Storm內部每個executor都會有一個接收隊列和一個發送隊列,一個工作線程和一個發送線程。每個worker內部都會有一個發送隊列,一個接收線程和一個發送線程。Storm中的隊列按職能分了三類:分別為executor接收隊列,executor發送隊列和worker發送隊列。這三類隊列的消費者以sleep()的方式不斷輪詢來接收消息,接收消息后的處理結果publish到下游隊列。
有很多Storm的技術文章中畫出了worker的接收線程和topology.receiver.buffer.size, 事實上在Storm1.0.x中 worker的接收線程已被移除,改為push的方式,在Storm server接收到消息后直接反序列化然后寫到各個executor的receive-queue中。
Buffer的讀寫
Storm實現的生產者消費者模式使用到的緩存隊列為LAMX Disruptor中的RingBuffer。LAMX Disruptor號稱最快的無鎖并發框架。在Storm的使用場景中,flush到RingBuffer時使用的等待策略為TimeoutBlockingWaitStrategy是通過ReentrantLock加鎖阻塞的, 且flush到RingBuffer前也會通過鎖來避免并發調用publishDirect(ArrayList<Object> objs, boolean block)方法。
Buffer寫入
以Bolt->Bolt數據傳輸為例,Bolt中的tuple發送主要通過OutputCollector實現, 當一個bolt在execute()方法中調用了OutputCollector.emit(String streamId, Collection<Tuple> anchors, List<Object> tuple) 后,它的調用棧如下圖所示:
粉色部分為clojure實現,黃色部分是java實現。調用棧的最后調用了java實現的DisruptorQueue.publish(Object obj) 方法。DisruptorQueue是Storm對LAMX disruptor的封裝,主要增加了批量發布和超時發布功能。由上一章節可知,tuple會被發布到一個名為executor$id-send-queue的DisruptorQueue中。
DisruptorQueue的發布的代碼邏輯比較復雜,主要通過ThreadLocalInserter和Flusher分別實現Tuple的批量發布和超時發布。
批量發布部分主要實現如下:DisruptorQueue的公共方法publish(Object obj)中,先后調用 ThreadLocalInserter的add和flush方法
/**
變量解釋:
batcher即為ThreadLocalInserter對象實例
**/
public void publish(Object obj) {
...
batcher.add(obj);
batcher.flush(false);
}
batcher.add(obj)方法的功能是把tuple放進當前批次的緩存中,如果當前批次_currentBatch滿了,且當前沒有發送失改批次(為了保證順序性,未發送成功的批次需要先發送),會觸發flush到disruptor的ringbuffer。但不保證flush成功,如果因為ringbuffer空間不足flush失敗,會把失敗的批次放進無界隊列_overflow中緩存。注意:add方法不阻塞。
batcher.flush(boolean block)的功能是觸發發送失敗的批次flush到ringbuffer中。該方法還可能在定時調度的Flusher線程中被調用。注意:block==false時,flush方法不阻塞。
綜上,RingBuffer中的發布單元為一個批次大小的tuple(而不是單個tuple),publish方法不會阻塞,_overflow是個無界非阻塞隊列。因此,如果下游處理不及時且上游持續生產數據時,可能因為_overflow中緩存的對象過多而發生OOM。Storm提供了兩種方式來避免這種情況,留在后續Storm和flink實現對比再討論。
/**
變量解釋:
_currentBatch為ArrayList<Object>對象實例, 用于緩存當前批次的tuple
_overflow為ConcurrentLinkedList<ArrayList<Object>>,無界隊列,用于緩存發送失敗的tuple batch
_inputBatchSize為當前批次的最大緩存tuple數
**/
public synchronized void add(Object obj) {
...
//如果當前批次已滿
if (_currentBatch.size() >= _inputBatchSize) {
boolean flushed = false;
//如果當前批次已滿且緩存中沒有發送失敗的批次
if (_overflow.isEmpty()) {
try {
//發布到disruptor的ringbuffer中,非阻塞,當ringbuffer空間不足時拋出InsufficientCapacityException
publishDirect(_currentBatch, false);
_overflowCount.addAndGet(0 - _currentBatch.size());
_currentBatch.clear();
flushed = true;
} catch (InsufficientCapacityException e) {
//Ignored we will flush later
}
}
//如果當前批次已滿 且 (緩存中有發送失敗的批次 或 當前批次發送失敗)
if (!flushed) {
//把當前批次加入到未發送失敗的緩存隊列中
_overflow.add(_currentBatch);
_currentBatch = new ArrayList<Object>(_inputBatchSize);
}
}
}
//May be called by a background thread
public void flush(boolean block) {
if (block) {
_flushLock.lock();
} else if (!_flushLock.tryLock()) {
//Someone else if flushing so don't do anything
return;
}
try {
while (!_overflow.isEmpty()) {
publishDirect(_overflow.peek(), block);
_overflowCount.addAndGet(0 - _overflow.poll().size());
}
} catch (InsufficientCapacityException e) {
//Ignored we should not block
} finally {
_flushLock.unlock();
}
}
到這里,看過Flink通信機制的同學應該明白“Flink的反壓機制實現得更天然”的說法了。
DisruptorQueue是底層實現,直接暴露給用戶的發送數據到下游的接口是output collector。Storm output collector的實現相較Flink混亂,存在兩個問題:
- Collector命名比較混亂,例如有的實現類叫XXCollectorImpl,有的又不帶Impl后綴 , ISpoutOutputCollector和IOutputCollector是兩個完全不同的接口,兩者不在同一繼承樹中,分別實現Spout的數據發送接口和Bolt的數據發送接口。不通過關鍵字搜索比較難找出全部實現了“tuple發送”功能的代碼。
- Storm的collector實現耦合了tuple的發送邏輯和tuple的ack fail邏輯,因為ack/fail邏輯不同而劃分了兩種主要的OutputCollector , 分別是負責發送Spout tuple的ISpoutOutputCollector、負責發送IRichBolt tuple的IOutputCollector、其它Collector基本上是通過委托模式基于這兩個Collector實現的。例如帶有自動自動ack/fail tuple功能的IBasicOutputCollector,這個類把tuple發送邏輯委托給OutputCollector,而java實現的OutputCollector最后會委托給由clojure代碼executor.clj中實現的IOutputCollector匿名類。
Storm有兩個批量處理框架,相關框架的實現類分別以Transactional和Trident開頭,Transactional開頭的批處理實現已經被標記為廢棄,現主要維護Trident的實現。這兩個API中提供給用戶編程使用的ITridentSpout和ITransactionalSpout 最后都會在Bolt所在的executor中調用,所以批處理編程API中的Spout使用的Collector實際父類或委托類為IOutputCollector。
Buffer讀取
RingBuffer的讀取和處理邏輯通過com.lmax.disruptor.EventHandler接口實現,executor中的工作線程和發送線程以及worker中的發送線程都分別實現了該接口。以executor工作線程為例,executor工作線程讀取event后轉換為Tuple, 并調用IBolt.execute(Tuple tuple)接口觸發用戶實現的業務邏輯。
上圖的邏輯在一個輪詢間隔為0的無限循環中: 當隊列空閑時,cpu空轉。
(defnk consume-loop*
[^DisruptorQueue queue handler
:kill-fn (fn [error] (exit-process! 1 "Async loop died!"))]
(async-loop ;;定義一個循環
(fn [] (consume-batch-when-available queue handler) 0) ;;此處返回0, 代表sleep-time
:kill-fn kill-fn ;;接收到kill信號時執行的清理邏輯
:thread-name (.getName queue))) ;;線程名稱
;; afn returns amount of time to sleep
(defnk async-loop [afn
:daemon false
:kill-fn (fn [error] (exit-process! 1 "Async loop died!"))
:priority Thread/NORM_PRIORITY
:factory? false
:start true
:thread-name nil]
...
(let [sleep-time (afn)]
(when-not (nil? sleep-time)
(sleep-secs sleep-time)
(recur)) ;;循環調用
)
...
Storm和Flink對比
- | Storm | Flink |
---|---|---|
隊列 | Disruptor | ArrayDeque+synchronized |
隊列有無鎖 | 有鎖,使用ReentrantLock | 有鎖,使用synchronized |
隊列有無等待 | 等待,默認Condiction.await(timeout) | 等待,使用wait/notify |
緩存 | 有緩存,用ArrayList和ConcurrentLinkedList | 有緩存,用自定義的MemorySegment和ArrayDeque |
緩存大小 | 可配置,默認100條,和message大小無關 | 可配置,默認最小32768 byte,和條數無關,是消息序列化后的大小(消息可以跨多個buffer) |
生產方式 | 多生產者 | 多生產者 |
消費方式 | 單消費者 | 單消費者 |
序列化 | 默認kryo | 自定義 |
序列化的時機 | 遠程通信時 | 寫入緩存時(因此本地線程通信也會序列化) |
隊列數 | 每個工作線程一個接收隊列(both spout and bolt),每 | 每個工作線程一個消費隊列(source除外) |