By Alibaba Cloud EMR-OLAP Team
The Team is mainly responsible for the R&D of open-source big data OLAP engines, such as ClickHouse, StarRocks, and Trino. The Team also provides Alibaba Cloud users with comprehensive big data OLAP solutions through EMR.
Alibaba Cloud EMR OLAP works with Flink to support Exactly-Once writing from Flink to ClickHouse, which ensures data accuracy in the entire real-time data warehouse. This article describes the solution of an open-source real-time data warehouse based on EMR OLAP.
Contents:
Flink and ClickHouse are leaders in real-time streaming computing and OLAP, respectively. Clients in fields, such as the Internet, advertising, and gaming, have combined the two to build user profiles, real-time BI reports, metric queries of application monitoring, and monitoring services, which form a real-time data warehouse solution (shown in Figure 1). These services have strict requirements for data accuracy. Therefore, the entire procedure of real-time data warehouses needs to ensure end-to-end Exactly-Once.
Generally, the upstream of Flink is pull-based persistent storage (such as Kafka) that can be read or consumed repeatedly. We only need to trace back the reading progress on the Source side to implement Exactly-Once on the Source side. Exactly-Once on the Sink side is more complicated since the Sink is push-based and needs to be guaranteed by the transactions of the target output system. However, ClickHouse does not support transactions.
Therefore, Alibaba Cloud EMR ClickHouse works with Flink to support Exactly-Once writing from Flink to ClickHouse, which ensures the data accuracy of the entire real-time data warehouse. This article will introduce the existing mechanisms and implementation schemes.
Figure 1 - Real-Time Data Warehouse Architecture
ClickHouse is a columnar OLAP system with MPP architecture (shown in Figure 2). Each node is peer-to-peer. A large amount of data can be imported by writing local tables to eac h node concurrently using ZooKeeper to collaborate the data.
The data part of ClickHouse is the smallest unit for data storage. When data Blocks received by ClickHouse are written, they are split based on partitions to form one or more data parts. After the data parts are written to the disk, small data parts are merged into large ones through the background merge thread to reduce the storage and reading overhead.
When writing data to a local table, ClickHouse writes a temporary data part that is invisible to the client. Then, it performs the rename operation to make this temporary data part a formal one. At this time, the data in it is visible to the client. Almost all temporary data parts will be converted into formal data parts by the rename operation. Temporary data parts that are not converted will be deleted from the disk by the ClickHouse cleanup policy.
The preceding analysis shows that the data writing of ClickHouse has a mechanism of conversion from temporary data parts to formal data parts, which can be modified to conform to the two-phase commit protocol. This is an important protocol to realize the consistency of transaction committing in distributed systems.
Figure 2 – Write Flink Tasks to ClickHouse
Note: Multiple Flink tasks can be written to the same shard or replica.
As a distributed processing engine, Flink provides a transaction-based sink mechanism that can ensure Exactly-Once data writing. The corresponding data receiver must provide JDBC that complies with the XA specification. Since the complete XA specification is complex, we analyze the processing mechanism of Flink first and determine the scope of interfaces to be implemented based on the real-world situation of ClickHouse.
Flink uses the checkpoint mechanism to implement unified transaction commit during distributed writing. This mechanism can periodically generate snapshots of the state in each Operator and persist them in storage. There is a Coordinator that coordinates the behavior of all Operators in the checkpoint mechanism. From the perspective of the Operator, a checkpoint has three stages: initialization > snapshot generation > checkpoint completion/deprecation. From the perspective of the Coordinator, you need to trigger the checkpoint regularly and trigger the complete notification after all Operators generate and persist the snapshot. (Refer to Appendix 1)
The following section describes how the Operator in Flink uses the transaction and checkpoint mechanism to ensure Exactly-Once. The complete execution of the Operator includes the initial, writeData, snapshot, commit, and close stages.
After the snapshot stage of all Operators is completed normally, the Coordinator notifies all Operators to perform a complete operation on succeeded checkpoints. In the interaction with ClickHouse, Operators call the commit() interface provided by JDBC to commit the transaction in this stage.
As can be concluded from the preceding process, Flink uses the checkpoint and transaction mechanisms to divide input data into batches based on checkpoint cycles. After all the data is written, this ensures the Coordinator notifies all Operators to complete the commit operation together. When an Operator fails to write, it will return to the status of the last successful checkpoint and perform a rollback operation on all xids of this batch of checkpoints based on the xid recorded in the snapshot. If a commit operation fails, the commit operation will be retried. If it still fails, it will be handled manually.
According to the writing mechanisms of Flink and ClickHouse, we can draw a sequence diagram of transaction writing from Flink to ClickHouse (shown in Figure 3). Since the local tables of ClickHouse are written, and the Coordinator guarantees the unified submission of transactions, ClickHouse does not need to implement the standard distributed transactions in the XA specification. It only needs to implement a few key interfaces in the two-phase commit protocol. Other interfaces are configured by default on the JDBC side.
Figure 3 – A Sequence Diagram of Transaction Writing from Flink to ClickHouse
If we to implement ClickHouse transactions, we need to define several operations that allow the transactions to be implemented:
Transaction Status:
The complete state machine is shown in Figure 4:
Figure 4 – A State Machine that supports ClickHouse Server Transaction
All operations in the figure are idempotent. The steps from Committing to Committed and from Aborting to Aborted require no operations. When Commit or Rollback operation is performed, the transaction status is changed to Committing or Aborting. After Commit or Rollback operation is executed, the transaction status is set to Committed or Aborted.
The Client accesses ClickHouse Server through HTTP Restful API. The interaction process of a complete transaction between Client and ClickHouse Server is shown in Figure 5:
Figure 5 – A Sequence Diagram of ClickHouse Transaction Processing
According to the XA specification, a complete distributed transaction mechanism requires the implementation of a large number of standard interfaces (please refer to Appendix 2). In this design, only a small number of key interfaces need to be implemented. Therefore, the combination-based adapter mode is adopted to provide Flink with standard XAResource implementation based on XA interfaces. At the same time, ClickHouse Server is shielded from interfaces that do not need to be supported.
The inheritance-based adapter mode is adopted to implement XADataSource. Some default configurations, such as the number of retries of sending failure, are modified for the characteristics of Exactly-Once.
In addition, in the production environment, the load balancing of data writing is not usually conducted through distributed tables but SLB. In the Exactly-Once scenario, tasks on the Flink side must be connected to a ClickHouse Server node. Therefore, you cannot use SLB to perform load balancing. We borrowed the idea of BalanceClickHouseDataSource to solve this problem. XADataSource can have the capability of load balancing while ensuring Exactly-Once by configuring multiple IP addresses in the URL and setting write_mode to Random in the properties configuration.
As a streaming data processing engine, Flink supports data writing to various data receivers. Each receiver needs to implement a specific Connector. For Exactly-Once, ClickHouse Connector adds options for XADataSource to provide the Exactly-Once feature based on the configuration of the client.
Figure 6 – ClickHouse Writing Performance Stress Testing (1)
As shown in Figure 7, the throughput of ClickHouse is proportional to the volume of data in a single batch, regardless of whether ClickHouse enables transactions. When a transaction is enabled, smaller amounts of data in each batch result in a greater ClickHouse throughput difference. This happens because the time spent writing each batch is relatively small in the transaction process, and the transaction will have a certain impact on this. Therefore, more batches in a transaction result in a reduced impact on writing performance. When the transaction contains more batches, the proportion of transaction processing time in writing decreases, and the ClickHouse merge has an increasing impact. This affects the performance of writing. The performance when transactions are enabled is better than when it is not enabled.
Figure 7 – Stress Testing of ClickHouse Writing Performance (2)
Figure 8 – Data Writing Test from Flink to ClickHouse
The transactions implemented in this version of EMR ClickHouse are incomplete. Standalone transactions are supported, but distributed transactions are not. Distributed systems generally use Meta Server for unified metadata management to support the distributed transaction mechanism. We are also planning to design ClickHouse MetaServer to support distributed transactions and remove the dependency of ClickHouse on ZooKeeper.
[1] https://flink.apache.org/2020/10/15/from-aligned-to-unaligned-checkpoints-part-1.html
[2] https://pubs.opengroup.org/onlinepubs/009680699/toc.pdf
Best Practices for Flink on Zeppelin Stream Computing Processing
How to Build a Cloud-Native Open-Source Big Data Platform | Best Practices of InMobi
59 posts | 6 followers
FollowAlibaba EMR - January 10, 2023
Alibaba EMR - March 18, 2022
Alibaba EMR - March 18, 2022
Alibaba EMR - March 18, 2022
Hologres - July 7, 2021
ApsaraDB - June 7, 2022
59 posts | 6 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 real-time data warehouse for serving and analytics which is compatible with PostgreSQL.
Learn MoreMore Posts by Alibaba EMR