By Gaodi Zhang (Yaotian), Hologres R&D
Like traditional MySQL databases, Hologres supports Hologres Binlog, which records event logs of all data changes in the database. With Hologres Binlog, data replication and synchronization among different datasets can be easily and flexibly achieved. In big data scenarios, Hologres supports direct consumption of Hologres Binlog by Flink. Compared to the traditional data warehouse layering, Flink and Hologres Binlog enable complete event-driven processing, covering real-time processing from ODS to DWD and DWD to DWS. This allows for unified storage under layered governance, enhancing data reusability and significantly reducing end-to-end data processing latency. As a result, users can benefit from a comprehensive real-time data warehouse solution. This article will provide an overview of the implementation principles and best practices of Hologres Binlog.
Binlog, short for Binary Log, is commonly used in MySQL databases to record all events that may cause data changes. These events include table schema changes (e.g., CREATE and ALTER TABLE) and table data changes (e.g., INSERT, UPDATE, and DELETE).
Initially, MySQL Binlog served two main purposes:
Binlog is also used as a streaming data source. Users often utilize data collection tools like Debezium or the open-source Canal provided by Alibaba Group to collect Binlog as streaming data sources. This allows for data synchronization from MySQL to other types of databases. Typically, the collected data is first sent to message middleware such as Kafka. The Flink engine then consumes the data, performs calculations, and eventually writes the data to the desired target, which can be databases, data lakes, real-time data warehouses, or offline data warehouses.
The main difference between Hologres Binlog and traditional database Binlog is that the former is primarily used for data synchronization, while the latter is also utilized in high-availability scenarios like primary-secondary instance synchronization and data restoration. Consequently, there are some implementation differences between the two, mainly in the following aspects:
1. Hologres Binlog do not record DDL operations.
Hologres Binlog focuses on data consumption and does not record table schema changes such as CREATE TABLE and ALTER TABLE (DDL statements). It only records data changes (INSERT, UPDATE, and DELETE) in a manner similar to the ROW (row-based replication) mode of MySQL Binlog. This feature records the changes in each row of data.
2. Hologres Binlog is flexible and table-level, allowing you to enable or disable Binlog based on your business requirements and specify different TTL (Time To Live) for different tables.
In contrast, MySQL Binlog operates at the instance level, occupying a significant amount of storage and requiring a cluster restart when enabled. However, Hologres Binlog offers more granularity, allowing you to enable or disable Binlog for specific tables as needed. This does not affect other databases and tables in the instance. Enabling the Binlog feature only adds an additional table-level storage.
3. Hologres Binlog can be easily queried.
During usage, you can query the Binlog of a table by adding the hg_binlog_lsn
, hg_binlog_event_type
, and hg_binlog_timetsamp_us
fields in the query. If a query contains one of the preceding three fields, such as select hg_binlog_lsn, * from test_message_src;
, the query is automatically routed to the Binlog table. In contrast, MySQL users cannot directly view Binlog. They need to use the "show binlog events" command or MySQL Binlog tools to parse Binlog files.
Please note that Hologres Binlog uses row-oriented storage at the underlying layer. Therefore, we recommend including the hg_binlog_lsn field in filter conditions to ensure query efficiency. Querying data based on other business fields may result in a full table scan, which can be slow for large Hologres Binlog tables and should be avoided..
4. Hologres is a distributed real-time data warehouse, and Binlog is also distributed.
Similar to data in a Hologres table, Hologres Binlog is distributed across different shards. Operations such as reading Binlog are also fragmented by shards, similar to the partitioning of Kafka. For more information, refer to the implementation principle below.
In summary, Hologres Binlog can be defined as Binlog that can be enabled or disabled as needed to record data changes (INSERT, UPDATE, and DELETE) of a Hologres table
The following shows the fields of Hologres Binlog.
Field description:
Field name | Field type | Description |
hg_binlog_lsn | BIGINT | The ordinal number of the current Binlog. This field is a Binlog system field. For Binlog in the same shard, the values of this field monotonically increase and may be discontinuous. Values of this field in Binlog in different shards may be identical and out of order. |
hg_binlog_event_type | BIGINT | The type of operation recorded in the current Binlog. This field is a Binlog system field. There are the following valid values: • INSERT=5, which indicates that a row or column is inserted. • DELETE=2, which indicates that a row or column is deleted. • BEFORE_UPDATE=3, which indicates that a row or column is saved before it is updated. • AFTER_UPDATE=7, which indicates that a row or column is saved after it is updated. • HEARTBEAT_LOG_EVENT= 27, which indicates that the Binlog in the current shard generated before or at the timestamp specified by the hg_binlog_timestamp_us field have been consumed. This value takes effect only if Binlog are consumed by using JDBC or Holo Client and the Grand Unified Configuration (GUC) parameter hg_experimental_enable_binlog_heartbeat_record is set to on. For more information, see Use JDBC to consume Hologres Binlog (Beta). |
hg_binlog_timestamp_us | BIGINT | The timestamp of the system. Unit: microseconds. This field is a Binlog system field. |
user_table_column_1 | User-defined | A user-defined table field. |
... | ... | ... |
user_table_column_n | User-defined | A user-defined table field. |
Additional information:
• An UPDATE operation generates two Binlog, which indicate that a row or column is saved before and after the update. Binlog ensures that the ordinal numbers of the two Binlog are continuous and in the right order. The log that is generated after the update comes after the log that is generated before the update.
• The order of user-defined fields is consistent with the order that is defined in a DDL statement.
Hologres implements logical replication by using Binlog. This allows you to subscribe to Binlog for data replication and synchronization.
Typical logical replication scenarios:
1. Copying a row-oriented table in Hologres to a column-oriented table. Row-oriented tables provide point query and write capabilities, while column-oriented tables support multi-dimensional analytical requirements.
For example, in Alibaba Chief Customer Office (CCO) scenarios, data written to a row-oriented storage table in Hologres is subscribed through Hologres Connector Binlog. The common layer's detailed data is selectively recomputed and written back to the column-oriented storage details table at the application layer. This facilitates various application layer analysis and summary scenarios. Starting from Hologres version 1.1, row-oriented and column-oriented storage can coexist in the same table, reducing the need for manually converting row-oriented storage to column-oriented storage using Binlog.
2. Performing full row calculations after partial updates using Binlog
In some scenarios, when data at the DWD layer is updated, the downstream computational logic requires complete table fields. However, traditional data warehouses may not fully support this requirement. For example, in the previous architecture of Alibaba Cloud CCO, Lindorm was used. To obtain complete table fields, subscribing to Hlog, triggering the stream task to query the fact table, aligning wide table fields, and then outputting them downstream were necessary. With Hologres Binlog, even if only part of the fields are updated, a Binlog that contains the entire row of data is generated. This feature enables partial updates to drive the calculation of the entire row of data, which can be consumed by the application layer without the need for reverse searches.
3. Real-time data broadening
In certain scenarios, user data is separated into multiple tables based on topics and dimensions. In the final analysis, different tables need to be associated. Hologres Binlog can be used as the driver for operations such as dual-stream joins and dimension table associations. This enables real-time data broadening, followed by analysis using large wide tables.
4. Migrate data between different Hologres versions or instances
During development, many users have test instances and business instances. Using Binlog to synchronize part of the business data to test instances in real-time is convenient for new business development. When performing cross-version upgrades, some users can also use Binlog's logical replication to perform complete workflow verification on high-version instances. This ensures that the instance remains stable and available after the upgrade.
You can use Hologres Binlog and Flink CDC to implement event-driven data processing and development, such as real-time data processing from ODS to DWD and DWD to DWS. Based on layered governance, unified storage can greatly shorten the end-to-end data processing latency, providing users with a comprehensive real-time data warehouse solution.
The following is a traditional layered architecture of data warehouses.
The traditional layered architecture of data warehouses uses Kafka to drive Flink. Flink queries some KeyValue systems (such as HBase) to associate some dimension tables during computing to achieve data widening. After the widening, the result is re-written to another topic in Kafka and then aggregated and summarized twice to generate some DWS or ADS. Finally, the result is stored in the OLAP or HBase system. The results need to exist in both KV and data warehouse systems because the query and analysis business has different requirements, and a piece of data may need to exist in multiple places, so the traditional layered architecture of data warehouses often has some data redundancy.
The above figure shows the real-time data warehouse architecture implemented by Flink and Hologres. The architecture can achieve analysis service integration. This architecture not only integrates analysis services to reduce data fragmentation, but also streamlines data processing links. Hologres Binlog are used to implement event-driven capability, replacing the Message Queuing Services (Kafka) in each stage. As you can see, the layered architecture of data warehouses integrated with analysis services can not only effectively reduce costs, but also simplify business logic, thus greatly improving development efficiency and reducing operation and maintenance costs.
In database usage, especially in the development of new businesses, there may be scenarios where you want to know about data changes and determine whether your business logic is accurate. In this case, you can also enable Binlog to monitor data change.
The storage structure of a Hologres table is divided into row-oriented storage, column-oriented storage, and row-column hybrid storage introduced in version 1.1. Column-oriented storage tables are suitable for OLAP scenarios, including complex queries, data association, scanning, filtering, and statistics. Row-oriented storage tables are suitable for key-value (KV) scenarios, including point queries and scans based on primary keys. Row-column hybrid storage tables can be regarded as creating a row-oriented storage table and a column-oriented storage table at the same time. HOO ensures data consistency, achieving the effect of supporting both efficient point query and OLAP analysis.
A Binlog table can be regarded as a special row-oriented storage table.
The following uses a row-oriented storage table as an example to describe how Binlog is implemented.
A common row-oriented storage table is as follows. You can see that it is in the KV format. The key is the primary key of the table, and the value is other fields of the table.
Enabling Binlog for a table can be considered as creating a new row-oriented storage table with hg_binlog_lsn as the key and the business table original fields, hg_binlog_event_type, and hg_binlog_timestamp_us fields combined as the value. You can see that the Binlog table fields are fixed and have a strong schema. The order is consistent with the order defined by the business table DDL.
The following figure shows a row-oriented storage table with Binlog enabled. Note that instead of adding columns to the original table, a new internal special table is created (represented as a special index of the original table in holo, so operations such as writing on the original table and Binlog table are atomic). Both exist at the same time. Therefore, if you enable Binlog, more storage space is occupied.
How to ensure the correctness of Binlog
Let's briefly review the storage engine architecture and the process of writing to a single shard.
• Storage engine architecture
The basic abstraction of the Hologres storage engine is distributed tables. To make the system scalable, we divide tables into shards (table group shard, or shard for short). Each shard constitutes a storage management and recovery unit. The above figure shows the basic architecture of a shard. A shard consists of multiple tablets that share a write-ahead log (WAL).
The Hologres storage engine uses WAL to ensure data atomicity and durability. When INSERT, UPDATE, or DELETE operations occur, the storage engine writes the WAL first and then writes the WAL to the MemTable of the corresponding tablet. After the MemTable has accumulated to a certain size or for a certain period of time, the MemTable is switched to an immutable flushing MemTable, and a new MemTable is opened to receive new write requests. The immutable flushing MemTable can be flushed to the disk and become an immutable file. When the immutable file is generated, the data is persistent. When the system crashes due to an error, the system will go to WAL to read logs when it restarts and restore the data that has not persisted.
• Single-shard writing
The above figure shows the process of single-shard writing in Hologres. Here, we only focus on the first two steps: After receiving a single-shard writing request, the WAL manager (1) assigns a Log Sequence Number (LSN, consisting of a timestamp and an incremented sequence number, can be queried through hg_sequence_number, a hidden field in the holo table) for the write request, and (2) creates a new log and makes this log persistent in the file system. This log contains the information needed to resume the write operation, and writes are not committed to tablets until this log is fully retained.
• Binlog generation time
Binlog data is generated before the WAL is written, which is the first step in single-shard writing described above. The hg_binlog_lsn directly reuses the Log Sequence Number (query the hg_sequence_number and hg_binlog_lsn fields in the Binlog table, and you can find that the two fields are exactly the same). Use the system time as the hg_binlog_timestamp_us when generating the Log Sequence Number. The write type is determined before the WAL log is created to complete the write. In Holo, the Binlog table is a special index of the original table. Data changes in the original table and the Binlog table occur simultaneously. This ensures that the Binlog data and the original table data are completely consistent.
The following example uses a row-oriented storage table and a column-oriented storage table to demonstrate how to generate a Binlog.
The change process of a row-oriented storage table with Binlog feature enabled.
In the figure, three operations are performed:
The update process of a column-oriented storage table with Binlog feature enabled.
A common column-oriented storage table is as follows. You can see that it is significantly different from a row-oriented storage table in that data is stored by column.
The following figure shows the process of generating Binlog by updating a row-oriented storage table with three data records. Similar to the row-oriented storage table, the update operation generates two Binlog whose event_type parameters are 3 and 7 and have the same timestamp_us. It should be noted that this update operation only updates the body field. Because Binlog record the data of all fields in the whole row, in the process of generating Binlog, it is necessary to check the values of each field in this row of data in the column-oriented storage table through id, which is not the operation that a column-oriented storage table is good at and will consume more resources than row-oriented storage table. Therefore, we recommend that you enable the Binlog feature with row-oriented storage tables.
Different from the TTL (time_to_live_in_seconds) of a table, Binlog have a separate TTL, which can be regarded as the TTL of a Binlog table. Use the following DDL statement to set the Binlog TTL for the table separately. Binlog will be cleared after their lifetime expires, so we recommend that you set this time to no less than seven days.
call set_table_property('test_message_src', 'binlog.ttl', '86400'); -- Configure the binlog.ttl property to specify the TTL of Binlog. Unit: seconds.
The Binlog table has the same number of shards as the original table and also has the same Distribution key, which means that a piece of data and its Binlog data must be distributed on the same shard.
Binlog are single-shard order-preserving. The hg_binlog_lsn on a single shard increases monotonically. Different shards may have the same hg_binlog_lsn. In addition to the two hg_binlog_timestamp_us generated by the update operation being identical, the hg_binlog_timestamp_us on a single shard is also incremented monotonically.
After understanding the implementation principle of Binlog, let's take a look at the best practices for using Binlog.
The recommended table creation statement is as follows:
begin;
create table test_message_src(
id int primary key,
title text not null,
body text);
call set_table_property('test_message_src', 'orientation', 'row');-- Create a row-oriented table named test_message_src.
call set_table_property('test_message_src', 'binlog.level', 'replica');-- Configure the binlog.level property to enable Binlog.
call set_table_property('test_message_src', 'binlog.ttl', '864000');--binlog.ttl, the TTL of Binlog. Unit: seconds. The time is seven days.
commit;
There are several details to note in the DDL statements above:
Currently, Hologres Binlog support two consumption methods.
Realtime Compute for Apache Flink can consume Binlog of Hologres. The following are DDL statements used to define a Hologres Binlog source table.
create table binlog_source_table(
hg_binlog_lsn BIGINT,
hg_binlog_event_type BIGINT,
hg_binlog_timestamp_us BIGINT,
id INTEGER,
title VARCHAR,
body VARCHAR
) with (
'connector'='hologres',
'dbname'='<yourDbname>',
'tablename'='<yourTablename>',
'username'='<yourAccessID>',
'password'='<yourAccessSecret>',
'endpoint'='<yourEndpoint>',
'binlog' = 'true', -- Enable Binlog consumption.
'cdcMode' = 'true', -- CDC mode
'binlogStartUpMode' = 'initial', -- Read full historical data first and then consume Binlog incrementally.
);
When Flink consumes Binlog, it has the following features and usage details:
1) Support CDC mode.
For Binlog data consumed in CDC mode, the exact Flink RowKind types (INSERT, DELETE, UPDATE_BEFORE, and UPDATE_AFTER) are automatically set for each row of data based on the hg_binlog_event_type. In this way, the image replication of table data can be completed, similar to the CDC function of MySQL and Postgres.
2) Support full and incremental consumption
Full and incremental consumption will read the full historical data of the database first and smoothly switches to Binlog to read incremental data.
3) We recommend that you adjust the concurrency of the source table based on the number of shards in the table.
The order of Binlog in Hologres is preserved at the shard level. When you use Realtime Compute to consume Binlog, it is recommended to adjust the concurrency number of the source table to be the same as the shard number of the Binlog table. This way, each shard corresponds to one concurrency, and the data order is ensured.
Hologres is compatible with the logical replication interface of PostgreSQL to a certain extent. You can use the corresponding interface to consume Binlog of Hologres by using JDBC. The following is a simple example. For more information about permission and cleaning up created components, see Use JDBC to consume Hologres Binlog.
Preparation steps before use:
1. First, prepare a table to enable the Binlog feature. For more information, see the best practices above.
2. Open the extension of the Binlog. The extension is at the database level, and a database needs to be created only once.
create extension hg_binlog;
3. Create a publication for the table, which has a one-to-one relationship with the table.
create publication hg_publication_test_1 for table test_message_src;
4. Create a replication slot and bind it to a publication. A publication can be bound to multiple replication_slot.
A replication slot indicates a change stream of data. The replication slot is also bound to the current consumption progress and maintains the point information of Binlog consumption so that the consumer can recover from the point committed after failover and continue transferring from the breakpoint.
call hg_create_logical_replication_slot('replication_slot_name', 'hgoutput', 'publication_name');
Use JDBC to consume Binlog in the following way:
// Create a PGReplicationStream and bind it to the replication slot.
PGReplicationStream pgReplicationStream = pgConnection.getReplicationAPI().replicationStream()
.logical()
.withSlotName("hg_replication_slot_1") // Specify the name of the replication slot.
.withSlotOption("parallel_index", "0") // Concurrent sequence number, which corresponds to the shard ID of the table.
.withSlotOption("batch_size", "1024") // The maximum batch size of Binlog that can be obtained at a single time.
.withSlotOption("start_time", "2021-01-01 00:00:00") // Start consuming at a certain point in time.
.withSlotOption("start_lsn","0") // Start consuming after an lsn, with higher priority than start_time.
.start();
// Create a Holo Client object.
HoloConfig holoConfig = new HoloConfig();
holoConfig.setJdbcUrl(url);
holoConfig.setUsername(username);
holoConfig.setPassword(password);
HoloClient client = new HoloClient(holoConfig);
// Create a Binlog decoder and obtain the table schema by using Holo Client.
TableSchema schema = client.getTableSchema("test_message_src", true);
HoloBinlogDecoder decoder = new HoloBinlogDecoder(schema);
// Consume data.
ByteBuffer byteBuffer = pgReplicationStream.readPending();
while (true) {
if (byteBuffer != null) {
List<BinlogRecord> records = decoder.decode(byteBuffer);
Long latestLsn = 0L;
for (BinlogRecord record : records) {
latestLsn = record.getBinlogLsn();
// Do Something
System.out.println( "lsn: " + latestLsn + ", record: " + Arrays.toString(record.getValues()));
}
// Commit the Binlog offset.
pgReplicationStream.setFlushedLSN(LogSequenceNumber.valueOf(latestLsn));
pgReplicationStream.forceUpdateStatus();
}
byteBuffer = pgReplicationStream.readPending();
}
Holo-Client integrates JDBC's ability to consume Binlog. You can specify the start time to consume Binlog of a table. You can also specify a start point for each shard. We recommend that you use Holo-client to consume Binlog.
Preparation operations such as table creation are the same as using JDBC to consume Hologres Binlog. The following example shows how to consume Hologres Binlog on a client.
import com.alibaba.hologres.client.BinlogShardGroupReader;
import com.alibaba.hologres.client.HoloConfig;
import com.alibaba.hologres.client.HoloClient;
import com.alibaba.hologres.client.model.Record;
import com.alibaba.hologres.client.model.TableSchema;
import java.util.Arrays;
public class HoloBinlogExample {
public static void main(String[] args) throws Exception {
String username = "";
String password = "";
String url = "jdbc:postgresql://ip:port/database";
String tableName = "test_message_src";
String slotName = "hg_replication_slot_1";
// Create the client parameter.
HoloConfig holoConfig = new HoloConfig();
holoConfig.setJdbcUrl(url);
holoConfig.setUsername(username);
holoConfig.setPassword(password);
holoConfig.setBinlogReadBatchSize(128);
HoloClient client = new HoloClient(holoConfig);
// Initiate a request to consume Binlog. The tableName parameter and the slotName parameter are required parameters. You can use a StartTimeBuilder object or an OffsetBuilder object to subscribe to the consumption progress. In the following example, StartTimeBuilder is used:
Subscribe subscribe = Subscribe.newStartTimeBuilder(tableName, slotName)
.setBinlogReadStartTime("2021-01-01 12:00:00+08")
.build();
// Create a Binlog reader.
BinlogShardGroupReader reader = client.binlogSubscribe(subscribe);
BinlogRecord record;
while ((record = reader.getBinlogRecord()) != null) {
//handle record
}
}
}
You can specify the parameters that are described in the following table when you use Holo Client to consume Binlog.
Parameter | Required | Default value | Description |
binlogReadBatchSize | No | 1024 | The maximum number of Binlog to consume in each batch for each shard. Unit: rows. |
binlogHeartBeatIntervalMs | No | -1 | The intervals at which the Binlog reader sends heartbeat records. If you set this parameter to -1, the Binlog reader does not send heartbeat records. If the Binlog contains no new data, a BinlogHeartBeatRecord is sent every binlogHeartBeatIntervalMs interval. The timestamp of each heartbeat record indicates that Binlog generated before this timestamp are all consumed. |
binlogIgnoreDelete | No | false | Specifies whether to ignore Binlog of the DELETE type. |
binlogIgnoreBeforeUpdate | No | false | Specifies whether to ignore Binlog of the BEFORE_UPDATE type. |
retryCount | No | 3 | The number of retries when the consumption fails. The number of retries is reset when the consumption succeeds. |
The Binlog feature of Hologres is designed for consumption and can be easily enabled at the table granularity level. It provides flexibility and ease of use. By using Hologres Binlog in conjunction with Flink CDC, you can achieve event-driven data processing and development, such as real-time processing of data from ODS to DWD and DWD to DWS. With a layered governance approach and unified storage, it significantly reduces the end-to-end data processing latency, lowers the learning curve, and improves development efficiency.
Hologres Technology: Extreme Analysis Performance of JSON Semi-structured Data
Exploring DevOps in the Era of AI Foundation Models Part I: ReAct Engineering
40 posts | 1 followers
FollowAlibaba Cloud MaxCompute - January 21, 2022
Hologres - July 13, 2021
Alibaba Clouder - July 1, 2020
Hologres - December 2, 2022
Hologres - June 30, 2021
Hologres - June 16, 2022
40 posts | 1 followers
FollowA real-time data warehouse for serving and analytics which is compatible with PostgreSQL.
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 MoreHelp media companies build a discovery service for their customers to find the most appropriate content.
Learn MoreMore Posts by Alibaba Cloud Data Intelligence