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

fix(planner): correctly handle hidden columns for SourceBackfill #19578

Merged
merged 6 commits into from
Nov 27, 2024

Conversation

xxchan
Copy link
Member

@xxchan xxchan commented Nov 26, 2024

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

What's changed and what's your intention?

fix #19575

Previously we add the hidden column in StreamSource/StreamSourceScan::new.
It works for StreamSource (i.e., when create source, because it's a simple plan), but not for StreamSourceScan (create MV). The index will be wrong when there are JOIN, etc above.
We cannot put in LogicalSource neither, it should be in binder.

Also add e2e & planner nexmark & tpch tests Moved to #19589

Checklist

  • I have written necessary rustdoc comments
  • I have added necessary unit tests and integration tests
  • I have added test labels as necessary. See details.
  • 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.

@github-actions github-actions bot added the type/fix Bug fix label Nov 26, 2024
@xxchan xxchan requested a review from st1page November 26, 2024 09:58
@xxchan xxchan force-pushed the xxchan/sporting-gopher branch from 57fc1c4 to fd2f157 Compare November 26, 2024 13:52
@xxchan xxchan force-pushed the xxchan/sporting-gopher branch 3 times, most recently from fcd5805 to 64cc6ce Compare November 26, 2024 15:30
@xxchan xxchan requested a review from BugenZhao November 26, 2024 16:12
@graphite-app graphite-app bot requested a review from a team November 26, 2024 16:18
@xxchan xxchan requested review from chenzl25 and fuyufjh November 27, 2024 02:40
@xxchan xxchan changed the title fix: correctly handle hidden columns for SourceBackfill fix(planner): correctly handle hidden columns for SourceBackfill Nov 27, 2024
@BugenZhao
Copy link
Member

Shall we separate the newly introduced test cases into a different PR? 🤡

Copy link
Member Author

xxchan commented Nov 27, 2024

@xxchan
Copy link
Member Author

xxchan commented Nov 27, 2024

Shall we separate the newly introduced test cases into a different PR? 🤡

Done

@xxchan xxchan force-pushed the xxchan/sporting-gopher branch from e61510f to 39e77fb Compare November 27, 2024 06:14
Copy link
Member

@BugenZhao BugenZhao left a comment

Choose a reason for hiding this comment

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

rubber stamp

@xxchan xxchan requested a review from stdrc November 27, 2024 06:21
Comment on lines 287 to 297
let mut last_column_id = if skip_col_id {
// col id will be filled outside later. Here just use a placeholder.
ColumnId::new(0)
} else {
max_column_id(columns)
};
Copy link
Member

Choose a reason for hiding this comment

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

I don't see where we fill the ColumnId, can you elaborate?

Copy link
Member

Choose a reason for hiding this comment

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

Is this really safe?

Copy link
Member Author

Choose a reason for hiding this comment

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

// XXX: why do we use col_id_gen here? It doesn't seem to be very necessary.
// XXX: should we also chenge the col id for struct fields?
for c in &mut columns {
c.column_desc.column_id = col_id_gen.generate(c.name())
}
debug_assert_column_ids_distinct(&columns);

Copy link
Member

Choose a reason for hiding this comment

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

😄🤔 What is chenge?

Copy link
Member

@stdrc stdrc Nov 27, 2024

Choose a reason for hiding this comment

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

Can you link to the function in create_source.rs from the comment of ColumnId::new(0)? I don't feel confident in terms of future maintenance.

Copy link
Contributor

@tabVersion tabVersion Nov 27, 2024

Choose a reason for hiding this comment

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

+1, why we need a placeholder here

shall we always place a dummy column_id when handling additional columns and fill the real one by col_id_gen.generate

Copy link
Contributor

Choose a reason for hiding this comment

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

please use ColumnId::placeholder()

Copy link
Member Author

Choose a reason for hiding this comment

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

Because at this stage the col ids are ColumnId::placeholder() (i32::MAX - 1), and will overflow here.

Copy link
Member

Choose a reason for hiding this comment

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

Because at this stage the col ids are ColumnId::placeholder() (i32::MAX - 1), and will overflow here.

Then I suppose doing incrementation on a dummy column ID can be worse.🤔

Copy link
Member Author

Choose a reason for hiding this comment

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

updated

@xxchan
Copy link
Member Author

xxchan commented Nov 27, 2024

So previously, StreamSource plan node contains these columns, but SourceCatalog doesn't contain them.

Comment on lines +1561 to +1574
// For shared sources, we will include partition and offset cols in the SourceExecutor's *output*, to be used by the SourceBackfillExecutor.
// For shared CDC source, the schema is different. See debezium_cdc_source_schema, CDC_BACKFILL_TABLE_ADDITIONAL_COLUMNS
if is_shared_non_cdc {
let (columns_exist, additional_columns) = source_add_partition_offset_cols(
&columns,
&with_properties.get_connector().unwrap(),
true, // col_id filled below at col_id_gen.generate
);
for (existed, c) in columns_exist.into_iter().zip_eq_fast(additional_columns) {
if !existed {
columns.push(ColumnCatalog::hidden(c));
}
}
}
Copy link
Member Author

Choose a reason for hiding this comment

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

I'm thinking that previously due to the logic in the planner, shared CDC source will also emit columns _rw_mysql-cdc_offset, _rw_mysql-cdc_partition (besides _rw_table_name and _rw_offset).

.. Or maybe somehow not included for CDC source. Since at the beginning of this PR, (i.e., using is_shared, instead of is_shared_non_cdc here), CDC will actually fail with index out of range at Java side..

I'm not 100% sure how it works. But it seems current stage is working.

@xxchan xxchan requested a review from stdrc November 27, 2024 08:42
@xxchan
Copy link
Member Author

xxchan commented Nov 27, 2024

@st1page I can't be confident without your approval

@st1page
Copy link
Contributor

st1page commented Nov 27, 2024

@st1page I can't be confident without your approval

:lark_cry: To be honest I do not very familiar with the shared source but every details in this PR LGTM.

fix #19575

Signed-off-by: xxchan <[email protected]>

fix test

Signed-off-by: xxchan <[email protected]>

add tpch test

Signed-off-by: xxchan <[email protected]>

fix

Signed-off-by: xxchan <[email protected]>
Signed-off-by: xxchan <[email protected]>
Signed-off-by: xxchan <[email protected]>
Signed-off-by: xxchan <[email protected]>
Signed-off-by: xxchan <[email protected]>
@xxchan xxchan force-pushed the xxchan/sporting-gopher branch from ee18a7e to fd14a95 Compare November 27, 2024 12:01
Copy link
Member Author

xxchan commented Nov 27, 2024

Merge activity

  • Nov 27, 8:40 PM GMT+8: Graphite couldn't merge this PR because it failed for an unknown reason (This repository has GitHub's merge queue enabled, which is currently incompatible with Graphite).

@xxchan xxchan added this pull request to the merge queue Nov 27, 2024
Merged via the queue into main with commit ba225b2 Nov 27, 2024
30 of 31 checks passed
@xxchan xxchan deleted the xxchan/sporting-gopher branch November 27, 2024 17:31
xxchan added a commit that referenced this pull request Nov 28, 2024
github-merge-queue bot pushed a commit that referenced this pull request Nov 28, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
type/fix Bug fix
Projects
None yet
Development

Successfully merging this pull request may close these issues.

shared source doesn't work with joins
6 participants