8000 mysql cdc source: schema change while RW is offline fails the connector permanently · Issue #21801 · risingwavelabs/risingwave · GitHub
[go: up one dir, main page]
More Web Proxy on the site http://driver.im/
Skip to content

mysql cdc source: schema change while RW is offline fails the connector permanently #21801

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

Open
BugenZhao opened this issue May 9, 2025 · 6 comments
Assignees
Labels
S-need-design Status: A detailed design is needed before coding. Typically used for feat/refactor issues. type/bug Type: Bug. Only for issues.
Milestone

Comments

@BugenZhao
Copy link
Member
BugenZhao commented May 9, 2025

Describe the bug

After creating a MySQL CDC source/table, if we take RW offline, make schema changes on the MySQL side, and then restart RW, we will only receive the latest table schema. This results in a failure to parse the (unconsumed) binlog with the original schema, causing the connector to fail permanently.

Error message/log

2025-05-09T08:11:29.637686252Z ERROR risingwave_connector_node: Error processing after of row in t because it's different column size with internal schema size 3, but after size 2, restart connector with schema recovery mode. thread="blc-127.0.0.1:8306" class="io.debezium.connector.mysql.MySqlStreamingChangeEventSource"
2025-05-09T08:11:29.639607293Z ERROR risingwave_connector_node: Error during binlog processing. Last offset stored = {transaction_id=file=binlog.000003,pos=236, ts_sec=1746778289, file=binlog.000003, pos=236, server_id=1, event=2}, binlog reader near position = binlog.000003/314 thread="blc-127.0.0.1:8306" class="io.debezium.connector.mysql.MySqlStreamingChangeEventSource"
2025-05-09T08:11:29.640509709Z ERROR risingwave_connector_node: Producer failure thread="blc-127.0.0.1:8306" class="io.debezium.pipeline.ErrorHandler" error="io.debezium.DebeziumException: Error processing binlog event\n\tat io.debezium.connector.mysql.MySqlStreamingChangeEventSource.handleEvent(MySqlStreamingChangeEventSource.java:305)\n\tat io.debezium.connector.mysql.MySqlStreamingChangeEventSource.lambda$execute$30(MySqlStreamingChangeEventSource.java:880)\n\tat com.github.shyiko.mysql.binlog.BinaryLogClient.notifyEventListeners(BinaryLogClient.java:1272)\n\tat com.github.shyiko.mysql.binlog.BinaryLogClient.listenForEventPackets(BinaryLogClient.java:1098)\n\tat com.github.shyiko.mysql.binlog.BinaryLogClient.connect(BinaryLogClient.java:655)\n\tat com.github.shyiko.mysql.binlog.BinaryLogClient$7.run(BinaryLogClient.java:954)\n\tat java.base/java.lang.Thread.run(Thread.java:840)\nCaused by: io.debezium.DebeziumException: Error processing row in t, internal schema size 3, but row size 2 , restart connector with schema recovery mode.\n\tat io.debezium.connector.mysql.MySqlStreamingChangeEventSource.validateChangeEventWithTable(MySqlStreamingChangeEventSource.java:696)\n\tat io.debezium.connector.mysql.MySqlStreamingChangeEventSource.lambda$handleInsert$6(MySqlStreamingChangeEventSource.java:715)\n\tat io.debezium.connector.mysql.MySqlStreamingChangeEventSource.handleChange(MySqlStreamingChangeEventSource.java:778)\n\tat io.debezium.connector.mysql.MySqlStreamingChangeEventSource.handleInsert(MySqlStreamingChangeEventSource.java:710)\n\tat io.debezium.connector.mysql.MySqlStreamingChangeEventSource.lambda$execute$21(MySqlStreamingChangeEventSource.java:855)\n\tat io.debezium.connector.mysql.MySqlStreamingChangeEventSource.handleEvent(MySqlStreamingChangeEventSource.java:285)\n\t... 6 more\n"
2025-05-09T08:11:29.640656251Z  INFO risingwave_connector_node: Error processing binlog event, and propagating to Kafka Connect so it stops this connector. Future binlog events read before connector is shutdown will be ignored. thread="blc-127.0.0.1:8306" class="io.debezium.connector.mysql.MySqlStreamingChangeEventSource"

To Reproduce

a.sh

#!/bin/bash

set -euo pipefail

export MYSQL_HOST="127.0.0.1"
export MYSQL_TCP_PORT="8306"
export MYSQL_PWD=""

export USE_SYSTEM_RISINGWAVE="docker:v2.3.2"

# Cleanup
./risedev k && ./risedev clean-data

# Setup CDC table with initial schema
./risedev d bugen-mysql
./risedev slt a.slt

# Take RW offline
./risedev k

# Resume MySQL only, perform some writes, then schema change, then some more writes
./risedev d mysql-only
mysql -u root -D risedev -e "
    insert into t values (2, 'def');
    alter table t add column v2 int default 42;
    insert into t values (3, 'ghi', 88);
"
./risedev k

# Resume RW CDC
./risedev d bugen-mysql

# Verify data
# If the bug is reproduced, you won't see rows with k=2 and k=3, check the logs of compute-node!
sleep 2
./risedev psql -c "select * from t;"

a.slt

control substitution on

system ok
mysql -e "
    USE risedev;
    DROP TABLE IF EXISTS t;
    CREATE TABLE t (k int primary key, v text);
"


statement ok
create source s with (
  ${RISEDEV_MYSQL_WITH_OPTIONS_COMMON},
  username = '$RISEDEV_MYSQL_USER',
  password = '$MYSQL_PWD',
  database.name = 'risedev',
);


statement ok
create table t (
  k int primary key,
  v text
) from s table 'risedev.t';


system ok
mysql -e "
    USE risedev;
    INSERT INTO t VALUES (1, 'abc');
"


sleep 2s

query IT
select * from t;
----
1 abc

risedev-profiles.user.yml

bugen-mysql:
  steps:
    - use: sqlite
    - use: minio
    - use: meta-node
      meta-backend: sqlite
    - use: compute-node
    - use: frontend
    - use: compactor
    - use: mysql

mysql-only:
  steps:
    - use: mysql

Then run a.sh.

Expected behavior

Should sync all records correctly and never fail.

How did you deploy RisingWave?

No response

The version of RisingWave

2.3.2 (also latest main)

Additional context

Decoding binlog requires schema information. And if there's schema change, we need a complete schema history. However, we are using MemorySchemaHistory and this might be the cause.

schema.history.internal=io.debezium.relational.history.MemorySchemaHistory

After RW is taking back online, it can only see the latest MySQL table schema because all history is gone. If there's still unconsumed binlog with original schema, Debezium will fail to parse it.

This topic can be quite complicated. Please refer to the Debezium docs for a comprehensive understanding: https://debezium.io/documentation/reference/stable/connectors/mysql.html#understanding-why-initial-snapshots-capture-the-schema-history-for-all-tables

@BugenZhao BugenZhao added the type/bug Type: Bug. Only for issues. label May 9, 2025
@github-actions github-actions bot added this to the release-2.5 milestone May 9, 2025
@wcy-fdu wcy-fdu self-assigned this May 9, 2025
@BugenZhao BugenZhao added the S-need-design Status: A detailed design is needed before coding. Typically used for feat/refactor issues. label May 12, 2025
@hzxa21
Copy link
Collaborator
hzxa21 commented May 13, 2025

embedded dbz seems to suggest that MemorySchemaHistory is only intended to be used in test environment. In production, FileSchemaHistory should be used.

https://debezium.io/documentation/reference/2.4/development/engine.html
https://www.bookstack.cn/read/debezium-2.0-en/73537b6e75c89490.md?wd=SQL%20Server

@hzxa21
Copy link
Collaborator
hzxa21 commented May 13, 2025

Oh, we probably can use S3SchemaHistory

@wcy-fdu
Copy link
Contributor
wcy-fdu commented May 13, 2025

Thanks for sharing, will do some further investigation later.

@wcy-fdu
Copy link
Contributor
wcy-fdu commented May 13, 2025

Update: I just verified and solved this problem locally using FileSchemaHistory, and continued to investigate S3SchemaHistory.

@wcy-fdu
Copy link
Contributor
wcy-fdu commented May 22, 2025

We(me and @hzxa21 ) have identified the root cause of the issue. Basicly there are two key parameters, one is SchemaHistory, and the other is snapshot.mode.

For snapshot.mode,

  • no_data means that when no offset is provided, the snapshot schema is taken, and when an offset is offered, it recovers directly from the schema history.
  • recovery means assuming that the schema history is lost or corrupted, and restoring the schema history from the latest available state.

Previously, we were using MemorySchemaHistory, which would inevitably be lost after a cluster restart, so we had to use recovery mode. The solution here is to use a persistent SchemaHistory along with no_data as the schema.mode.

Moving forward, I will explore some implementations of persistent SchemaHistory to ensure compatibility with RisingWave Cloud (AWS, GCP, and Azure).

@wcy-fdu
Copy link
Contributor
wcy-fdu commented May 22, 2025

While debugging this issue, we discovered another issue.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
S-need-design Status: A detailed design is needed before coding. Typically used for feat/refactor issues. type/bug Type: Bug. Only for issues.
Projects
None yet
Development

No branches or pull requests

3 participants
0