This article is compiled from the speeches of Li Yu (Jueding) (ASF Member, Apache Flink & HBase PMC, and Alibaba Senior Technical Expert) and Tang Yun (Chagan) (Apache Flink Committer and Alibaba Technical Expert) at the Flink Forward Asia 2021 Core Technology Session. It includes:
In 2021, the Flink community state-backend module has developed immensely. Before version 1.13, users lacked monitoring methods for the performance of state-related operators, and there was no good way to learn about the latency of state read and write operations.
The state latency tracking has been introduced. The principle is to use system.nowTime to count the access delay before and after each state access and then store it in a histgram-type indicator. The monitoring function has a relatively small impact on its performance, especially the performance impact of state access.
For the configurations, emphasis needs to be placed on the sampling interval and historical data retention. The smaller the sampling interval, the more accurate the data results, but the greater the impact on the performance of daily access. The larger the number of historical data retention, the more accurate the data results, but the larger the memory usage.
In Flink 1.14, RocksDB is finally upgraded from version 5.17 to version 6.20. In addition to fixing bugs in RocksDB, the new version of RocksDB also adds some features that can be used in Flink 1.14 and Flink 1.15. First, it supports the ARM platform, which ensures Flink jobs can run on ARM. Second, it provides finer-grained memory control for WriteBuffer to improve the stability of memory control. In addition, the deleteRange interface is provided, which helps improve the performance in the scale-out scenario.
With the increasing popularity of cloud-native, more vendors choose to run Flink in container environments through Kubernetes. Users need to consider how to run restricted resources stably, especially the control of memory usage. However, RocksDB, which was born in 2010, has some inherent deficiencies in this area. Flink 1.10 first introduced memory control. RocksDB made some progress and improvements in memory control throughout 2021.
First of all, let's review the memory problems of RocksDB. Before talking about this problem, we should understand how Flink uses state and RocksDB.
Due to the three reasons above, even if the memory management limitations of RocksDB are ignored, the use of Flink may lead to unlimited memory usage in theory.
The figure above defines multiple RocksDB instances of a SQL class, sharing a write buffer manager and its corresponding block cache. The manager in it who manages multiple write buffers reserves the memory it applies for in block caches, and the data blocks are cached in block caches. The cache includes data blocks, index blocks, and filter blocks. It can be simply understood as a write cache and a read cache.
Thus, the collaboration between the write buffer manager and block cache is that manager accounts in the block cache.
After the manager applies for the buffer process, it uses io blocks as the basic unit to upgrade the memory in the block cache. The default io block is 1/8 of a single write buffer, and the configuration of the write buffer is 64MB, so the size of the io block is 8MB. The 8MB memory application will be split into several dummy entries and allocated to several shards of the block. In particular, after Flink is upgraded to RocksDB, the minimum unit of dummy entry is reduced to 256KB, which reduces the probability of memory application excess.
Since RocksDB is designed for multithreading, where multiple shards exist in a cache, its memory application is more complicated.
In the memory generated by the internal implementation of the write buffer manager, a mutable WriteBuffer is transformed into an immutable write buffer. When the immutable table is brushed to the disk, the usage of the mutable write buffer is limited by default. When reaching the limits, these write buffers are flushed in advance. It causes a problem. Even if the amount of data written is small, once you apply for arena blocks, especially if there are a large number of arena blocks, the member table flush will be triggered in advance. From the perspective of users, there are a large number of small SST files locally. The overall read and write performance is also poor. Therefore, the Flink community has implemented configuration verification functions of arena block size for this.
Currently, RocksDB has insufficient memory control and limits. Therefore, it needs to reserve a portion of external memory for RocksDB overuse in specific scenarios. Compared with the Flink process memory model in the preceding figure, it can be seen that the memory needs to be properly reserved on the jvm-overhead to prevent RocksDB from being overused. The table on the left shows the default configuration values related to jvm-overhead. If you want to configure jvm-overhead to 512MB, you only need to configure mini and max to 512MB.
In scenarios with limited memory, data blocks, index blocks, and filter blocks are competing with each other. The block instance in the figure above is drawn according to the actual size. Let's take the SST of a 256MB file as an example. The index block is about 0.5MB, the filter block is about 5MB, and the data block is generally 4KB-64KB. It can be seen that the competition of blocks will lead to a large number of swap-in and swap-out, which affects the reading performance significantly.
To solve the preceding problems, we encapsulate the partition-index and partition-filter functions of RocksDB to optimize the performance in the case of limited memory. The index and filter are stored hierarchically so data blocks can be stored as much as possible in limited memory, reducing the read probability of disks and improving overall performance.
In addition to stability-related improvements, Flink focuses on refactoring state-related APIs, which will be friendlier to novices.
The previous API mixed the concepts of statebackend for state read and write and checkpoint for fault-tolerant backup. Let's take MemoryStatebackend and FsStateBackend as examples. The two are identical in terms of state reading, writing, and accessing objects. The difference lies only in fault-tolerant backup, so it is easy for beginners to confuse the difference.
The preceding figure shows the differences between the Flink state read/write and fault tolerance point check APIs before and after the update.
In the new version, we set the status access and fault-tolerant backup separately. The figure above is a comparison table between the new version of API and the old version of API. HashMaoStateBackend state storage is responsible for both MemoryStatebackend and FsStateBackend state reading and writing.
The biggest difference between the two is that in terms of checkpoint fault tolerance, one corresponds to full memory ManagercCheckpointStorage. The other corresponds to file-based FileSystemSCheckpointStorage. Developers can have a deeper understanding through refactoring API.
SavePoint is coupled with state-backend. It is not limited to what state-backend it is implemented. However, the SavePoint format is different for different state-backend in the previous version of Flink. However, the community has unified the relevant SavePoint format in the new version. For the same job, users can seamlessly switch the state-backend without losing the status.
In addition, the community has further enhanced the stability of unaligned checkpoints. Use the buffer in the channel as in-flight data and as part of the operator state for advance persistence to avoid the time of barrier alignment.
In addition, the community supports automatic switching between traditional aligned and unaligned in the new version. As long as a global timeout period is set, Flink will automatically switch after the checkpoint reaches the threshold. The introduction of this feature can help developers achieve better checkpoint performance.
The production use ease of the RocksDB backend will be improved in the future. On the one hand, some key performance indicators of RocksDB will be added (such as block cache hit rate) to the standard monitoring indicators, so the performance of RocksDB can be adjusted easier. On the other hand, we plan to redirect the RocksDB log file to the TM log directory or TM log, which makes it easier to view the RocksDB log to locate problems and tune.
Also, we will further clarify the snapshot semantics of Flink. Currently, there are three types of snapshots in Flink: checkpoint, savepoint, and retained checkpoint:
The community proposed FLIP-193 to solve this problem, which requires users to declare whether to use the claim or no-claim mode when starting a job based on a retained checkpoint:
In the future, we plan to give clearer semantics to user-controlled snapshots and introduce the concept of savepoint in native format instead of contained checkpoints.
Finally, we will present the work in progress for FLIP-158. It introduces a Changelog-based state backend to implement faster and more stable incremental snapshots, which is equivalent to introducing a snapshot based on log marking. Compared with the existing snapshot incremental snapshot mechanism, it has a shorter snapshot interval, but at the same time, it will sacrifice some state data processing delay. This is a trade-off and equilibrium between latency and fault tolerance.
New Generation Stream Computing and Fault Tolerance of Flink - Periodic Summary and Prospect
Deconstructing Stream Storage - Pravega and Flink Build an End-to-End Big Data Pipeline
152 posts | 44 followers
FollowApache Flink Community - May 27, 2024
Apache Flink Community China - January 11, 2021
Apache Flink Community China - September 27, 2020
Apache Flink Community - May 9, 2024
Apache Flink Community - July 5, 2024
Alibaba Cloud Native - July 18, 2024
152 posts | 44 followers
FollowRealtime Compute for Apache Flink offers a highly integrated platform for real-time data processing, which optimizes the computing of Apache Flink.
Learn MoreAlibaba Cloud provides big data consulting services to help enterprises leverage advanced data technology.
Learn MoreAlibaba Cloud experts provide retailers with a lightweight and customized big data consulting service to help you assess your big data maturity and plan your big data journey.
Learn MoreA fully-managed Apache Kafka service to help you quickly build data pipelines for your big data analytics.
Learn MoreMore Posts by Apache Flink Community