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

Yohei Yoshimuta updated FLINK-36945:
------------------------------------
    Description: 
[The Vitess schema migration 
tool|https://vitess.io/docs/user-guides/schema-changes/ddl-strategies/] uses 
`RENAME TABLE` to perform schema changes.

However, the MySQL CDC connector does not account for these changes, causing 
the schema history topic in Debezium to become stale. While this issue does not 
immediately affect a running job, it prevents the job from restarting 
successfully from a checkpoint and results in the following error:
{code:java}
Caused by: 
com.ververica.cdc.connectors.shaded.org.apache.kafka.connect.errors.ConnectException:
 Data row is smaller than a column index, internal schema representation is 
probably out of sync with real database schema
at 
io.debezium.relational.TableSchemaBuilder.validateIncomingRowToInternalMetadata(TableSchemaBuilder.java:254)
at 
io.debezium.relational.TableSchemaBuilder.lambda$createValueGenerator$5(TableSchemaBuilder.java:283)
at io.debezium.relational.TableSchema.valueFromColumnData(TableSchema.java:141)
at 
io.debezium.relational.RelationalChangeRecordEmitter.emitUpdateRecord(RelationalChangeRecordEmitter.java:139)
at 
io.debezium.relational.RelationalChangeRecordEmitter.emitChangeRecords(RelationalChangeRecordEmitter.java:60)
at 
io.debezium.pipeline.EventDispatcher.dispatchDataChangeEvent(EventDispatcher.java:209)
... 12 more
{code}
When this happens, the database history topic needs to be rebuilt, but the job 
cannot be automatically recovered.

The current workaround is to set `scan.startup.mode` to `specific-offset`, 
which forces CDC to pass `schema_only_recovery` to Debezium. However, this 
requires manual intervention.

Examples of potential solutions include:
 - Enhancing the schema history mechanism to capture and process `RENAME TABLE` 
events during schema migrations.

 - Implementing a fallback mechanism to reconcile schema differences during 
recovery, reducing the need for manual intervention.

  was:
[The Vitess schema migration 
tool|https://vitess.io/docs/user-guides/schema-changes/ddl-strategies/] uses 
`RENAME TABLE` to perform schema changes.

However, the MySQL CDC connector does not account for these changes, causing 
the schema history topic in Debezium to become stale. While this issue does not 
immediately affect a running job, it prevents the job from restarting 
successfully from a checkpoint and results in the following error:

 
{code:java}
Caused by: 
com.ververica.cdc.connectors.shaded.org.apache.kafka.connect.errors.ConnectException:
 Data row is smaller than a column index, internal schema representation is 
probably out of sync with real database schema
at 
io.debezium.relational.TableSchemaBuilder.validateIncomingRowToInternalMetadata(TableSchemaBuilder.java:254)
at 
io.debezium.relational.TableSchemaBuilder.lambda$createValueGenerator$5(TableSchemaBuilder.java:283)
at io.debezium.relational.TableSchema.valueFromColumnData(TableSchema.java:141)
at 
io.debezium.relational.RelationalChangeRecordEmitter.emitUpdateRecord(RelationalChangeRecordEmitter.java:139)
at 
io.debezium.relational.RelationalChangeRecordEmitter.emitChangeRecords(RelationalChangeRecordEmitter.java:60)
at 
io.debezium.pipeline.EventDispatcher.dispatchDataChangeEvent(EventDispatcher.java:209)
... 12 more
{code}
 

When this happens, the database history topic needs to be rebuilt, but the job 
cannot be automatically recovered.

 

The current workaround is to set `scan.startup.mode` to `specific-offset`, 
which forces CDC to pass `schema_only_recovery` to Debezium. However, this 
requires manual intervention.

 

Examples of potential solutions include:
 - Enhancing the schema history mechanism to capture and process `RENAME TABLE` 
events during schema migrations.

 - Implementing a fallback mechanism to reconcile schema differences during 
recovery, reducing the need for manual intervention.


> MySQL CDC internal schema representation becomes out of sync with the real 
> database schema when restarting a job
> ----------------------------------------------------------------------------------------------------------------
>
>                 Key: FLINK-36945
>                 URL: https://issues.apache.org/jira/browse/FLINK-36945
>             Project: Flink
>          Issue Type: Bug
>          Components: Flink CDC
>            Reporter: Yohei Yoshimuta
>            Priority: Major
>
> [The Vitess schema migration 
> tool|https://vitess.io/docs/user-guides/schema-changes/ddl-strategies/] uses 
> `RENAME TABLE` to perform schema changes.
> However, the MySQL CDC connector does not account for these changes, causing 
> the schema history topic in Debezium to become stale. While this issue does 
> not immediately affect a running job, it prevents the job from restarting 
> successfully from a checkpoint and results in the following error:
> {code:java}
> Caused by: 
> com.ververica.cdc.connectors.shaded.org.apache.kafka.connect.errors.ConnectException:
>  Data row is smaller than a column index, internal schema representation is 
> probably out of sync with real database schema
> at 
> io.debezium.relational.TableSchemaBuilder.validateIncomingRowToInternalMetadata(TableSchemaBuilder.java:254)
> at 
> io.debezium.relational.TableSchemaBuilder.lambda$createValueGenerator$5(TableSchemaBuilder.java:283)
> at 
> io.debezium.relational.TableSchema.valueFromColumnData(TableSchema.java:141)
> at 
> io.debezium.relational.RelationalChangeRecordEmitter.emitUpdateRecord(RelationalChangeRecordEmitter.java:139)
> at 
> io.debezium.relational.RelationalChangeRecordEmitter.emitChangeRecords(RelationalChangeRecordEmitter.java:60)
> at 
> io.debezium.pipeline.EventDispatcher.dispatchDataChangeEvent(EventDispatcher.java:209)
> ... 12 more
> {code}
> When this happens, the database history topic needs to be rebuilt, but the 
> job cannot be automatically recovered.
> The current workaround is to set `scan.startup.mode` to `specific-offset`, 
> which forces CDC to pass `schema_only_recovery` to Debezium. However, this 
> requires manual intervention.
> Examples of potential solutions include:
>  - Enhancing the schema history mechanism to capture and process `RENAME 
> TABLE` events during schema migrations.
>  - Implementing a fallback mechanism to reconcile schema differences during 
> recovery, reducing the need for manual intervention.



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

Reply via email to