The author of this article is Ding Yang, who works at the R&D Center of the Agricultural Bank of China. He downloaded, used Flink CDC version 2.1 as soon as it was released, and realized real-time data capture and performance tuning of Oracle. This article shares the details of the trial process. The main contents include:
Flink CDC released the latest version 2.1 on November 15, 2021. This version adds support for Oracle by introducing the built-in Debezium component. The author downloaded this version and realized the real-time data capture and performance tuning of Oracle. Now, I will share some details of the trial process.
Note: This article aims to share some useful information based on a troubleshooting experience and internal execution principles. Therefore, Flink CDC and the basic usage of its built-in Debezium module are not covered. Readers can refer to the following addresses for basic usage methods and parameters:
Trial Environment:
According to the official document description, input the following statement in the Flink SQL CLI:
create table TEST (A string)
WITH ('connector'='oracle-cdc',
'hostname'='10.230.179.125',
'port'='1521',
'username'='myname',
'password'='***',
'database-name'='MY_SERVICE_NAME',
'schema-name'='MY_SCHEMA',
'table-name'='TEST' );
After that, we tried to observe through the select * from TEST and found we could not connect to Oracle normally. The following error was reported:
[ERROR] Could not execute SQL statement. Reason:
oracle.net.ns.NetException: Listener refused the connection with the following error:
ORA-12505, TNS:listener does not currently know of SID given in connect descriptor
Judging from the error message, Flink CDC may mistakenly identify the MY_SERVICE_NAME (Oracle service name) provided in the connection information as SID. Therefore, we tried to read the source code of Flink CDC involving Oracle Connector and found that in the com.ververica.cdc.connectors.oracle.OracleValidator, the code for Oracle connection is:
public static Connection openConnection(Properties properties) throws SQLException {
DriverManager.registerDriver(new oracle.jdbc.OracleDriver());
String hostname = properties.getProperty("database.hostname");
String port = properties.getProperty("database.port");
String dbname = properties.getProperty("database.dbname");
String userName = properties.getProperty("database.user");
String userpwd = properties.getProperty("database.password");
return DriverManager.getConnection(
"jdbc:oracle:thin:@" + hostname + ":" + port + ":" + dbname, userName, userpwd);
}
In the current version of Flink CDC, the connection mode of SID and Service Name is not distinguished, but the connection mode of SID is directly written in the code (port and dbname are separated by ":").
Starting from Oracle 8i, Oracle has introduced the concept of Service Name to support cluster (RAC) deployment of databases. A service name can be used as a logical concept of a database to unify the connection to different SID instances of the database. Therefore, the following two methods can be considered:
"jdbc:oracle:thin:@" + hostname + ":" + port + "/" + dbname, userName, userpwd)
The author adopts the second method, which realizes the normal connection to the database while retaining the Oracle Service Name features.
This issue has been submitted to the Flink CDC Issue 701.
Follow the preceding steps to observe the select * from TEST again and find that the data still cannot be obtained normally. The following error is reported:
[ERROR] Could not execute SQL statement. Reason:
io.debezium.DebeziumException: Supplemental logging not configured for table MY_SERVICE_NAME.MY_SCHEMA.test. Use command: ALTER TABLE MY_SCHEMA.test ADD SUPPLEMENTAL LOG DATA (ALL) COLUMNS
The table mentioned in the error log is MY_SERVICE_NAME.MY_SCHEMA.test. Why are the database names and schema names all uppercase, while the table names are lowercase?
Note: The error was reported by the io.debezium package. The following code is found in the io.debezium.relational.Tables by analyzing the source code of the package (according to the pom.xml file of Flink CDC, Debezium version 1.5.4 is currently used):
private TableId toLowerCaseIfNeeded(TableId tableId) {
return tableIdCaseInsensitive ? tableId.toLowercase() : tableId;
}
The developers of Debezium uniformly defined case insensitive as a need to convert table names to lowercase. This is true for PostgreSQL and MySQL supported by Debezium. For Oracle databases, case insensitive means the table name needs to be converted to uppercase when storing internal metadata.
Please refer to this document Nonquoted identifiers are not case sensitive. Oracle interprets them as uppercase.
Therefore, after Debezium reads the case insensitive* configuration, according to the preceding code logic, it will report an error because it attempts to read lowercase table names.
Since Debezium did not fix the problem until the latest stable version 1.7.1 and the latest development version 1.8.0, we can bypass the problem in the following two ways:
create table TEST (A string)
WITH ('connector'='oracle-cdc',
'hostname'='10.230.179.125',
'port'='1521',
'username'='myname',
'password'='***',
'database-name'='MY_SERVICE_NAME',
'schema-name'='MY_SCHEMA',
'table-name'='TEST',
'debezium.database.tablename.case.insensitive'='false' );
The disadvantage of this method is that it loses the Oracle case insensitive feature, and the uppercase table name must be specified in the 'table-name'.
Note: For database.tablename.case.insensitive parameters, Debezium is currently set to true by default only for Oracle 11g, and false by default for other Oracle versions. Therefore, if you are not using Oracle 11g, you do not need to modify this parameter but still need to specify the uppercase table name.
This issue has been submitted to the Flink CDC Issue 702.
The data latency is large, and sometimes it takes 3-5 minutes to capture data changes. A clear solution for this problem has been given in the Flink CDC FAQ. Add the following two configuration items to the create statement:
'debezium.log.mining.strategy'='online_catalog',
'debezium.log.mining.continuous.mine'= 'true'
Why do we do this? We can still deepen our understanding of the tool by analyzing source code and logs and combining how Oracle Logminer works.
The extraction of Logminer is mainly performed by the execute method in the Debezium io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource. This article does not list the actual source code and only extracts the key process drawn in the following flowchart to save space. Readers interested in this can compare the process map and analyze combined with the actual source code:
You can use the redo_log_catalog method to monitor the DDL information of a data table. Since archive logs are permanently stored on the disk, you can still obtain all DDL and DML operations before the downtime after the database is down. However, since it involves more information monitoring than the online catalog, the resulting frequent log switching and log dump operations are also staggering.
According to the author's test situation, if the debezium.log.mining.strategy is the default configuration redo_log_catalog, it is necessary to operate in step one. (The operation takes about half a minute to one minute.) In step four, according to the amount of archived logs data, the time consumption will also fluctuate from one minute to four minutes. In step five, query V$LOGMNR_CONTENTS views often take more than ten seconds to complete.
In addition, since archive logs grow rapidly in the actual system, it is often used to delete or dump expired logs regularly. Due to the time-consuming step four, the author observed that during the execution of step four, under a certain probability, the archive logs added in step two will be deleted and dumped. Therefore, during the query in step five, the following error will be reported because the log added in step two fails to be found:
ORA-00308: cannot open archive log '/path/to/archive/log /...'
ORA-27037: unable to obtain file status
In general, tables that Flink CDC needs to monitor, especially tables of great significance to business systems, do not perform DDL operations. You only need to capture DML operations. In special cases (such as database downtime), you can use the method of updating full data after the database is restored to ensure data consistency. Therefore, the online_catalog method is sufficient to meet our needs.
In addition, no matter using online_catalog or the default redo_log_catalog, there will be a problem that the logs found in step two and the logs needed in step five are out of sync. Therefore, adding the "debezium.log.mining.continuous.mine'='true" parameter and handing over the real-time log collection to Oracle to automatically complete can avoid this problem.
After the author configures according to these two parameters, the data delay can be reduced from a few minutes to about five seconds.
Step three and step seven of the preceding process map refer to determining the LogMiner monitoring timing range and determining the sleep time based on the configuration items. The process is further analyzed and a general methodology is given for further tuning of a single table.
You can understand how Debezium adjusts the monitoring timing range and sleep time by observing the getEndScn method in the io.debezium.connector.oracle.logminer.LogMinerHelper class. The method is described in the following process map:
Debezium gives two sets of parameters, log.mining.batch.size. and log.mining.sleep.time. to make the step size of each logMiner run as consistent as possible with the step size increased by the SCN of the database itself. It can be seen that:
if (currentBatchSize == batchSizeMax) {
LOGGER.info("LogMiner is now using the maximum batch size {}. This could be indicative of large SCN gaps", currentBatchSize);
}
If the current monitoring time range reaches log.mining.batch.size.max, Debezium will give the preceding message in the log. In actual application, check whether the log generated by Flink CDC contains the prompt to find out whether the log.mining.batch.size.max value is reasonable.
We have learned from the preceding two hidden parameters: debezium.database.tablename.case.insensitive (section 2) and debezium.log.mining.continuous.mine (section 3), which are not explained in Debezium's official documentation but can be used. All the hidden parameters of the Debezium Oracle Connector are given by analyzing the source code, and the following are their descriptions:
We think the log.mining.history.recorder.class parameters deserve focus in addition to the two parameters we used previously. The default value of this parameter is io.debezium.connector.oracle.logminer.NeverHistoryRecorder, which is an empty class. Therefore, when analyzing Flink CDC behavior, we can customize the class that implements the io.debezium.connector.oracle.logminer.HistoryRecorder interface and realize personalized monitoring of Flink CDC behavior without modifying the source code.
Flink CDC Series – Part 5: Implement Real-Time Writing of MySQL Data to Apache Doris
150 posts | 43 followers
FollowApache Flink Community China - May 18, 2022
ApsaraDB - February 29, 2024
Apache Flink Community China - May 18, 2022
Apache Flink Community China - June 2, 2022
Alibaba Cloud Indonesia - March 23, 2023
Alibaba Clouder - January 7, 2021
150 posts | 43 followers
FollowMigrate your legacy Oracle databases to Alibaba Cloud to save on long-term costs and take advantage of improved scalability, reliability, robust security, high performance, and cloud-native features.
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