You are viewing an old version of this page. View the current version.

Compare with Current View Page History

« Previous Version 8 Next »

Status

Current state[Under Discussion]

Discussion threadhere (<- link to https://mail-archives.apache.org/mod_mbox/flink-dev/)

JIRAhere (<- link to https://issues.apache.org/jira/browse/FLINK-XXXX)

Released: <Flink Version>

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

Motivation

Brief Introduction to Doris  

Apache Doris is a high-performance, real-time analytical database based on MPP architecture, known for its extreme speed and ease of use. It only requires a sub-second response time to return query results under massive data and can support not only high-concurrent point query scenarios but also high-throughput complex analysis scenarios. All this makes Apache Doris an ideal tool for scenarios including report analysis, ad-hoc query, unified data warehouse, and data lake query acceleration. On Apache Doris, users can build various applications, such as user behavior analysis, AB test platform, log retrieval analysis, user portrait analysis, and order analysis.

Apache Doris, formerly known as Palo, was initially created to support Baidu's ad reporting business. It was officially open-sourced in 2017 and donated by Baidu to the Apache Foundation for incubation in July 2018, where it was operated by members of the incubator project management committee under the guidance of Apache mentors. Currently, the Apache Doris community has gathered more than 500 contributors from hundreds of companies in different industries, and the number of active contributors is more than 100 per month. In June 2022, Apache Doris graduated from Apache incubator as a Top-Level Project.

Apache Doris now has a wide user base in China and around the world, and as of today, Apache Doris is used in production environments in over 2000 companies worldwide. Of the top 50 Chinese Internet companies by market capitalization (or valuation), more than 80% are long-term users of Apache Doris, including Baidu, Meituan, Xiaomi, Jingdong, Bytedance, Tencent, NetEase, Kwai, Weibo, and Ke Holdings. It is also widely used in some traditional industries such as finance, energy, manufacturing, and telecommunications.


Why is Flink Doris Connector  useful?

Apache Flink is a popular stream processing framework that allows users to analyze and operate on data on streams in real time.
The Flink Doris Connector allows Flink users to seamlessly integrate Flink with Doris, allowing them to perform real-time data analysis and write the results directly to Doris.
At the same time, with the help of Flink Doris Connector, Flink users can efficiently read data in Doris and analyze it with data from other data sources.


Scenarios for using Flink Doris Connector include:

  1. Data synchronization: Flink Doris Connector can pull out data from Doris and synchronize it to other data sources.
  2. Correlation analysis: Flink Doris Connector can perform correlation analysis and query between other data sources and the data in Doris.
  3. Real-time writing: Flink Doris Connector can write upstream raw data or ETL-cleaned data into Doris.
  4. CDC database synchronization: FlinkCDC is integrated into Flink Doris Connector, which can help users provide entire database synchronization more conveniently.


Overall, the Flink Doris Connector is a powerful tool for enterprises looking to perform real-time data analysis and integrate Flink and Doris workflows. By using this connector, enterprises can get the best of both worlds, leveraging the real-time processing power of Flink with the scalability and cost-effectiveness of Doris.


Public Interfaces

Flink Doris Connector does not introduce any new interfaces or any existing interfaces that will be removed or changed.

Proposed Changes

Overall design

1. Source

There are two types of reading from DorisSource:

1.1. SCAN

Batch reading of Doris data is currently a bounded stream, usually used for data synchronization or joint analysis with other data sources.
1. First, the query will be spliced according to the query and sent to Doris to obtain the query plan.
2. The above Response will return the Tablet and BE node information where the query is located.
3. Use taskmanager to query specific tablet information concurrently


1.2. LOOKUP JOIN

For the scenario where the dimension table is in Doris, lookup join is performed, and JDBC is mainly used for querying.

2. Sink

Writing on the Doris side is mainly done through the Stream Load API , At the same time, Doris Sink will provide two writing methods

2.1. Streaming writing

When the Sink operator receives data, it will initiate a Stream Load request and maintain the http link until the Checkpoint ends to complete the data writing.

Exactly-Once

Stream Load provides two-phase commit api, refer to https://github.com/apache/doris/issues/7141
Combined with Stream Load's two-phase commit, end-to-end data consistency can be achieved based on Flink's two-phase commit.

2.2. Save batch writing

Streaming writing is submitted based on the checkpoint method and is strongly bound to the checkpoint, that is, the data visibility is the checkpoint interval. However, in some scenarios, the delay of user data needs to be decoupled from the checkpoint interval.

Batch writing is to cache data to the Sink, trigger writing based on thresholds such as the number of records, or periodically write the data in the cache to Doris.

Note:that batch writing provides at-least-once semantics and does not guarantee Exactly-Once semantics. However, it can be combined with Doris' primary key table to achieve Exactly-Once.

3. Datatype Mapping

Doris TypeFlink Type
NULL_TYPENULL
BOOLEANBOOLEAN
TINYINTTINYINT
SMALLINTSMALLINT
INTINT
BIGINTBIGINT
FLOATFLOAT
DOUBLEDOUBLE
DATEDATE
DATETIMETIMESTAMP
DECIMALDECIMAL
CHARSTRING
LARGEINTSTRING
VARCHARSTRING
STRINGSTRING
BitmapUnsupported datatype
HLLUnsupported datatype


Compatibility, Deprecation, and Migration Plan

1. Reference to Flink Doris Connecotor through DataStream will be affected (FlinkSQL will not be affected)
The current package path of Flink Doris Connector is org.apache.doris. After migrating to the Flink community, the path may become org.apache.flink.connector.doris. For the DataStream program written by the user, the job may need to be modified.

2. Considering that Flink Doris Connector is already organized in Apache, I think we can just transfer the ownership of the project and rename it. This avoids the need to recreate the project and push code.

Test Plan

1. Unit test cases to test methods in Flink Doris Connector.
2. The E2E test suite will be added in flink-connector-doris-e2e-tests.

Rejected Alternatives

If there are alternative ways of accomplishing the same thing, what were they? The purpose of this section is to motivate why the design is the way it is and not some other way.

  • No labels