This article is organized from the sharing of FFA 2023 by Yuan Mei from Alibaba Cloud. Mei shared the evolution of state management and the selection of Flink 2.0 storage-computing separation architecture based on Alibaba internal practices. The content is mainly divided into the following five parts:
We revisit the topic of state storage this time because state storage is the core of stream computing. Since the release of the foundational work on VLDB in 2017 to introduce the state storage, Flink has not changed much on the architecture of the state. However, the time is constantly evolving and changing. For the deployment mode, storage mode, and job load, great changes have taken place.
From the perspective of deployment mode, we have experienced the deployment without resource isolation on Cluster in the map-reduce era, and Kubernetes containers are deployed with granularity isolation in the cloud-native era.
At the same time, the improvement of hardware has also brought about changes in storage mode. The most significant thing is that the network bandwidth has started from several hundred Mbps 10 years ago to 25 Gbps now which is comparable to the width of a local SSD. Moreover, the internal network transmission speed is fast and the cost is low, which makes the storage mode change from local storage to distributed storage and then to cloud storage. Although cloud storage has a high delay, it has the advantages of large capacity and low cost, which is very suitable for the background of cost reduction and efficiency increase.
In addition, great changes have taken place in the job load. When Flink was born, hundreds of MB of the state was considered a large state scenario. However, in some scenarios such as logistics, hundreds of TB of the state is also very common. Advances in technology and changes in user needs have brought us some new thoughts on the part of state storage, which is the topic of this article.
This article mainly discusses the evolution direction of Flink storage-computing separation architecture in three aspects.
• In the first part, the reason why the state is very important to Flink is explained. That is, because the state access is on a single data processing path, processing delay has a great impact on the overall performance.
• In the second part, the recent state storage community and some of our work and exploration within Alibaba are introduced.
• In the third part, what the Flink storage-computing separation architecture should be based on these explorations is talked about.
In Flink stream processing, if subsequent calculations need to rely on the previous input, you need to use a state table to store intermediate data. In Task Manager, a service component called StateBackend provides state read/write services for the operator layer. If the state is too large to be loaded in the memory, StateBackend organizes the state data in the form of files and stores them on the local disk. However, local disks usually do not have data reliability. Therefore, Flink needs to take regular snapshots of the state and upload local files to distributed file systems such as OSS, HDFS, and S3. In addition, multiple tasks in a Flink job often contain state operators. Therefore, Flink needs to ensure that the state snapshots on these different tasks are globally consistent, that is, the input is aligned. This process is the Flink Checkpointing process.
In this part, we mainly discuss the current requirements of Flink state storage and the existing problems. With this premise, we can solve the problem.
First, state processing is the critical path of stream computing. Currently, Flink state interfaces are of single-thread and single-KV access mode. Therefore, the delay of state reads and writes greatly affects the processing performance (TPS) of Flink jobs. Most Flink online jobs mount local disks, and their access delay is basically at the us level. If the local state storage is directly replaced with DFS storage, the access delay will encounter a large bottleneck. This is also the reason why many people have encountered performance problems when trying Flink remote state storage. In addition, the process of taking a snapshot of the state also needs to be as light and stable as possible and does not affect normal data processing.
Under the background of cloud-native, higher requirements are put forward for the state.
The first is the limit on the size of local disks. Containerization technology can achieve good resource isolation, which means that resources need to be pre-allocated. In Flink scenarios, the size of the local disk of Task Manager is fixed when a job is started. It is troublesome to remount the disk while the job is running. In general, the fully managed services of major cloud vendors do not support this advanced feature. Therefore, if the local disk is full during the running of the job, it can only be solved by scaling out the concurrency. However, public cloud users are often reluctant to pay more because the disks are filled with the state.
In addition to local disk resources, the same is true for other resources such as CPU and network I /O. The containerized architecture hopes that the use of these resources can be as smooth as possible to avoid peaks and achieve better resource isolation. At last, there is efficient auto-scaling elasticity. The community can now combine the Kubernetes operator to achieve in-place rescale, and use the local rescale to be done in 1.19 to partially solve the auto-scaling problem, but there will still be a problem of slow state download when the state is relatively large.
In addition, there are some requirements from the perspective of users, such as the queryable and reusable state . The queryable state is frequently queried by users in the community. However, due to the architecture defect, the normal data processing of task managers will be affected if the queryable state is enabled. Currently, the queryable state is in the Deprecate state in the community. We also need to fully consider such requirements when designing the storage-computing separation architecture.
Based on the above requirements and challenges, let's take a look at what attempts we have made and what problems we still have with the state storage optimization in the community and our commercial version.
Starting from Flink version 1.11, the distributed snapshot has gone through six versions in the community. We have done some work on the stability and completion speed of checkpoints, mainly including three functions: dynamic adjustment of cached data (Buffer Debloating), unaligned snapshot (Unaligned Checkpoint), and general incremental checkpoint. These three functions have been implemented on a large scale in major companies, giving full play to their due value.
Unaligned Checkpoints and Buffer Debloating are used to prevent or reduce the case that the Checkpoint Barrier is blocked by data in pipelines between operators during the back pressure. From the experimental results, the Unaligned Checkpoint can shorten the snapshot completion time by 90% during the back pressure (this premise is that the state size cannot be too large and the barrier alignment time is the main factor). In addition, because the Unaligned Checkpoint stores data in the operator pipeline, we have also tested its impact on the full checkpoint size and state recovery time. The test results show that the increase in the full checkpoint size and the extra time for state recovery are basically negligible.
When the job state is relatively large, the state upload time will become the dominant factor of the checkpoint completion time. In this case, we can use generic incremental snapshots to accelerate checkpoints. A generic incremental snapshot decouples the checkpointing process from the state snapshot process. It can continuously upload the incremental state changelog to DFS so that only a small number of data needs to be uploaded when making checkpoints. Therefore, checkpoints can be completed within 10 seconds regardless of the state size (all state query test results on Nexmark are consistent).
General incremental checkpoints were originally designed to accelerate checkpoints, but later we found some additional benefits. As shown in the above figure, the peak usage of CPU and network traffic is reduced by 70%, which is critical to the stability and resource-saving of the overall cluster. The reason for this optimization is that Flink periodic snapshots allow all the RocksDB state backends in the job to periodically trigger compactions and upload files. As a result, a large amount of CPU and network I/O are consumed during checkpoints, which is likely to cause CPU and network I/O peaks. After the generic incremental snapshot decouples the checkpointing and state snapshot processes, the timing of state snapshot triggers can be randomized to avoid the formation of CPU and network usage peaks.
In addition, because generic incremental snapshots are fast, the number of data replays can be greatly reduced during fault tolerance, which is critical for jobs that are sensitive to duplicate data. Everything has two sides. The additional overhead of generic incremental snapshots is the need for data dual-writes. We have also used Nexmark Stateful Query to test its impact on TPS. The results show that the impact of generic incremental snapshots on TPS is less than 5% when the network bandwidth is not a bottleneck. For detailed data, see Generic Log-based Incremental Checkpoint --- Performance Evaluation & Analytics.
In addition to checkpoints, the community has been experimenting with more efficient elastic scaling since Flink 1.17.
Firstly, the Adaptive Scheduler in Flink 1.18 supports the Rest API. Kubernetes auto scaler can call this API for in-place scaling. In other words, the task manager and job manager do not need to be restarted after the job is updated, which can maximize the reuse of existing resources.
Secondly, for the file download part, Flink 1.18 begins to support parallel download, reducing the average time consumption by 30%. Flink 1.19 is expected to support Local Rescale, which can avoid repeated downloads during scaling. With the above-mentioned Kubernetes in-place scaling, this part of the improvement should be great.
Finally, in the aspect of state reconstruction, WAL is disenabled when the state is restored in Flink 1.18, which can reduce the additional overhead during recovery. After Flink 1.19 is issued, state reconstruction can support file-level merge tailoring, and the time required to reconstruct the state backend is reduced by 50% to 80%.
In this section, we will share some of our explorations on state cloud-native within Alibaba.
Gemini has been fully launched in Alibaba, serving jobs of different state sizes (from hundreds of MB to hundreds of TB). It has also been tested by the Double 11 Shopping Festival from 2021 to 2023 and achieved a very good effect of cost reduction and efficiency increase. In addition, we use Nexmark Benchmark based on Flink 1.15 to compare the performance of Gemini and RocksDB at the pure state backend level. The test results show that Gemini significantly improves the job performance (single-core throughput), with an average of 50% +.
Gemini can be regarded as a hierarchical state storage. Memory data is stored in Mem Table (read/write) and Cache (read-only), hot data is written to the local disk, and some cold data is written to DFS after the local disk is full. Putting aside other DB features, let's focus on the exploration of Gemini storage-computing separation. First of all, fully managed cloud services are limited by disk capacity. Most users are unwilling to pay extra for full local disks to increase concurrency. Therefore, Gemini implements centralized local and remote file management, using local disks as the primary storage and DFS as the next level of storage. If the amount of data on DFS is not large, the performance is acceptable. However, if DFS is frequently accessed, its performance will be greatly influenced.
In terms of rapid scaling, Gemini mainly optimizes two aspects: file granularity merging and tailoring and LazyRestore. Among them, file-level merge tailoring is slightly different from the current practice of the community. Gemini file tailoring is asynchronous, which allows jobs to recover faster. LazyRestore refers to the asynchronous downloading of remote files and loading on demand. After the metadata is restored, the Flink job can start running at half speed. From our test results, we can see that hot update and LazyRestore can reduce the time for the job to stop processing from 200s to 20s.
Under the hierarchical architecture of state management, there are currently some solutions to the problems of state use, but these methods are not perfect. We have also encountered some difficulties in the implementation process. In the last section, we will share our experience and possible solutions to the evolution of the storage-computing separation architecture.
In the unified file management architecture of Gemini, state files are flushed to DFS after the local disk is full, but the local storage is still used as the primary storage. The disadvantage of this method is that file management is more complicated and not easy to expand. For example, when we have more layers of storage, we need to make major changes to the architecture. This method is more complex than using local storage as a cache, especially for handling exceptions such as file write failures. Secondly, in the model with the local state as the primary memory, each task lacks the global view of the overall state of the operator, which makes it difficult to share the state. For some jobs with extremely large states and sharing requirements, the implementation cost will increase exponentially. More importantly, in this mode, many DB designs are based on the premise that local storage is the primary memory, which makes the state DB compaction and hot-cold separation policies relatively complex. The binding of computing nodes and storage nodes is relatively tight.
For this reason, we can consider doing this differently. We can use DFS as primary storage and local disk as an optional cache, and DFS data is the source of truth. In this method, state data is directly and continuously written to DFS, and memory and local disks are used as caches to serve operator state access at the upper layer. The benefits of this architecture are:
• When a checkpoint is triggered, most of the state files have already been in DFS. Checkpoints can be quickly executed.
• State files can be shared among operators. In some scenarios, storage resources can be greatly reduced by sharing state computing.
• State query can be implemented based on remote checkpoint files and file-level APIs can be provided.
• Compaction and state cleanup are no longer bound to the task manager. We can use methods such as remote compaction and load balancing to balance the overall resource usage of the cluster.
However, by just switching the state primary memory from the local disk to the remote file system, can the storage-computing separation architecture be perfectly implemented? The answer is No. If this is the case, the job performance will be particularly degraded, because compared with the local state architecture, the access performance will be greatly rolled back after the state is written to the remote end.
The reason for the performance degradation is that due to the current Flink thread access model, data processing must be performed in a single-thread and single-record processing mode, which is extremely sensitive to the delay of state access. Among them, state access takes a single KV as the granularity. Operator data processing will be encapsulated into Mail and executed sequentially in task threads, so the delay of a single record has a significant impact on performance. From the specific analysis of the read-write link, the write path to the state is non-blocking. Writes are appended to writeBuffer, and writeBuffer is asynchronously flushed to the disk after the writerBuffer is full, so the writing process has little effect on the overall TPS. However, the read path greatly affects the overall performance. If the cache miss appears in Block Cache when reading data, you need to read data from the disk. The reading process will block data processing and affect the overall performance of the job.
Furthermore, we can see that when the task thread executes computing, the time is mainly consumed in the following parts:
• Operator CPU: it is used to execute operator logic.
• State CPU: it is used for read-write access and serialization and deserialization.
• State IO: it is the time when the state is read from the disk. Since the disk read operation is Blocking IO, and the access latency of DFS is much larger than that of local disks, the direct read speed of DFS in a single-threaded model is much slower than that of local disks.
We can consider transforming the Blocking IO of the task thread into the Non-Blocking IO to make use of CPU and IO resources more efficiently to solve the problem. In the preliminary experiment of the state multi-thread concurrent access, when we adjust the thread pool size to be sufficiently large, the performance is significantly improved (about 6 times). In fact, the issues to be considered in the Non-Blocking IO architecture are much more complicated than the current state multi-thread access experiment, such as how to ensure the execution sequence of the same Key. We have also made some POC and got some preliminary conclusions. Please look forward to the sharings in subsequent articles.
To wrap up this presentation, here are some key takeaways:
First, state access plays a key role in stream computing. On the access path of a single record, the processing speed of state access will seriously affect the TPS. This is also the reason why many previous attempts to Flink remote state storage access are very slow.
Second, the cloud-native era has put forward higher requirements for state storage, such as limited local disks, fast scaling, and smooth resource usage. Based on this, we have done a lot of optimization work, but it is not enough. Therefore, we propose that the storage access architecture of the state needs further evolution.
Third, the two functions of Unaligned Checkpoint and generic Incremental Checkpoint have been introduced by the community, and they have been widely practiced and used in major companies and manufacturers. We have also made small-scale attempts internally, and the results are very good.
Fourth, we have introduced Alibaba internal hierarchical state storage for stream computing. In this process, we have made some explorations in cloud-native and storage-computing separation.
Finally, we've shared an overview of the Flink 2.0 storage-computing separation architecture. We plan to initiate discussions on this topic within the community, providing more detailed insights into our design rationale, challenges faced, and the solutions we've devised. Stay tuned.
150 posts | 43 followers
FollowApache Flink Community - May 7, 2024
Alibaba Cloud Community - May 10, 2024
Alibaba Cloud Community - December 9, 2021
Apache Flink Community - January 31, 2024
Alibaba EMR - May 11, 2021
ApsaraDB - February 4, 2024
150 posts | 43 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