-
Notifications
You must be signed in to change notification settings - Fork 2k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[FLINK-36945] MySQL CDC supports parsing RENAME TABLE Statement with multiple tables #3876
base: master
Are you sure you want to change the base?
Conversation
…presentation becomes out of sync with the real database schema when restarting a job
…presentation with the real database schema when restarting a job
customers.getTableId(), | ||
customers.getTableId(), | ||
customers.getTableId(), | ||
customers.getTableId())); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
📝 This issue does not occur when two separate RENAME TABLE statements are executed. The following test diff demonstrates how to confirm this behavior before fixing the source code:
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlRenameTablesSchemaFixITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlRenameTablesSchemaFixITCase.java
index 7d885c47..5e773ddd 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlRenameTablesSchemaFixITCase.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlRenameTablesSchemaFixITCase.java
@@ -198,11 +198,13 @@ public class MySqlRenameTablesSchemaFixITCase {
LOG.info("Step 3: Swap the tables");
executeStatements(
String.format(
- "RENAME TABLE %s TO %s_old, %s_copy TO %s;",
- customers.getTableId(),
- customers.getTableId(),
- customers.getTableId(),
- customers.getTableId()));
+ "RENAME TABLE %s TO %s_old;",
+ customers.getTableId(), customers.getTableId()));
+
+ executeStatements(
+ String.format(
+ "RENAME TABLE %s_copy TO %s;",
+ customers.getTableId(), customers.getTableId()));
LOG.info("Step 4: Insert data into the altered table before the savepoint");
executeStatements(
Here is the current (master branch) difference in DataChangeEvent between renaming a single table and renaming multiple tables: The first log comes from renaming a single table in a single statement. The second log is one of the DataChangeEvents generated when multiple tables are renamed in a single statement, specifically the event relevant to this case. The comparison focuses on these two logs to highlight how diff$ diff rename_single.txt rename_multi.txt
6c6
< ts_sec=1737529154,
---
> ts_sec=1737425209,
8,9c8,9
< pos=1054,
< gtids=5cb040cc-d88e-11ef-aaf7-0242ac110004:1-46,
---
> pos=1798,
> gtids=5891eef6-d79c-11ef-b2d3-0242ac110004:44-48,
17c17
< databaseName=customer_1ttteik
---
> databaseName=customer_1nkreoe
27,29c27,29
< ts_ms=1737529154419,
< db=customer_1ttteik,
< table=customers,
---
> ts_ms=1737425209617,
> db=customer_1nkreoe,
> table=customers_old,customers,
31c31
< gtid=5cb040cc-d88e-11ef-aaf7-0242ac110004:47,
---
> gtid=5891eef6-d79c-11ef-b2d3-0242ac110004:49,
33c33
< pos=898,
---
> pos=1580,
39c39
< "pos": 898,
---
> "pos": 1580,
44c44
< "ts_sec": 1737529154,
---
> "ts_sec": 1737425209,
46,47c46,47
< "pos": 1054,
< "gtids": "5cb040cc-d88e-11ef-aaf7-0242ac110004:1-46",
---
> "pos": 1798,
> "gtids": "5891eef6-d79c-11ef-b2d3-0242ac110004:44-48",
50,51c50,51
< "databaseName": "customer_1ttteik",
< "ddl": "RENAME TABLE customer_1ttteik.customers_copy TO customer_1ttteik.customers",
---
> "databaseName": "customer_1nkreoe",
> "ddl": "RENAME TABLE customer_1nkreoe.customers_copy TO customer_1nkreoe.customers",
55,56c55,56
< "id": "\"customer_1ttteik\".\"customers\"",
< "previousId": "\"customer_1ttteik\".\"customers_copy\"",
---
> "id": "\"customer_1nkreoe\".\"customers\"",
> "previousId": "\"customer_1nkreoe\".\"customers_copy\"",
Excluding random values such as timestamps, positions, GTIDs, and database names, the only meaningful difference lies in the value of source.table. - table=customers,
+ table=customers_old,customers, In the case of renaming a single table, the source.table contains one table name. However, for renaming multiple tables in a single statement, the source.table includes multiple table names, separated by commas. This behavior was observed from logs printed during the BinlogSplitReader execution, which shows how the DataChangeEvent differs in these scenarios. diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/debezium/reader/BinlogSplitReader.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/debezium/reader/BinlogSplitReader.java
index cd3c697e..49f567a9 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/debezium/reader/BinlogSplitReader.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/debezium/reader/BinlogSplitReader.java
@@ -165,6 +165,7 @@ public class BinlogSplitReader implements DebeziumReader<SourceRecords, MySqlSpl
if (currentTaskRunning) {
List<DataChangeEvent> batch = queue.poll();
for (DataChangeEvent event : batch) {
+ LOG.info("Read binlog event: {}", event);
if (isParsingOnLineSchemaChanges) {
Optional<SourceRecord> oscRecord =
parseOnLineSchemaChangeEvent(event.getRecord()); |
Quick question isn't this related to #3668? |
@mielientiev It's similar but not related. The fixed PR for #3668 does not address the issue presented here. And, I’m unsure if this fix resolves potential issues with gh-ost or pt-osc. |
@leonardBang Two tests failed:
|
Thanks for @yoheimuta's great work! I believe MySQL UT timeout failure is a known issue, and the migration test failure is irrelevant to this change. Will take a look after the new year holiday. |
Thanks for confirming! Happy holidays! 👍 |
This closes FLINK-36945.
The Vitess schema migration tool renames multiple tables within a single statement to perform schema changes, which are not correctly recognized by MySQL CDC.
The issue arises because the schema change event emitted by Debezium includes multiple table IDs, which MySQL CDC does not expect. As a result, changes for tables specified in the table filter configuration may be unintentionally filtered out. This can prevent schema changes from being saved in the state, which is critical for recovering the job from a snapshot.
This PR updates the source info with the correct table name to align with MySQL CDC expectations, ensuring schema changes are saved correctly.