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: introduce watermark on source #7750

Merged
merged 13 commits into from
Feb 9, 2023
Merged

feat: introduce watermark on source #7750

merged 13 commits into from
Feb 9, 2023

Conversation

yuhao-su
Copy link
Contributor

@yuhao-su yuhao-su commented Feb 7, 2023

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

What's changed and what's your intention?

Done:

  • introduce watermark on source.

TODO:

  • watermark expr display in frontend
  • define multiple watermark on source
  • define watermark on table

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).
  • I have demonstrated that backward compatibility is not broken by breaking changes and created issues to track deprecated features to be removed in the future. (Please refer the issue)
  • All checks passed in ./risedev check (or alias, ./risedev c)

Documentation

We can create at most 1 watermark on the source column now by specifying WATERMARK FOR [column] as <expr>. Then a WatermarkFilterExecutor will be generated after the SourceExecutor

Just a simple example, not the actual use case:

To create a watermark on bid.auction

CREATE SOURCE bid (
auction INTEGER, 
bidder INTEGER, 
price INTEGER, 
WATERMARK FOR auction AS auction - 1,
"date_time" TIMESTAMP)
with (
  connector = 'nexmark',
  nexmark.table.type = 'Bid',
  nexmark.split.num = '12',
  nexmark.min.event.gap.in.ns = '0'
) ROW FORMAT JSON;

Release note

Please create a release note for your changes. In the release note, focus on the impact on users, and mention the environment or conditions where the impact may occur.

Refer to a related PR or issue link (optional)

#6952
#6750

Copy link
Contributor

@st1page st1page left a comment

Choose a reason for hiding this comment

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

generally LGTM

proto/catalog.proto Outdated Show resolved Hide resolved
proto/catalog.proto Outdated Show resolved Hide resolved
src/frontend/src/binder/create.rs Show resolved Hide resolved
src/frontend/src/binder/create.rs Outdated Show resolved Hide resolved
Comment on lines 262 to 288
fn bind_source_watermark(
session: &SessionImpl,
name: String,
source_watermarks: Vec<SourceWatermark>,
column_catalogs: &[ColumnCatalog],
) -> Result<Vec<WatermarkDesc>> {
let mut binder = Binder::new(session);
binder.bind_column_defs(name.clone(), column_catalogs.to_vec())?;

let watermark_descs = source_watermarks
.into_iter()
.map(|source_watermark| {
let col_name = source_watermark.column.real_value();
let watermark_idx = binder.get_column_binding_index(name.clone(), &col_name)?;

let expr = binder.bind_expr(source_watermark.expr)?.to_expr_proto();

Ok::<_, RwError>(WatermarkDesc {
watermark_idx: Some(ProstColumnIndex {
index: watermark_idx as u64,
}),
expr: Some(expr),
})
})
.try_collect()?;
Ok(watermark_descs)
}
Copy link
Contributor

Choose a reason for hiding this comment

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

Maybe it is too heavy here to construct a new binder. we can extract some bind expression logic later.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Maybe in later PR

proto/stream_plan.proto Outdated Show resolved Hide resolved
src/sqlparser/src/parser.rs Show resolved Hide resolved
src/sqlparser/src/parser.rs Outdated Show resolved Hide resolved
@codecov
Copy link

codecov bot commented Feb 8, 2023

Codecov Report

Merging #7750 (e47e7f6) into main (fa96b32) will increase coverage by 0.00%.
The diff coverage is 71.62%.

@@           Coverage Diff            @@
##             main    #7750    +/-   ##
========================================
  Coverage   71.67%   71.68%            
========================================
  Files        1101     1103     +2     
  Lines      175867   176072   +205     
========================================
+ Hits       126058   126214   +156     
- Misses      49809    49858    +49     
Flag Coverage Δ
rust 71.68% <71.62%> (+<0.01%) ⬆️

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

Impacted Files Coverage Δ
src/frontend/src/binder/mod.rs 100.00% <ø> (ø)
src/frontend/src/optimizer/plan_node/mod.rs 88.14% <ø> (ø)
src/meta/src/stream/stream_graph.rs 75.28% <0.00%> (-0.58%) ⬇️
src/sqlparser/src/ast/mod.rs 88.80% <ø> (ø)
src/sqlparser/src/keywords.rs 100.00% <ø> (ø)
src/stream/src/common/table/state_table.rs 80.06% <0.00%> (ø)
src/stream/src/from_proto/watermark_filter.rs 0.00% <0.00%> (ø)
...src/optimizer/plan_node/stream_watermark_filter.rs 40.00% <40.00%> (ø)
src/sqlparser/src/parser.rs 92.08% <89.65%> (-0.04%) ⬇️
src/sqlparser/src/ast/statement.rs 73.04% <92.59%> (+1.26%) ⬆️
... and 12 more

📣 We’re building smart automated test selection to slash your CI/CD build times. Learn more

@fuyufjh
Copy link
Member

fuyufjh commented Feb 9, 2023

define multiple watermark on source

Do we have any simple solution for this? @st1page If not, we may defer it until real-world cases appear.

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.

Generally LGTM

src/frontend/src/handler/create_source.rs Show resolved Hide resolved
@@ -356,7 +357,10 @@ impl ToBatch for LogicalSource {
impl ToStream for LogicalSource {
fn to_stream(&self, _ctx: &mut ToStreamContext) -> Result<PlanRef> {
let mut plan: PlanRef = StreamSource::new(self.clone()).into();
if let Some(row_id_index) = self.core.row_id_index && self.core.gen_row_id{
if let Some(catalog) = self.source_catalog() && !catalog.watermark_descs.is_empty(){
plan = StreamWatermarkFilter::new(plan, catalog.watermark_descs.clone()).into();
Copy link
Member

Choose a reason for hiding this comment

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

How to test this in e2e or planner test? 🤔

(Perhaps after supporting watermark on append-only tables?)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Added a planner test.

Copy link
Contributor

Choose a reason for hiding this comment

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

we can check the state cleaning in the e2e test because our state cleaning is based on the consistent range_delete

// The table used to persist watermark, the key is vnode.
catalog.Table table = 3;
// The watermark descs
repeated catalog.WatermarkDesc watermark_descs = 1;
Copy link
Member

Choose a reason for hiding this comment

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

Don't worry. repeated field is compatible with non-repeated one, so you may leave them as non-repeated now and change it until needed, in order to make the code cleaner.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Already changed🤣

@yuhao-su
Copy link
Contributor Author

yuhao-su commented Feb 9, 2023

define multiple watermark on source

Do we have any simple solution for this? @st1page If not, we may defer it until real-world cases appear.

The only thing we need to do is to refine the WatermarkFilterExecutor. It is not really hard.

@mergify mergify bot merged commit 42f37b5 into main Feb 9, 2023
@mergify mergify bot deleted the yuhao/watermark_source branch February 9, 2023 08:27
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.

4 participants