Skip to content
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

feat(cdc): in-memory backfill for single mysql cdc table #11707

Merged
merged 80 commits into from
Aug 30, 2023

Conversation

StrikeW
Copy link
Contributor

@StrikeW StrikeW commented Aug 16, 2023

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

What's changed and what's your intention?

This PR is the first part of the risingwavelabs/rfcs#63. It implements a CdcBackfillExecutor to take care of the backfill logic for a CDC table. Right now, we will instantiate a CdcBackfillExecutor to wrap a source executor so that we can use the changelog events to backfill the table.

Main changes are as follows:

  • Add a session variable cdc_backfill to gate the this new feature
  • Add a StreamChunkMeta field to the StreamChunk for storing offsets of change log events, because the backfill algorithm needs to filter events based on offset (snapshot chunk doesn't have offsets)
  • ExternalTableReader component for reading snapshot and binlog from upstream table, this PR implements a table reader for MySQL
  • Frontend will store external table information as a field in StreamSourceInfo in the process of CREATE TABLE

TODO:

  • This PR supports backfill for the MySQL table, Postgres support will be done in another PR
  • Records backfill offset so that we can resume the backfill progress upon recovery
    - Rewrite primary key comparison conditions for MySQL
    - Support more MySQL data types

tracked: #11079

Checklist

  • I have written necessary rustdoc comments
  • I have added necessary unit tests and integration tests
  • I have added fuzzing tests or opened an issue to track them. (Optional, recommended for new SQL features Sqlsmith: Sql feature generation #7934).
  • My PR contains breaking changes. (If it deprecates some features, please create a tracking issue to remove them in the future).
  • All checks passed in ./risedev check (or alias, ./risedev c)
  • My PR changes performance-critical code. (Please run macro/micro-benchmarks and show the results.)
  • My PR contains critical fixes that are necessary to be merged into the latest release. (Please check out the details)

Documentation

  • My PR needs documentation updates. (Please use the Release note section below to summarize the impact on users)

Release note

Enhance the mysql-cdc connector to allow lock-free ingestion, the connector won't need to lock upstream tables.
set cdc_backfill="true" to enable this feature.

@StrikeW StrikeW force-pushed the siyuan/cdc-backfill-framework-single-table branch from e7b76ff to 05d3de8 Compare August 30, 2023 03:10
src/connector/src/source/external.rs Show resolved Hide resolved
};
(pk.clone(), val)
} else {
(pk.clone(), Value::NULL)
Copy link
Collaborator

Choose a reason for hiding this comment

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

This will generate filter conditions like a > NULL or a = NULL, which are always evaluated to false. Is this expected?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Since primary key cannot be null in mysql, I fixed by returning an error to the caller.

}

/// A wrapper of upstream table for snapshot read
/// because we need to customize the snapshot read for managed upstream table (e.g. mv, index)
Copy link
Collaborator

Choose a reason for hiding this comment

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

UpstreamTableReader is only used for cdc backfill. Based on the doc here, are we planning to use the same trait for cdc backfill and table/mv backfill?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Ideally, I want to unify the implementation of backfill for mv and backfill for external table. But the fact is that it is a little hard to tune the implementation details to become a unified backfill executor. May revisit this part in future.

@@ -506,10 +506,11 @@ where
)
.await?;
Copy link
Collaborator

Choose a reason for hiding this comment

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

nits: we can use map to call KeyedRow::into_owned_row to avoid introdcuing the owned_row_iter method.

@@ -540,7 +540,7 @@ where
.await?;
Copy link
Collaborator

Choose a reason for hiding this comment

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

nits: we can use map to call KeyedRow::into_owned_row to avoid introdcuing the owned_row_iter method.

let order_key = primary_keys.iter().join(",");
let sql = if start_pk.is_none() {
format!(
"SELECT {} FROM {} ORDER BY {}",
Copy link
Member

Choose a reason for hiding this comment

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

Could there be potential injection attacks here?

Copy link
Collaborator

@hzxa21 hzxa21 Aug 30, 2023

Choose a reason for hiding this comment

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

Could there be potential injection attacks here?

Are spaces allowed in the RW field name?

Copy link
Member

Choose a reason for hiding this comment

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

It seems possible.

dev=> create table t ("1; DROP DATABASE foo; SELECT 1" int);
CREATE_TABLE
dev=> select * from t;
 1; DROP DATABASE foo; SELECT 1 
--------------------------------
(0 rows)

Similar issues: #4217

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Thanks for the reminder🥵

Copy link
Contributor Author

@StrikeW StrikeW Aug 30, 2023

Choose a reason for hiding this comment

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

I will unify to binary protocol in a separate PR.

@StrikeW StrikeW enabled auto-merge August 30, 2023 07:34
@StrikeW StrikeW added this pull request to the merge queue Aug 30, 2023
@StrikeW StrikeW removed this pull request from the merge queue due to a manual request Aug 30, 2023
@StrikeW StrikeW enabled auto-merge August 30, 2023 09:42
@StrikeW StrikeW added this pull request to the merge queue Aug 30, 2023
Merged via the queue into main with commit 15f4fdd Aug 30, 2023
36 of 37 checks passed
@StrikeW StrikeW deleted the siyuan/cdc-backfill-framework-single-table branch August 30, 2023 11:39
@kwannoel
Copy link
Contributor

Add a session variable cdc_backfill to gate the this new feature

Seems like it is set to true by default https://github.com/risingwavelabs/risingwave/pull/11707/files#diff-4e19861816f5cf913890db3abd8088d01fa135d85f43080d5e6ee5c6c8ac40aaR337

Is that expected?

@StrikeW
Copy link
Contributor Author

StrikeW commented Aug 31, 2023

Add a session variable cdc_backfill to gate the this new feature

Seems like it is set to true by default https://github.com/risingwavelabs/risingwave/pull/11707/files#diff-4e19861816f5cf913890db3abd8088d01fa135d85f43080d5e6ee5c6c8ac40aaR337

Is that expected?

Yes.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

8 participants