[ 
https://issues.apache.org/jira/browse/FLINK-36619?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

linqigeng updated FLINK-36619:
------------------------------
    Description: 
#CDC 3.2:

If some tables are removed from the `tables` parameter and then re-added, the 
`CreateTableEvent` will be ignored and not sent to downstream.

This way to reproduce:
 # set `tables` to `{*}test_db.\.{*},test_db2.\.{*}{{*}}`
 # remove `test_db2.\.*` from `tables` and restart pipeline job 
 # re-add  `test_db2.\.*` to `tables` and restart pipeline job

!image-2024-10-29-11-52-41-353.png!

 

#CDC 3.3

CreateTableEvent could be sent, but another exception occured in TM.
{code:java}
2025-01-21 19:11:49,934 ERROR 
io.debezium.connector.mysql.MySqlStreamingChangeEventSource  [] - Encountered 
change event 'Event{header=EventHeaderV4{timestamp=1737457909000, 
eventType=TABLE_MAP, serverId=1, headerLength=19, dataLength=79, 
nextPosition=945183051, flags=0}, data=TableMapEventData{tableId=2859, 
database='test_db2', table='t1', columnTypes=8, 15, 1, 15, 18, 18, 15, 15, 15, 
15, 15, 15, 15, 15, 15, 15, 15, 15, columnMetadata=0, 765, 0, 765, 0, 0, 2295, 
765, 765, 765, 765, 765, 765, 765, 765, 765, 765, 765, columnNullability={1, 3, 
6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17}, eventMetadata=null}}' at offset 
{transaction_id=null, ts_sec=1737457863, file=mysql-bin.000036, pos=945182869, 
server_id=1, event=1} for table test_db2.t1 whose schema isn't known to this 
connector. One possible cause is an incomplete database history topic. Take a 
new snapshot in this case.Use the mysqlbinlog tool to view the problematic 
event: mysqlbinlog --start-position=945182953 --stop-position=945183051 
--verbose mysql-bin.0000362025-01-21 19:11:49,935 ERROR 
io.debezium.connector.mysql.MySqlStreamingChangeEventSource  [] - Error during 
binlog processing. Last offset stored = {transaction_id=null, 
ts_sec=1737457863, file=mysql-bin.000036, pos=945182869, server_id=1, event=1}, 
binlog reader near position = mysql-bin.000036/9451829532025-01-21 19:12:15,472 
INFO  org.apache.doris.flink.sink.batch.DorisBatchWriter           [] - 
checkpoint flush triggered.2025-01-21 19:12:15,481 INFO  
org.apache.flink.cdc.connectors.mysql.source.reader.MySqlSourceReader [] - 
Binlog offset on checkpoint 5: {transaction_id=null, ts_sec=1737457863, 
file=mysql-bin.000036, pos=945192687, kind=SPECIFIC, row=0, event=0, 
server_id=1} {code}

  was:
## CDC 3.2:

If some tables are removed from the `tables` parameter and then re-added, the 
`CreateTableEvent` will be ignored and not sent to downstream.

This way to reproduce:
 # set `tables` to `{*}test_db.\.*,test_db2.\.*{*}`
 # remove `test_db2.\.*` from `tables` and restart pipeline job 
 # re-add  `test_db2.\.*` to `tables` and restart pipeline job

!image-2024-10-29-11-52-41-353.png!

 

## CDC 3.3

CreateTableEvent could be sent, but another exception occured in TM.
{code:java}
2025-01-21 19:11:49,934 ERROR 
io.debezium.connector.mysql.MySqlStreamingChangeEventSource  [] - Encountered 
change event 'Event{header=EventHeaderV4{timestamp=1737457909000, 
eventType=TABLE_MAP, serverId=1, headerLength=19, dataLength=79, 
nextPosition=945183051, flags=0}, data=TableMapEventData{tableId=2859, 
database='test_db2', table='t1', columnTypes=8, 15, 1, 15, 18, 18, 15, 15, 15, 
15, 15, 15, 15, 15, 15, 15, 15, 15, columnMetadata=0, 765, 0, 765, 0, 0, 2295, 
765, 765, 765, 765, 765, 765, 765, 765, 765, 765, 765, columnNullability={1, 3, 
6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17}, eventMetadata=null}}' at offset 
{transaction_id=null, ts_sec=1737457863, file=mysql-bin.000036, pos=945182869, 
server_id=1, event=1} for table test_db2.t1 whose schema isn't known to this 
connector. One possible cause is an incomplete database history topic. Take a 
new snapshot in this case.Use the mysqlbinlog tool to view the problematic 
event: mysqlbinlog --start-position=945182953 --stop-position=945183051 
--verbose mysql-bin.0000362025-01-21 19:11:49,935 ERROR 
io.debezium.connector.mysql.MySqlStreamingChangeEventSource  [] - Error during 
binlog processing. Last offset stored = {transaction_id=null, 
ts_sec=1737457863, file=mysql-bin.000036, pos=945182869, server_id=1, event=1}, 
binlog reader near position = mysql-bin.000036/9451829532025-01-21 19:12:15,472 
INFO  org.apache.doris.flink.sink.batch.DorisBatchWriter           [] - 
checkpoint flush triggered.2025-01-21 19:12:15,481 INFO  
org.apache.flink.cdc.connectors.mysql.source.reader.MySqlSourceReader [] - 
Binlog offset on checkpoint 5: {transaction_id=null, ts_sec=1737457863, 
file=mysql-bin.000036, pos=945192687, kind=SPECIFIC, row=0, event=0, 
server_id=1} {code}


> Downstream lost the CreateTableEvent when tables were removed and then 
> re-added
> -------------------------------------------------------------------------------
>
>                 Key: FLINK-36619
>                 URL: https://issues.apache.org/jira/browse/FLINK-36619
>             Project: Flink
>          Issue Type: Bug
>          Components: Flink CDC
>    Affects Versions: cdc-3.2.0, cdc-3.3.0
>            Reporter: linqigeng
>            Priority: Major
>         Attachments: image-2024-10-29-11-52-41-353.png, 
> image-2024-10-29-12-04-26-718.png
>
>
> #CDC 3.2:
> If some tables are removed from the `tables` parameter and then re-added, the 
> `CreateTableEvent` will be ignored and not sent to downstream.
> This way to reproduce:
>  # set `tables` to `{*}test_db.\.{*},test_db2.\.{*}{{*}}`
>  # remove `test_db2.\.*` from `tables` and restart pipeline job 
>  # re-add  `test_db2.\.*` to `tables` and restart pipeline job
> !image-2024-10-29-11-52-41-353.png!
>  
> #CDC 3.3
> CreateTableEvent could be sent, but another exception occured in TM.
> {code:java}
> 2025-01-21 19:11:49,934 ERROR 
> io.debezium.connector.mysql.MySqlStreamingChangeEventSource  [] - Encountered 
> change event 'Event{header=EventHeaderV4{timestamp=1737457909000, 
> eventType=TABLE_MAP, serverId=1, headerLength=19, dataLength=79, 
> nextPosition=945183051, flags=0}, data=TableMapEventData{tableId=2859, 
> database='test_db2', table='t1', columnTypes=8, 15, 1, 15, 18, 18, 15, 15, 
> 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, columnMetadata=0, 765, 0, 765, 0, 0, 
> 2295, 765, 765, 765, 765, 765, 765, 765, 765, 765, 765, 765, 
> columnNullability={1, 3, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17}, 
> eventMetadata=null}}' at offset {transaction_id=null, ts_sec=1737457863, 
> file=mysql-bin.000036, pos=945182869, server_id=1, event=1} for table 
> test_db2.t1 whose schema isn't known to this connector. One possible cause is 
> an incomplete database history topic. Take a new snapshot in this case.Use 
> the mysqlbinlog tool to view the problematic event: mysqlbinlog 
> --start-position=945182953 --stop-position=945183051 --verbose 
> mysql-bin.0000362025-01-21 19:11:49,935 ERROR 
> io.debezium.connector.mysql.MySqlStreamingChangeEventSource  [] - Error 
> during binlog processing. Last offset stored = {transaction_id=null, 
> ts_sec=1737457863, file=mysql-bin.000036, pos=945182869, server_id=1, 
> event=1}, binlog reader near position = mysql-bin.000036/9451829532025-01-21 
> 19:12:15,472 INFO  org.apache.doris.flink.sink.batch.DorisBatchWriter         
>   [] - checkpoint flush triggered.2025-01-21 19:12:15,481 INFO  
> org.apache.flink.cdc.connectors.mysql.source.reader.MySqlSourceReader [] - 
> Binlog offset on checkpoint 5: {transaction_id=null, ts_sec=1737457863, 
> file=mysql-bin.000036, pos=945192687, kind=SPECIFIC, row=0, event=0, 
> server_id=1} {code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to