本文首發於泊浮目的簡書:https://www.jianshu.com/u/204...
版本 | 日期 | 備註 |
---|---|---|
1.0 | 2021.12.20 | 文章首發 |
1.1 | 2021.12.22 | 錯別字修正 |
0. 前言
在最初接觸到Flink時,是來自於業界裡一些頭部玩家的分享——大家會用其來處理海量資料。在這種場景下,如何避免JVM GC帶來StopTheWorld帶來的副作用
這樣的問題一直盤繞在我心頭。直到用了Flink以後,閱讀了相關的原始碼(以1.14.0為基準),終於有了一些答案。在這篇文章裡也是會分享給大家。
1. JVM記憶體管理的不足
除了上述提到的StopTheWorld,JVM的記憶體管理還會帶來以下問題:
- 記憶體浪費:一個Java物件在記憶體中儲存時會分為三個部分:物件頭、例項資料、對齊填充部分。首先,32位和64位的實現中,物件頭分別要佔用32bit和64bit。而為了提供整體的使用效率,JVM記憶體中的資料不是連續儲存的,而是按照8byte的整數倍進行儲存。哪怕你只有1byte,會自動padding7byte。
- 快取未命中:大家都知道CPU是有L1、2、3級快取的,當CPU去讀取記憶體中的資料時,會將記憶體中鄰近的資料讀到快取中——這是程式區域性性原理的一種實踐手段。
最近被CPU訪問的資料,短期內CPU還要訪問(時間);被CPU訪問的資料附近的資料,CPU短期內還要訪問(空間)。
但我們前面提到,Java物件在堆上儲存的時候並不是連續的,所以CPU去讀取JVM上的物件時,快取的鄰近記憶體區域資料往往不是CPU下一步計算所需要的。這時CPU只能空轉等待從記憶體裡讀取資料(兩者的速度不是一個量級)。如果資料恰好被swap到硬碟裡,那就是難上加難了。
2. Flink的演進方案
在v0.10之前,Flink使用了堆上記憶體的實現。簡單來說就是通過Unsafe來分配記憶體,並用byte陣列的方式將其引用起來,應用層自己維護型別資訊來獲取相應的資料。但這樣仍然會有問題:
- 在堆內記憶體過大的情況下,JVM啟動時間會很長,而且Full GC會到達分鐘級。
- IO效率低:堆上記憶體寫磁碟或網路至少需要1次記憶體複製。
因此在v0.10後,Flink引入了堆外記憶體管理功能。見Jira:Add an off-heap variant of the managed memory。除了解決堆內記憶體的問題,還會帶來一些好處:
- 堆外記憶體可以做成程式之間共享。這意味Flink可以以此來做故障恢復。
當然,凡事都是有雙面性的,缺點是:
- 分配短生命週期的物件,比起堆上記憶體,在堆外記憶體上分配開銷更高。
- 堆外記憶體出錯時排錯更為複雜。
這種實現在Spark中也可以找到,它叫做MemoryPool
,同時支援堆內和堆外的記憶體方式,具體見MemoryMode.scala
;Kafka也有類似的思路——通過Java NIO的ByteBuffer來儲存它的訊息。
3. 原始碼分析
總的來說,Flink在這一塊的實現是比較清晰的——和作業系統一樣有記憶體段,也有記憶體頁這樣的資料結構。
3.1 記憶體段
主要實現為MemorySegment
。在v1.12前MemorySegment
僅僅為一個介面,它的實現有兩個HybridMemorySegment
和HeapMemorySegment
。在之後的發展中,大家發現HeapMemorySegment
基本都沒有人用了,而是都用HybridMemorySegment
了,為了優化效能——避免執行時每次都去查函式表確認呼叫的函式,去掉了HeapMemorySegment
,並將HybridMemorySegment
移到了MemorySegment
中——這會見帶來近2.7倍的呼叫速度優化。:Off-heap Memory in Apache Flink and the curious JIT compiler以及Jira:Don't explicitly use HeapMemorySegment in raw format serde。
MemorySegment
主要負責引用記憶體段,並其中資料進行讀寫——它對基本型別支援的很好,而複雜型別則需要外部來做序列化。具體的實現還是比較簡單的,從field的宣告中就可以大致看出實現了。唯一需要講一下的是LITTLE_ENDIAN
:不同的CPU架構會才不同的儲存順序——PowerPC會採用Big Endian方式,低地址存放最低有效位元組;而x86會採用Little Endian方式儲存資料,低地址存放最高有效位元組。
說實話,讀到這個程式碼的時候筆者還是略震驚的,因為寫Java這麼多年幾乎對底層的硬體是無感知的。沒想到Java程式碼還要考慮相容CPU架構的邏輯。
這個時候就會有同學問了,那這個MemorySegments是如何在Flink中運作的呢?我們可以看個測試用例:BinaryRowDataTest裡的testPagesSer:
先是有MemorySegments,通過對應的BinaryRowWriter寫入資料到RowData,再用BinaryRowDataSerializer寫RowData到RandomAccessOutputView:
@Test
public void testPagesSer() throws IOException {
MemorySegment[] memorySegments = new MemorySegment[5];
ArrayList<MemorySegment> memorySegmentList = new ArrayList<>();
for (int i = 0; i < 5; i++) {
memorySegments[i] = MemorySegmentFactory.wrap(new byte[64]);
memorySegmentList.add(memorySegments[i]);
}
{
// multi memorySegments
String str = "啦啦啦啦啦我是快樂的粉刷匠,啦啦啦啦啦我是快樂的粉刷匠," + "啦啦啦啦啦我是快樂的粉刷匠。";
BinaryRowData row = new BinaryRowData(1);
BinaryRowWriter writer = new BinaryRowWriter(row);
writer.writeString(0, fromString(str));
writer.complete();
RandomAccessOutputView out = new RandomAccessOutputView(memorySegments, 64);
BinaryRowDataSerializer serializer = new BinaryRowDataSerializer(1);
serializer.serializeToPages(row, out);
BinaryRowData mapRow = serializer.createInstance();
mapRow =
serializer.mapFromPages(
mapRow, new RandomAccessInputView(memorySegmentList, 64));
writer.reset();
writer.writeString(0, mapRow.getString(0));
writer.complete();
assertEquals(str, row.getString(0).toString());
BinaryRowData deserRow =
serializer.deserializeFromPages(
new RandomAccessInputView(memorySegmentList, 64));
writer.reset();
writer.writeString(0, deserRow.getString(0));
writer.complete();
assertEquals(str, row.getString(0).toString());
}
// ignore some code
}
3.2 記憶體頁
一個MemorySegment預設對應了32KB大小的記憶體塊。在流處理中,很容易出現超過32KB的資料,這時就需要跨MemorySegment。那麼對於編寫相應邏輯的人就需要持有多個MemorySegment,因此Flink提供了記憶體頁的實現,它會持有多個MemorySegment例項,方便框架的開發人員來快速的編寫Memory相關的程式碼,而無需關注一個個的MemorySegment。
其抽象為DataInputView和DataOutputView,分別對了資料讀取和資料寫入。
接下來,還是關聯實際的程式碼看一下。我們以我們最常見的KafkaProducer使用為例:
|-- KafkaProducer#invoke //在這裡指定了serializedValue
\-- KeyedSerializationSchema#serializeValue //序列化record 的value
我們挑一個實現看看,以TypeInformationKeyValueSerializationSchema
為例:
|-- TypeInformationKeyValueSerializationSchema#deserialize //KeyedSerializationSchema的實現類
|-- DataInputDeserializer#setBuffer // 這是DataInputView的實現,用內部的byte陣列儲存資料。這裡很奇怪的是並沒有使用MemorySegement。
|-- TypeSerializer#deserialize // 它的實現會針對不同的型別,從DataInputView裡讀出資料返回
其實這裡的例子不太恰當。因為KeyedSerializationSchema已經被標記為了廢棄。社群更建議我們使用KafkaSerializationSchema。第一個原因是因為KeyedSerializationSchema的抽象並不合適Kafka,當Kafka在Record新加欄位時,是很難抽象當這個介面裡的——這個介面僅僅關注了key、value以及topic。
以KafkaSerializationSchema
展開的話,我們可以看典型的實現——KafkaSerializationSchemaWrapper
,我們關心的地方很容找到:
@Override
public ProducerRecord<byte[], byte[]> serialize(T element, @Nullable Long timestamp) {
byte[] serialized = serializationSchema.serialize(element);
final Integer partition;
if (partitioner != null) {
partition = partitioner.partition(element, null, serialized, topic, partitions);
} else {
partition = null;
}
final Long timestampToWrite;
if (writeTimestamp) {
timestampToWrite = timestamp;
} else {
timestampToWrite = null;
}
return new ProducerRecord<>(topic, partition, timestampToWrite, null, serialized);
}
這個serializationSchema的宣告是一個名為SerializationSchema
的介面。可以看到它有大量的實現,其中很多對應了DataStream還有SQL API中的format。我們以TypeInformationSerializationSchema
為例繼續跟蹤:
@Public
public class TypeInformationSerializationSchema<T>
implements DeserializationSchema<T>, SerializationSchema<T> {
//ignore some filed
/** The serializer for the actual de-/serialization. */
private final TypeSerializer<T> serializer;
....
又看到我們熟悉的介面TypeSerializer
了。就像上面說的,它的實現會針對不同的型別,從DataInputView、DataOutputView進行互動,提供序列化和反序列化的能力。在它的方法簽名中也是可以看到的:
/**
* Serializes the given record to the given target output view.
*
* @param record The record to serialize.
* @param target The output view to write the serialized data to.
* @throws IOException Thrown, if the serialization encountered an I/O related error. Typically
* raised by the output view, which may have an underlying I/O channel to which it
* delegates.
*/
public abstract void serialize(T record, DataOutputView target) throws IOException;
/**
* De-serializes a record from the given source input view.
*
* @param source The input view from which to read the data.
* @return The deserialized element.
* @throws IOException Thrown, if the de-serialization encountered an I/O related error.
* Typically raised by the input view, which may have an underlying I/O channel from which
* it reads.
*/
public abstract T deserialize(DataInputView source) throws IOException;
/**
* De-serializes a record from the given source input view into the given reuse record instance
* if mutable.
*
* @param reuse The record instance into which to de-serialize the data.
* @param source The input view from which to read the data.
* @return The deserialized element.
* @throws IOException Thrown, if the de-serialization encountered an I/O related error.
* Typically raised by the input view, which may have an underlying I/O channel from which
* it reads.
*/
public abstract T deserialize(T reuse, DataInputView source) throws IOException;
/**
* Copies exactly one record from the source input view to the target output view. Whether this
* operation works on binary data or partially de-serializes the record to determine its length
* (such as for records of variable length) is up to the implementer. Binary copies are
* typically faster. A copy of a record containing two integer numbers (8 bytes total) is most
* efficiently implemented as {@code target.write(source, 8);}.
*
* @param source The input view from which to read the record.
* @param target The target output view to which to write the record.
* @throws IOException Thrown if any of the two views raises an exception.
*/
public abstract void copy(DataInputView source, DataOutputView target) throws IOException;
那麼TypeSerializer#deserialize
到底是怎麼被呼叫到的呢?這些細節並不是這篇文章需要關心的。在這裡我們展示一下呼叫鏈,有興趣的讀者可以沿著這個呼叫鏈看一下具體的程式碼:
|-- TypeSerializer#deserialize
|-- StreamElementSerializer#deserialize
|-- TypeInformationKeyValueSerializationSchema#deserialize
|-- KafkaDeserializationSchema#deserialize
|-- KafkaFetcher#partitionConsumerRecordsHandler //到這裡已經很清楚了,這裡是由FlinkKafkaConsumer new出來的物件
3.3 緩衝池
還有一個比較有意思的類是LocalBufferPool
,封裝了MemorySegment
。一般用於網路緩衝器(NetworkBuffer),NetworkBuffer
是網路交換資料的包裝,當結果分割槽(ResultParition)開始寫出資料的時候,需要向LocalBufferPool申請Buffer資源。
寫入邏輯:
|-- Task#constructor //構造任務
|-- NettyShuffleEnvironment#createResultPartitionWriters // 建立用於寫入結果的結果分割槽
|-- ResultPartitionFactory#create
\-- ResultPartitionFactory#createBufferPoolFactory //在這裡建立了一個簡單的BufferPoolFactory
|-- PipelinedResultPartition#constructor
|-- BufferWritingResultPartition#constructor
|-- SortMergeResultPartition#constructor or BufferWritingResultPartition#constructor
|-- ResultPartition#constructor
\-- ResultPartition#steup // 註冊緩衝池到這個結果分割槽中
另外,NetworkBuffer
實現了Netty的AbstractReferenceCountedByteBuf
。這意味著這裡採用了經典的引用計數演算法,當Buffer不再被需要時,會被回收。
4. 其他
4.1 相關Flink Jira
以下是我在寫本文時參考過的Jira列表:
- Add an off-heap variant of the managed memory:https://issues.apache.org/jir...
- Separate type specific memory segments.:https://issues.apache.org/jir...
- Investigate potential out-of-memory problems due to managed unsafe memory allocation:https://issues.apache.org/jir...
- Adjust GC Cleaner for unsafe memory and Java 11:https://issues.apache.org/jir...
- FLIP-49 Unified Memory Configuration for TaskExecutors:https://issues.apache.org/jir...
- Don't explicitly use HeapMemorySegment in raw format serde:https://issues.apache.org/jir...
- Refactor HybridMemorySegment:https://issues.apache.org/jir...
- use flink's buffers in netty:https://issues.apache.org/jir...
- Add copyToUnsafe, copyFromUnsafe and equalTo to MemorySegment:https://issues.apache.org/jir...