The following is the content of the article:
In Flink Forward Asia 2020, our team shared a topic entitled "The Past, Present, and Future of Pravega Flink Connector." The topic introduced the process of Pravega Flink Connector FLIP-95 Table API from zero to one, enabling the Pravega stream to establish links with the Flink table API and transform each other. Users can use simple SQL statements to query and write data in the Pravega stream. Over a year later, we have also made some advanced functions based on promoting the integration of flow and tables, further enriching the usage scenarios and simplifying the difficulty of use. Next, this article takes Catalog API integration and Debezium support as examples to explore the technical details of these advanced functions.
Pravega Schema Registry is a new function introduced by the Pravega ecological community in 2020, similar to Confluent Schema Registry and AWS Glue, which is a common solution that uses Pravega to store and manage data schema structures. Depending on Pravega Schema Registry, you can implement the Catalog API in the connector to implement the functions of the Pravega catalog. You can use Flink to access data in the name of SQL statements without rewriting the DDL statement of CREATE TABLE to establish a connection with a Pravega stream.
Pravega stream stores raw binary data after serialization. This requires the read and write ends to agree on the schema of the written data. This is easy to solve on a small scale of data and development. However, when the scale of load continues to expand, it is difficult to face hundreds of stream reads and writes and the case that multiple development departments cooperate. This requires Pravega Schema Registry components to ensure schema consistency.
As shown in the preceding figure, when the writer writes an event, the reader must deserialize this event with the same schema to obtain accurate data from the binary. Some sensitive information may even be further encrypted and encoded based on metadata.
We need to have centralized schema metadata for Pravega's stream, including the storage of the encoding format, which allows writing and definition. Then, all readers can read it, obtain the data structure and anti-sequence method, and reach a schema consensus.
We hope this kind of metadata storage does not need to rely on additional components. It is best to use Pravega to save the cost of operation and maintenance and take advantage of Pravega's efficient and persistent storage features.
Schema change is a common business scenario. In the same stream, as the business expands, the written semi-structured data may introduce new fields or change some fields to support more workflow and jobs. This is compatible with many format standards, such as Avro. Specifically, when another writer writes the data of the new schema, we need to ensure that the original online reader can continue to work and support data reading for the new reader that needs to use the new field. Therefore, Pravega Schema Registry should ensure that it is unnecessary to wait for the reader to try deserialization to know the change but to configure format compatibility on the writer side and make some intervention when registering a new schema. Therefore, the management of reading and writing client applications can be carried out quickly.
Based on these motives, we developed Pravega Schema Registry.
It is a schema for storing and managing semi-structured data in Pravega. It provides RESTful interfaces to manage the stored schema, encoding format of data, and compatibility policies. The interfaces we provide are open. It has built-in common serialization formats (such as Avro, protobuf, and JSON) and common compression algorithms (such as lz4 and snappy) but also supports some custom serialization methods. This abstraction is a more general solution than other similar projects in the industry. In all serialization management, we can customize the corresponding encoding format and compatibility policy, so users can freely use serialization methods to process data. The entire project is stored with the Pravega Key Value Table function, which is the underlying implementation of the Schema Registry. Pravega Key Value Table was only one of the components that stored metadata inside Pravega previously. We developed a public API, made the 0.8 version publicly available, and guaranteed a stable version of the beta in the 0.10 version.
Pravega Schema Registry maintains relative independence. Besides the Pravega key value table used in the underlying implementation, all abstractions in the upper layer are independent and not limited to Pravega. The whole project is not open-source inside Pravega but is a separate project in the ecology. As such, it can be used by more common storage systems, including common files and object storage services, as a solution to schema management.
The system architecture of the whole project is shown in the picture below:
Schema Registry can interact with clients through RESTful APIs and GRPC protocols. Group corresponds to a schema management unit, and Pravega corresponds to a stream, which stores the default serialization format, compatibility configuration, serialization information, and encoding information of multiple versions. As mentioned before, this information is stored on the Pravega segment in the form of key-value pairs.
From the data link point of view, the writing side needs to use a special event serialization method with header bytes of protocol version and encoding id. Therefore, the Schema Registry can intervene to register or verify the schema of the data to check whether it meets the encoding and compatibility requirements. Then, it allows the valid data of the schema to enter the storage of the Pravega stream. Similarly, the read side also needs to use this special deserialization for reading.
With Schema Registry, the mapping relationship between Catalog and Pravega is clear.
As shown in the figure, the stream data in the stream plus the schema stored in Schema Registry can deserialize and synthesize a table in Flink Catalog according to the table structure. Similarly, the table-to-stream conversion can be completed in the opposite link. Therefore, traditional Catalog operations, such as table creation, table deletion, and database deletion, can be abstracted as metadata changes to Pravega and Schema Registry.
Based on this theory, we have initially implemented the catalog interface, allowing users to use the following DDL to establish a catalog and SQL to manipulate Pravega's metadata.
CREATE CATALOG pravega_catalog WITH(
'type' = 'pravega',
'default-database' = 'scope1',
'controller-uri' = 'tcp://localhost:9090',
'schema-registry-uri' = 'http://localhost:9092'
);
However, when we polished the prototype of the first edition, we encountered three difficulties:
RowData
of row records in Flink tables was the internal and private class of Flink under format. Therefore, if we want to reuse this part of the code to maintain consistency with Flink's schema and serialization conversion behavior, we have to copy the entire class library. Hence, we put forward such an idea to the community. Can it be abstracted and converted into public classes? After communication, we opened the corresponding JIRA FLINK-19098 and contributed code to fix this problem. We opened the link of data serialization converted in the Catalog table.CREATE TABLE clicks (
...
) WITH (
'connector' = 'pravega',
...
'format' = 'pravega-registry',
'pravega-registry.uri' = 'tcp://localhost:9092',
'pravega-registry.namespace' = 'scope1',
'pravega-registry.group-id' = 'stream1',
'pravega-registry.format' = 'Avro'
);
Then, we get this information in the implementation to call the API of Schema Registry and fix the problems mentioned earlier. We have completed interoperability between the complete binary data and Flink RowData
. Thus, the entire link is connected.
First of all, let's introduce the concept of CDC. The full name of CDC is Change Data Capture, which is a methodology for identifying and tracking data changes and taking action. However, it is a broad concept. In the practical experience of modern industries, CDC is a narrow sense of technical terms, which means a new technology for database scenarios, for database log analysis, and then transforming it into a specific format of data flow, such as Debezium and Canal.
As the most widely used CDC technology in the industry, Debezium is implemented based on Kafka Connect and is a distributed platform that transforms database row-level changes into event streams. Currently, the application scenarios of CDC technology in the industry are extensive, including data synchronization for backup and disaster recovery, data distribution to multiple downstream systems, and ETL integration for connecting to data lakes.
Currently, Debezium is deployed in three ways:
We maintained the integrity of the implementation and provided two write methods during the integration of Debezium and Pravega: general write and transactional write.
Debezium server is a periodic batch pull process in the source connector. The interface side will receive an upsert stream of the Debezium batch. As shown in the figure on the right, each bracket is a batch pulled once, where yellow represents update and white represents insert.
If it is a normal write, we will write all events, whether it is inserted or updated, sequentially in the form of individual events. You don't have to worry about the order in the distributed environment since each Debezium event carries the key in the database table. When Pravega writes, it can carry the corresponding routing key to make the event of the same key in the same segment. Pravega can ensure the order of data on the same routing key.
The next one is transactional writing. For each Debezium batch, Pravega encapsulates it in a transaction and submits the Pravega transaction when the batch is complete. As such, when there is a failover in the Debezium server, all events can be played back and output without repetition due to the atomicity and idempotence of the Pravega transaction, thus ensuring accurate semantics.
After version 1.6, users can use the parameters shown in the table to configure the Debezium server. They only need to fill in the corresponding Pravega connection parameters and specify the scope name and transactional write switch to synchronize the changes of all tables in a database (such as MySQL in real-time) and write them into the Pravega stream with the same name as the table in the message format of Debezium.
In addition to the contribution of Pravega in Debezium, we need to integrate the Pravega Flink connector side of the computing end to consume data using the stream processing of Flink.
We have implemented Table Factory on the existing FLIP-95 Table API to support basic read and write functions. The community has provided debezium-json
implementation of format factory, which can be applied directly using the same format. People that know the Flink Table API may ask one question. What are the difficulties?
At first, we thought so too, but things are far from simple. There are two main difficulties.
One difficulty is the need for additional support for the inverse sequence of multiple events.
Debezium provides upsert streams, which is a common one-to-one deserialization process for inserting events when converting the Rowdata
abstraction of the Flink table. However, we need to convert to two events in the pre-update and post-update states to update. This is contrary to our default one-to-one implementation of connector serialization.
Since Pravega's serializer interface is a one-to-one mapping, we have made a lot of code support to ensure the interoperability between the Pravega serializer and Flink DeserializationSchema interface, which is convenient for users. If we want to meet this new requirement, we have to reconstruct the previous deserialization link and introduce the original deserialization process from the Pravega client to the inside of the Pravega connector, thus using the following deserialize method with the collector.
default void deserialize(byte[] message, Collector<T> out) throws IOException {
At the same time, the entire code modification link needs to be careful to ensure the compatibility between the original Pravega serializer and the Flink API conversion API. This process will not affect the upgradeability of online users.
The other difficulty is the metadata support of FLIP-107 on the Table source side.
Pravega Table Source implements the SupportsReadingMetadata
interfaces provided by Flink and provides support. Users can use the from-format
prefix to specify metadata from the format factory, such as the ingestion timestamp of Debezium, the table name, and other metadata to refine and enrich the table information. At the same time, we support metadata from Pravega, which is called EventPointer
, to record the location information of the current event in the stream. Recording this information helps users store this data and complete the subsequent random reading and even indexing requirements. The following is the DDL statement for creating a table that contains metadata. These new columns are listed in sequence after the original data structure in the abstract of the Flink table.
CREATE TABLE debezium_source (
// Raw Schema
id INT NOT NULL,
// Format metadata
origin_ts TIMESTAMP(3) METADATA FROM 'from_format.ingestion-timestamp' VIRTUAL,
origin_table STRING METADATA FROM 'from_format.source.table' VIRTUAL,
// Connector metadata
event_pointer BYTES METADATA VIRTUAL
);
The section above is the complete Debezium support process from Pravega to Connector.
Flink CDC Series – Part 1: How Flink CDC Simplifies Real-Time Data Ingestion
150 posts | 43 followers
FollowApache Flink Community China - July 21, 2020
Apache Flink Community China - August 12, 2022
Apache Flink Community - June 28, 2024
Apache Flink Community China - January 11, 2021
Apache Flink Community China - September 16, 2020
Alibaba Clouder - April 25, 2021
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 MoreApsaraDB for HBase is a NoSQL database engine that is highly optimized and 100% compatible with the community edition of HBase.
Learn MoreMore Posts by Apache Flink Community