|
29 | 29 | import io.debezium.pipeline.source.spi.EventMetadataProvider; |
30 | 30 | import io.debezium.pipeline.spi.ChangeEventCreator; |
31 | 31 | import io.debezium.pipeline.spi.SchemaChangeEventEmitter; |
| 32 | +import io.debezium.relational.TableId; |
32 | 33 | import io.debezium.relational.history.HistoryRecord; |
| 34 | +import io.debezium.relational.history.TableChanges; |
33 | 35 | import io.debezium.schema.DataCollectionFilters; |
34 | 36 | import io.debezium.schema.DataCollectionId; |
35 | 37 | import io.debezium.schema.DatabaseSchema; |
|
45 | 47 | import org.slf4j.LoggerFactory; |
46 | 48 |
|
47 | 49 | import java.io.IOException; |
| 50 | +import java.util.Arrays; |
48 | 51 | import java.util.Collection; |
49 | 52 | import java.util.HashMap; |
| 53 | +import java.util.List; |
50 | 54 | import java.util.Map; |
| 55 | +import java.util.stream.Collectors; |
51 | 56 |
|
| 57 | +import static io.debezium.connector.AbstractSourceInfo.TABLE_NAME_KEY; |
52 | 58 | import static org.apache.flink.cdc.connectors.mysql.debezium.dispatcher.SignalEventDispatcher.BINLOG_FILENAME_OFFSET_KEY; |
53 | 59 | import static org.apache.flink.cdc.connectors.mysql.debezium.dispatcher.SignalEventDispatcher.BINLOG_POSITION_OFFSET_KEY; |
54 | 60 |
|
@@ -206,11 +212,103 @@ private Struct schemaChangeRecordValue(SchemaChangeEvent event) throws IOExcepti |
206 | 212 | String historyStr = DOCUMENT_WRITER.write(historyRecord.document()); |
207 | 213 |
|
208 | 214 | Struct value = new Struct(schemaChangeValueSchema); |
209 | | - value.put(HistoryRecord.Fields.SOURCE, event.getSource()); |
| 215 | + value.put(HistoryRecord.Fields.SOURCE, rewriteTableNameIfNeeded(event)); |
210 | 216 | value.put(HISTORY_RECORD_FIELD, historyStr); |
211 | 217 | return value; |
212 | 218 | } |
213 | 219 |
|
| 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 | + |
214 | 312 | @Override |
215 | 313 | public void schemaChangeEvent(SchemaChangeEvent event) throws InterruptedException { |
216 | 314 | historizedSchema.applySchemaChange(event); |
|
0 commit comments