-
Notifications
You must be signed in to change notification settings - Fork 578
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(streaming): watermark on append only table #8207
Merged
Merged
Changes from all commits
Commits
Show all changes
3 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -25,18 +25,20 @@ use risingwave_common::catalog::{ | |
use risingwave_common::error::{ErrorCode, Result}; | ||
use risingwave_pb::catalog::{ | ||
ColumnIndex as ProstColumnIndex, Source as ProstSource, StreamSourceInfo, Table as ProstTable, | ||
WatermarkDesc, | ||
}; | ||
use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism; | ||
use risingwave_sqlparser::ast::{ | ||
ColumnDef, ColumnOption, DataType as AstDataType, ObjectName, SourceSchema, TableConstraint, | ||
ColumnDef, ColumnOption, DataType as AstDataType, ObjectName, SourceSchema, SourceWatermark, | ||
TableConstraint, | ||
}; | ||
|
||
use super::create_source::resolve_source_schema; | ||
use super::RwPgResponse; | ||
use crate::binder::{bind_data_type, bind_struct_field}; | ||
use crate::catalog::table_catalog::TableVersion; | ||
use crate::catalog::{check_valid_column_name, ColumnId}; | ||
use crate::handler::create_source::UPSTREAM_SOURCE_KEY; | ||
use crate::handler::create_source::{bind_source_watermark, UPSTREAM_SOURCE_KEY}; | ||
use crate::handler::HandlerArgs; | ||
use crate::optimizer::plan_node::LogicalSource; | ||
use crate::optimizer::property::{Order, RequiredDist}; | ||
|
@@ -285,6 +287,7 @@ pub(crate) async fn gen_create_table_plan_with_source( | |
columns: Vec<ColumnDef>, | ||
constraints: Vec<TableConstraint>, | ||
source_schema: SourceSchema, | ||
source_watermarks: Vec<SourceWatermark>, | ||
mut col_id_gen: ColumnIdGenerator, | ||
) -> Result<(PlanRef, Option<ProstSource>, ProstTable)> { | ||
let (column_descs, pk_column_id_from_columns) = bind_sql_columns(columns, &mut col_id_gen)?; | ||
|
@@ -293,6 +296,15 @@ pub(crate) async fn gen_create_table_plan_with_source( | |
let (mut columns, mut pk_column_ids, mut row_id_index) = | ||
bind_sql_table_constraints(column_descs, pk_column_id_from_columns, constraints)?; | ||
|
||
let watermark_descs = bind_source_watermark( | ||
context.session_ctx(), | ||
table_name.real_value(), | ||
source_watermarks, | ||
&columns, | ||
)?; | ||
// TODO(yuhao): allow multiple watermark on source. | ||
assert!(watermark_descs.len() <= 1); | ||
|
||
let definition = context.normalized_sql().to_owned(); | ||
|
||
let source_info = resolve_source_schema( | ||
|
@@ -313,6 +325,7 @@ pub(crate) async fn gen_create_table_plan_with_source( | |
row_id_index, | ||
Some(source_info), | ||
definition, | ||
watermark_descs, | ||
Some(col_id_gen.into_version()), | ||
) | ||
} | ||
|
@@ -325,16 +338,19 @@ pub(crate) fn gen_create_table_plan( | |
columns: Vec<ColumnDef>, | ||
constraints: Vec<TableConstraint>, | ||
mut col_id_gen: ColumnIdGenerator, | ||
source_watermarks: Vec<SourceWatermark>, | ||
) -> Result<(PlanRef, Option<ProstSource>, ProstTable)> { | ||
let definition = context.normalized_sql().to_owned(); | ||
let (column_descs, pk_column_id_from_columns) = bind_sql_columns(columns, &mut col_id_gen)?; | ||
|
||
gen_create_table_plan_without_bind( | ||
context, | ||
table_name, | ||
column_descs, | ||
pk_column_id_from_columns, | ||
constraints, | ||
definition, | ||
source_watermarks, | ||
Some(col_id_gen.into_version()), | ||
) | ||
} | ||
|
@@ -347,11 +363,19 @@ pub(crate) fn gen_create_table_plan_without_bind( | |
pk_column_id_from_columns: Option<ColumnId>, | ||
constraints: Vec<TableConstraint>, | ||
definition: String, | ||
source_watermarks: Vec<SourceWatermark>, | ||
version: Option<TableVersion>, | ||
) -> Result<(PlanRef, Option<ProstSource>, ProstTable)> { | ||
let (columns, pk_column_ids, row_id_index) = | ||
bind_sql_table_constraints(column_descs, pk_column_id_from_columns, constraints)?; | ||
|
||
let watermark_descs = bind_source_watermark( | ||
context.session_ctx(), | ||
table_name.real_value(), | ||
source_watermarks, | ||
&columns, | ||
)?; | ||
|
||
gen_table_plan_inner( | ||
context.into(), | ||
table_name, | ||
|
@@ -360,6 +384,7 @@ pub(crate) fn gen_create_table_plan_without_bind( | |
row_id_index, | ||
None, | ||
definition, | ||
watermark_descs, | ||
version, | ||
) | ||
} | ||
|
@@ -373,6 +398,7 @@ fn gen_table_plan_inner( | |
row_id_index: Option<usize>, | ||
source_info: Option<StreamSourceInfo>, | ||
definition: String, | ||
watermark_descs: Vec<WatermarkDesc>, | ||
version: Option<TableVersion>, /* TODO: this should always be `Some` if we support `ALTER | ||
* TABLE` for `CREATE TABLE AS`. */ | ||
) -> Result<(PlanRef, Option<ProstSource>, ProstTable)> { | ||
|
@@ -395,7 +421,7 @@ fn gen_table_plan_inner( | |
properties: context.with_options().inner().clone().into_iter().collect(), | ||
info: Some(source_info), | ||
owner: session.user_id(), | ||
watermark_descs: vec![], | ||
watermark_descs: watermark_descs.clone(), | ||
}); | ||
|
||
let source_catalog = source.as_ref().map(|source| Rc::new((source).into())); | ||
|
@@ -408,6 +434,7 @@ fn gen_table_plan_inner( | |
pk_column_ids, | ||
row_id_index, | ||
false, | ||
true, | ||
context.clone(), | ||
) | ||
.into(); | ||
|
@@ -438,12 +465,21 @@ fn gen_table_plan_inner( | |
.into()); | ||
} | ||
|
||
if !append_only && !watermark_descs.is_empty() { | ||
return Err(ErrorCode::NotSupported( | ||
"Defining watermarks on table requires the table to be append only.".to_owned(), | ||
"Set the option `appendonly=true`".to_owned(), | ||
) | ||
.into()); | ||
} | ||
|
||
Comment on lines
+468
to
+475
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @xx01cyx append only part |
||
let materialize = plan_root.gen_table_plan( | ||
name, | ||
columns, | ||
definition, | ||
row_id_index, | ||
append_only, | ||
watermark_descs, | ||
version, | ||
)?; | ||
|
||
|
@@ -460,6 +496,7 @@ pub async fn handle_create_table( | |
constraints: Vec<TableConstraint>, | ||
if_not_exists: bool, | ||
source_schema: Option<SourceSchema>, | ||
source_watermarks: Vec<SourceWatermark>, | ||
) -> Result<RwPgResponse> { | ||
let session = handler_args.session.clone(); | ||
|
||
|
@@ -487,6 +524,7 @@ pub async fn handle_create_table( | |
columns, | ||
constraints, | ||
source_schema, | ||
source_watermarks, | ||
col_id_gen, | ||
) | ||
.await? | ||
|
@@ -497,6 +535,7 @@ pub async fn handle_create_table( | |
columns, | ||
constraints, | ||
col_id_gen, | ||
source_watermarks, | ||
)?, | ||
}; | ||
let mut graph = build_graph(plan); | ||
|
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -62,6 +62,7 @@ impl LogicalSource { | |
pk_col_ids: Vec<ColumnId>, | ||
row_id_index: Option<usize>, | ||
gen_row_id: bool, | ||
for_table: bool, | ||
ctx: OptimizerContextRef, | ||
) -> Self { | ||
let core = generic::Source { | ||
|
@@ -70,6 +71,7 @@ impl LogicalSource { | |
pk_col_ids, | ||
row_id_index, | ||
gen_row_id, | ||
for_table, | ||
}; | ||
|
||
let schema = core.schema(); | ||
|
@@ -356,9 +358,11 @@ 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(catalog) = self.source_catalog() && !catalog.watermark_descs.is_empty(){ | ||
if let Some(catalog) = self.source_catalog() && !catalog.watermark_descs.is_empty() && !self.core.for_table{ | ||
plan = StreamWatermarkFilter::new(plan, catalog.watermark_descs.clone()).into(); | ||
} | ||
|
||
assert!(!(self.core.gen_row_id && self.core.for_table)); | ||
Comment on lines
+361
to
+365
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. May add some comments here. |
||
if let Some(row_id_index) = self.core.row_id_index && self.core.gen_row_id { | ||
plan = StreamRowIdGen::new(plan, row_id_index).into(); | ||
} | ||
|
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -90,6 +90,7 @@ impl Planner { | |
pk_col_ids, | ||
row_id_index, | ||
gen_row_id, | ||
false, | ||
self.ctx(), | ||
) | ||
.into()) | ||
|
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
emm, our usual usage is without the external connector. So maybe removing it is better?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think we can keep both behaviors undocumented for test. I'll add a new planner test.