diff --git a/.github/labeler.yml b/.github/labeler.yml index fa35ca556e6..f94de94c736 100644 --- a/.github/labeler.yml +++ b/.github/labeler.yml @@ -72,3 +72,9 @@ vitess-cdc-connector: - flink-cdc-connect/flink-cdc-source-connectors/flink-sql-connector-vitess-cdc/**/* values-pipeline-connector: - flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/**/* +mysql-pipeline-connector: + - flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/**/* +doris-pipeline-connector: + - flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/**/* +starrocks-pipeline-connector: + - flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/**/* diff --git a/docs/content/connectors/db2-cdc.md b/docs/content/connectors/db2-cdc.md index 28a44708b63..b73fece9c03 100644 --- a/docs/content/connectors/db2-cdc.md +++ b/docs/content/connectors/db2-cdc.md @@ -23,7 +23,7 @@ using a build automation tool (such as Maven or SBT) and SQL Client with SQL JAR com.ververica flink-connector-db2-cdc - 2.5-SNAPSHOT + 3.0-SNAPSHOT ``` @@ -31,7 +31,7 @@ using a build automation tool (such as Maven or SBT) and SQL Client with SQL JAR ```Download link is available only for stable releases.``` -Download flink-sql-connector-db2-cdc-2.5-SNAPSHOT.jar and +Download flink-sql-connector-db2-cdc-3.0-SNAPSHOT.jar and put it under `/lib/`. **Note:** flink-sql-connector-db2-cdc-XXX-SNAPSHOT version is the code corresponding to the development branch. Users diff --git a/docs/content/connectors/mongodb-cdc(ZH).md b/docs/content/connectors/mongodb-cdc(ZH).md index baea53a3ab2..8886b5edc87 100644 --- a/docs/content/connectors/mongodb-cdc(ZH).md +++ b/docs/content/connectors/mongodb-cdc(ZH).md @@ -13,7 +13,7 @@ MongoDB CDC 连接器允许从 MongoDB 读取快照数据和增量数据。 本 com.ververica flink-connector-mongodb-cdc - 2.5-SNAPSHOT + 3.0-SNAPSHOT ``` @@ -21,7 +21,7 @@ MongoDB CDC 连接器允许从 MongoDB 读取快照数据和增量数据。 本 ```下载链接仅适用于稳定版本。``` -下载 [flink-sql-connector-mongodb-cdc-2.5-SNAPSHOT.jar](https://repo1.maven.org/maven2/com/ververica/flink-sql-connector-mongodb-cdc/2.5-SNAPSHOT/flink-sql-connector-mongodb-cdc-2.5-SNAPSHOT.jar) 把它放在 `/lib/`. +下载 [flink-sql-connector-mongodb-cdc-3.0-SNAPSHOT.jar](https://repo1.maven.org/maven2/com/ververica/flink-sql-connector-mongodb-cdc/3.0-SNAPSHOT/flink-sql-connector-mongodb-cdc-3.0-SNAPSHOT.jar) 把它放在 `/lib/`. **注意:** flink-sql-connector-mongodb-cdc-XXX-SNAPSHOT 版本是与开发分支相对应的代码。 用户需要下载源代码并编译相应的jar。 用户应使用已发布的版本,例如 [flink-sql-connector-mongodb-cdc-2.3.0.jar](https://mvnrepository.com/artifact/com.ververica/flink-sql-connector-mongodb-cdc), 发布的版本将在 Maven 中央仓库中提供。 diff --git a/docs/content/connectors/mongodb-cdc.md b/docs/content/connectors/mongodb-cdc.md index abf10d238ce..0cdfa52c65a 100644 --- a/docs/content/connectors/mongodb-cdc.md +++ b/docs/content/connectors/mongodb-cdc.md @@ -13,7 +13,7 @@ In order to setup the MongoDB CDC connector, the following table provides depend com.ververica flink-connector-mongodb-cdc - 2.5-SNAPSHOT + 3.0-SNAPSHOT ``` @@ -21,7 +21,7 @@ In order to setup the MongoDB CDC connector, the following table provides depend ```Download link is available only for stable releases.``` -Download [flink-sql-connector-mongodb-cdc-2.5-SNAPSHOT.jar](https://repo1.maven.org/maven2/com/ververica/flink-sql-connector-mongodb-cdc/2.5-SNAPSHOT/flink-sql-connector-mongodb-cdc-2.5-SNAPSHOT.jar) and put it under `/lib/`. +Download [flink-sql-connector-mongodb-cdc-3.0-SNAPSHOT.jar](https://repo1.maven.org/maven2/com/ververica/flink-sql-connector-mongodb-cdc/3.0-SNAPSHOT/flink-sql-connector-mongodb-cdc-3.0-SNAPSHOT.jar) and put it under `/lib/`. **Note:** flink-sql-connector-mongodb-cdc-XXX-SNAPSHOT version is the code corresponding to the development branch. Users need to download the source code and compile the corresponding jar. Users should use the released version, such as [flink-sql-connector-mongodb-cdc-2.2.1.jar](https://mvnrepository.com/artifact/com.ververica/flink-sql-connector-mongodb-cdc), the released version will be available in the Maven central warehouse. diff --git a/docs/content/connectors/mysql-cdc(ZH).md b/docs/content/connectors/mysql-cdc(ZH).md index bd8e1036e9e..6fe8364aae6 100644 --- a/docs/content/connectors/mysql-cdc(ZH).md +++ b/docs/content/connectors/mysql-cdc(ZH).md @@ -20,7 +20,7 @@ MySQL CDC 连接器允许从 MySQL 数据库读取快照数据和增量数据。 com.ververica flink-connector-mysql-cdc - 2.5-SNAPSHOT + 3.0-SNAPSHOT ``` @@ -28,7 +28,7 @@ MySQL CDC 连接器允许从 MySQL 数据库读取快照数据和增量数据。 ```下载链接仅在已发布版本可用,请在文档网站左下角选择浏览已发布的版本。``` -下载 flink-sql-connector-mysql-cdc-2.5-SNAPSHOT.jar 到 `/lib/` 目录下。 +下载 flink-sql-connector-mysql-cdc-3.0-SNAPSHOT.jar 到 `/lib/` 目录下。 **注意:** flink-sql-connector-mysql-cdc-XXX-SNAPSHOT 版本是开发分支`release-XXX`对应的快照版本,快照版本用户需要下载源代码并编译相应的 jar。用户应使用已经发布的版本,例如 [flink-sql-connector-mysql-cdc-2.2.1.jar](https://mvnrepository.com/artifact/com.ververica/flink-sql-connector-mysql-cdc) 当前已发布的所有版本都可以在 Maven 中央仓库获取。 @@ -369,55 +369,68 @@ Flink SQL> SELECT * FROM orders; TIMESTAMP_LTZ(3) NOT NULL 当前记录表在数据库中更新的时间。
如果从表的快照而不是 binlog 读取记录,该值将始终为0。 + + row_kind + STRING NOT NULL + 当前记录对应的 changelog 类型。注意:当 Source 算子选择为每条记录输出 row_kind 字段后,下游 SQL 算子在处理消息撤回时会因为这个字段不同而比对失败, +建议只在简单的同步作业中引用该元数据列。
'+I' 表示 INSERT 数据,'-D' 表示 DELETE 数据,'-U' 表示 UPDATE_BEFORE 数据,'+U' 表示 UPDATE_AFTER 数据。 + + 下述创建表示例展示元数据列的用法: + ```sql -CREATE TABLE products ( - db_name STRING METADATA FROM 'database_name' VIRTUAL, - table_name STRING METADATA FROM 'table_name' VIRTUAL, - operation_ts TIMESTAMP_LTZ(3) METADATA FROM 'op_ts' VIRTUAL, - order_id INT, - order_date TIMESTAMP(0), +CREATE TABLE products +( + db_name STRING METADATA FROM 'database_name' VIRTUAL, + table_name STRING METADATA FROM 'table_name' VIRTUAL, + operation_ts TIMESTAMP_LTZ(3) METADATA FROM 'op_ts' VIRTUAL, + operation STRING METADATA FROM 'row_kind' VIRTUAL, + order_id INT, + order_date TIMESTAMP(0), customer_name STRING, - price DECIMAL(10, 5), - product_id INT, - order_status BOOLEAN, - PRIMARY KEY(order_id) NOT ENFORCED + price DECIMAL(10, 5), + product_id INT, + order_status BOOLEAN, + PRIMARY KEY (order_id) NOT ENFORCED ) WITH ( - 'connector' = 'mysql-cdc', - 'hostname' = 'localhost', - 'port' = '3306', - 'username' = 'root', - 'password' = '123456', - 'database-name' = 'mydb', - 'table-name' = 'orders' -); + 'connector' = 'mysql-cdc', + 'hostname' = 'localhost', + 'port' = '3306', + 'username' = 'root', + 'password' = '123456', + 'database-name' = 'mydb', + 'table-name' = 'orders' + ); ``` 下述创建表示例展示使用正则表达式匹配多张库表的用法: + ```sql -CREATE TABLE products ( - db_name STRING METADATA FROM 'database_name' VIRTUAL, - table_name STRING METADATA FROM 'table_name' VIRTUAL, - operation_ts TIMESTAMP_LTZ(3) METADATA FROM 'op_ts' VIRTUAL, - order_id INT, - order_date TIMESTAMP(0), +CREATE TABLE products +( + db_name STRING METADATA FROM 'database_name' VIRTUAL, + table_name STRING METADATA FROM 'table_name' VIRTUAL, + operation_ts TIMESTAMP_LTZ(3) METADATA FROM 'op_ts' VIRTUAL, + operation STRING METADATA FROM 'row_kind' VIRTUAL, + order_id INT, + order_date TIMESTAMP(0), customer_name STRING, - price DECIMAL(10, 5), - product_id INT, - order_status BOOLEAN, - PRIMARY KEY(order_id) NOT ENFORCED + price DECIMAL(10, 5), + product_id INT, + order_status BOOLEAN, + PRIMARY KEY (order_id) NOT ENFORCED ) WITH ( - 'connector' = 'mysql-cdc', - 'hostname' = 'localhost', - 'port' = '3306', - 'username' = 'root', - 'password' = '123456', - 'database-name' = '(^(test).*|^(tpc).*|txc|.*[p$]|t{2})', - 'table-name' = '(t[5-8]|tt)' -); + 'connector' = 'mysql-cdc', + 'hostname' = 'localhost', + 'port' = '3306', + 'username' = 'root', + 'password' = '123456', + 'database-name' = '(^(test).*|^(tpc).*|txc|.*[p$]|t{2})', + 'table-name' = '(t[5-8]|tt)' + ); ``` diff --git a/docs/content/connectors/mysql-cdc.md b/docs/content/connectors/mysql-cdc.md index cd2bd371c3b..91be5926d2c 100644 --- a/docs/content/connectors/mysql-cdc.md +++ b/docs/content/connectors/mysql-cdc.md @@ -21,7 +21,7 @@ In order to setup the MySQL CDC connector, the following table provides dependen com.ververicaflink-connector-mysql-cdc - 2.5-SNAPSHOT + 3.0-SNAPSHOT ``` @@ -29,7 +29,7 @@ In order to setup the MySQL CDC connector, the following table provides dependen ```Download link is available only for stable releases.``` -Download flink-sql-connector-mysql-cdc-2.5-SNAPSHOT.jar and put it under `/lib/`. +Download flink-sql-connector-mysql-cdc-3.0-SNAPSHOT.jar and put it under `/lib/`. **Note:** flink-sql-connector-mysql-cdc-XXX-SNAPSHOT version is the code corresponding to the development branch. Users need to download the source code and compile the corresponding jar. Users should use the released version, such as [flink-sql-connector-mysql-cdc-2.3.0.jar](https://mvnrepository.com/artifact/com.ververica/flink-connector-mysql-cdc), the released version will be available in the Maven central warehouse. @@ -377,55 +377,68 @@ The following format metadata can be exposed as read-only (VIRTUAL) columns in a + + + + +
TIMESTAMP_LTZ(3) NOT NULL It indicates the time that the change was made in the database.
If the record is read from snapshot of the table instead of the binlog, the value is always 0.
row_kindSTRING NOT NULLIt indicates the row kind of the changelog,Note: The downstream SQL operator may fail to compare due to this new added column when processing the row retraction if +the source operator chooses to output the 'row_kind' column for each record. It is recommended to use this metadata column only in simple synchronization jobs. +
'+I' means INSERT message, '-D' means DELETE message, '-U' means UPDATE_BEFORE message and '+U' means UPDATE_AFTER message.
The extended CREATE TABLE example demonstrates the syntax for exposing these metadata fields: + ```sql -CREATE TABLE products ( - db_name STRING METADATA FROM 'database_name' VIRTUAL, - table_name STRING METADATA FROM 'table_name' VIRTUAL, - operation_ts TIMESTAMP_LTZ(3) METADATA FROM 'op_ts' VIRTUAL, - order_id INT, - order_date TIMESTAMP(0), +CREATE TABLE products +( + db_name STRING METADATA FROM 'database_name' VIRTUAL, + table_name STRING METADATA FROM 'table_name' VIRTUAL, + operation_ts TIMESTAMP_LTZ(3) METADATA FROM 'op_ts' VIRTUAL, + operation STRING METADATA FROM 'row_kind' VIRTUAL, + order_id INT, + order_date TIMESTAMP(0), customer_name STRING, - price DECIMAL(10, 5), - product_id INT, - order_status BOOLEAN, - PRIMARY KEY(order_id) NOT ENFORCED + price DECIMAL(10, 5), + product_id INT, + order_status BOOLEAN, + PRIMARY KEY (order_id) NOT ENFORCED ) WITH ( - 'connector' = 'mysql-cdc', - 'hostname' = 'localhost', - 'port' = '3306', - 'username' = 'root', - 'password' = '123456', - 'database-name' = 'mydb', - 'table-name' = 'orders' -); + 'connector' = 'mysql-cdc', + 'hostname' = 'localhost', + 'port' = '3306', + 'username' = 'root', + 'password' = '123456', + 'database-name' = 'mydb', + 'table-name' = 'orders' + ); ``` The extended CREATE TABLE example demonstrates the usage of regex to match multi-tables: + ```sql -CREATE TABLE products ( - db_name STRING METADATA FROM 'database_name' VIRTUAL, - table_name STRING METADATA FROM 'table_name' VIRTUAL, - operation_ts TIMESTAMP_LTZ(3) METADATA FROM 'op_ts' VIRTUAL, - order_id INT, - order_date TIMESTAMP(0), +CREATE TABLE products +( + db_name STRING METADATA FROM 'database_name' VIRTUAL, + table_name STRING METADATA FROM 'table_name' VIRTUAL, + operation_ts TIMESTAMP_LTZ(3) METADATA FROM 'op_ts' VIRTUAL, + operation STRING METADATA FROM 'row_kind' VIRTUAL, + order_id INT, + order_date TIMESTAMP(0), customer_name STRING, - price DECIMAL(10, 5), - product_id INT, - order_status BOOLEAN, - PRIMARY KEY(order_id) NOT ENFORCED + price DECIMAL(10, 5), + product_id INT, + order_status BOOLEAN, + PRIMARY KEY (order_id) NOT ENFORCED ) WITH ( - 'connector' = 'mysql-cdc', - 'hostname' = 'localhost', - 'port' = '3306', - 'username' = 'root', - 'password' = '123456', - 'database-name' = '(^(test).*|^(tpc).*|txc|.*[p$]|t{2})', - 'table-name' = '(t[5-8]|tt)' -); + 'connector' = 'mysql-cdc', + 'hostname' = 'localhost', + 'port' = '3306', + 'username' = 'root', + 'password' = '123456', + 'database-name' = '(^(test).*|^(tpc).*|txc|.*[p$]|t{2})', + 'table-name' = '(t[5-8]|tt)' + ); ``` diff --git a/docs/content/connectors/oceanbase-cdc(ZH).md b/docs/content/connectors/oceanbase-cdc(ZH).md index 9940d5f0a2a..27f1346bf69 100644 --- a/docs/content/connectors/oceanbase-cdc(ZH).md +++ b/docs/content/connectors/oceanbase-cdc(ZH).md @@ -11,7 +11,7 @@ OceanBase CDC 连接器允许从 OceanBase 读取快照数据和增量数据。 com.ververicaflink-connector-oceanbase-cdc - 2.5-SNAPSHOT + 3.0-SNAPSHOT ``` @@ -29,7 +29,7 @@ OceanBase CDC 连接器允许从 OceanBase 读取快照数据和增量数据。 ```下载链接仅在已发布版本可用,请在文档网站左下角选择浏览已发布的版本。``` -下载[flink-sql-connector-oceanbase-cdc-2.5-SNAPSHOT.jar](https://repo1.maven.org/maven2/com/ververica/flink-sql-connector-oceanbase-cdc/2.5-SNAPSHOT/flink-sql-connector-oceanbase-cdc-2.5-SNAPSHOT.jar) 到 `/lib/` 目录下。 +下载[flink-sql-connector-oceanbase-cdc-3.0-SNAPSHOT.jar](https://repo1.maven.org/maven2/com/ververica/flink-sql-connector-oceanbase-cdc/2.5-SNAPSHOT/flink-sql-connector-oceanbase-cdc-2.5-SNAPSHOT.jar) 到 `/lib/` 目录下。 **注意:** flink-sql-connector-oceanbase-cdc-XXX-SNAPSHOT 版本是开发分支`release-XXX`对应的快照版本,快照版本用户需要下载源代码并编译相应的 jar。用户应使用已经发布的版本,例如 [flink-sql-connector-oceanbase-cdc-2.3.0.jar](https://mvnrepository.com/artifact/com.ververica/flink-sql-connector-oceanbase-cdc) 当前已发布的所有版本都可以在 Maven 中央仓库获取。 diff --git a/docs/content/connectors/oceanbase-cdc.md b/docs/content/connectors/oceanbase-cdc.md index e421df6fd31..0df3e9d3355 100644 --- a/docs/content/connectors/oceanbase-cdc.md +++ b/docs/content/connectors/oceanbase-cdc.md @@ -12,7 +12,7 @@ In order to set up the OceanBase CDC connector, the following table provides dep com.ververica flink-connector-oceanbase-cdc - 2.5-SNAPSHOT + 3.0-SNAPSHOT ``` @@ -30,7 +30,7 @@ If you want to use OceanBase JDBC driver to connect to the enterprise edition da ```Download link is available only for stable releases.``` -Download [flink-sql-connector-oceanbase-cdc-2.5-SNAPSHOT.jar](https://repo1.maven.org/maven2/com/ververica/flink-sql-connector-oceanbase-cdc/2.5-SNAPSHOT/flink-sql-connector-oceanbase-cdc-2.5-SNAPSHOT.jar) and put it under `/lib/`. +Download [flink-sql-connector-oceanbase-cdc-3.0-SNAPSHOT.jar](https://repo1.maven.org/maven2/com/ververica/flink-sql-connector-oceanbase-cdc/3.0-SNAPSHOT/flink-sql-connector-oceanbase-cdc-3.0-SNAPSHOT.jar) and put it under `/lib/`. **Note:** flink-sql-connector-oceanbase-cdc-XXX-SNAPSHOT version is the code corresponding to the development branch. Users need to download the source code and compile the corresponding jar. Users should use the released version, such as [flink-sql-connector-oceanbase-cdc-2.2.1.jar](https://mvnrepository.com/artifact/com.ververica/flink-sql-connector-oceanbase-cdc), the released version will be available in the Maven central warehouse. diff --git a/docs/content/connectors/oracle-cdc.md b/docs/content/connectors/oracle-cdc.md index 47080537230..9842a06bdd9 100644 --- a/docs/content/connectors/oracle-cdc.md +++ b/docs/content/connectors/oracle-cdc.md @@ -14,7 +14,7 @@ In order to setup the Oracle CDC connector, the following table provides depende com.ververica flink-connector-oracle-cdc - 2.5-SNAPSHOT + 3.0-SNAPSHOT ``` @@ -22,7 +22,7 @@ In order to setup the Oracle CDC connector, the following table provides depende **Download link is available only for stable releases.** -Download [flink-sql-connector-oracle-cdc-2.5-SNAPSHOT.jar](https://repo1.maven.org/maven2/com/ververica/flink-sql-connector-oracle-cdc/2.5-SNAPSHOT/flink-sql-connector-oracle-cdc-2.5-SNAPSHOT.jar) and put it under `/lib/`. +Download [flink-sql-connector-oracle-cdc-3.0-SNAPSHOT.jar](https://repo1.maven.org/maven2/com/ververica/flink-sql-connector-oracle-cdc/3.0-SNAPSHOT/flink-sql-connector-oracle-cdc-3.0-SNAPSHOT.jar) and put it under `/lib/`. **Note:** flink-sql-connector-oracle-cdc-XXX-SNAPSHOT version is the code corresponding to the development branch. Users need to download the source code and compile the corresponding jar. Users should use the released version, such as [flink-sql-connector-oracle-cdc-2.3.0.jar](https://mvnrepository.com/artifact/com.ververica/flink-sql-connector-oracle-cdc), the released version will be available in the Maven central warehouse. diff --git a/docs/content/connectors/postgres-cdc.md b/docs/content/connectors/postgres-cdc.md index 364daf8f4d0..91c6847928f 100644 --- a/docs/content/connectors/postgres-cdc.md +++ b/docs/content/connectors/postgres-cdc.md @@ -14,7 +14,7 @@ In order to setup the Postgres CDC connector, the following table provides depen com.ververica flink-connector-postgres-cdc - 2.5-SNAPSHOT + 3.0-SNAPSHOT ``` @@ -22,7 +22,7 @@ In order to setup the Postgres CDC connector, the following table provides depen ```Download link is available only for stable releases.``` -Download flink-sql-connector-postgres-cdc-2.5-SNAPSHOT.jar and put it under `/lib/`. +Download flink-sql-connector-postgres-cdc-3.0-SNAPSHOT.jar and put it under `/lib/`. **Note:** flink-sql-connector-postgres-cdc-XXX-SNAPSHOT version is the code corresponding to the development branch. Users need to download the source code and compile the corresponding jar. Users should use the released version, such as [flink-sql-connector-postgres-cdc-2.3.0.jar](https://mvnrepository.com/artifact/com.ververica/flink-sql-connector-postgres-cdc), the released version will be available in the Maven central warehouse. diff --git a/docs/content/connectors/sqlserver-cdc.md b/docs/content/connectors/sqlserver-cdc.md index 04a7be967da..4f6c2d81872 100644 --- a/docs/content/connectors/sqlserver-cdc.md +++ b/docs/content/connectors/sqlserver-cdc.md @@ -14,7 +14,7 @@ In order to setup the SQLServer CDC connector, the following table provides depe com.ververica flink-connector-sqlserver-cdc - 2.5-SNAPSHOT + 3.0-SNAPSHOT ``` @@ -22,7 +22,7 @@ In order to setup the SQLServer CDC connector, the following table provides depe ```Download link is available only for stable releases.``` -Download [flink-sql-connector-sqlserver-cdc-2.5-SNAPSHOT.jar](https://repo1.maven.org/maven2/com/ververica/flink-sql-connector-sqlserver-cdc/2.5-SNAPSHOT/flink-sql-connector-sqlserver-cdc-2.5-SNAPSHOT.jar) and put it under `/lib/`. +Download [flink-sql-connector-sqlserver-cdc-3.0-SNAPSHOT.jar](https://repo1.maven.org/maven2/com/ververica/flink-sql-connector-sqlserver-cdc/3.0-SNAPSHOT/flink-sql-connector-sqlserver-cdc-3.0-SNAPSHOT.jar) and put it under `/lib/`. **Note:** flink-sql-connector-sqlserver-cdc-XXX-SNAPSHOT version is the code corresponding to the development branch. Users need to download the source code and compile the corresponding jar. Users should use the released version, such as [flink-sql-connector-sqlserver-cdc-2.2.1.jar](https://mvnrepository.com/artifact/com.ververica/flink-sql-connector-sqlserver-cdc), the released version will be available in the Maven central warehouse. diff --git a/docs/content/connectors/tidb-cdc.md b/docs/content/connectors/tidb-cdc.md index 67e900171d5..2ab6ebdc201 100644 --- a/docs/content/connectors/tidb-cdc.md +++ b/docs/content/connectors/tidb-cdc.md @@ -14,7 +14,7 @@ In order to setup the TiDB CDC connector, the following table provides dependenc com.ververica flink-connector-tidb-cdc - 2.5-SNAPSHOT + 3.0-SNAPSHOT ``` @@ -22,7 +22,7 @@ In order to setup the TiDB CDC connector, the following table provides dependenc ```Download link is available only for stable releases.``` -Download [flink-sql-connector-tidb-cdc-2.5-SNAPSHOT.jar](https://repo1.maven.org/maven2/com/ververica/flink-sql-connector-tidb-cdc/2.5-SNAPSHOT/flink-sql-connector-tidb-cdc-2.5-SNAPSHOT.jar) and put it under `/lib/`. +Download [flink-sql-connector-tidb-cdc-3.0-SNAPSHOT.jar](https://repo1.maven.org/maven2/com/ververica/flink-sql-connector-tidb-cdc/3.0-SNAPSHOT/flink-sql-connector-tidb-cdc-3.0-SNAPSHOT.jar) and put it under `/lib/`. **Note:** flink-sql-connector-tidb-cdc-XXX-SNAPSHOT version is the code corresponding to the development branch. Users need to download the source code and compile the corresponding jar. Users should use the released version, such as [flink-sql-connector-tidb-cdc-2.2.1.jar](https://mvnrepository.com/artifact/com.ververica/flink-sql-connector-tidb-cdc), the released version will be available in the Maven central warehouse. diff --git a/docs/content/connectors/vitess-cdc.md b/docs/content/connectors/vitess-cdc.md index 67a40cdfc32..94c2875abdf 100644 --- a/docs/content/connectors/vitess-cdc.md +++ b/docs/content/connectors/vitess-cdc.md @@ -14,13 +14,13 @@ In order to setup the Vitess CDC connector, the following table provides depende com.ververica flink-connector-vitess-cdc - 2.5-SNAPSHOT + 3.0-SNAPSHOT ``` ### SQL Client JAR -Download [flink-sql-connector-vitess-cdc-2.5-SNAPSHOT.jar](https://repo1.maven.org/maven2/com/ververica/flink-sql-connector-vitess-cdc/2.5-SNAPSHOT/flink-sql-connector-vitess-cdc-2.5-SNAPSHOT.jar) and put it under `/lib/`. +Download [flink-sql-connector-vitess-cdc-3.0-SNAPSHOT.jar](https://repo1.maven.org/maven2/com/ververica/flink-sql-connector-vitess-cdc/3.0-SNAPSHOT/flink-sql-connector-vitess-cdc-3.0-SNAPSHOT.jar) and put it under `/lib/`. Setup Vitess server ---------------- diff --git a/docs/content/overview/cdc-connectors.md b/docs/content/overview/cdc-connectors.md index f68b54e3424..106adaa6b13 100644 --- a/docs/content/overview/cdc-connectors.md +++ b/docs/content/overview/cdc-connectors.md @@ -99,7 +99,7 @@ Include following Maven dependency (available through Maven Central): flink-connector-mysql-cdc - 2.5-SNAPSHOT + 3.0-SNAPSHOT ``` diff --git a/docs/content/overview/cdc-pipeline.md b/docs/content/overview/cdc-pipeline.md index d974554764d..60b99d2e2e0 100644 --- a/docs/content/overview/cdc-pipeline.md +++ b/docs/content/overview/cdc-pipeline.md @@ -11,6 +11,22 @@ CDC Streaming ELT Framework is a stream data integration framework that aims to * ✅ Synchronization of entire databases * ✅ Schema evolution capability + +## Supported Connectors + +| Connector | Database | +|----------------------------------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| [doris-pipeline](../pipelines/doris-pipeline.md) |
  • [Doris](https://doris.apache.org/): 1.2.x, 2.x.x | +| [mysql-pipeline](../pipelines/mysql-pipeline.md) |
  • [MySQL](https://dev.mysql.com/doc): 5.6, 5.7, 8.0.x
  • [RDS MySQL](https://www.aliyun.com/product/rds/mysql): 5.6, 5.7, 8.0.x
  • [PolarDB MySQL](https://www.aliyun.com/product/polardb): 5.6, 5.7, 8.0.x
  • [Aurora MySQL](https://aws.amazon.com/cn/rds/aurora): 5.6, 5.7, 8.0.x
  • [MariaDB](https://mariadb.org): 10.x
  • [PolarDB X](https://github.com/ApsaraDB/galaxysql): 2.0.1 | +| [starrocks-pipeline](../pipelines/starrocks-pipeline.md) |
  • [StarRocks](https://www.starrocks.io/): 2.x, 3.x | + +## Supported Flink Versions +The following table shows the version mapping between Flink® CDC Pipeline and Flink®: + +| Flink® CDC Version | Flink® Version | +|:-----------------------------------:|:-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------:| +| 3.0.* | 1.14.\*, 1.15.\*, 1.16.\*, 1.17.\*, 1.18.\* | + ## Core Concepts ![CDC Design](/_static/fig/design.png "CDC Design") diff --git a/docs/content/pipelines/doris-pipeline(ZH).md b/docs/content/pipelines/doris-pipeline(ZH).md index e84d195b6b2..47c793eb4bd 100644 --- a/docs/content/pipelines/doris-pipeline(ZH).md +++ b/docs/content/pipelines/doris-pipeline(ZH).md @@ -1,4 +1,4 @@ -# Doris Pipeline +# Doris Pipeline 连接器 本文介绍了Pipeline Doris Connector的使用 diff --git a/docs/content/pipelines/doris-pipeline.md b/docs/content/pipelines/doris-pipeline.md index 0714fca0a6b..badb4429568 100644 --- a/docs/content/pipelines/doris-pipeline.md +++ b/docs/content/pipelines/doris-pipeline.md @@ -1,4 +1,4 @@ -# Doris Pipeline +# Doris Pipeline Connector This article introduces of Doris Pipeline Connector diff --git a/docs/content/quickstart/build-real-time-data-lake-tutorial.md b/docs/content/quickstart/build-real-time-data-lake-tutorial.md index 0a95e224365..b7f3b6278ba 100644 --- a/docs/content/quickstart/build-real-time-data-lake-tutorial.md +++ b/docs/content/quickstart/build-real-time-data-lake-tutorial.md @@ -17,7 +17,7 @@ Prepare a Linux or MacOS computer with Docker installed. ## Preparing JAR package required **Download links are available only for stable releases, SNAPSHOT dependencies need to be built based on master or release- branches by yourself.** -- flink-sql-connector-mysql-cdc-2.5-SNAPSHOT.jar +- flink-sql-connector-mysql-cdc-3.0-SNAPSHOT.jar - [flink-shaded-hadoop-2-uber-2.7.5-10.0.jar](https://repo.maven.apache.org/maven2/org/apache/flink/flink-shaded-hadoop-2-uber/2.7.5-10.0/flink-shaded-hadoop-2-uber-2.7.5-10.0.jar) - [iceberg-flink-runtime-1.16-1.3.1.jar](https://repo.maven.apache.org/maven2/org/apache/iceberg/iceberg-flink-runtime-1.16/1.3.1/iceberg-flink-runtime-1.16-1.3.1.jar) diff --git a/docs/content/quickstart/datastream-api-package-guidance.md b/docs/content/quickstart/datastream-api-package-guidance.md index dff6cdbc08a..cd536dc06b7 100644 --- a/docs/content/quickstart/datastream-api-package-guidance.md +++ b/docs/content/quickstart/datastream-api-package-guidance.md @@ -36,7 +36,7 @@ flink 1.17.2 flink mysql cdc 2.4.2 1.17.2 1.7.15 2.17.1 - 1.9.7.Final + 1.9.8.Final diff --git a/docs/content/quickstart/db2-tutorial.md b/docs/content/quickstart/db2-tutorial.md index 3eed958653a..e1e2cf851cb 100644 --- a/docs/content/quickstart/db2-tutorial.md +++ b/docs/content/quickstart/db2-tutorial.md @@ -62,7 +62,7 @@ docker-compose down **Download links are available only for stable releases, SNAPSHOT dependencies need to be built based on master or release- branches by yourself.** - [flink-sql-connector-elasticsearch7-3.0.1-1.17.jar](https://repo.maven.apache.org/maven2/org/apache/flink/flink-sql-connector-elasticsearch7/3.0.1-1.17/flink-sql-connector-elasticsearch7-3.0.1-1.17.jar) -- flink-sql-connector-db2-cdc-2.5-SNAPSHOT.jar +- flink-sql-connector-db2-cdc-3.0-SNAPSHOT.jar **3. Launch a Flink cluster and start a Flink SQL CLI** diff --git a/docs/content/quickstart/mongodb-tutorial.md b/docs/content/quickstart/mongodb-tutorial.md index def5ce30589..66777b00b17 100644 --- a/docs/content/quickstart/mongodb-tutorial.md +++ b/docs/content/quickstart/mongodb-tutorial.md @@ -110,7 +110,7 @@ db.customers.insertMany([ ```Download links are available only for stable releases, SNAPSHOT dependencies need to be built based on master or release- branches by yourself. ``` - [flink-sql-connector-elasticsearch7-3.0.1-1.17.jar](https://repo.maven.apache.org/maven2/org/apache/flink/flink-sql-connector-elasticsearch7/3.0.1-1.17/flink-sql-connector-elasticsearch7-3.0.1-1.17.jar) -- [flink-sql-connector-mongodb-cdc-2.5-SNAPSHOT.jar](https://repo1.maven.org/maven2/com/ververica/flink-sql-connector-mongodb-cdc/2.5-SNAPSHOT/flink-sql-connector-mongodb-cdc-2.5-SNAPSHOT.jar) +- [flink-sql-connector-mongodb-cdc-3.0-SNAPSHOT.jar](https://repo1.maven.org/maven2/com/ververica/flink-sql-connector-mongodb-cdc/3.0-SNAPSHOT/flink-sql-connector-mongodb-cdc-3.0-SNAPSHOT.jar) 4. Launch a Flink cluster, then start a Flink SQL CLI and execute following SQL statements inside: diff --git a/docs/content/quickstart/mysql-doris-pipeline-tutorial.md b/docs/content/quickstart/mysql-doris-pipeline-tutorial.md index 0a217731559..f001661e00e 100644 --- a/docs/content/quickstart/mysql-doris-pipeline-tutorial.md +++ b/docs/content/quickstart/mysql-doris-pipeline-tutorial.md @@ -298,7 +298,7 @@ Specifically, `source-table` support regular expression matching with multiple t ```yaml route: - - source-table: app_db.order.* + - source-table: app_db.order\.* sink-table: ods_db.ods_orders ``` diff --git a/docs/content/quickstart/mysql-starrocks-pipeline-tutorial.md b/docs/content/quickstart/mysql-starrocks-pipeline-tutorial.md index baade23152d..f4b515d2f7e 100644 --- a/docs/content/quickstart/mysql-starrocks-pipeline-tutorial.md +++ b/docs/content/quickstart/mysql-starrocks-pipeline-tutorial.md @@ -265,7 +265,7 @@ Specifically, `source-table` support regular expression matching with multiple t ```yaml route: - - source-table: app_db.order.* + - source-table: app_db.order\.* sink-table: ods_db.ods_orders ``` diff --git a/docs/content/quickstart/oceanbase-tutorial.md b/docs/content/quickstart/oceanbase-tutorial.md index 11f957fe036..74b2eef81d3 100644 --- a/docs/content/quickstart/oceanbase-tutorial.md +++ b/docs/content/quickstart/oceanbase-tutorial.md @@ -11,15 +11,20 @@ Create `docker-compose.yml`. +*Note*: `host` network mode is required in this demo, so it can only work on Linux, see [network-tutorial-host](https://docs.docker.com/network/network-tutorial-host/). + ```yaml version: '2.1' services: observer: - image: oceanbase/oceanbase-ce:4.0.0.0 + image: oceanbase/oceanbase-ce:4.2.0.0 container_name: observer + environment: + - 'MODE=slim' + - 'OB_ROOT_PASSWORD=pswd' network_mode: "host" oblogproxy: - image: whhe/oblogproxy:1.1.0_4x + image: whhe/oblogproxy:1.1.3_4x container_name: oblogproxy environment: - 'OB_SYS_USERNAME=root' @@ -60,20 +65,6 @@ docker-compose up -d ### Set password -There is no password for 'root' user by default, but we need a user of 'sys' tenant with non-empty password in oblogproxy. So here we should set a password for 'root@sys' firstly. - -Login 'root' user of 'sys' tenant. - -```shell -docker-compose exec observer obclient -h127.0.0.1 -P2881 -uroot@sys -``` - -Set a password, note that the password needs to be consistent with the environment variable 'OB_SYS_PASSWORD' of oblogproxy service. - -```mysql -ALTER USER root IDENTIFIED BY 'pswd'; -``` - From OceanBase 4.0.0.0 CE, we can only fetch the commit log of non-sys tenant. Here we use the 'test' tenant for example. diff --git "a/docs/content/\345\277\253\351\200\237\344\270\212\346\211\213/datastream-api-package-guidance-zh.md" "b/docs/content/\345\277\253\351\200\237\344\270\212\346\211\213/datastream-api-package-guidance-zh.md" index 5afc879fb32..9e134e752ef 100644 --- "a/docs/content/\345\277\253\351\200\237\344\270\212\346\211\213/datastream-api-package-guidance-zh.md" +++ "b/docs/content/\345\277\253\351\200\237\344\270\212\346\211\213/datastream-api-package-guidance-zh.md" @@ -36,7 +36,7 @@ flink 1.17.2 flink mysql cdc 2.4.2 1.17.2 1.7.15 2.17.1 - 1.9.7.Final + 1.9.8.Final @@ -244,4 +244,4 @@ public class CdcTest { } } -``` \ No newline at end of file +``` diff --git "a/docs/content/\345\277\253\351\200\237\344\270\212\346\211\213/mysql-doris-pipeline-tutorial-zh.md" "b/docs/content/\345\277\253\351\200\237\344\270\212\346\211\213/mysql-doris-pipeline-tutorial-zh.md" index 41bb52c2745..923f9fe3ac0 100644 --- "a/docs/content/\345\277\253\351\200\237\344\270\212\346\211\213/mysql-doris-pipeline-tutorial-zh.md" +++ "b/docs/content/\345\277\253\351\200\237\344\270\212\346\211\213/mysql-doris-pipeline-tutorial-zh.md" @@ -300,7 +300,7 @@ Flink CDC 提供了将源表的表结构/数据路由到其他表名的配置, ```yaml route: - - source-table: app_db.order.* + - source-table: app_db.order\.* sink-table: ods_db.ods_orders ``` diff --git "a/docs/content/\345\277\253\351\200\237\344\270\212\346\211\213/mysql-starrocks-pipeline-tutorial-zh.md" "b/docs/content/\345\277\253\351\200\237\344\270\212\346\211\213/mysql-starrocks-pipeline-tutorial-zh.md" index e6d3d9dda72..28d509619a6 100644 --- "a/docs/content/\345\277\253\351\200\237\344\270\212\346\211\213/mysql-starrocks-pipeline-tutorial-zh.md" +++ "b/docs/content/\345\277\253\351\200\237\344\270\212\346\211\213/mysql-starrocks-pipeline-tutorial-zh.md" @@ -265,7 +265,7 @@ Flink CDC 提供了将源表的表结构/数据路由到其他表名的配置, ```yaml route: - - source-table: app_db.order.* + - source-table: app_db.order\.* sink-table: ods_db.ods_orders ``` diff --git "a/docs/content/\345\277\253\351\200\237\344\270\212\346\211\213/oceanbase-tutorial-zh.md" "b/docs/content/\345\277\253\351\200\237\344\270\212\346\211\213/oceanbase-tutorial-zh.md" index 8a63ea4436f..9269ec20119 100644 --- "a/docs/content/\345\277\253\351\200\237\344\270\212\346\211\213/oceanbase-tutorial-zh.md" +++ "b/docs/content/\345\277\253\351\200\237\344\270\212\346\211\213/oceanbase-tutorial-zh.md" @@ -12,15 +12,20 @@ 配置 `docker-compose.yml`。 +*注意*:本示例需要使用`host`网络,所以只能在 Linux 系统上运行,更多信息见 [network-tutorial-host](https://docs.docker.com/network/network-tutorial-host/)。 + ```yaml version: '2.1' services: observer: - image: oceanbase/oceanbase-ce:4.0.0.0 + image: oceanbase/oceanbase-ce:4.2.0.0 container_name: observer + environment: + - 'MODE=slim' + - 'OB_ROOT_PASSWORD=pswd' network_mode: "host" oblogproxy: - image: whhe/oblogproxy:1.1.0_4x + image: whhe/oblogproxy:1.1.3_4x container_name: oblogproxy environment: - 'OB_SYS_USERNAME=root' @@ -61,20 +66,6 @@ docker-compose up -d ### 设置密码 -OceanBase 中 root 用户默认是没有密码的,但是 oblogproxy 需要配置一个使用非空密码的系统租户用户,因此这里我们需要先为 root@sys 用户设置一个密码。 - -登陆 sys 租户的 root 用户: - -```shell -docker-compose exec observer obclient -h127.0.0.1 -P2881 -uroot@sys -``` - -设置密码,注意这里的密码需要与上一步中 oblogproxy 服务的环境变量 'OB_SYS_PASSWORD' 保持一样。 - -```mysql -ALTER USER root IDENTIFIED BY 'pswd'; -``` - OceanBase 从社区版 4.0.0.0 开始只支持对非 sys 租户的增量数据拉取,这里我们使用 test 租户的 root 用户作为示例。 登陆 test 租户的 root 用户: diff --git a/docs/site/index.html b/docs/site/index.html index 81633ca1fd8..6fb3b3c3178 100644 --- a/docs/site/index.html +++ b/docs/site/index.html @@ -65,7 +65,7 @@

    Embrace Database Changes

    - +

    Exactly-once Semantics

    @@ -74,7 +74,7 @@

    Exactly-once Semantics

    CDC Connectors for Apache Flink® supports reading database snapshots and continues to read binlogs with exactly-once processing, even after failures.
    - +

    Table/SQL API

    @@ -83,7 +83,7 @@

    Table/SQL API

    Users can use SQL DDL to create a CDC source to monitor changes on a single table.
    - +

    DataStream API

    diff --git a/flink-cdc-common/src/main/java/com/ververica/cdc/common/types/utils/DataTypeUtils.java b/flink-cdc-common/src/main/java/com/ververica/cdc/common/types/utils/DataTypeUtils.java index 8388e64d71b..fc4c5f53684 100644 --- a/flink-cdc-common/src/main/java/com/ververica/cdc/common/types/utils/DataTypeUtils.java +++ b/flink-cdc-common/src/main/java/com/ververica/cdc/common/types/utils/DataTypeUtils.java @@ -112,7 +112,7 @@ public static org.apache.flink.table.types.DataType toFlinkDataType(DataType typ case DATE: return org.apache.flink.table.api.DataTypes.DATE(); case TIME_WITHOUT_TIME_ZONE: - return org.apache.flink.table.api.DataTypes.TIME(length); + return org.apache.flink.table.api.DataTypes.TIME(precision); case BIGINT: return org.apache.flink.table.api.DataTypes.BIGINT(); case FLOAT: @@ -120,13 +120,13 @@ public static org.apache.flink.table.types.DataType toFlinkDataType(DataType typ case DOUBLE: return org.apache.flink.table.api.DataTypes.DOUBLE(); case TIMESTAMP_WITHOUT_TIME_ZONE: - return org.apache.flink.table.api.DataTypes.TIMESTAMP_WITH_TIME_ZONE(length); - case TIMESTAMP_WITH_LOCAL_TIME_ZONE: - return org.apache.flink.table.api.DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(length); case TIMESTAMP_WITH_TIME_ZONE: - return org.apache.flink.table.api.DataTypes.TIMESTAMP_WITH_TIME_ZONE(length); + return org.apache.flink.table.api.DataTypes.TIMESTAMP_WITH_TIME_ZONE(precision); + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + return org.apache.flink.table.api.DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE( + precision); case ARRAY: - Preconditions.checkState(children != null && children.size() > 0); + Preconditions.checkState(children != null && !children.isEmpty()); return org.apache.flink.table.api.DataTypes.ARRAY(toFlinkDataType(children.get(0))); case MAP: Preconditions.checkState(children != null && children.size() > 1); diff --git a/flink-cdc-common/src/main/java/com/ververica/cdc/common/utils/InstantiationUtil.java b/flink-cdc-common/src/main/java/com/ververica/cdc/common/utils/InstantiationUtil.java index fbed9409bd6..34a07467d00 100644 --- a/flink-cdc-common/src/main/java/com/ververica/cdc/common/utils/InstantiationUtil.java +++ b/flink-cdc-common/src/main/java/com/ververica/cdc/common/utils/InstantiationUtil.java @@ -19,7 +19,6 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.core.memory.DataInputViewStreamWrapper; import org.apache.flink.core.memory.DataOutputViewStreamWrapper; -import org.apache.flink.util.CollectionUtil; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -112,8 +111,7 @@ protected Class resolveProxyClass(String[] interfaces) // ------------------------------------------------ - private static final HashMap> primitiveClasses = - CollectionUtil.newHashMapWithExpectedSize(9); + private static final HashMap> primitiveClasses = new HashMap<>(9); static { primitiveClasses.put("boolean", boolean.class); diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/main/java/com/ververica/cdc/connectors/doris/sink/DorisEventSerializer.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/main/java/com/ververica/cdc/connectors/doris/sink/DorisEventSerializer.java index 1c47b082e5f..f26a66b6624 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/main/java/com/ververica/cdc/connectors/doris/sink/DorisEventSerializer.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/main/java/com/ververica/cdc/connectors/doris/sink/DorisEventSerializer.java @@ -35,7 +35,6 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; -import java.text.SimpleDateFormat; import java.time.ZoneId; import java.time.format.DateTimeFormatter; import java.util.HashMap; @@ -50,7 +49,8 @@ public class DorisEventSerializer implements DorisRecordSerializer { private Map schemaMaps = new HashMap<>(); /** Format DATE type data. */ - public static final SimpleDateFormat DATE_FORMATTER = new SimpleDateFormat("yyyy-MM-dd"); + public static final DateTimeFormatter DATE_FORMATTER = + DateTimeFormatter.ofPattern("yyyy-MM-dd"); /** Format timestamp-related type data. */ public static final DateTimeFormatter DATE_TIME_FORMATTER = diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/main/java/com/ververica/cdc/connectors/doris/sink/DorisMetadataApplier.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/main/java/com/ververica/cdc/connectors/doris/sink/DorisMetadataApplier.java index 7ad1235f61e..a7691a5aa0c 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/main/java/com/ververica/cdc/connectors/doris/sink/DorisMetadataApplier.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/main/java/com/ververica/cdc/connectors/doris/sink/DorisMetadataApplier.java @@ -79,7 +79,7 @@ public void applySchemaChange(SchemaChangeEvent event) { } else if (event instanceof RenameColumnEvent) { applyRenameColumnEvent((RenameColumnEvent) event); } else if (event instanceof AlterColumnTypeEvent) { - throw new RuntimeException("Unsupport schema change event, " + event); + throw new RuntimeException("Unsupported schema change event, " + event); } } catch (Exception ex) { throw new RuntimeException( diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/main/java/com/ververica/cdc/connectors/doris/sink/DorisRowConverter.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/main/java/com/ververica/cdc/connectors/doris/sink/DorisRowConverter.java index bfef35ded71..1788a7071d7 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/main/java/com/ververica/cdc/connectors/doris/sink/DorisRowConverter.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/main/java/com/ververica/cdc/connectors/doris/sink/DorisRowConverter.java @@ -32,7 +32,6 @@ import java.io.IOException; import java.io.Serializable; -import java.sql.Date; import java.time.LocalDate; import java.time.ZoneId; import java.time.ZonedDateTime; @@ -100,8 +99,7 @@ static SerializationConverter createExternalConverter(DataType type, ZoneId pipe return (index, val) -> val.getDouble(index); case DATE: return (index, val) -> - DATE_FORMATTER.format( - Date.valueOf(LocalDate.ofEpochDay(val.getInt(index)))); + LocalDate.ofEpochDay(val.getInt(index)).format(DATE_FORMATTER); case TIMESTAMP_WITHOUT_TIME_ZONE: return (index, val) -> val.getTimestamp(index, DataTypeChecks.getPrecision(type)) diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/com/ververica/cdc/connectors/mysql/factory/MySqlDataSourceFactory.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/com/ververica/cdc/connectors/mysql/factory/MySqlDataSourceFactory.java index dfd7af01273..33372132764 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/com/ververica/cdc/connectors/mysql/factory/MySqlDataSourceFactory.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/com/ververica/cdc/connectors/mysql/factory/MySqlDataSourceFactory.java @@ -203,6 +203,8 @@ public String identifier() { } private static final String SCAN_STARTUP_MODE_VALUE_INITIAL = "initial"; + + private static final String SCAN_STARTUP_MODE_VALUE_SNAPSHOT = "snapshot"; private static final String SCAN_STARTUP_MODE_VALUE_EARLIEST = "earliest-offset"; private static final String SCAN_STARTUP_MODE_VALUE_LATEST = "latest-offset"; private static final String SCAN_STARTUP_MODE_VALUE_SPECIFIC_OFFSET = "specific-offset"; @@ -221,7 +223,8 @@ private static StartupOptions getStartupOptions(Configuration config) { switch (modeString.toLowerCase()) { case SCAN_STARTUP_MODE_VALUE_INITIAL: return StartupOptions.initial(); - + case SCAN_STARTUP_MODE_VALUE_SNAPSHOT: + return StartupOptions.snapshot(); case SCAN_STARTUP_MODE_VALUE_LATEST: return StartupOptions.latest(); @@ -238,9 +241,10 @@ private static StartupOptions getStartupOptions(Configuration config) { default: throw new ValidationException( String.format( - "Invalid value for option '%s'. Supported values are [%s, %s, %s, %s, %s], but was: %s", + "Invalid value for option '%s'. Supported values are [%s, %s, %s, %s, %s, %s], but was: %s", SCAN_STARTUP_MODE.key(), SCAN_STARTUP_MODE_VALUE_INITIAL, + SCAN_STARTUP_MODE_VALUE_SNAPSHOT, SCAN_STARTUP_MODE_VALUE_LATEST, SCAN_STARTUP_MODE_VALUE_EARLIEST, SCAN_STARTUP_MODE_VALUE_SPECIFIC_OFFSET, diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/com/ververica/cdc/connectors/mysql/source/parser/CustomAlterTableParserListener.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/com/ververica/cdc/connectors/mysql/source/parser/CustomAlterTableParserListener.java index 93d738767a2..b313960ac36 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/com/ververica/cdc/connectors/mysql/source/parser/CustomAlterTableParserListener.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/com/ververica/cdc/connectors/mysql/source/parser/CustomAlterTableParserListener.java @@ -44,7 +44,7 @@ import static com.ververica.cdc.connectors.mysql.utils.MySqlTypeUtils.fromDbzColumn; -/** Copied from {@link AlterTableParserListener} in Debezium 1.9.7.Final. */ +/** Copied from {@link AlterTableParserListener} in Debezium 1.9.8.Final. */ public class CustomAlterTableParserListener extends MySqlParserBaseListener { private static final int STARTING_INDEX = 1; diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/com/ververica/cdc/connectors/mysql/source/parser/CustomMySqlAntlrDdlParserListener.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/com/ververica/cdc/connectors/mysql/source/parser/CustomMySqlAntlrDdlParserListener.java index 089a81208af..be99d36536d 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/com/ververica/cdc/connectors/mysql/source/parser/CustomMySqlAntlrDdlParserListener.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/com/ververica/cdc/connectors/mysql/source/parser/CustomMySqlAntlrDdlParserListener.java @@ -48,7 +48,7 @@ import java.util.concurrent.CopyOnWriteArrayList; /** - * Copied from {@link MySqlAntlrDdlParserListener} in Debezium 1.9.7.final. + * Copied from {@link MySqlAntlrDdlParserListener} in Debezium 1.9.8.final. * *

    This listener's constructor will use some modified listener. */ diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/com/ververica/cdc/connectors/mysql/source/MySqlFullTypesITCase.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/com/ververica/cdc/connectors/mysql/source/MySqlFullTypesITCase.java index 498f02c092c..a74e62b2fa6 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/com/ververica/cdc/connectors/mysql/source/MySqlFullTypesITCase.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/com/ververica/cdc/connectors/mysql/source/MySqlFullTypesITCase.java @@ -45,6 +45,7 @@ import java.math.BigDecimal; import java.sql.Connection; import java.sql.Statement; +import java.sql.Timestamp; import java.time.Instant; import java.time.ZoneId; import java.util.Arrays; @@ -118,6 +119,7 @@ public void testMysql57TimeDataTypes() throws Throwable { DataTypes.TIMESTAMP(0), DataTypes.TIMESTAMP(3), DataTypes.TIMESTAMP(6), + DataTypes.TIMESTAMP_LTZ(0), DataTypes.TIMESTAMP_LTZ(0)); Object[] expectedSnapshot = @@ -131,10 +133,11 @@ public void testMysql57TimeDataTypes() throws Throwable { // Because Flink's BinaryWriter force write int value for TIME(6). // See BinaryWriter#write for detail. 64822123, - TimestampData.fromMillis(1595008822000L), - TimestampData.fromMillis(1595008822123L), - TimestampData.fromMillis(1595008822123L, 456000), - LocalZonedTimestampData.fromEpochMillis(1595008822000L, 0) + TimestampData.fromTimestamp(Timestamp.valueOf("2020-07-17 18:00:22")), + TimestampData.fromTimestamp(Timestamp.valueOf("2020-07-17 18:00:22.123")), + TimestampData.fromTimestamp(Timestamp.valueOf("2020-07-17 18:00:22.123456")), + LocalZonedTimestampData.fromInstant(toInstant("2020-07-17 18:00:22")), + null }; Object[] expectedStreamRecord = @@ -145,10 +148,11 @@ public void testMysql57TimeDataTypes() throws Throwable { 64822000, 64822123, null, - TimestampData.fromMillis(1595008822000L), - TimestampData.fromMillis(1595008822123L), - TimestampData.fromMillis(1595008822123L, 456000), - LocalZonedTimestampData.fromEpochMillis(1595008822000L, 0) + TimestampData.fromTimestamp(Timestamp.valueOf("2020-07-17 18:00:22")), + TimestampData.fromTimestamp(Timestamp.valueOf("2020-07-17 18:00:22.123")), + TimestampData.fromTimestamp(Timestamp.valueOf("2020-07-17 18:00:22.123456")), + LocalZonedTimestampData.fromInstant(toInstant("2020-07-17 18:00:22")), + LocalZonedTimestampData.fromInstant(toInstant("2000-01-01 00:00:00")) }; testTimeDataTypes( @@ -170,7 +174,8 @@ public void testMysql8TimeDataTypes() throws Throwable { DataTypes.TIMESTAMP(6), DataTypes.TIMESTAMP_LTZ(0), DataTypes.TIMESTAMP_LTZ(3), - DataTypes.TIMESTAMP_LTZ(6)); + DataTypes.TIMESTAMP_LTZ(6), + DataTypes.TIMESTAMP_LTZ(0)); Object[] expectedSnapshot = new Object[] { @@ -183,13 +188,13 @@ public void testMysql8TimeDataTypes() throws Throwable { // Because Flink's BinaryWriter force write int value for TIME(6). // See BinaryWriter#write for detail. 64822123, - TimestampData.fromMillis(1595008822000L), - TimestampData.fromMillis(1595008822123L), - TimestampData.fromMillis(1595008822123L, 456000), - LocalZonedTimestampData.fromInstant(Instant.parse("2020-07-17T18:00:22Z")), - LocalZonedTimestampData.fromInstant(Instant.parse("2020-07-17T18:00:22.123Z")), - LocalZonedTimestampData.fromInstant( - Instant.parse("2020-07-17T18:00:22.123456Z")) + TimestampData.fromTimestamp(Timestamp.valueOf("2020-07-17 18:00:22")), + TimestampData.fromTimestamp(Timestamp.valueOf("2020-07-17 18:00:22.123")), + TimestampData.fromTimestamp(Timestamp.valueOf("2020-07-17 18:00:22.123456")), + LocalZonedTimestampData.fromInstant(toInstant("2020-07-17 18:00:22")), + LocalZonedTimestampData.fromInstant(toInstant("2020-07-17 18:00:22.123")), + LocalZonedTimestampData.fromInstant(toInstant("2020-07-17 18:00:22.123456")), + null }; Object[] expectedStreamRecord = @@ -200,13 +205,13 @@ public void testMysql8TimeDataTypes() throws Throwable { 64822000, 64822123, null, - TimestampData.fromMillis(1595008822000L), - TimestampData.fromMillis(1595008822123L), - TimestampData.fromMillis(1595008822123L, 456000), - LocalZonedTimestampData.fromInstant(Instant.parse("2020-07-17T18:00:22Z")), - LocalZonedTimestampData.fromInstant(Instant.parse("2020-07-17T18:00:22.123Z")), - LocalZonedTimestampData.fromInstant( - Instant.parse("2020-07-17T18:00:22.123456Z")) + TimestampData.fromTimestamp(Timestamp.valueOf("2020-07-17 18:00:22")), + TimestampData.fromTimestamp(Timestamp.valueOf("2020-07-17 18:00:22.123")), + TimestampData.fromTimestamp(Timestamp.valueOf("2020-07-17 18:00:22.123456")), + LocalZonedTimestampData.fromInstant(toInstant("2020-07-17 18:00:22")), + LocalZonedTimestampData.fromInstant(toInstant("2020-07-17 18:00:22.123")), + LocalZonedTimestampData.fromInstant(toInstant("2020-07-17 18:00:22.123456")), + LocalZonedTimestampData.fromInstant(toInstant("2000-01-01 00:00:00")) }; testTimeDataTypes( @@ -317,6 +322,10 @@ private void testCommonDataTypes(UniqueDatabase database) throws Exception { assertThat(recordFields(streamRecord, COMMON_TYPES)).isEqualTo(expectedStreamRecord); } + private Instant toInstant(String ts) { + return Timestamp.valueOf(ts).toLocalDateTime().atZone(ZoneId.of("UTC")).toInstant(); + } + private void testTimeDataTypes( UniqueDatabase database, RowType recordType, @@ -340,7 +349,8 @@ private void testTimeDataTypes( try (Connection connection = database.getJdbcConnection(); Statement statement = connection.createStatement()) { - statement.execute("UPDATE time_types SET time_6_c = null WHERE id = 1;"); + statement.execute( + "UPDATE time_types SET time_6_c = null, timestamp_def_c = default WHERE id = 1;"); } List streamResults = fetchResults(iterator, 1); diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/com/ververica/cdc/connectors/mysql/source/MySqlMetadataAccessorITCase.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/com/ververica/cdc/connectors/mysql/source/MySqlMetadataAccessorITCase.java index 1c3466320f4..e31f713b6de 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/com/ververica/cdc/connectors/mysql/source/MySqlMetadataAccessorITCase.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/com/ververica/cdc/connectors/mysql/source/MySqlMetadataAccessorITCase.java @@ -130,6 +130,7 @@ public void testMysql57AccessTimeTypesSchema() { DataTypes.TIMESTAMP(0), DataTypes.TIMESTAMP(3), DataTypes.TIMESTAMP(6), + DataTypes.TIMESTAMP_LTZ(0), DataTypes.TIMESTAMP_LTZ(0) }, new String[] { @@ -142,7 +143,8 @@ public void testMysql57AccessTimeTypesSchema() { "datetime_c", "datetime3_c", "datetime6_c", - "timestamp_c" + "timestamp_c", + "timestamp_def_c" })) .build(); assertThat(actualSchema).isEqualTo(expectedSchema); @@ -176,7 +178,8 @@ public void testMysql8AccessTimeTypesSchema() { DataTypes.TIMESTAMP(6), DataTypes.TIMESTAMP_LTZ(0), DataTypes.TIMESTAMP_LTZ(3), - DataTypes.TIMESTAMP_LTZ(6) + DataTypes.TIMESTAMP_LTZ(6), + DataTypes.TIMESTAMP_LTZ(0) }, new String[] { "id", @@ -190,7 +193,8 @@ public void testMysql8AccessTimeTypesSchema() { "datetime6_c", "timestamp_c", "timestamp3_c", - "timestamp6_c" + "timestamp6_c", + "timestamp_def_c" })) .build(); assertThat(actualSchema).isEqualTo(expectedSchema); diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/com/ververica/cdc/connectors/mysql/source/MySqlPipelineITCase.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/com/ververica/cdc/connectors/mysql/source/MySqlPipelineITCase.java index 0e4b78358ad..00bdda4bae9 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/com/ververica/cdc/connectors/mysql/source/MySqlPipelineITCase.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/com/ververica/cdc/connectors/mysql/source/MySqlPipelineITCase.java @@ -522,6 +522,12 @@ private List executeAlterAndProvideExpected(TableId tableId, Statement st tableId, Collections.singletonList( Column.physicalColumn("DESC3", DataTypes.BIGINT())))); + + // Should not catch SchemaChangeEvent of tables other than `products` + statement.execute( + String.format( + "ALTER TABLE `%s`.`orders` ADD COLUMN `desc1` VARCHAR(45) NULL;", + inventoryDatabase.getDatabaseName())); return expected; } } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/resources/ddl/column_type_test.sql b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/resources/ddl/column_type_test.sql index dbb8f4885af..d0fa3514ab3 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/resources/ddl/column_type_test.sql +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/resources/ddl/column_type_test.sql @@ -106,6 +106,7 @@ CREATE TABLE time_types datetime3_c DATETIME(3), datetime6_c DATETIME(6), timestamp_c TIMESTAMP NULL, + timestamp_def_c TIMESTAMP NULL DEFAULT '2000-01-01 00:00:00', PRIMARY KEY (id) ) DEFAULT CHARSET=utf8; @@ -119,4 +120,5 @@ VALUES (DEFAULT, '2020-07-17 18:00:22', '2020-07-17 18:00:22.123', '2020-07-17 18:00:22.123456', - '2020-07-17 18:00:22'); \ No newline at end of file + '2020-07-17 18:00:22', + NULL); \ No newline at end of file diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/resources/ddl/column_type_test_mysql8.sql b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/resources/ddl/column_type_test_mysql8.sql index 6dc3e13ebf8..05843ef625e 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/resources/ddl/column_type_test_mysql8.sql +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/resources/ddl/column_type_test_mysql8.sql @@ -108,6 +108,7 @@ CREATE TABLE time_types timestamp_c TIMESTAMP(0), timestamp3_c TIMESTAMP(3), timestamp6_c TIMESTAMP(6), + timestamp_def_c TIMESTAMP NULL DEFAULT '2000-01-01 00:00:00', PRIMARY KEY (id) ) DEFAULT CHARSET=utf8; @@ -123,4 +124,5 @@ VALUES (DEFAULT, '2020-07-17 18:00:22.123456', '2020-07-17 18:00:22', '2020-07-17 18:00:22.123', - '2020-07-17 18:00:22.123456'); \ No newline at end of file + '2020-07-17 18:00:22.123456', + NULL); \ No newline at end of file diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/main/java/com/ververica/cdc/connectors/starrocks/sink/StarRocksMetadataApplier.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/main/java/com/ververica/cdc/connectors/starrocks/sink/StarRocksMetadataApplier.java index afb083ba233..ba5c5b4515b 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/main/java/com/ververica/cdc/connectors/starrocks/sink/StarRocksMetadataApplier.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/main/java/com/ververica/cdc/connectors/starrocks/sink/StarRocksMetadataApplier.java @@ -119,7 +119,7 @@ private void applyAddColumn(AddColumnEvent addColumnEvent) { .setColumnName(column.getName()) .setOrdinalPosition(-1) .setColumnComment(column.getComment()); - toStarRocksDataType(column, builder); + toStarRocksDataType(column, false, builder); addColumns.add(builder.build()); } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/main/java/com/ververica/cdc/connectors/starrocks/sink/StarRocksUtils.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/main/java/com/ververica/cdc/connectors/starrocks/sink/StarRocksUtils.java index 9b3f01ec910..3c7a7535a5b 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/main/java/com/ververica/cdc/connectors/starrocks/sink/StarRocksUtils.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/main/java/com/ververica/cdc/connectors/starrocks/sink/StarRocksUtils.java @@ -38,8 +38,6 @@ import com.ververica.cdc.common.types.TinyIntType; import com.ververica.cdc.common.types.VarCharType; -import java.sql.Date; -import java.text.SimpleDateFormat; import java.time.LocalDate; import java.time.ZoneId; import java.time.ZonedDateTime; @@ -76,6 +74,7 @@ public static StarRocksTable toStarRocksTable( } } + int primaryKeyCount = schema.primaryKeys().size(); List starRocksColumns = new ArrayList<>(); for (int i = 0; i < orderedColumns.size(); i++) { Column column = orderedColumns.get(i); @@ -84,7 +83,7 @@ public static StarRocksTable toStarRocksTable( .setColumnName(column.getName()) .setOrdinalPosition(i) .setColumnComment(column.getComment()); - toStarRocksDataType(column, builder); + toStarRocksDataType(column, i < primaryKeyCount, builder); starRocksColumns.add(builder.build()); } @@ -106,13 +105,16 @@ public static StarRocksTable toStarRocksTable( } /** Convert CDC data type to StarRocks data type. */ - public static void toStarRocksDataType(Column cdcColumn, StarRocksColumn.Builder builder) { - CdcDataTypeTransformer dataTypeTransformer = new CdcDataTypeTransformer(builder); + public static void toStarRocksDataType( + Column cdcColumn, boolean isPrimaryKeys, StarRocksColumn.Builder builder) { + CdcDataTypeTransformer dataTypeTransformer = + new CdcDataTypeTransformer(isPrimaryKeys, builder); cdcColumn.getType().accept(dataTypeTransformer); } /** Format DATE type data. */ - private static final SimpleDateFormat DATE_FORMATTER = new SimpleDateFormat("yyyy-MM-dd"); + private static final DateTimeFormatter DATE_FORMATTER = + DateTimeFormatter.ofPattern("yyyy-MM-dd"); /** Format timestamp-related type data. */ private static final DateTimeFormatter DATETIME_FORMATTER = @@ -168,9 +170,8 @@ record -> case DATE: fieldGetter = record -> - DATE_FORMATTER.format( - Date.valueOf( - LocalDate.ofEpochDay(record.getInt(fieldPos)))); + LocalDate.ofEpochDay(record.getInt(fieldPos)) + .format(DATE_FORMATTER); break; case TIMESTAMP_WITHOUT_TIME_ZONE: fieldGetter = @@ -236,8 +237,10 @@ public static class CdcDataTypeTransformer extends DataTypeDefaultVisitor { private final StarRocksColumn.Builder builder; + private final boolean isPrimaryKeys; - public CdcDataTypeTransformer(StarRocksColumn.Builder builder) { + public CdcDataTypeTransformer(boolean isPrimaryKeys, StarRocksColumn.Builder builder) { + this.isPrimaryKeys = isPrimaryKeys; this.builder = builder; } @@ -309,7 +312,11 @@ public StarRocksColumn.Builder visit(CharType charType) { // varchar type int length = charType.getLength(); long starRocksLength = length * 3L; - if (starRocksLength <= MAX_CHAR_SIZE) { + // In the StarRocks, The primary key columns can be any of the following data types: + // BOOLEAN, TINYINT, SMALLINT, INT, BIGINT, LARGEINT, STRING, VARCHAR, DATE, and + // DATETIME, But it doesn't include CHAR. When a char type appears in the primary key of + // MySQL, creating a table in StarRocks requires conversion to varchar type. + if (starRocksLength <= MAX_CHAR_SIZE && !isPrimaryKeys) { builder.setDataType(CHAR); builder.setNullable(charType.isNullable()); builder.setColumnSize((int) starRocksLength); diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/test/java/com/ververical/cdc/connectors/starrocks/sink/CdcDataTypeTransformerTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/test/java/com/ververical/cdc/connectors/starrocks/sink/CdcDataTypeTransformerTest.java index f2ce9344b73..54b78c472a9 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/test/java/com/ververical/cdc/connectors/starrocks/sink/CdcDataTypeTransformerTest.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/test/java/com/ververical/cdc/connectors/starrocks/sink/CdcDataTypeTransformerTest.java @@ -33,7 +33,8 @@ public void testCharType() { // map to char of StarRocks if CDC length <= StarRocksUtils.MAX_CHAR_SIZE StarRocksColumn.Builder smallLengthBuilder = new StarRocksColumn.Builder().setColumnName("small_char").setOrdinalPosition(0); - new CharType(1).accept(new StarRocksUtils.CdcDataTypeTransformer(smallLengthBuilder)); + new CharType(1) + .accept(new StarRocksUtils.CdcDataTypeTransformer(false, smallLengthBuilder)); StarRocksColumn smallLengthColumn = smallLengthBuilder.build(); assertEquals("small_char", smallLengthColumn.getColumnName()); assertEquals(0, smallLengthColumn.getOrdinalPosition()); @@ -45,7 +46,7 @@ public void testCharType() { StarRocksColumn.Builder largeLengthBuilder = new StarRocksColumn.Builder().setColumnName("large_char").setOrdinalPosition(1); new CharType(StarRocksUtils.MAX_CHAR_SIZE) - .accept(new StarRocksUtils.CdcDataTypeTransformer(largeLengthBuilder)); + .accept(new StarRocksUtils.CdcDataTypeTransformer(false, largeLengthBuilder)); StarRocksColumn largeLengthColumn = largeLengthBuilder.build(); assertEquals("large_char", largeLengthColumn.getColumnName()); assertEquals(1, largeLengthColumn.getOrdinalPosition()); @@ -56,13 +57,28 @@ public void testCharType() { assertTrue(largeLengthColumn.isNullable()); } + @Test + public void testCharTypeForPrimaryKey() { + // map to varchar of StarRocks if column is primary key + StarRocksColumn.Builder smallLengthBuilder = + new StarRocksColumn.Builder().setColumnName("primary_key").setOrdinalPosition(0); + new CharType(1).accept(new StarRocksUtils.CdcDataTypeTransformer(true, smallLengthBuilder)); + StarRocksColumn smallLengthColumn = smallLengthBuilder.build(); + assertEquals("primary_key", smallLengthColumn.getColumnName()); + assertEquals(0, smallLengthColumn.getOrdinalPosition()); + assertEquals(StarRocksUtils.VARCHAR, smallLengthColumn.getDataType()); + assertEquals(Integer.valueOf(3), smallLengthColumn.getColumnSize().orElse(null)); + assertTrue(smallLengthColumn.isNullable()); + } + @Test public void testVarCharType() { // the length fo StarRocks should be 3 times as that of CDC if CDC length * 3 <= // StarRocksUtils.MAX_VARCHAR_SIZE StarRocksColumn.Builder smallLengthBuilder = new StarRocksColumn.Builder().setColumnName("small_varchar").setOrdinalPosition(0); - new VarCharType(3).accept(new StarRocksUtils.CdcDataTypeTransformer(smallLengthBuilder)); + new VarCharType(3) + .accept(new StarRocksUtils.CdcDataTypeTransformer(false, smallLengthBuilder)); StarRocksColumn smallLengthColumn = smallLengthBuilder.build(); assertEquals("small_varchar", smallLengthColumn.getColumnName()); assertEquals(0, smallLengthColumn.getOrdinalPosition()); @@ -75,7 +91,7 @@ public void testVarCharType() { StarRocksColumn.Builder largeLengthBuilder = new StarRocksColumn.Builder().setColumnName("large_varchar").setOrdinalPosition(1); new CharType(StarRocksUtils.MAX_VARCHAR_SIZE + 1) - .accept(new StarRocksUtils.CdcDataTypeTransformer(largeLengthBuilder)); + .accept(new StarRocksUtils.CdcDataTypeTransformer(false, largeLengthBuilder)); StarRocksColumn largeLengthColumn = largeLengthBuilder.build(); assertEquals("large_varchar", largeLengthColumn.getColumnName()); assertEquals(1, largeLengthColumn.getOrdinalPosition()); diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/config/JdbcSourceConfigFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/config/JdbcSourceConfigFactory.java index 3c1474874e6..61607a7160f 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/config/JdbcSourceConfigFactory.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/config/JdbcSourceConfigFactory.java @@ -201,6 +201,7 @@ public JdbcSourceConfigFactory chunkKeyColumn(String chunkKeyColumn) { public JdbcSourceConfigFactory startupOptions(StartupOptions startupOptions) { switch (startupOptions.startupMode) { case INITIAL: + case SNAPSHOT: case LATEST_OFFSET: break; default: diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/dialect/DataSourceDialect.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/dialect/DataSourceDialect.java index c41eee8af6d..402ee609f52 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/dialect/DataSourceDialect.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/dialect/DataSourceDialect.java @@ -79,4 +79,12 @@ public interface DataSourceDialect */ @Override default void notifyCheckpointComplete(long checkpointId) throws Exception {} + + /** + * We may need the offset corresponding to the checkpointId. For example, we should commit LSN + * of checkpoint to postgres's slot. + */ + default void notifyCheckpointComplete(long checkpointId, Offset offset) throws Exception { + notifyCheckpointComplete(checkpointId); + } } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/options/StartupMode.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/options/StartupMode.java index ef6381fc856..67a3285fda9 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/options/StartupMode.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/options/StartupMode.java @@ -30,5 +30,6 @@ public enum StartupMode { SPECIFIC_OFFSETS, - TIMESTAMP + TIMESTAMP, + SNAPSHOT } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/options/StartupOptions.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/options/StartupOptions.java index d365b9e91d3..880d09210ce 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/options/StartupOptions.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/options/StartupOptions.java @@ -38,6 +38,14 @@ public static StartupOptions initial() { return new StartupOptions(StartupMode.INITIAL, null, null, null); } + /** + * Performs an initial snapshot on the monitored database tables upon first startup, and not + * read the binlog anymore . + */ + public static StartupOptions snapshot() { + return new StartupOptions(StartupMode.SNAPSHOT, null, null, null); + } + /** * Never to perform snapshot on the monitored database tables upon first startup, just read from * the beginning of the change log. This should be used with care, as it is only valid when the @@ -89,6 +97,7 @@ private StartupOptions( switch (startupMode) { case INITIAL: + case SNAPSHOT: case EARLIEST_OFFSET: case LATEST_OFFSET: break; @@ -104,6 +113,17 @@ private StartupOptions( } } + public boolean isStreamOnly() { + return startupMode == StartupMode.EARLIEST_OFFSET + || startupMode == StartupMode.LATEST_OFFSET + || startupMode == StartupMode.SPECIFIC_OFFSETS + || startupMode == StartupMode.TIMESTAMP; + } + + public boolean isSnapshotOnly() { + return startupMode == StartupMode.SNAPSHOT; + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/IncrementalSource.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/IncrementalSource.java index 55f676a7222..8b490e8d66f 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/IncrementalSource.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/IncrementalSource.java @@ -33,7 +33,6 @@ import com.ververica.cdc.common.annotation.VisibleForTesting; import com.ververica.cdc.connectors.base.config.SourceConfig; import com.ververica.cdc.connectors.base.dialect.DataSourceDialect; -import com.ververica.cdc.connectors.base.options.StartupMode; import com.ververica.cdc.connectors.base.source.assigner.HybridSplitAssigner; import com.ververica.cdc.connectors.base.source.assigner.SplitAssigner; import com.ververica.cdc.connectors.base.source.assigner.StreamSplitAssigner; @@ -79,7 +78,7 @@ public class IncrementalSource // This field is introduced for testing purpose, for example testing if changes made in the // snapshot phase are correctly backfilled into the snapshot by registering a pre high watermark // hook for generating changes. - private SnapshotPhaseHooks snapshotHooks = SnapshotPhaseHooks.empty(); + protected SnapshotPhaseHooks snapshotHooks = SnapshotPhaseHooks.empty(); public IncrementalSource( SourceConfig.Factory configFactory, @@ -101,7 +100,12 @@ public OffsetFactory getOffsetFactory() { @Override public Boundedness getBoundedness() { - return Boundedness.CONTINUOUS_UNBOUNDED; + C sourceConfig = configFactory.create(0); + if (sourceConfig.getStartupOptions().isSnapshotOnly()) { + return Boundedness.BOUNDED; + } else { + return Boundedness.CONTINUOUS_UNBOUNDED; + } } @Override @@ -139,7 +143,7 @@ public SplitEnumerator createEnumerator( SplitEnumeratorContext enumContext) { C sourceConfig = configFactory.create(0); final SplitAssigner splitAssigner; - if (sourceConfig.getStartupOptions().startupMode == StartupMode.INITIAL) { + if (!sourceConfig.getStartupOptions().isStreamOnly()) { try { final List remainingTables = dataSourceDialect.discoverDataCollections(sourceConfig); @@ -161,7 +165,8 @@ public SplitEnumerator createEnumerator( splitAssigner = new StreamSplitAssigner(sourceConfig, dataSourceDialect, offsetFactory); } - return new IncrementalSourceEnumerator(enumContext, sourceConfig, splitAssigner); + return new IncrementalSourceEnumerator( + enumContext, sourceConfig, splitAssigner, getBoundedness()); } @Override @@ -189,7 +194,8 @@ public SplitEnumerator restoreEnumerator( throw new UnsupportedOperationException( "Unsupported restored PendingSplitsState: " + checkpoint); } - return new IncrementalSourceEnumerator(enumContext, sourceConfig, splitAssigner); + return new IncrementalSourceEnumerator( + enumContext, sourceConfig, splitAssigner, getBoundedness()); } @Override diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/HybridSplitAssigner.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/HybridSplitAssigner.java index 98bfbfeb8cb..db7f1d1fc2f 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/HybridSplitAssigner.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/HybridSplitAssigner.java @@ -46,6 +46,7 @@ public class HybridSplitAssigner implements SplitAssigne private static final String STREAM_SPLIT_ID = "stream-split"; private final int splitMetaGroupSize; + private final C sourceConfig; private boolean isStreamSplitAssigned; @@ -61,6 +62,7 @@ public HybridSplitAssigner( DataSourceDialect dialect, OffsetFactory offsetFactory) { this( + sourceConfig, new SnapshotSplitAssigner<>( sourceConfig, currentParallelism, @@ -80,6 +82,7 @@ public HybridSplitAssigner( DataSourceDialect dialect, OffsetFactory offsetFactory) { this( + sourceConfig, new SnapshotSplitAssigner<>( sourceConfig, currentParallelism, @@ -92,10 +95,12 @@ public HybridSplitAssigner( } private HybridSplitAssigner( + C sourceConfig, SnapshotSplitAssigner snapshotSplitAssigner, boolean isStreamSplitAssigned, int splitMetaGroupSize, OffsetFactory offsetFactory) { + this.sourceConfig = sourceConfig; this.snapshotSplitAssigner = snapshotSplitAssigner; this.isStreamSplitAssigned = isStreamSplitAssigned; this.splitMetaGroupSize = splitMetaGroupSize; @@ -179,8 +184,8 @@ public void notifyCheckpointComplete(long checkpointId) { } @Override - public boolean isStreamSplitAssigned() { - return isStreamSplitAssigned; + public boolean noMoreSplits() { + return snapshotSplitAssigner.noMoreSplits() && isStreamSplitAssigned; } @Override @@ -199,13 +204,17 @@ public StreamSplit createStreamSplit() { Map splitFinishedOffsets = snapshotSplitAssigner.getSplitFinishedOffsets(); final List finishedSnapshotSplitInfos = new ArrayList<>(); - Offset minOffset = null; + Offset minOffset = null, maxOffset = null; for (SchemalessSnapshotSplit split : assignedSnapshotSplit) { - // find the min offset of change log + // find the min and max offset of change log Offset changeLogOffset = splitFinishedOffsets.get(split.splitId()); if (minOffset == null || changeLogOffset.isBefore(minOffset)) { minOffset = changeLogOffset; } + if (maxOffset == null || changeLogOffset.isAfter(maxOffset)) { + maxOffset = changeLogOffset; + } + finishedSnapshotSplitInfos.add( new FinishedSnapshotSplitInfo( split.getTableId(), @@ -216,14 +225,21 @@ public StreamSplit createStreamSplit() { offsetFactory)); } + // If the source is running in snapshot mode, we use the highest watermark among + // snapshot splits as the ending offset to provide a consistent snapshot view at the moment + // of high watermark. + Offset stoppingOffset = offsetFactory.createNoStoppingOffset(); + if (sourceConfig.getStartupOptions().isSnapshotOnly()) { + stoppingOffset = maxOffset; + } + // the finishedSnapshotSplitInfos is too large for transmission, divide it to groups and // then transfer them - boolean divideMetaToGroups = finishedSnapshotSplitInfos.size() > splitMetaGroupSize; return new StreamSplit( STREAM_SPLIT_ID, minOffset == null ? offsetFactory.createInitialOffset() : minOffset, - offsetFactory.createNoStoppingOffset(), + stoppingOffset, divideMetaToGroups ? new ArrayList<>() : finishedSnapshotSplitInfos, new HashMap<>(), finishedSnapshotSplitInfos.size()); diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/SnapshotSplitAssigner.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/SnapshotSplitAssigner.java index 5dea133fa63..673cb51bf59 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/SnapshotSplitAssigner.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/SnapshotSplitAssigner.java @@ -290,7 +290,7 @@ public void notifyCheckpointComplete(long checkpointId) { @Override public void close() {} - /** Indicates there is no more splits available in this assigner. */ + @Override public boolean noMoreSplits() { return remainingTables.isEmpty() && remainingSplits.isEmpty(); } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/SplitAssigner.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/SplitAssigner.java index 4fca7cffd9e..bd776b26246 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/SplitAssigner.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/SplitAssigner.java @@ -55,10 +55,8 @@ public interface SplitAssigner { */ boolean waitingForFinishedSplits(); - /** Whether the split assigner is finished stream split assigning. */ - default boolean isStreamSplitAssigned() { - throw new UnsupportedOperationException("Not support to assigning StreamSplit."); - } + /** Indicates there is no more splits available in this assigner. */ + boolean noMoreSplits(); /** * Gets the finished splits' information. This is useful metadata to generate a stream split diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/StreamSplitAssigner.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/StreamSplitAssigner.java index 82cbd23b8f0..9c5f8d62a24 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/StreamSplitAssigner.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/StreamSplitAssigner.java @@ -119,7 +119,7 @@ public void notifyCheckpointComplete(long checkpointId) { } @Override - public boolean isStreamSplitAssigned() { + public boolean noMoreSplits() { return isStreamSplitAssigned; } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/enumerator/IncrementalSourceEnumerator.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/enumerator/IncrementalSourceEnumerator.java index 291f6d973f1..46544629937 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/enumerator/IncrementalSourceEnumerator.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/enumerator/IncrementalSourceEnumerator.java @@ -16,6 +16,7 @@ package com.ververica.cdc.connectors.base.source.enumerator; +import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.connector.source.SourceEvent; import org.apache.flink.api.connector.source.SplitEnumerator; import org.apache.flink.api.connector.source.SplitEnumeratorContext; @@ -66,14 +67,18 @@ public class IncrementalSourceEnumerator private final TreeSet readersAwaitingSplit; private List> finishedSnapshotSplitMeta; + private Boundedness boundedness; + public IncrementalSourceEnumerator( SplitEnumeratorContext context, SourceConfig sourceConfig, - SplitAssigner splitAssigner) { + SplitAssigner splitAssigner, + Boundedness boundedness) { this.context = context; this.sourceConfig = sourceConfig; this.splitAssigner = splitAssigner; this.readersAwaitingSplit = new TreeSet<>(); + this.boundedness = boundedness; } @Override @@ -163,7 +168,7 @@ private void assignSplits() { continue; } - if (splitAssigner.isStreamSplitAssigned() && sourceConfig.isCloseIdleReaders()) { + if (shouldCloseIdleReader()) { // close idle readers when snapshot phase finished. context.signalNoMoreSplits(nextAwaiting); awaitingReader.remove(); @@ -184,6 +189,17 @@ private void assignSplits() { } } + private boolean shouldCloseIdleReader() { + // When no unassigned split anymore, Signal NoMoreSplitsEvent to awaiting reader in two + // situations: + // 1. When Set StartupMode = snapshot mode(also bounded), there's no more splits in the + // assigner. + // 2. When set scan.incremental.close-idle-reader.enabled = true, there's no more splits in + // the assigner. + return splitAssigner.noMoreSplits() + && (boundedness == Boundedness.BOUNDED || (sourceConfig.isCloseIdleReaders())); + } + private int[] getRegisteredReader() { return this.context.registeredReaders().keySet().stream() .mapToInt(Integer::intValue) diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/SourceSplitSerializer.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/SourceSplitSerializer.java index 378ea21cfb9..12a17237758 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/SourceSplitSerializer.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/SourceSplitSerializer.java @@ -159,7 +159,7 @@ public SourceSplitBase deserializeSplit(int version, byte[] serialized) throws I readFinishedSplitsInfo(version, in); Map tableChangeMap = readTableSchemas(version, in); int totalFinishedSplitSize = finishedSplitsInfo.size(); - if (version == 3) { + if (version >= 3) { totalFinishedSplitSize = in.readInt(); } in.releaseArrays(); diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/IncrementalSourceReader.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/IncrementalSourceReader.java index a9f943b14e8..c22a9e32975 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/IncrementalSourceReader.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/IncrementalSourceReader.java @@ -77,7 +77,7 @@ public class IncrementalSourceReader private final int subtaskId; private final SourceSplitSerializer sourceSplitSerializer; private final C sourceConfig; - private final DataSourceDialect dialect; + protected final DataSourceDialect dialect; public IncrementalSourceReader( FutureCompletingBlockingQueue> elementQueue, @@ -141,6 +141,10 @@ public void notifyCheckpointComplete(long checkpointId) throws Exception { protected void onSplitFinished(Map finishedSplitIds) { for (SourceSplitState splitState : finishedSplitIds.values()) { SourceSplitBase sourceSplit = splitState.toSourceSplit(); + if (sourceConfig.getStartupOptions().isSnapshotOnly() && sourceSplit.isStreamSplit()) { + // when startupMode = SNAPSHOT. the stream split could finish. + continue; + } checkState( sourceSplit.isSnapshotSplit(), String.format( diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/IncrementalSourceReaderWithCommit.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/IncrementalSourceReaderWithCommit.java new file mode 100644 index 00000000000..f538bbcafe6 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/IncrementalSourceReaderWithCommit.java @@ -0,0 +1,100 @@ +/* + * Copyright 2023 Ververica Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.ververica.cdc.connectors.base.source.reader; + +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.base.source.reader.RecordEmitter; +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; + +import com.ververica.cdc.connectors.base.config.SourceConfig; +import com.ververica.cdc.connectors.base.dialect.DataSourceDialect; +import com.ververica.cdc.connectors.base.source.meta.offset.Offset; +import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitBase; +import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitSerializer; +import com.ververica.cdc.connectors.base.source.meta.split.StreamSplit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; +import java.util.TreeMap; +import java.util.function.Supplier; + +/** + * Record the LSN of checkpoint {@link StreamSplit}, which can be used to submit to the CDC source. + */ +public class IncrementalSourceReaderWithCommit extends IncrementalSourceReader { + private static final Logger LOG = + LoggerFactory.getLogger(IncrementalSourceReaderWithCommit.class); + + private final TreeMap lastCheckPointOffset; + private long maxCompletedCheckpointId; + + public IncrementalSourceReaderWithCommit( + FutureCompletingBlockingQueue elementQueue, + Supplier supplier, + RecordEmitter recordEmitter, + Configuration config, + SourceReaderContext context, + SourceConfig sourceConfig, + SourceSplitSerializer sourceSplitSerializer, + DataSourceDialect dialect) { + super( + elementQueue, + supplier, + recordEmitter, + config, + context, + sourceConfig, + sourceSplitSerializer, + dialect); + this.lastCheckPointOffset = new TreeMap<>(); + this.maxCompletedCheckpointId = 0; + } + + @Override + public List snapshotState(long checkpointId) { + final List stateSplits = super.snapshotState(checkpointId); + + stateSplits.stream() + .filter(SourceSplitBase::isStreamSplit) + .findAny() + .map(SourceSplitBase::asStreamSplit) + .ifPresent( + streamSplit -> { + lastCheckPointOffset.put(checkpointId, streamSplit.getStartingOffset()); + LOG.debug( + "Starting offset of stream split is: {}, and checkpoint id is {}.", + streamSplit.getStartingOffset(), + checkpointId); + }); + + return stateSplits; + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + // checkpointId might be for a checkpoint that was triggered earlier. see + // CheckpointListener#notifyCheckpointComplete(long). + if (checkpointId > maxCompletedCheckpointId) { + Offset offset = lastCheckPointOffset.get(checkpointId); + dialect.notifyCheckpointComplete(checkpointId, offset); + lastCheckPointOffset.headMap(checkpointId, true).clear(); + maxCompletedCheckpointId = checkpointId; + } + } +} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/IncrementalSourceSplitReader.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/IncrementalSourceSplitReader.java index 849cad1bc33..be6c3f36a55 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/IncrementalSourceSplitReader.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/IncrementalSourceSplitReader.java @@ -84,7 +84,7 @@ public RecordsWithSplitIds fetch() throws IOException { throw new IOException(e); } return dataIt == null - ? finishedSnapshotSplit() + ? finishedSplit() : ChangeEventRecords.forRecords(currentSplitId, dataIt); } @@ -154,7 +154,7 @@ public boolean canAssignNextSplit() { return currentFetcher == null || currentFetcher.isFinished(); } - private ChangeEventRecords finishedSnapshotSplit() { + private ChangeEventRecords finishedSplit() { final ChangeEventRecords finishedRecords = ChangeEventRecords.forFinishedSplit(currentSplitId); currentSplitId = null; diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/IncrementalSourceStreamFetcher.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/IncrementalSourceStreamFetcher.java index 28755ff5148..9a0c55e17a9 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/IncrementalSourceStreamFetcher.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/IncrementalSourceStreamFetcher.java @@ -35,7 +35,6 @@ import javax.annotation.Nullable; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -88,13 +87,18 @@ public void submitTask(FetchTask fetchTask) { try { streamFetchTask.execute(taskContext); } catch (Exception e) { - this.currentTaskRunning = false; LOG.error( String.format( "Execute stream read task for stream split %s fail", currentStreamSplit), e); readException = e; + } finally { + try { + stopReadTask(); + } catch (Exception e) { + throw new RuntimeException(e); + } } }); } @@ -122,7 +126,7 @@ public Iterator pollSplitRecords() throws InterruptedException { sourceRecordsSet.add(new SourceRecords(sourceRecords)); return sourceRecordsSet.iterator(); } else { - return Collections.emptyIterator(); + return null; } } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-debezium/src/main/java/com/ververica/cdc/debezium/event/DebeziumSchemaDataTypeInference.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-debezium/src/main/java/com/ververica/cdc/debezium/event/DebeziumSchemaDataTypeInference.java index 460d760a2be..dc40b36531a 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-debezium/src/main/java/com/ververica/cdc/debezium/event/DebeziumSchemaDataTypeInference.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-debezium/src/main/java/com/ververica/cdc/debezium/event/DebeziumSchemaDataTypeInference.java @@ -142,7 +142,7 @@ protected DataType inferString(Object value, Schema schema) { if (ZonedTimestamp.SCHEMA_NAME.equals(schema.name())) { int nano = Optional.ofNullable((String) value) - .map(Instant::parse) + .map(s -> ZonedTimestamp.FORMATTER.parse(s, Instant::from)) .map(Instant::getNano) .orElse(0); diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-debezium/src/main/java/com/ververica/cdc/debezium/table/AppendMetadataCollector.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-debezium/src/main/java/com/ververica/cdc/debezium/table/AppendMetadataCollector.java index cb8c58f3a43..888b0b485fa 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-debezium/src/main/java/com/ververica/cdc/debezium/table/AppendMetadataCollector.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-debezium/src/main/java/com/ververica/cdc/debezium/table/AppendMetadataCollector.java @@ -44,7 +44,14 @@ public AppendMetadataCollector(MetadataConverter[] metadataConverters) { public void collect(RowData physicalRow) { GenericRowData metaRow = new GenericRowData(metadataConverters.length); for (int i = 0; i < metadataConverters.length; i++) { - Object meta = metadataConverters[i].read(inputRecord); + MetadataConverter metadataConverter = metadataConverters[i]; + Object meta; + if (metadataConverter instanceof RowDataMetadataConverter) { + meta = ((RowDataMetadataConverter) metadataConverter).read(physicalRow); + } else { + meta = metadataConverter.read(inputRecord); + } + metaRow.setField(i, meta); } RowData outRow = new JoinedRowData(physicalRow.getRowKind(), physicalRow, metaRow); diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-debezium/src/main/java/com/ververica/cdc/debezium/table/MetadataConverter.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-debezium/src/main/java/com/ververica/cdc/debezium/table/MetadataConverter.java index f42e27cbb6c..4c07af54a35 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-debezium/src/main/java/com/ververica/cdc/debezium/table/MetadataConverter.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-debezium/src/main/java/com/ververica/cdc/debezium/table/MetadataConverter.java @@ -25,5 +25,6 @@ @FunctionalInterface @Internal public interface MetadataConverter extends Serializable { + Object read(SourceRecord record); } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/table/StartupMode.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-debezium/src/main/java/com/ververica/cdc/debezium/table/RowDataMetadataConverter.java similarity index 58% rename from flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/table/StartupMode.java rename to flink-cdc-connect/flink-cdc-source-connectors/flink-connector-debezium/src/main/java/com/ververica/cdc/debezium/table/RowDataMetadataConverter.java index c0f62da60b2..ef88197aa0d 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/table/StartupMode.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-debezium/src/main/java/com/ververica/cdc/debezium/table/RowDataMetadataConverter.java @@ -5,7 +5,7 @@ * you may not use this file except in compliance with the License. * You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,15 +14,15 @@ * limitations under the License. */ -package com.ververica.cdc.connectors.sqlserver.table; +package com.ververica.cdc.debezium.table; -/** - * Startup modes for the SqlServer CDC Consumer. - * - * @see StartupOptions - */ -public enum StartupMode { - INITIAL, - INITIAL_ONLY, - LATEST_OFFSET, +import org.apache.flink.table.data.RowData; + +import com.ververica.cdc.common.annotation.Internal; + +/** A converter converts {@link RowData} metadata into Flink internal data structures. */ +@Internal +public interface RowDataMetadataConverter extends MetadataConverter { + + Object read(RowData rowData); } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-debezium/src/main/java/io/debezium/relational/RelationalChangeRecordEmitter.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-debezium/src/main/java/io/debezium/relational/RelationalChangeRecordEmitter.java index 6bb18a75c84..2b0688a5dc4 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-debezium/src/main/java/io/debezium/relational/RelationalChangeRecordEmitter.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-debezium/src/main/java/io/debezium/relational/RelationalChangeRecordEmitter.java @@ -22,7 +22,7 @@ import java.util.Optional; /** - * Copied from Debezium 1.9.7.Final. + * Copied from Debezium 1.9.8.Final. * *

    Base class for {@link ChangeRecordEmitter} implementations based on a relational database. * diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/com/ververica/cdc/connectors/mongodb/MongoDBSource.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/com/ververica/cdc/connectors/mongodb/MongoDBSource.java index 6ea94e03772..d0448f4af19 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/com/ververica/cdc/connectors/mongodb/MongoDBSource.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/com/ververica/cdc/connectors/mongodb/MongoDBSource.java @@ -20,7 +20,6 @@ import com.mongodb.kafka.connect.source.MongoSourceConfig; import com.mongodb.kafka.connect.source.MongoSourceConfig.ErrorTolerance; import com.mongodb.kafka.connect.source.MongoSourceConfig.OutputFormat; -import com.ververica.cdc.common.annotation.PublicEvolving; import com.ververica.cdc.connectors.base.options.StartupOptions; import com.ververica.cdc.connectors.mongodb.internal.MongoDBConnectorSourceConnector; import com.ververica.cdc.debezium.DebeziumDeserializationSchema; @@ -55,7 +54,7 @@ * A builder to build a SourceFunction which can read snapshot and continue to consume change stream * events. */ -@PublicEvolving +@Deprecated public class MongoDBSource { public static final String FULL_DOCUMENT_UPDATE_LOOKUP = FullDocument.UPDATE_LOOKUP.getValue(); diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/com/ververica/cdc/connectors/mongodb/source/config/MongoDBSourceConfigFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/com/ververica/cdc/connectors/mongodb/source/config/MongoDBSourceConfigFactory.java index cc2c33c602e..2ca4ff16712 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/com/ververica/cdc/connectors/mongodb/source/config/MongoDBSourceConfigFactory.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/com/ververica/cdc/connectors/mongodb/source/config/MongoDBSourceConfigFactory.java @@ -164,6 +164,7 @@ public MongoDBSourceConfigFactory startupOptions(StartupOptions startupOptions) checkNotNull(startupOptions); switch (startupOptions.startupMode) { case INITIAL: + case SNAPSHOT: case LATEST_OFFSET: case TIMESTAMP: this.startupOptions = startupOptions; diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/com/ververica/cdc/connectors/mongodb/source/reader/fetch/MongoDBStreamFetchTask.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/com/ververica/cdc/connectors/mongodb/source/reader/fetch/MongoDBStreamFetchTask.java index d4db251944a..eebdc575ac9 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/com/ververica/cdc/connectors/mongodb/source/reader/fetch/MongoDBStreamFetchTask.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/com/ververica/cdc/connectors/mongodb/source/reader/fetch/MongoDBStreamFetchTask.java @@ -48,7 +48,6 @@ import org.slf4j.LoggerFactory; import java.time.Instant; -import java.util.Map; import java.util.Optional; import static com.ververica.cdc.connectors.mongodb.internal.MongoDBEnvelope.CLUSTER_TIME_FIELD; @@ -170,8 +169,7 @@ public void execute(Context context) throws Exception { LOG.info("Ignored {} record: {}", operationType, changeStreamDocument); } } - - if (changeRecord != null) { + if (changeRecord != null && !isBoundedRead()) { queue.enqueue(new DataChangeEvent(changeRecord)); } @@ -179,6 +177,11 @@ public void execute(Context context) throws Exception { ChangeStreamOffset currentOffset; if (changeRecord != null) { currentOffset = new ChangeStreamOffset(getResumeToken(changeRecord)); + // The log after the high watermark won't emit. + if (currentOffset.isAtOrBefore(streamSplit.getEndingOffset())) { + queue.enqueue(new DataChangeEvent(changeRecord)); + } + } else { // Heartbeat is not turned on or there is no update event currentOffset = new ChangeStreamOffset(getCurrentClusterTime(mongoClient)); @@ -222,13 +225,14 @@ public StreamSplit getSplit() { } @Override - public void close() {} + public void close() { + taskRunning = false; + } private MongoChangeStreamCursor openChangeStreamCursor( ChangeStreamDescriptor changeStreamDescriptor) { ChangeStreamOffset offset = - new ChangeStreamOffset( - (Map) streamSplit.getStartingOffset().getOffset()); + new ChangeStreamOffset(streamSplit.getStartingOffset().getOffset()); ChangeStreamIterable changeStreamIterable = getChangeStreamIterable(sourceConfig, changeStreamDescriptor); diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/com/ververica/cdc/connectors/mongodb/table/MongoDBTableSourceFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/com/ververica/cdc/connectors/mongodb/table/MongoDBTableSourceFactory.java index d4265bb937d..20e13b4eb07 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/com/ververica/cdc/connectors/mongodb/table/MongoDBTableSourceFactory.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/com/ververica/cdc/connectors/mongodb/table/MongoDBTableSourceFactory.java @@ -153,6 +153,8 @@ private void checkPrimaryKey(UniqueConstraint pk, String message) { } private static final String SCAN_STARTUP_MODE_VALUE_INITIAL = "initial"; + + private static final String SCAN_STARTUP_MODE_VALUE_SNAPSHOT = "snapshot"; private static final String SCAN_STARTUP_MODE_VALUE_LATEST = "latest-offset"; private static final String SCAN_STARTUP_MODE_VALUE_TIMESTAMP = "timestamp"; @@ -162,6 +164,8 @@ private static StartupOptions getStartupOptions(ReadableConfig config) { switch (modeString.toLowerCase()) { case SCAN_STARTUP_MODE_VALUE_INITIAL: return StartupOptions.initial(); + case SCAN_STARTUP_MODE_VALUE_SNAPSHOT: + return StartupOptions.snapshot(); case SCAN_STARTUP_MODE_VALUE_LATEST: return StartupOptions.latest(); case SCAN_STARTUP_MODE_VALUE_TIMESTAMP: @@ -174,9 +178,10 @@ private static StartupOptions getStartupOptions(ReadableConfig config) { default: throw new ValidationException( String.format( - "Invalid value for option '%s'. Supported values are [%s, %s, %s], but was: %s", + "Invalid value for option '%s'. Supported values are [%s, %s, %s, %s], but was: %s", SCAN_STARTUP_MODE.key(), SCAN_STARTUP_MODE_VALUE_INITIAL, + SCAN_STARTUP_MODE_VALUE_SNAPSHOT, SCAN_STARTUP_MODE_VALUE_LATEST, SCAN_STARTUP_MODE_VALUE_TIMESTAMP, modeString)); diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/com/ververica/cdc/connectors/mongodb/source/MongoDBFullChangelogITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/com/ververica/cdc/connectors/mongodb/source/MongoDBFullChangelogITCase.java index 86f936d6d06..02306b10358 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/com/ververica/cdc/connectors/mongodb/source/MongoDBFullChangelogITCase.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/com/ververica/cdc/connectors/mongodb/source/MongoDBFullChangelogITCase.java @@ -73,6 +73,7 @@ public class MongoDBFullChangelogITCase extends MongoDBSourceTestBase { private static final int USE_POST_LOWWATERMARK_HOOK = 1; private static final int USE_PRE_HIGHWATERMARK_HOOK = 2; + private static final int USE_POST_HIGHWATERMARK_HOOK = 3; @Rule public final Timeout timeoutPerTest = Timeout.seconds(300); @@ -218,6 +219,78 @@ public void testReadSingleTableWithSingleParallelismAndSkipBackfill() throws Exc true); } + @Test + public void testSnapshotOnlyModeWithDMLPostHighWaterMark() throws Exception { + if (!parallelismSnapshot) { + return; + } + // The data num is 21, set fetchSize = 22 to test whether the job is bounded. + List records = + testBackfillWhenWritingEvents( + false, 22, USE_POST_HIGHWATERMARK_HOOK, StartupOptions.snapshot()); + List expectedRecords = + Arrays.asList( + "+I[101, user_1, Shanghai, 123567891234]", + "+I[102, user_2, Shanghai, 123567891234]", + "+I[103, user_3, Shanghai, 123567891234]", + "+I[109, user_4, Shanghai, 123567891234]", + "+I[110, user_5, Shanghai, 123567891234]", + "+I[111, user_6, Shanghai, 123567891234]", + "+I[118, user_7, Shanghai, 123567891234]", + "+I[121, user_8, Shanghai, 123567891234]", + "+I[123, user_9, Shanghai, 123567891234]", + "+I[1009, user_10, Shanghai, 123567891234]", + "+I[1010, user_11, Shanghai, 123567891234]", + "+I[1011, user_12, Shanghai, 123567891234]", + "+I[1012, user_13, Shanghai, 123567891234]", + "+I[1013, user_14, Shanghai, 123567891234]", + "+I[1014, user_15, Shanghai, 123567891234]", + "+I[1015, user_16, Shanghai, 123567891234]", + "+I[1016, user_17, Shanghai, 123567891234]", + "+I[1017, user_18, Shanghai, 123567891234]", + "+I[1018, user_19, Shanghai, 123567891234]", + "+I[1019, user_20, Shanghai, 123567891234]", + "+I[2000, user_21, Shanghai, 123567891234]"); + assertEqualsInAnyOrder(expectedRecords, records); + } + + @Test + public void testSnapshotOnlyModeWithDMLPreHighWaterMark() throws Exception { + if (!parallelismSnapshot) { + return; + } + // The data num is 21, set fetchSize = 22 to test whether the job is bounded + List records = + testBackfillWhenWritingEvents( + false, 22, USE_PRE_HIGHWATERMARK_HOOK, StartupOptions.snapshot()); + List expectedRecords = + Arrays.asList( + "+I[101, user_1, Shanghai, 123567891234]", + "+I[102, user_2, Shanghai, 123567891234]", + "+I[103, user_3, Shanghai, 123567891234]", + "+I[109, user_4, Shanghai, 123567891234]", + "+I[110, user_5, Shanghai, 123567891234]", + "+I[111, user_6, Shanghai, 123567891234]", + "+I[118, user_7, Shanghai, 123567891234]", + "+I[121, user_8, Shanghai, 123567891234]", + "+I[123, user_9, Shanghai, 123567891234]", + "+I[1009, user_10, Shanghai, 123567891234]", + "+I[1010, user_11, Shanghai, 123567891234]", + "+I[1011, user_12, Shanghai, 123567891234]", + "+I[1012, user_13, Shanghai, 123567891234]", + "+I[1013, user_14, Shanghai, 123567891234]", + "+I[1014, user_15, Shanghai, 123567891234]", + "+I[1015, user_16, Shanghai, 123567891234]", + "+I[1016, user_17, Shanghai, 123567891234]", + "+I[1017, user_18, Shanghai, 123567891234]", + "+I[1018, user_19, Shanghai, 123567891234]", + "+I[2000, user_21, Pittsburgh, 123567891234]", + "+I[15213, user_15213, Shanghai, 123567891234]"); + // when enable backfill, the wal log between (snapshot, high_watermark) will be + // applied as snapshot image + assertEqualsInAnyOrder(expectedRecords, records); + } + @Test public void testEnableBackfillWithDMLPreHighWaterMark() throws Exception { if (!parallelismSnapshot) { @@ -225,7 +298,8 @@ public void testEnableBackfillWithDMLPreHighWaterMark() throws Exception { } List records = - testBackfillWhenWritingEvents(false, 21, USE_PRE_HIGHWATERMARK_HOOK, true); + testBackfillWhenWritingEvents( + false, 21, USE_PRE_HIGHWATERMARK_HOOK, StartupOptions.initial()); List expectedRecords = Arrays.asList( @@ -262,7 +336,8 @@ public void testEnableBackfillWithDMLPostLowWaterMark() throws Exception { } List records = - testBackfillWhenWritingEvents(false, 21, USE_POST_LOWWATERMARK_HOOK, true); + testBackfillWhenWritingEvents( + false, 21, USE_POST_LOWWATERMARK_HOOK, StartupOptions.initial()); List expectedRecords = Arrays.asList( @@ -292,6 +367,44 @@ public void testEnableBackfillWithDMLPostLowWaterMark() throws Exception { assertEqualsInAnyOrder(expectedRecords, records); } + @Test + public void testEnableBackfillWithDMLPostHighWaterMark() throws Exception { + if (!parallelismSnapshot) { + return; + } + List records = + testBackfillWhenWritingEvents( + false, 25, USE_POST_HIGHWATERMARK_HOOK, StartupOptions.initial()); + List expectedRecords = + Arrays.asList( + "+I[101, user_1, Shanghai, 123567891234]", + "+I[102, user_2, Shanghai, 123567891234]", + "+I[103, user_3, Shanghai, 123567891234]", + "+I[109, user_4, Shanghai, 123567891234]", + "+I[110, user_5, Shanghai, 123567891234]", + "+I[111, user_6, Shanghai, 123567891234]", + "+I[118, user_7, Shanghai, 123567891234]", + "+I[121, user_8, Shanghai, 123567891234]", + "+I[123, user_9, Shanghai, 123567891234]", + "+I[1009, user_10, Shanghai, 123567891234]", + "+I[1010, user_11, Shanghai, 123567891234]", + "+I[1011, user_12, Shanghai, 123567891234]", + "+I[1012, user_13, Shanghai, 123567891234]", + "+I[1013, user_14, Shanghai, 123567891234]", + "+I[1014, user_15, Shanghai, 123567891234]", + "+I[1015, user_16, Shanghai, 123567891234]", + "+I[1016, user_17, Shanghai, 123567891234]", + "+I[1017, user_18, Shanghai, 123567891234]", + "+I[1018, user_19, Shanghai, 123567891234]", + "+I[1019, user_20, Shanghai, 123567891234]", + "+I[2000, user_21, Shanghai, 123567891234]", + "+I[15213, user_15213, Shanghai, 123567891234]", + "-U[2000, user_21, Shanghai, 123567891234]", + "+U[2000, user_21, Pittsburgh, 123567891234]", + "-D[1019, user_20, Shanghai, 123567891234]"); + assertEqualsInAnyOrder(expectedRecords, records); + } + @Test public void testSkipBackfillWithDMLPreHighWaterMark() throws Exception { if (!parallelismSnapshot) { @@ -299,7 +412,8 @@ public void testSkipBackfillWithDMLPreHighWaterMark() throws Exception { } List records = - testBackfillWhenWritingEvents(true, 25, USE_PRE_HIGHWATERMARK_HOOK, true); + testBackfillWhenWritingEvents( + true, 25, USE_PRE_HIGHWATERMARK_HOOK, StartupOptions.initial()); List expectedRecords = Arrays.asList( @@ -340,7 +454,8 @@ public void testSkipBackfillWithDMLPostLowWaterMark() throws Exception { } List records = - testBackfillWhenWritingEvents(true, 25, USE_POST_LOWWATERMARK_HOOK, true); + testBackfillWhenWritingEvents( + true, 25, USE_POST_LOWWATERMARK_HOOK, StartupOptions.initial()); List expectedRecords = Arrays.asList( @@ -376,7 +491,7 @@ public void testSkipBackfillWithDMLPostLowWaterMark() throws Exception { } private List testBackfillWhenWritingEvents( - boolean skipBackFill, int fetchSize, int hookType, boolean enableFullDocPrePostImage) + boolean skipBackFill, int fetchSize, int hookType, StartupOptions startupOptions) throws Exception { String customerDatabase = @@ -398,7 +513,7 @@ private List testBackfillWhenWritingEvents( env.enableCheckpointing(1000); env.setParallelism(1); - ResolvedSchema customersSchame = + ResolvedSchema customersSchema = new ResolvedSchema( Arrays.asList( physical("cid", BIGINT().notNull()), @@ -407,19 +522,19 @@ private List testBackfillWhenWritingEvents( physical("phone_number", STRING())), new ArrayList<>(), UniqueConstraint.primaryKey("pk", Collections.singletonList("cid"))); - TestTable customerTable = new TestTable(customerDatabase, "customers", customersSchame); + TestTable customerTable = new TestTable(customerDatabase, "customers", customersSchema); MongoDBSource source = new MongoDBSourceBuilder() .hosts(CONTAINER.getHostAndPort()) .databaseList(customerDatabase) .username(FLINK_USER) .password(FLINK_USER_PASSWORD) - .startupOptions(StartupOptions.initial()) - .scanFullChangelog(enableFullDocPrePostImage) + .startupOptions(startupOptions) + .scanFullChangelog(true) .collectionList( getCollectionNameRegex( customerDatabase, new String[] {"customers"})) - .deserializer(customerTable.getDeserializer(enableFullDocPrePostImage)) + .deserializer(customerTable.getDeserializer(true)) .skipSnapshotBackfill(skipBackFill) .build(); @@ -443,10 +558,16 @@ private List testBackfillWhenWritingEvents( mongoCollection.deleteOne(Filters.eq("cid", 1019L)); }; - if (hookType == USE_POST_LOWWATERMARK_HOOK) { - hooks.setPostLowWatermarkAction(snapshotPhaseHook); - } else if (hookType == USE_PRE_HIGHWATERMARK_HOOK) { - hooks.setPreHighWatermarkAction(snapshotPhaseHook); + switch (hookType) { + case USE_POST_LOWWATERMARK_HOOK: + hooks.setPostLowWatermarkAction(snapshotPhaseHook); + break; + case USE_PRE_HIGHWATERMARK_HOOK: + hooks.setPreHighWatermarkAction(snapshotPhaseHook); + break; + case USE_POST_HIGHWATERMARK_HOOK: + hooks.setPostHighWatermarkAction(snapshotPhaseHook); + break; } source.setSnapshotHooks(hooks); diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/com/ververica/cdc/connectors/mongodb/source/MongoDBParallelSourceITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/com/ververica/cdc/connectors/mongodb/source/MongoDBParallelSourceITCase.java index 9ac86fbe40f..a7b52be5290 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/com/ververica/cdc/connectors/mongodb/source/MongoDBParallelSourceITCase.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/test/java/com/ververica/cdc/connectors/mongodb/source/MongoDBParallelSourceITCase.java @@ -67,6 +67,7 @@ public class MongoDBParallelSourceITCase extends MongoDBSourceTestBase { private static final int USE_POST_LOWWATERMARK_HOOK = 1; private static final int USE_PRE_HIGHWATERMARK_HOOK = 2; + private static final int USE_POST_HIGHWATERMARK_HOOK = 3; @Rule public final Timeout timeoutPerTest = Timeout.seconds(300); @@ -147,11 +148,78 @@ public void testReadSingleTableWithSingleParallelismAndSkipBackfill() throws Exc true); } + @Test + public void testSnapshotOnlyModeWithDMLPostHighWaterMark() throws Exception { + // The data num is 21, set fetchSize = 22 to test whether the job is bounded. + List records = + testBackfillWhenWritingEvents( + false, 22, USE_POST_HIGHWATERMARK_HOOK, StartupOptions.snapshot()); + List expectedRecords = + Arrays.asList( + "+I[101, user_1, Shanghai, 123567891234]", + "+I[102, user_2, Shanghai, 123567891234]", + "+I[103, user_3, Shanghai, 123567891234]", + "+I[109, user_4, Shanghai, 123567891234]", + "+I[110, user_5, Shanghai, 123567891234]", + "+I[111, user_6, Shanghai, 123567891234]", + "+I[118, user_7, Shanghai, 123567891234]", + "+I[121, user_8, Shanghai, 123567891234]", + "+I[123, user_9, Shanghai, 123567891234]", + "+I[1009, user_10, Shanghai, 123567891234]", + "+I[1010, user_11, Shanghai, 123567891234]", + "+I[1011, user_12, Shanghai, 123567891234]", + "+I[1012, user_13, Shanghai, 123567891234]", + "+I[1013, user_14, Shanghai, 123567891234]", + "+I[1014, user_15, Shanghai, 123567891234]", + "+I[1015, user_16, Shanghai, 123567891234]", + "+I[1016, user_17, Shanghai, 123567891234]", + "+I[1017, user_18, Shanghai, 123567891234]", + "+I[1018, user_19, Shanghai, 123567891234]", + "+I[1019, user_20, Shanghai, 123567891234]", + "+I[2000, user_21, Shanghai, 123567891234]"); + assertEqualsInAnyOrder(expectedRecords, records); + } + + @Test + public void testSnapshotOnlyModeWithDMLPreHighWaterMark() throws Exception { + // The data num is 21, set fetchSize = 22 to test whether the job is bounded + List records = + testBackfillWhenWritingEvents( + false, 22, USE_PRE_HIGHWATERMARK_HOOK, StartupOptions.snapshot()); + List expectedRecords = + Arrays.asList( + "+I[101, user_1, Shanghai, 123567891234]", + "+I[102, user_2, Shanghai, 123567891234]", + "+I[103, user_3, Shanghai, 123567891234]", + "+I[109, user_4, Shanghai, 123567891234]", + "+I[110, user_5, Shanghai, 123567891234]", + "+I[111, user_6, Shanghai, 123567891234]", + "+I[118, user_7, Shanghai, 123567891234]", + "+I[121, user_8, Shanghai, 123567891234]", + "+I[123, user_9, Shanghai, 123567891234]", + "+I[1009, user_10, Shanghai, 123567891234]", + "+I[1010, user_11, Shanghai, 123567891234]", + "+I[1011, user_12, Shanghai, 123567891234]", + "+I[1012, user_13, Shanghai, 123567891234]", + "+I[1013, user_14, Shanghai, 123567891234]", + "+I[1014, user_15, Shanghai, 123567891234]", + "+I[1015, user_16, Shanghai, 123567891234]", + "+I[1016, user_17, Shanghai, 123567891234]", + "+I[1017, user_18, Shanghai, 123567891234]", + "+I[1018, user_19, Shanghai, 123567891234]", + "+I[2000, user_21, Pittsburgh, 123567891234]", + "+I[15213, user_15213, Shanghai, 123567891234]"); + // when enable backfill, the wal log between (snapshot, high_watermark) will be + // applied as snapshot image + assertEqualsInAnyOrder(expectedRecords, records); + } + @Test public void testEnableBackfillWithDMLPreHighWaterMark() throws Exception { List records = - testBackfillWhenWritingEvents(false, 21, USE_PRE_HIGHWATERMARK_HOOK, false); + testBackfillWhenWritingEvents( + false, 21, USE_PRE_HIGHWATERMARK_HOOK, StartupOptions.initial()); List expectedRecords = Arrays.asList( @@ -185,7 +253,8 @@ public void testEnableBackfillWithDMLPreHighWaterMark() throws Exception { public void testEnableBackfillWithDMLPostLowWaterMark() throws Exception { List records = - testBackfillWhenWritingEvents(false, 21, USE_POST_LOWWATERMARK_HOOK, false); + testBackfillWhenWritingEvents( + false, 21, USE_POST_LOWWATERMARK_HOOK, StartupOptions.initial()); List expectedRecords = Arrays.asList( @@ -215,11 +284,49 @@ public void testEnableBackfillWithDMLPostLowWaterMark() throws Exception { assertEqualsInAnyOrder(expectedRecords, records); } + @Test + public void testEnableBackfillWithDMLPostHighWaterMark() throws Exception { + + List records = + testBackfillWhenWritingEvents( + false, 24, USE_POST_HIGHWATERMARK_HOOK, StartupOptions.initial()); + List expectedRecords = + Arrays.asList( + "+I[101, user_1, Shanghai, 123567891234]", + "+I[102, user_2, Shanghai, 123567891234]", + "+I[103, user_3, Shanghai, 123567891234]", + "+I[109, user_4, Shanghai, 123567891234]", + "+I[110, user_5, Shanghai, 123567891234]", + "+I[111, user_6, Shanghai, 123567891234]", + "+I[118, user_7, Shanghai, 123567891234]", + "+I[121, user_8, Shanghai, 123567891234]", + "+I[123, user_9, Shanghai, 123567891234]", + "+I[1009, user_10, Shanghai, 123567891234]", + "+I[1010, user_11, Shanghai, 123567891234]", + "+I[1011, user_12, Shanghai, 123567891234]", + "+I[1012, user_13, Shanghai, 123567891234]", + "+I[1013, user_14, Shanghai, 123567891234]", + "+I[1014, user_15, Shanghai, 123567891234]", + "+I[1015, user_16, Shanghai, 123567891234]", + "+I[1016, user_17, Shanghai, 123567891234]", + "+I[1017, user_18, Shanghai, 123567891234]", + "+I[1018, user_19, Shanghai, 123567891234]", + "+I[1019, user_20, Shanghai, 123567891234]", + "+I[2000, user_21, Shanghai, 123567891234]", + "+I[15213, user_15213, Shanghai, 123567891234]", + "+U[2000, user_21, Pittsburgh, 123567891234]", + // delete message only contains _id, sql job contain value because of + // changelog normalization + "-D[0, null, null, null]"); + assertEqualsInAnyOrder(expectedRecords, records); + } + @Test public void testSkipBackfillWithDMLPreHighWaterMark() throws Exception { List records = - testBackfillWhenWritingEvents(true, 24, USE_PRE_HIGHWATERMARK_HOOK, false); + testBackfillWhenWritingEvents( + true, 24, USE_PRE_HIGHWATERMARK_HOOK, StartupOptions.initial()); List expectedRecords = Arrays.asList( @@ -258,7 +365,8 @@ public void testSkipBackfillWithDMLPreHighWaterMark() throws Exception { public void testSkipBackfillWithDMLPostLowWaterMark() throws Exception { List records = - testBackfillWhenWritingEvents(true, 24, USE_POST_LOWWATERMARK_HOOK, false); + testBackfillWhenWritingEvents( + true, 24, USE_POST_LOWWATERMARK_HOOK, StartupOptions.initial()); List expectedRecords = Arrays.asList( @@ -295,14 +403,14 @@ public void testSkipBackfillWithDMLPostLowWaterMark() throws Exception { } private List testBackfillWhenWritingEvents( - boolean skipBackFill, int fetchSize, int hookType, boolean enableFullDocPrePostImage) + boolean skipBackFill, int fetchSize, int hookType, StartupOptions startupOptions) throws Exception { String customerDatabase = CONTAINER.executeCommandFileInSeparateDatabase("customer"); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.enableCheckpointing(1000); env.setParallelism(1); - ResolvedSchema customersSchame = + ResolvedSchema customersSchema = new ResolvedSchema( Arrays.asList( physical("cid", BIGINT().notNull()), @@ -311,7 +419,7 @@ private List testBackfillWhenWritingEvents( physical("phone_number", STRING())), new ArrayList<>(), UniqueConstraint.primaryKey("pk", Collections.singletonList("cid"))); - TestTable customerTable = new TestTable(customerDatabase, "customers", customersSchame); + TestTable customerTable = new TestTable(customerDatabase, "customers", customersSchema); MongoDBSource source = new MongoDBSourceBuilder() .hosts(CONTAINER.getHostAndPort()) @@ -319,12 +427,13 @@ private List testBackfillWhenWritingEvents( .username(FLINK_USER) .password(FLINK_USER_PASSWORD) .startupOptions(StartupOptions.initial()) - .scanFullChangelog(enableFullDocPrePostImage) + .scanFullChangelog(false) .collectionList( getCollectionNameRegex( customerDatabase, new String[] {"customers"})) - .deserializer(customerTable.getDeserializer(enableFullDocPrePostImage)) + .deserializer(customerTable.getDeserializer(false)) .skipSnapshotBackfill(skipBackFill) + .startupOptions(startupOptions) .build(); // Do some database operations during hook in snapshot phase. @@ -345,17 +454,18 @@ private List testBackfillWhenWritingEvents( mongoCollection.updateOne( Filters.eq("cid", 2000L), Updates.set("address", "Pittsburgh")); mongoCollection.deleteOne(Filters.eq("cid", 1019L)); - try { - Thread.sleep(500L); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } }; - if (hookType == USE_POST_LOWWATERMARK_HOOK) { - hooks.setPostLowWatermarkAction(snapshotPhaseHook); - } else if (hookType == USE_PRE_HIGHWATERMARK_HOOK) { - hooks.setPreHighWatermarkAction(snapshotPhaseHook); + switch (hookType) { + case USE_POST_LOWWATERMARK_HOOK: + hooks.setPostLowWatermarkAction(snapshotPhaseHook); + break; + case USE_PRE_HIGHWATERMARK_HOOK: + hooks.setPreHighWatermarkAction(snapshotPhaseHook); + break; + case USE_POST_HIGHWATERMARK_HOOK: + hooks.setPostHighWatermarkAction(snapshotPhaseHook); + break; } source.setSnapshotHooks(hooks); diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/DebeziumUtils.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/DebeziumUtils.java index 80d98e0726f..1b9d87f5fea 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/DebeziumUtils.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/DebeziumUtils.java @@ -19,6 +19,9 @@ import org.apache.flink.util.FlinkRuntimeException; import com.github.shyiko.mysql.binlog.BinaryLogClient; +import com.github.shyiko.mysql.binlog.event.EventData; +import com.github.shyiko.mysql.binlog.event.EventHeaderV4; +import com.github.shyiko.mysql.binlog.event.RotateEventData; import com.ververica.cdc.connectors.mysql.source.config.MySqlSourceConfig; import com.ververica.cdc.connectors.mysql.source.connection.JdbcConnectionFactory; import com.ververica.cdc.connectors.mysql.source.offset.BinlogOffset; @@ -41,11 +44,14 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; import java.sql.SQLException; +import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.concurrent.ArrayBlockingQueue; import java.util.function.Predicate; import static com.ververica.cdc.connectors.mysql.source.utils.TableDiscoveryUtils.listTables; @@ -233,4 +239,96 @@ private static Map querySystemVariables( return variables; } + + public static BinlogOffset findBinlogOffset(long targetMs, MySqlConnection connection) { + MySqlConnection.MySqlConnectionConfiguration config = connection.connectionConfig(); + BinaryLogClient client = + new BinaryLogClient( + config.hostname(), config.port(), config.username(), config.password()); + + List binlogFiles = new ArrayList<>(); + JdbcConnection.ResultSetConsumer rsc = + rs -> { + while (rs.next()) { + String fileName = rs.getString(1); + long fileSize = rs.getLong(2); + if (fileSize > 0) { + binlogFiles.add(fileName); + } + } + }; + + try { + connection.query("SHOW BINARY LOGS", rsc); + LOG.info("Total search binlog: {}", binlogFiles); + + if (binlogFiles.isEmpty()) { + return BinlogOffset.ofBinlogFilePosition("", 0); + } + + String binlogName = searchBinlogName(client, targetMs, binlogFiles); + return BinlogOffset.ofBinlogFilePosition(binlogName, 0); + } catch (Exception e) { + throw new FlinkRuntimeException(e); + } + } + + private static String searchBinlogName( + BinaryLogClient client, long targetMs, List binlogFiles) + throws IOException, InterruptedException { + int startIdx = 0; + int endIdx = binlogFiles.size() - 1; + + while (startIdx <= endIdx) { + int mid = startIdx + (endIdx - startIdx) / 2; + long midTs = getBinlogTimestamp(client, binlogFiles.get(mid)); + if (midTs < targetMs) { + startIdx = mid + 1; + } else if (targetMs < midTs) { + endIdx = mid - 1; + } else { + return binlogFiles.get(mid); + } + } + + return endIdx < 0 ? binlogFiles.get(0) : binlogFiles.get(endIdx); + } + + private static long getBinlogTimestamp(BinaryLogClient client, String binlogFile) + throws IOException, InterruptedException { + + ArrayBlockingQueue binlogTimestamps = new ArrayBlockingQueue<>(1); + BinaryLogClient.EventListener eventListener = + event -> { + EventData data = event.getData(); + if (data instanceof RotateEventData) { + // We skip RotateEventData because it does not contain the timestamp we are + // interested in. + return; + } + + EventHeaderV4 header = event.getHeader(); + long timestamp = header.getTimestamp(); + if (timestamp > 0) { + binlogTimestamps.offer(timestamp); + try { + client.disconnect(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + }; + + try { + client.registerEventListener(eventListener); + client.setBinlogFilename(binlogFile); + client.setBinlogPosition(0); + + LOG.info("begin parse binlog: {}", binlogFile); + client.connect(); + } finally { + client.unregisterEventListener(eventListener); + } + return binlogTimestamps.take(); + } } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/reader/BinlogSplitReader.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/reader/BinlogSplitReader.java index f8fc488fef4..c9c95d67f70 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/reader/BinlogSplitReader.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/reader/BinlogSplitReader.java @@ -37,7 +37,6 @@ import io.debezium.connector.base.ChangeEventQueue; import io.debezium.connector.mysql.MySqlStreamingChangeEventSourceMetrics; import io.debezium.pipeline.DataChangeEvent; -import io.debezium.pipeline.source.spi.ChangeEventSource; import io.debezium.relational.TableId; import io.debezium.relational.Tables; import org.apache.kafka.connect.data.Struct; @@ -65,6 +64,8 @@ import static com.ververica.cdc.connectors.mysql.source.utils.RecordUtils.getStructContainsChunkKey; import static com.ververica.cdc.connectors.mysql.source.utils.RecordUtils.getTableId; import static com.ververica.cdc.connectors.mysql.source.utils.RecordUtils.isDataChangeRecord; +import static com.ververica.cdc.connectors.mysql.source.utils.RecordUtils.isSchemaChangeEvent; +import static com.ververica.cdc.connectors.mysql.source.utils.RecordUtils.isTableChangeRecord; /** * A Debezium binlog reader implementation that also support reads binlog and filter overlapping @@ -87,6 +88,8 @@ public class BinlogSplitReader implements DebeziumReader maxSplitHighWatermarkMap; private final Set pureBinlogPhaseTables; private Tables.TableFilter capturedTableFilter; + private final StoppableChangeEventSourceContext changeEventSourceContext = + new StoppableChangeEventSourceContext(); private static final long READER_CLOSE_TIMEOUT = 30L; @@ -124,29 +127,22 @@ public void submitSplit(MySqlSplit mySqlSplit) { () -> { try { binlogSplitReadTask.execute( - new BinlogSplitChangeEventSourceContextImpl(), + changeEventSourceContext, statefulTaskContext.getMySqlPartition(), statefulTaskContext.getOffsetContext()); } catch (Exception e) { - currentTaskRunning = false; LOG.error( String.format( "Execute binlog read task for mysql split %s fail", currentBinlogSplit), e); readException = e; + } finally { + stopBinlogReadTask(); } }); } - private class BinlogSplitChangeEventSourceContextImpl - implements ChangeEventSource.ChangeEventSourceContext { - @Override - public boolean isRunning() { - return currentTaskRunning; - } - } - @Override public boolean isFinished() { return currentBinlogSplit == null || !currentTaskRunning; @@ -191,9 +187,8 @@ public void close() { if (statefulTaskContext.getBinaryLogClient() != null) { statefulTaskContext.getBinaryLogClient().disconnect(); } - // set currentTaskRunning to false to terminate the - // while loop in MySqlStreamingChangeEventSource's execute method - currentTaskRunning = false; + + stopBinlogReadTask(); if (executorService != null) { executorService.shutdown(); if (!executorService.awaitTermination(READER_CLOSE_TIMEOUT, TimeUnit.SECONDS)) { @@ -252,9 +247,15 @@ private boolean shouldEmit(SourceRecord sourceRecord) { } // not in the monitored splits scope, do not emit return false; + } else if (isSchemaChangeEvent(sourceRecord)) { + if (isTableChangeRecord(sourceRecord)) { + TableId tableId = getTableId(sourceRecord); + return capturedTableFilter.isIncluded(tableId); + } else { + // Not related to changes in table structure, like `CREATE/DROP DATABASE`, skip it + return false; + } } - // always send the schema change event and signal event - // we need record them to state of Flink return true; } @@ -328,7 +329,9 @@ private Predicate createEventFilter(BinlogOffset startingOffset) { } public void stopBinlogReadTask() { - this.currentTaskRunning = false; + currentTaskRunning = false; + // Terminate the while loop in MySqlStreamingChangeEventSource's execute method + changeEventSourceContext.stopChangeEventSource(); } @VisibleForTesting diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/reader/SnapshotSplitReader.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/reader/SnapshotSplitReader.java index 70e37f010b5..a34bd8691dd 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/reader/SnapshotSplitReader.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/debezium/reader/SnapshotSplitReader.java @@ -92,6 +92,9 @@ public class SnapshotSplitReader implements DebeziumReader createEnumerator( validator.validate(); final MySqlSplitAssigner splitAssigner; - if (sourceConfig.getStartupOptions().startupMode == StartupMode.INITIAL) { + if (!sourceConfig.getStartupOptions().isStreamOnly()) { try (JdbcConnection jdbc = openJdbcConnection(sourceConfig)) { boolean isTableIdCaseSensitive = DebeziumUtils.isTableIdCaseSensitive(jdbc); splitAssigner = @@ -214,7 +218,8 @@ public SplitEnumerator createEnumerator( splitAssigner = new MySqlBinlogSplitAssigner(sourceConfig); } - return new MySqlSourceEnumerator(enumContext, sourceConfig, splitAssigner); + return new MySqlSourceEnumerator( + enumContext, sourceConfig, splitAssigner, getBoundedness()); } @Override @@ -238,7 +243,8 @@ public SplitEnumerator restoreEnumerator( throw new UnsupportedOperationException( "Unsupported restored PendingSplitsState: " + checkpoint); } - return new MySqlSourceEnumerator(enumContext, sourceConfig, splitAssigner); + return new MySqlSourceEnumerator( + enumContext, sourceConfig, splitAssigner, getBoundedness()); } @Override diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/assigners/MySqlBinlogSplitAssigner.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/assigners/MySqlBinlogSplitAssigner.java index ca4eedf61bb..d3b6ad3c634 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/assigners/MySqlBinlogSplitAssigner.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/assigners/MySqlBinlogSplitAssigner.java @@ -110,7 +110,7 @@ public AssignerStatus getAssignerStatus() { } @Override - public boolean isStreamSplitAssigned() { + public boolean noMoreSplits() { return isBinlogSplitAssigned; } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/assigners/MySqlHybridSplitAssigner.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/assigners/MySqlHybridSplitAssigner.java index 4c7b26eba42..e100c97f675 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/assigners/MySqlHybridSplitAssigner.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/assigners/MySqlHybridSplitAssigner.java @@ -51,6 +51,7 @@ public class MySqlHybridSplitAssigner implements MySqlSplitAssigner { private static final String BINLOG_SPLIT_ID = "binlog-split"; private final int splitMetaGroupSize; + private final MySqlSourceConfig sourceConfig; private boolean isBinlogSplitAssigned; @@ -62,6 +63,7 @@ public MySqlHybridSplitAssigner( List remainingTables, boolean isTableIdCaseSensitive) { this( + sourceConfig, new MySqlSnapshotSplitAssigner( sourceConfig, currentParallelism, remainingTables, isTableIdCaseSensitive), false, @@ -73,6 +75,7 @@ public MySqlHybridSplitAssigner( int currentParallelism, HybridPendingSplitsState checkpoint) { this( + sourceConfig, new MySqlSnapshotSplitAssigner( sourceConfig, currentParallelism, checkpoint.getSnapshotPendingSplits()), checkpoint.isBinlogSplitAssigned(), @@ -80,9 +83,11 @@ public MySqlHybridSplitAssigner( } private MySqlHybridSplitAssigner( + MySqlSourceConfig sourceConfig, MySqlSnapshotSplitAssigner snapshotSplitAssigner, boolean isBinlogSplitAssigned, int splitMetaGroupSize) { + this.sourceConfig = sourceConfig; this.snapshotSplitAssigner = snapshotSplitAssigner; this.isBinlogSplitAssigned = isBinlogSplitAssigned; this.splitMetaGroupSize = splitMetaGroupSize; @@ -99,7 +104,7 @@ public Optional getNext() { // do not assign split until the adding table process finished return Optional.empty(); } - if (snapshotSplitAssigner.noMoreSnapshotSplits()) { + if (snapshotSplitAssigner.noMoreSplits()) { // binlog split assigning if (isBinlogSplitAssigned) { // no more splits for the assigner @@ -129,11 +134,6 @@ public boolean waitingForFinishedSplits() { return snapshotSplitAssigner.waitingForFinishedSplits(); } - @Override - public boolean isStreamSplitAssigned() { - return isBinlogSplitAssigned; - } - @Override public List getFinishedSplitInfos() { return snapshotSplitAssigner.getFinishedSplitInfos(); @@ -174,6 +174,11 @@ public AssignerStatus getAssignerStatus() { return snapshotSplitAssigner.getAssignerStatus(); } + @Override + public boolean noMoreSplits() { + return snapshotSplitAssigner.noMoreSplits() && isBinlogSplitAssigned; + } + @Override public void startAssignNewlyAddedTables() { snapshotSplitAssigner.startAssignNewlyAddedTables(); @@ -189,10 +194,6 @@ public void close() { snapshotSplitAssigner.close(); } - public boolean noMoreSnapshotSplits() { - return snapshotSplitAssigner.noMoreSnapshotSplits(); - } - // -------------------------------------------------------------------------------------------- private MySqlBinlogSplit createBinlogSplit() { @@ -206,12 +207,17 @@ private MySqlBinlogSplit createBinlogSplit() { final List finishedSnapshotSplitInfos = new ArrayList<>(); BinlogOffset minBinlogOffset = null; + BinlogOffset maxBinlogOffset = null; for (MySqlSchemalessSnapshotSplit split : assignedSnapshotSplit) { - // find the min binlog offset + // find the min and max binlog offset BinlogOffset binlogOffset = splitFinishedOffsets.get(split.splitId()); if (minBinlogOffset == null || binlogOffset.isBefore(minBinlogOffset)) { minBinlogOffset = binlogOffset; } + if (maxBinlogOffset == null || binlogOffset.isAfter(maxBinlogOffset)) { + maxBinlogOffset = binlogOffset; + } + finishedSnapshotSplitInfos.add( new FinishedSnapshotSplitInfo( split.getTableId(), @@ -221,14 +227,21 @@ private MySqlBinlogSplit createBinlogSplit() { binlogOffset)); } + // If the source is running in snapshot mode, we use the highest watermark among + // snapshot splits as the ending offset to provide a consistent snapshot view at the moment + // of high watermark. + BinlogOffset stoppingOffset = BinlogOffset.ofNonStopping(); + if (sourceConfig.getStartupOptions().isSnapshotOnly()) { + stoppingOffset = maxBinlogOffset; + } + // the finishedSnapshotSplitInfos is too large for transmission, divide it to groups and // then transfer them - boolean divideMetaToGroups = finishedSnapshotSplitInfos.size() > splitMetaGroupSize; return new MySqlBinlogSplit( BINLOG_SPLIT_ID, minBinlogOffset == null ? BinlogOffset.ofEarliest() : minBinlogOffset, - BinlogOffset.ofNonStopping(), + stoppingOffset, divideMetaToGroups ? new ArrayList<>() : finishedSnapshotSplitInfos, new HashMap<>(), finishedSnapshotSplitInfos.size()); diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/assigners/MySqlSnapshotSplitAssigner.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/assigners/MySqlSnapshotSplitAssigner.java index 0536a2621fe..8753f8aebed 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/assigners/MySqlSnapshotSplitAssigner.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/assigners/MySqlSnapshotSplitAssigner.java @@ -216,7 +216,9 @@ else if (!isRemainingTablesCheckpointed && !isSnapshotAssigningFinished(assigner } private void captureNewlyAddedTables() { - if (sourceConfig.isScanNewlyAddedTableEnabled()) { + // Don't scan newly added table in snapshot mode. + if (sourceConfig.isScanNewlyAddedTableEnabled() + || !sourceConfig.getStartupOptions().isSnapshotOnly()) { // check whether we got newly added tables try (JdbcConnection jdbc = openJdbcConnection(sourceConfig)) { final List currentCapturedTables = @@ -517,7 +519,7 @@ private void waitTableDiscoveryReady() { } /** Indicates there is no more splits available in this assigner. */ - public boolean noMoreSnapshotSplits() { + public boolean noMoreSplits() { return !needToDiscoveryTables() && remainingTables.isEmpty() && remainingSplits.isEmpty(); } @@ -547,7 +549,7 @@ public Map getSplitFinishedOffsets() { * are finished. */ private boolean allSnapshotSplitsFinished() { - return noMoreSnapshotSplits() && assignedSplits.size() == splitFinishedOffsets.size(); + return noMoreSplits() && assignedSplits.size() == splitFinishedOffsets.size(); } private void splitChunksForRemainingTables() { diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/assigners/MySqlSplitAssigner.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/assigners/MySqlSplitAssigner.java index 52dfcd325c9..57f003eb95b 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/assigners/MySqlSplitAssigner.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/assigners/MySqlSplitAssigner.java @@ -18,6 +18,7 @@ import org.apache.flink.api.common.state.CheckpointListener; +import com.ververica.cdc.common.annotation.Internal; import com.ververica.cdc.connectors.mysql.source.assigners.state.PendingSplitsState; import com.ververica.cdc.connectors.mysql.source.offset.BinlogOffset; import com.ververica.cdc.connectors.mysql.source.split.FinishedSnapshotSplitInfo; @@ -32,6 +33,7 @@ * The {@code MySqlSplitAssigner} is responsible for deciding what split should be processed. It * determines split processing order. */ +@Internal public interface MySqlSplitAssigner { /** @@ -53,11 +55,6 @@ public interface MySqlSplitAssigner { */ boolean waitingForFinishedSplits(); - /** Whether the split assigner is finished stream split assigning. */ - default boolean isStreamSplitAssigned() { - throw new UnsupportedOperationException("Not support to assigning StreamSplit."); - } - /** * Gets the finished splits' information. This is useful metadata to generate a binlog split * that considering finished snapshot splits. @@ -110,6 +107,9 @@ default boolean isStreamSplitAssigned() { /** Starts assign newly added tables. */ void startAssignNewlyAddedTables(); + /** Indicates there is no more splits available in this assigner. */ + boolean noMoreSplits(); + /** * Callback to handle the binlog split has been updated in the newly added tables process. This * is useful to check the newly added tables has been finished or not. diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/enumerator/MySqlSourceEnumerator.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/enumerator/MySqlSourceEnumerator.java index 3b46bd521c8..f475ed60b80 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/enumerator/MySqlSourceEnumerator.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/enumerator/MySqlSourceEnumerator.java @@ -16,6 +16,7 @@ package com.ververica.cdc.connectors.mysql.source.enumerator; +import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.connector.source.SourceEvent; import org.apache.flink.api.connector.source.SplitEnumerator; import org.apache.flink.api.connector.source.SplitEnumeratorContext; @@ -24,7 +25,6 @@ import org.apache.flink.shaded.guava31.com.google.common.collect.Lists; import com.ververica.cdc.common.annotation.Internal; -import com.ververica.cdc.connectors.mysql.source.assigners.MySqlBinlogSplitAssigner; import com.ververica.cdc.connectors.mysql.source.assigners.MySqlHybridSplitAssigner; import com.ververica.cdc.connectors.mysql.source.assigners.MySqlSplitAssigner; import com.ververica.cdc.connectors.mysql.source.assigners.state.PendingSplitsState; @@ -71,6 +71,8 @@ public class MySqlSourceEnumerator implements SplitEnumerator readersAwaitingSplit; private List> binlogSplitMeta; @@ -80,10 +82,12 @@ public class MySqlSourceEnumerator implements SplitEnumerator context, MySqlSourceConfig sourceConfig, - MySqlSplitAssigner splitAssigner) { + MySqlSplitAssigner splitAssigner, + Boundedness boundedness) { this.context = context; this.sourceConfig = sourceConfig; this.splitAssigner = splitAssigner; + this.boundedness = boundedness; this.readersAwaitingSplit = new TreeSet<>(); } @@ -204,10 +208,7 @@ private void assignSplits() { continue; } - if (splitAssigner.isStreamSplitAssigned() - && sourceConfig.isCloseIdleReaders() - && noMoreSnapshotSplits() - && (binlogSplitTaskId != null && !binlogSplitTaskId.equals(nextAwaiting))) { + if (shouldCloseIdleReader(nextAwaiting)) { // close idle readers when snapshot phase finished. context.signalNoMoreSplits(nextAwaiting); awaitingReader.remove(); @@ -232,14 +233,18 @@ && noMoreSnapshotSplits() } } - private boolean noMoreSnapshotSplits() { - if (splitAssigner instanceof MySqlHybridSplitAssigner) { - return ((MySqlHybridSplitAssigner) splitAssigner).noMoreSnapshotSplits(); - } else if (splitAssigner instanceof MySqlBinlogSplitAssigner) { - return true; - } - throw new IllegalStateException( - "Unexpected subtype of MySqlSplitAssigner class when invoking noMoreSnapshotSplits."); + private boolean shouldCloseIdleReader(int nextAwaiting) { + // When no unassigned split anymore, Signal NoMoreSplitsEvent to awaiting reader in two + // situations: + // 1. When Set StartupMode = snapshot mode(also bounded), there's no more splits in the + // assigner. + // 2. When set scan.incremental.close-idle-reader.enabled = true, there's no more splits in + // the assigner. + return splitAssigner.noMoreSplits() + && (boundedness == Boundedness.BOUNDED + || (sourceConfig.isCloseIdleReaders() + && (binlogSplitTaskId != null + && !binlogSplitTaskId.equals(nextAwaiting)))); } private int[] getRegisteredReader() { diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/offset/BinlogOffsetUtils.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/offset/BinlogOffsetUtils.java index 560b9964cfc..b2dff89a46f 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/offset/BinlogOffsetUtils.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/offset/BinlogOffsetUtils.java @@ -47,8 +47,9 @@ public static BinlogOffset initializeEffectiveOffset( BinlogOffsetKind offsetKind = offset.getOffsetKind(); switch (offsetKind) { case EARLIEST: - case TIMESTAMP: return BinlogOffset.ofBinlogFilePosition("", 0); + case TIMESTAMP: + return DebeziumUtils.findBinlogOffset(offset.getTimestampSec() * 1000, connection); case LATEST: return DebeziumUtils.currentBinlogOffset(connection); default: diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/reader/MySqlSourceReader.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/reader/MySqlSourceReader.java index f42bf2ab0b0..e38ebed5fa9 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/reader/MySqlSourceReader.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/reader/MySqlSourceReader.java @@ -176,15 +176,26 @@ protected void onSplitFinished(Map finishedSplitIds) { for (MySqlSplitState mySqlSplitState : finishedSplitIds.values()) { MySqlSplit mySqlSplit = mySqlSplitState.toMySqlSplit(); if (mySqlSplit.isBinlogSplit()) { - suspendedBinlogSplit = toSuspendedBinlogSplit(mySqlSplit.asBinlogSplit()); - LOG.info( - "Source reader {} suspended binlog split reader success after the newly added table process, current offset {}", - subtaskId, - suspendedBinlogSplit.getStartingOffset()); - context.sendSourceEventToCoordinator( - new LatestFinishedSplitsNumberRequestEvent()); - // do not request next split when the reader is suspended - requestNextSplit = false; + // Two possibilities that finish a binlog split: + // + // 1. Binlog reader is suspended by enumerator because new tables have been + // finished its snapshot reading. + // Under this case mySqlSourceReaderContext.isBinlogSplitReaderSuspended() is + // true and need to request the latest finished splits number. + // + // 2. Binlog reader reaches the ending offset of the split. We need to do + // nothing under this case. + if (mySqlSourceReaderContext.isBinlogSplitReaderSuspended()) { + suspendedBinlogSplit = toSuspendedBinlogSplit(mySqlSplit.asBinlogSplit()); + LOG.info( + "Source reader {} suspended binlog split reader success after the newly added table process, current offset {}", + subtaskId, + suspendedBinlogSplit.getStartingOffset()); + context.sendSourceEventToCoordinator( + new LatestFinishedSplitsNumberRequestEvent()); + // do not request next split when the reader is suspended + requestNextSplit = false; + } } else { finishedUnackedSplits.put(mySqlSplit.splitId(), mySqlSplit.asSnapshotSplit()); } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/utils/RecordUtils.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/utils/RecordUtils.java index e1cb52055a8..8423c6a6445 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/utils/RecordUtils.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/utils/RecordUtils.java @@ -18,6 +18,7 @@ import org.apache.flink.table.types.logical.RowType; +import com.ververica.cdc.common.utils.StringUtils; import com.ververica.cdc.connectors.mysql.debezium.dispatcher.SignalEventDispatcher.WatermarkKind; import com.ververica.cdc.connectors.mysql.debezium.reader.DebeziumReader; import com.ververica.cdc.connectors.mysql.source.offset.BinlogOffset; @@ -387,6 +388,13 @@ public static TableId getTableId(SourceRecord dataRecord) { return new TableId(dbName, null, tableName); } + public static boolean isTableChangeRecord(SourceRecord dataRecord) { + Struct value = (Struct) dataRecord.value(); + Struct source = value.getStruct(Envelope.FieldName.SOURCE); + String tableName = source.getString(TABLE_NAME_KEY); + return !StringUtils.isNullOrWhitespaceOnly(tableName); + } + public static Object[] getSplitKey( RowType splitBoundaryType, SchemaNameAdjuster nameAdjuster, Struct target) { // the split key field contains single field now diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/table/MySqlReadableMetadata.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/table/MySqlReadableMetadata.java index fee3ef62f09..95539555aa1 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/table/MySqlReadableMetadata.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/table/MySqlReadableMetadata.java @@ -17,11 +17,13 @@ package com.ververica.cdc.connectors.mysql.table; import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.StringData; import org.apache.flink.table.data.TimestampData; import org.apache.flink.table.types.DataType; import com.ververica.cdc.debezium.table.MetadataConverter; +import com.ververica.cdc.debezium.table.RowDataMetadataConverter; import io.debezium.connector.AbstractSourceInfo; import io.debezium.data.Envelope; import org.apache.kafka.connect.data.Struct; @@ -78,6 +80,28 @@ public Object read(SourceRecord record) { return TimestampData.fromEpochMillis( (Long) sourceStruct.get(AbstractSourceInfo.TIMESTAMP_KEY)); } + }), + + /** + * It indicates the row kind of the changelog. '+I' means INSERT message, '-D' means DELETE + * message, '-U' means UPDATE_BEFORE message and '+U' means UPDATE_AFTER message + */ + ROW_KIND( + "row_kind", + DataTypes.STRING().notNull(), + new RowDataMetadataConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object read(RowData rowData) { + return StringData.fromString(rowData.getRowKind().shortString()); + } + + @Override + public Object read(SourceRecord record) { + throw new UnsupportedOperationException( + "Please call read(RowData rowData) method instead."); + } }); private final String key; diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/table/MySqlTableSourceFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/table/MySqlTableSourceFactory.java index 755cea74fc0..f136b243d0c 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/table/MySqlTableSourceFactory.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/table/MySqlTableSourceFactory.java @@ -213,6 +213,7 @@ public Set> optionalOptions() { } private static final String SCAN_STARTUP_MODE_VALUE_INITIAL = "initial"; + private static final String SCAN_STARTUP_MODE_VALUE_SNAPSHOT = "snapshot"; private static final String SCAN_STARTUP_MODE_VALUE_EARLIEST = "earliest-offset"; private static final String SCAN_STARTUP_MODE_VALUE_LATEST = "latest-offset"; private static final String SCAN_STARTUP_MODE_VALUE_SPECIFIC_OFFSET = "specific-offset"; @@ -224,6 +225,8 @@ private static StartupOptions getStartupOptions(ReadableConfig config) { switch (modeString.toLowerCase()) { case SCAN_STARTUP_MODE_VALUE_INITIAL: return StartupOptions.initial(); + case SCAN_STARTUP_MODE_VALUE_SNAPSHOT: + return StartupOptions.snapshot(); case SCAN_STARTUP_MODE_VALUE_LATEST: return StartupOptions.latest(); @@ -241,9 +244,10 @@ private static StartupOptions getStartupOptions(ReadableConfig config) { default: throw new ValidationException( String.format( - "Invalid value for option '%s'. Supported values are [%s, %s, %s, %s, %s], but was: %s", + "Invalid value for option '%s'. Supported values are [%s, %s, %s, %s, %s, %s], but was: %s", SCAN_STARTUP_MODE.key(), SCAN_STARTUP_MODE_VALUE_INITIAL, + SCAN_STARTUP_MODE_VALUE_SNAPSHOT, SCAN_STARTUP_MODE_VALUE_LATEST, SCAN_STARTUP_MODE_VALUE_EARLIEST, SCAN_STARTUP_MODE_VALUE_SPECIFIC_OFFSET, diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/table/StartupMode.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/table/StartupMode.java index 6cf91932188..e41ee248e17 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/table/StartupMode.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/table/StartupMode.java @@ -30,5 +30,6 @@ public enum StartupMode { SPECIFIC_OFFSETS, - TIMESTAMP + TIMESTAMP, + SNAPSHOT } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/table/StartupOptions.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/table/StartupOptions.java index 9e90502b021..03a64d07f2a 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/table/StartupOptions.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/table/StartupOptions.java @@ -40,6 +40,14 @@ public static StartupOptions initial() { return new StartupOptions(StartupMode.INITIAL, null); } + /** + * Performs an initial snapshot on the monitored database tables upon first startup, and not + * read the binlog anymore . + */ + public static StartupOptions snapshot() { + return new StartupOptions(StartupMode.SNAPSHOT, null); + } + /** * Never to perform snapshot on the monitored database tables upon first startup, just read from * the beginning of the binlog. This should be used with care, as it is only valid when the @@ -92,12 +100,23 @@ public static StartupOptions timestamp(long startupTimestampMillis) { private StartupOptions(StartupMode startupMode, BinlogOffset binlogOffset) { this.startupMode = startupMode; this.binlogOffset = binlogOffset; - if (startupMode != StartupMode.INITIAL) { + if (isStreamOnly()) { checkNotNull( binlogOffset, "Binlog offset is required if startup mode is %s", startupMode); } } + public boolean isStreamOnly() { + return startupMode == StartupMode.EARLIEST_OFFSET + || startupMode == StartupMode.LATEST_OFFSET + || startupMode == StartupMode.SPECIFIC_OFFSETS + || startupMode == StartupMode.TIMESTAMP; + } + + public boolean isSnapshotOnly() { + return startupMode == StartupMode.SNAPSHOT; + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/io/debezium/connector/mysql/MySqlConnection.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/io/debezium/connector/mysql/MySqlConnection.java index faa4997a87d..84f5c230a69 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/io/debezium/connector/mysql/MySqlConnection.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/io/debezium/connector/mysql/MySqlConnection.java @@ -38,7 +38,7 @@ import java.util.Properties; /** - * Copied from Debezium project(1.9.7.final) to add custom jdbc properties in the jdbc url. The new + * Copied from Debezium project(1.9.8.final) to add custom jdbc properties in the jdbc url. The new * parameter {@code jdbcProperties} in the constructor of {@link MySqlConnectionConfiguration} will * be used to generate the jdbc url pattern, and may overwrite the default value. * diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/io/debezium/connector/mysql/MySqlDefaultValueConverter.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/io/debezium/connector/mysql/MySqlDefaultValueConverter.java index 674fa68f7e7..9840369188d 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/io/debezium/connector/mysql/MySqlDefaultValueConverter.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/io/debezium/connector/mysql/MySqlDefaultValueConverter.java @@ -35,7 +35,7 @@ import java.util.regex.Pattern; /** - * Copied from Debezium project(1.9.7.final) to add BIGINT and SMALLINT to TRIM_DATA_TYPES. Remove + * Copied from Debezium project(1.9.8.final) to add BIGINT and SMALLINT to TRIM_DATA_TYPES. Remove * this when https://issues.redhat.com/browse/DBZ-6824 is fixed. * *

    Line 81 & 82: add BIGINT and SMALLINT to TRIM_DATA_TYPES. diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/io/debezium/connector/mysql/MySqlStreamingChangeEventSource.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/io/debezium/connector/mysql/MySqlStreamingChangeEventSource.java index a35aa93da61..298ac7268c9 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/io/debezium/connector/mysql/MySqlStreamingChangeEventSource.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/io/debezium/connector/mysql/MySqlStreamingChangeEventSource.java @@ -84,7 +84,7 @@ import static io.debezium.util.Strings.isNullOrEmpty; /** - * Copied from Debezium project to fix + * Copied from Debezium 1.9.8.Final project to fix * https://github.com/ververica/flink-cdc-connectors/issues/1944. * *

    Line 1427-1433 : Adjust GTID merging logic to support recovering from job which previously diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/io/debezium/connector/mysql/MySqlValueConverters.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/io/debezium/connector/mysql/MySqlValueConverters.java index 686e9722cc0..1d357c8f64e 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/io/debezium/connector/mysql/MySqlValueConverters.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/io/debezium/connector/mysql/MySqlValueConverters.java @@ -48,8 +48,8 @@ import java.util.regex.Pattern; /** - * Copied from Debezium project to fix FLOAT converted to FLOAT64 type issue. See DBZ-3865, - * DBZ-5843. Remove it when debezium version is upgraded above 2.0.0.Final. + * Copied from Debezium 1.9.8.Final project to fix FLOAT converted to FLOAT64 type issue. See + * DBZ-3865, DBZ-5843. Remove it when debezium version is upgraded above 2.0.0.Final. * *

    Line 240 & 246: add FLOAT type adjustment. * diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/io/debezium/connector/mysql/antlr/listener/DefaultValueParserListener.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/io/debezium/connector/mysql/antlr/listener/DefaultValueParserListener.java index d96a7a5ebd4..8a956123e31 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/io/debezium/connector/mysql/antlr/listener/DefaultValueParserListener.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/io/debezium/connector/mysql/antlr/listener/DefaultValueParserListener.java @@ -13,11 +13,11 @@ import java.util.concurrent.atomic.AtomicReference; /** - * Copied from Debezium project(v1.9.7.Final) to fix + * Copied from Debezium project(v1.9.8.Final) to fix * https://github.com/ververica/flink-cdc-connectors/issues/1506. * *

    Line 48~59: use the actual default string value when the sql contains COLLATE. We should - * remove this class after we bumped a higher debezium version where the + * remove this class after we bumped debezium 2.0 where the * https://issues.redhat.com/browse/DBZ-5587 has been fixed. */ public class DefaultValueParserListener extends MySqlParserBaseListener { @@ -95,7 +95,10 @@ public void exitDefaultValue(boolean skipIfUnknownOptional) { } private String unquote(String stringLiteral) { - return stringLiteral.substring(1, stringLiteral.length() - 1); + if (stringLiteral != null && stringLiteral.startsWith("'") && stringLiteral.endsWith("'")) { + return stringLiteral.substring(1, stringLiteral.length() - 1); + } + return stringLiteral; } private String unquoteBinary(String stringLiteral) { diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/source/MySqlSourceITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/source/MySqlSourceITCase.java index 2ca4038d24f..32cd0a21031 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/source/MySqlSourceITCase.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/source/MySqlSourceITCase.java @@ -160,6 +160,8 @@ public class MySqlSourceITCase extends MySqlSourceTestBase { private static final int USE_POST_LOWWATERMARK_HOOK = 1; private static final int USE_PRE_HIGHWATERMARK_HOOK = 2; + private static final int USE_POST_HIGHWATERMARK_HOOK = 3; + @Parameterized.Parameters(name = "table: {0}, chunkColumn: {1}") public static Collection parameters() { return Arrays.asList( @@ -350,10 +352,76 @@ public void testStartFromLatestOffset() throws Exception { testStartingOffset(StartupOptions.latest(), Collections.emptyList()); } + @Test + public void testSnapshotOnlyModeWithDMLPostHighWaterMark() throws Exception { + List records = + testBackfillWhenWritingEvents( + false, 21, USE_POST_HIGHWATERMARK_HOOK, StartupOptions.snapshot()); + List expectedRecords = + Arrays.asList( + "+I[101, user_1, Shanghai, 123567891234]", + "+I[102, user_2, Shanghai, 123567891234]", + "+I[103, user_3, Shanghai, 123567891234]", + "+I[109, user_4, Shanghai, 123567891234]", + "+I[110, user_5, Shanghai, 123567891234]", + "+I[111, user_6, Shanghai, 123567891234]", + "+I[118, user_7, Shanghai, 123567891234]", + "+I[121, user_8, Shanghai, 123567891234]", + "+I[123, user_9, Shanghai, 123567891234]", + "+I[1009, user_10, Shanghai, 123567891234]", + "+I[1010, user_11, Shanghai, 123567891234]", + "+I[1011, user_12, Shanghai, 123567891234]", + "+I[1012, user_13, Shanghai, 123567891234]", + "+I[1013, user_14, Shanghai, 123567891234]", + "+I[1014, user_15, Shanghai, 123567891234]", + "+I[1015, user_16, Shanghai, 123567891234]", + "+I[1016, user_17, Shanghai, 123567891234]", + "+I[1017, user_18, Shanghai, 123567891234]", + "+I[1018, user_19, Shanghai, 123567891234]", + "+I[1019, user_20, Shanghai, 123567891234]", + "+I[2000, user_21, Shanghai, 123567891234]"); + assertEqualsInAnyOrder(expectedRecords, records); + } + + @Test + public void testSnapshotOnlyModeWithDMLPreHighWaterMark() throws Exception { + List records = + testBackfillWhenWritingEvents( + false, 21, USE_PRE_HIGHWATERMARK_HOOK, StartupOptions.snapshot()); + List expectedRecords = + Arrays.asList( + "+I[101, user_1, Shanghai, 123567891234]", + "+I[102, user_2, Shanghai, 123567891234]", + "+I[103, user_3, Shanghai, 123567891234]", + "+I[109, user_4, Shanghai, 123567891234]", + "+I[110, user_5, Shanghai, 123567891234]", + "+I[111, user_6, Shanghai, 123567891234]", + "+I[118, user_7, Shanghai, 123567891234]", + "+I[121, user_8, Shanghai, 123567891234]", + "+I[123, user_9, Shanghai, 123567891234]", + "+I[1009, user_10, Shanghai, 123567891234]", + "+I[1010, user_11, Shanghai, 123567891234]", + "+I[1011, user_12, Shanghai, 123567891234]", + "+I[1012, user_13, Shanghai, 123567891234]", + "+I[1013, user_14, Shanghai, 123567891234]", + "+I[1014, user_15, Shanghai, 123567891234]", + "+I[1015, user_16, Shanghai, 123567891234]", + "+I[1016, user_17, Shanghai, 123567891234]", + "+I[1017, user_18, Shanghai, 123567891234]", + "+I[1018, user_19, Shanghai, 123567891234]", + "+I[2000, user_21, Pittsburgh, 123567891234]", + "+I[15213, user_15213, Shanghai, 123567891234]"); + // when enable backfill, the wal log between (snapshot, high_watermark) will be + // applied as snapshot image + assertEqualsInAnyOrder(expectedRecords, records); + } + @Test public void testEnableBackfillWithDMLPreHighWaterMark() throws Exception { - List records = testBackfillWhenWritingEvents(false, 21, USE_PRE_HIGHWATERMARK_HOOK); + List records = + testBackfillWhenWritingEvents( + false, 21, USE_PRE_HIGHWATERMARK_HOOK, StartupOptions.initial()); List expectedRecords = Arrays.asList( @@ -386,7 +454,9 @@ public void testEnableBackfillWithDMLPreHighWaterMark() throws Exception { @Test public void testEnableBackfillWithDMLPostLowWaterMark() throws Exception { - List records = testBackfillWhenWritingEvents(false, 21, USE_POST_LOWWATERMARK_HOOK); + List records = + testBackfillWhenWritingEvents( + false, 21, USE_POST_LOWWATERMARK_HOOK, StartupOptions.initial()); List expectedRecords = Arrays.asList( @@ -419,7 +489,9 @@ public void testEnableBackfillWithDMLPostLowWaterMark() throws Exception { @Test public void testSkipBackfillWithDMLPreHighWaterMark() throws Exception { - List records = testBackfillWhenWritingEvents(true, 25, USE_PRE_HIGHWATERMARK_HOOK); + List records = + testBackfillWhenWritingEvents( + true, 25, USE_PRE_HIGHWATERMARK_HOOK, StartupOptions.initial()); List expectedRecords = Arrays.asList( @@ -456,7 +528,9 @@ public void testSkipBackfillWithDMLPreHighWaterMark() throws Exception { @Test public void testSkipBackfillWithDMLPostLowWaterMark() throws Exception { - List records = testBackfillWhenWritingEvents(true, 25, USE_POST_LOWWATERMARK_HOOK); + List records = + testBackfillWhenWritingEvents( + true, 25, USE_POST_LOWWATERMARK_HOOK, StartupOptions.initial()); List expectedRecords = Arrays.asList( @@ -492,7 +566,11 @@ public void testSkipBackfillWithDMLPostLowWaterMark() throws Exception { } private List testBackfillWhenWritingEvents( - boolean skipSnapshotBackfill, int fetchSize, int hookType) throws Exception { + boolean skipSnapshotBackfill, + int fetchSize, + int hookType, + StartupOptions startupOptions) + throws Exception { customDatabase.createAndInitialize(); TestTable customerTable = new TestTable(customDatabase, "customers", TestTableSchemas.CUSTOMERS); @@ -509,6 +587,7 @@ private List testBackfillWhenWritingEvents( .tableList(customerTable.getTableId()) .deserializer(customerTable.getDeserializer()) .skipSnapshotBackfill(skipSnapshotBackfill) + .startupOptions(startupOptions) .build(); String[] statements = @@ -528,16 +607,18 @@ private List testBackfillWhenWritingEvents( connection.setAutoCommit(false); connection.execute(statements); connection.commit(); - try { - Thread.sleep(500L); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } }; - if (hookType == USE_PRE_HIGHWATERMARK_HOOK) { - hooks.setPreHighWatermarkAction(snapshotPhaseHook); - } else if (hookType == USE_POST_LOWWATERMARK_HOOK) { - hooks.setPostLowWatermarkAction(snapshotPhaseHook); + + switch (hookType) { + case USE_POST_LOWWATERMARK_HOOK: + hooks.setPostLowWatermarkAction(snapshotPhaseHook); + break; + case USE_PRE_HIGHWATERMARK_HOOK: + hooks.setPreHighWatermarkAction(snapshotPhaseHook); + break; + case USE_POST_HIGHWATERMARK_HOOK: + hooks.setPostHighWatermarkAction(snapshotPhaseHook); + break; } source.setSnapshotHooks(hooks); diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/source/SpecificStartingOffsetITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/source/SpecificStartingOffsetITCase.java index b30750c754f..254c99fb112 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/source/SpecificStartingOffsetITCase.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/source/SpecificStartingOffsetITCase.java @@ -47,6 +47,7 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; import org.junit.rules.TemporaryFolder; +import org.locationtech.jts.util.Assert; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testcontainers.containers.output.Slf4jLogConsumer; @@ -256,6 +257,77 @@ void testStartingFromSpecificOffset() throws Exception { restoredJobClient.cancel().get(); } + @Test + void testBinlogSplitFromTimestampOffset() throws Exception { + // Purge binary log at first + purgeBinaryLogs(); + + Assert.equals( + BinlogOffset.ofBinlogFilePosition("mysql-bin.000004", 0), + DebeziumUtils.findBinlogOffset(System.currentTimeMillis(), connection)); + + executeStatements( + String.format( + "INSERT INTO %s VALUES (15213, 'Alice', 'Rome', '123456987');", + customers.getTableId())); + Thread.sleep(1000); + long t1 = System.currentTimeMillis(); + flushLogs(); + + executeStatements( + String.format( + "INSERT INTO %s VALUES (15513, 'Bob', 'Milan', '123456987');", + customers.getTableId())); + Thread.sleep(1000); + long t2 = System.currentTimeMillis(); + flushLogs(); + + executeStatements( + String.format( + "INSERT INTO %s VALUES (18213, 'Charlie', 'Paris', '123456987');", + customers.getTableId())); + Thread.sleep(1000); + long t3 = System.currentTimeMillis(); + flushLogs(); + + executeStatements( + String.format( + "INSERT INTO %s VALUES (19613, 'Tom', 'NewYork', '123456987');", + customers.getTableId())); + Thread.sleep(1000); + long t4 = System.currentTimeMillis(); + flushLogs(); + + executeStatements( + String.format( + "INSERT INTO %s VALUES (20913, 'Cat', 'Washington', '123456987');", + customers.getTableId())); + Thread.sleep(1000); + long t5 = System.currentTimeMillis(); + flushLogs(); + + Assert.equals( + BinlogOffset.ofBinlogFilePosition("mysql-bin.000005", 0), + DebeziumUtils.findBinlogOffset(t1, connection)); + Assert.equals( + BinlogOffset.ofBinlogFilePosition("mysql-bin.000006", 0), + DebeziumUtils.findBinlogOffset(t2, connection)); + Assert.equals( + BinlogOffset.ofBinlogFilePosition("mysql-bin.000007", 0), + DebeziumUtils.findBinlogOffset(t3, connection)); + Assert.equals( + BinlogOffset.ofBinlogFilePosition("mysql-bin.000008", 0), + DebeziumUtils.findBinlogOffset(t4, connection)); + Assert.equals( + BinlogOffset.ofBinlogFilePosition("mysql-bin.000009", 0), + DebeziumUtils.findBinlogOffset(t5, connection)); + + purgeBinaryLogs(); + Assert.equals( + BinlogOffset.ofBinlogFilePosition("mysql-bin.000009", 0), + DebeziumUtils.findBinlogOffset(t3, connection)); + } + @Test void testStartingFromTimestampOffset() throws Exception { // Purge binary log at first @@ -275,6 +347,9 @@ void testStartingFromTimestampOffset() throws Exception { "INSERT INTO %s VALUES (18213, 'Charlie', 'Paris', '123456987');", customers.getTableId())); + // switch new log + flushLogs(); + // Record current timestamp Thread.sleep(1000); StartupOptions startupOptions = StartupOptions.timestamp(System.currentTimeMillis()); diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/source/assigners/MySqlHybridSplitAssignerTest.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/source/assigners/MySqlHybridSplitAssignerTest.java index 2bd4c1ff9d5..8c303ce852b 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/source/assigners/MySqlHybridSplitAssignerTest.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/source/assigners/MySqlHybridSplitAssignerTest.java @@ -31,6 +31,7 @@ import com.ververica.cdc.connectors.mysql.source.split.FinishedSnapshotSplitInfo; import com.ververica.cdc.connectors.mysql.source.split.MySqlBinlogSplit; import com.ververica.cdc.connectors.mysql.source.split.MySqlSchemalessSnapshotSplit; +import com.ververica.cdc.connectors.mysql.source.split.MySqlSnapshotSplit; import com.ververica.cdc.connectors.mysql.source.split.MySqlSplit; import com.ververica.cdc.connectors.mysql.table.StartupOptions; import com.ververica.cdc.connectors.mysql.testutils.UniqueDatabase; @@ -49,6 +50,7 @@ import java.util.stream.Collectors; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; /** Tests for {@link MySqlHybridSplitAssigner}. */ public class MySqlHybridSplitAssignerTest extends MySqlSourceTestBase { @@ -65,7 +67,8 @@ public static void init() { public void testAssignMySqlBinlogSplitAfterAllSnapshotSplitsFinished() { final String captureTable = "customers"; - MySqlSourceConfig configuration = getConfig(new String[] {captureTable}); + MySqlSourceConfig configuration = + getConfig(new String[] {captureTable}, StartupOptions.initial()); // Step 1. Mock MySqlHybridSplitAssigner Object TableId tableId = new TableId(null, customerDatabase.getDatabaseName(), captureTable); @@ -139,14 +142,54 @@ public void testAssignMySqlBinlogSplitAfterAllSnapshotSplitsFinished() { assigner.close(); } - private MySqlSourceConfig getConfig(String[] captureTables) { + @Test + public void testAssigningInSnapshotOnlyMode() { + final String captureTable = "customers"; + + MySqlSourceConfig sourceConfig = + getConfig(new String[] {captureTable}, StartupOptions.snapshot()); + + // Create and initialize assigner + MySqlHybridSplitAssigner assigner = + new MySqlHybridSplitAssigner(sourceConfig, 1, new ArrayList<>(), false); + assigner.open(); + + // Get all snapshot splits + List snapshotSplits = drainSnapshotSplits(assigner); + + // Generate fake finished offsets from 0 to snapshotSplits.size() - 1 + int i = 0; + Map finishedOffsets = new HashMap<>(); + for (MySqlSnapshotSplit snapshotSplit : snapshotSplits) { + BinlogOffset binlogOffset = + BinlogOffset.builder().setBinlogFilePosition("foo", i++).build(); + finishedOffsets.put(snapshotSplit.splitId(), binlogOffset); + } + assigner.onFinishedSplits(finishedOffsets); + + // Get the binlog split + Optional split = assigner.getNext(); + assertTrue(split.isPresent()); + assertTrue(split.get() instanceof MySqlBinlogSplit); + MySqlBinlogSplit binlogSplit = split.get().asBinlogSplit(); + + // Validate if the stopping offset of the binlog split is the maximum among all finished + // offsets, which should be snapshotSplits.size() - 1 + assertEquals( + BinlogOffset.builder() + .setBinlogFilePosition("foo", snapshotSplits.size() - 1) + .build(), + binlogSplit.getEndingOffset()); + } + + private MySqlSourceConfig getConfig(String[] captureTables, StartupOptions startupOptions) { String[] captureTableIds = Arrays.stream(captureTables) .map(tableName -> customerDatabase.getDatabaseName() + "." + tableName) .toArray(String[]::new); return new MySqlSourceConfigFactory() - .startupOptions(StartupOptions.initial()) + .startupOptions(startupOptions) .databaseList(customerDatabase.getDatabaseName()) .tableList(captureTableIds) .hostname(MYSQL_CONTAINER.getHost()) @@ -156,4 +199,17 @@ private MySqlSourceConfig getConfig(String[] captureTables) { .serverTimeZone(ZoneId.of("UTC").toString()) .createConfig(0); } + + private List drainSnapshotSplits(MySqlHybridSplitAssigner assigner) { + List snapshotSplits = new ArrayList<>(); + while (true) { + Optional split = assigner.getNext(); + if (!split.isPresent()) { + break; + } + assertTrue(split.get() instanceof MySqlSnapshotSplit); + snapshotSplits.add(split.get().asSnapshotSplit()); + } + return snapshotSplits; + } } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/table/MySqlConnectorITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/table/MySqlConnectorITCase.java index 931d6734ce4..9c994e645f7 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/table/MySqlConnectorITCase.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/table/MySqlConnectorITCase.java @@ -933,6 +933,7 @@ public void testMetadataColumns() throws Exception { "CREATE TABLE mysql_users (" + " db_name STRING METADATA FROM 'database_name' VIRTUAL," + " table_name STRING METADATA VIRTUAL," + + " row_kind STRING METADATA FROM 'row_kind' VIRTUAL," + " `id` DECIMAL(20, 0) NOT NULL," + " name STRING," + " address STRING," @@ -967,6 +968,7 @@ public void testMetadataColumns() throws Exception { "CREATE TABLE sink (" + " database_name STRING," + " table_name STRING," + + " row_kind STRING," + " `id` DECIMAL(20, 0) NOT NULL," + " name STRING," + " address STRING," @@ -1004,15 +1006,15 @@ public void testMetadataColumns() throws Exception { List expected = Stream.of( - "+I[%s, user_table_1_1, 111, user_111, Shanghai, 123567891234, user_111@foo.com, null]", - "+I[%s, user_table_1_2, 121, user_121, Shanghai, 123567891234, null, null]", - "+I[%s, user_table_1_2, 200, user_200, Wuhan, 123567891234, null, null]", - "+I[%s, user_table_1_1, 300, user_300, Hangzhou, 123567891234, user_300@foo.com, null]", - "+U[%s, user_table_1_1, 300, user_300, Beijing, 123567891234, user_300@foo.com, null]", - "+U[%s, user_table_1_2, 121, user_121, Shanghai, 88888888, null, null]", - "-D[%s, user_table_1_1, 111, user_111, Shanghai, 123567891234, user_111@foo.com, null]", - "-U[%s, user_table_1_1, 300, user_300, Hangzhou, 123567891234, user_300@foo.com, null]", - "-U[%s, user_table_1_2, 121, user_121, Shanghai, 123567891234, null, null]") + "+I[%s, user_table_1_1, +I, 111, user_111, Shanghai, 123567891234, user_111@foo.com, null]", + "+I[%s, user_table_1_2, +I, 121, user_121, Shanghai, 123567891234, null, null]", + "+I[%s, user_table_1_2, +I, 200, user_200, Wuhan, 123567891234, null, null]", + "+I[%s, user_table_1_1, +I, 300, user_300, Hangzhou, 123567891234, user_300@foo.com, null]", + "+U[%s, user_table_1_1, +U, 300, user_300, Beijing, 123567891234, user_300@foo.com, null]", + "+U[%s, user_table_1_2, +U, 121, user_121, Shanghai, 88888888, null, null]", + "-D[%s, user_table_1_1, -D, 111, user_111, Shanghai, 123567891234, user_111@foo.com, null]", + "-U[%s, user_table_1_1, -U, 300, user_300, Hangzhou, 123567891234, user_300@foo.com, null]", + "-U[%s, user_table_1_2, -U, 121, user_121, Shanghai, 123567891234, null, null]") .map(s -> String.format(s, userDatabase1.getDatabaseName())) .sorted() .collect(Collectors.toList()); diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/table/MySqlTableSourceFactoryTest.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/table/MySqlTableSourceFactoryTest.java index f9da62ccfc2..4f2b7cfa446 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/table/MySqlTableSourceFactoryTest.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/table/MySqlTableSourceFactoryTest.java @@ -720,7 +720,7 @@ public void testValidation() { } catch (Throwable t) { String msg = "Invalid value for option 'scan.startup.mode'. Supported values are " - + "[initial, latest-offset, earliest-offset, specific-offset, timestamp], " + + "[initial, snapshot, latest-offset, earliest-offset, specific-offset, timestamp], " + "but was: abc"; assertTrue(ExceptionUtils.findThrowableWithMessage(t, msg).isPresent()); } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/testutils/UniqueDatabase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/testutils/UniqueDatabase.java index 6a34fe15847..0dd4ea8d61a 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/testutils/UniqueDatabase.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/com/ververica/cdc/connectors/mysql/testutils/UniqueDatabase.java @@ -142,7 +142,7 @@ public void createAndInitialize() { } } - /** Drop the database if is exists. */ + /** Drop the database if it is existing. */ public void dropDatabase() { try { try (Connection connection = diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/com/ververica/cdc/connectors/oceanbase/source/OceanBaseConnection.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/com/ververica/cdc/connectors/oceanbase/source/OceanBaseConnection.java index e45497e9c1d..3e3b85a5908 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/com/ververica/cdc/connectors/oceanbase/source/OceanBaseConnection.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/com/ververica/cdc/connectors/oceanbase/source/OceanBaseConnection.java @@ -20,13 +20,17 @@ import io.debezium.jdbc.JdbcConfiguration; import io.debezium.jdbc.JdbcConnection; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.sql.DatabaseMetaData; import java.sql.ResultSet; import java.sql.SQLException; +import java.sql.Timestamp; import java.time.Duration; import java.util.ArrayList; import java.util.List; +import java.util.Optional; import java.util.Properties; import java.util.regex.Pattern; import java.util.stream.Collectors; @@ -34,6 +38,8 @@ /** {@link JdbcConnection} extension to be used with OceanBase server. */ public class OceanBaseConnection extends JdbcConnection { + private static final Logger LOG = LoggerFactory.getLogger(OceanBaseConnection.class); + private static final String QUOTED_CHARACTER = "`"; private static final Properties DEFAULT_JDBC_PROPERTIES = initializeDefaultJdbcProperties(); private static final String MYSQL_URL_PATTERN = @@ -105,20 +111,42 @@ private static JdbcConnection.ConnectionFactory factory( formatJdbcUrl(jdbcDriver, jdbcProperties), jdbcDriver, classLoader); } - private String getSystemSchema() { - return "mysql".equalsIgnoreCase(compatibleMode) ? "oceanbase" : "SYS"; - } - /** - * Get current timestamp in nanoseconds from GTS (Global Timestamp Service). + * Get current timestamp number in seconds. * - * @return the global timestamp. + * @return current timestamp number. * @throws SQLException If a database access error occurs. */ - public Long getGlobalTimestamp() throws SQLException { + public long getCurrentTimestampS() throws SQLException { + try { + long globalTimestamp = getGlobalTimestamp(); + LOG.info("Global timestamp: {}", globalTimestamp); + return Long.parseLong(String.valueOf(globalTimestamp).substring(0, 10)); + } catch (Exception e) { + LOG.warn("Failed to get global timestamp, use local timestamp instead"); + } + return getCurrentTimestamp() + .orElseThrow(IllegalStateException::new) + .toInstant() + .getEpochSecond(); + } + + private long getGlobalTimestamp() throws SQLException { + String schema = "mysql".equalsIgnoreCase(compatibleMode) ? "oceanbase" : "SYS"; + return querySingleValue( + connection(), + "SELECT TS_VALUE FROM " + schema + ".V$OB_TIMESTAMP_SERVICE", + ps -> {}, + rs -> rs.getLong(1)); + } + + @Override + public Optional getCurrentTimestamp() throws SQLException { return queryAndMap( - String.format("SELECT TS_VALUE FROM %s.V$OB_TIMESTAMP_SERVICE", getSystemSchema()), - rs -> rs.next() ? rs.getLong(1) : null); + "mysql".equalsIgnoreCase(compatibleMode) + ? "SELECT CURRENT_TIMESTAMP" + : "SELECT CURRENT_TIMESTAMP FROM DUAL", + rs -> rs.next() ? Optional.of(rs.getTimestamp(1)) : Optional.empty()); } /** diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/com/ververica/cdc/connectors/oceanbase/source/OceanBaseRichSourceFunction.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/com/ververica/cdc/connectors/oceanbase/source/OceanBaseRichSourceFunction.java index e243243b0e2..8520f48a505 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/com/ververica/cdc/connectors/oceanbase/source/OceanBaseRichSourceFunction.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/com/ververica/cdc/connectors/oceanbase/source/OceanBaseRichSourceFunction.java @@ -152,11 +152,7 @@ public void run(SourceContext ctx) throws Exception { initTableWhiteList(); if (shouldReadSnapshot()) { - Long globalTimestamp = getSnapshotConnection().getGlobalTimestamp(); - if (globalTimestamp == null || globalTimestamp <= 0) { - throw new RuntimeException("Got invalid global timestamp: " + globalTimestamp); - } - long startTimestamp = globalTimestamp / 1000_000_000; + long startTimestamp = getSnapshotConnection().getCurrentTimestampS(); LOG.info("Snapshot reading started from timestamp: {}", startTimestamp); readSnapshotRecords(); LOG.info("Snapshot reading finished"); diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/com/ververica/cdc/connectors/oracle/source/reader/fetch/EventProcessorFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/com/ververica/cdc/connectors/oracle/source/reader/fetch/EventProcessorFactory.java new file mode 100644 index 00000000000..9b0b8c8bede --- /dev/null +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/com/ververica/cdc/connectors/oracle/source/reader/fetch/EventProcessorFactory.java @@ -0,0 +1,282 @@ +/* + * Copyright 2023 Ververica Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.ververica.cdc.connectors.oracle.source.reader.fetch; + +import com.ververica.cdc.common.annotation.Internal; +import com.ververica.cdc.connectors.base.relational.JdbcSourceEventDispatcher; +import com.ververica.cdc.connectors.base.source.meta.split.StreamSplit; +import com.ververica.cdc.connectors.base.source.meta.wartermark.WatermarkKind; +import com.ververica.cdc.connectors.oracle.source.meta.offset.RedoLogOffset; +import io.debezium.DebeziumException; +import io.debezium.connector.oracle.OracleConnection; +import io.debezium.connector.oracle.OracleConnectorConfig; +import io.debezium.connector.oracle.OracleDatabaseSchema; +import io.debezium.connector.oracle.OracleOffsetContext; +import io.debezium.connector.oracle.OraclePartition; +import io.debezium.connector.oracle.OracleStreamingChangeEventSourceMetrics; +import io.debezium.connector.oracle.logminer.events.LogMinerEventRow; +import io.debezium.connector.oracle.logminer.processor.LogMinerEventProcessor; +import io.debezium.connector.oracle.logminer.processor.infinispan.EmbeddedInfinispanLogMinerEventProcessor; +import io.debezium.connector.oracle.logminer.processor.infinispan.RemoteInfinispanLogMinerEventProcessor; +import io.debezium.connector.oracle.logminer.processor.memory.MemoryLogMinerEventProcessor; +import io.debezium.pipeline.ErrorHandler; +import io.debezium.pipeline.source.spi.ChangeEventSource; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.SQLException; + +import static com.ververica.cdc.connectors.oracle.source.meta.offset.RedoLogOffset.NO_STOPPING_OFFSET; + +/** + * Factory to produce a LogMinerEventProcessor with enhanced processRow method to distinguish + * whether is bounded. + */ +@Internal +public class EventProcessorFactory { + private static final Logger LOG = LoggerFactory.getLogger(EventProcessorFactory.class); + + private EventProcessorFactory() {} + + public static LogMinerEventProcessor createProcessor( + ChangeEventSource.ChangeEventSourceContext context, + OracleConnectorConfig connectorConfig, + OracleConnection jdbcConnection, + JdbcSourceEventDispatcher dispatcher, + OraclePartition partition, + OracleOffsetContext offsetContext, + OracleDatabaseSchema schema, + OracleStreamingChangeEventSourceMetrics metrics, + ErrorHandler errorHandler, + StreamSplit redoLogSplit) { + final OracleConnectorConfig.LogMiningBufferType bufferType = + connectorConfig.getLogMiningBufferType(); + if (bufferType.equals(OracleConnectorConfig.LogMiningBufferType.MEMORY)) { + return new CDCMemoryLogMinerEventProcessor( + context, + connectorConfig, + jdbcConnection, + dispatcher, + partition, + offsetContext, + schema, + metrics, + errorHandler, + redoLogSplit); + } else if (bufferType.equals( + OracleConnectorConfig.LogMiningBufferType.INFINISPAN_EMBEDDED)) { + return new CDCEmbeddedInfinispanLogMinerEventProcessor( + context, + connectorConfig, + jdbcConnection, + dispatcher, + partition, + offsetContext, + schema, + metrics, + errorHandler, + redoLogSplit); + } else if (bufferType.equals(OracleConnectorConfig.LogMiningBufferType.INFINISPAN_REMOTE)) { + return new CDCRemoteInfinispanLogMinerEventProcessor( + context, + connectorConfig, + jdbcConnection, + dispatcher, + partition, + offsetContext, + schema, + metrics, + errorHandler, + redoLogSplit); + } else { + throw new IllegalArgumentException( + "not support this type of bufferType: " + bufferType); + } + } + + /** + * A {@link MemoryLogMinerEventProcessor} with enhanced processRow method to distinguish whether + * is bounded. + */ + public static class CDCMemoryLogMinerEventProcessor extends MemoryLogMinerEventProcessor { + private final StreamSplit redoLogSplit; + private final ErrorHandler errorHandler; + + private ChangeEventSource.ChangeEventSourceContext context; + private final JdbcSourceEventDispatcher dispatcher; + + public CDCMemoryLogMinerEventProcessor( + ChangeEventSource.ChangeEventSourceContext context, + OracleConnectorConfig connectorConfig, + OracleConnection jdbcConnection, + JdbcSourceEventDispatcher dispatcher, + OraclePartition partition, + OracleOffsetContext offsetContext, + OracleDatabaseSchema schema, + OracleStreamingChangeEventSourceMetrics metrics, + ErrorHandler errorHandler, + StreamSplit redoLogSplit) { + super( + context, + connectorConfig, + jdbcConnection, + dispatcher, + partition, + offsetContext, + schema, + metrics); + this.redoLogSplit = redoLogSplit; + this.errorHandler = errorHandler; + this.context = context; + this.dispatcher = dispatcher; + } + + @Override + protected void processRow(OraclePartition partition, LogMinerEventRow row) + throws SQLException, InterruptedException { + super.processRow(partition, row); + afterRowProcess(partition, row, redoLogSplit, errorHandler, dispatcher, context); + } + } + + /** + * A {@link EmbeddedInfinispanLogMinerEventProcessor} with enhanced processRow method to + * distinguish whether is bounded. + */ + public static class CDCEmbeddedInfinispanLogMinerEventProcessor + extends EmbeddedInfinispanLogMinerEventProcessor { + private final StreamSplit redoLogSplit; + private final ErrorHandler errorHandler; + + private ChangeEventSource.ChangeEventSourceContext context; + private final JdbcSourceEventDispatcher dispatcher; + + public CDCEmbeddedInfinispanLogMinerEventProcessor( + ChangeEventSource.ChangeEventSourceContext context, + OracleConnectorConfig connectorConfig, + OracleConnection jdbcConnection, + JdbcSourceEventDispatcher dispatcher, + OraclePartition partition, + OracleOffsetContext offsetContext, + OracleDatabaseSchema schema, + OracleStreamingChangeEventSourceMetrics metrics, + ErrorHandler errorHandler, + StreamSplit redoLogSplit) { + super( + context, + connectorConfig, + jdbcConnection, + dispatcher, + partition, + offsetContext, + schema, + metrics); + this.redoLogSplit = redoLogSplit; + this.errorHandler = errorHandler; + this.context = context; + this.dispatcher = dispatcher; + } + + @Override + protected void processRow(OraclePartition partition, LogMinerEventRow row) + throws SQLException, InterruptedException { + super.processRow(partition, row); + afterRowProcess(partition, row, redoLogSplit, errorHandler, dispatcher, context); + } + } + + /** + * A {@link CDCRemoteInfinispanLogMinerEventProcessor} with enhanced processRow method to + * distinguish whether is bounded. + */ + public static class CDCRemoteInfinispanLogMinerEventProcessor + extends RemoteInfinispanLogMinerEventProcessor { + private final StreamSplit redoLogSplit; + private final ErrorHandler errorHandler; + + private ChangeEventSource.ChangeEventSourceContext context; + private final JdbcSourceEventDispatcher dispatcher; + + public CDCRemoteInfinispanLogMinerEventProcessor( + ChangeEventSource.ChangeEventSourceContext context, + OracleConnectorConfig connectorConfig, + OracleConnection jdbcConnection, + JdbcSourceEventDispatcher dispatcher, + OraclePartition partition, + OracleOffsetContext offsetContext, + OracleDatabaseSchema schema, + OracleStreamingChangeEventSourceMetrics metrics, + ErrorHandler errorHandler, + StreamSplit redoLogSplit) { + super( + context, + connectorConfig, + jdbcConnection, + dispatcher, + partition, + offsetContext, + schema, + metrics); + this.redoLogSplit = redoLogSplit; + this.errorHandler = errorHandler; + this.context = context; + this.dispatcher = dispatcher; + } + + @Override + protected void processRow(OraclePartition partition, LogMinerEventRow row) + throws SQLException, InterruptedException { + super.processRow(partition, row); + afterRowProcess(partition, row, redoLogSplit, errorHandler, dispatcher, context); + } + } + + public static void afterRowProcess( + OraclePartition partition, + LogMinerEventRow row, + StreamSplit redoLogSplit, + ErrorHandler errorHandler, + JdbcSourceEventDispatcher dispatcher, + ChangeEventSource.ChangeEventSourceContext context) { + // check do we need to stop for fetch redo log for snapshot split. + if (isBoundedRead(redoLogSplit)) { + final RedoLogOffset currentRedoLogOffset = new RedoLogOffset(row.getScn().longValue()); + // reach the high watermark, the redo log fetcher should be finished + if (currentRedoLogOffset.isAtOrAfter(redoLogSplit.getEndingOffset())) { + // send redo log end event + try { + dispatcher.dispatchWatermarkEvent( + partition.getSourcePartition(), + redoLogSplit, + currentRedoLogOffset, + WatermarkKind.END); + } catch (InterruptedException e) { + LOG.error("Send signal event error.", e); + errorHandler.setProducerThrowable( + new DebeziumException("Error processing redo log signal event", e)); + } + // tell fetcher the redo log task finished + ((OracleScanFetchTask.OracleSnapshotRedoLogSplitChangeEventSourceContext) context) + .finished(); + } + } + } + + private static boolean isBoundedRead(StreamSplit redoLogSplit) { + return !NO_STOPPING_OFFSET.equals(redoLogSplit.getEndingOffset()); + } +} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/com/ververica/cdc/connectors/oracle/source/reader/fetch/OracleStreamFetchTask.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/com/ververica/cdc/connectors/oracle/source/reader/fetch/OracleStreamFetchTask.java index 097f763e191..3dd45bf2dc3 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/com/ververica/cdc/connectors/oracle/source/reader/fetch/OracleStreamFetchTask.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/com/ververica/cdc/connectors/oracle/source/reader/fetch/OracleStreamFetchTask.java @@ -16,13 +16,11 @@ package com.ververica.cdc.connectors.oracle.source.reader.fetch; +import com.ververica.cdc.common.annotation.Internal; import com.ververica.cdc.connectors.base.relational.JdbcSourceEventDispatcher; import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitBase; import com.ververica.cdc.connectors.base.source.meta.split.StreamSplit; -import com.ververica.cdc.connectors.base.source.meta.wartermark.WatermarkKind; import com.ververica.cdc.connectors.base.source.reader.external.FetchTask; -import com.ververica.cdc.connectors.oracle.source.meta.offset.RedoLogOffset; -import io.debezium.DebeziumException; import io.debezium.config.Configuration; import io.debezium.connector.oracle.OracleConnection; import io.debezium.connector.oracle.OracleConnectorConfig; @@ -31,18 +29,15 @@ import io.debezium.connector.oracle.OraclePartition; import io.debezium.connector.oracle.OracleStreamingChangeEventSourceMetrics; import io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSource; +import io.debezium.connector.oracle.logminer.processor.LogMinerEventProcessor; import io.debezium.pipeline.ErrorHandler; import io.debezium.pipeline.source.spi.ChangeEventSource; import io.debezium.util.Clock; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.HashMap; -import java.util.Map; - -import static com.ververica.cdc.connectors.oracle.source.meta.offset.RedoLogOffset.NO_STOPPING_OFFSET; - /** The task to work for fetching data of Oracle table stream split. */ +@Internal public class OracleStreamFetchTask implements FetchTask { private final StreamSplit split; @@ -100,6 +95,12 @@ public static class RedoLogSplitReadTask extends LogMinerStreamingChangeEventSou private final JdbcSourceEventDispatcher dispatcher; private final ErrorHandler errorHandler; private ChangeEventSourceContext context; + private final OracleConnectorConfig connectorConfig; + private final OracleConnection connection; + + private final OracleDatabaseSchema schema; + + private final OracleStreamingChangeEventSourceMetrics metrics; public RedoLogSplitReadTask( OracleConnectorConfig connectorConfig, @@ -122,6 +123,10 @@ public RedoLogSplitReadTask( this.redoLogSplit = redoLogSplit; this.dispatcher = dispatcher; this.errorHandler = errorHandler; + this.connectorConfig = connectorConfig; + this.connection = connection; + this.metrics = metrics; + this.schema = schema; } @Override @@ -133,48 +138,26 @@ public void execute( super.execute(context, partition, offsetContext); } + /** + * Delegate {@link EventProcessorFactory} to produce a LogMinerEventProcessor with enhanced + * processRow method to distinguish whether is bounded. + */ @Override - protected void afterHandleScn( - OraclePartition partition, OracleOffsetContext offsetContext) { - super.afterHandleScn(partition, offsetContext); - // check do we need to stop for fetch redo log for snapshot split. - if (isBoundedRead()) { - final RedoLogOffset currentRedoLogOffset = - getCurrentRedoLogOffset(offsetContext.getOffset()); - // reach the high watermark, the redo log fetcher should be finished - if (currentRedoLogOffset.isAtOrAfter(redoLogSplit.getEndingOffset())) { - // send redo log end event - try { - dispatcher.dispatchWatermarkEvent( - partition.getSourcePartition(), - redoLogSplit, - currentRedoLogOffset, - WatermarkKind.END); - } catch (InterruptedException e) { - LOG.error("Send signal event error.", e); - errorHandler.setProducerThrowable( - new DebeziumException("Error processing redo log signal event", e)); - } - // tell fetcher the redo log task finished - ((OracleScanFetchTask.OracleSnapshotRedoLogSplitChangeEventSourceContext) - context) - .finished(); - } - } - } - - private boolean isBoundedRead() { - return !NO_STOPPING_OFFSET.equals(redoLogSplit.getEndingOffset()); - } - - public static RedoLogOffset getCurrentRedoLogOffset(Map offset) { - Map offsetStrMap = new HashMap<>(); - for (Map.Entry entry : offset.entrySet()) { - offsetStrMap.put( - entry.getKey(), - entry.getValue() == null ? null : entry.getValue().toString()); - } - return new RedoLogOffset(offsetStrMap); + protected LogMinerEventProcessor createProcessor( + ChangeEventSourceContext context, + OraclePartition partition, + OracleOffsetContext offsetContext) { + return EventProcessorFactory.createProcessor( + context, + connectorConfig, + connection, + dispatcher, + partition, + offsetContext, + schema, + metrics, + errorHandler, + redoLogSplit); } } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/OracleConnection.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/OracleConnection.java deleted file mode 100644 index 5fd6b07cd7b..00000000000 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/OracleConnection.java +++ /dev/null @@ -1,640 +0,0 @@ -/* - * Copyright 2023 Ververica Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package io.debezium.connector.oracle; - -import io.debezium.DebeziumException; -import io.debezium.config.Field; -import io.debezium.connector.oracle.OracleConnectorConfig.ConnectorAdapter; -import io.debezium.jdbc.JdbcConfiguration; -import io.debezium.jdbc.JdbcConnection; -import io.debezium.relational.Column; -import io.debezium.relational.ColumnEditor; -import io.debezium.relational.TableId; -import io.debezium.relational.Tables; -import io.debezium.relational.Tables.ColumnNameFilter; -import io.debezium.util.Strings; -import oracle.jdbc.OracleTypes; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.sql.Clob; -import java.sql.DatabaseMetaData; -import java.sql.PreparedStatement; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.sql.Statement; -import java.sql.Timestamp; -import java.time.OffsetDateTime; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.function.Supplier; -import java.util.regex.Pattern; -import java.util.stream.Collectors; - -/** - * Copied from Debezium 1.9.7.Final. - * - *

    Line 213-225: Fixed for DBZ-5738. - */ -public class OracleConnection extends JdbcConnection { - - private static final Logger LOGGER = LoggerFactory.getLogger(OracleConnection.class); - - /** Returned by column metadata in Oracle if no scale is set; */ - private static final int ORACLE_UNSET_SCALE = -127; - - /** Pattern to identify system generated indices and column names. */ - private static final Pattern SYS_NC_PATTERN = - Pattern.compile("^SYS_NC(?:_OID|_ROWINFO|[0-9][0-9][0-9][0-9][0-9])\\$$"); - - /** Pattern to identify abstract data type indices and column names. */ - private static final Pattern ADT_INDEX_NAMES_PATTERN = Pattern.compile("^\".*\"\\.\".*\".*"); - - /** - * Pattern to identify a hidden column based on redefining a table with the {@code ROWID} - * option. - */ - private static final Pattern MROW_PATTERN = Pattern.compile("^M_ROW\\$\\$"); - - /** A field for the raw jdbc url. This field has no default value. */ - private static final Field URL = Field.create("url", "Raw JDBC url"); - - /** The database version. */ - private final OracleDatabaseVersion databaseVersion; - - private static final String QUOTED_CHARACTER = "\""; - - public OracleConnection(JdbcConfiguration config, Supplier classLoaderSupplier) { - this(config, classLoaderSupplier, true); - } - - public OracleConnection( - JdbcConfiguration config, - Supplier classLoaderSupplier, - boolean showVersion) { - super( - config, - resolveConnectionFactory(config), - classLoaderSupplier, - QUOTED_CHARACTER, - QUOTED_CHARACTER); - this.databaseVersion = resolveOracleDatabaseVersion(); - if (showVersion) { - LOGGER.info("Database Version: {}", databaseVersion.getBanner()); - } - } - - public void setSessionToPdb(String pdbName) { - Statement statement = null; - - try { - statement = connection().createStatement(); - statement.execute("alter session set container=" + pdbName); - } catch (SQLException e) { - throw new RuntimeException(e); - } finally { - if (statement != null) { - try { - statement.close(); - } catch (SQLException e) { - LOGGER.error("Couldn't close statement", e); - } - } - } - } - - public void resetSessionToCdb() { - Statement statement = null; - - try { - statement = connection().createStatement(); - statement.execute("alter session set container=cdb$root"); - } catch (SQLException e) { - throw new RuntimeException(e); - } finally { - if (statement != null) { - try { - statement.close(); - } catch (SQLException e) { - LOGGER.error("Couldn't close statement", e); - } - } - } - } - - public OracleDatabaseVersion getOracleVersion() { - return databaseVersion; - } - - private OracleDatabaseVersion resolveOracleDatabaseVersion() { - String versionStr; - try { - try { - // Oracle 18.1 introduced BANNER_FULL as the new column rather than BANNER - // This column uses a different format than the legacy BANNER. - versionStr = - queryAndMap( - "SELECT BANNER_FULL FROM V$VERSION WHERE BANNER_FULL LIKE 'Oracle Database%'", - (rs) -> { - if (rs.next()) { - return rs.getString(1); - } - return null; - }); - } catch (SQLException e) { - // exception ignored - if (e.getMessage().contains("ORA-00904: \"BANNER_FULL\"")) { - LOGGER.debug( - "BANNER_FULL column not in V$VERSION, using BANNER column as fallback"); - versionStr = null; - } else { - throw e; - } - } - - // For databases prior to 18.1, a SQLException will be thrown due to BANNER_FULL not - // being a column and - // this will cause versionStr to remain null, use fallback column BANNER for versions - // prior to 18.1. - if (versionStr == null) { - versionStr = - queryAndMap( - "SELECT BANNER FROM V$VERSION WHERE BANNER LIKE 'Oracle Database%'", - (rs) -> { - if (rs.next()) { - return rs.getString(1); - } - return null; - }); - } - } catch (SQLException e) { - throw new RuntimeException("Failed to resolve Oracle database version", e); - } - - if (versionStr == null) { - throw new RuntimeException("Failed to resolve Oracle database version"); - } - - return OracleDatabaseVersion.parse(versionStr); - } - - @Override - public Set readTableNames( - String databaseCatalog, - String schemaNamePattern, - String tableNamePattern, - String[] tableTypes) - throws SQLException { - - Set tableIds = - super.readTableNames(null, schemaNamePattern, tableNamePattern, tableTypes); - - return tableIds.stream() - .map(t -> new TableId(databaseCatalog, t.schema(), t.table())) - .collect(Collectors.toSet()); - } - - /** - * Retrieves all {@code TableId} in a given database catalog, filtering certain ids that should - * be omitted from the returned set such as special spatial tables and index-organized tables. - * - * @param catalogName the catalog/database name - * @return set of all table ids for existing table objects - * @throws SQLException if a database exception occurred - */ - protected Set getAllTableIds(String catalogName) throws SQLException { - final String query = - "select owner, table_name from all_tables " - + - // filter special spatial tables - "where table_name NOT LIKE 'MDRT_%' " - + "and table_name NOT LIKE 'MDRS_%' " - + "and table_name NOT LIKE 'MDXT_%' " - + - // filter index-organized-tables - "and (table_name NOT LIKE 'SYS_IOT_OVER_%' and IOT_NAME IS NULL) " - // filter nested tables - + "and nested = 'NO'" - // filter parent tables of nested tables - + "and table_name not in (select PARENT_TABLE_NAME from ALL_NESTED_TABLES)"; - - Set tableIds = new HashSet<>(); - query( - query, - (rs) -> { - while (rs.next()) { - tableIds.add(new TableId(catalogName, rs.getString(1), rs.getString(2))); - } - LOGGER.trace("TableIds are: {}", tableIds); - }); - - return tableIds; - } - - // todo replace metadata with something like this - private ResultSet getTableColumnsInfo(String schemaNamePattern, String tableName) - throws SQLException { - String columnQuery = - "select column_name, data_type, data_length, data_precision, data_scale, default_length, density, char_length from " - + "all_tab_columns where owner like '" - + schemaNamePattern - + "' and table_name='" - + tableName - + "'"; - - PreparedStatement statement = connection().prepareStatement(columnQuery); - return statement.executeQuery(); - } - - // this is much faster, we will use it until full replacement of the metadata usage TODO - public void readSchemaForCapturedTables( - Tables tables, - String databaseCatalog, - String schemaNamePattern, - ColumnNameFilter columnFilter, - boolean removeTablesNotFoundInJdbc, - Set capturedTables) - throws SQLException { - - Set tableIdsBefore = new HashSet<>(tables.tableIds()); - - DatabaseMetaData metadata = connection().getMetaData(); - Map> columnsByTable = new HashMap<>(); - - for (TableId tableId : capturedTables) { - try (ResultSet columnMetadata = - metadata.getColumns( - databaseCatalog, schemaNamePattern, tableId.table(), null)) { - while (columnMetadata.next()) { - // add all whitelisted columns - readTableColumn(columnMetadata, tableId, columnFilter) - .ifPresent( - column -> { - columnsByTable - .computeIfAbsent(tableId, t -> new ArrayList<>()) - .add(column.create()); - }); - } - } - } - - // Read the metadata for the primary keys ... - for (Map.Entry> tableEntry : columnsByTable.entrySet()) { - // First get the primary key information, which must be done for *each* table ... - List pkColumnNames = readPrimaryKeyNames(metadata, tableEntry.getKey()); - - // Then define the table ... - List columns = tableEntry.getValue(); - Collections.sort(columns); - tables.overwriteTable(tableEntry.getKey(), columns, pkColumnNames, null); - } - - if (removeTablesNotFoundInJdbc) { - // Remove any definitions for tables that were not found in the database metadata ... - tableIdsBefore.removeAll(columnsByTable.keySet()); - tableIdsBefore.forEach(tables::removeTable); - } - } - - @Override - protected String resolveCatalogName(String catalogName) { - final String pdbName = config().getString("pdb.name"); - return (!Strings.isNullOrEmpty(pdbName) ? pdbName : config().getString("dbname")) - .toUpperCase(); - } - - @Override - public List readTableUniqueIndices(DatabaseMetaData metadata, TableId id) - throws SQLException { - return super.readTableUniqueIndices(metadata, id.toDoubleQuoted()); - } - - @Override - public Optional getCurrentTimestamp() throws SQLException { - return queryAndMap( - "SELECT CURRENT_TIMESTAMP FROM DUAL", - rs -> rs.next() ? Optional.of(rs.getTimestamp(1)) : Optional.empty()); - } - - @Override - protected boolean isTableUniqueIndexIncluded(String indexName, String columnName) { - if (columnName != null) { - return !SYS_NC_PATTERN.matcher(columnName).matches() - && !ADT_INDEX_NAMES_PATTERN.matcher(columnName).matches() - && !MROW_PATTERN.matcher(columnName).matches(); - } - return false; - } - - /** - * Get the current, most recent system change number. - * - * @return the current system change number - * @throws SQLException if an exception occurred - * @throws IllegalStateException if the query does not return at least one row - */ - public Scn getCurrentScn() throws SQLException { - return queryAndMap( - "SELECT CURRENT_SCN FROM V$DATABASE", - (rs) -> { - if (rs.next()) { - return Scn.valueOf(rs.getString(1)); - } - throw new IllegalStateException("Could not get SCN"); - }); - } - - /** - * Generate a given table's DDL metadata. - * - * @param tableId table identifier, should never be {@code null} - * @return generated DDL - * @throws SQLException if an exception occurred obtaining the DDL metadata - * @throws NonRelationalTableException the table is not a relational table - */ - public String getTableMetadataDdl(TableId tableId) - throws SQLException, NonRelationalTableException { - try { - // This table contains all available objects that are considered relational & object - // based. - // By querying for TABLE_TYPE is null, we are explicitly confirming what if an entry - // exists - // that the table is in-fact a relational table and if the result set is empty, the - // object - // is another type, likely an object-based table, in which case we cannot generate DDL. - final String tableType = - "SELECT COUNT(1) FROM ALL_ALL_TABLES WHERE OWNER='" - + tableId.schema() - + "' AND TABLE_NAME='" - + tableId.table() - + "' AND TABLE_TYPE IS NULL"; - if (queryAndMap(tableType, rs -> rs.next() ? rs.getInt(1) : 0) == 0) { - throw new NonRelationalTableException( - "Table " + tableId + " is not a relational table"); - } - - // The storage and segment attributes aren't necessary - executeWithoutCommitting( - "begin dbms_metadata.set_transform_param(DBMS_METADATA.SESSION_TRANSFORM, 'STORAGE', false); end;"); - executeWithoutCommitting( - "begin dbms_metadata.set_transform_param(DBMS_METADATA.SESSION_TRANSFORM, 'SEGMENT_ATTRIBUTES', false); end;"); - // In case DDL is returned as multiple DDL statements, this allows the parser to parse - // each separately. - // This is only critical during streaming as during snapshot the table structure is - // built from JDBC driver queries. - executeWithoutCommitting( - "begin dbms_metadata.set_transform_param(DBMS_METADATA.SESSION_TRANSFORM, 'SQLTERMINATOR', true); end;"); - return queryAndMap( - "SELECT dbms_metadata.get_ddl('TABLE','" - + tableId.table() - + "','" - + tableId.schema() - + "') FROM DUAL", - rs -> { - if (!rs.next()) { - throw new DebeziumException( - "Could not get DDL metadata for table: " + tableId); - } - - Object res = rs.getObject(1); - return ((Clob) res).getSubString(1, (int) ((Clob) res).length()); - }); - } finally { - executeWithoutCommitting( - "begin dbms_metadata.set_transform_param(DBMS_METADATA.SESSION_TRANSFORM, 'DEFAULT'); end;"); - } - } - - /** - * Get the current connection's session statistic by name. - * - * @param name the name of the statistic to be fetched, must not be {@code null} - * @return the session statistic value, never {@code null} - * @throws SQLException if an exception occurred obtaining the session statistic value - */ - public Long getSessionStatisticByName(String name) throws SQLException { - return queryAndMap( - "SELECT VALUE FROM v$statname n, v$mystat m WHERE n.name='" - + name - + "' AND n.statistic#=m.statistic#", - rs -> rs.next() ? rs.getLong(1) : 0L); - } - - /** - * Returns whether the given table exists or not. - * - * @param tableName table name, should not be {@code null} - * @return true if the table exists, false if it does not - * @throws SQLException if a database exception occurred - */ - public boolean isTableExists(String tableName) throws SQLException { - return queryAndMap( - "SELECT COUNT(1) FROM USER_TABLES WHERE TABLE_NAME = '" + tableName + "'", - rs -> rs.next() && rs.getLong(1) > 0); - } - - public boolean isTableExists(TableId tableId) throws SQLException { - return queryAndMap( - "SELECT COUNT(1) FROM ALL_TABLES WHERE OWNER = '" - + tableId.schema() - + "' AND TABLE_NAME = '" - + tableId.table() - + "'", - rs -> rs.next() && rs.getLong(1) > 0); - } - - /** - * Returns whether the given table is empty or not. - * - * @param tableName table name, should not be {@code null} - * @return true if the table has no records, false otherwise - * @throws SQLException if a database exception occurred - */ - public boolean isTableEmpty(String tableName) throws SQLException { - return getRowCount(tableName) == 0L; - } - - /** - * Returns the number of rows in a given table. - * - * @param tableName table name, should not be {@code null} - * @return the number of rows - * @throws SQLException if a database exception occurred - */ - public long getRowCount(String tableName) throws SQLException { - return queryAndMap( - "SELECT COUNT(1) FROM " + tableName, - rs -> { - if (rs.next()) { - return rs.getLong(1); - } - return 0L; - }); - } - - public T singleOptionalValue(String query, ResultSetExtractor extractor) - throws SQLException { - return queryAndMap(query, rs -> rs.next() ? extractor.apply(rs) : null); - } - - @Override - public String buildSelectWithRowLimits( - TableId tableId, - int limit, - String projection, - Optional condition, - String orderBy) { - final TableId table = new TableId(null, tableId.schema(), tableId.table()); - final StringBuilder sql = new StringBuilder("SELECT "); - sql.append(projection).append(" FROM "); - sql.append(quotedTableIdString(table)); - if (condition.isPresent()) { - sql.append(" WHERE ").append(condition.get()); - } - if (getOracleVersion().getMajor() < 12) { - sql.insert(0, " SELECT * FROM (") - .append(" ORDER BY ") - .append(orderBy) - .append(")") - .append(" WHERE ROWNUM <=") - .append(limit); - } else { - sql.append(" ORDER BY ") - .append(orderBy) - .append(" FETCH NEXT ") - .append(limit) - .append(" ROWS ONLY"); - } - return sql.toString(); - } - - public static String connectionString(JdbcConfiguration config) { - return config.getString(URL) != null - ? config.getString(URL) - : ConnectorAdapter.parse(config.getString("connection.adapter")).getConnectionUrl(); - } - - private static ConnectionFactory resolveConnectionFactory(JdbcConfiguration config) { - return JdbcConnection.patternBasedFactory(connectionString(config)); - } - - /** - * Determine whether the Oracle server has the archive log enabled. - * - * @return {@code true} if the server's {@code LOG_MODE} is set to {@code ARCHIVELOG}, or {@code - * false} otherwise - */ - protected boolean isArchiveLogMode() { - try { - final String mode = - queryAndMap( - "SELECT LOG_MODE FROM V$DATABASE", - rs -> rs.next() ? rs.getString(1) : ""); - LOGGER.debug("LOG_MODE={}", mode); - return "ARCHIVELOG".equalsIgnoreCase(mode); - } catch (SQLException e) { - throw new DebeziumException( - "Unexpected error while connecting to Oracle and looking at LOG_MODE mode: ", - e); - } - } - - /** - * Resolve a system change number to a timestamp, return value is in database timezone. - * - *

    The SCN to TIMESTAMP mapping is only retained for the duration of the flashback query - * area. This means that eventually the mapping between these values are no longer kept by - * Oracle and making a call with a SCN value that has aged out will result in an ORA-08181 - * error. This function explicitly checks for this use case and if a ORA-08181 error is thrown, - * it is therefore treated as if a value does not exist returning an empty optional value. - * - * @param scn the system change number, must not be {@code null} - * @return an optional timestamp when the system change number occurred - * @throws SQLException if a database exception occurred - */ - public Optional getScnToTimestamp(Scn scn) throws SQLException { - try { - return queryAndMap( - "SELECT scn_to_timestamp('" + scn + "') FROM DUAL", - rs -> - rs.next() - ? Optional.of(rs.getObject(1, OffsetDateTime.class)) - : Optional.empty()); - } catch (SQLException e) { - if (e.getMessage().startsWith("ORA-08181")) { - // ORA-08181 specified number is not a valid system change number - // This happens when the SCN provided is outside the flashback area range - // This should be treated as a value is not available rather than an error - return Optional.empty(); - } - // Any other SQLException should be thrown - throw e; - } - } - - @Override - protected ColumnEditor overrideColumn(ColumnEditor column) { - // This allows the column state to be overridden before default-value resolution so that the - // output of the default value is within the same precision as that of the column values. - if (OracleTypes.TIMESTAMP == column.jdbcType()) { - column.length(column.scale().orElse(Column.UNSET_INT_VALUE)).scale(null); - } else if (OracleTypes.NUMBER == column.jdbcType()) { - column.scale().filter(s -> s == ORACLE_UNSET_SCALE).ifPresent(s -> column.scale(null)); - } - return column; - } - - @Override - protected Map> getColumnsDetails( - String databaseCatalog, - String schemaNamePattern, - String tableName, - Tables.TableFilter tableFilter, - ColumnNameFilter columnFilter, - DatabaseMetaData metadata, - Set viewIds) - throws SQLException { - // The Oracle JDBC driver expects that if the table name contains a "/" character that - // the table name is pre-escaped prior to the JDBC driver call, or else it throws an - // exception about the character sequence being improperly escaped. - if (tableName != null && tableName.contains("/")) { - tableName = tableName.replace("/", "//"); - } - return super.getColumnsDetails( - databaseCatalog, - schemaNamePattern, - tableName, - tableFilter, - columnFilter, - metadata, - viewIds); - } - - /** - * An exception that indicates the operation failed because the table is not a relational table. - */ - public static class NonRelationalTableException extends DebeziumException { - public NonRelationalTableException(String message) { - super(message); - } - } -} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/logminer/LogMinerAdapter.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/logminer/LogMinerAdapter.java index 53387e3cdbd..59a5cdd6621 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/logminer/LogMinerAdapter.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/logminer/LogMinerAdapter.java @@ -58,7 +58,7 @@ import java.util.stream.Collectors; /** - * Copied from Debezium 1.9.7. + * Copied from Debezium 1.9.8.Final. * *

    Line 356: Replace < condition with <= to be able to catch ongoing transactions during snapshot * if current SCN points to START/INSERT/DELETE/UPDATE event. diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/logminer/LogMinerChangeRecordEmitter.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/logminer/LogMinerChangeRecordEmitter.java index 4a743dfa4f2..4969c1abdca 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/logminer/LogMinerChangeRecordEmitter.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/logminer/LogMinerChangeRecordEmitter.java @@ -22,7 +22,12 @@ import java.util.Optional; -/** Emits change records based on an event read from Oracle LogMiner. */ +/** + * Copied from Debezium 1.9.8.Final. Emits change records based on an event read from Oracle + * LogMiner. + * + *

    This class add RowId and overrides the emit methods to put rowId in the header. + */ public class LogMinerChangeRecordEmitter extends BaseChangeRecordEmitter { private final Operation operation; diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/logminer/LogMinerStreamingChangeEventSource.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/logminer/LogMinerStreamingChangeEventSource.java index 216c1e31310..76a6d2bb83f 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/logminer/LogMinerStreamingChangeEventSource.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/logminer/LogMinerStreamingChangeEventSource.java @@ -63,9 +63,11 @@ import static io.debezium.connector.oracle.logminer.LogMinerHelper.setLogFilesForMining; /** - * Copied from Debezium 1.9.7. Diff: added afterHandleScn() method. A {@link - * StreamingChangeEventSource} based on Oracle's LogMiner utility. The event handler loop is - * executed in a separate executor. + * Copied from Debezium 1.9.8.Final. A {@link StreamingChangeEventSource} based on Oracle's LogMiner + * utility. The event handler loop is executed in a separate executor. + * + *

    Diff: Make createProcessor method as protected to produce a LogMinerEventProcessor with + * enhanced processRow method to distinguish whether is bounded. */ public class LogMinerStreamingChangeEventSource implements StreamingChangeEventSource { @@ -251,8 +253,6 @@ public void execute( } pauseBetweenMiningSessions(); } - - afterHandleScn(partition, offsetContext); } } } @@ -266,8 +266,6 @@ public void execute( } } - protected void afterHandleScn(OraclePartition partition, OracleOffsetContext offsetContext) {} - /** * Computes the start SCN for the first mining session. * @@ -361,7 +359,7 @@ private void captureSessionMemoryStatistics(OracleConnection connection) throws format.format(sessionProcessGlobalAreaMaxMemory / 1024.f / 1024.f)); } - private LogMinerEventProcessor createProcessor( + protected LogMinerEventProcessor createProcessor( ChangeEventSourceContext context, OraclePartition partition, OracleOffsetContext offsetContext) { diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/logminer/processor/AbstractLogMinerEventProcessor.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/logminer/processor/AbstractLogMinerEventProcessor.java index af868c9bb91..8a0b516678d 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/logminer/processor/AbstractLogMinerEventProcessor.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/io/debezium/connector/oracle/logminer/processor/AbstractLogMinerEventProcessor.java @@ -54,7 +54,7 @@ import java.util.regex.Pattern; /** - * Copied from Debezium 1.9.7.Final. + * Copied from Debezium 1.9.8.Final. * *

    An abstract implementation of {@link LogMinerEventProcessor} that all processors should * extend. diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/test/java/com/ververica/cdc/connectors/oracle/source/OracleSourceITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/test/java/com/ververica/cdc/connectors/oracle/source/OracleSourceITCase.java index ce158932134..dd5572667e5 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/test/java/com/ververica/cdc/connectors/oracle/source/OracleSourceITCase.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/test/java/com/ververica/cdc/connectors/oracle/source/OracleSourceITCase.java @@ -65,6 +65,7 @@ public class OracleSourceITCase extends OracleSourceTestBase { private static final int USE_POST_LOWWATERMARK_HOOK = 1; private static final int USE_PRE_HIGHWATERMARK_HOOK = 2; + private static final int USE_POST_HIGHWATERMARK_HOOK = 3; private static final Logger LOG = LoggerFactory.getLogger(OracleSourceITCase.class); @@ -194,6 +195,39 @@ public void testEnableBackfillWithDMLPostLowWaterMark() throws Exception { assertEqualsInAnyOrder(expectedRecords, records); } + @Test + public void testEnableBackfillWithDMLPostHighWaterMark() throws Exception { + List records = + testBackfillWhenWritingEvents(false, 21, USE_POST_HIGHWATERMARK_HOOK); + + List expectedRecords = + Arrays.asList( + "+I[101, user_1, Shanghai, 123567891234]", + "+I[102, user_2, Shanghai, 123567891234]", + "+I[103, user_3, Shanghai, 123567891234]", + "+I[109, user_4, Shanghai, 123567891234]", + "+I[110, user_5, Shanghai, 123567891234]", + "+I[111, user_6, Shanghai, 123567891234]", + "+I[118, user_7, Shanghai, 123567891234]", + "+I[121, user_8, Shanghai, 123567891234]", + "+I[123, user_9, Shanghai, 123567891234]", + "+I[1009, user_10, Shanghai, 123567891234]", + "+I[1010, user_11, Shanghai, 123567891234]", + "+I[1011, user_12, Shanghai, 123567891234]", + "+I[1012, user_13, Shanghai, 123567891234]", + "+I[1013, user_14, Shanghai, 123567891234]", + "+I[1014, user_15, Shanghai, 123567891234]", + "+I[1015, user_16, Shanghai, 123567891234]", + "+I[1016, user_17, Shanghai, 123567891234]", + "+I[1017, user_18, Shanghai, 123567891234]", + "+I[1018, user_19, Shanghai, 123567891234]", + "+I[1019, user_20, Shanghai, 123567891234]", + "+I[2000, user_21, Shanghai, 123567891234]"); + // when enable backfill, the wal log between [low_watermark, snapshot) will be applied + // as snapshot image + assertEqualsInAnyOrder(expectedRecords, records); + } + @Test public void testSkipBackfillWithDMLPreHighWaterMark() throws Exception { List records = testBackfillWhenWritingEvents(true, 25, USE_PRE_HIGHWATERMARK_HOOK); @@ -275,7 +309,7 @@ private List testBackfillWhenWritingEvents( env.enableCheckpointing(200L); env.setParallelism(1); - ResolvedSchema customersSchame = + ResolvedSchema customersSchema = new ResolvedSchema( Arrays.asList( physical("ID", BIGINT().notNull()), @@ -285,7 +319,7 @@ private List testBackfillWhenWritingEvents( new ArrayList<>(), UniqueConstraint.primaryKey("pk", Collections.singletonList("ID"))); TestTable customerTable = - new TestTable(ORACLE_DATABASE, ORACLE_SCHEMA, "CUSTOMERS", customersSchame); + new TestTable(ORACLE_DATABASE, ORACLE_SCHEMA, "CUSTOMERS", customersSchema); String tableId = customerTable.getTableId(); OracleSourceBuilder.OracleIncrementalSource source = @@ -326,16 +360,19 @@ private List testBackfillWhenWritingEvents( try (OracleConnection oracleConnection = OracleConnectionUtils.createOracleConnection(configuration)) { oracleConnection.execute(statements); - Thread.sleep(500L); - } catch (InterruptedException e) { - throw new RuntimeException(e); } }; - if (hookType == USE_POST_LOWWATERMARK_HOOK) { - hooks.setPostLowWatermarkAction(snapshotPhaseHook); - } else if (hookType == USE_PRE_HIGHWATERMARK_HOOK) { - hooks.setPreHighWatermarkAction(snapshotPhaseHook); + switch (hookType) { + case USE_POST_LOWWATERMARK_HOOK: + hooks.setPostLowWatermarkAction(snapshotPhaseHook); + break; + case USE_PRE_HIGHWATERMARK_HOOK: + hooks.setPreHighWatermarkAction(snapshotPhaseHook); + break; + case USE_POST_HIGHWATERMARK_HOOK: + hooks.setPostHighWatermarkAction(snapshotPhaseHook); + break; } source.setSnapshotHooks(hooks); diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/PostgreSQLSource.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/PostgreSQLSource.java index 21ee0219a3b..9468adbb357 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/PostgreSQLSource.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/PostgreSQLSource.java @@ -30,6 +30,7 @@ * A builder to build a SourceFunction which can read snapshot and continue to consume binlog for * PostgreSQL. */ +@Deprecated public class PostgreSQLSource { private static final long DEFAULT_HEARTBEAT_MS = Duration.ofMinutes(5).toMillis(); diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/PostgresDialect.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/PostgresDialect.java index 325dc9cf74f..806745e0b24 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/PostgresDialect.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/PostgresDialect.java @@ -214,9 +214,9 @@ public JdbcSourceFetchTaskContext createFetchTaskContext( } @Override - public void notifyCheckpointComplete(long checkpointId) throws Exception { + public void notifyCheckpointComplete(long checkpointId, Offset offset) throws Exception { if (streamFetchTask != null) { - streamFetchTask.commitCurrentOffset(); + streamFetchTask.commitCurrentOffset(offset); } } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/PostgresSourceBuilder.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/PostgresSourceBuilder.java index c3ec1ee0e60..9a054d49cb2 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/PostgresSourceBuilder.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/PostgresSourceBuilder.java @@ -16,19 +16,27 @@ package com.ververica.cdc.connectors.postgres.source; +import org.apache.flink.api.connector.source.SourceReaderContext; import org.apache.flink.api.connector.source.SplitEnumerator; import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; import org.apache.flink.util.FlinkRuntimeException; import com.ververica.cdc.common.annotation.Experimental; -import com.ververica.cdc.connectors.base.options.StartupMode; +import com.ververica.cdc.connectors.base.config.JdbcSourceConfig; import com.ververica.cdc.connectors.base.options.StartupOptions; import com.ververica.cdc.connectors.base.source.assigner.HybridSplitAssigner; import com.ververica.cdc.connectors.base.source.assigner.SplitAssigner; import com.ververica.cdc.connectors.base.source.assigner.StreamSplitAssigner; import com.ververica.cdc.connectors.base.source.assigner.state.PendingSplitsState; import com.ververica.cdc.connectors.base.source.jdbc.JdbcIncrementalSource; +import com.ververica.cdc.connectors.base.source.meta.split.SourceRecords; import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitBase; +import com.ververica.cdc.connectors.base.source.metrics.SourceReaderMetrics; +import com.ververica.cdc.connectors.base.source.reader.IncrementalSourceReader; +import com.ververica.cdc.connectors.base.source.reader.IncrementalSourceReaderWithCommit; +import com.ververica.cdc.connectors.base.source.reader.IncrementalSourceSplitReader; import com.ververica.cdc.connectors.postgres.source.config.PostgresSourceConfig; import com.ververica.cdc.connectors.postgres.source.config.PostgresSourceConfigFactory; import com.ververica.cdc.connectors.postgres.source.enumerator.PostgresSourceEnumerator; @@ -39,6 +47,7 @@ import java.time.Duration; import java.util.List; import java.util.Properties; +import java.util.function.Supplier; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -286,12 +295,42 @@ public PostgresIncrementalSource( super(configFactory, deserializationSchema, offsetFactory, dataSourceDialect); } + @Override + public IncrementalSourceReader createReader( + SourceReaderContext readerContext) throws Exception { + // create source config for the given subtask (e.g. unique server id) + JdbcSourceConfig sourceConfig = configFactory.create(readerContext.getIndexOfSubtask()); + FutureCompletingBlockingQueue> elementsQueue = + new FutureCompletingBlockingQueue<>(); + + final SourceReaderMetrics sourceReaderMetrics = + new SourceReaderMetrics(readerContext.metricGroup()); + + sourceReaderMetrics.registerMetrics(); + Supplier> splitReaderSupplier = + () -> + new IncrementalSourceSplitReader<>( + readerContext.getIndexOfSubtask(), + dataSourceDialect, + sourceConfig, + snapshotHooks); + return new IncrementalSourceReaderWithCommit( + elementsQueue, + splitReaderSupplier, + createRecordEmitter(sourceConfig, sourceReaderMetrics), + readerContext.getConfiguration(), + readerContext, + sourceConfig, + sourceSplitSerializer, + dataSourceDialect); + } + @Override public SplitEnumerator createEnumerator( SplitEnumeratorContext enumContext) { final SplitAssigner splitAssigner; PostgresSourceConfig sourceConfig = (PostgresSourceConfig) configFactory.create(0); - if (sourceConfig.getStartupOptions().startupMode == StartupMode.INITIAL) { + if (!sourceConfig.getStartupOptions().isStreamOnly()) { try { final List remainingTables = dataSourceDialect.discoverDataCollections(sourceConfig); @@ -315,7 +354,11 @@ public SplitEnumerator createEnumerator( } return new PostgresSourceEnumerator( - enumContext, sourceConfig, splitAssigner, (PostgresDialect) dataSourceDialect); + enumContext, + sourceConfig, + splitAssigner, + (PostgresDialect) dataSourceDialect, + this.getBoundedness()); } public static PostgresSourceBuilder builder() { diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/enumerator/PostgresSourceEnumerator.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/enumerator/PostgresSourceEnumerator.java index 40164b9e462..485efbdbea3 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/enumerator/PostgresSourceEnumerator.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/enumerator/PostgresSourceEnumerator.java @@ -16,9 +16,11 @@ package com.ververica.cdc.connectors.postgres.source.enumerator; +import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.connector.source.SplitEnumeratorContext; import org.apache.flink.util.FlinkRuntimeException; +import com.ververica.cdc.common.annotation.Internal; import com.ververica.cdc.connectors.base.source.assigner.SplitAssigner; import com.ververica.cdc.connectors.base.source.enumerator.IncrementalSourceEnumerator; import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitBase; @@ -32,6 +34,7 @@ * The Postgres source enumerator that enumerates receive the split request and assign the split to * source readers. */ +@Internal public class PostgresSourceEnumerator extends IncrementalSourceEnumerator { private final PostgresDialect postgresDialect; @@ -40,8 +43,9 @@ public PostgresSourceEnumerator( SplitEnumeratorContext context, PostgresSourceConfig sourceConfig, SplitAssigner splitAssigner, - PostgresDialect postgresDialect) { - super(context, sourceConfig, splitAssigner); + PostgresDialect postgresDialect, + Boundedness boundedness) { + super(context, sourceConfig, splitAssigner, boundedness); this.postgresDialect = postgresDialect; } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/fetch/PostgresScanFetchTask.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/fetch/PostgresScanFetchTask.java index 24425bcf5d6..c01afc1c4e4 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/fetch/PostgresScanFetchTask.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/fetch/PostgresScanFetchTask.java @@ -36,7 +36,6 @@ import io.debezium.connector.postgresql.spi.SlotState; import io.debezium.pipeline.EventDispatcher; import io.debezium.pipeline.source.AbstractSnapshotChangeEventSource; -import io.debezium.pipeline.source.spi.ChangeEventSource; import io.debezium.pipeline.source.spi.SnapshotProgressListener; import io.debezium.pipeline.spi.SnapshotResult; import io.debezium.relational.RelationalSnapshotChangeEventSource; @@ -104,8 +103,8 @@ protected void executeDataSnapshot(Context context) throws Exception { ctx.getSnapshotChangeEventSourceMetrics(), snapshotSplit); - PostgresChangeEventSourceContext changeEventSourceContext = - new PostgresChangeEventSourceContext(); + StoppableChangeEventSourceContext changeEventSourceContext = + new StoppableChangeEventSourceContext(); SnapshotResult snapshotResult = snapshotSplitReadTask.execute( changeEventSourceContext, ctx.getPartition(), ctx.getOffsetContext()); @@ -146,7 +145,7 @@ protected void executeBackfillTask(Context context, StreamSplit backfillStreamSp snapshotSplit, ((PostgresSourceConfig) ctx.getSourceConfig()).getSlotNameForBackfillTask()); backfillReadTask.execute( - new PostgresChangeEventSourceContext(), ctx.getPartition(), postgresOffsetContext); + new StoppableChangeEventSourceContext(), ctx.getPartition(), postgresOffsetContext); } /** @@ -205,18 +204,6 @@ private void maybeDropSlotForBackFillReadTask( } } - class PostgresChangeEventSourceContext implements ChangeEventSource.ChangeEventSourceContext { - - public void finished() { - taskRunning = false; - } - - @Override - public boolean isRunning() { - return taskRunning; - } - } - /** A SnapshotChangeEventSource implementation for Postgres to read snapshot split. */ public static class PostgresSnapshotSplitReadTask extends AbstractSnapshotChangeEventSource { diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/fetch/PostgresStreamFetchTask.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/fetch/PostgresStreamFetchTask.java index 40ed521fe35..30cc32bb2d9 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/fetch/PostgresStreamFetchTask.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/fetch/PostgresStreamFetchTask.java @@ -19,6 +19,7 @@ import org.apache.flink.util.FlinkRuntimeException; import com.ververica.cdc.connectors.base.relational.JdbcSourceEventDispatcher; +import com.ververica.cdc.connectors.base.source.meta.offset.Offset; import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitBase; import com.ververica.cdc.connectors.base.source.meta.split.StreamSplit; import com.ververica.cdc.connectors.base.source.meta.wartermark.WatermarkKind; @@ -42,6 +43,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + import java.util.HashMap; import java.util.Map; @@ -90,8 +93,8 @@ public void execute(Context context) throws Exception { sourceFetchContext.getTaskContext(), sourceFetchContext.getReplicationConnection(), split); - StreamSplitChangeEventSourceContext changeEventSourceContext = - new StreamSplitChangeEventSourceContext(); + StoppableChangeEventSourceContext changeEventSourceContext = + new StoppableChangeEventSourceContext(); streamSplitReadTask.execute( changeEventSourceContext, sourceFetchContext.getPartition(), @@ -102,7 +105,8 @@ public void execute(Context context) throws Exception { public void close() { LOG.debug("stopping StreamFetchTask for split: {}", split); if (streamSplitReadTask != null) { - ((StreamSplitChangeEventSourceContext) streamSplitReadTask.context).finished(); + ((StoppableChangeEventSourceContext) (streamSplitReadTask.context)) + .stopChangeEventSource(); } stopped = true; taskRunning = false; @@ -118,7 +122,7 @@ public SourceSplitBase getSplit() { return split; } - public void commitCurrentOffset() { + public void commitCurrentOffset(@Nullable Offset offsetToCommit) { if (streamSplitReadTask != null && streamSplitReadTask.offsetContext != null) { PostgresOffsetContext postgresOffsetContext = streamSplitReadTask.offsetContext; @@ -128,6 +132,21 @@ public void commitCurrentOffset() { postgresOffsetContext .getOffset() .get(PostgresOffsetContext.LAST_COMMIT_LSN_KEY); + + if (offsetToCommit != null) { + // We should commit the checkpoint's LSN instead of postgresOffsetContext's LSN to + // the slot. + // If the checkpoint succeeds and a table UPDATE message arrives before the + // notifyCheckpoint is called, which is represented as a BEGIN/UPDATE/COMMIT WAL + // event sequence. The LSN of postgresOffsetContext will be updated to the LSN of + // the COMMIT event. Committing the COMMIT LSN to the slot is incorrect because if a + // failover occurs after the successful commission, Flink will recover from that + // checkpoint and consume WAL starting from the slot LSN that is the LSN of COMMIT + // event, rather than from the checkpoint's LSN. Therefore, UPDATE messages cannot + // be consumed, resulting in data loss. + commitLsn = ((PostgresOffset) offsetToCommit).getLsn().asLong(); + } + if (commitLsn != null && (lastCommitLsn == null || Lsn.valueOf(commitLsn).compareTo(Lsn.valueOf(lastCommitLsn)) > 0)) { @@ -144,19 +163,6 @@ public void commitCurrentOffset() { } } - private class StreamSplitChangeEventSourceContext - implements ChangeEventSource.ChangeEventSourceContext { - - public void finished() { - taskRunning = false; - } - - @Override - public boolean isRunning() { - return taskRunning; - } - } - /** A {@link ChangeEventSource} implementation for Postgres to read streaming changes. */ public static class StreamSplitReadTask extends PostgresStreamingChangeEventSource { private static final Logger LOG = LoggerFactory.getLogger(StreamSplitReadTask.class); @@ -228,7 +234,7 @@ public void execute( new FlinkRuntimeException("Error processing WAL signal event", e)); } - ((PostgresScanFetchTask.PostgresChangeEventSourceContext) context).finished(); + ((StoppableChangeEventSourceContext) context).stopChangeEventSource(); } } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/fetch/StoppableChangeEventSourceContext.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/fetch/StoppableChangeEventSourceContext.java new file mode 100644 index 00000000000..37ce7564fca --- /dev/null +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/fetch/StoppableChangeEventSourceContext.java @@ -0,0 +1,38 @@ +/* + * Copyright 2023 Ververica Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.ververica.cdc.connectors.postgres.source.fetch; + +import io.debezium.pipeline.source.spi.ChangeEventSource; + +/** + * A change event source context that can stop the running source by invoking {@link + * #stopChangeEventSource()}. + */ +public class StoppableChangeEventSourceContext + implements ChangeEventSource.ChangeEventSourceContext { + + private volatile boolean isRunning = true; + + public void stopChangeEventSource() { + isRunning = false; + } + + @Override + public boolean isRunning() { + return isRunning; + } +} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/table/PostgreSQLTableFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/table/PostgreSQLTableFactory.java index c388c28b5b1..aa8ba051daa 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/table/PostgreSQLTableFactory.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/table/PostgreSQLTableFactory.java @@ -200,6 +200,8 @@ public Set> optionalOptions() { } private static final String SCAN_STARTUP_MODE_VALUE_INITIAL = "initial"; + + private static final String SCAN_STARTUP_MODE_VALUE_SNAPSHOT = "snapshot"; private static final String SCAN_STARTUP_MODE_VALUE_LATEST = "latest-offset"; private static StartupOptions getStartupOptions(ReadableConfig config) { @@ -208,16 +210,18 @@ private static StartupOptions getStartupOptions(ReadableConfig config) { switch (modeString.toLowerCase()) { case SCAN_STARTUP_MODE_VALUE_INITIAL: return StartupOptions.initial(); - + case SCAN_STARTUP_MODE_VALUE_SNAPSHOT: + return StartupOptions.snapshot(); case SCAN_STARTUP_MODE_VALUE_LATEST: return StartupOptions.latest(); default: throw new ValidationException( String.format( - "Invalid value for option '%s'. Supported values are [%s, %s], but was: %s", + "Invalid value for option '%s'. Supported values are [%s, %s, %s], but was: %s", SCAN_STARTUP_MODE.key(), SCAN_STARTUP_MODE_VALUE_INITIAL, + SCAN_STARTUP_MODE_VALUE_SNAPSHOT, SCAN_STARTUP_MODE_VALUE_LATEST, modeString)); } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/io/debezium/connector/postgresql/PostgresStreamingChangeEventSource.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/io/debezium/connector/postgresql/PostgresStreamingChangeEventSource.java index 4b97321bef3..e309a3b29a4 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/io/debezium/connector/postgresql/PostgresStreamingChangeEventSource.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/io/debezium/connector/postgresql/PostgresStreamingChangeEventSource.java @@ -35,7 +35,7 @@ import java.util.concurrent.atomic.AtomicReference; /** - * Copied from Debezium 1.9.7. + * Copied from Debezium 1.9.8.final * *

    Line 150~151 : set the ending lsn for the replication connection. */ diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/io/debezium/connector/postgresql/connection/Lsn.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/io/debezium/connector/postgresql/connection/Lsn.java index 28e856c0362..64dc479eff4 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/io/debezium/connector/postgresql/connection/Lsn.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/io/debezium/connector/postgresql/connection/Lsn.java @@ -11,10 +11,10 @@ import java.nio.ByteBuffer; /** - * Copied from Debezium 1.9.7 without changes due to the NoSuchMethodError, caused by the fact that - * current Debezium release java version is 11, so we need to compile this file by java 8 compiler. - * More - * info. Abstraction of PostgreSQL log sequence number, adapted from {@link LogSequenceNumber}. + * Copied from Debezium 1.9.8 due to the NoSuchMethodError, caused by the fact that current Debezium + * release java version is 11, so we need to compile this file by java 8 compiler. More info. + * Abstraction of PostgreSQL log sequence number, adapted from {@link LogSequenceNumber}. * *

    Line 32: add NO_STOPPING_LSN */ diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/io/debezium/connector/postgresql/connection/PostgresConnection.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/io/debezium/connector/postgresql/connection/PostgresConnection.java index 29fb9f3208e..931540051c6 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/io/debezium/connector/postgresql/connection/PostgresConnection.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/io/debezium/connector/postgresql/connection/PostgresConnection.java @@ -48,19 +48,24 @@ import java.time.Duration; import java.util.Objects; import java.util.Optional; +import java.util.Set; import java.util.concurrent.atomic.AtomicLong; +import java.util.regex.Pattern; /** * {@link JdbcConnection} connection extension used for connecting to Postgres instances. * * @author Horia Chiorean - *

    Copied from Debezium 1.9.2-Final with two additional methods: + *

    Copied from Debezium 1.9.8-Final with two additional methods: *

      *
    • Constructor PostgresConnection( Configuration config, PostgresValueConverterBuilder * valueConverterBuilder, ConnectionFactory factory) to allow passing a custom * ConnectionFactory *
    • override connection() to return a unwrapped PgConnection (otherwise, it will complain * about HikariProxyConnection cannot be cast to class org.postgresql.core.BaseConnection) + *
    • override isTableUniqueIndexIncluded: Copied DBZ-5398 from Debezium 2.0.0.Final to fix + * https://github.com/ververica/flink-cdc-connectors/issues/2710. Remove this comment + * after bumping debezium version to 2.0.0.Final. *
    */ public class PostgresConnection extends JdbcConnection { @@ -72,6 +77,10 @@ public class PostgresConnection extends JdbcConnection { public static final String CONNECTION_HEARTBEAT = "Debezium Heartbeat"; public static final String CONNECTION_GENERAL = "Debezium General"; + private static final Pattern FUNCTION_DEFAULT_PATTERN = + Pattern.compile("^[(]?[A-Za-z0-9_.]+\\((?:.+(?:, ?.+)*)?\\)"); + private static final Pattern EXPRESSION_DEFAULT_PATTERN = + Pattern.compile("\\(+(?:.+(?:[+ - * / < > = ~ ! @ # % ^ & | ` ?] ?.+)+)+\\)"); private static Logger LOGGER = LoggerFactory.getLogger(PostgresConnection.class); private static final String URL_PATTERN = @@ -541,7 +550,7 @@ public synchronized void close() { public Long currentTransactionId() throws SQLException { AtomicLong txId = new AtomicLong(0); query( - "select * from txid_current()", + "select (case pg_is_in_recovery() when 't' then 0 else txid_current() end) AS pg_current_txid", rs -> { if (rs.next()) { txId.compareAndSet(0, rs.getLong(1)); @@ -562,7 +571,7 @@ public long currentXLogLocation() throws SQLException { int majorVersion = connection().getMetaData().getDatabaseMajorVersion(); query( majorVersion >= 10 - ? "select * from pg_current_wal_lsn()" + ? "select (case pg_is_in_recovery() when 't' then pg_last_wal_receive_lsn() else pg_current_wal_lsn() end) AS pg_current_wal_lsn" : "select * from pg_current_xlog_location()", rs -> { if (!rs.next()) { @@ -830,6 +839,26 @@ protected boolean isTableType(String tableType) { return "TABLE".equals(tableType) || "PARTITIONED TABLE".equals(tableType); } + @Override + protected boolean isTableUniqueIndexIncluded(String indexName, String columnName) { + if (columnName != null) { + return !FUNCTION_DEFAULT_PATTERN.matcher(columnName).matches() + && !EXPRESSION_DEFAULT_PATTERN.matcher(columnName).matches(); + } + return false; + } + + /** + * Retrieves all {@code TableId}s in a given database catalog, including partitioned tables. + * + * @param catalogName the catalog/database name + * @return set of all table ids for existing table objects + * @throws SQLException if a database exception occurred + */ + public Set getAllTableIds(String catalogName) throws SQLException { + return readTableNames(catalogName, null, null, new String[] {"TABLE", "PARTITIONED TABLE"}); + } + @FunctionalInterface public interface PostgresValueConverterBuilder { PostgresValueConverter build(TypeRegistry registry); diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/io/debezium/connector/postgresql/connection/PostgresReplicationConnection.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/io/debezium/connector/postgresql/connection/PostgresReplicationConnection.java index 364cb59bd76..0c607045e7d 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/io/debezium/connector/postgresql/connection/PostgresReplicationConnection.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/io/debezium/connector/postgresql/connection/PostgresReplicationConnection.java @@ -49,14 +49,14 @@ import static java.lang.Math.toIntExact; /** - * Copied from Debezium 1.9.7. + * Copied from Debezium 1.9.8.Final * *

    The {@link ReplicationConnection} created from {@code createReplicationStream} will hang when * the wal logs only contain the keepAliveMessage. Support to set an ending Lsn to stop hanging. * - *

    Line 82, 694~695 : add endingPos and its setter. + *

    Line 83, 711~713 : add endingPos and its setter. * - *

    Line 554~559, 578~583: ReplicationStream from {@code createReplicationStream} will stop when + *

    Line 571~576, 595~600: ReplicationStream from {@code createReplicationStream} will stop when * endingPos reached. */ public class PostgresReplicationConnection extends JdbcConnection implements ReplicationConnection { @@ -72,6 +72,7 @@ public class PostgresReplicationConnection extends JdbcConnection implements Rep private final PostgresConnectorConfig connectorConfig; private final Duration statusUpdateInterval; private final MessageDecoder messageDecoder; + private final PostgresConnection jdbcConnection; private final TypeRegistry typeRegistry; private final Properties streamParams; @@ -98,7 +99,7 @@ public class PostgresReplicationConnection extends JdbcConnection implements Rep * @param statusUpdateInterval the interval at which the replication connection should * periodically send status * @param doSnapshot whether the connector is doing snapshot - * @param jdbcConnection general POstgreSQL JDBC connection + * @param jdbcConnection general PostgreSQL JDBC connection * @param typeRegistry registry with PostgreSQL types * @param streamParams additional parameters to pass to the replication stream * @param schema the schema; must not be null @@ -136,6 +137,7 @@ private PostgresReplicationConnection( this.statusUpdateInterval = statusUpdateInterval; this.messageDecoder = plugin.messageDecoder(new MessageDecoderContext(config, schema), jdbcConnection); + this.jdbcConnection = jdbcConnection; this.typeRegistry = typeRegistry; this.streamParams = streamParams; this.slotCreationInfo = null; @@ -204,44 +206,24 @@ protected void initPublication() { stmt.execute(createPublicationStmt); break; case FILTERED: - try { - Set tablesToCapture = determineCapturedTables(); - tableFilterString = - tablesToCapture.stream() - .map(TableId::toDoubleQuotedString) - .collect(Collectors.joining(", ")); - if (tableFilterString.isEmpty()) { - throw new DebeziumException( - String.format( - "No table filters found for filtered publication %s", - publicationName)); - } - createPublicationStmt = - String.format( - "CREATE PUBLICATION %s FOR TABLE %s;", - publicationName, tableFilterString); - LOGGER.info( - "Creating Publication with statement '{}'", - createPublicationStmt); - // Publication doesn't exist, create it but restrict to the - // tableFilter. - stmt.execute(createPublicationStmt); - } catch (Exception e) { - throw new ConnectException( - String.format( - "Unable to create filtered publication %s for %s", - publicationName, tableFilterString), - e); - } + createOrUpdatePublicationModeFilterted( + tableFilterString, stmt, false); break; } } else { - LOGGER.trace( - "A logical publication named '{}' for plugin '{}' and database '{}' is already active on the server " - + "and will be used by the plugin", - publicationName, - plugin, - database()); + switch (publicationAutocreateMode) { + case FILTERED: + createOrUpdatePublicationModeFilterted( + tableFilterString, stmt, true); + break; + default: + LOGGER.trace( + "A logical publication named '{}' for plugin '{}' and database '{}' is already active on the server " + + "and will be used by the plugin", + publicationName, + plugin, + database()); + } } } } @@ -253,11 +235,46 @@ protected void initPublication() { } } + private void createOrUpdatePublicationModeFilterted( + String tableFilterString, Statement stmt, boolean isUpdate) { + String createOrUpdatePublicationStmt; + try { + Set tablesToCapture = determineCapturedTables(); + tableFilterString = + tablesToCapture.stream() + .map(TableId::toDoubleQuotedString) + .collect(Collectors.joining(", ")); + if (tableFilterString.isEmpty()) { + throw new DebeziumException( + String.format( + "No table filters found for filtered publication %s", + publicationName)); + } + createOrUpdatePublicationStmt = + isUpdate + ? String.format( + "ALTER PUBLICATION %s SET TABLE %s;", + publicationName, tableFilterString) + : String.format( + "CREATE PUBLICATION %s FOR TABLE %s;", + publicationName, tableFilterString); + LOGGER.info( + isUpdate + ? "Updating Publication with statement '{}'" + : "Creating Publication with statement '{}'", + createOrUpdatePublicationStmt); + stmt.execute(createOrUpdatePublicationStmt); + } catch (Exception e) { + throw new ConnectException( + String.format( + "Unable to %s filtered publication %s for %s", + isUpdate ? "update" : "create", publicationName, tableFilterString), + e); + } + } + private Set determineCapturedTables() throws Exception { - Set allTableIds = - this.connect() - .readTableNames( - pgConnection().getCatalog(), null, null, new String[] {"TABLE"}); + Set allTableIds = jdbcConnection.getAllTableIds(connectorConfig.databaseName()); Set capturedTables = new HashSet<>(); diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/PostgresTestBase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/PostgresTestBase.java index 5c16aa74187..eaf12f3d8af 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/PostgresTestBase.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/PostgresTestBase.java @@ -18,6 +18,7 @@ import org.apache.flink.table.planner.factories.TestValuesTableFactory; import org.apache.flink.test.util.AbstractTestBase; +import org.apache.flink.types.Row; import com.ververica.cdc.connectors.postgres.source.PostgresConnectionPoolFactory; import com.ververica.cdc.connectors.postgres.source.config.PostgresSourceConfigFactory; @@ -41,7 +42,9 @@ import java.sql.DriverManager; import java.sql.SQLException; import java.sql.Statement; +import java.util.ArrayList; import java.util.Arrays; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Random; @@ -219,6 +222,16 @@ protected PostgresSourceConfigFactory getMockPostgresSourceConfigFactory( return postgresSourceConfigFactory; } + public static List fetchRows(Iterator iter, int size) { + List rows = new ArrayList<>(size); + while (size > 0 && iter.hasNext()) { + Row row = iter.next(); + rows.add(row.toString()); + size--; + } + return rows; + } + public static void assertEqualsInAnyOrder(List expected, List actual) { assertTrue(expected != null && actual != null); assertEqualsInOrder( diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/source/MockPostgresDialect.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/source/MockPostgresDialect.java new file mode 100644 index 00000000000..9a5bb17fe41 --- /dev/null +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/source/MockPostgresDialect.java @@ -0,0 +1,46 @@ +/* + * Copyright 2023 Ververica Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.ververica.cdc.connectors.postgres.source; + +import org.apache.flink.util.Preconditions; + +import com.ververica.cdc.connectors.base.source.meta.offset.Offset; +import com.ververica.cdc.connectors.postgres.source.config.PostgresSourceConfig; + +import java.util.function.Consumer; + +/** Mock postgres dialect used to test changelog when checkpoint. */ +public class MockPostgresDialect extends PostgresDialect { + + private static Consumer callback = null; + + public MockPostgresDialect(PostgresSourceConfig sourceConfig) { + super(sourceConfig); + } + + @Override + public void notifyCheckpointComplete(long checkpointId, Offset offset) throws Exception { + if (callback != null) { + callback.accept(checkpointId); + } + super.notifyCheckpointComplete(checkpointId, offset); + } + + public static void setNotifyCheckpointCompleteCallback(Consumer callback) { + MockPostgresDialect.callback = Preconditions.checkNotNull(callback); + } +} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/source/PostgresSourceITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/source/PostgresSourceITCase.java index f4c364978a9..f08d30326ba 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/source/PostgresSourceITCase.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/source/PostgresSourceITCase.java @@ -33,7 +33,9 @@ import org.apache.flink.types.Row; import org.apache.flink.util.CloseableIterator; import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FlinkRuntimeException; +import com.ververica.cdc.connectors.base.options.StartupOptions; import com.ververica.cdc.connectors.base.source.utils.hooks.SnapshotPhaseHook; import com.ververica.cdc.connectors.base.source.utils.hooks.SnapshotPhaseHooks; import com.ververica.cdc.connectors.postgres.PostgresTestBase; @@ -59,12 +61,14 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.FutureTask; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; import java.util.stream.Collectors; @@ -89,6 +93,7 @@ public class PostgresSourceITCase extends PostgresTestBase { private static final int USE_POST_LOWWATERMARK_HOOK = 1; private static final int USE_PRE_HIGHWATERMARK_HOOK = 2; + private static final int USE_POST_HIGHWATERMARK_HOOK = 3; private final String scanStartupMode; @@ -323,13 +328,81 @@ public void testDebeziumSlotDropOnStop() throws Exception { tableResult.getJobClient().get().cancel().get(); } + @Test + public void testSnapshotOnlyModeWithDMLPostHighWaterMark() throws Exception { + // The data num is 21, set fetchSize = 22 to test the job is bounded. + List records = + testBackfillWhenWritingEvents( + false, 22, USE_POST_HIGHWATERMARK_HOOK, StartupOptions.snapshot()); + List expectedRecords = + Arrays.asList( + "+I[101, user_1, Shanghai, 123567891234]", + "+I[102, user_2, Shanghai, 123567891234]", + "+I[103, user_3, Shanghai, 123567891234]", + "+I[109, user_4, Shanghai, 123567891234]", + "+I[110, user_5, Shanghai, 123567891234]", + "+I[111, user_6, Shanghai, 123567891234]", + "+I[118, user_7, Shanghai, 123567891234]", + "+I[121, user_8, Shanghai, 123567891234]", + "+I[123, user_9, Shanghai, 123567891234]", + "+I[1009, user_10, Shanghai, 123567891234]", + "+I[1010, user_11, Shanghai, 123567891234]", + "+I[1011, user_12, Shanghai, 123567891234]", + "+I[1012, user_13, Shanghai, 123567891234]", + "+I[1013, user_14, Shanghai, 123567891234]", + "+I[1014, user_15, Shanghai, 123567891234]", + "+I[1015, user_16, Shanghai, 123567891234]", + "+I[1016, user_17, Shanghai, 123567891234]", + "+I[1017, user_18, Shanghai, 123567891234]", + "+I[1018, user_19, Shanghai, 123567891234]", + "+I[1019, user_20, Shanghai, 123567891234]", + "+I[2000, user_21, Shanghai, 123567891234]"); + assertEqualsInAnyOrder(expectedRecords, records); + } + + @Test + public void testSnapshotOnlyModeWithDMLPreHighWaterMark() throws Exception { + // The data num is 21, set fetchSize = 22 to test the job is bounded + List records = + testBackfillWhenWritingEvents( + false, 22, USE_PRE_HIGHWATERMARK_HOOK, StartupOptions.snapshot()); + List expectedRecords = + Arrays.asList( + "+I[101, user_1, Shanghai, 123567891234]", + "+I[102, user_2, Shanghai, 123567891234]", + "+I[103, user_3, Shanghai, 123567891234]", + "+I[109, user_4, Shanghai, 123567891234]", + "+I[110, user_5, Shanghai, 123567891234]", + "+I[111, user_6, Shanghai, 123567891234]", + "+I[118, user_7, Shanghai, 123567891234]", + "+I[121, user_8, Shanghai, 123567891234]", + "+I[123, user_9, Shanghai, 123567891234]", + "+I[1009, user_10, Shanghai, 123567891234]", + "+I[1010, user_11, Shanghai, 123567891234]", + "+I[1011, user_12, Shanghai, 123567891234]", + "+I[1012, user_13, Shanghai, 123567891234]", + "+I[1013, user_14, Shanghai, 123567891234]", + "+I[1014, user_15, Shanghai, 123567891234]", + "+I[1015, user_16, Shanghai, 123567891234]", + "+I[1016, user_17, Shanghai, 123567891234]", + "+I[1017, user_18, Shanghai, 123567891234]", + "+I[1018, user_19, Shanghai, 123567891234]", + "+I[2000, user_21, Pittsburgh, 123567891234]", + "+I[15213, user_15213, Shanghai, 123567891234]"); + // when enable backfill, the wal log between (snapshot, high_watermark) will be + // applied as snapshot image + assertEqualsInAnyOrder(expectedRecords, records); + } + @Test public void testEnableBackfillWithDMLPreHighWaterMark() throws Exception { if (!DEFAULT_SCAN_STARTUP_MODE.equals(scanStartupMode)) { return; } - List records = testBackfillWhenWritingEvents(false, 21, USE_PRE_HIGHWATERMARK_HOOK); + List records = + testBackfillWhenWritingEvents( + false, 21, USE_PRE_HIGHWATERMARK_HOOK, StartupOptions.initial()); List expectedRecords = Arrays.asList( @@ -365,7 +438,9 @@ public void testEnableBackfillWithDMLPostLowWaterMark() throws Exception { return; } - List records = testBackfillWhenWritingEvents(false, 21, USE_POST_LOWWATERMARK_HOOK); + List records = + testBackfillWhenWritingEvents( + false, 21, USE_POST_LOWWATERMARK_HOOK, StartupOptions.initial()); List expectedRecords = Arrays.asList( @@ -401,7 +476,9 @@ public void testSkipBackfillWithDMLPreHighWaterMark() throws Exception { return; } - List records = testBackfillWhenWritingEvents(true, 25, USE_PRE_HIGHWATERMARK_HOOK); + List records = + testBackfillWhenWritingEvents( + true, 25, USE_PRE_HIGHWATERMARK_HOOK, StartupOptions.initial()); List expectedRecords = Arrays.asList( @@ -441,7 +518,9 @@ public void testSkipBackfillWithDMLPostLowWaterMark() throws Exception { return; } - List records = testBackfillWhenWritingEvents(true, 25, USE_POST_LOWWATERMARK_HOOK); + List records = + testBackfillWhenWritingEvents( + true, 25, USE_POST_LOWWATERMARK_HOOK, StartupOptions.initial()); List expectedRecords = Arrays.asList( @@ -476,13 +555,83 @@ public void testSkipBackfillWithDMLPostLowWaterMark() throws Exception { assertEqualsInAnyOrder(expectedRecords, records); } + @Test + public void testNewLsnCommittedWhenCheckpoint() throws Exception { + int parallelism = 1; + FailoverType failoverType = FailoverType.JM; + FailoverPhase failoverPhase = FailoverPhase.STREAM; + String[] captureCustomerTables = new String[] {"customers"}; + RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration = + RestartStrategies.fixedDelayRestart(1, 0); + boolean skipSnapshotBackfill = false; + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tEnv = StreamTableEnvironment.create(env); + + env.setParallelism(parallelism); + env.enableCheckpointing(200L); + env.setRestartStrategy(restartStrategyConfiguration); + String sourceDDL = + format( + "CREATE TABLE customers (" + + " id BIGINT NOT NULL," + + " name STRING," + + " address STRING," + + " phone_number STRING," + + " primary key (id) not enforced" + + ") WITH (" + + " 'connector' = 'postgres-cdc-mock'," + + " 'scan.incremental.snapshot.enabled' = 'true'," + + " 'hostname' = '%s'," + + " 'port' = '%s'," + + " 'username' = '%s'," + + " 'password' = '%s'," + + " 'database-name' = '%s'," + + " 'schema-name' = '%s'," + + " 'table-name' = '%s'," + + " 'scan.startup.mode' = '%s'," + + " 'scan.incremental.snapshot.chunk.size' = '100'," + + " 'slot.name' = '%s'," + + " 'scan.incremental.snapshot.backfill.skip' = '%s'" + + ")", + customDatabase.getHost(), + customDatabase.getDatabasePort(), + customDatabase.getUsername(), + customDatabase.getPassword(), + customDatabase.getDatabaseName(), + SCHEMA_NAME, + getTableNameRegex(captureCustomerTables), + scanStartupMode, + slotName, + skipSnapshotBackfill); + tEnv.executeSql(sourceDDL); + TableResult tableResult = tEnv.executeSql("select * from customers"); + + // first step: check the snapshot data + if (DEFAULT_SCAN_STARTUP_MODE.equals(scanStartupMode)) { + checkSnapshotData(tableResult, failoverType, failoverPhase, captureCustomerTables); + } + + // second step: check the stream data + checkStreamDataWithHook(tableResult, failoverType, failoverPhase, captureCustomerTables); + + tableResult.getJobClient().get().cancel().get(); + + // sleep 1000ms to wait until connections are closed. + Thread.sleep(1000L); + } + private List testBackfillWhenWritingEvents( - boolean skipSnapshotBackfill, int fetchSize, int hookType) throws Exception { + boolean skipSnapshotBackfill, + int fetchSize, + int hookType, + StartupOptions startupOptions) + throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.enableCheckpointing(1000); env.setParallelism(1); - ResolvedSchema customersSchame = + ResolvedSchema customersSchema = new ResolvedSchema( Arrays.asList( physical("id", BIGINT().notNull()), @@ -492,7 +641,7 @@ private List testBackfillWhenWritingEvents( new ArrayList<>(), UniqueConstraint.primaryKey("pk", Collections.singletonList("id"))); TestTable customerTable = - new TestTable(customDatabase, "customer", "customers", customersSchame); + new TestTable(customDatabase, "customer", "customers", customersSchema); String tableId = customerTable.getTableId(); PostgresSourceBuilder.PostgresIncrementalSource source = @@ -504,6 +653,7 @@ private List testBackfillWhenWritingEvents( .database(customDatabase.getDatabaseName()) .slotName(slotName) .tableList(tableId) + .startupOptions(startupOptions) .skipSnapshotBackfill(skipSnapshotBackfill) .deserializer(customerTable.getDeserializer()) .build(); @@ -525,16 +675,19 @@ private List testBackfillWhenWritingEvents( try (PostgresConnection postgresConnection = dialect.openJdbcConnection()) { postgresConnection.execute(statements); postgresConnection.commit(); - Thread.sleep(500L); - } catch (InterruptedException e) { - throw new RuntimeException(e); } }; - if (hookType == USE_POST_LOWWATERMARK_HOOK) { - hooks.setPostLowWatermarkAction(snapshotPhaseHook); - } else if (hookType == USE_PRE_HIGHWATERMARK_HOOK) { - hooks.setPreHighWatermarkAction(snapshotPhaseHook); + switch (hookType) { + case USE_POST_LOWWATERMARK_HOOK: + hooks.setPostLowWatermarkAction(snapshotPhaseHook); + break; + case USE_PRE_HIGHWATERMARK_HOOK: + hooks.setPreHighWatermarkAction(snapshotPhaseHook); + break; + case USE_POST_HIGHWATERMARK_HOOK: + hooks.setPostHighWatermarkAction(snapshotPhaseHook); + break; } source.setSnapshotHooks(hooks); @@ -727,6 +880,75 @@ private void checkStreamData( assertTrue(!hasNextData(iterator)); } + private void checkStreamDataWithHook( + TableResult tableResult, + FailoverType failoverType, + FailoverPhase failoverPhase, + String[] captureCustomerTables) + throws Exception { + waitUntilJobRunning(tableResult); + CloseableIterator iterator = tableResult.collect(); + JobID jobId = tableResult.getJobClient().get().getJobID(); + + final AtomicLong savedCheckpointId = new AtomicLong(0); + final CountDownLatch countDownLatch = new CountDownLatch(1); + + MockPostgresDialect.setNotifyCheckpointCompleteCallback( + checkpointId -> { + try { + if (savedCheckpointId.get() == 0) { + savedCheckpointId.set(checkpointId); + + for (String tableId : captureCustomerTables) { + makeFirstPartStreamEvents( + getConnection(), + customDatabase.getDatabaseName() + + '.' + + SCHEMA_NAME + + '.' + + tableId); + } + // wait for the stream reading + Thread.sleep(2000L); + + triggerFailover( + failoverType, + jobId, + miniClusterResource.getMiniCluster(), + () -> sleepMs(200)); + countDownLatch.countDown(); + } + } catch (Exception e) { + throw new FlinkRuntimeException(e); + } + }); + + countDownLatch.await(); + waitUntilJobRunning(tableResult); + + if (failoverPhase == FailoverPhase.STREAM) { + triggerFailover( + failoverType, jobId, miniClusterResource.getMiniCluster(), () -> sleepMs(200)); + waitUntilJobRunning(tableResult); + } + for (String tableId : captureCustomerTables) { + makeSecondPartStreamEvents( + getConnection(), + customDatabase.getDatabaseName() + '.' + SCHEMA_NAME + '.' + tableId); + } + + List expectedStreamData = new ArrayList<>(); + for (int i = 0; i < captureCustomerTables.length; i++) { + expectedStreamData.addAll(firstPartStreamEvents); + expectedStreamData.addAll(secondPartStreamEvents); + } + // wait for the stream reading + Thread.sleep(2000L); + + assertEqualsInAnyOrder(expectedStreamData, fetchRows(iterator, expectedStreamData.size())); + assertTrue(!hasNextData(iterator)); + } + private void checkStreamDataWithDDLDuringFailover( TableResult tableResult, FailoverType failoverType, @@ -811,16 +1033,6 @@ private static List fetchRowData( return rows.stream().map(stringifier).collect(Collectors.toList()); } - private static List fetchRows(Iterator iter, int size) { - List rows = new ArrayList<>(size); - while (size > 0 && iter.hasNext()) { - Row row = iter.next(); - rows.add(row.toString()); - size--; - } - return rows; - } - /** * Make some changes on the specified customer table. Changelog in string could be accessed by * {@link #firstPartStreamEvents}. diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/table/MockPostgreSQLTableFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/table/MockPostgreSQLTableFactory.java new file mode 100644 index 00000000000..fe6859436da --- /dev/null +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/table/MockPostgreSQLTableFactory.java @@ -0,0 +1,37 @@ +/* + * Copyright 2023 Ververica Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.ververica.cdc.connectors.postgres.table; + +import org.apache.flink.table.connector.source.DynamicTableSource; + +/** Mock {@link PostgreSQLTableFactory}. */ +public class MockPostgreSQLTableFactory extends PostgreSQLTableFactory { + public static final String IDENTIFIER = "postgres-cdc-mock"; + + @Override + public DynamicTableSource createDynamicTableSource(Context context) { + PostgreSQLTableSource postgreSQLTableSource = + (PostgreSQLTableSource) super.createDynamicTableSource(context); + + return new MockPostgreSQLTableSource(postgreSQLTableSource); + } + + @Override + public String factoryIdentifier() { + return IDENTIFIER; + } +} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/table/MockPostgreSQLTableSource.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/table/MockPostgreSQLTableSource.java new file mode 100644 index 00000000000..4955337081f --- /dev/null +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/table/MockPostgreSQLTableSource.java @@ -0,0 +1,111 @@ +/* + * Copyright 2023 Ververica Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.ververica.cdc.connectors.postgres.table; + +import org.apache.flink.api.connector.source.Source; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.connector.source.SourceProvider; +import org.apache.flink.table.data.RowData; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.Preconditions; + +import com.ververica.cdc.connectors.base.options.StartupOptions; +import com.ververica.cdc.connectors.base.source.IncrementalSource; +import com.ververica.cdc.connectors.postgres.source.MockPostgresDialect; +import com.ververica.cdc.connectors.postgres.source.PostgresSourceBuilder; +import com.ververica.cdc.connectors.postgres.source.config.PostgresSourceConfigFactory; +import com.ververica.cdc.debezium.table.DebeziumChangelogMode; + +import java.lang.reflect.Field; +import java.time.Duration; +import java.util.Properties; + +/** Mock {@link PostgreSQLTableSource}. */ +public class MockPostgreSQLTableSource extends PostgreSQLTableSource { + public MockPostgreSQLTableSource(PostgreSQLTableSource postgreSQLTableSource) { + super( + (ResolvedSchema) get(postgreSQLTableSource, "physicalSchema"), + (int) get(postgreSQLTableSource, "port"), + (String) get(postgreSQLTableSource, "hostname"), + (String) get(postgreSQLTableSource, "database"), + (String) get(postgreSQLTableSource, "schemaName"), + (String) get(postgreSQLTableSource, "tableName"), + (String) get(postgreSQLTableSource, "username"), + (String) get(postgreSQLTableSource, "password"), + (String) get(postgreSQLTableSource, "pluginName"), + (String) get(postgreSQLTableSource, "slotName"), + (DebeziumChangelogMode) get(postgreSQLTableSource, "changelogMode"), + (Properties) get(postgreSQLTableSource, "dbzProperties"), + (boolean) get(postgreSQLTableSource, "enableParallelRead"), + (int) get(postgreSQLTableSource, "splitSize"), + (int) get(postgreSQLTableSource, "splitMetaGroupSize"), + (int) get(postgreSQLTableSource, "fetchSize"), + (Duration) get(postgreSQLTableSource, "connectTimeout"), + (int) get(postgreSQLTableSource, "connectMaxRetries"), + (int) get(postgreSQLTableSource, "connectionPoolSize"), + (double) get(postgreSQLTableSource, "distributionFactorUpper"), + (double) get(postgreSQLTableSource, "distributionFactorLower"), + (Duration) get(postgreSQLTableSource, "heartbeatInterval"), + (StartupOptions) get(postgreSQLTableSource, "startupOptions"), + (String) get(postgreSQLTableSource, "chunkKeyColumn"), + (boolean) get(postgreSQLTableSource, "closeIdleReaders"), + (boolean) get(postgreSQLTableSource, "skipSnapshotBackfill")); + } + + @Override + public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) { + ScanRuntimeProvider scanRuntimeProvider = super.getScanRuntimeProvider(scanContext); + + if (scanRuntimeProvider instanceof SourceProvider) { + Source source = ((SourceProvider) scanRuntimeProvider).createSource(); + Preconditions.checkState( + source instanceof PostgresSourceBuilder.PostgresIncrementalSource); + + PostgresSourceBuilder.PostgresIncrementalSource incrementalSource = + (PostgresSourceBuilder.PostgresIncrementalSource) source; + + try { + Field configFactoryField = + IncrementalSource.class.getDeclaredField("configFactory"); + configFactoryField.setAccessible(true); + PostgresSourceConfigFactory configFactory = + (PostgresSourceConfigFactory) configFactoryField.get(incrementalSource); + MockPostgresDialect mockPostgresDialect = + new MockPostgresDialect(configFactory.create(0)); + + Field dataSourceDialectField = + IncrementalSource.class.getDeclaredField("dataSourceDialect"); + dataSourceDialectField.setAccessible(true); + dataSourceDialectField.set(incrementalSource, mockPostgresDialect); + } catch (NoSuchFieldException | IllegalArgumentException | IllegalAccessException e) { + throw new FlinkRuntimeException(e); + } + } + + return scanRuntimeProvider; + } + + private static Object get(PostgreSQLTableSource postgreSQLTableSource, String name) { + try { + Field field = postgreSQLTableSource.getClass().getDeclaredField(name); + field.setAccessible(true); + return field.get(postgreSQLTableSource); + } catch (NoSuchFieldException | IllegalArgumentException | IllegalAccessException e) { + throw new FlinkRuntimeException(e); + } + } +} diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/table/PostgreSQLConnectorITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/table/PostgreSQLConnectorITCase.java index 5df140e0882..907a32b5b21 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/table/PostgreSQLConnectorITCase.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/table/PostgreSQLConnectorITCase.java @@ -23,6 +23,9 @@ import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import org.apache.flink.table.planner.factories.TestValuesTableFactory; import org.apache.flink.table.utils.LegacyRowResource; +import org.apache.flink.types.Row; +import org.apache.flink.types.RowUtils; +import org.apache.flink.util.CloseableIterator; import org.apache.flink.util.ExceptionUtils; import com.ververica.cdc.connectors.postgres.PostgresTestBase; @@ -35,6 +38,7 @@ import java.sql.Connection; import java.sql.SQLException; import java.sql.Statement; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -707,4 +711,69 @@ public void testUpsertMode() throws Exception { result.getJobClient().get().cancel().get(); } + + @Test + public void testUniqueIndexIncludingFunction() throws Exception { + // Clear the influence of usesLegacyRows which set USE_LEGACY_TO_STRING = true. + // In this test, print +I,-U, +U to see more clearly. + RowUtils.USE_LEGACY_TO_STRING = false; + initializePostgresTable(POSTGRES_CONTAINER, "index_type_test"); + + String sourceDDL = + String.format( + "CREATE TABLE functional_unique_index (" + + " id INTEGER NOT NULL," + + " char_c STRING" + + ") WITH (" + + " 'connector' = 'postgres-cdc'," + + " 'hostname' = '%s'," + + " 'port' = '%s'," + + " 'username' = '%s'," + + " 'password' = '%s'," + + " 'database-name' = '%s'," + + " 'schema-name' = '%s'," + + " 'table-name' = '%s'," + + " 'scan.incremental.snapshot.enabled' = '%s'," + // In the snapshot phase of increment snapshot mode, table without + // primary key is not allowed now.Thus, when + // scan.incremental.snapshot.enabled = true, use 'latest-offset' + // startup mode. + + (parallelismSnapshot + ? " 'scan.startup.mode' = 'latest-offset'," + : "") + + " 'slot.name' = '%s'" + + ")", + POSTGRES_CONTAINER.getHost(), + POSTGRES_CONTAINER.getMappedPort(POSTGRESQL_PORT), + POSTGRES_CONTAINER.getUsername(), + POSTGRES_CONTAINER.getPassword(), + POSTGRES_CONTAINER.getDatabaseName(), + "indexes", + "functional_unique_index", + parallelismSnapshot, + getSlotName()); + tEnv.executeSql(sourceDDL); + + // async submit job + TableResult tableResult = tEnv.executeSql("SELECT * FROM functional_unique_index"); + List expected = new ArrayList<>(); + if (!parallelismSnapshot) { + expected.add("+I[1, a]"); + } + + // wait a bit to make sure the replication slot is ready + Thread.sleep(5000L); + + // generate WAL + try (Connection connection = getJdbcConnection(POSTGRES_CONTAINER); + Statement statement = connection.createStatement()) { + statement.execute("UPDATE indexes.functional_unique_index SET char_c=NULL WHERE id=1;"); + } + + expected.addAll(Arrays.asList("-U[1, a]", "+U[1, null]")); + CloseableIterator iterator = tableResult.collect(); + assertEqualsInAnyOrder(expected, fetchRows(iterator, expected.size())); + tableResult.getJobClient().get().cancel().get(); + RowUtils.USE_LEGACY_TO_STRING = true; + } } diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory new file mode 100644 index 00000000000..51f1231e3ce --- /dev/null +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -0,0 +1,14 @@ +# Copyright 2023 Ververica Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +com.ververica.cdc.connectors.postgres.table.MockPostgreSQLTableFactory diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/resources/ddl/column_type_test.sql b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/resources/ddl/column_type_test.sql index c9ccf78b979..ed3794a3c37 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/resources/ddl/column_type_test.sql +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/resources/ddl/column_type_test.sql @@ -53,4 +53,4 @@ INSERT INTO full_types VALUES (1, '2', 32767, 65535, 2147483647, 5.5, 6.6, 123.12345, 404.4443, true, 'Hello World', 'a', 'abc', 'abcd..xyz', '2020-07-17 18:00:22.123', '2020-07-17 18:00:22.123456', '2020-07-17', '18:00:22', 500, 'SRID=3187;POINT(174.9479 -36.7208)'::geometry, - 'MULTILINESTRING((169.1321 -44.7032, 167.8974 -44.6414))'::geography); + 'MULTILINESTRING((169.1321 -44.7032, 167.8974 -44.6414))'::geography); \ No newline at end of file diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/resources/ddl/index_type_test.sql b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/resources/ddl/index_type_test.sql new file mode 100644 index 00000000000..aee2951cb5c --- /dev/null +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/test/resources/ddl/index_type_test.sql @@ -0,0 +1,35 @@ +-- Copyright 2023 Ververica Inc. +-- +-- Licensed under the Apache License, Version 2.0 (the "License"); +-- you may not use this file except in compliance with the License. +-- You may obtain a copy of the License at +-- http://www.apache.org/licenses/LICENSE-2.0 +-- Unless required by applicable law or agreed to in writing, +-- software distributed under the License is distributed on an +-- "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +-- KIND, either express or implied. See the License for the +-- specific language governing permissions and limitations +-- under the License. + +-- ---------------------------------------------------------------------------------------------------------------- +-- DATABASE: key_type_test +-- ---------------------------------------------------------------------------------------------------------------- +-- Generate a number of tables to cover as many of the PG index situation (primary key, unique index) as possible +DROP SCHEMA IF EXISTS indexes CASCADE; +CREATE SCHEMA indexes; +SET search_path TO indexes; + +-- Generate a table without primary key but a functional unique index +CREATE TABLE functional_unique_index +( + id INTEGER NOT NULL, + char_c CHAR +); +create unique index test_tbl_idx + on functional_unique_index(id, COALESCE(char_c, ''::text)); + +ALTER TABLE functional_unique_index + REPLICA IDENTITY FULL; + +INSERT INTO functional_unique_index +VALUES (1, 'a'); diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/SqlServerSource.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/SqlServerSource.java index 7d085a2683a..bda1286c486 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/SqlServerSource.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/SqlServerSource.java @@ -29,6 +29,7 @@ * A builder to build a SourceFunction which can read snapshot and continue to consume transaction * log for SqlServer. */ +@Deprecated public class SqlServerSource { private static final String DATABASE_SERVER_NAME = "sqlserver_transaction_log_source"; diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/io/debezium/connector/sqlserver/SqlServerStreamingChangeEventSource.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/io/debezium/connector/sqlserver/SqlServerStreamingChangeEventSource.java index eadba8160cd..80b4cf235bb 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/io/debezium/connector/sqlserver/SqlServerStreamingChangeEventSource.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/io/debezium/connector/sqlserver/SqlServerStreamingChangeEventSource.java @@ -37,9 +37,9 @@ import java.util.stream.Collectors; /** - * Copied from Debezium project(1.9.7.final) to add method {@link + * Copied from Debezium project(1.9.8.final) to add method {@link * SqlServerStreamingChangeEventSource#afterHandleLsn(SqlServerPartition, Lsn)}. Also implemented - * {@link SqlServerStreamingChangeEventSource#execute( ChangeEventSourceContext, SqlServerPartition, + * {@link SqlServerStreamingChangeEventSource#execute(ChangeEventSourceContext, SqlServerPartition, * SqlServerOffsetContext)}. A {@link StreamingChangeEventSource} based on SQL Server change data * capture functionality. A main loop polls database DDL change and change data tables and turns * them into change events. diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/com/ververica/cdc/connectors/sqlserver/source/SqlServerSourceITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/com/ververica/cdc/connectors/sqlserver/source/SqlServerSourceITCase.java index c563199ab57..9e2a55e3721 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/com/ververica/cdc/connectors/sqlserver/source/SqlServerSourceITCase.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/com/ververica/cdc/connectors/sqlserver/source/SqlServerSourceITCase.java @@ -123,7 +123,7 @@ public void testReadSingleTableWithSingleParallelismAndSkipBackfill() throws Exc @Test public void testEnableBackfillWithDMLPreHighWaterMark() throws Exception { - List records = testBackfillWhenWritingEvents(false, 21, USE_PRE_HIGHWATERMARK_HOOK); + List records = testBackfillWhenWritingEvents(false, 25, USE_PRE_HIGHWATERMARK_HOOK); List expectedRecords = Arrays.asList( @@ -146,17 +146,23 @@ public void testEnableBackfillWithDMLPreHighWaterMark() throws Exception { "+I[1016, user_17, Shanghai, 123567891234]", "+I[1017, user_18, Shanghai, 123567891234]", "+I[1018, user_19, Shanghai, 123567891234]", - "+I[2000, user_21, Pittsburgh, 123567891234]", - "+I[15213, user_15213, Shanghai, 123567891234]"); - // when enable backfill, the wal log between [snapshot, high_watermark) will be - // applied as snapshot image + "+I[1019, user_20, Shanghai, 123567891234]", + "+I[2000, user_21, Shanghai, 123567891234]", + "+I[15213, user_15213, Shanghai, 123567891234]", + "-U[2000, user_21, Shanghai, 123567891234]", + "+U[2000, user_21, Pittsburgh, 123567891234]", + "-D[1019, user_20, Shanghai, 123567891234]"); + // In sqlserver database, because the capture process extracts change data from the + // transaction log, there is a built-in latency between the time that a change is committed + // to a source table and the time that the change appears within its associated change + // table.Then in streaming phase, the log which should be ignored will be read again. assertEqualsInAnyOrder(expectedRecords, records); } @Test public void testEnableBackfillWithDMLPostLowWaterMark() throws Exception { - List records = testBackfillWhenWritingEvents(false, 21, USE_POST_LOWWATERMARK_HOOK); + List records = testBackfillWhenWritingEvents(false, 25, USE_POST_LOWWATERMARK_HOOK); List expectedRecords = Arrays.asList( @@ -180,9 +186,15 @@ public void testEnableBackfillWithDMLPostLowWaterMark() throws Exception { "+I[1017, user_18, Shanghai, 123567891234]", "+I[1018, user_19, Shanghai, 123567891234]", "+I[2000, user_21, Pittsburgh, 123567891234]", - "+I[15213, user_15213, Shanghai, 123567891234]"); - // when enable backfill, the wal log between [low_watermark, snapshot) will be applied - // as snapshot image + "+I[15213, user_15213, Shanghai, 123567891234]", + "+I[15213, user_15213, Shanghai, 123567891234]", + "-U[2000, user_21, Shanghai, 123567891234]", + "+U[2000, user_21, Pittsburgh, 123567891234]", + "-D[1019, user_20, Shanghai, 123567891234]"); + // In sqlserver database, because the capture process extracts change data from the + // transaction log, there is a built-in latency between the time that a change is committed + // to a source table and the time that the change appears within its associated change + // table.Then in streaming phase, the log which should be ignored will be read again. assertEqualsInAnyOrder(expectedRecords, records); } @@ -272,7 +284,7 @@ private List testBackfillWhenWritingEvents( env.enableCheckpointing(1000); env.setParallelism(1); - ResolvedSchema customersSchame = + ResolvedSchema customersSchema = new ResolvedSchema( Arrays.asList( physical("id", BIGINT().notNull()), @@ -281,7 +293,7 @@ private List testBackfillWhenWritingEvents( physical("phone_number", STRING())), new ArrayList<>(), UniqueConstraint.primaryKey("pk", Collections.singletonList("id"))); - TestTable customerTable = new TestTable(databaseName, "dbo", "customers", customersSchame); + TestTable customerTable = new TestTable(databaseName, "dbo", "customers", customersSchema); String tableId = customerTable.getTableId(); SqlServerSourceBuilder.SqlServerIncrementalSource source = @@ -310,14 +322,10 @@ private List testBackfillWhenWritingEvents( (sourceConfig, split) -> { SqlServerDialect dialect = new SqlServerDialect((SqlServerSourceConfig) sourceConfig); - JdbcConnection sqlServerConnection = - dialect.openJdbcConnection((JdbcSourceConfig) sourceConfig); - sqlServerConnection.execute(statements); - sqlServerConnection.commit(); - try { - Thread.sleep(5000L); - } catch (InterruptedException e) { - throw new RuntimeException(e); + try (JdbcConnection sqlServerConnection = + dialect.openJdbcConnection((JdbcSourceConfig) sourceConfig)) { + sqlServerConnection.execute(statements); + sqlServerConnection.commit(); } }; diff --git a/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/Db2E2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/Db2E2eITCase.java index 1ac25022713..44b1d5a3d8b 100644 --- a/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/Db2E2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/Db2E2eITCase.java @@ -20,7 +20,6 @@ import com.ververica.cdc.connectors.tests.utils.JdbcProxy; import com.ververica.cdc.connectors.tests.utils.TestUtils; import org.junit.After; -import org.junit.AfterClass; import org.junit.Before; import org.junit.Test; import org.slf4j.Logger; @@ -114,23 +113,6 @@ public void after() { super.after(); } - @AfterClass - public static void afterClass() { - // Cleanup the db2 image, because it's too large and will cause the next test to fail. - db2Container.getDockerClient().removeImageCmd(DB2_CUSTOM_IMAGE).exec(); - db2Container - .getDockerClient() - .listImagesCmd() - .withImageNameFilter(DB2_IMAGE) - .exec() - .forEach( - image -> - db2Container - .getDockerClient() - .removeImageCmd(image.getId()) - .exec()); - } - @Test public void testDb2CDC() throws Exception { List sqlLines = diff --git a/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/OceanBaseE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/OceanBaseE2eITCase.java index c517b75c99d..25b5d6a959a 100644 --- a/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/OceanBaseE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/OceanBaseE2eITCase.java @@ -16,17 +16,14 @@ package com.ververica.cdc.connectors.tests; -import com.github.dockerjava.api.DockerClient; import com.ververica.cdc.connectors.tests.utils.FlinkContainerTestEnvironment; import com.ververica.cdc.connectors.tests.utils.JdbcProxy; import com.ververica.cdc.connectors.tests.utils.TestUtils; -import org.junit.AfterClass; import org.junit.Before; import org.junit.ClassRule; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.testcontainers.DockerClientFactory; import org.testcontainers.containers.GenericContainer; import org.testcontainers.containers.output.Slf4jLogConsumer; import org.testcontainers.containers.wait.strategy.Wait; @@ -46,7 +43,6 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; import java.util.stream.Collectors; -import java.util.stream.Stream; import static org.junit.Assert.assertNotNull; @@ -116,21 +112,6 @@ private Connection getTestConnection(String databaseName) { } } - @AfterClass - public static void afterClass() { - Stream.of(OB_SERVER, LOG_PROXY).forEach(GenericContainer::stop); - - DockerClient client = DockerClientFactory.instance().client(); - client.listImagesCmd() - .withImageNameFilter(OB_SERVER_IMAGE) - .exec() - .forEach(image -> client.removeImageCmd(image.getId()).exec()); - client.listImagesCmd() - .withImageNameFilter(OB_LOG_PROXY_IMAGE) - .exec() - .forEach(image -> client.removeImageCmd(image.getId()).exec()); - } - @Test public void testOceanBaseCDC() throws Exception { List sqlLines = diff --git a/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/OracleE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/OracleE2eITCase.java index 54d503d4665..11d9a6a3b00 100644 --- a/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/OracleE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/OracleE2eITCase.java @@ -20,7 +20,6 @@ import com.ververica.cdc.connectors.tests.utils.JdbcProxy; import com.ververica.cdc.connectors.tests.utils.TestUtils; import org.junit.After; -import org.junit.AfterClass; import org.junit.Before; import org.junit.Test; import org.slf4j.Logger; @@ -84,16 +83,6 @@ public void after() { super.after(); } - @AfterClass - public static void afterClass() { - // Cleanup the oracle image, because it's too large and will cause the next test to fail. - oracle.getDockerClient() - .listImagesCmd() - .withImageNameFilter(ORACLE_IMAGE) - .exec() - .forEach(image -> oracle.getDockerClient().removeImageCmd(image.getId()).exec()); - } - @Test public void testOracleCDC() throws Exception { List sqlLines = diff --git a/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/SqlServerE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/SqlServerE2eITCase.java index a31a3c75477..3f7b8518e00 100644 --- a/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/SqlServerE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/SqlServerE2eITCase.java @@ -20,14 +20,12 @@ import com.ververica.cdc.connectors.tests.utils.JdbcProxy; import com.ververica.cdc.connectors.tests.utils.TestUtils; import org.junit.After; -import org.junit.AfterClass; import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.runners.Parameterized; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.testcontainers.DockerClientFactory; import org.testcontainers.containers.MSSQLServerContainer; import org.testcontainers.containers.output.Slf4jLogConsumer; import org.testcontainers.lifecycle.Startables; @@ -103,12 +101,6 @@ public void after() { super.after(); } - @AfterClass - public static void afterClass() { - // Cleanup the sqlserver image, because it's too large and will cause the next test to fail. - DockerClientFactory.instance().client().removeImageCmd(MSSQL_SERVER_IMAGE).exec(); - } - @Test public void testSqlServerCDC() throws Exception { List sqlLines = diff --git a/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/TiDBE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/TiDBE2eITCase.java index 8a4756786c7..5a117cc1b5e 100644 --- a/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/TiDBE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/TiDBE2eITCase.java @@ -20,7 +20,6 @@ import com.ververica.cdc.connectors.tests.utils.JdbcProxy; import com.ververica.cdc.connectors.tests.utils.TestUtils; import org.junit.After; -import org.junit.AfterClass; import org.junit.Before; import org.junit.ClassRule; import org.junit.Test; @@ -142,16 +141,6 @@ public void after() { super.after(); } - @AfterClass - public static void afterClass() { - // Cleanup the TIDB image, because it's too large and will cause the next test to fail. - TIDB.getDockerClient() - .listImagesCmd() - .withImageNameFilter("pingcap/tikv") - .exec() - .forEach(image -> TIDB.getDockerClient().removeImageCmd(image.getId()).exec()); - } - @Test public void testTIDBCDC() throws Exception { List sqlLines = diff --git a/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/utils/FlinkContainerTestEnvironment.java b/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/utils/FlinkContainerTestEnvironment.java index 609b9056e04..bcd434a65b1 100644 --- a/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/utils/FlinkContainerTestEnvironment.java +++ b/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/utils/FlinkContainerTestEnvironment.java @@ -26,10 +26,12 @@ import org.apache.flink.table.api.ValidationException; import org.apache.flink.util.TestLogger; +import com.github.dockerjava.api.DockerClient; import com.ververica.cdc.connectors.mysql.testutils.MySqlContainer; import com.ververica.cdc.connectors.mysql.testutils.MySqlVersion; import com.ververica.cdc.connectors.mysql.testutils.UniqueDatabase; import org.junit.After; +import org.junit.AfterClass; import org.junit.Before; import org.junit.ClassRule; import org.junit.Rule; @@ -38,6 +40,7 @@ import org.junit.runners.Parameterized; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.testcontainers.DockerClientFactory; import org.testcontainers.containers.Container.ExecResult; import org.testcontainers.containers.GenericContainer; import org.testcontainers.containers.Network; @@ -174,6 +177,45 @@ public void after() { mysqlInventoryDatabase.dropDatabase(); } + @AfterClass + public static void afterClass() { + DockerClient dockerClient = DockerClientFactory.instance().client(); + + // List all containers and remove the ones that are not testcontainers related. + dockerClient.listContainersCmd().exec().stream() + .filter(container -> !container.getImage().startsWith("testcontainers")) + .forEach( + container -> { + dockerClient.stopContainerCmd(container.getId()).exec(); + dockerClient.removeContainerCmd(container.getId()).exec(); + }); + + // List all images and remove the ones that are not flink、mysql、testcontainers related. + dockerClient.listImagesCmd().exec().stream() + .filter( + image -> + image.getRepoTags() != null + && Arrays.stream(image.getRepoTags()) + .anyMatch( + tag -> + !tag.startsWith("flink:") + && !tag.startsWith( + "testcontainers") + && !tag.equals( + MYSQL + .getDockerImageName()))) + .forEach( + image -> { + try { + dockerClient.removeImageCmd(image.getId()).exec(); + } catch (Exception e) { + LOG.warn( + "Failed to remove image: {}", + String.join(",", image.getRepoTags())); + } + }); + } + /** Allow overriding the default flink properties. */ public void overrideFlinkProperties(String properties) { jobManager.withEnv("FLINK_PROPERTIES", properties); diff --git a/flink-cdc-runtime/src/main/java/com/ververica/cdc/runtime/serializer/data/binary/BinaryRecordDataSerializer.java b/flink-cdc-runtime/src/main/java/com/ververica/cdc/runtime/serializer/data/binary/BinaryRecordDataSerializer.java index cb8e6a9a16a..9fbf42dd75e 100644 --- a/flink-cdc-runtime/src/main/java/com/ververica/cdc/runtime/serializer/data/binary/BinaryRecordDataSerializer.java +++ b/flink-cdc-runtime/src/main/java/com/ververica/cdc/runtime/serializer/data/binary/BinaryRecordDataSerializer.java @@ -89,13 +89,16 @@ public BinaryRecordData deserialize(DataInputView source) throws IOException { } @Override + @SuppressWarnings("unused") public BinaryRecordData deserialize(BinaryRecordData reuse, DataInputView source) throws IOException { MemorySegment[] segments = reuse.getSegments(); checkArgument( segments == null || (segments.length == 1 && reuse.getOffset() == 0), "Reuse BinaryRecordData should have no segments or only one segment and offset start at 0."); - + // Note: arity is not used in BinaryRecordData, so we can ignore it here.But we still need + // to read it. + int arity = source.readInt(); int length = source.readInt(); if (segments == null || segments[0].size() < length) { segments = new MemorySegment[] {MemorySegmentFactory.wrap(new byte[length])}; diff --git a/mysqlToDoris.yml b/mysqlToDoris.yml deleted file mode 100644 index ec46dd5d3f9..00000000000 --- a/mysqlToDoris.yml +++ /dev/null @@ -1,37 +0,0 @@ -################################################################################ -# Copyright 2023 Ververica Inc. -# -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -################################################################################ -source: - type: mysql - name: Mysql Source - hostname: localhost - port: 3306 - username: root - password: 123456 - tables: test.test_tbl1 - server-id: 5400-5408 - server-time-zone: Asia/Shanghai - -sink: - type: doris - name: Doris Sink - fenodes: 10.16.10.6:28737 - username: root - password: "" - table.create.properties.replication_num: 1 - table.create.properties.light_schema_change: true - - -pipeline: - parallelism: 1 diff --git a/pom.xml b/pom.xml index 43297e2408c..e82baec99b5 100644 --- a/pom.xml +++ b/pom.xml @@ -74,7 +74,7 @@ under the License. 1.18.0 1.18 17.0 - 1.9.7.Final + 1.9.8.Final 3.2.0 2.2.0 1.18.3