×
Community Blog Flink Shuffle 3.0: Vision, Roadmap, and Progress

Flink Shuffle 3.0: Vision, Roadmap, and Progress

This article was compiled from a speech by Song Xintong (Wuzang) (an Alibaba Cloud Senior Technical Expert) during FFA 2022, discussing Flink Shuffle 3.

This article was compiled from a speech by Song Xintong (Wuzang) (an Alibaba Cloud Senior Technical Expert) during the core technology session of FFA 2022. The content of this article is mainly divided into five parts:

  1. The Evolution of Flink Shuffle
  2. Stream-Batch Integration
  3. Cloud-Native
  4. Adaptivity
  5. Shuffle 3.0

1. The Evolution of Flink Shuffle

_1

During the entire evolution of Shuffle, the concept of Shuffle 1.0 and 2.0 has not been proposed. From its technological development, we can divide it into two stages, as shown in the figure.

In Shuffle 1.0, Shuffle only has basic data transmission capabilities, and the Flink project is at a young stage.

In Shuffle 2.0, we made a series of optimizations.

  • In terms of performance, we have optimized the serialization of data and the memory copy of the underlying network and designed a Sort-Based Blocking Shuffle for the batch scenarios. This shuffle mode may be friendly to disk IO.
  • In terms of stability, we have introduced the Credit-Based flow control mechanism, which is more stable than the backpressure mechanism that originally depends on TCP. In addition, the community has introduced a Buffer-Debloating mechanism that enables it to reduce the impact of data backlog on checkpoints in a state of backpressure.
  • In terms of stream and batch integration, we reconstruct the Shuffle module for the Service plug-in, making it possible to implement Shuffle developed by a third party. In addition, we have paved the way for Remote Shuffle Service technology in the batch scenario.

In summary, we can find that both performance and stability are necessary capabilities for mass production on Flink. The integration of stream and batch is one of the main development directions of the Flink community in the past. From the entire Shuffle 2.0 phase, we found that Flink Shuffle has been mature and performed well in production.

What issues should we focus on when it comes to Shuffle 3.0? With the development of the times and the progress of technology, what new challenges have been posed to Shuffle?

Here we list three keywords: stream-batch integration, cloud-native, and adaptivity. Next, I will explore them one by one.

_2

2. Stream-Batch Integration

_3

What is the connection and difference between stream-batch integration and stream-batch integration?

As shown in the preceding figure, the classic Flink stream-batch integration architecture is on the left. In this architecture, Flink provides a unified API expression for streams and batches and uses a unified engine (namely Flink) to process stream and batch data. In addition, we usually schedule real-time tasks and offline tasks to the same cluster for mixing. This improves R&D and O&M efficiency and resource utilization.

Currently, the Flink stream-batch integration architecture is user-oriented. If you look inside the engine, you will find that the difference between the stream mode and the batch mode of a Flink task is clear. There are two offline and real-time data links in the entire architecture. Thus, the integration of stream and batch is mainly a user-oriented concept.

_4

Stream-batch integration is the Flink stream and batch integration capability. It puts the stream and batch technology into one engine. We hope to break the technical boundary between stream and batch on the engine side. Both stream technology and batch technology serve different scenarios at the same time.

In terms of stream-batch integration, there are two main points:

  1. In batch processing scenarios, Flink, as a streaming-based engine, draws on and learns from mature batch technology experience and has many unique advantages. For example, upstream and downstream tasks run at the same time during stream processing. The streaming kernel engine can ensure that data is directly transmitted without falling to the disk, thus reducing IO overhead and improving performance. In addition, there is a checkpoint-based fault tolerance mechanism for stream processing, which is more flexible and refined.
  2. After the streaming engine has the capability of batch processing, it can serve stream processing scenarios better. For example, batch job data usually needs to be sorted, which has better performance and effect on state access. In addition, the intermediate data of batch data will fall on the disk, which can be repeatedly consumed, improving fault tolerance.

Stream-batch integration mainly emphasizes the break of the boundary between stream and batch. Put all the technologies together from the engine side to serve different scenarios. It is not difficult to see that the concept of stream-batch integration is an end-to-end thing. It runs through scenarios (such as execution plan optimization, compilation, scheduling, running, shuffling, and fault tolerance) and needs to be changed and improved according to the concept of stream-batch integration.

_5

Hybrid Shuffle is a technology that applies streaming technology to batch scenarios.

Currently, Flink Shuffle mainly includes Pipelined Shuffle and Blocking Shuffle. The upstream and downstream tasks of stream pipelined shuffle run at the same time, which reduces the running time of the tasks. At the same time, its data can be directly transferred between tasks without falling.

However, Pipelined Shuffle is still in the production unavailable state in batch scenarios. When it runs both upstream and downstream, the resource demand is high. If multiple tasks exist at the same time, each task can only receive part of the resources, which makes it easy to form a deadlock in resource scheduling.

Batch Blocking Shuffle provides better resource adaptability. We can use one slot to perform all tasks in extreme cases. However, its performance is slow since batch tasks run as stage scheduling. Each stage needs to wait for the long tail task to be complete. Secondly, all its data needs to be fallen to the disk, resulting in high IO overhead.

This shows that both streaming Shuffle and batch Shuffle will cause resource fragmentation under certain circumstances. Although they hold resources, they cannot schedule and execute tasks, which will cause resource waste.

_6

Hybrid Shuffle is designed to combine the features of stream Shuffle with batch Shuffle. You can write data to the memory for direct consumption. You can write data to the disk for later consumption when too much data cannot be stored in the memory and downstream consumption is not timely enough. Through adaptive switchover, the downstream can consume data at any time during and after the upstream output is completed, which eliminates resource fragmentation.

Hybrid Shuffle allows all upstream and downstream tasks to run at the same time when resources are sufficient. Its performance is the same as a streaming pipeline shuffle. If resources are limited, Hybrid Shuffle can executed tasksi n the upstream. After the data is written on the disck, then Hybrid Shuffle can consume tasks in the downstream. Its resources are more adaptive than Blocking Shuffle.

In addition, Hybrid Shuffle switches between memory and disk, which is a dynamic adaptive switching, not a static one-time switching. In the process of data consumption, we can switch to disk mode at any time when the memory is full. When the data in the memory is consumed, leaving more space, it can switch back to the memory for consumption.

Hybrid Shuffle has been released in Flink 1.16. After testing, Hybrid Shuffle has improved performance by 7.2% compared with Blocking with limited resources. If resources are sufficient, Hybrid Shuffle can significantly improve the performance, compared with Blocking.

We will continue to improve and optimize Hybrid Shuffle during Flink 1.17. It mainly includes performance optimization for broadcast data and compatibility with other important characteristics of batch processing in large-scale production.

_7

Single Task Failover is a technology that applies batch technology to streaming scenarios. In a streaming task of Flink, if a task fails, the associated upstream and downstream tasks must be restarted globally to ensure data consistency. This global restart is costly, especially for large-scale and complex jobs.

A single-point failover enables you to only restart the current failed task when a failover occurs. Currently, we support three consistency semantics: best-effort, at-least-once, and exactly-once. The stronger the guarantee of consistency, the higher the corresponding overhead. Best-effort needs to restore the task status. In order to solve this problem, we use distributed local snapshots to make scheduled local snapshots for each task to avoid global synchronization overhead. Under at-least-once semantics, we need to replay upstream data to avoid data loss. Under exactly-once semantics, we need to replay the data and deduplicate the data downstream.

Whether it is to replay input or deduplicate output, it is completed at the Shuffle level. They are similar to Blocking Shuffle's semi-persistent data disk and support repeated consumption. Therefore, in practice, we have carried out expansion and secondary development based on the existing batch Shuffle capability.

Single Task Failover work is still in the internal practice stage. At-least-once semantics will soon be launched within Alibaba Cloud, while exactly-once is still under development.

3. Cloud-Native

_8

The Shuffle 3.0 Practice in Cloud-Native Scenarios

Since Flink 1.9, we have been building a Flink cloud-native deployment system, including the deployment mode of Native Kubernetes, application mode of lightweight clients, Native Kubernetes HA mode, and resource management mode of Reactive Scaling.

The Flink cloud-native deployment system is becoming perfect. The production of Flink streaming tasks is mature and has passed a large number of production tests. However, we will still encounter problems when running batch tasks.

_9

Among them, the main problem is batch Shuffle data storage. We need to disk a large amount of intermediate data in batch tasks. Where the data is stored? Flink has two mainstream Shuffle modes: Internal Shuffle and Remote Shuffle.

Internal Shuffle's data is written directly in TM. There are two problems:

  1. Resource Efficiency: The Auto Scaling ability of resources is important in the cloud or cloud computing environment. In the internal shuffle of Flink, when we write data on the local TM, TM cannot release resources timely, which limits the elasticity of computing resources.
  2. Disk Cost: The disk of a physical machine is in a containerized environment. We cannot accurately define how much disk space each TM needs to configure. If the configuration space is large, the cost is high, resulting in a waste of resources. If the configuration space is insufficient, the stability of data processing is affected.

Although a cloud disk has the capabilities of dynamic mounting and shared storage space, its cost is higher than the disk, the access speed is slower than local access, and dynamic mounting is also time-consuming.

In summary, the main problems of Internal Shuffle are resource efficiency and disk costs.

The problem with Remote Shuffle is data transmission overhead. Originally, Shuffle data only needed to be transmitted between two TMs. Now, we need to transmit it from the upstream TM to a remote system and then the downstream TM to consume it from the remote system. This will at least double the cost of transmission.

In addition, we need to deploy Flink clusters and an additional set of Remote Shuffle Service clusters, which will incur some costs and overheads from deployment and O&M.

Finally, Remote Shuffle Service can alleviate disk space and disk cost problems (to a certain extent). Since it can build a Remote Shuffle Service and serve a large number of different Flink instances at the same time, it can play a role in peak cut. However, it cannot fundamentally eliminate disk space problems.

Therefore, neither Internal Shuffle nor Remote Shuffle has a complete solution to store batch data in the cloud-native scenarios of Flink.

_10

When using cloud products, we often use Object Storage Service (OSS). Object Storage Service-based Shuffle has flexible resource elasticity and low cost. However, the Object Storage Service is often unmodifiable. When the upstream writes data, the data is invisible to the downstream. Once the downstream data is visible, the upstream cannot modify or append the data. In addition, its performance still lags behind local disks or cloud disks.

Therefore, Object Storage Service-based shuffle still faces some challenges in stream processing scenarios. On the one hand, it needs to implement the ability to read and write based on an unmodifiable Object Storage Service. On the other hand, it is difficult for Object Storage Service to meet the low-latency demand. Although it is difficult for Object Storage Service to support Shuffle data management independently, when local disks are not enough, Object Storage Service can be used as a supplement to other data storage methods to achieve a balance between performance and cost.

Object Storage Service-based Shuffle is still in the internal practice stage and is expected to be released in Flink 1.18.

4. Adaptivity

_11

In the latest Flink 1.16, Adaptivity has four different types of Shuffle: Pipelined Shuffle, Hash Blocking Shuffle, Sort-Based Blocking, and the latest Hybrid Shuffle. In the future, Flink may introduce Single Task Failover, Object Storage Service Shuffle, Merge-Based Shuffle, and more. In addition, in third-party projects, Flink Remote Shuffle is implemented based on the Flink Shuffle interface.

_12

A large number of different Shuffle implementations exist at the same time, which brings some problems. Users do not know how to select the Shuffle type, which is difficult to use. Select a suitable Shuffle type based on the scenario, which requires users to have an in-depth understanding of the internal principles of Shuffle. After selecting the Shuffle type, in actual production, users also face the problem of different Shuffle type tuning parameters and different principles. In addition, some users may need to use multiple shuffle types at the same time since they have a wide range of scenarios. How do these Shuffle types match? Its complexity brings difficulties to users.

In terms of developer maintenance, as more shuffles appear, the staff needs to maintain more code and even repeat development. In addition, the complexity of Shuffle begins to spread to the entire Flink link (such as SQL compilation, scheduling, and running). It has brought a certain influence on the long-term maintenance of the project.

_13

We propose three methods to improve adaptability.

  1. Complexity Reversal: Adapts Shuffle to external conditions and determines which Shuffle implementation needs to be selected. This reduces the complexity of operations.
  2. Reduce External Information Dependence: We hope to make the best decision based on the information we have. We can convert unnecessary information into supplementary information. At the same time, we can obtain the information that can be automatically obtained as much as possible, reducing the information dependence of Shuffle and other modules.
  3. We hope that Shuffle can automatically adjust its behavior according to changes in the use environment during operation, eliminating the boundaries between different Shuffle types to adapt to the dynamic changes of runtime.

5. Shuffle 3.0

_14

Finally, we introduce the Flink Shuffle 3.0 architecture design based on the preceding keywords. This architecture is called an adaptive tiered storage architecture. In this architecture, we abstract the data exchange process between the upstream and downstream of Shuffle into the process of upstream writing data to a certain storage and then downstream extracting the data to be queried from the storage.

A write selector and a read selector are included in the tiered adaptive storage architecture, which are responsible for writing and reading data to different storage media. The internal implementation details are hidden and have a unified abstraction in the middle storage layer.

In terms of dynamic adaptation, the write end writes data at the storage layer according to the priority. If you encounter a problem (such as insufficient space), the storage layer will feedback that the data cannot be received currently and continue to write to the next priority storage layer. On the read end, we query the desired data in order of priority. Through tiered storage plus dynamic adaptation, we integrate and complement multiple storage layer media to meet our needs in different situations.

_15

In terms of storage layer planning, the Local TM layer mainly includes memory and disks. In the Remote TM layer, users write data to the memory and disk of the third-party TM for management. There is also a remote storage media layer.

_16

We have encountered the following key technical problems in the exploration of Shuffle 3.0 adaptive storage architecture.

In terms of data grouping, the data grouping methods stored in different locations are different, which determines the difference between the data index structure and the file storage format.

In terms of data management granularity, a large granularity is used to switch between storage tiers, which reduces the switching frequency and search cost. Different storage tiers are suitable for different granularities. Inside the storage layer, memory storage is more applicable to a smaller granularity. It requires higher real-time visibility and has lower costs for managing data. We will pay attention to how to reduce the number of files and tend to remote storage services (like Object Storage Service) to have a large data management granularity.

In terms of data indexing, the location of data storage determines the application of different indexing methods. For example, the query performance of the memory index mode is better on the local TM and remote TM. Due to the lack of external service processes in the Object Storage Service, the data is managed. Therefore, we perform a simple list operation on the file based on the file naming method and judge whether the current desired data is in the file according to the file name.

_17

Currently, we are exploring Shuffle 3.0. When Flink 1.18 is launched, the community will launch the first version of tiered adaptive architecture storage, which includes the storage layer of local TM memory and disks and supports remote Object Storage Service capabilities. In the future, we will increase the memory and disk capabilities of stream processing, Single Task Failover, and remote TM.

0 2 1
Share on

Apache Flink Community

132 posts | 41 followers

You may also like

Comments

Apache Flink Community

132 posts | 41 followers

Related Products