Skip to content

fix(mysql-cdc): use connection pool to avoid data loss caused by unsafe cancellation #22128

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
wants to merge 11 commits into
base: main
Choose a base branch
from

Conversation

wcy-fdu
Copy link
Contributor

@wcy-fdu wcy-fdu commented Jun 5, 2025

I hereby agree to the terms of the RisingWave Labs, Inc. Contributor License Agreement.

What's changed and what's your intention?

In the implementation of MySQL CDC, we use mysql_async to read historical data from MySQL. We utilize the exec_drop interface, but there is a cancellation safety issue: if a drop occurs while exec_drop is running, and if the connection is not rebuilt, subsequent calls to read_stream will return empty. At this point, the upper layer will mistakenly believe that the backfill has ended, leading to data loss.

One way to simulate and reproduce this error is to construct a case where exec_drop is dropped:

 let query  = conn.exec_drop("SET time_zone = if(not sleep(2), \"+00:00\", \"\")", ());

        match tokio::time::timeout(Duration::from_secs(1), query).await {
            Ok(result) => {
                println!("Operation completed: {:?}", result);
            },
            Err(_) => {
                println!("Operation timed out");
            }
        };

In the above example, after exec_drop times out and is dropped, using this connection afterwards will result in an error: Io(Io(Custom { kind: UnexpectedEof, error: "can't parse: buf doesn't have enough data" }))

After reviewing the official documentation for mysql_async, I believe we should use the recommended connection pool. When we need a connection, we can use get_conn and then drop the connection with drop(conn) once we're done. Additionally, after backfilling and reading all historical data, we should first run pool.disconnect before dropping snapshot_stream. This approach can avoid cancellation safety issues and elegantly reclaim resources.

Note that, it's a bit hard to construct a scenario where a drop occurs exactly during the execution of exec_drop, so we constructed a unit test using minimal reproducible code and ran it using the MySQL environment in e2e-source-test.

Checklist

  • I have written necessary rustdoc comments.
  • I have added necessary unit tests and integration tests.
  • I have added test labels as necessary.
  • I have added fuzzing tests or opened an issue to track them.
  • My PR contains breaking changes.
  • My PR changes performance-critical code, so I will run (micro) benchmarks and present the results.
  • I have checked the Release Timeline and Currently Supported Versions to determine which release branches I need to cherry-pick this PR into.

Documentation

  • My PR needs documentation updates.
Release note

@github-actions github-actions bot added the type/fix Type: Bug fix. Only for pull requests. label Jun 5, 2025
@wcy-fdu
Copy link
Contributor Author

wcy-fdu commented Jun 5, 2025

CI failed, under investigation

@wcy-fdu wcy-fdu requested review from hzxa21 and yuhao-su June 9, 2025 02:20
@wcy-fdu
Copy link
Contributor Author

wcy-fdu commented Jun 9, 2025

cc @hzxa21 for reviewing, @yuhao-su I heard you found another potential data loss situation. This PR is caused by mysql_async, just to let you know.

@@ -375,10 +374,15 @@ impl ExternalTableReader for MySqlExternalTableReader {
) -> BoxStream<'_, ConnectorResult<OwnedRow>> {
self.snapshot_read_inner(table_name, start_pk, primary_keys, limit)
}

async fn disconnect(&self) -> ConnectorResult<()> {
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You can pass ownership of self here so that you don't need to clone the pool: async fn disconnect(self)

Comment on lines +247 to +250
async fn disconnect(&self) -> StreamExecutorResult<()> {
self.reader.disconnect().await?;
Ok(())
}
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ditto

@@ -200,6 +200,12 @@ pub type CdcOffsetParseFunc = Box<dyn Fn(&str) -> ConnectorResult<CdcOffset> + S
pub trait ExternalTableReader {
async fn current_cdc_offset(&self) -> ConnectorResult<CdcOffset>;

// Currently, MySQL cdc uses a connection pool to manage connections to MySQL, and other CDC processes do not require the disconnect step for now.
#[allow(clippy::unused_async)]
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why do we need this? I thought it is always used.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is because other cdc(pg, mssql..) does not need this method.

@hzxa21
Copy link
Collaborator

hzxa21 commented Jun 10, 2025

@cyliu0 @wcy-fdu I remembered we have a test pipeline for cdc. Given that this is a critical correctness fix, can we trigger the test pipeline before mering this PR?

@wcy-fdu wcy-fdu added the A-cdc Area: CDC. label Jun 10, 2025
@wcy-fdu wcy-fdu added mysql-cdc and removed A-cdc Area: CDC. mysql-cdc labels Jun 10, 2025
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
mysql-cdc type/fix Type: Bug fix. Only for pull requests.
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants