Skip to content

Commit 39608ed

Browse files
authored
[FLINK-36945][cdc-connector/mysql] Support parsing rename multiple tables in one statement
This closes #3876.
1 parent b44e570 commit 39608ed

File tree

2 files changed

+494
-1
lines changed

2 files changed

+494
-1
lines changed

flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/debezium/dispatcher/EventDispatcherImpl.java

Lines changed: 99 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -29,7 +29,9 @@
2929
import io.debezium.pipeline.source.spi.EventMetadataProvider;
3030
import io.debezium.pipeline.spi.ChangeEventCreator;
3131
import io.debezium.pipeline.spi.SchemaChangeEventEmitter;
32+
import io.debezium.relational.TableId;
3233
import io.debezium.relational.history.HistoryRecord;
34+
import io.debezium.relational.history.TableChanges;
3335
import io.debezium.schema.DataCollectionFilters;
3436
import io.debezium.schema.DataCollectionId;
3537
import io.debezium.schema.DatabaseSchema;
@@ -45,10 +47,14 @@
4547
import org.slf4j.LoggerFactory;
4648

4749
import java.io.IOException;
50+
import java.util.Arrays;
4851
import java.util.Collection;
4952
import java.util.HashMap;
53+
import java.util.List;
5054
import java.util.Map;
55+
import java.util.stream.Collectors;
5156

57+
import static io.debezium.connector.AbstractSourceInfo.TABLE_NAME_KEY;
5258
import static org.apache.flink.cdc.connectors.mysql.debezium.dispatcher.SignalEventDispatcher.BINLOG_FILENAME_OFFSET_KEY;
5359
import static org.apache.flink.cdc.connectors.mysql.debezium.dispatcher.SignalEventDispatcher.BINLOG_POSITION_OFFSET_KEY;
5460

@@ -206,11 +212,103 @@ private Struct schemaChangeRecordValue(SchemaChangeEvent event) throws IOExcepti
206212
String historyStr = DOCUMENT_WRITER.write(historyRecord.document());
207213

208214
Struct value = new Struct(schemaChangeValueSchema);
209-
value.put(HistoryRecord.Fields.SOURCE, event.getSource());
215+
value.put(HistoryRecord.Fields.SOURCE, rewriteTableNameIfNeeded(event));
210216
value.put(HISTORY_RECORD_FIELD, historyStr);
211217
return value;
212218
}
213219

220+
/**
221+
* Rewrites the table name in the Source if needed to handle schema changes properly.
222+
*
223+
* <p>This method addresses a specific issue when renaming multiple tables within a single
224+
* statement, such as: {@code RENAME TABLE customers TO customers_old, customers_copy TO
225+
* customers;}.
226+
*
227+
* <p>In such cases, Debezium's {@link io.debezium.connector.mysql.MySqlDatabaseSchema}
228+
* emits two separate change events:
229+
*
230+
* <ul>
231+
* <li>{@code RENAME TABLE customers TO customers_old}
232+
* <li>{@code RENAME TABLE customers_copy TO customers}
233+
* </ul>
234+
*
235+
* <p>Both events share a table name of {@code customers, customers_old} in their source
236+
* info, which includes multiple table IDs in a single string.
237+
*
238+
* <p>On the other hand, the {@code TableChanges.TableChange#id} correctly identifies the
239+
* schema change:
240+
*
241+
* <ul>
242+
* <li>The change for {@code RENAME TABLE customers_copy TO customers} has the {@code
243+
* customers} ID.
244+
* <li>The change for {@code RENAME TABLE customers TO customers_old} is empty.
245+
* </ul>
246+
*
247+
* <p>The problem arises because {@link
248+
* org.apache.flink.cdc.connectors.mysql.debezium.reader.BinlogSplitReader} does not expect
249+
* multiple table IDs in the source info. As a result, changes for tables defined by the
250+
* table filter configuration (e.g., {@code customers}) may be filtered out unintentionally.
251+
* This can lead to schema changes not being saved in the state, which is crucial for
252+
* recovering the job from a snapshot.
253+
*
254+
* <p>To resolve this issue, this method:
255+
*
256+
* <ol>
257+
* <li>Checks if the source info contains multiple table names.
258+
* <li>Verifies if the {@code TableChange#id} matches one of the table names.
259+
* <li>Updates the source info with the correct table name that conforms to Flink CDC
260+
* expectations, ensuring the schema change is saved correctly.
261+
* </ol>
262+
*
263+
* @param event the schema change event emitted by Debezium.
264+
* @return the updated source info with the corrected table name if necessary.
265+
*/
266+
private Struct rewriteTableNameIfNeeded(SchemaChangeEvent event) {
267+
Struct sourceInfo = event.getSource();
268+
String tableName = sourceInfo.getString(TABLE_NAME_KEY);
269+
if (tableName == null || tableName.isEmpty()) {
270+
return sourceInfo;
271+
}
272+
273+
List<String> tableNames = parseTableNames(tableName);
274+
if (2 <= tableNames.size() && event.getDdl().toLowerCase().startsWith("rename")) {
275+
for (TableChanges.TableChange tableChange : event.getTableChanges()) {
276+
String changedTableName = getMatchingTableName(tableNames, tableChange.getId());
277+
if (changedTableName != null) {
278+
LOG.debug(
279+
"Rewrite table name from {} to {} on swapping tables",
280+
tableName,
281+
changedTableName);
282+
sourceInfo.put(TABLE_NAME_KEY, changedTableName);
283+
}
284+
}
285+
}
286+
return sourceInfo;
287+
}
288+
289+
/**
290+
* Decodes table names from a comma-separated string.
291+
*
292+
* <p>This method extracts individual table names from a string where multiple table names
293+
* are separated by commas. The input string is constructed by {@link
294+
* io.debezium.connector.mysql.SourceInfo}.
295+
*
296+
* @param tableName a comma-separated string containing multiple table names
297+
* @return a list of trimmed table names
298+
*/
299+
private List<String> parseTableNames(String tableName) {
300+
return Arrays.stream(tableName.split(","))
301+
.map(String::trim)
302+
.collect(Collectors.toList());
303+
}
304+
305+
private String getMatchingTableName(List<String> tableNames, TableId tableId) {
306+
return tableNames.stream()
307+
.filter(name -> name.equals(tableId.table()))
308+
.findFirst()
309+
.orElse(null);
310+
}
311+
214312
@Override
215313
public void schemaChangeEvent(SchemaChangeEvent event) throws InterruptedException {
216314
historizedSchema.applySchemaChange(event);

0 commit comments

Comments
 (0)