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): persist the backfill state for table-on-source #13276

Merged
merged 22 commits into from
Nov 14, 2023

Conversation

StrikeW
Copy link
Contributor

@StrikeW StrikeW commented Nov 7, 2023

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

What's changed and what's your intention?

Add a state table (table_id | pk columns... | backfill_finished | row_count | cdc_offset) to the CdcBackfill executor to persist the backfill progress during backfill. The pk columns would be multiple columns if the table has composite primary keys.

close #13204

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

If this PR includes changes that directly affect users or other significant modifications relevant to the community, kindly draft a release note to provide a concise summary of these changes. Please prioritize highlighting the impact these changes will have on users.

@StrikeW StrikeW force-pushed the siyuan/cdc-backfill-state branch from 96991d1 to 7b20a95 Compare November 7, 2023 05:15
Copy link

codecov bot commented Nov 7, 2023

Codecov Report

Attention: 185 lines in your changes are missing coverage. Please review.

Comparison is base (0020507) 67.87% compared to head (d8fd8a2) 67.86%.
Report is 2 commits behind head on main.

Files Patch % Lines
src/stream/src/executor/backfill/cdc/state.rs 32.71% 72 Missing ⚠️
src/meta/service/src/ddl_service.rs 0.00% 62 Missing ⚠️
...d/src/optimizer/plan_node/stream_cdc_table_scan.rs 0.00% 24 Missing ⚠️
...c/stream/src/executor/backfill/cdc/cdc_backfill.rs 35.71% 9 Missing ⚠️
src/stream/src/from_proto/stream_scan.rs 0.00% 8 Missing ⚠️
src/meta/src/manager/catalog/mod.rs 0.00% 3 Missing ⚠️
src/stream/src/from_proto/source/trad_source.rs 0.00% 2 Missing ⚠️
src/common/src/catalog/external_table.rs 0.00% 1 Missing ⚠️
src/connector/src/parser/mod.rs 0.00% 1 Missing ⚠️
src/connector/src/source/external.rs 66.66% 1 Missing ⚠️
... and 2 more
Additional details and impacted files
@@            Coverage Diff             @@
##             main   #13276      +/-   ##
==========================================
- Coverage   67.87%   67.86%   -0.01%     
==========================================
  Files        1526     1526              
  Lines      259952   260043      +91     
==========================================
+ Hits       176432   176479      +47     
- Misses      83520    83564      +44     
Flag Coverage Δ
rust 67.86% <19.21%> (-0.01%) ⬇️

Flags with carried forward coverage won't be shown. Click here to find out more.

☔ View full report in Codecov by Sentry.
📢 Have feedback on the report? Share it here.

@StrikeW StrikeW marked this pull request as draft November 8, 2023 13:30
@kwannoel
Copy link
Contributor

kwannoel commented Nov 9, 2023

Is there some test for cdc backfill over recovery?

@StrikeW
Copy link
Contributor Author

StrikeW commented Nov 9, 2023

Is there some test for cdc backfill over recovery?

Right now only has e2e test. I insert new data into MV created in cdc.share_stream.slt after the cluster has been shutdown, then restart the cluster and check that new data has been ingested.

@StrikeW StrikeW marked this pull request as ready for review November 9, 2023 05:49
@StrikeW StrikeW requested a review from fuyufjh November 9, 2023 05:49
}

/// Mark the backfill has done and save the last cdc offset
pub async fn mutate_state(&mut self, state_item: CdcStateItem) -> StreamExecutorResult<()> {
Copy link
Contributor

Choose a reason for hiding this comment

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

Suggested change
pub async fn mutate_state(&mut self, state_item: CdcStateItem) -> StreamExecutorResult<()> {
pub async fn finish_state(&mut self, state_item: CdcStateItem) -> StreamExecutorResult<()> {

Copy link
Contributor Author

Choose a reason for hiding this comment

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

The comment is misleading, the function is not called only after the backfill is complete. We use this function to update the backfill state.


Ok(CdcStateItem {
is_finished: finished,
..Default::default()
Copy link
Contributor

Choose a reason for hiding this comment

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

Hmm should we only recover is_finished? What about last cdc offset, row count etc

Copy link
Contributor Author

Choose a reason for hiding this comment

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

The cdc offset and row count is only for observability, the executor doesn't rely on these fields.

@StrikeW StrikeW requested a review from kwannoel November 9, 2023 11:31
@StrikeW
Copy link
Contributor Author

StrikeW commented Nov 9, 2023

Existing comments addressed. PTAL @hzxa21 @fuyufjh @BugenZhao

Copy link
Member

@fuyufjh fuyufjh left a comment

Choose a reason for hiding this comment

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

Mostly LGTM.

This PR only records a boolean flag - completed or not in the persisted state, intead of a real progress (marked by PK perhaps). I think we had better complete this before releasing to users, because we can't just tell users the progress is persisted but either 0% or 100%.

src/frontend/src/optimizer/plan_node/stream_table_scan.rs Outdated Show resolved Hide resolved
Comment on lines 39 to +42
pub enum CdcBackfillStateImpl<S: StateStore> {
Undefined,
SingleTable(SingleTableState<S>),
SingleTable(SingleBackfillState<S>),
MultiTable(MultiBackfillState<S>),
}
Copy link
Member

Choose a reason for hiding this comment

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

In the future, shall we deprecate the SingleTable option (and remove the enum)? It seems not worth to maintain it because MultiTable can completely cover its use case.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I also want to deprecate it. But we need to change the original Table job plan from source -> mview to source -> stream_scan -> mview, so that it can has its own state, technically it is feasible.

Copy link
Collaborator

Choose a reason for hiding this comment

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

+1. Currently we have 3 ways to use mysql-cdc:

  • create table xxx with (....)
  • set cdc_backfill='true' + create table xxx with (...)
  • set cdc_backfill=true + create source with (...) + create table from ...

It is confused and not easy to explain to user which one to choose. Ideally we should only have 2 ways:

  1. create table xxx with (....)
  2. create source with (...) + create table from ...

1 and 2 can share the same implementation and 1 can be a syntax suger for 2 if only user only needs to cdc one table from the db.

Copy link
Collaborator

Choose a reason for hiding this comment

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

The question is whether we need to ensure compatibility to make existing jobs created via set cdc_backfill='true' + create table xxx with (...) to be compatible. IMO, we can keep it simple and avoid compatibility codes here since cdc_backfill is announced as experimental feature only.

src/stream/src/from_proto/stream_scan.rs Show resolved Hide resolved
src/meta/service/src/ddl_service.rs Show resolved Hide resolved
src/meta/service/src/ddl_service.rs Show resolved Hide resolved
Copy link
Contributor Author

@StrikeW StrikeW left a comment

Choose a reason for hiding this comment

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

I updated the pr to support recoverable backfill, that is to record the backfill progress and can continue the progress upon cluster recovery.

src/stream/src/from_proto/stream_scan.rs Show resolved Hide resolved
src/meta/service/src/ddl_service.rs Show resolved Hide resolved
Comment on lines 39 to +42
pub enum CdcBackfillStateImpl<S: StateStore> {
Undefined,
SingleTable(SingleTableState<S>),
SingleTable(SingleBackfillState<S>),
MultiTable(MultiBackfillState<S>),
}
Copy link
Contributor Author

Choose a reason for hiding this comment

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

I also want to deprecate it. But we need to change the original Table job plan from source -> mview to source -> stream_scan -> mview, so that it can has its own state, technically it is feasible.

@StrikeW StrikeW force-pushed the siyuan/cdc-backfill-state branch from bec0206 to 26a6113 Compare November 13, 2023 03:31
@StrikeW StrikeW requested review from hzxa21 and fuyufjh November 13, 2023 03:31
let mut key = self.split_id.to_string();
key.push_str(BACKFILL_STATE_KEY_SUFFIX);
// write backfill finished flag
self.source_state_handler
.set(key.into(), JsonbVal::from(Value::Bool(true)))
.set(
key.into(),
Copy link
Collaborator

Choose a reason for hiding this comment

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

I just realized that we are treating the source state table as a key value store and setting two keys (<split_id>_backfill here and <split_id> in L298). I feel that this can make the source state table contains different primary key schema. Will this break the read path of the source state table?

Copy link
Contributor Author

@StrikeW StrikeW Nov 13, 2023

Choose a reason for hiding this comment

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

Since for the backfill for single table scenario, we didn't introduce a new operator to the query plan. Just wraps the source executor in the cdc backfill executor and reuse the state table of Source. We access source state table only via point query with the key column, so it won't break.

Copy link
Collaborator

Choose a reason for hiding this comment

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

True. If only get is used in the streaming job, there won't be unexpected result. But when querying source state via SQL, we can see something like this:

partition_id  |                                                                                                                                                                      offset_info                                                                                                                                                                       
---------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
 1002          | {"split_info": {"mysql_split": {"inner": {"snapshot_done": false, "split_id": 1002, "start_offset": "{\"sourcePartition\":{\"server\":\"RW_CDC_1002\"},\"sourceOffset\":{\"transaction_id\":null,\"ts_sec\":1699866101,\"file\":\"binlog.000012\",\"pos\":157,\"server_id\":1},\"isHeartbeat\":true}"}}, "pg_split": null}, "split_type": "mysql-cdc"}
 1002_backfill | true
(2 rows)

This looks very strange to me. How about storing the backfill finished flag inside the json as a new field under "mysql_split"? This issue is introduced in #12535 and not related to the changes in this PR. I am okay with handling it separately.

Copy link
Contributor Author

@StrikeW StrikeW Nov 13, 2023

Choose a reason for hiding this comment

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

Yeah, it is a bit dirty. As mentioned in #13276 (comment), I plan to refactor the plan of single table cdc backfill to make the cdc backfill become a separate operator instead of just wraps the source inside it. So that we don't need to write state to source state table.

Copy link
Collaborator

@hzxa21 hzxa21 left a comment

Choose a reason for hiding this comment

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

LGTM

@kwannoel
Copy link
Contributor

I updated the pr to support recoverable backfill, that is to record the backfill progress and can continue the progress upon cluster recovery.

Could you update the PR description and add a test for this as well (separate PR is fine as well, as long as there's an issue to track it).

Previously the test only tests for case where cdc backfill is done.

@kwannoel
Copy link
Contributor

Also do we need both pk and cdc offset?
If we have pk already, seems we don't need cdc offset?

@StrikeW
Copy link
Contributor Author

StrikeW commented Nov 14, 2023

Thanks for the reminder, I will add more test for the recoverable backfill part, tracked by #13400

Also do we need both pk and cdc offset? If we have pk already, seems we don't need cdc offset?

The persisted cdc_offset acts as a low watermark for upstream events to ignore events with offset that less than the cdc_offset. And also may help us debuging the backfill process.

Copy link
Member

@fuyufjh fuyufjh left a comment

Choose a reason for hiding this comment

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

LGTM

@StrikeW StrikeW enabled auto-merge November 14, 2023 05:28
@StrikeW StrikeW added this pull request to the merge queue Nov 14, 2023
@github-merge-queue github-merge-queue bot removed this pull request from the merge queue due to failed status checks Nov 14, 2023
@StrikeW StrikeW added this pull request to the merge queue Nov 14, 2023
Merged via the queue into main with commit 3b7036c Nov 14, 2023
8 of 9 checks passed
@StrikeW StrikeW deleted the siyuan/cdc-backfill-state branch November 14, 2023 07:57
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.

Persist the backfill state to record backfill progress
4 participants