From f901182c17018b16039ca315efc8850ea5607692 Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Thu, 16 Nov 2023 13:34:04 +0800 Subject: [PATCH 01/16] fix(test): missing rowsort in udf.slt (#13453) --- e2e_test/udf/udf.slt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/e2e_test/udf/udf.slt b/e2e_test/udf/udf.slt index 62979196e74c..b6d3161b7d3f 100644 --- a/e2e_test/udf/udf.slt +++ b/e2e_test/udf/udf.slt @@ -239,7 +239,7 @@ insert into t values (1), (2), (3); statement ok flush; -query II +query II rowsort select * from mv; ---- 1 0 From fb7fa70f0a351b098b5f341746657acc65ba27cb Mon Sep 17 00:00:00 2001 From: xxchan Date: Thu, 16 Nov 2023 13:36:47 +0800 Subject: [PATCH 02/16] chore: turn off more test targets (#13427) --- src/ctl/Cargo.toml | 3 +++ src/tests/compaction_test/Cargo.toml | 2 ++ src/tests/sqlsmith/Cargo.toml | 2 ++ 3 files changed, 7 insertions(+) diff --git a/src/ctl/Cargo.toml b/src/ctl/Cargo.toml index 4c9fe916d29b..f8bd5d9d4d27 100644 --- a/src/ctl/Cargo.toml +++ b/src/ctl/Cargo.toml @@ -58,3 +58,6 @@ risingwave_hummock_sdk = { workspace = true, features = ["enable_test_epoch"] } [lints] workspace = true + +[lib] +test = false diff --git a/src/tests/compaction_test/Cargo.toml b/src/tests/compaction_test/Cargo.toml index 98fe4ce6eedd..786ddb269fbd 100644 --- a/src/tests/compaction_test/Cargo.toml +++ b/src/tests/compaction_test/Cargo.toml @@ -53,10 +53,12 @@ workspace-hack = { path = "../../workspace-hack" } [[bin]] name = "compaction-test" path = "src/bin/compaction.rs" +test = false [[bin]] name = "delete-range-test" path = "src/bin/delete_range.rs" +test = false [lints] workspace = true diff --git a/src/tests/sqlsmith/Cargo.toml b/src/tests/sqlsmith/Cargo.toml index 402c6119cd1c..726b8d15ea1e 100644 --- a/src/tests/sqlsmith/Cargo.toml +++ b/src/tests/sqlsmith/Cargo.toml @@ -43,10 +43,12 @@ libtest-mimic = "0.6" [[bin]] name = "sqlsmith" path = "src/bin/main.rs" +test = false [[bin]] name = "sqlsmith-reducer" path = "src/bin/reducer.rs" +test = false [features] enable_sqlsmith_unit_test = [] From 8bcae118e4c09cd345cfa093f5ca0664c626dfa9 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Thu, 16 Nov 2023 15:12:16 +0800 Subject: [PATCH 03/16] refactor(risedev): refine error reporting (#13455) Signed-off-by: Bugen Zhao --- src/risedevtool/src/bin/risedev-dev.rs | 15 ++-- src/risedevtool/src/task.rs | 72 +++++++++---------- .../src/task/task_etcd_ready_check.rs | 2 +- .../src/task/task_kafka_ready_check.rs | 6 +- src/risedevtool/src/wait.rs | 20 ++++-- 5 files changed, 64 insertions(+), 51 deletions(-) diff --git a/src/risedevtool/src/bin/risedev-dev.rs b/src/risedevtool/src/bin/risedev-dev.rs index 474e8dd0cbd1..3d922b161bda 100644 --- a/src/risedevtool/src/bin/risedev-dev.rs +++ b/src/risedevtool/src/bin/risedev-dev.rs @@ -349,6 +349,10 @@ fn task_main( } fn main() -> Result<()> { + // Intentionally disable backtrace to provide more compact error message for `risedev dev`. + // Backtraces for RisingWave components are enabled in `Task::execute`. + std::env::set_var("RUST_BACKTRACE", "0"); + preflight_check()?; let task_name = std::env::args() @@ -440,11 +444,11 @@ fn main() -> Result<()> { } Err(err) => { println!( - "{} - Failed to start: {}\nCaused by:\n\t{}", + "{} - Failed to start: {:?}", // with `Caused by` style("ERROR").red().bold(), err, - err.root_cause().to_string().trim(), ); + println!(); println!( "* Use `{}` to enable new components, if they are missing.", style("./risedev configure").blue().bold(), @@ -464,9 +468,12 @@ fn main() -> Result<()> { ); println!("---"); println!(); - println!(); - Err(err) + // As we have already printed the error above, we don't need to print that error again. + // However, to return with a proper exit code, still return an error here. + Err(anyhow!( + "Failed to start all services. See details and instructions above." + )) } } } diff --git a/src/risedevtool/src/task.rs b/src/risedevtool/src/task.rs index a2e4ec9bf46d..32d488654489 100644 --- a/src/risedevtool/src/task.rs +++ b/src/risedevtool/src/task.rs @@ -42,9 +42,9 @@ use std::process::{Command, Output}; use std::sync::Arc; use std::time::Duration; -use anyhow::{anyhow, Result}; +use anyhow::{anyhow, Context, Result}; use indicatif::ProgressBar; -use reqwest::blocking::Client; +use reqwest::blocking::{Client, Response}; use tempfile::TempDir; pub use utils::*; @@ -172,7 +172,9 @@ where let addr = server.as_ref().parse()?; wait( || { - TcpStream::connect_timeout(&addr, Duration::from_secs(1))?; + TcpStream::connect_timeout(&addr, Duration::from_secs(1)).with_context(|| { + format!("failed to establish tcp connection to {}", server.as_ref()) + })?; Ok(()) }, &mut self.log, @@ -184,7 +186,11 @@ where Ok(()) } - pub fn wait_http(&mut self, server: impl AsRef) -> anyhow::Result<()> { + fn wait_http_with_response_cb( + &mut self, + server: impl AsRef, + cb: impl Fn(Response) -> anyhow::Result<()>, + ) -> anyhow::Result<()> { let server = server.as_ref(); wait( || { @@ -192,12 +198,13 @@ where .get(server) .timeout(Duration::from_secs(1)) .body("") - .send()?; - if resp.status().is_success() { - Ok(()) - } else { - Err(anyhow!("http failed with status: {}", resp.status())) - } + .send()? + .error_for_status() + .with_context(|| { + format!("failed to establish http connection to {}", server) + })?; + + cb(resp) }, &mut self.log, self.status_file.as_ref().unwrap(), @@ -207,39 +214,26 @@ where ) } - pub fn wait_http_with_cb( + pub fn wait_http(&mut self, server: impl AsRef) -> anyhow::Result<()> { + self.wait_http_with_response_cb(server, |_| Ok(())) + } + + pub fn wait_http_with_text_cb( &mut self, server: impl AsRef, cb: impl Fn(&str) -> bool, ) -> anyhow::Result<()> { - let server = server.as_ref(); - wait( - || { - let resp = Client::new() - .get(server) - .timeout(Duration::from_secs(1)) - .body("") - .send()?; - if resp.status().is_success() { - let data = resp.text()?; - if cb(&data) { - Ok(()) - } else { - Err(anyhow!( - "http health check callback failed with body: {:?}", - data - )) - } - } else { - Err(anyhow!("http failed with status: {}", resp.status())) - } - }, - &mut self.log, - self.status_file.as_ref().unwrap(), - self.id.as_ref().unwrap(), - Some(Duration::from_secs(30)), - true, - ) + self.wait_http_with_response_cb(server, |resp| { + let data = resp.text()?; + if cb(&data) { + Ok(()) + } else { + Err(anyhow!( + "http health check callback failed with body: {:?}", + data + )) + } + }) } pub fn wait(&mut self, wait_func: impl FnMut() -> Result<()>) -> anyhow::Result<()> { diff --git a/src/risedevtool/src/task/task_etcd_ready_check.rs b/src/risedevtool/src/task/task_etcd_ready_check.rs index ffc62e370702..9c4bea769ddd 100644 --- a/src/risedevtool/src/task/task_etcd_ready_check.rs +++ b/src/risedevtool/src/task/task_etcd_ready_check.rs @@ -44,7 +44,7 @@ impl Task for EtcdReadyCheckTask { response.health == "true" }; - ctx.wait_http_with_cb(health_check_addr, online_cb)?; + ctx.wait_http_with_text_cb(health_check_addr, online_cb)?; ctx.pb .set_message(format!("api {}:{}", self.config.address, self.config.port)); ctx.complete_spin(); diff --git a/src/risedevtool/src/task/task_kafka_ready_check.rs b/src/risedevtool/src/task/task_kafka_ready_check.rs index 1b53c96473ac..2e7b8cf436b7 100644 --- a/src/risedevtool/src/task/task_kafka_ready_check.rs +++ b/src/risedevtool/src/task/task_kafka_ready_check.rs @@ -14,7 +14,7 @@ use std::time::Duration; -use anyhow::{anyhow, Result}; +use anyhow::{Context, Result}; use rdkafka::config::FromClientConfig; use rdkafka::consumer::{BaseConsumer, Consumer}; use rdkafka::ClientConfig; @@ -48,7 +48,7 @@ impl Task for KafkaReadyCheckTask { let consumer = rt.block_on(async { BaseConsumer::from_config(&config) .await - .map_err(|e| anyhow!("{}", e)) + .context("failed to create consumer") })?; ctx.wait(|| { @@ -56,7 +56,7 @@ impl Task for KafkaReadyCheckTask { let _metadata = consumer .fetch_metadata(None, Duration::from_secs(1)) .await - .map_err(|e| anyhow!("{}", e))?; + .context("failed to fetch metadata")?; Ok(()) }) })?; diff --git a/src/risedevtool/src/wait.rs b/src/risedevtool/src/wait.rs index 3a47c7a4b643..e5c0671b0c3e 100644 --- a/src/risedevtool/src/wait.rs +++ b/src/risedevtool/src/wait.rs @@ -48,7 +48,13 @@ pub fn wait( if let Some(ref timeout) = timeout { if std::time::Instant::now() - start_time >= *timeout { - return Err(anyhow!("failed to connect, last error: {:?}", last_error)); + let context = "timeout when trying to connect"; + + return Err(if let Some(last_error) = last_error { + last_error.context(context) + } else { + anyhow!(context) + }); } } @@ -56,11 +62,17 @@ pub fn wait( let mut buf = String::new(); fs_err::File::open(p)?.read_to_string(&mut buf)?; - return Err(anyhow!( + let context = format!( "{} exited while waiting for connection: {}", style(id).red().bold(), - buf, - )); + buf.trim(), + ); + + return Err(if let Some(last_error) = last_error { + last_error.context(context) + } else { + anyhow!(context) + }); } sleep(Duration::from_millis(30)); From ef53d65e640f96e86c8c7ef208d9e5051408986f Mon Sep 17 00:00:00 2001 From: Dylan Date: Thu, 16 Nov 2023 15:39:26 +0800 Subject: [PATCH 04/16] fix(optimizer): skip subquery optimization if no apply (#13460) --- .../tests/testdata/output/cse_expr.yaml | 2 +- .../tests/testdata/output/explain.yaml | 14 +------------- src/frontend/src/optimizer/logical_optimization.rs | 10 ++++++++-- 3 files changed, 10 insertions(+), 16 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/output/cse_expr.yaml b/src/frontend/planner_test/tests/testdata/output/cse_expr.yaml index eba0edf5b1f0..5057fa13473c 100644 --- a/src/frontend/planner_test/tests/testdata/output/cse_expr.yaml +++ b/src/frontend/planner_test/tests/testdata/output/cse_expr.yaml @@ -84,5 +84,5 @@ sql: | with t(v, arr) as (select 1, array[2, 3]) select v < all(arr), v < some(arr) from t; batch_plan: |- - BatchProject { exprs: [All((1:Int32 < $expr10063)) as $expr1, Some((1:Int32 < $expr10063)) as $expr2] } + BatchProject { exprs: [All((1:Int32 < $expr10048)) as $expr1, Some((1:Int32 < $expr10048)) as $expr2] } └─BatchValues { rows: [[1:Int32, ARRAY[2, 3]:List(Int32)]] } diff --git a/src/frontend/planner_test/tests/testdata/output/explain.yaml b/src/frontend/planner_test/tests/testdata/output/explain.yaml index f416e9155a02..eced1aaf718b 100644 --- a/src/frontend/planner_test/tests/testdata/output/explain.yaml +++ b/src/frontend/planner_test/tests/testdata/output/explain.yaml @@ -26,11 +26,6 @@ LogicalProject { exprs: [1:Int32] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - Predicate Push Down: - - LogicalProject { exprs: [1:Int32] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - Convert Over Window: apply TrivialProjectToValuesRule 1 time(s) @@ -70,7 +65,7 @@ "stages": { "0": { "root": { - "plan_node_id": 10037, + "plan_node_id": 10031, "plan_node_type": "BatchValues", "schema": [ { @@ -133,13 +128,6 @@ Predicate Push Down: - LogicalProject { exprs: [t1.v1, t2.v2] } - └─LogicalJoin { type: Inner, on: (t1.v1 = t2.v2) } - ├─LogicalScan { table: t1, columns: [v1, _row_id] } - └─LogicalScan { table: t2, columns: [v2, _row_id] } - - Predicate Push Down: - LogicalProject { exprs: [t1.v1, t2.v2] } └─LogicalJoin { type: Inner, on: (t1.v1 = t2.v2) } ├─LogicalScan { table: t1, columns: [v1, _row_id] } diff --git a/src/frontend/src/optimizer/logical_optimization.rs b/src/frontend/src/optimizer/logical_optimization.rs index f8d9e6f78c73..887f393e81c6 100644 --- a/src/frontend/src/optimizer/logical_optimization.rs +++ b/src/frontend/src/optimizer/logical_optimization.rs @@ -426,6 +426,10 @@ impl LogicalOptimizer { explain_trace: bool, ctx: &OptimizerContextRef, ) -> Result { + // Bail our if no apply operators. + if !has_logical_apply(plan.clone()) { + return Ok(plan); + } // Simple Unnesting. plan = plan.optimize_by_rules(&SIMPLE_UNNESTING); if HasMaxOneRowApply().visit(plan.clone()) { @@ -437,8 +441,6 @@ impl LogicalOptimizer { // Predicate push down before translate apply, because we need to calculate the domain // and predicate push down can reduce the size of domain. plan = Self::predicate_pushdown(plan, explain_trace, ctx); - // In order to unnest a table function, we need to convert it into a `project_set` first. - plan = plan.optimize_by_rules(&TABLE_FUNCTION_TO_PROJECT_SET); // In order to unnest values with correlated input ref, we need to extract project first. plan = plan.optimize_by_rules(&VALUES_EXTRACT_PROJECT); // General Unnesting. @@ -535,6 +537,8 @@ impl LogicalOptimizer { } plan = plan.optimize_by_rules(&SET_OPERATION_MERGE); plan = plan.optimize_by_rules(&SET_OPERATION_TO_JOIN); + // In order to unnest a table function, we need to convert it into a `project_set` first. + plan = plan.optimize_by_rules(&TABLE_FUNCTION_TO_PROJECT_SET); plan = Self::subquery_unnesting(plan, enable_share_plan, explain_trace, &ctx)?; @@ -628,6 +632,8 @@ impl LogicalOptimizer { plan = plan.optimize_by_rules(&SET_OPERATION_MERGE); plan = plan.optimize_by_rules(&SET_OPERATION_TO_JOIN); plan = plan.optimize_by_rules(&ALWAYS_FALSE_FILTER); + // In order to unnest a table function, we need to convert it into a `project_set` first. + plan = plan.optimize_by_rules(&TABLE_FUNCTION_TO_PROJECT_SET); plan = Self::subquery_unnesting(plan, false, explain_trace, &ctx)?; From be3d41807165286a4cb236f491f724612e9699a8 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Thu, 16 Nov 2023 15:49:10 +0800 Subject: [PATCH 05/16] refactor(frontend): separate sys_scan and stream_scan (#13452) --- .../tests/testdata/output/pg_catalog.yaml | 8 +- .../tests/testdata/output/subquery.yaml | 22 +- src/frontend/src/optimizer/mod.rs | 8 +- .../src/optimizer/plan_node/batch_seq_scan.rs | 97 ++--- .../optimizer/plan_node/batch_sys_seq_scan.rs | 224 +++++++++++ .../src/optimizer/plan_node/generic/mod.rs | 2 + .../src/optimizer/plan_node/generic/scan.rs | 5 - .../optimizer/plan_node/generic/sys_scan.rs | 286 +++++++++++++ .../src/optimizer/plan_node/logical_scan.rs | 19 +- .../optimizer/plan_node/logical_sys_scan.rs | 378 ++++++++++++++++++ src/frontend/src/optimizer/plan_node/mod.rs | 8 + .../plan_visitor/cardinality_visitor.rs | 4 +- .../relation_collector_visitor.rs | 12 +- .../plan_visitor/sys_table_visitor.rs | 15 +- src/frontend/src/planner/relation.rs | 7 +- src/frontend/src/scheduler/plan_fragmenter.rs | 33 +- 16 files changed, 985 insertions(+), 143 deletions(-) create mode 100644 src/frontend/src/optimizer/plan_node/batch_sys_seq_scan.rs create mode 100644 src/frontend/src/optimizer/plan_node/generic/sys_scan.rs create mode 100644 src/frontend/src/optimizer/plan_node/logical_sys_scan.rs diff --git a/src/frontend/planner_test/tests/testdata/output/pg_catalog.yaml b/src/frontend/planner_test/tests/testdata/output/pg_catalog.yaml index 5ae1827ad95a..7842e311e47a 100644 --- a/src/frontend/planner_test/tests/testdata/output/pg_catalog.yaml +++ b/src/frontend/planner_test/tests/testdata/output/pg_catalog.yaml @@ -6,8 +6,8 @@ └─LogicalShare { id: 3 } └─LogicalProject { exprs: [rw_types.id, rw_types.name, rw_types.typelem, rw_types.typarray, rw_types.input_oid, false:Boolean, 0:Int32, -1:Int32, 0:Int32, 0:Int32, rw_schemas.id, 'b':Varchar, 0:Int32, null:Varchar, null:Varchar, null:Int32] } └─LogicalJoin { type: Inner, on: (rw_schemas.name = 'pg_catalog':Varchar), output: all } - ├─LogicalScan { table: rw_types, columns: [rw_types.id, rw_types.name, rw_types.input_oid, rw_types.typelem, rw_types.typarray] } - └─LogicalScan { table: rw_schemas, columns: [rw_schemas.id, rw_schemas.name, rw_schemas.owner, rw_schemas.acl] } + ├─LogicalSysScan { table: rw_types, columns: [rw_types.id, rw_types.name, rw_types.input_oid, rw_types.typelem, rw_types.typarray] } + └─LogicalSysScan { table: rw_schemas, columns: [rw_schemas.id, rw_schemas.name, rw_schemas.owner, rw_schemas.acl] } batch_plan: |- BatchProject { exprs: [rw_types.id, rw_types.name, rw_types.typelem, rw_types.typarray, rw_types.input_oid, false:Boolean, 0:Int32, -1:Int32, 0:Int32, 0:Int32, rw_schemas.id, 'b':Varchar, 0:Int32, null:Varchar, null:Varchar, null:Int32] } └─BatchNestedLoopJoin { type: Inner, predicate: true, output: all } @@ -21,7 +21,7 @@ LogicalProject { exprs: [rw_schemas.id, rw_schemas.name, rw_schemas.owner, rw_schemas.acl] } └─LogicalShare { id: 2 } └─LogicalProject { exprs: [rw_schemas.id, rw_schemas.name, rw_schemas.owner, rw_schemas.acl] } - └─LogicalScan { table: rw_schemas, columns: [rw_schemas.id, rw_schemas.name, rw_schemas.owner, rw_schemas.acl] } + └─LogicalSysScan { table: rw_schemas, columns: [rw_schemas.id, rw_schemas.name, rw_schemas.owner, rw_schemas.acl] } batch_plan: 'BatchScan { table: rw_schemas, columns: [rw_schemas.id, rw_schemas.name, rw_schemas.owner, rw_schemas.acl], distribution: Single }' - sql: | select * from pg_catalog.pg_cast @@ -211,7 +211,7 @@ └─LogicalFilter { predicate: (1:Int32 = rw_users.id) } └─LogicalShare { id: 2 } └─LogicalProject { exprs: [rw_users.id, rw_users.name, rw_users.create_db, rw_users.is_super, '********':Varchar] } - └─LogicalScan { table: rw_users, columns: [rw_users.id, rw_users.name, rw_users.is_super, rw_users.create_db, rw_users.create_user, rw_users.can_login] } + └─LogicalSysScan { table: rw_users, columns: [rw_users.id, rw_users.name, rw_users.is_super, rw_users.create_db, rw_users.create_user, rw_users.can_login] } batch_plan: |- BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchValues { rows: [[]] } diff --git a/src/frontend/planner_test/tests/testdata/output/subquery.yaml b/src/frontend/planner_test/tests/testdata/output/subquery.yaml index eb76926c3593..5143c7fdee85 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery.yaml @@ -204,13 +204,13 @@ └─LogicalFilter { predicate: (1:Int32 = rw_users.id) } └─LogicalShare { id: 2 } └─LogicalProject { exprs: [rw_users.id, rw_users.name, rw_users.create_db, rw_users.is_super, '********':Varchar] } - └─LogicalScan { table: rw_users, columns: [rw_users.id, rw_users.name, rw_users.is_super, rw_users.create_db, rw_users.create_user, rw_users.can_login] } + └─LogicalSysScan { table: rw_users, columns: [rw_users.id, rw_users.name, rw_users.is_super, rw_users.create_db, rw_users.create_user, rw_users.can_login] } optimized_logical_plan_for_batch: |- LogicalJoin { type: LeftOuter, on: true, output: all } ├─LogicalValues { rows: [[]], schema: Schema { fields: [] } } └─LogicalJoin { type: LeftOuter, on: true, output: all } ├─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - └─LogicalScan { table: rw_users, output_columns: [rw_users.name], required_columns: [rw_users.name, rw_users.id], predicate: (1:Int32 = rw_users.id) } + └─LogicalSysScan { table: rw_users, output_columns: [rw_users.name], required_columns: [rw_users.name, rw_users.id], predicate: (1:Int32 = rw_users.id) } batch_plan: |- BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchValues { rows: [[]] } @@ -247,27 +247,27 @@ │ │ │ │ │ ├─LogicalUnion { all: true } │ │ │ │ │ │ ├─LogicalUnion { all: true } │ │ │ │ │ │ │ ├─LogicalProject { exprs: [rw_tables.id, rw_tables.name, 'table':Varchar, rw_tables.schema_id, rw_tables.owner, rw_tables.definition, rw_tables.acl] } - │ │ │ │ │ │ │ │ └─LogicalScan { table: rw_tables, columns: [rw_tables.id, rw_tables.name, rw_tables.schema_id, rw_tables.owner, rw_tables.definition, rw_tables.acl, rw_tables.initialized_at, rw_tables.created_at] } + │ │ │ │ │ │ │ │ └─LogicalSysScan { table: rw_tables, columns: [rw_tables.id, rw_tables.name, rw_tables.schema_id, rw_tables.owner, rw_tables.definition, rw_tables.acl, rw_tables.initialized_at, rw_tables.created_at] } │ │ │ │ │ │ │ └─LogicalProject { exprs: [rw_system_tables.id, rw_system_tables.name, 'system table':Varchar, rw_system_tables.schema_id, rw_system_tables.owner, rw_system_tables.definition, rw_system_tables.acl] } - │ │ │ │ │ │ │ └─LogicalScan { table: rw_system_tables, columns: [rw_system_tables.id, rw_system_tables.name, rw_system_tables.schema_id, rw_system_tables.owner, rw_system_tables.definition, rw_system_tables.acl] } + │ │ │ │ │ │ │ └─LogicalSysScan { table: rw_system_tables, columns: [rw_system_tables.id, rw_system_tables.name, rw_system_tables.schema_id, rw_system_tables.owner, rw_system_tables.definition, rw_system_tables.acl] } │ │ │ │ │ │ └─LogicalProject { exprs: [rw_sources.id, rw_sources.name, 'source':Varchar, rw_sources.schema_id, rw_sources.owner, rw_sources.definition, rw_sources.acl] } - │ │ │ │ │ │ └─LogicalScan { table: rw_sources, columns: [rw_sources.id, rw_sources.name, rw_sources.schema_id, rw_sources.owner, rw_sources.connector, rw_sources.columns, rw_sources.format, rw_sources.row_encode, rw_sources.append_only, rw_sources.connection_id, rw_sources.definition, rw_sources.acl, rw_sources.initialized_at, rw_sources.created_at] } + │ │ │ │ │ │ └─LogicalSysScan { table: rw_sources, columns: [rw_sources.id, rw_sources.name, rw_sources.schema_id, rw_sources.owner, rw_sources.connector, rw_sources.columns, rw_sources.format, rw_sources.row_encode, rw_sources.append_only, rw_sources.connection_id, rw_sources.definition, rw_sources.acl, rw_sources.initialized_at, rw_sources.created_at] } │ │ │ │ │ └─LogicalProject { exprs: [rw_indexes.id, rw_indexes.name, 'index':Varchar, rw_indexes.schema_id, rw_indexes.owner, rw_indexes.definition, rw_indexes.acl] } - │ │ │ │ │ └─LogicalScan { table: rw_indexes, columns: [rw_indexes.id, rw_indexes.name, rw_indexes.primary_table_id, rw_indexes.original_column_ids, rw_indexes.schema_id, rw_indexes.owner, rw_indexes.definition, rw_indexes.acl, rw_indexes.initialized_at, rw_indexes.created_at] } + │ │ │ │ │ └─LogicalSysScan { table: rw_indexes, columns: [rw_indexes.id, rw_indexes.name, rw_indexes.primary_table_id, rw_indexes.original_column_ids, rw_indexes.schema_id, rw_indexes.owner, rw_indexes.definition, rw_indexes.acl, rw_indexes.initialized_at, rw_indexes.created_at] } │ │ │ │ └─LogicalProject { exprs: [rw_sinks.id, rw_sinks.name, 'sink':Varchar, rw_sinks.schema_id, rw_sinks.owner, rw_sinks.definition, rw_sinks.acl] } - │ │ │ │ └─LogicalScan { table: rw_sinks, columns: [rw_sinks.id, rw_sinks.name, rw_sinks.schema_id, rw_sinks.owner, rw_sinks.connector, rw_sinks.sink_type, rw_sinks.connection_id, rw_sinks.definition, rw_sinks.acl, rw_sinks.initialized_at, rw_sinks.created_at] } + │ │ │ │ └─LogicalSysScan { table: rw_sinks, columns: [rw_sinks.id, rw_sinks.name, rw_sinks.schema_id, rw_sinks.owner, rw_sinks.connector, rw_sinks.sink_type, rw_sinks.connection_id, rw_sinks.definition, rw_sinks.acl, rw_sinks.initialized_at, rw_sinks.created_at] } │ │ │ └─LogicalProject { exprs: [rw_materialized_views.id, rw_materialized_views.name, 'materialized view':Varchar, rw_materialized_views.schema_id, rw_materialized_views.owner, rw_materialized_views.definition, rw_materialized_views.acl] } - │ │ │ └─LogicalScan { table: rw_materialized_views, columns: [rw_materialized_views.id, rw_materialized_views.name, rw_materialized_views.schema_id, rw_materialized_views.owner, rw_materialized_views.definition, rw_materialized_views.acl, rw_materialized_views.initialized_at, rw_materialized_views.created_at] } + │ │ │ └─LogicalSysScan { table: rw_materialized_views, columns: [rw_materialized_views.id, rw_materialized_views.name, rw_materialized_views.schema_id, rw_materialized_views.owner, rw_materialized_views.definition, rw_materialized_views.acl, rw_materialized_views.initialized_at, rw_materialized_views.created_at] } │ │ └─LogicalProject { exprs: [rw_views.id, rw_views.name, 'view':Varchar, rw_views.schema_id, rw_views.owner, rw_views.definition, rw_views.acl] } - │ │ └─LogicalScan { table: rw_views, columns: [rw_views.id, rw_views.name, rw_views.schema_id, rw_views.owner, rw_views.definition, rw_views.acl] } + │ │ └─LogicalSysScan { table: rw_views, columns: [rw_views.id, rw_views.name, rw_views.schema_id, rw_views.owner, rw_views.definition, rw_views.acl] } │ └─LogicalShare { id: 18 } │ └─LogicalProject { exprs: [rw_schemas.id, rw_schemas.name, rw_schemas.owner, rw_schemas.acl] } - │ └─LogicalScan { table: rw_schemas, columns: [rw_schemas.id, rw_schemas.name, rw_schemas.owner, rw_schemas.acl] } + │ └─LogicalSysScan { table: rw_schemas, columns: [rw_schemas.id, rw_schemas.name, rw_schemas.owner, rw_schemas.acl] } └─LogicalProject { exprs: [rw_users.name] } └─LogicalFilter { predicate: (CorrelatedInputRef { index: 3, correlated_id: 1 } = rw_users.id) } └─LogicalShare { id: 22 } └─LogicalProject { exprs: [rw_users.id, rw_users.name, rw_users.create_db, rw_users.is_super, '********':Varchar] } - └─LogicalScan { table: rw_users, columns: [rw_users.id, rw_users.name, rw_users.is_super, rw_users.create_db, rw_users.create_user, rw_users.can_login] } + └─LogicalSysScan { table: rw_users, columns: [rw_users.id, rw_users.name, rw_users.is_super, rw_users.create_db, rw_users.create_user, rw_users.can_login] } batch_plan: |- BatchExchange { order: [rw_schemas.name ASC, rw_tables.name ASC], dist: Single } └─BatchProject { exprs: [rw_schemas.name, rw_tables.name, Case(($expr1 = 'r':Varchar), 'table':Varchar, ($expr1 = 'v':Varchar), 'view':Varchar, ($expr1 = 'm':Varchar), 'materialized view':Varchar, ($expr1 = 'i':Varchar), 'index':Varchar, ($expr1 = 'S':Varchar), 'sequence':Varchar, ($expr1 = 's':Varchar), 'special':Varchar, ($expr1 = 't':Varchar), 'TOAST table':Varchar, ($expr1 = 'f':Varchar), 'foreign table':Varchar, ($expr1 = 'p':Varchar), 'partitioned table':Varchar, ($expr1 = 'I':Varchar), 'partitioned index':Varchar) as $expr2, rw_users.name] } diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index eb7773233955..70d5ae1769c8 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -747,9 +747,7 @@ fn exist_and_no_exchange_before(plan: &PlanRef, is_candidate: fn(&PlanRef) -> bo /// Returns `true` if we must insert an additional exchange to ensure this. fn require_additional_exchange_on_root_in_distributed_mode(plan: PlanRef) -> bool { fn is_user_table(plan: &PlanRef) -> bool { - plan.as_batch_seq_scan() - .map(|node| !node.core().is_sys_table()) - .unwrap_or(false) + plan.node_type() == PlanNodeType::BatchSeqScan } fn is_source(plan: &PlanRef) -> bool { @@ -780,9 +778,7 @@ fn require_additional_exchange_on_root_in_distributed_mode(plan: PlanRef) -> boo /// them for the different requirement of plan node in different execute mode. fn require_additional_exchange_on_root_in_local_mode(plan: PlanRef) -> bool { fn is_user_table(plan: &PlanRef) -> bool { - plan.as_batch_seq_scan() - .map(|node| !node.core().is_sys_table()) - .unwrap_or(false) + plan.node_type() == PlanNodeType::BatchSeqScan } fn is_source(plan: &PlanRef) -> bool { diff --git a/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs b/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs index 65487e537420..1b2582a7e2db 100644 --- a/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs +++ b/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs @@ -21,8 +21,7 @@ use risingwave_common::types::ScalarImpl; use risingwave_common::util::scan_range::{is_full_range, ScanRange}; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::row_seq_scan_node::ChunkSize; -use risingwave_pb::batch_plan::{RowSeqScanNode, SysRowSeqScanNode}; -use risingwave_pb::plan_common::PbColumnDesc; +use risingwave_pb::batch_plan::RowSeqScanNode; use super::batch::prelude::*; use super::utils::{childless_record, Distill}; @@ -78,29 +77,25 @@ impl BatchSeqScan { fn clone_with_dist(&self) -> Self { Self::new_inner( self.core.clone(), - if self.core.is_sys_table() { - Distribution::Single - } else { - match self.core.distribution_key() { - None => Distribution::SomeShard, - Some(distribution_key) => { - if distribution_key.is_empty() { - Distribution::Single - } else { - // For other batch operators, `HashShard` is a simple hashing, i.e., - // `target_shard = hash(dist_key) % shard_num` - // - // But MV is actually sharded by consistent hashing, i.e., - // `target_shard = vnode_mapping.map(hash(dist_key) % vnode_num)` - // - // They are incompatible, so we just specify its distribution as - // `SomeShard` to force an exchange is - // inserted. - Distribution::UpstreamHashShard( - distribution_key, - self.core.table_desc.table_id, - ) - } + match self.core.distribution_key() { + None => Distribution::SomeShard, + Some(distribution_key) => { + if distribution_key.is_empty() { + Distribution::Single + } else { + // For other batch operators, `HashShard` is a simple hashing, i.e., + // `target_shard = hash(dist_key) % shard_num` + // + // But MV is actually sharded by consistent hashing, i.e., + // `target_shard = vnode_mapping.map(hash(dist_key) % vnode_num)` + // + // They are incompatible, so we just specify its distribution as + // `SomeShard` to force an exchange is + // inserted. + Distribution::UpstreamHashShard( + distribution_key, + self.core.table_desc.table_id, + ) } } }, @@ -214,45 +209,29 @@ impl ToDistributedBatch for BatchSeqScan { impl ToBatchPb for BatchSeqScan { fn to_batch_prost_body(&self) -> NodeBody { - let column_descs = self - .core - .column_descs() - .iter() - .map(PbColumnDesc::from) - .collect(); - - if self.core.is_sys_table() { - NodeBody::SysRowSeqScan(SysRowSeqScanNode { - table_id: self.core.table_desc.table_id.table_id, - column_descs, - }) - } else { - NodeBody::RowSeqScan(RowSeqScanNode { - table_desc: Some(self.core.table_desc.to_protobuf()), - column_ids: self - .core - .output_column_ids() - .iter() - .map(ColumnId::get_id) - .collect(), - scan_ranges: self.scan_ranges.iter().map(|r| r.to_protobuf()).collect(), - // To be filled by the scheduler. - vnode_bitmap: None, - ordered: !self.order().is_any(), - chunk_size: self - .core - .chunk_size - .map(|chunk_size| ChunkSize { chunk_size }), - }) - } + NodeBody::RowSeqScan(RowSeqScanNode { + table_desc: Some(self.core.table_desc.to_protobuf()), + column_ids: self + .core + .output_column_ids() + .iter() + .map(ColumnId::get_id) + .collect(), + scan_ranges: self.scan_ranges.iter().map(|r| r.to_protobuf()).collect(), + // To be filled by the scheduler. + vnode_bitmap: None, + ordered: !self.order().is_any(), + chunk_size: self + .core + .chunk_size + .map(|chunk_size| ChunkSize { chunk_size }), + }) } } impl ToLocalBatch for BatchSeqScan { fn to_local(&self) -> Result { - let dist = if self.core.is_sys_table() { - Distribution::Single - } else if let Some(distribution_key) = self.core.distribution_key() + let dist = if let Some(distribution_key) = self.core.distribution_key() && !distribution_key.is_empty() { Distribution::UpstreamHashShard(distribution_key, self.core.table_desc.table_id) diff --git a/src/frontend/src/optimizer/plan_node/batch_sys_seq_scan.rs b/src/frontend/src/optimizer/plan_node/batch_sys_seq_scan.rs new file mode 100644 index 000000000000..22c24233ada3 --- /dev/null +++ b/src/frontend/src/optimizer/plan_node/batch_sys_seq_scan.rs @@ -0,0 +1,224 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::ops::Bound; + +use itertools::Itertools; +use pretty_xmlish::{Pretty, XmlNode}; +use risingwave_common::error::Result; +use risingwave_common::types::ScalarImpl; +use risingwave_common::util::scan_range::{is_full_range, ScanRange}; +use risingwave_pb::batch_plan::plan_node::NodeBody; +use risingwave_pb::batch_plan::SysRowSeqScanNode; +use risingwave_pb::plan_common::PbColumnDesc; + +use super::batch::prelude::*; +use super::utils::{childless_record, Distill}; +use super::{generic, ExprRewritable, PlanBase, PlanRef, ToBatchPb, ToDistributedBatch}; +use crate::expr::ExprRewriter; +use crate::optimizer::plan_node::ToLocalBatch; +use crate::optimizer::property::{Distribution, DistributionDisplay, Order}; + +/// `BatchSysSeqScan` implements [`super::LogicalSysScan`] to scan from a row-oriented table +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct BatchSysSeqScan { + pub base: PlanBase, + core: generic::SysScan, + scan_ranges: Vec, +} + +impl BatchSysSeqScan { + fn new_inner(core: generic::SysScan, dist: Distribution, scan_ranges: Vec) -> Self { + let order = if scan_ranges.len() > 1 { + Order::any() + } else { + core.get_out_column_index_order() + }; + let base = PlanBase::new_batch_with_core(&core, dist, order); + + { + // validate scan_range + scan_ranges.iter().for_each(|scan_range| { + assert!(!scan_range.is_full_table_scan()); + let scan_pk_prefix_len = scan_range.eq_conds.len(); + let order_len = core.table_desc.order_column_indices().len(); + assert!( + scan_pk_prefix_len < order_len + || (scan_pk_prefix_len == order_len && is_full_range(&scan_range.range)), + "invalid scan_range", + ); + }) + } + + Self { + base, + core, + scan_ranges, + } + } + + pub fn new(core: generic::SysScan, scan_ranges: Vec) -> Self { + // Use `Single` by default, will be updated later with `clone_with_dist`. + Self::new_inner(core, Distribution::Single, scan_ranges) + } + + fn clone_with_dist(&self) -> Self { + Self::new_inner( + self.core.clone(), + Distribution::Single, + self.scan_ranges.clone(), + ) + } + + /// Get a reference to the batch seq scan's logical. + #[must_use] + pub fn core(&self) -> &generic::SysScan { + &self.core + } + + pub fn scan_ranges(&self) -> &[ScanRange] { + &self.scan_ranges + } + + fn scan_ranges_as_strs(&self, verbose: bool) -> Vec { + let order_names = match verbose { + true => self.core.order_names_with_table_prefix(), + false => self.core.order_names(), + }; + let mut range_strs = vec![]; + + let explain_max_range = 20; + for scan_range in self.scan_ranges.iter().take(explain_max_range) { + #[expect(clippy::disallowed_methods)] + let mut range_str = scan_range + .eq_conds + .iter() + .zip(order_names.iter()) + .map(|(v, name)| match v { + Some(v) => format!("{} = {:?}", name, v), + None => format!("{} IS NULL", name), + }) + .collect_vec(); + if !is_full_range(&scan_range.range) { + let i = scan_range.eq_conds.len(); + range_str.push(range_to_string(&order_names[i], &scan_range.range)) + } + range_strs.push(range_str.join(" AND ")); + } + if self.scan_ranges.len() > explain_max_range { + range_strs.push("...".to_string()); + } + range_strs + } +} + +impl_plan_tree_node_for_leaf! { BatchSysSeqScan } + +fn lb_to_string(name: &str, lb: &Bound) -> String { + let (op, v) = match lb { + Bound::Included(v) => (">=", v), + Bound::Excluded(v) => (">", v), + Bound::Unbounded => unreachable!(), + }; + format!("{} {} {:?}", name, op, v) +} +fn ub_to_string(name: &str, ub: &Bound) -> String { + let (op, v) = match ub { + Bound::Included(v) => ("<=", v), + Bound::Excluded(v) => ("<", v), + Bound::Unbounded => unreachable!(), + }; + format!("{} {} {:?}", name, op, v) +} +fn range_to_string(name: &str, range: &(Bound, Bound)) -> String { + match (&range.0, &range.1) { + (Bound::Unbounded, Bound::Unbounded) => unreachable!(), + (Bound::Unbounded, ub) => ub_to_string(name, ub), + (lb, Bound::Unbounded) => lb_to_string(name, lb), + (lb, ub) => { + format!("{} AND {}", lb_to_string(name, lb), ub_to_string(name, ub)) + } + } +} + +impl Distill for BatchSysSeqScan { + fn distill<'a>(&self) -> XmlNode<'a> { + let verbose = self.base.ctx().is_explain_verbose(); + let mut vec = Vec::with_capacity(4); + vec.push(("table", Pretty::from(self.core.table_name.clone()))); + vec.push(("columns", self.core.columns_pretty(verbose))); + + if !self.scan_ranges.is_empty() { + let range_strs = self.scan_ranges_as_strs(verbose); + vec.push(( + "scan_ranges", + Pretty::Array(range_strs.into_iter().map(Pretty::from).collect()), + )); + } + + if verbose { + let dist = Pretty::display(&DistributionDisplay { + distribution: self.distribution(), + input_schema: self.base.schema(), + }); + vec.push(("distribution", dist)); + } + + childless_record("BatchScan", vec) + } +} + +impl ToDistributedBatch for BatchSysSeqScan { + fn to_distributed(&self) -> Result { + Ok(self.clone_with_dist().into()) + } +} + +impl ToBatchPb for BatchSysSeqScan { + fn to_batch_prost_body(&self) -> NodeBody { + let column_descs = self + .core + .column_descs() + .iter() + .map(PbColumnDesc::from) + .collect(); + NodeBody::SysRowSeqScan(SysRowSeqScanNode { + table_id: self.core.table_desc.table_id.table_id, + column_descs, + }) + } +} + +impl ToLocalBatch for BatchSysSeqScan { + fn to_local(&self) -> Result { + Ok(Self::new_inner( + self.core.clone(), + Distribution::Single, + self.scan_ranges.clone(), + ) + .into()) + } +} + +impl ExprRewritable for BatchSysSeqScan { + fn has_rewritable_expr(&self) -> bool { + true + } + + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + let mut core = self.core.clone(); + core.rewrite_exprs(r); + Self::new(core, self.scan_ranges.clone()).into() + } +} diff --git a/src/frontend/src/optimizer/plan_node/generic/mod.rs b/src/frontend/src/optimizer/plan_node/generic/mod.rs index fe85e0919804..c0baacffec8a 100644 --- a/src/frontend/src/optimizer/plan_node/generic/mod.rs +++ b/src/frontend/src/optimizer/plan_node/generic/mod.rs @@ -42,6 +42,8 @@ mod source; pub use source::*; mod scan; pub use scan::*; +mod sys_scan; +pub use sys_scan::*; mod union; pub use union::*; mod top_n; diff --git a/src/frontend/src/optimizer/plan_node/generic/scan.rs b/src/frontend/src/optimizer/plan_node/generic/scan.rs index 7d83ac0ba7e4..3c615183f9ad 100644 --- a/src/frontend/src/optimizer/plan_node/generic/scan.rs +++ b/src/frontend/src/optimizer/plan_node/generic/scan.rs @@ -34,7 +34,6 @@ use crate::utils::{ColIndexMappingRewriteExt, Condition}; pub enum ScanTableType { #[default] General, - SysTable, CdcTable, } @@ -424,10 +423,6 @@ impl Scan { } } - pub fn is_sys_table(&self) -> bool { - matches!(self.scan_table_type, ScanTableType::SysTable) - } - pub fn is_cdc_table(&self) -> bool { matches!(self.scan_table_type, ScanTableType::CdcTable) } diff --git a/src/frontend/src/optimizer/plan_node/generic/sys_scan.rs b/src/frontend/src/optimizer/plan_node/generic/sys_scan.rs new file mode 100644 index 000000000000..dadc959fce33 --- /dev/null +++ b/src/frontend/src/optimizer/plan_node/generic/sys_scan.rs @@ -0,0 +1,286 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::HashMap; +use std::rc::Rc; + +use educe::Educe; +use pretty_xmlish::Pretty; +use risingwave_common::catalog::{ColumnDesc, Field, Schema, TableDesc}; +use risingwave_common::util::column_index_mapping::ColIndexMapping; +use risingwave_common::util::sort_util::ColumnOrder; + +use super::GenericPlanNode; +use crate::catalog::ColumnId; +use crate::expr::ExprRewriter; +use crate::optimizer::optimizer_context::OptimizerContextRef; +use crate::optimizer::property::{Cardinality, FunctionalDependencySet, Order}; +use crate::utils::{ColIndexMappingRewriteExt, Condition}; + +/// [`SysScan`] returns contents of a table or other equivalent object +#[derive(Debug, Clone, Educe)] +#[educe(PartialEq, Eq, Hash)] +pub struct SysScan { + pub table_name: String, + /// Include `output_col_idx` and columns required in `predicate` + pub required_col_idx: Vec, + pub output_col_idx: Vec, + /// Descriptor of the table + pub table_desc: Rc, + /// The pushed down predicates. It refers to column indexes of the table. + pub predicate: Condition, + /// Help RowSeqSysScan executor use a better chunk size + pub chunk_size: Option, + /// The cardinality of the table **without** applying the predicate. + pub table_cardinality: Cardinality, + #[educe(PartialEq(ignore))] + #[educe(Hash(ignore))] + pub ctx: OptimizerContextRef, +} + +impl SysScan { + pub(crate) fn rewrite_exprs(&mut self, r: &mut dyn ExprRewriter) { + self.predicate = self.predicate.clone().rewrite_expr(r); + } + + /// Get the ids of the output columns. + pub fn output_column_ids(&self) -> Vec { + self.output_col_idx + .iter() + .map(|i| self.get_table_columns()[*i].column_id) + .collect() + } + + pub fn primary_key(&self) -> &[ColumnOrder] { + &self.table_desc.pk + } + + pub(crate) fn column_names_with_table_prefix(&self) -> Vec { + self.output_col_idx + .iter() + .map(|&i| format!("{}.{}", self.table_name, self.get_table_columns()[i].name)) + .collect() + } + + pub(crate) fn column_names(&self) -> Vec { + self.output_col_idx + .iter() + .map(|&i| self.get_table_columns()[i].name.clone()) + .collect() + } + + pub(crate) fn order_names(&self) -> Vec { + self.table_desc + .order_column_indices() + .iter() + .map(|&i| self.get_table_columns()[i].name.clone()) + .collect() + } + + pub(crate) fn order_names_with_table_prefix(&self) -> Vec { + self.table_desc + .order_column_indices() + .iter() + .map(|&i| format!("{}.{}", self.table_name, self.get_table_columns()[i].name)) + .collect() + } + + /// Return indices of fields the output is ordered by and + /// corresponding direction + pub fn get_out_column_index_order(&self) -> Order { + let id_to_tb_idx = self.table_desc.get_id_to_op_idx_mapping(); + let order = Order::new( + self.table_desc + .pk + .iter() + .map(|order| { + let idx = id_to_tb_idx + .get(&self.table_desc.columns[order.column_index].column_id) + .unwrap(); + ColumnOrder::new(*idx, order.order_type) + }) + .collect(), + ); + self.i2o_col_mapping().rewrite_provided_order(&order) + } + + /// get the Mapping of columnIndex from internal column index to output column index + pub fn i2o_col_mapping(&self) -> ColIndexMapping { + ColIndexMapping::with_remaining_columns( + &self.output_col_idx, + self.get_table_columns().len(), + ) + } + + /// Get the ids of the output columns and primary key columns. + pub fn output_and_pk_column_ids(&self) -> Vec { + let mut ids = self.output_column_ids(); + for column_order in self.primary_key() { + let id = self.get_table_columns()[column_order.column_index].column_id; + if !ids.contains(&id) { + ids.push(id); + } + } + ids + } + + /// Create a `LogicalSysScan` node. Used internally by optimizer. + #[allow(clippy::too_many_arguments)] + pub(crate) fn new( + table_name: String, + output_col_idx: Vec, // the column index in the table + table_desc: Rc, + ctx: OptimizerContextRef, + predicate: Condition, // refers to column indexes of the table + table_cardinality: Cardinality, + ) -> Self { + Self::new_inner( + table_name, + output_col_idx, + table_desc, + ctx, + predicate, + table_cardinality, + ) + } + + #[allow(clippy::too_many_arguments)] + pub(crate) fn new_inner( + table_name: String, + output_col_idx: Vec, // the column index in the table + table_desc: Rc, + ctx: OptimizerContextRef, + predicate: Condition, // refers to column indexes of the table + table_cardinality: Cardinality, + ) -> Self { + // here we have 3 concepts + // 1. column_id: ColumnId, stored in catalog and a ID to access data from storage. + // 2. table_idx: usize, column index in the TableDesc or tableCatalog. + // 3. operator_idx: usize, column index in the SysScanOperator's schema. + // In a query we get the same version of catalog, so the mapping from column_id and + // table_idx will not change. And the `required_col_idx` is the `table_idx` of the + // required columns, i.e., the mapping from operator_idx to table_idx. + + let mut required_col_idx = output_col_idx.clone(); + let predicate_col_idx = predicate.collect_input_refs(table_desc.columns.len()); + predicate_col_idx.ones().for_each(|idx| { + if !required_col_idx.contains(&idx) { + required_col_idx.push(idx); + } + }); + + Self { + table_name, + required_col_idx, + output_col_idx, + table_desc, + predicate, + chunk_size: None, + ctx, + table_cardinality, + } + } + + pub(crate) fn columns_pretty<'a>(&self, verbose: bool) -> Pretty<'a> { + Pretty::Array( + match verbose { + true => self.column_names_with_table_prefix(), + false => self.column_names(), + } + .into_iter() + .map(Pretty::from) + .collect(), + ) + } + + pub(crate) fn fields_pretty_schema(&self) -> Schema { + let fields = self + .table_desc + .columns + .iter() + .map(|col| Field::from_with_table_name_prefix(col, &self.table_name)) + .collect(); + Schema { fields } + } +} + +impl GenericPlanNode for SysScan { + fn schema(&self) -> Schema { + let fields = self + .output_col_idx + .iter() + .map(|tb_idx| { + let col = &self.get_table_columns()[*tb_idx]; + Field::from_with_table_name_prefix(col, &self.table_name) + }) + .collect(); + Schema { fields } + } + + fn stream_key(&self) -> Option> { + let id_to_op_idx = Self::get_id_to_op_idx_mapping(&self.output_col_idx, &self.table_desc); + self.table_desc + .stream_key + .iter() + .map(|&c| { + id_to_op_idx + .get(&self.table_desc.columns[c].column_id) + .copied() + }) + .collect::>>() + } + + fn ctx(&self) -> OptimizerContextRef { + self.ctx.clone() + } + + fn functional_dependency(&self) -> FunctionalDependencySet { + let pk_indices = self.stream_key(); + let col_num = self.output_col_idx.len(); + match &pk_indices { + Some(pk_indices) => FunctionalDependencySet::with_key(col_num, pk_indices), + None => FunctionalDependencySet::new(col_num), + } + } +} + +impl SysScan { + pub fn get_table_columns(&self) -> &[ColumnDesc] { + &self.table_desc.columns + } + + /// Get the descs of the output columns. + pub fn column_descs(&self) -> Vec { + self.output_col_idx + .iter() + .map(|&i| self.get_table_columns()[i].clone()) + .collect() + } + + /// Helper function to create a mapping from `column_id` to `operator_idx` + pub fn get_id_to_op_idx_mapping( + output_col_idx: &[usize], + table_desc: &Rc, + ) -> HashMap { + let mut id_to_op_idx = HashMap::new(); + output_col_idx + .iter() + .enumerate() + .for_each(|(op_idx, tb_idx)| { + let col = &table_desc.columns[*tb_idx]; + id_to_op_idx.insert(col.column_id, op_idx); + }); + id_to_op_idx + } +} diff --git a/src/frontend/src/optimizer/plan_node/logical_scan.rs b/src/frontend/src/optimizer/plan_node/logical_scan.rs index 4a8d9bd872d8..bd1297a4cb92 100644 --- a/src/frontend/src/optimizer/plan_node/logical_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_scan.rs @@ -19,7 +19,7 @@ use fixedbitset::FixedBitSet; use itertools::Itertools; use pretty_xmlish::{Pretty, XmlNode}; use risingwave_common::catalog::{CdcTableDesc, ColumnDesc, TableDesc}; -use risingwave_common::error::{ErrorCode, Result, RwError}; +use risingwave_common::error::Result; use risingwave_common::util::sort_util::ColumnOrder; use super::generic::{GenericPlanNode, GenericPlanRef}; @@ -107,10 +107,6 @@ impl LogicalScan { &self.core.scan_table_type } - pub fn is_sys_table(&self) -> bool { - self.core.is_sys_table() - } - pub fn is_cdc_table(&self) -> bool { matches!(self.core.scan_table_type, ScanTableType::CdcTable) } @@ -557,12 +553,6 @@ impl ToBatch for LogicalScan { impl ToStream for LogicalScan { fn to_stream(&self, ctx: &mut ToStreamContext) -> Result { - if self.is_sys_table() { - return Err(RwError::from(ErrorCode::NotImplemented( - "streaming on system table is not allowed".to_string(), - None.into(), - ))); - } if self.predicate().always_true() { if self.is_cdc_table() { Ok(StreamCdcTableScan::new(self.core.clone()).into()) @@ -583,13 +573,6 @@ impl ToStream for LogicalScan { &self, _ctx: &mut RewriteStreamContext, ) -> Result<(PlanRef, ColIndexMapping)> { - if self.is_sys_table() { - return Err(RwError::from(ErrorCode::NotImplemented( - "streaming on system table is not allowed".to_string(), - None.into(), - ))); - } - if self.is_cdc_table() { return Ok(( self.clone().into(), diff --git a/src/frontend/src/optimizer/plan_node/logical_sys_scan.rs b/src/frontend/src/optimizer/plan_node/logical_sys_scan.rs new file mode 100644 index 000000000000..56985d81a5c2 --- /dev/null +++ b/src/frontend/src/optimizer/plan_node/logical_sys_scan.rs @@ -0,0 +1,378 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::rc::Rc; + +use itertools::Itertools; +use pretty_xmlish::{Pretty, XmlNode}; +use risingwave_common::catalog::{ColumnDesc, TableDesc}; +use risingwave_common::error::{ErrorCode, Result, RwError}; + +use super::generic::{GenericPlanNode, GenericPlanRef}; +use super::utils::{childless_record, Distill}; +use super::{ + generic, BatchFilter, BatchProject, ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, + PredicatePushdown, ToBatch, ToStream, +}; +use crate::expr::{CorrelatedInputRef, ExprImpl, ExprRewriter, ExprVisitor, InputRef}; +use crate::optimizer::optimizer_context::OptimizerContextRef; +use crate::optimizer::plan_node::{ + BatchSysSeqScan, ColumnPruningContext, LogicalFilter, LogicalValues, PredicatePushdownContext, + RewriteStreamContext, ToStreamContext, +}; +use crate::optimizer::property::{Cardinality, Order}; +use crate::utils::{ColIndexMapping, Condition, ConditionDisplay}; + +/// `LogicalSysScan` returns contents of a table or other equivalent object +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct LogicalSysScan { + pub base: PlanBase, + core: generic::SysScan, +} + +impl From for LogicalSysScan { + fn from(core: generic::SysScan) -> Self { + let base = PlanBase::new_logical_with_core(&core); + Self { base, core } + } +} + +impl From for PlanRef { + fn from(core: generic::SysScan) -> Self { + LogicalSysScan::from(core).into() + } +} + +impl LogicalSysScan { + /// Create a [`LogicalSysScan`] node. Used by planner. + pub fn create( + table_name: String, // explain-only + table_desc: Rc, + ctx: OptimizerContextRef, + table_cardinality: Cardinality, + ) -> Self { + generic::SysScan::new( + table_name, + (0..table_desc.columns.len()).collect(), + table_desc, + ctx, + Condition::true_cond(), + table_cardinality, + ) + .into() + } + + pub fn table_name(&self) -> &str { + &self.core.table_name + } + + /// The cardinality of the table **without** applying the predicate. + pub fn table_cardinality(&self) -> Cardinality { + self.core.table_cardinality + } + + /// Get a reference to the logical scan's table desc. + pub fn table_desc(&self) -> &TableDesc { + self.core.table_desc.as_ref() + } + + /// Get the descs of the output columns. + pub fn column_descs(&self) -> Vec { + self.core.column_descs() + } + + /// Get the logical scan's filter predicate + pub fn predicate(&self) -> &Condition { + &self.core.predicate + } + + /// a vec of `InputRef` corresponding to `output_col_idx`, which can represent a pulled project. + fn output_idx_to_input_ref(&self) -> Vec { + let output_idx = self + .output_col_idx() + .iter() + .enumerate() + .map(|(i, &col_idx)| { + InputRef::new(i, self.table_desc().columns[col_idx].data_type.clone()).into() + }) + .collect_vec(); + output_idx + } + + /// Undo predicate push down when predicate in scan is not supported. + pub fn predicate_pull_up(&self) -> (generic::SysScan, Condition, Option>) { + let mut predicate = self.predicate().clone(); + if predicate.always_true() { + return (self.core.clone(), Condition::true_cond(), None); + } + + let mut inverse_mapping = { + let mapping = ColIndexMapping::new( + self.required_col_idx().iter().map(|i| Some(*i)).collect(), + self.table_desc().columns.len(), + ); + // Since `required_col_idx` mapping is not invertible, we need to inverse manually. + let mut inverse_map = vec![None; mapping.target_size()]; + for (src, dst) in mapping.mapping_pairs() { + inverse_map[dst] = Some(src); + } + ColIndexMapping::new(inverse_map, mapping.source_size()) + }; + + predicate = predicate.rewrite_expr(&mut inverse_mapping); + + let scan_without_predicate = generic::SysScan::new( + self.table_name().to_string(), + self.required_col_idx().to_vec(), + self.core.table_desc.clone(), + self.ctx(), + Condition::true_cond(), + self.table_cardinality(), + ); + let project_expr = if self.required_col_idx() != self.output_col_idx() { + Some(self.output_idx_to_input_ref()) + } else { + None + }; + (scan_without_predicate, predicate, project_expr) + } + + fn clone_with_predicate(&self, predicate: Condition) -> Self { + generic::SysScan::new_inner( + self.table_name().to_string(), + self.output_col_idx().to_vec(), + self.core.table_desc.clone(), + self.base.ctx().clone(), + predicate, + self.table_cardinality(), + ) + .into() + } + + pub fn clone_with_output_indices(&self, output_col_idx: Vec) -> Self { + generic::SysScan::new_inner( + self.table_name().to_string(), + output_col_idx, + self.core.table_desc.clone(), + self.base.ctx().clone(), + self.predicate().clone(), + self.table_cardinality(), + ) + .into() + } + + pub fn output_col_idx(&self) -> &Vec { + &self.core.output_col_idx + } + + pub fn required_col_idx(&self) -> &Vec { + &self.core.required_col_idx + } +} + +impl_plan_tree_node_for_leaf! {LogicalSysScan} + +impl Distill for LogicalSysScan { + fn distill<'a>(&self) -> XmlNode<'a> { + let verbose = self.base.ctx().is_explain_verbose(); + let mut vec = Vec::with_capacity(5); + vec.push(("table", Pretty::from(self.table_name().to_owned()))); + let key_is_columns = + self.predicate().always_true() || self.output_col_idx() == self.required_col_idx(); + let key = if key_is_columns { + "columns" + } else { + "output_columns" + }; + vec.push((key, self.core.columns_pretty(verbose))); + if !key_is_columns { + vec.push(( + "required_columns", + Pretty::Array( + self.required_col_idx() + .iter() + .map(|i| { + let col_name = &self.table_desc().columns[*i].name; + Pretty::from(if verbose { + format!("{}.{}", self.table_name(), col_name) + } else { + col_name.to_string() + }) + }) + .collect(), + ), + )); + } + + if !self.predicate().always_true() { + let input_schema = self.core.fields_pretty_schema(); + vec.push(( + "predicate", + Pretty::display(&ConditionDisplay { + condition: self.predicate(), + input_schema: &input_schema, + }), + )) + } + + if self.table_cardinality() != Cardinality::unknown() { + vec.push(("cardinality", Pretty::display(&self.table_cardinality()))); + } + + childless_record("LogicalSysScan", vec) + } +} + +impl ColPrunable for LogicalSysScan { + fn prune_col(&self, required_cols: &[usize], _ctx: &mut ColumnPruningContext) -> PlanRef { + let output_col_idx: Vec = required_cols + .iter() + .map(|i| self.required_col_idx()[*i]) + .collect(); + assert!(output_col_idx + .iter() + .all(|i| self.output_col_idx().contains(i))); + + self.clone_with_output_indices(output_col_idx).into() + } +} + +impl ExprRewritable for LogicalSysScan { + fn has_rewritable_expr(&self) -> bool { + true + } + + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + let mut core = self.core.clone(); + core.rewrite_exprs(r); + Self { + base: self.base.clone_with_new_plan_id(), + core, + } + .into() + } +} + +impl PredicatePushdown for LogicalSysScan { + // TODO(kwannoel): Unify this with logical_scan. + fn predicate_pushdown( + &self, + mut predicate: Condition, + _ctx: &mut PredicatePushdownContext, + ) -> PlanRef { + // If the predicate contains `CorrelatedInputRef` or `now()`. We don't push down. + // This case could come from the predicate push down before the subquery unnesting. + struct HasCorrelated {} + impl ExprVisitor for HasCorrelated { + type Result = bool; + + fn merge(a: bool, b: bool) -> bool { + a | b + } + + fn visit_correlated_input_ref(&mut self, _: &CorrelatedInputRef) -> bool { + true + } + } + let non_pushable_predicate: Vec<_> = predicate + .conjunctions + .extract_if(|expr| expr.count_nows() > 0 || HasCorrelated {}.visit_expr(expr)) + .collect(); + let predicate = predicate.rewrite_expr(&mut ColIndexMapping::new( + self.output_col_idx().iter().map(|i| Some(*i)).collect(), + self.table_desc().columns.len(), + )); + if non_pushable_predicate.is_empty() { + self.clone_with_predicate(predicate.and(self.predicate().clone())) + .into() + } else { + return LogicalFilter::create( + self.clone_with_predicate(predicate.and(self.predicate().clone())) + .into(), + Condition { + conjunctions: non_pushable_predicate, + }, + ); + } + } +} + +impl LogicalSysScan { + // TODO(kwannoel): Unify this with logical_scan. + fn to_batch_inner_with_required(&self, required_order: &Order) -> Result { + if self.predicate().always_true() { + required_order + .enforce_if_not_satisfies(BatchSysSeqScan::new(self.core.clone(), vec![]).into()) + } else { + let (scan_ranges, predicate) = self.predicate().clone().split_to_scan_ranges( + self.core.table_desc.clone(), + self.base + .ctx() + .session_ctx() + .config() + .get_max_split_range_gap(), + )?; + let mut scan = self.clone(); + scan.core.predicate = predicate; // We want to keep `required_col_idx` unchanged, so do not call `clone_with_predicate`. + + let plan: PlanRef = if scan.core.predicate.always_false() { + LogicalValues::create(vec![], scan.core.schema(), scan.core.ctx).to_batch()? + } else { + let (scan, predicate, project_expr) = scan.predicate_pull_up(); + + let mut plan: PlanRef = BatchSysSeqScan::new(scan, scan_ranges).into(); + if !predicate.always_true() { + plan = BatchFilter::new(generic::Filter::new(predicate, plan)).into(); + } + if let Some(exprs) = project_expr { + plan = BatchProject::new(generic::Project::new(exprs, plan)).into() + } + plan + }; + + assert_eq!(plan.schema(), self.schema()); + required_order.enforce_if_not_satisfies(plan) + } + } +} + +impl ToBatch for LogicalSysScan { + fn to_batch(&self) -> Result { + self.to_batch_with_order_required(&Order::any()) + } + + fn to_batch_with_order_required(&self, required_order: &Order) -> Result { + let new = self.clone_with_predicate(self.predicate().clone()); + new.to_batch_inner_with_required(required_order) + } +} + +impl ToStream for LogicalSysScan { + fn to_stream(&self, _ctx: &mut ToStreamContext) -> Result { + Err(RwError::from(ErrorCode::NotImplemented( + "streaming on system table is not allowed".to_string(), + None.into(), + ))) + } + + fn logical_rewrite_for_stream( + &self, + _ctx: &mut RewriteStreamContext, + ) -> Result<(PlanRef, ColIndexMapping)> { + Err(RwError::from(ErrorCode::NotImplemented( + "streaming on system table is not allowed".to_string(), + None.into(), + ))) + } +} diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index e048f0a4707d..77f8fdb60607 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -768,6 +768,7 @@ mod batch_simple_agg; mod batch_sort; mod batch_sort_agg; mod batch_source; +mod batch_sys_seq_scan; mod batch_table_function; mod batch_topn; mod batch_union; @@ -793,6 +794,7 @@ mod logical_project_set; mod logical_scan; mod logical_share; mod logical_source; +mod logical_sys_scan; mod logical_table_function; mod logical_topn; mod logical_union; @@ -854,6 +856,7 @@ pub use batch_simple_agg::BatchSimpleAgg; pub use batch_sort::BatchSort; pub use batch_sort_agg::BatchSortAgg; pub use batch_source::BatchSource; +pub use batch_sys_seq_scan::BatchSysSeqScan; pub use batch_table_function::BatchTableFunction; pub use batch_topn::BatchTopN; pub use batch_union::BatchUnion; @@ -879,6 +882,7 @@ pub use logical_project_set::LogicalProjectSet; pub use logical_scan::LogicalScan; pub use logical_share::LogicalShare; pub use logical_source::LogicalSource; +pub use logical_sys_scan::LogicalSysScan; pub use logical_table_function::LogicalTableFunction; pub use logical_topn::LogicalTopN; pub use logical_union::LogicalUnion; @@ -944,6 +948,7 @@ macro_rules! for_all_plan_nodes { , { Logical, Filter } , { Logical, Project } , { Logical, Scan } + , { Logical, SysScan } , { Logical, Source } , { Logical, Insert } , { Logical, Delete } @@ -973,6 +978,7 @@ macro_rules! for_all_plan_nodes { , { Batch, Delete } , { Batch, Update } , { Batch, SeqScan } + , { Batch, SysSeqScan } , { Batch, HashJoin } , { Batch, NestedLoopJoin } , { Batch, Values } @@ -1035,6 +1041,7 @@ macro_rules! for_logical_plan_nodes { , { Logical, Filter } , { Logical, Project } , { Logical, Scan } + , { Logical, SysScan } , { Logical, Source } , { Logical, Insert } , { Logical, Delete } @@ -1070,6 +1077,7 @@ macro_rules! for_batch_plan_nodes { , { Batch, Project } , { Batch, Filter } , { Batch, SeqScan } + , { Batch, SysSeqScan } , { Batch, HashJoin } , { Batch, NestedLoopJoin } , { Batch, Values } diff --git a/src/frontend/src/optimizer/plan_visitor/cardinality_visitor.rs b/src/frontend/src/optimizer/plan_visitor/cardinality_visitor.rs index a9eb5639fcf8..2072defa6272 100644 --- a/src/frontend/src/optimizer/plan_visitor/cardinality_visitor.rs +++ b/src/frontend/src/optimizer/plan_visitor/cardinality_visitor.rs @@ -56,8 +56,8 @@ impl CardinalityVisitor { return None; } let scan = proj.input(); - let scan = scan.as_logical_scan()?; - if scan.is_sys_table() && scan.table_name() == "rw_schemas" { + let scan = scan.as_logical_sys_scan()?; + if scan.table_name() == "rw_schemas" { if let Some(name) = scan .output_col_idx() .iter() diff --git a/src/frontend/src/optimizer/plan_visitor/relation_collector_visitor.rs b/src/frontend/src/optimizer/plan_visitor/relation_collector_visitor.rs index e899021b8bde..daabe4d4b471 100644 --- a/src/frontend/src/optimizer/plan_visitor/relation_collector_visitor.rs +++ b/src/frontend/src/optimizer/plan_visitor/relation_collector_visitor.rs @@ -52,22 +52,16 @@ impl PlanVisitor for RelationCollectorVisitor { } fn visit_batch_seq_scan(&mut self, plan: &crate::optimizer::plan_node::BatchSeqScan) { - if !plan.core().is_sys_table() { - self.relations.insert(plan.core().table_desc.table_id); - } + self.relations.insert(plan.core().table_desc.table_id); } fn visit_logical_scan(&mut self, plan: &LogicalScan) { - if !plan.is_sys_table() { - self.relations.insert(plan.table_desc().table_id); - } + self.relations.insert(plan.table_desc().table_id); } fn visit_stream_table_scan(&mut self, plan: &StreamTableScan) { let logical = plan.core(); - if !logical.is_sys_table() { - self.relations.insert(logical.table_desc.table_id); - } + self.relations.insert(logical.table_desc.table_id); } fn visit_batch_source(&mut self, plan: &BatchSource) { diff --git a/src/frontend/src/optimizer/plan_visitor/sys_table_visitor.rs b/src/frontend/src/optimizer/plan_visitor/sys_table_visitor.rs index e93a5c4e0bb1..0f679f7f3030 100644 --- a/src/frontend/src/optimizer/plan_visitor/sys_table_visitor.rs +++ b/src/frontend/src/optimizer/plan_visitor/sys_table_visitor.rs @@ -13,7 +13,7 @@ // limitations under the License. use super::{DefaultBehavior, Merge}; -use crate::optimizer::plan_node::{BatchSeqScan, LogicalScan, StreamTableScan}; +use crate::optimizer::plan_node::{BatchSysSeqScan, LogicalSysScan, StreamTableScan}; use crate::optimizer::plan_visitor::PlanVisitor; use crate::PlanRef; @@ -36,15 +36,16 @@ impl PlanVisitor for SysTableVisitor { Merge(|a, b| a | b) } - fn visit_batch_seq_scan(&mut self, batch_seq_scan: &BatchSeqScan) -> bool { - batch_seq_scan.core().is_sys_table() + fn visit_batch_sys_seq_scan(&mut self, _batch_seq_scan: &BatchSysSeqScan) -> bool { + true } - fn visit_logical_scan(&mut self, logical_scan: &LogicalScan) -> bool { - logical_scan.is_sys_table() + fn visit_logical_sys_scan(&mut self, _logical_scan: &LogicalSysScan) -> bool { + true } - fn visit_stream_table_scan(&mut self, stream_table_scan: &StreamTableScan) -> bool { - stream_table_scan.core().is_sys_table() + // Sys scan not allowed for streaming. + fn visit_stream_table_scan(&mut self, _stream_table_scan: &StreamTableScan) -> bool { + false } } diff --git a/src/frontend/src/planner/relation.rs b/src/frontend/src/planner/relation.rs index d4686bbf15c2..b4987403a792 100644 --- a/src/frontend/src/planner/relation.rs +++ b/src/frontend/src/planner/relation.rs @@ -27,7 +27,7 @@ use crate::expr::{Expr, ExprImpl, ExprType, FunctionCall, InputRef}; use crate::optimizer::plan_node::generic::ScanTableType; use crate::optimizer::plan_node::{ LogicalApply, LogicalHopWindow, LogicalJoin, LogicalProject, LogicalScan, LogicalShare, - LogicalSource, LogicalTableFunction, LogicalValues, PlanRef, + LogicalSource, LogicalSysScan, LogicalTableFunction, LogicalValues, PlanRef, }; use crate::optimizer::property::Cardinality; use crate::planner::Planner; @@ -57,13 +57,10 @@ impl Planner { } pub(crate) fn plan_sys_table(&mut self, sys_table: BoundSystemTable) -> Result { - Ok(LogicalScan::create( + Ok(LogicalSysScan::create( sys_table.sys_table_catalog.name().to_string(), - ScanTableType::SysTable, Rc::new(sys_table.sys_table_catalog.table_desc()), - vec![], self.ctx(), - false, Cardinality::unknown(), // TODO(card): cardinality of system table ) .into()) diff --git a/src/frontend/src/scheduler/plan_fragmenter.rs b/src/frontend/src/scheduler/plan_fragmenter.rs index b20197f9a491..fdc546a7d735 100644 --- a/src/frontend/src/scheduler/plan_fragmenter.rs +++ b/src/frontend/src/scheduler/plan_fragmenter.rs @@ -916,26 +916,25 @@ impl BatchPlanFragmenter { // Do not visit next stage. return Ok(None); } + if let Some(scan_node) = node.as_batch_sys_seq_scan() { + let name = scan_node.core().table_name.to_owned(); + return Ok(Some(TableScanInfo::system_table(name))); + } if let Some(scan_node) = node.as_batch_seq_scan() { let name = scan_node.core().table_name.to_owned(); - let info = if scan_node.core().is_sys_table() { - TableScanInfo::system_table(name) - } else { - let table_desc = &*scan_node.core().table_desc; - let table_catalog = self - .catalog_reader - .read_guard() - .get_table_by_id(&table_desc.table_id) - .cloned() - .map_err(RwError::from)?; - let vnode_mapping = self - .worker_node_manager - .fragment_mapping(table_catalog.fragment_id)?; - let partitions = - derive_partitions(scan_node.scan_ranges(), table_desc, &vnode_mapping); - TableScanInfo::new(name, partitions) - }; + let table_desc = &*scan_node.core().table_desc; + let table_catalog = self + .catalog_reader + .read_guard() + .get_table_by_id(&table_desc.table_id) + .cloned() + .map_err(RwError::from)?; + let vnode_mapping = self + .worker_node_manager + .fragment_mapping(table_catalog.fragment_id)?; + let partitions = derive_partitions(scan_node.scan_ranges(), table_desc, &vnode_mapping); + let info = TableScanInfo::new(name, partitions); Ok(Some(info)) } else { node.inputs() From 7dd89f26146c206d106583be392472018af6c5c2 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Thu, 16 Nov 2023 16:28:15 +0800 Subject: [PATCH 06/16] fix(ci): remove sink json payload and jdbc sink from connector integration test (#13449) Co-authored-by: Noel Kwan <47273164+kwannoel@users.noreply.github.com> --- ci/scripts/connector-node-integration-test.sh | 16 ++---- .../python-client/integration_tests.py | 54 ------------------- 2 files changed, 3 insertions(+), 67 deletions(-) diff --git a/ci/scripts/connector-node-integration-test.sh b/ci/scripts/connector-node-integration-test.sh index 5971cd8986ab..6b3147bbeb19 100755 --- a/ci/scripts/connector-node-integration-test.sh +++ b/ci/scripts/connector-node-integration-test.sh @@ -99,10 +99,9 @@ else exit 1 fi -sink_input_feature=("" "--input_binary_file=./data/sink_input --data_format_use_json=False") -upsert_sink_input_feature=("--input_file=./data/upsert_sink_input.json" - "--input_binary_file=./data/upsert_sink_input --data_format_use_json=False") -type=("Json format" "StreamChunk format") +sink_input_feature=("--input_binary_file=./data/sink_input --data_format_use_json=False") +upsert_sink_input_feature=("--input_binary_file=./data/upsert_sink_input --data_format_use_json=False") +type=("StreamChunk format") ${MC_PATH} mb minio/bucket for ((i=0; i<${#type[@]}; i++)); do @@ -115,15 +114,6 @@ for ((i=0; i<${#type[@]}; i++)); do exit 1 fi - echo "--- running jdbc ${type[i]} integration tests" - cd ${RISINGWAVE_ROOT}/java/connector-node/python-client - if python3 integration_tests.py --jdbc_sink ${sink_input_feature[i]}; then - echo "Jdbc sink ${type[i]} test passed" - else - echo "Jdbc sink ${type[i]} test failed" - exit 1 - fi - # test upsert mode echo "--- running iceberg upsert mode ${type[i]} integration tests" cd ${RISINGWAVE_ROOT}/java/connector-node/python-client diff --git a/java/connector-node/python-client/integration_tests.py b/java/connector-node/python-client/integration_tests.py index 64fa949f48ce..99d91b74dd4b 100644 --- a/java/connector-node/python-client/integration_tests.py +++ b/java/connector-node/python-client/integration_tests.py @@ -211,45 +211,6 @@ def test_sink(prop, format, payload_input, table_schema, is_coordinated=False): exit(1) -def validate_jdbc_sink(input_file): - conn = psycopg2.connect( - "dbname=test user=test password=connector host=localhost port=5432" - ) - cur = conn.cursor() - cur.execute("SELECT * FROM test") - rows = cur.fetchall() - expected = [list(row.values()) for batch in load_input(input_file) for row in batch] - - def convert(b): - return [(item[1]["id"], item[1]["name"]) for item in b] - - expected = convert(expected) - - if len(rows) != len(expected): - print( - "Integration test failed: expected {} rows, but got {}".format( - len(expected), len(rows) - ) - ) - exit(1) - for i in range(len(rows)): - if len(rows[i]) != len(expected[i]): - print( - "Integration test failed: expected {} columns, but got {}".format( - len(expected[i]), len(rows[i]) - ) - ) - exit(1) - for j in range(len(rows[i])): - if rows[i][j] != expected[i][j]: - print( - "Integration test failed: expected {} at row {}, column {}, but got {}".format( - expected[i][j], i, j, rows[i][j] - ) - ) - exit(1) - - def test_file_sink(param): prop = { "connector": "file", @@ -258,18 +219,6 @@ def test_file_sink(param): test_sink(prop, **param) -def test_jdbc_sink(input_file, param): - prop = { - "connector": "jdbc", - "jdbc.url": "jdbc:postgresql://localhost:5432/test?user=test&password=connector", - "table.name": "test", - "type": "upsert", - } - test_sink(prop, **param) - # validate results - validate_jdbc_sink(input_file) - - def test_elasticsearch_sink(param): prop = { "connector": "elasticsearch", @@ -333,7 +282,6 @@ def test_stream_chunk_data_format(param): formatter_class=argparse.ArgumentDefaultsHelpFormatter ) parser.add_argument("--file_sink", action="store_true", help="run file sink test") - parser.add_argument("--jdbc_sink", action="store_true", help="run jdbc sink test") parser.add_argument( "--stream_chunk_format_test", action="store_true", @@ -390,8 +338,6 @@ def test_stream_chunk_data_format(param): if args.file_sink: test_file_sink(param) - if args.jdbc_sink: - test_jdbc_sink(args.input_file, param) if args.iceberg_sink: test_iceberg_sink(param) if args.deltalake_sink: From 98f8e1a4464eee346b09f0d8ed0201a05c9e2e0e Mon Sep 17 00:00:00 2001 From: stonepage <40830455+st1page@users.noreply.github.com> Date: Thu, 16 Nov 2023 16:41:59 +0800 Subject: [PATCH 07/16] perf(optimizer): omit unnecessary clone (#13466) --- .../tests/testdata/output/cse_expr.yaml | 2 +- .../tests/testdata/output/explain.yaml | 2 +- .../planner_test/tests/testdata/output/expr.yaml | 4 ++-- .../tests/testdata/output/nexmark.yaml | 2 +- .../src/optimizer/heuristic_optimizer.rs | 16 ++++++++++++++-- 5 files changed, 19 insertions(+), 7 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/output/cse_expr.yaml b/src/frontend/planner_test/tests/testdata/output/cse_expr.yaml index 5057fa13473c..fe41cb83794e 100644 --- a/src/frontend/planner_test/tests/testdata/output/cse_expr.yaml +++ b/src/frontend/planner_test/tests/testdata/output/cse_expr.yaml @@ -84,5 +84,5 @@ sql: | with t(v, arr) as (select 1, array[2, 3]) select v < all(arr), v < some(arr) from t; batch_plan: |- - BatchProject { exprs: [All((1:Int32 < $expr10048)) as $expr1, Some((1:Int32 < $expr10048)) as $expr2] } + BatchProject { exprs: [All((1:Int32 < $expr10015)) as $expr1, Some((1:Int32 < $expr10015)) as $expr2] } └─BatchValues { rows: [[1:Int32, ARRAY[2, 3]:List(Int32)]] } diff --git a/src/frontend/planner_test/tests/testdata/output/explain.yaml b/src/frontend/planner_test/tests/testdata/output/explain.yaml index eced1aaf718b..0faca9c10291 100644 --- a/src/frontend/planner_test/tests/testdata/output/explain.yaml +++ b/src/frontend/planner_test/tests/testdata/output/explain.yaml @@ -65,7 +65,7 @@ "stages": { "0": { "root": { - "plan_node_id": 10031, + "plan_node_id": 10020, "plan_node_type": "BatchValues", "schema": [ { diff --git a/src/frontend/planner_test/tests/testdata/output/expr.yaml b/src/frontend/planner_test/tests/testdata/output/expr.yaml index 65df2522c81d..7f117e64be4e 100644 --- a/src/frontend/planner_test/tests/testdata/output/expr.yaml +++ b/src/frontend/planner_test/tests/testdata/output/expr.yaml @@ -450,7 +450,7 @@ └─LogicalProject { exprs: [Array(1:Int32) as $expr1] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: |- - BatchProject { exprs: [Some((1:Int32 < ArrayCat($expr10041, ARRAY[2]:List(Int32)))) as $expr1] } + BatchProject { exprs: [Some((1:Int32 < ArrayCat($expr10013, ARRAY[2]:List(Int32)))) as $expr1] } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchValues { rows: [[]] } └─BatchValues { rows: [[ARRAY[1]:List(Int32)]] } @@ -473,7 +473,7 @@ └─LogicalProject { exprs: [Array(1:Int32) as $expr1] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: |- - BatchProject { exprs: [All((1:Int32 < ArrayCat($expr10041, ARRAY[2]:List(Int32)))) as $expr1] } + BatchProject { exprs: [All((1:Int32 < ArrayCat($expr10013, ARRAY[2]:List(Int32)))) as $expr1] } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchValues { rows: [[]] } └─BatchValues { rows: [[ARRAY[1]:List(Int32)]] } diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml index 9b754512b1da..592696190a86 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml @@ -1127,7 +1127,7 @@ JOIN side_input FOR SYSTEM_TIME AS OF PROCTIME() S ON mod(B.auction, 10000) = S.key sink_plan: |- - StreamSink { type: append-only, columns: [auction, bidder, price, date_time, value, bid._row_id(hidden), $expr10042(hidden), side_input.key(hidden)] } + StreamSink { type: append-only, columns: [auction, bidder, price, date_time, value, bid._row_id(hidden), $expr10022(hidden), side_input.key(hidden)] } └─StreamTemporalJoin { type: Inner, predicate: $expr1 = side_input.key, output: [bid.auction, bid.bidder, bid.price, bid.date_time, side_input.value, bid._row_id, $expr1, side_input.key] } ├─StreamExchange { dist: HashShard($expr1) } │ └─StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.date_time, (bid.auction % 10000:Int32) as $expr1, bid._row_id] } diff --git a/src/frontend/src/optimizer/heuristic_optimizer.rs b/src/frontend/src/optimizer/heuristic_optimizer.rs index 5e4a6b2c8570..7db756977bbe 100644 --- a/src/frontend/src/optimizer/heuristic_optimizer.rs +++ b/src/frontend/src/optimizer/heuristic_optimizer.rs @@ -62,12 +62,17 @@ impl<'a> HeuristicOptimizer<'a> { } fn optimize_inputs(&mut self, plan: PlanRef) -> PlanRef { + let pre_applied = self.stats.total_applied(); let inputs = plan .inputs() .into_iter() .map(|sub_tree| self.optimize(sub_tree)) .collect_vec(); - plan.clone_with_inputs(&inputs) + if pre_applied != self.stats.total_applied() { + plan.clone_with_inputs(&inputs) + } else { + plan + } } pub fn optimize(&mut self, mut plan: PlanRef) -> PlanRef { @@ -102,6 +107,7 @@ impl<'a> HeuristicOptimizer<'a> { } pub struct Stats { + total_applied: usize, rule_counter: HashMap, } @@ -109,10 +115,12 @@ impl Stats { pub fn new() -> Self { Self { rule_counter: HashMap::new(), + total_applied: 0, } } pub fn count_rule(&mut self, rule: &BoxedRule) { + self.total_applied += 1; match self.rule_counter.entry(rule.description().to_string()) { Entry::Occupied(mut entry) => { *entry.get_mut() += 1; @@ -124,7 +132,11 @@ impl Stats { } pub fn has_applied_rule(&self) -> bool { - !self.rule_counter.is_empty() + self.total_applied != 0 + } + + pub fn total_applied(&self) -> usize { + self.total_applied } } From 53d19565c650d3deb6679039c46ad6c0f089ec4a Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Thu, 16 Nov 2023 17:04:18 +0800 Subject: [PATCH 08/16] chore(ci): notify for more tests (#13463) --- ci/scripts/notify.py | 3 +++ ci/workflows/main-cron.yml | 3 +++ 2 files changed, 6 insertions(+) diff --git a/ci/scripts/notify.py b/ci/scripts/notify.py index 5ca526115810..7ea47781a1a8 100755 --- a/ci/scripts/notify.py +++ b/ci/scripts/notify.py @@ -9,6 +9,9 @@ TEST_MAP = { "test-notify": ["noelkwan", "noelkwan"], "backfill-tests": ["noelkwan"], + "backwards-compat-tests": ["noelkwan"], + "fuzz-test": ["noelkwan"], + "e2e-test-release": ["zhi"], "e2e-iceberg-sink-tests": ["renjie"], "e2e-java-binding-tests": ["yiming"], "e2e-clickhouse-sink-tests": ["bohan"], diff --git a/ci/workflows/main-cron.yml b/ci/workflows/main-cron.yml index 42fccac95af2..ac7867623dbf 100644 --- a/ci/workflows/main-cron.yml +++ b/ci/workflows/main-cron.yml @@ -59,6 +59,7 @@ steps: retry: *auto-retry - label: "end-to-end test (release)" + key: "e2e-test-release" command: "ci/scripts/cron-e2e-test.sh -p ci-release -m ci-3streaming-2serving-3fe" if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-e2e-test") depends_on: @@ -141,6 +142,7 @@ steps: retry: *auto-retry - label: "fuzz test" + key: "fuzz-test" command: "ci/scripts/cron-fuzz-test.sh -p ci-release" if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-sqlsmith-fuzzing-tests") depends_on: @@ -481,6 +483,7 @@ steps: # Backwards compatibility tests - label: "Backwards compatibility tests" + key: "backwards-compat-tests" command: "RW_COMMIT=$BUILDKITE_COMMIT ci/scripts/backwards-compat-test.sh -p ci-release" if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-backwards-compat-tests") depends_on: From c6dffdf391dc64fa832061b938cb2eebc3591699 Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Thu, 16 Nov 2023 17:51:08 +0800 Subject: [PATCH 09/16] feat(datagen): support `timestamptz` (#13451) --- e2e_test/source/basic/datagen.slt | 6 ++++- src/common/src/array/data_chunk.rs | 7 ++++++ src/common/src/field_generator/mod.rs | 23 +++++++++++++++++- src/common/src/field_generator/timestamp.rs | 24 ++++++++++++++++++- .../src/source/datagen/source/reader.rs | 23 ++++++++++++------ 5 files changed, 73 insertions(+), 10 deletions(-) diff --git a/e2e_test/source/basic/datagen.slt b/e2e_test/source/basic/datagen.slt index dcda3718e49d..91c51f624a1e 100644 --- a/e2e_test/source/basic/datagen.slt +++ b/e2e_test/source/basic/datagen.slt @@ -102,7 +102,7 @@ statement ok drop table s1; statement ok -create table s1 (v1 struct, t1 timestamp, c1 varchar) with ( +create table s1 (v1 struct, t1 timestamp, z1 timestamptz, c1 varchar) with ( connector = 'datagen', fields.v1.v2.kind = 'random', fields.v1.v2.min = '1', @@ -112,6 +112,10 @@ create table s1 (v1 struct, t1 timestamp, c1 varchar) with ( fields.t1.max_past = '2h 37min', fields.t1.max_past_mode = 'relative', fields.t1.seed = '3', + fields.z1.kind = 'random', + fields.z1.max_past = '2h 37min', + fields.z1.max_past_mode = 'relative', + fields.z1.seed = '3', fields.c1.kind = 'random', fields.c1.length = '100', fields.c1.seed = '3', diff --git a/src/common/src/array/data_chunk.rs b/src/common/src/array/data_chunk.rs index 98a237814176..7cdeeeee4e2a 100644 --- a/src/common/src/array/data_chunk.rs +++ b/src/common/src/array/data_chunk.rs @@ -919,6 +919,13 @@ impl DataChunkTestExt for DataChunk { .generate_datum(offset); array_builder.append(datum); } + DataType::Timestamptz => { + let datum = + FieldGeneratorImpl::with_timestamptz(None, None, None, Self::SEED) + .expect("create timestamptz generator should succeed") + .generate_datum(offset); + array_builder.append(datum); + } _ if data_type.is_numeric() => { let mut data_gen = FieldGeneratorImpl::with_number_random( data_type.clone(), diff --git a/src/common/src/field_generator/mod.rs b/src/common/src/field_generator/mod.rs index 02a958aa0ebe..0daa2b640c60 100644 --- a/src/common/src/field_generator/mod.rs +++ b/src/common/src/field_generator/mod.rs @@ -26,7 +26,7 @@ pub use timestamp::*; pub use varchar::*; use crate::array::{ListValue, StructValue}; -use crate::types::{DataType, Datum, ScalarImpl, Timestamp}; +use crate::types::{DataType, Datum, ScalarImpl, Timestamp, Timestamptz}; pub const DEFAULT_MIN: i16 = i16::MIN; pub const DEFAULT_MAX: i16 = i16::MAX; @@ -96,6 +96,7 @@ pub enum FieldGeneratorImpl { VarcharRandomFixedLength(VarcharRandomFixedLengthField), VarcharConstant, Timestamp(ChronoField), + Timestamptz(ChronoField), Struct(Vec<(String, FieldGeneratorImpl)>), List(Box, usize), } @@ -189,6 +190,20 @@ impl FieldGeneratorImpl { )?)) } + pub fn with_timestamptz( + base: Option>, + max_past: Option, + max_past_mode: Option, + seed: u64, + ) -> Result { + Ok(FieldGeneratorImpl::Timestamptz(ChronoField::new( + base, + max_past, + max_past_mode, + seed, + )?)) + } + pub fn with_varchar(varchar_property: &VarcharProperty, seed: u64) -> Self { match varchar_property { VarcharProperty::RandomFixedLength(length_option) => { @@ -235,6 +250,7 @@ impl FieldGeneratorImpl { FieldGeneratorImpl::VarcharRandomVariableLength(f) => f.generate(offset), FieldGeneratorImpl::VarcharConstant => VarcharConstant::generate_json(), FieldGeneratorImpl::Timestamp(f) => f.generate(offset), + FieldGeneratorImpl::Timestamptz(f) => f.generate(offset), FieldGeneratorImpl::Struct(fields) => { let map = fields .iter_mut() @@ -267,6 +283,7 @@ impl FieldGeneratorImpl { FieldGeneratorImpl::VarcharRandomVariableLength(f) => f.generate_datum(offset), FieldGeneratorImpl::VarcharConstant => VarcharConstant::generate_datum(), FieldGeneratorImpl::Timestamp(f) => f.generate_datum(offset), + FieldGeneratorImpl::Timestamptz(f) => f.generate_datum(offset), FieldGeneratorImpl::Struct(fields) => { let data = fields .iter_mut() @@ -328,6 +345,7 @@ mod tests { DataType::Float64, DataType::Varchar, DataType::Timestamp, + DataType::Timestamptz, ] { let mut generator = match data_type { DataType::Varchar => FieldGeneratorImpl::with_varchar( @@ -337,6 +355,9 @@ mod tests { DataType::Timestamp => { FieldGeneratorImpl::with_timestamp(None, None, None, seed).unwrap() } + DataType::Timestamptz => { + FieldGeneratorImpl::with_timestamptz(None, None, None, seed).unwrap() + } _ => FieldGeneratorImpl::with_number_random(data_type, None, None, seed).unwrap(), }; diff --git a/src/common/src/field_generator/timestamp.rs b/src/common/src/field_generator/timestamp.rs index 54c55de27396..faddd6d0a32a 100644 --- a/src/common/src/field_generator/timestamp.rs +++ b/src/common/src/field_generator/timestamp.rs @@ -22,7 +22,7 @@ use serde_json::Value; use tracing::debug; use super::DEFAULT_MAX_PAST; -use crate::types::{Datum, Scalar, Timestamp}; +use crate::types::{Datum, Scalar, Timestamp, Timestamptz}; pub struct ChronoField { max_past: Duration, @@ -106,3 +106,25 @@ impl ChronoFieldInner for Timestamp { Value::String(self.0.to_string()) } } + +impl ChronoFieldInner for Timestamptz { + fn from_now() -> Self { + Timestamptz::from( + Utc::now() + .duration_round(Duration::microseconds(1)) + .unwrap(), + ) + } + + fn from_base(base: DateTime) -> Self { + Timestamptz::from(base) + } + + fn minus(&self, duration: Duration) -> Self { + Timestamptz::from(self.to_datetime_utc() - duration) + } + + fn to_json(&self) -> Value { + Value::String(self.to_string()) + } +} diff --git a/src/connector/src/source/datagen/source/reader.rs b/src/connector/src/source/datagen/source/reader.rs index 11cb9db08c48..2cc56924876d 100644 --- a/src/connector/src/source/datagen/source/reader.rs +++ b/src/connector/src/source/datagen/source/reader.rs @@ -213,7 +213,7 @@ fn generator_from_data_type( None => split_index, }; match data_type { - DataType::Timestamp => { + ty @ (DataType::Timestamp | DataType::Timestamptz) => { let max_past_key = format!("fields.{}.max_past", name); let max_past_value = fields_option_map.get(&max_past_key).cloned(); let max_past_mode_key = format!("fields.{}.max_past_mode", name); @@ -230,12 +230,21 @@ fn generator_from_data_type( None => None, }; - FieldGeneratorImpl::with_timestamp( - basetime, - max_past_value, - max_past_mode_value, - random_seed, - ) + if ty == DataType::Timestamptz { + FieldGeneratorImpl::with_timestamptz( + basetime, + max_past_value, + max_past_mode_value, + random_seed, + ) + } else { + FieldGeneratorImpl::with_timestamp( + basetime, + max_past_value, + max_past_mode_value, + random_seed, + ) + } } DataType::Varchar => { let length_key = format!("fields.{}.length", name); From acb33762b65767539deb0e7b2a9b3c56259ceb5f Mon Sep 17 00:00:00 2001 From: Yingjun Wu Date: Thu, 16 Nov 2023 02:31:32 -0800 Subject: [PATCH 10/16] chore: Update README.md (#13468) Co-authored-by: hengm3467 <100685635+hengm3467@users.noreply.github.com> --- README.md | 28 +++++++++++----------------- 1 file changed, 11 insertions(+), 17 deletions(-) diff --git a/README.md b/README.md index f2caa86a2332..ab83b8b80a3e 100644 --- a/README.md +++ b/README.md @@ -60,7 +60,7 @@ -RisingWave is a distributed SQL streaming database that enables cost-efficient and reliable processing of streaming data. +RisingWave is a distributed SQL streaming database that enables simple, efficient, and reliable processing of streaming data. ![RisingWave](https://github.com/risingwavelabs/risingwave-docs/blob/main/docs/images/new_archi_grey.png) @@ -96,30 +96,24 @@ For **Kubernetes deployments**, please refer to [Kubernetes with Helm](https://d ## Why RisingWave for stream processing? -RisingWave adaptly addresses some of the most challenging problems in stream processing. Compared to existing stream processing systems like [Apache Flink](https://flink.apache.org/), [Apache Spark Streaming](https://spark.apache.org/docs/latest/streaming-programming-guide.html), and [KsqlDB](https://ksqldb.io/), RisingWave stands out in two primary dimensions: **Ease-of-use** and **efficiency**, thanks to its **[PostgreSQL](https://www.postgresql.org/)-style interaction experience** and **[Snowflake](https://snowflake.com/)-like architectural design** (i.e., compute-storage decoupling). +RisingWave specializes in providing **incrementally updated, consistent materialized views** — a persistent data structure that represents the results of stream processing. RisingWave significantly reduces the complexity of building stream processing applications by allowing developers to express intricate stream processing logic through cascaded materialized views. Furthermore, it allows users to persist data directly within the system, eliminating the need to deliver results to external databases for storage and query serving. + +Compared to existing stream processing systems like [Apache Flink](https://flink.apache.org/), [Apache Spark Streaming](https://spark.apache.org/docs/latest/streaming-programming-guide.html), and [KsqlDB](https://ksqldb.io/), RisingWave stands out in two primary dimensions: **Ease-of-use** and **cost efficiency**, thanks to its **[PostgreSQL](https://www.postgresql.org/)-style interaction experience** and **[Snowflake](https://snowflake.com/)-like architectural design** (i.e., decoupled storage and compute). ### Ease-of-use * **Simple to learn** * RisingWave speaks PostgreSQL-style SQL, enabling users to dive into stream processing in much the same way as operating a PostgreSQL database. -* **Simple to verify correctness** - * RisingWave persists results in materialized views and allow users to break down complex stream computation programs into stacked materialized views, simplifying program development and result verification. -* **Simple to maintain and operate** - * RisingWave abstracts away unnecessary low-level details, allowing users to concentrate solely on SQL code-level issues. +* **Simple to develop** + * RisingWave operates as a relational database, allowing users to decompose stream processing logic into smaller, manageable, stacked materialized views, rather than dealing with extensive computational programs. * **Simple to integrate** * With integrations to a diverse range of cloud systems and the PostgreSQL ecosystem, RisingWave boasts a rich and expansive ecosystem, making it straightforward to incorporate into existing infrastructures. -### Efficiency -* **High resource utilization** - * Queries in RisingWave leverage shared computational resources, eliminating the need for users to manually allocate resources for each query. -* **No compromise on large state management** - * The decoupled compute-storage architecture of RisingWave ensures remote persistence of internal states, and users never need to worry about the size of internal states when handling complex queries. -* **Highly efficient in multi-stream joins** - * RisingWave has made significant optimizations for multiple stream join scenarios. Users can easily join 10-20 streams (or more) efficiently in a production environment. +### Cost efficiency +* **Highly efficient in complex queries** + * RisingWave persists internal states in remote storages (e.g., S3), and users can confidently and efficiently perform complex streaming queries (e.g., joining dozens of data streams) in a production environment, without worrying about state size. * **Transparent dynamic scaling** - * RisingWave supports near-instantaneous dynamic scaling without any service interruptions. + * RisingWave's state management mechanism enables near-instantaneous dynamic scaling without any service interruptions. * **Instant failure recovery** - * RisingWave's state management mechanism allows it to recover from failure in seconds, not minutes or hours. -* **Simplified data stack** - * RisingWave's ability to store data and serve queries eliminates the need for separate maintenance of stream processors and databases. Users can effortlessly connect RisingWave to their preferred BI tools or through client libraries. + * RisingWave's state management mechanism also allows it to recover from failure in seconds, not minutes or hours. ## RisingWave's limitations RisingWave isn’t a panacea for all data engineering hurdles. It has its own set of limitations: From 059a84037b157ff4465f862e58a4968003dc2790 Mon Sep 17 00:00:00 2001 From: stonepage <40830455+st1page@users.noreply.github.com> Date: Thu, 16 Nov 2023 20:47:53 +0800 Subject: [PATCH 11/16] fix(test): fix non deterministic group topn test (#13473) --- e2e_test/streaming/group_top_n/main.slt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/e2e_test/streaming/group_top_n/main.slt b/e2e_test/streaming/group_top_n/main.slt index 16d3187e16d7..620c93503e28 100644 --- a/e2e_test/streaming/group_top_n/main.slt +++ b/e2e_test/streaming/group_top_n/main.slt @@ -14,7 +14,7 @@ where rank <= 3; statement ok create materialized view mv_with_expr_in_window as select x, y from ( - select *, ROW_NUMBER() OVER (PARTITION BY x/2 ORDER BY 6-y) as rank from t + select *, ROW_NUMBER() OVER (PARTITION BY x/2 ORDER BY 6-y, x) as rank from t ) where rank <= 3; @@ -68,7 +68,7 @@ select * from mv; 3 3 query II rowsort -select * from mv_with_expr_in_window; +select * from mv_with_expr_in_window order by x, y; ---- 1 3 1 4 From 942a526c3a79368072849fa6937b4188871b7882 Mon Sep 17 00:00:00 2001 From: Bohan Zhang Date: Thu, 16 Nov 2023 21:13:10 +0800 Subject: [PATCH 12/16] feat: allow overwrite stream_rate_control in with clause (#13009) Signed-off-by: tabVersion --- e2e_test/ddl/throttle.slt | 23 ++++++++ e2e_test/sink/kafka/create_sink.slt | 29 +++++++++++ e2e_test/source/basic/kafka.slt | 13 +++++ src/frontend/src/handler/create_mv.rs | 10 +++- .../src/optimizer/optimizer_context.rs | 14 ++++- .../optimizer/plan_node/stream_fs_fetch.rs | 7 +-- .../src/optimizer/plan_node/stream_source.rs | 7 +-- .../optimizer/plan_node/stream_table_scan.rs | 8 +-- src/frontend/src/utils/mod.rs | 3 +- src/frontend/src/utils/overwrite_options.rs | 52 +++++++++++++++++++ src/stream/src/from_proto/source/fs_fetch.rs | 2 +- .../src/from_proto/source/trad_source.rs | 2 +- src/stream/src/from_proto/stream_scan.rs | 3 +- 13 files changed, 147 insertions(+), 26 deletions(-) create mode 100644 e2e_test/ddl/throttle.slt create mode 100644 src/frontend/src/utils/overwrite_options.rs diff --git a/e2e_test/ddl/throttle.slt b/e2e_test/ddl/throttle.slt new file mode 100644 index 000000000000..6395697f0df7 --- /dev/null +++ b/e2e_test/ddl/throttle.slt @@ -0,0 +1,23 @@ +# streaming_rate_limit also applies to create sink and create source, please refer to +# e2e_test/source/basic/kafka.slt and e2e_test/sink/kafka/create_sink.slt for this part + +statement ok +create table t1 (v1 int); + +# tracked in https://github.com/risingwavelabs/risingwave/issues/13474 +# create with duplicate streaming_rate_limit +# statement error +# create materialized view mv1 with (streaming_rate_limit = 1000, streaming_rate_limit = 2000) as select * from t1; + +# create with unknown fields +statement error unexpected options in WITH clause +create materialized view mv1 with (streaming_rate_limit = 1000, unknown_field = 2000) as select * from t1; + +statement ok +create materialized view mv1 with (streaming_rate_limit = 1000) as select * from t1; + +statement ok +drop materialized view mv1; + +statement ok +drop table t1; diff --git a/e2e_test/sink/kafka/create_sink.slt b/e2e_test/sink/kafka/create_sink.slt index 4aabec688840..6a447bd9fc7c 100644 --- a/e2e_test/sink/kafka/create_sink.slt +++ b/e2e_test/sink/kafka/create_sink.slt @@ -134,9 +134,38 @@ create sink multiple_pk from t_kafka with ( primary_key = 'id,v_varchar' ); +# throttle option +statement ok +create sink multiple_pk_throttle from t_kafka with ( + connector = 'kafka', + properties.bootstrap.server = 'message_queue:29092', + topic = 'test-rw-sink-debezium', + type = 'debezium', + primary_key = 'id,v_varchar', + streaming_rate_limit = 200 +); + +statement ok +create sink multiple_pk_throttle_1 + as select * from t_kafka + with ( + connector = 'kafka', + properties.bootstrap.server = 'message_queue:29092', + topic = 'test-rw-sink-debezium', + type = 'debezium', + primary_key = 'id,v_varchar', + streaming_rate_limit = 200 +); + statement ok drop sink multiple_pk; +statement ok +drop sink multiple_pk_throttle; + +statement ok +drop sink multiple_pk_throttle_1; + statement error Sink primary key column not found: invalid. create sink invalid_pk_column from t_kafka with ( connector = 'kafka', diff --git a/e2e_test/source/basic/kafka.slt b/e2e_test/source/basic/kafka.slt index 941d6ae98c0a..56097de50809 100644 --- a/e2e_test/source/basic/kafka.slt +++ b/e2e_test/source/basic/kafka.slt @@ -372,6 +372,16 @@ create table s28 (id bytea, PRIMARY KEY(_rw_key)) with ( scan.startup.mode = 'earliest' ) FORMAT PLAIN ENCODE BYTES +# throttle option +statement ok +create table s29 (id bytea, PRIMARY KEY(_rw_key)) with ( + connector = 'kafka', + topic = 'kafka_source_format_bytes', + properties.bootstrap.server = 'message_queue:29092', + scan.startup.mode = 'earliest', + streaming_rate_limit = 200 +) FORMAT PLAIN ENCODE BYTES + statement ok CREATE TABLE mongo_customers ( _id BIGINT PRIMARY KEY, @@ -842,6 +852,9 @@ drop table s27 statement ok drop table s28 +statement ok +drop table s29 + statement ok DROP TABLE mongo_customers; diff --git a/src/frontend/src/handler/create_mv.rs b/src/frontend/src/handler/create_mv.rs index aeadf35aa2d8..4c3beaf922ac 100644 --- a/src/frontend/src/handler/create_mv.rs +++ b/src/frontend/src/handler/create_mv.rs @@ -16,7 +16,8 @@ use either::Either; use itertools::Itertools; use pgwire::pg_response::{PgResponse, StatementType}; use risingwave_common::acl::AclMode; -use risingwave_common::error::{ErrorCode, Result}; +use risingwave_common::error::ErrorCode::ProtocolError; +use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_pb::catalog::{CreateType, PbTable}; use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism; use risingwave_sqlparser::ast::{EmitMode, Ident, ObjectName, Query}; @@ -165,6 +166,13 @@ pub async fn handle_create_mv( let (mut table, graph) = { let context = OptimizerContext::from_handler_args(handler_args); + if !context.with_options().is_empty() { + // get other useful fields by `remove`, the logic here is to reject unknown options. + return Err(RwError::from(ProtocolError(format!( + "unexpected options in WITH clause: {:?}", + context.with_options().keys() + )))); + } let has_order_by = !query.order_by.is_empty(); if has_order_by { diff --git a/src/frontend/src/optimizer/optimizer_context.rs b/src/frontend/src/optimizer/optimizer_context.rs index e4b8d3c56681..7f7f867daa3b 100644 --- a/src/frontend/src/optimizer/optimizer_context.rs +++ b/src/frontend/src/optimizer/optimizer_context.rs @@ -24,7 +24,7 @@ use crate::expr::{CorrelatedId, SessionTimezone}; use crate::handler::HandlerArgs; use crate::optimizer::plan_node::PlanNodeId; use crate::session::SessionImpl; -use crate::WithOptions; +use crate::utils::{OverwriteOptions, WithOptions}; const RESERVED_ID_NUM: u16 = 10000; @@ -50,6 +50,9 @@ pub struct OptimizerContext { session_timezone: RefCell, /// Store expr display id. next_expr_display_id: RefCell, + /// Store the configs can be overwritten in with clause + /// if not specified, use the value from session variable. + overwrite_options: OverwriteOptions, } // Still not sure if we need to introduce "on_optimization_finish" or other common callback methods, @@ -71,10 +74,11 @@ impl OptimizerContext { } /// Create a new [`OptimizerContext`] from the given [`HandlerArgs`] and [`ExplainOptions`]. - pub fn new(handler_args: HandlerArgs, explain_options: ExplainOptions) -> Self { + pub fn new(mut handler_args: HandlerArgs, explain_options: ExplainOptions) -> Self { let session_timezone = RefCell::new(SessionTimezone::new( handler_args.session.config().get_timezone().to_owned(), )); + let overwrite_options = OverwriteOptions::new(&mut handler_args); Self { session_ctx: handler_args.session, next_plan_node_id: RefCell::new(RESERVED_ID_NUM.into()), @@ -87,6 +91,7 @@ impl OptimizerContext { with_options: handler_args.with_options, session_timezone, next_expr_display_id: RefCell::new(RESERVED_ID_NUM.into()), + overwrite_options, } } @@ -106,6 +111,7 @@ impl OptimizerContext { with_options: Default::default(), session_timezone: RefCell::new(SessionTimezone::new("UTC".into())), next_expr_display_id: RefCell::new(0), + overwrite_options: OverwriteOptions::default(), } .into() } @@ -189,6 +195,10 @@ impl OptimizerContext { &self.with_options } + pub fn overwrite_options(&self) -> &OverwriteOptions { + &self.overwrite_options + } + pub fn session_ctx(&self) -> &Arc { &self.session_ctx } diff --git a/src/frontend/src/optimizer/plan_node/stream_fs_fetch.rs b/src/frontend/src/optimizer/plan_node/stream_fs_fetch.rs index 2fc1aba8127f..87cbc65b212d 100644 --- a/src/frontend/src/optimizer/plan_node/stream_fs_fetch.rs +++ b/src/frontend/src/optimizer/plan_node/stream_fs_fetch.rs @@ -111,12 +111,7 @@ impl StreamNode for StreamFsFetch { .map(|c| c.to_protobuf()) .collect_vec(), properties: source_catalog.properties.clone().into_iter().collect(), - rate_limit: self - .base - .ctx() - .session_ctx() - .config() - .get_streaming_rate_limit(), + rate_limit: self.base.ctx().overwrite_options().streaming_rate_limit, }); NodeBody::StreamFsFetch(StreamFsFetchNode { node_inner: source_inner, diff --git a/src/frontend/src/optimizer/plan_node/stream_source.rs b/src/frontend/src/optimizer/plan_node/stream_source.rs index 51a764b53fab..1f2cd6fc98ad 100644 --- a/src/frontend/src/optimizer/plan_node/stream_source.rs +++ b/src/frontend/src/optimizer/plan_node/stream_source.rs @@ -87,12 +87,7 @@ impl StreamNode for StreamSource { .map(|c| c.to_protobuf()) .collect_vec(), properties: source_catalog.properties.clone().into_iter().collect(), - rate_limit: self - .base - .ctx() - .session_ctx() - .config() - .get_streaming_rate_limit(), + rate_limit: self.base.ctx().overwrite_options().streaming_rate_limit, }); PbNodeBody::Source(SourceNode { source_inner }) } diff --git a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs index e506c69ffb3b..750548a315e9 100644 --- a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs @@ -283,6 +283,7 @@ impl StreamTableScan { }) .collect_vec(); + // TODO: snapshot read of upstream mview let batch_plan_node = BatchPlanNode { table_desc: Some(self.core.table_desc.to_protobuf()), column_ids: upstream_column_ids.clone(), @@ -301,12 +302,7 @@ impl StreamTableScan { // The table desc used by backfill executor table_desc: Some(self.core.table_desc.to_protobuf()), state_table: Some(catalog), - rate_limit: self - .base - .ctx() - .session_ctx() - .config() - .get_streaming_rate_limit(), + rate_limit: self.base.ctx().overwrite_options().streaming_rate_limit, ..Default::default() }); diff --git a/src/frontend/src/utils/mod.rs b/src/frontend/src/utils/mod.rs index 3951da525bb0..6f105e5960b8 100644 --- a/src/frontend/src/utils/mod.rs +++ b/src/frontend/src/utils/mod.rs @@ -31,8 +31,9 @@ mod index_set; pub use index_set::*; pub(crate) mod group_by; pub mod infer_stmt_row_desc; - +pub mod overwrite_options; pub use group_by::*; +pub use overwrite_options::*; use crate::expr::{Expr, ExprImpl, ExprRewriter, InputRef}; diff --git a/src/frontend/src/utils/overwrite_options.rs b/src/frontend/src/utils/overwrite_options.rs new file mode 100644 index 000000000000..14e838e689d7 --- /dev/null +++ b/src/frontend/src/utils/overwrite_options.rs @@ -0,0 +1,52 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use crate::handler::HandlerArgs; + +#[derive(Debug, Clone, Default)] +pub struct OverwriteOptions { + pub streaming_rate_limit: Option, + // ttl has been deprecated + pub ttl: Option, +} + +impl OverwriteOptions { + const STREAMING_RATE_LIMIT_KEY: &'static str = "streaming_rate_limit"; + const TTL_KEY: &'static str = "ttl"; + + pub fn new(args: &mut HandlerArgs) -> Self { + let streaming_rate_limit = { + if let Some(x) = args + .with_options + .inner_mut() + .remove(Self::STREAMING_RATE_LIMIT_KEY) + { + // FIXME(tabVersion): validate the value + Some(x.parse::().unwrap()) + } else { + args.session.config().get_streaming_rate_limit() + } + }; + let ttl = args + .with_options + .inner_mut() + .remove(Self::TTL_KEY) + // FIXME(tabVersion): validate the value + .map(|x| x.parse::().unwrap()); + Self { + streaming_rate_limit, + ttl, + } + } +} diff --git a/src/stream/src/from_proto/source/fs_fetch.rs b/src/stream/src/from_proto/source/fs_fetch.rs index aad6914d4bce..8c7b79852d6e 100644 --- a/src/stream/src/from_proto/source/fs_fetch.rs +++ b/src/stream/src/from_proto/source/fs_fetch.rs @@ -97,7 +97,7 @@ impl ExecutorBuilder for FsFetchExecutorBuilder { ) .boxed(); - let rate_limit = source.get_rate_limit().cloned().ok(); + let rate_limit = source.rate_limit.map(|x| x as _); Ok(FlowControlExecutor::new(executor, rate_limit).boxed()) } } diff --git a/src/stream/src/from_proto/source/trad_source.rs b/src/stream/src/from_proto/source/trad_source.rs index 3232c49d8cbb..ac303e27ee9b 100644 --- a/src/stream/src/from_proto/source/trad_source.rs +++ b/src/stream/src/from_proto/source/trad_source.rs @@ -194,7 +194,7 @@ impl ExecutorBuilder for SourceExecutorBuilder { } } }; - let rate_limit = source.get_rate_limit().cloned().ok(); + let rate_limit = source.rate_limit.map(|x| x as _); Ok(FlowControlExecutor::new(executor, rate_limit).boxed()) } else { // If there is no external stream source, then no data should be persisted. We pass a diff --git a/src/stream/src/from_proto/stream_scan.rs b/src/stream/src/from_proto/stream_scan.rs index 5b6136fda9ea..059a4c983fe2 100644 --- a/src/stream/src/from_proto/stream_scan.rs +++ b/src/stream/src/from_proto/stream_scan.rs @@ -218,7 +218,6 @@ impl ExecutorBuilder for StreamScanExecutorBuilder { } StreamScanType::Unspecified => unreachable!(), }; - let rate_limit = node.get_rate_limit().cloned().ok(); - Ok(FlowControlExecutor::new(executor, rate_limit).boxed()) + Ok(FlowControlExecutor::new(executor, node.rate_limit.map(|x| x as _)).boxed()) } } From 7ebab64057673e92b22b149a1d7de536f655cbe1 Mon Sep 17 00:00:00 2001 From: Dylan Date: Thu, 16 Nov 2023 23:38:16 +0800 Subject: [PATCH 13/16] feat(optimizer): avoid predicate pushdown for batch queries if unnecessary (#13470) Co-authored-by: stonepage <40830455+st1page@users.noreply.github.com> Co-authored-by: st1page <1245835950@qq.com> --- .../tests/testdata/output/cse_expr.yaml | 2 +- .../tests/testdata/output/explain.yaml | 26 +--------------- .../tests/testdata/output/expr.yaml | 4 +-- .../tests/testdata/output/nexmark_source.yaml | 19 +++++++----- .../testdata/output/nexmark_watermark.yaml | 30 +++++++++---------- .../tests/testdata/output/share.yaml | 6 ++-- .../src/optimizer/logical_optimization.rs | 19 ++++++++++-- .../src/optimizer/optimizer_context.rs | 12 ++++++++ 8 files changed, 63 insertions(+), 55 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/output/cse_expr.yaml b/src/frontend/planner_test/tests/testdata/output/cse_expr.yaml index fe41cb83794e..4682f091ad1b 100644 --- a/src/frontend/planner_test/tests/testdata/output/cse_expr.yaml +++ b/src/frontend/planner_test/tests/testdata/output/cse_expr.yaml @@ -84,5 +84,5 @@ sql: | with t(v, arr) as (select 1, array[2, 3]) select v < all(arr), v < some(arr) from t; batch_plan: |- - BatchProject { exprs: [All((1:Int32 < $expr10015)) as $expr1, Some((1:Int32 < $expr10015)) as $expr2] } + BatchProject { exprs: [All((1:Int32 < $expr10009)) as $expr1, Some((1:Int32 < $expr10009)) as $expr2] } └─BatchValues { rows: [[1:Int32, ARRAY[2, 3]:List(Int32)]] } diff --git a/src/frontend/planner_test/tests/testdata/output/explain.yaml b/src/frontend/planner_test/tests/testdata/output/explain.yaml index 0faca9c10291..e44431ffcd3d 100644 --- a/src/frontend/planner_test/tests/testdata/output/explain.yaml +++ b/src/frontend/planner_test/tests/testdata/output/explain.yaml @@ -16,16 +16,6 @@ LogicalProject { exprs: [1:Int32] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - Predicate Push Down: - - LogicalProject { exprs: [1:Int32] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - - Predicate Push Down: - - LogicalProject { exprs: [1:Int32] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - Convert Over Window: apply TrivialProjectToValuesRule 1 time(s) @@ -65,7 +55,7 @@ "stages": { "0": { "root": { - "plan_node_id": 10020, + "plan_node_id": 10016, "plan_node_type": "BatchValues", "schema": [ { @@ -121,13 +111,6 @@ Predicate Push Down: - LogicalProject { exprs: [t1.v1, t2.v2] } - └─LogicalJoin { type: Inner, on: (t1.v1 = t2.v2) } - ├─LogicalScan { table: t1, columns: [v1, _row_id] } - └─LogicalScan { table: t2, columns: [v2, _row_id] } - - Predicate Push Down: - LogicalProject { exprs: [t1.v1, t2.v2] } └─LogicalJoin { type: Inner, on: (t1.v1 = t2.v2) } ├─LogicalScan { table: t1, columns: [v1, _row_id] } @@ -151,13 +134,6 @@ Prune Columns: - LogicalProject { exprs: [t1.v1, t2.v2] } - └─LogicalJoin { type: Inner, on: (t1.v1 = t2.v2) } - ├─LogicalScan { table: t1, columns: [v1] } - └─LogicalScan { table: t2, columns: [v2] } - - Predicate Push Down: - LogicalProject { exprs: [t1.v1, t2.v2] } └─LogicalJoin { type: Inner, on: (t1.v1 = t2.v2) } ├─LogicalScan { table: t1, columns: [v1] } diff --git a/src/frontend/planner_test/tests/testdata/output/expr.yaml b/src/frontend/planner_test/tests/testdata/output/expr.yaml index 7f117e64be4e..d0ab974ff81b 100644 --- a/src/frontend/planner_test/tests/testdata/output/expr.yaml +++ b/src/frontend/planner_test/tests/testdata/output/expr.yaml @@ -450,7 +450,7 @@ └─LogicalProject { exprs: [Array(1:Int32) as $expr1] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: |- - BatchProject { exprs: [Some((1:Int32 < ArrayCat($expr10013, ARRAY[2]:List(Int32)))) as $expr1] } + BatchProject { exprs: [Some((1:Int32 < ArrayCat($expr10011, ARRAY[2]:List(Int32)))) as $expr1] } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchValues { rows: [[]] } └─BatchValues { rows: [[ARRAY[1]:List(Int32)]] } @@ -473,7 +473,7 @@ └─LogicalProject { exprs: [Array(1:Int32) as $expr1] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: |- - BatchProject { exprs: [All((1:Int32 < ArrayCat($expr10013, ARRAY[2]:List(Int32)))) as $expr1] } + BatchProject { exprs: [All((1:Int32 < ArrayCat($expr10011, ARRAY[2]:List(Int32)))) as $expr1] } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchValues { rows: [[]] } └─BatchValues { rows: [[ARRAY[1]:List(Int32)]] } diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml index 6a7d3c0fa04a..654bc8b0f33e 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml @@ -116,8 +116,8 @@ sql: SELECT auction, price FROM bid WHERE auction = 1007 OR auction = 1020 OR auction = 2001 OR auction = 2019 OR auction = 2087; batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [auction, price] } - └─BatchFilter { predicate: (((((auction = 1007:Int32) OR (auction = 1020:Int32)) OR (auction = 2001:Int32)) OR (auction = 2019:Int32)) OR (auction = 2087:Int32)) } + └─BatchFilter { predicate: (((((auction = 1007:Int32) OR (auction = 1020:Int32)) OR (auction = 2001:Int32)) OR (auction = 2019:Int32)) OR (auction = 2087:Int32)) } + └─BatchProject { exprs: [auction, price] } └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [auction, price, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } @@ -152,10 +152,12 @@ └─BatchHashJoin { type: Inner, predicate: seller = id, output: [name, city, state, id] } ├─BatchExchange { order: [], dist: HashShard(seller) } │ └─BatchFilter { predicate: (category = 10:Int32) } - │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + │ └─BatchProject { exprs: [id, seller, category] } + │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } └─BatchExchange { order: [], dist: HashShard(id) } └─BatchFilter { predicate: (((state = 'or':Varchar) OR (state = 'id':Varchar)) OR (state = 'ca':Varchar)) } - └─BatchSource { source: person, columns: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id], filter: (None, None) } + └─BatchProject { exprs: [id, name, city, state] } + └─BatchSource { source: person, columns: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [name, city, state, id, _row_id(hidden), seller(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, seller], pk_columns: [_row_id, _row_id#1, seller], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(_row_id, seller, _row_id) } @@ -1015,7 +1017,8 @@ BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [auction, bidder, (0.908:Decimal * price::Decimal) as $expr1, Case(((Extract('HOUR':Varchar, date_time) >= 8:Decimal) AND (Extract('HOUR':Varchar, date_time) <= 18:Decimal)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, date_time) <= 6:Decimal) OR (Extract('HOUR':Varchar, date_time) >= 20:Decimal)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr2, date_time, extra] } └─BatchFilter { predicate: ((0.908:Decimal * price::Decimal) > 1000000:Decimal) AND ((0.908:Decimal * price::Decimal) < 50000000:Decimal) } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchProject { exprs: [auction, bidder, price, date_time, extra] } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, bidtimetype, date_time, extra, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } └─StreamProject { exprs: [auction, bidder, (0.908:Decimal * price::Decimal) as $expr1, Case(((Extract('HOUR':Varchar, date_time) >= 8:Decimal) AND (Extract('HOUR':Varchar, date_time) <= 18:Decimal)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, date_time) <= 6:Decimal) OR (Extract('HOUR':Varchar, date_time) >= 20:Decimal)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr2, date_time, extra, _row_id] } @@ -1634,7 +1637,8 @@ │ └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } └─BatchExchange { order: [], dist: HashShard(id) } └─BatchFilter { predicate: (category = 10:Int32) } - └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + └─BatchProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category] } + └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, auction], pk_columns: [_row_id, _row_id#1, auction], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(auction, _row_id, _row_id) } @@ -1705,7 +1709,8 @@ BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [auction, bidder, price, channel, Case((Lower(channel) = 'apple':Varchar), '0':Varchar, (Lower(channel) = 'google':Varchar), '1':Varchar, (Lower(channel) = 'facebook':Varchar), '2':Varchar, (Lower(channel) = 'baidu':Varchar), '3':Varchar, ArrayAccess(RegexpMatch(url, '(&|^)channel_id=([^&]*)':Varchar), 2:Int32)) as $expr1] } └─BatchFilter { predicate: (IsNotNull(ArrayAccess(RegexpMatch(url, '(&|^)channel_id=([^&]*)':Varchar), 2:Int32)) OR In(Lower(channel), 'apple':Varchar, 'google':Varchar, 'facebook':Varchar, 'baidu':Varchar)) } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchProject { exprs: [auction, bidder, price, channel, url] } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, channel_id, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } └─StreamProject { exprs: [auction, bidder, price, channel, Case((Lower(channel) = 'apple':Varchar), '0':Varchar, (Lower(channel) = 'google':Varchar), '1':Varchar, (Lower(channel) = 'facebook':Varchar), '2':Varchar, (Lower(channel) = 'baidu':Varchar), '3':Varchar, ArrayAccess(RegexpMatch(url, '(&|^)channel_id=([^&]*)':Varchar), 2:Int32)) as $expr1, _row_id] } diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml index 7f41cfffe04f..53df7f872633 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml @@ -118,16 +118,16 @@ A.category = 10 and (P.state = 'or' OR P.state = 'id' OR P.state = 'ca'); batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchHashJoin { type: Inner, predicate: $expr3 = $expr5, output: [$expr6, $expr7, $expr8, $expr2] } - ├─BatchExchange { order: [], dist: HashShard($expr3) } - │ └─BatchProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 7:Int32) as $expr3] } + └─BatchHashJoin { type: Inner, predicate: $expr2 = $expr3, output: [$expr4, $expr5, $expr6, $expr1] } + ├─BatchExchange { order: [], dist: HashShard($expr2) } + │ └─BatchProject { exprs: [Field(auction, 0:Int32) as $expr1, Field(auction, 7:Int32) as $expr2] } │ └─BatchFilter { predicate: (Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32) } - │ └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } + │ └─BatchProject { exprs: [event_type, auction] } │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } - └─BatchExchange { order: [], dist: HashShard($expr5) } - └─BatchProject { exprs: [Field(person, 0:Int32) as $expr5, Field(person, 1:Int32) as $expr6, Field(person, 4:Int32) as $expr7, Field(person, 5:Int32) as $expr8] } + └─BatchExchange { order: [], dist: HashShard($expr3) } + └─BatchProject { exprs: [Field(person, 0:Int32) as $expr3, Field(person, 1:Int32) as $expr4, Field(person, 4:Int32) as $expr5, Field(person, 5:Int32) as $expr6] } └─BatchFilter { predicate: (((Field(person, 5:Int32) = 'or':Varchar) OR (Field(person, 5:Int32) = 'id':Varchar)) OR (Field(person, 5:Int32) = 'ca':Varchar)) AND (event_type = 0:Int32) } - └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr4, _row_id] } + └─BatchProject { exprs: [event_type, person] } └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [name, city, state, id, _row_id(hidden), $expr3(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, $expr3], pk_columns: [_row_id, _row_id#1, $expr3], pk_conflict: NoCheck } @@ -1719,12 +1719,12 @@ ├─BatchExchange { order: [], dist: HashShard($expr2) } │ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, Field(bid, 3:Int32) as $expr5, Field(bid, 4:Int32) as $expr6, $expr1] } │ └─BatchFilter { predicate: (event_type = 2:Int32) } - │ └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } + │ └─BatchProject { exprs: [event_type, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1] } │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } └─BatchExchange { order: [], dist: HashShard($expr8) } └─BatchProject { exprs: [Field(auction, 0:Int32) as $expr8, Field(auction, 1:Int32) as $expr9, Field(auction, 2:Int32) as $expr10, Field(auction, 3:Int32) as $expr11, Field(auction, 4:Int32) as $expr12, $expr7, Field(auction, 6:Int32) as $expr13, Field(auction, 7:Int32) as $expr14, Field(auction, 8:Int32) as $expr15] } └─BatchFilter { predicate: (Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32) } - └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr7, _row_id] } + └─BatchProject { exprs: [event_type, auction, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr7] } └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, auction], pk_columns: [_row_id, _row_id#1, auction], pk_conflict: NoCheck } @@ -1819,10 +1819,10 @@ lower(channel) in ('apple', 'google', 'facebook', 'baidu'); batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [$expr2, $expr3, $expr4, $expr5, Case((Lower($expr5) = 'apple':Varchar), '0':Varchar, (Lower($expr5) = 'google':Varchar), '1':Varchar, (Lower($expr5) = 'facebook':Varchar), '2':Varchar, (Lower($expr5) = 'baidu':Varchar), '3':Varchar, ArrayAccess(RegexpMatch($expr6, '(&|^)channel_id=([^&]*)':Varchar), 2:Int32)) as $expr7] } - └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, Field(bid, 3:Int32) as $expr5, Field(bid, 4:Int32) as $expr6] } + └─BatchProject { exprs: [$expr1, $expr2, $expr3, $expr4, Case((Lower($expr4) = 'apple':Varchar), '0':Varchar, (Lower($expr4) = 'google':Varchar), '1':Varchar, (Lower($expr4) = 'facebook':Varchar), '2':Varchar, (Lower($expr4) = 'baidu':Varchar), '3':Varchar, ArrayAccess(RegexpMatch($expr5, '(&|^)channel_id=([^&]*)':Varchar), 2:Int32)) as $expr6] } + └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr1, Field(bid, 1:Int32) as $expr2, Field(bid, 2:Int32) as $expr3, Field(bid, 3:Int32) as $expr4, Field(bid, 4:Int32) as $expr5] } └─BatchFilter { predicate: (IsNotNull(ArrayAccess(RegexpMatch(Field(bid, 4:Int32), '(&|^)channel_id=([^&]*)':Varchar), 2:Int32)) OR In(Lower(Field(bid, 3:Int32)), 'apple':Varchar, 'google':Varchar, 'facebook':Varchar, 'baidu':Varchar)) AND (event_type = 2:Int32) } - └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } + └─BatchProject { exprs: [event_type, bid] } └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, channel_id, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } @@ -1861,10 +1861,10 @@ SPLIT_PART(url, '/', 6) as dir3 FROM bid; batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [$expr2, $expr3, $expr4, $expr5, SplitPart($expr6, '/':Varchar, 4:Int32) as $expr7, SplitPart($expr6, '/':Varchar, 5:Int32) as $expr8, SplitPart($expr6, '/':Varchar, 6:Int32) as $expr9] } - └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, Field(bid, 3:Int32) as $expr5, Field(bid, 4:Int32) as $expr6] } + └─BatchProject { exprs: [$expr1, $expr2, $expr3, $expr4, SplitPart($expr5, '/':Varchar, 4:Int32) as $expr6, SplitPart($expr5, '/':Varchar, 5:Int32) as $expr7, SplitPart($expr5, '/':Varchar, 6:Int32) as $expr8] } + └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr1, Field(bid, 1:Int32) as $expr2, Field(bid, 2:Int32) as $expr3, Field(bid, 3:Int32) as $expr4, Field(bid, 4:Int32) as $expr5] } └─BatchFilter { predicate: (event_type = 2:Int32) } - └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } + └─BatchProject { exprs: [event_type, bid] } └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, dir1, dir2, dir3, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } diff --git a/src/frontend/planner_test/tests/testdata/output/share.yaml b/src/frontend/planner_test/tests/testdata/output/share.yaml index d446b23fab88..799eadf1cdf8 100644 --- a/src/frontend/planner_test/tests/testdata/output/share.yaml +++ b/src/frontend/planner_test/tests/testdata/output/share.yaml @@ -337,10 +337,12 @@ └─BatchHashJoin { type: Inner, predicate: id = id, output: [id, id, date_time, date_time] } ├─BatchExchange { order: [], dist: HashShard(id) } │ └─BatchFilter { predicate: (initial_bid = 1:Int32) AND (AtTimeZone(date_time, 'UTC':Varchar) > '2021-03-31 23:59:59+00:00':Timestamptz) } - │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + │ └─BatchProject { exprs: [id, initial_bid, date_time] } + │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } └─BatchExchange { order: [], dist: HashShard(id) } └─BatchFilter { predicate: (initial_bid = 2:Int32) } - └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + └─BatchProject { exprs: [id, initial_bid, date_time] } + └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [a_id, b_id, a_ts, b_ts, _row_id(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, a_id], pk_columns: [_row_id, _row_id#1, a_id], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(id, _row_id, _row_id) } diff --git a/src/frontend/src/optimizer/logical_optimization.rs b/src/frontend/src/optimizer/logical_optimization.rs index 887f393e81c6..cfae8a04504a 100644 --- a/src/frontend/src/optimizer/logical_optimization.rs +++ b/src/frontend/src/optimizer/logical_optimization.rs @@ -47,6 +47,7 @@ impl PlanRef { ctx.trace(format!("{}", stats)); ctx.trace(plan.explain_to_string()); } + ctx.add_rule_applied(stats.total_applied()); plan } @@ -71,6 +72,7 @@ impl PlanRef { ctx.trace(format!("{}", stats)); ctx.trace(output_plan.explain_to_string()); } + ctx.add_rule_applied(stats.total_applied()); if !stats.has_applied_rule() { return output_plan; @@ -638,6 +640,7 @@ impl LogicalOptimizer { plan = Self::subquery_unnesting(plan, false, explain_trace, &ctx)?; // Predicate Push-down + let mut last_total_rule_applied_before_predicate_pushdown = ctx.total_rule_applied(); plan = Self::predicate_pushdown(plan, explain_trace, &ctx); if plan.ctx().session_ctx().config().get_enable_join_ordering() { @@ -652,7 +655,10 @@ impl LogicalOptimizer { // Predicate Push-down: apply filter pushdown rules again since we pullup all join // conditions into a filter above the multijoin. - plan = Self::predicate_pushdown(plan, explain_trace, &ctx); + if last_total_rule_applied_before_predicate_pushdown != ctx.total_rule_applied() { + last_total_rule_applied_before_predicate_pushdown = ctx.total_rule_applied(); + plan = Self::predicate_pushdown(plan, explain_trace, &ctx); + } // Push down the calculation of inputs of join's condition. plan = plan.optimize_by_rules(&PUSH_CALC_OF_JOIN); @@ -660,7 +666,10 @@ impl LogicalOptimizer { plan = plan.optimize_by_rules(&SPLIT_OVER_WINDOW); // Must push down predicates again after split over window so that OverWindow can be // optimized to TopN. - plan = Self::predicate_pushdown(plan, explain_trace, &ctx); + if last_total_rule_applied_before_predicate_pushdown != ctx.total_rule_applied() { + last_total_rule_applied_before_predicate_pushdown = ctx.total_rule_applied(); + plan = Self::predicate_pushdown(plan, explain_trace, &ctx); + } plan = plan.optimize_by_rules(&CONVERT_OVER_WINDOW); plan = plan.optimize_by_rules(&MERGE_OVER_WINDOW); @@ -673,7 +682,11 @@ impl LogicalOptimizer { // Do a final column pruning and predicate pushing down to clean up the plan. plan = Self::column_pruning(plan, explain_trace, &ctx); - plan = Self::predicate_pushdown(plan, explain_trace, &ctx); + if last_total_rule_applied_before_predicate_pushdown != ctx.total_rule_applied() { + #[allow(unused_assignments)] + last_total_rule_applied_before_predicate_pushdown = ctx.total_rule_applied(); + plan = Self::predicate_pushdown(plan, explain_trace, &ctx); + } plan = plan.optimize_by_rules(&PROJECT_REMOVE); diff --git a/src/frontend/src/optimizer/optimizer_context.rs b/src/frontend/src/optimizer/optimizer_context.rs index 7f7f867daa3b..abf86b5fa5b7 100644 --- a/src/frontend/src/optimizer/optimizer_context.rs +++ b/src/frontend/src/optimizer/optimizer_context.rs @@ -50,6 +50,8 @@ pub struct OptimizerContext { session_timezone: RefCell, /// Store expr display id. next_expr_display_id: RefCell, + /// Total number of optimization rules have been applied. + total_rule_applied: RefCell, /// Store the configs can be overwritten in with clause /// if not specified, use the value from session variable. overwrite_options: OverwriteOptions, @@ -91,6 +93,7 @@ impl OptimizerContext { with_options: handler_args.with_options, session_timezone, next_expr_display_id: RefCell::new(RESERVED_ID_NUM.into()), + total_rule_applied: RefCell::new(0), overwrite_options, } } @@ -111,6 +114,7 @@ impl OptimizerContext { with_options: Default::default(), session_timezone: RefCell::new(SessionTimezone::new("UTC".into())), next_expr_display_id: RefCell::new(0), + total_rule_applied: RefCell::new(0), overwrite_options: OverwriteOptions::default(), } .into() @@ -147,6 +151,14 @@ impl OptimizerContext { *self.next_correlated_id.borrow() } + pub fn add_rule_applied(&self, num: usize) { + *self.total_rule_applied.borrow_mut() += num; + } + + pub fn total_rule_applied(&self) -> usize { + *self.total_rule_applied.borrow() + } + pub fn is_explain_verbose(&self) -> bool { self.explain_options.verbose } From 483347725eacbe39a9a60b0817ac145aaf205f80 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 16 Nov 2023 19:21:47 +0000 Subject: [PATCH 14/16] chore(deps): Bump itertools from 0.11.0 to 0.12.0 (#13444) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: dependabot[bot] Co-authored-by: TennyZhuang --- Cargo.lock | 72 +++++++++++++++------------- src/batch/Cargo.toml | 2 +- src/bench/Cargo.toml | 2 +- src/common/Cargo.toml | 2 +- src/compute/Cargo.toml | 2 +- src/connector/Cargo.toml | 2 +- src/ctl/Cargo.toml | 2 +- src/expr/core/Cargo.toml | 2 +- src/expr/impl/Cargo.toml | 2 +- src/expr/macro/Cargo.toml | 2 +- src/frontend/Cargo.toml | 2 +- src/frontend/planner_test/Cargo.toml | 2 +- src/jni_core/Cargo.toml | 2 +- src/meta/Cargo.toml | 2 +- src/meta/node/Cargo.toml | 2 +- src/meta/service/Cargo.toml | 2 +- src/object_store/Cargo.toml | 2 +- src/risedevtool/Cargo.toml | 2 +- src/risedevtool/config/Cargo.toml | 2 +- src/rpc_client/Cargo.toml | 2 +- src/source/Cargo.toml | 2 +- src/sqlparser/Cargo.toml | 2 +- src/storage/Cargo.toml | 2 +- src/storage/backup/Cargo.toml | 2 +- src/storage/hummock_sdk/Cargo.toml | 2 +- src/storage/hummock_test/Cargo.toml | 2 +- src/stream/Cargo.toml | 2 +- src/tests/simulation/Cargo.toml | 2 +- src/tests/sqlsmith/Cargo.toml | 2 +- src/utils/pgwire/Cargo.toml | 2 +- src/workspace-hack/Cargo.toml | 6 +-- 31 files changed, 71 insertions(+), 65 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 6be570b2eabd..b44d512affc5 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4250,6 +4250,15 @@ dependencies = [ "either", ] +[[package]] +name = "itertools" +version = "0.12.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "25db6b064527c5d482d0423354fcd07a89a2dfe07b67892e62411946db7f07b0" +dependencies = [ + "either", +] + [[package]] name = "itoa" version = "1.0.9" @@ -6024,7 +6033,7 @@ dependencies = [ "byteorder", "bytes", "futures", - "itertools 0.11.0", + "itertools 0.12.0", "madsim-tokio", "openssl", "panic-message", @@ -6560,7 +6569,7 @@ checksum = "8bdf592881d821b83d471f8af290226c8d51402259e9bb5be7f9f8bdebbb11ac" dependencies = [ "bytes", "heck 0.4.1", - "itertools 0.11.0", + "itertools 0.10.5", "log", "multimap 0.8.3", "once_cell", @@ -6594,7 +6603,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "265baba7fabd416cf5078179f7d2cbeca4ce7a9041111900675ea7c4cb8a4c32" dependencies = [ "anyhow", - "itertools 0.11.0", + "itertools 0.10.5", "proc-macro2", "quote", "syn 2.0.37", @@ -7163,7 +7172,7 @@ dependencies = [ "glob", "google-cloud-pubsub", "indicatif", - "itertools 0.11.0", + "itertools 0.12.0", "madsim-rdkafka", "madsim-tokio", "redis", @@ -7190,7 +7199,7 @@ dependencies = [ "dialoguer", "enum-iterator", "fs-err", - "itertools 0.11.0", + "itertools 0.12.0", ] [[package]] @@ -7201,7 +7210,7 @@ dependencies = [ "async-trait", "bincode 1.3.3", "bytes", - "itertools 0.11.0", + "itertools 0.12.0", "parking_lot 0.12.1", "prost 0.12.1", "risingwave_common", @@ -7230,7 +7239,7 @@ dependencies = [ "futures-util", "hashbrown 0.14.0", "hytra", - "itertools 0.11.0", + "itertools 0.12.0", "madsim-tokio", "madsim-tonic", "parking_lot 0.12.1", @@ -7273,7 +7282,7 @@ dependencies = [ "clap", "futures", "hdrhistogram", - "itertools 0.11.0", + "itertools 0.12.0", "libc", "madsim-tokio", "nix 0.27.1", @@ -7384,7 +7393,7 @@ dependencies = [ "humantime", "hyper", "hytra", - "itertools 0.11.0", + "itertools 0.12.0", "itoa", "jsonbb", "libc", @@ -7548,7 +7557,7 @@ dependencies = [ "futures", "futures-async-stream", "hyper", - "itertools 0.11.0", + "itertools 0.12.0", "madsim-tokio", "madsim-tonic", "maplit", @@ -7618,7 +7627,7 @@ dependencies = [ "hyper-tls", "icelake", "indexmap 1.9.3", - "itertools 0.11.0", + "itertools 0.12.0", "jni", "jsonschema-transpiler", "madsim-rdkafka", @@ -7689,7 +7698,7 @@ dependencies = [ "comfy-table", "futures", "inquire", - "itertools 0.11.0", + "itertools 0.12.0", "madsim-etcd-client", "madsim-tokio", "regex", @@ -7762,7 +7771,7 @@ dependencies = [ "expect-test", "futures-async-stream", "futures-util", - "itertools 0.11.0", + "itertools 0.12.0", "madsim-tokio", "num-traits", "parse-display", @@ -7793,7 +7802,7 @@ dependencies = [ "futures-async-stream", "futures-util", "hex", - "itertools 0.11.0", + "itertools 0.12.0", "jsonbb", "madsim-tokio", "md5", @@ -7817,7 +7826,7 @@ dependencies = [ name = "risingwave_expr_macro" version = "0.1.0" dependencies = [ - "itertools 0.11.0", + "itertools 0.12.0", "proc-macro2", "quote", "syn 2.0.37", @@ -7848,7 +7857,7 @@ dependencies = [ "futures", "futures-async-stream", "iana-time-zone", - "itertools 0.11.0", + "itertools 0.12.0", "madsim-tokio", "madsim-tonic", "maplit", @@ -7897,7 +7906,7 @@ version = "1.3.0-alpha" dependencies = [ "bytes", "hex", - "itertools 0.11.0", + "itertools 0.12.0", "parse-display", "risingwave_common", "risingwave_pb", @@ -7917,7 +7926,7 @@ dependencies = [ "fail", "futures", "futures-async-stream", - "itertools 0.11.0", + "itertools 0.12.0", "madsim-tokio", "parking_lot 0.12.1", "rand", @@ -7982,7 +7991,7 @@ dependencies = [ "cfg-or-panic", "expect-test", "futures", - "itertools 0.11.0", + "itertools 0.12.0", "jni", "madsim-tokio", "paste", @@ -8022,7 +8031,7 @@ dependencies = [ "futures", "hex", "hyper", - "itertools 0.11.0", + "itertools 0.12.0", "madsim-etcd-client", "madsim-tokio", "madsim-tonic", @@ -8092,7 +8101,7 @@ dependencies = [ "clap", "either", "futures", - "itertools 0.11.0", + "itertools 0.12.0", "madsim-etcd-client", "madsim-tokio", "madsim-tonic", @@ -8120,7 +8129,7 @@ dependencies = [ "async-trait", "either", "futures", - "itertools 0.11.0", + "itertools 0.12.0", "madsim-tokio", "madsim-tonic", "rand", @@ -8156,7 +8165,7 @@ dependencies = [ "hyper", "hyper-rustls", "hyper-tls", - "itertools 0.11.0", + "itertools 0.12.0", "madsim-aws-sdk-s3", "madsim-tokio", "opendal", @@ -8194,7 +8203,7 @@ version = "1.3.0-alpha" dependencies = [ "anyhow", "expect-test", - "itertools 0.11.0", + "itertools 0.12.0", "libtest-mimic", "madsim-tokio", "paste", @@ -8234,7 +8243,7 @@ dependencies = [ "either", "futures", "hyper", - "itertools 0.11.0", + "itertools 0.12.0", "lru 0.10.1", "madsim-tokio", "madsim-tonic", @@ -8291,7 +8300,7 @@ dependencies = [ "console", "futures", "glob", - "itertools 0.11.0", + "itertools 0.12.0", "lru 0.7.6", "madsim", "madsim-aws-sdk-s3", @@ -8338,7 +8347,7 @@ dependencies = [ "criterion", "futures", "futures-async-stream", - "itertools 0.11.0", + "itertools 0.12.0", "madsim-tokio", "parking_lot 0.12.1", "paste", @@ -8355,7 +8364,7 @@ dependencies = [ name = "risingwave_sqlparser" version = "1.3.0-alpha" dependencies = [ - "itertools 0.11.0", + "itertools 0.12.0", "matches", "serde", "tracing", @@ -8386,7 +8395,7 @@ dependencies = [ "chrono", "clap", "expect-test", - "itertools 0.11.0", + "itertools 0.12.0", "libtest-mimic", "madsim-tokio", "rand", @@ -8448,7 +8457,7 @@ dependencies = [ "futures", "futures-async-stream", "hex", - "itertools 0.11.0", + "itertools 0.12.0", "libc", "lz4", "mach2", @@ -8511,7 +8520,7 @@ dependencies = [ "futures-async-stream", "governor", "hytra", - "itertools 0.11.0", + "itertools 0.12.0", "local_stats_alloc", "lru 0.7.6", "madsim-tokio", @@ -11420,7 +11429,6 @@ dependencies = [ "hyper", "indexmap 1.9.3", "itertools 0.10.5", - "itertools 0.11.0", "jni", "lazy_static", "lexical-core", diff --git a/src/batch/Cargo.toml b/src/batch/Cargo.toml index 60141e1638c2..c91023f5b1b2 100644 --- a/src/batch/Cargo.toml +++ b/src/batch/Cargo.toml @@ -24,7 +24,7 @@ futures-async-stream = { workspace = true } futures-util = "0.3" hashbrown = { workspace = true } hytra = "0.1.2" -itertools = "0.11" +itertools = "0.12" parking_lot = { version = "0.12", features = ["arc_lock"] } paste = "1" prometheus = { version = "0.13", features = ["process"] } diff --git a/src/bench/Cargo.toml b/src/bench/Cargo.toml index 5227c39272c3..e5f6d8e7ede1 100644 --- a/src/bench/Cargo.toml +++ b/src/bench/Cargo.toml @@ -19,7 +19,7 @@ bytesize = { version = "1", features = ["serde"] } clap = { version = "4", features = ["derive"] } futures = { version = "0.3", default-features = false, features = ["alloc"] } hdrhistogram = "7" -itertools = "0.11" +itertools = "0.12" libc = "0.2" opentelemetry = { version = "0.20", default-features = false, features = ["rt-tokio"], optional = true } parking_lot = "0.12" diff --git a/src/common/Cargo.toml b/src/common/Cargo.toml index 27b3a4249e3b..921c02ee6ae4 100644 --- a/src/common/Cargo.toml +++ b/src/common/Cargo.toml @@ -48,7 +48,7 @@ http = "0.2" humantime = "2.1" hyper = "0.14" hytra = { workspace = true } -itertools = "0.11" +itertools = "0.12" itoa = "1.0" jsonbb = "0.1.2" lru = { git = "https://github.com/risingwavelabs/lru-rs.git", rev = "cb2d7c7" } diff --git a/src/compute/Cargo.toml b/src/compute/Cargo.toml index 5dc1b90f14ba..54c8b6896f0a 100644 --- a/src/compute/Cargo.toml +++ b/src/compute/Cargo.toml @@ -23,7 +23,7 @@ either = "1" futures = { version = "0.3", default-features = false, features = ["alloc"] } futures-async-stream = { workspace = true } hyper = "0.14" -itertools = "0.11" +itertools = "0.12" maplit = "1.0.2" pprof = { version = "0.13", features = ["flamegraph"] } prometheus = { version = "0.13" } diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index d336b482d4f9..db1cae16bb67 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -65,7 +65,7 @@ hyper = { version = "0.14", features = [ hyper-tls = "0.5" icelake = { workspace = true } indexmap = { version = "1.9.3", features = ["serde"] } -itertools = "0.11" +itertools = "0.12" jni = { version = "0.21.1", features = ["invocation"] } jst = { package = 'jsonschema-transpiler', git = "https://github.com/mozilla/jsonschema-transpiler", rev = "c1a89d720d118843d8bcca51084deb0ed223e4b4" } maplit = "1.0.2" diff --git a/src/ctl/Cargo.toml b/src/ctl/Cargo.toml index f8bd5d9d4d27..0097f5624f70 100644 --- a/src/ctl/Cargo.toml +++ b/src/ctl/Cargo.toml @@ -22,7 +22,7 @@ comfy-table = "7" etcd-client = { workspace = true } futures = { version = "0.3", default-features = false, features = ["alloc"] } inquire = "0.6.2" -itertools = "0.11" +itertools = "0.12" regex = "1.10.0" risingwave_common = { workspace = true } risingwave_connector = { workspace = true } diff --git a/src/expr/core/Cargo.toml b/src/expr/core/Cargo.toml index ab8dd697e220..db3795b8a20e 100644 --- a/src/expr/core/Cargo.toml +++ b/src/expr/core/Cargo.toml @@ -34,7 +34,7 @@ either = "1" enum-as-inner = "0.6" futures-async-stream = { workspace = true } futures-util = "0.3" -itertools = "0.11" +itertools = "0.12" num-traits = "0.2" parse-display = "0.8" paste = "1" diff --git a/src/expr/impl/Cargo.toml b/src/expr/impl/Cargo.toml index dbc3e60c880e..ee6d52977cf6 100644 --- a/src/expr/impl/Cargo.toml +++ b/src/expr/impl/Cargo.toml @@ -28,7 +28,7 @@ fancy-regex = "0.12" futures-async-stream = { workspace = true } futures-util = "0.3" hex = "0.4" -itertools = "0.11" +itertools = "0.12" jsonbb = "0.1.2" md5 = "0.7" num-traits = "0.2" diff --git a/src/expr/macro/Cargo.toml b/src/expr/macro/Cargo.toml index bf761b142061..9b388920d2eb 100644 --- a/src/expr/macro/Cargo.toml +++ b/src/expr/macro/Cargo.toml @@ -8,7 +8,7 @@ edition = "2021" proc-macro = true [dependencies] -itertools = "0.11" +itertools = "0.12" proc-macro2 = "1" quote = "1" syn = { version = "2", features = ["full", "extra-traits"] } diff --git a/src/frontend/Cargo.toml b/src/frontend/Cargo.toml index 73fcb205fafc..ada566ad8179 100644 --- a/src/frontend/Cargo.toml +++ b/src/frontend/Cargo.toml @@ -35,7 +35,7 @@ fixedbitset = "0.4.2" futures = { version = "0.3", default-features = false, features = ["alloc"] } futures-async-stream = { workspace = true } iana-time-zone = "0.1" -itertools = "0.11" +itertools = "0.12" maplit = "1" md5 = "0.7.0" num-integer = "0.1" diff --git a/src/frontend/planner_test/Cargo.toml b/src/frontend/planner_test/Cargo.toml index 47d6ba3a6828..4de959bf16ce 100644 --- a/src/frontend/planner_test/Cargo.toml +++ b/src/frontend/planner_test/Cargo.toml @@ -16,7 +16,7 @@ normal = ["workspace-hack"] [dependencies] anyhow = "1" expect-test = "1" -itertools = "0.11" +itertools = "0.12" paste = "1" risingwave_expr_impl = { workspace = true } risingwave_frontend = { workspace = true } diff --git a/src/jni_core/Cargo.toml b/src/jni_core/Cargo.toml index c1bdde44b43d..8953d15a99a9 100644 --- a/src/jni_core/Cargo.toml +++ b/src/jni_core/Cargo.toml @@ -14,7 +14,7 @@ anyhow = "1" bytes = "1" cfg-or-panic = "0.2" futures = { version = "0.3", default-features = false, features = ["alloc"] } -itertools = "0.11" +itertools = "0.12" jni = "0.21.1" paste = "1" prost = { workspace = true } diff --git a/src/meta/Cargo.toml b/src/meta/Cargo.toml index e0b58defa9b5..9acbf84ff121 100644 --- a/src/meta/Cargo.toml +++ b/src/meta/Cargo.toml @@ -33,7 +33,7 @@ function_name = "0.3.0" futures = { version = "0.3", default-features = false, features = ["alloc"] } hex = "0.4" hyper = "0.14" -itertools = "0.11" +itertools = "0.12" memcomparable = { version = "0.2" } mime_guess = "2" num-integer = "0.1" diff --git a/src/meta/node/Cargo.toml b/src/meta/node/Cargo.toml index e7597c0d041f..04fbea0705a8 100644 --- a/src/meta/node/Cargo.toml +++ b/src/meta/node/Cargo.toml @@ -19,7 +19,7 @@ clap = { version = "4", features = ["derive", "env"] } either = "1" etcd-client = { workspace = true } futures = { version = "0.3", default-features = false, features = ["alloc"] } -itertools = "0.11" +itertools = "0.12" prometheus-http-query = "0.7" redact = "0.1.5" regex = "1" diff --git a/src/meta/service/Cargo.toml b/src/meta/service/Cargo.toml index d1c08a642c8c..b734b6210649 100644 --- a/src/meta/service/Cargo.toml +++ b/src/meta/service/Cargo.toml @@ -18,7 +18,7 @@ anyhow = "1" async-trait = "0.1" either = "1" futures = { version = "0.3", default-features = false, features = ["alloc"] } -itertools = "0.11" +itertools = "0.12" rand = "0.8" regex = "1" risingwave_common = { workspace = true } diff --git a/src/object_store/Cargo.toml b/src/object_store/Cargo.toml index 87fd07be7810..a3f77e101bd4 100644 --- a/src/object_store/Cargo.toml +++ b/src/object_store/Cargo.toml @@ -25,7 +25,7 @@ futures = { version = "0.3", default-features = false, features = ["alloc"] } hyper = "0.14" hyper-rustls = { version = "0.24.2", features = ["webpki-roots"] } hyper-tls = "0.5.0" -itertools = "0.11" +itertools = "0.12" opendal = "0.41" prometheus = { version = "0.13", features = ["process"] } risingwave_common = { workspace = true } diff --git a/src/risedevtool/Cargo.toml b/src/risedevtool/Cargo.toml index e3caf3c7d34d..ff2acfe3cbec 100644 --- a/src/risedevtool/Cargo.toml +++ b/src/risedevtool/Cargo.toml @@ -22,7 +22,7 @@ fs-err = "2.10.0" glob = "0.3" google-cloud-pubsub = "0.20" indicatif = "0.17" -itertools = "0.11" +itertools = "0.12" rdkafka = { workspace = true } redis = "0.23" regex = "1" diff --git a/src/risedevtool/config/Cargo.toml b/src/risedevtool/config/Cargo.toml index c0eecd8ce0c4..bafa8584922e 100644 --- a/src/risedevtool/config/Cargo.toml +++ b/src/risedevtool/config/Cargo.toml @@ -14,7 +14,7 @@ console = "0.15" dialoguer = "0.11" enum-iterator = "1" fs-err = "2.10.0" -itertools = "0.11" +itertools = "0.12" [lints] workspace = true diff --git a/src/rpc_client/Cargo.toml b/src/rpc_client/Cargo.toml index f49a9b26ad5e..6fb11d7f5a7f 100644 --- a/src/rpc_client/Cargo.toml +++ b/src/rpc_client/Cargo.toml @@ -20,7 +20,7 @@ easy-ext = "1" either = "1.9.0" futures = { version = "0.3", default-features = false, features = ["alloc"] } hyper = "0.14" -itertools = "0.11.0" +itertools = "0.12.0" lru = "0.10.1" moka = { version = "0.12", features = ["future"] } rand = "0.8" diff --git a/src/source/Cargo.toml b/src/source/Cargo.toml index aedb0b915890..735ca5f10d9b 100644 --- a/src/source/Cargo.toml +++ b/src/source/Cargo.toml @@ -17,7 +17,7 @@ normal = ["workspace-hack"] anyhow = "1" futures = { version = "0.3", default-features = false, features = ["alloc"] } futures-async-stream = { workspace = true } -itertools = "0.11" +itertools = "0.12" parking_lot = "0.12" rand = "0.8" risingwave_common = { workspace = true } diff --git a/src/sqlparser/Cargo.toml b/src/sqlparser/Cargo.toml index 56566b4090a5..2bb58461302b 100644 --- a/src/sqlparser/Cargo.toml +++ b/src/sqlparser/Cargo.toml @@ -25,7 +25,7 @@ ignored = ["workspace-hack"] normal = ["workspace-hack"] [dependencies] -itertools = "0.11" +itertools = "0.12" serde = { version = "1.0", features = ["derive"], optional = true } tracing = "0.1" diff --git a/src/storage/Cargo.toml b/src/storage/Cargo.toml index 6150a6e862fd..a53aca9ff1bc 100644 --- a/src/storage/Cargo.toml +++ b/src/storage/Cargo.toml @@ -30,7 +30,7 @@ foyer = { git = "https://github.com/MrCroxx/foyer", rev = "ce2e222" } futures = { version = "0.3", default-features = false, features = ["alloc"] } futures-async-stream = { workspace = true } hex = "0.4" -itertools = "0.11" +itertools = "0.12" libc = "0.2" lz4 = "1.24.0" memcomparable = "0.2" diff --git a/src/storage/backup/Cargo.toml b/src/storage/backup/Cargo.toml index 9e98e6c9076a..2b4f1c60ec5c 100644 --- a/src/storage/backup/Cargo.toml +++ b/src/storage/backup/Cargo.toml @@ -18,7 +18,7 @@ anyhow = "1" async-trait = "0.1" bincode = "1.3" bytes = { version = "1", features = ["serde"] } -itertools = "0.11" +itertools = "0.12" parking_lot = { version = "0.12", features = ["arc_lock"] } prost = { workspace = true } risingwave_common = { workspace = true } diff --git a/src/storage/hummock_sdk/Cargo.toml b/src/storage/hummock_sdk/Cargo.toml index 4e8e47a019c2..2bc06424280e 100644 --- a/src/storage/hummock_sdk/Cargo.toml +++ b/src/storage/hummock_sdk/Cargo.toml @@ -16,7 +16,7 @@ normal = ["workspace-hack"] [dependencies] bytes = "1" hex = "0.4" -itertools = "0.11" +itertools = "0.12" parse-display = "0.8" risingwave_common = { workspace = true } risingwave_pb = { workspace = true } diff --git a/src/storage/hummock_test/Cargo.toml b/src/storage/hummock_test/Cargo.toml index c3482f142d46..39af3db64241 100644 --- a/src/storage/hummock_test/Cargo.toml +++ b/src/storage/hummock_test/Cargo.toml @@ -21,7 +21,7 @@ clap = { version = "4", features = ["derive"] } fail = "0.5" futures = { version = "0.3", default-features = false, features = ["alloc"] } futures-async-stream = "0.2.9" -itertools = "0.11" +itertools = "0.12" parking_lot = "0.12" rand = "0.8" risingwave_common = { workspace = true } diff --git a/src/stream/Cargo.toml b/src/stream/Cargo.toml index d85914a87b0d..4cdb6ad02b36 100644 --- a/src/stream/Cargo.toml +++ b/src/stream/Cargo.toml @@ -33,7 +33,7 @@ governor = { version = "0.6", default-features = false, features = [ "jitter", ] } hytra = "0.1.2" -itertools = "0.11" +itertools = "0.12" local_stats_alloc = { path = "../utils/local_stats_alloc" } lru = { git = "https://github.com/risingwavelabs/lru-rs.git", rev = "cb2d7c7" } maplit = "1.0.2" diff --git a/src/tests/simulation/Cargo.toml b/src/tests/simulation/Cargo.toml index c2e37b78e14a..eceab4c434eb 100644 --- a/src/tests/simulation/Cargo.toml +++ b/src/tests/simulation/Cargo.toml @@ -20,7 +20,7 @@ console = "0.15" etcd-client = { workspace = true } futures = { version = "0.3", default-features = false, features = ["alloc"] } glob = "0.3" -itertools = "0.11" +itertools = "0.12" lru = { git = "https://github.com/risingwavelabs/lru-rs.git", rev = "cb2d7c7" } madsim = "0.2.22" paste = "1" diff --git a/src/tests/sqlsmith/Cargo.toml b/src/tests/sqlsmith/Cargo.toml index 726b8d15ea1e..2ddaa196eb15 100644 --- a/src/tests/sqlsmith/Cargo.toml +++ b/src/tests/sqlsmith/Cargo.toml @@ -17,7 +17,7 @@ normal = ["workspace-hack"] anyhow = "1" chrono = "0.4" clap = { version = "4", features = ["derive"] } -itertools = "0.11" +itertools = "0.12" rand = { version = "0.8", features = ["small_rng"] } rand_chacha = { version = "0.3.1" } regex = "1" diff --git a/src/utils/pgwire/Cargo.toml b/src/utils/pgwire/Cargo.toml index c62def532c36..73ed5a4200b1 100644 --- a/src/utils/pgwire/Cargo.toml +++ b/src/utils/pgwire/Cargo.toml @@ -20,7 +20,7 @@ auto_enums = { version = "0.8", features = ["tokio1"] } byteorder = "1.5" bytes = "1" futures = { version = "0.3", default-features = false, features = ["alloc"] } -itertools = "0.11" +itertools = "0.12" openssl = "0.10.57" panic-message = "0.3" risingwave_common = { workspace = true } diff --git a/src/workspace-hack/Cargo.toml b/src/workspace-hack/Cargo.toml index b263a9c5bcef..6a38db45a276 100644 --- a/src/workspace-hack/Cargo.toml +++ b/src/workspace-hack/Cargo.toml @@ -59,8 +59,7 @@ hashbrown-5ef9efb8ec2df382 = { package = "hashbrown", version = "0.12", features hmac = { version = "0.12", default-features = false, features = ["reset"] } hyper = { version = "0.14", features = ["full"] } indexmap = { version = "1", default-features = false, features = ["serde", "std"] } -itertools-93f6ce9d446188ac = { package = "itertools", version = "0.10" } -itertools-a6292c17cd707f01 = { package = "itertools", version = "0.11" } +itertools = { version = "0.10" } jni = { version = "0.21", features = ["invocation"] } lazy_static = { version = "1", default-features = false, features = ["spin_no_std"] } lexical-core = { version = "0.8", features = ["format"] } @@ -158,8 +157,7 @@ either = { version = "1", features = ["serde"] } fixedbitset = { version = "0.4" } frunk_core = { version = "0.4", default-features = false, features = ["std"] } hashbrown-582f2526e08bb6a0 = { package = "hashbrown", version = "0.14", features = ["nightly", "raw"] } -itertools-93f6ce9d446188ac = { package = "itertools", version = "0.10" } -itertools-a6292c17cd707f01 = { package = "itertools", version = "0.11" } +itertools = { version = "0.10" } lazy_static = { version = "1", default-features = false, features = ["spin_no_std"] } libc = { version = "0.2", features = ["extra_traits"] } log = { version = "0.4", default-features = false, features = ["kv_unstable", "std"] } From a34f46a0ac3d79363e93f11f710dbdf32233e0c6 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Fri, 17 Nov 2023 18:38:04 +0800 Subject: [PATCH 15/16] refactor: separate `CdcScan` from `Scan` in `logical` and `core` (#13494) Co-authored-by: Eric Fu --- .../tests/testdata/input/create_source.yaml | 20 ++ .../tests/testdata/output/create_source.yaml | 20 ++ src/frontend/src/handler/create_index.rs | 2 - src/frontend/src/handler/create_table.rs | 4 +- .../optimizer/plan_node/generic/cdc_scan.rs | 173 +++++++++++++++ .../src/optimizer/plan_node/generic/mod.rs | 4 + .../src/optimizer/plan_node/generic/scan.rs | 99 ++------- .../optimizer/plan_node/logical_cdc_scan.rs | 206 ++++++++++++++++++ .../src/optimizer/plan_node/logical_scan.rs | 56 +---- src/frontend/src/optimizer/plan_node/mod.rs | 4 + .../plan_node/stream_cdc_table_scan.rs | 8 +- .../optimizer/rule/index_selection_rule.rs | 7 +- src/frontend/src/planner/relation.rs | 2 - .../src/scheduler/distributed/query.rs | 2 - 14 files changed, 452 insertions(+), 155 deletions(-) create mode 100644 src/frontend/src/optimizer/plan_node/generic/cdc_scan.rs create mode 100644 src/frontend/src/optimizer/plan_node/logical_cdc_scan.rs diff --git a/src/frontend/planner_test/tests/testdata/input/create_source.yaml b/src/frontend/planner_test/tests/testdata/input/create_source.yaml index 0496ad1c13b1..32fd9c806ae3 100644 --- a/src/frontend/planner_test/tests/testdata/input/create_source.yaml +++ b/src/frontend/planner_test/tests/testdata/input/create_source.yaml @@ -33,6 +33,26 @@ ) FORMAT PLAIN ENCODE CSV (delimiter = E'\t', without_header = true); expected_outputs: - explain_output +- id: create_source_with_cdc_backfill + sql: | + create source mysql_mydb with ( + connector = 'mysql-cdc', + hostname = '127.0.0.1', + port = '8306', + username = 'root', + password = '123456', + database.name = 'mydb', + server.id = 5888 + ); + explain (logical) create table t1_rw ( + v1 int, + v2 int, + primary key(v1) + ) from mysql_mydb table 'mydb.t1'; + expected_outputs: + - explain_output + with_config_map: + CDC_BACKFILL: 'true' - id: create_source_with_cdc_backfill sql: | create source mysql_mydb with ( diff --git a/src/frontend/planner_test/tests/testdata/output/create_source.yaml b/src/frontend/planner_test/tests/testdata/output/create_source.yaml index c8082fb3b713..2aeea6f914d2 100644 --- a/src/frontend/planner_test/tests/testdata/output/create_source.yaml +++ b/src/frontend/planner_test/tests/testdata/output/create_source.yaml @@ -45,6 +45,26 @@ └─StreamExchange { dist: HashShard(_row_id) } └─StreamDml { columns: [v1, v2, _row_id] } └─StreamSource +- id: create_source_with_cdc_backfill + sql: | + create source mysql_mydb with ( + connector = 'mysql-cdc', + hostname = '127.0.0.1', + port = '8306', + username = 'root', + password = '123456', + database.name = 'mydb', + server.id = 5888 + ); + explain (logical) create table t1_rw ( + v1 int, + v2 int, + primary key(v1) + ) from mysql_mydb table 'mydb.t1'; + explain_output: | + LogicalCdcScan { table: mydb.t1, columns: [v1, v2] } + with_config_map: + CDC_BACKFILL: 'true' - id: create_source_with_cdc_backfill sql: | create source mysql_mydb with ( diff --git a/src/frontend/src/handler/create_index.rs b/src/frontend/src/handler/create_index.rs index 3eb93f2900c9..2cc7bb3b49df 100644 --- a/src/frontend/src/handler/create_index.rs +++ b/src/frontend/src/handler/create_index.rs @@ -35,7 +35,6 @@ use crate::catalog::root_catalog::SchemaPath; use crate::expr::{Expr, ExprImpl, InputRef}; use crate::handler::privilege::ObjectCheckItem; use crate::handler::HandlerArgs; -use crate::optimizer::plan_node::generic::ScanTableType; use crate::optimizer::plan_node::{Explain, LogicalProject, LogicalScan, StreamMaterialize}; use crate::optimizer::property::{Cardinality, Distribution, Order, RequiredDist}; use crate::optimizer::{OptimizerContext, OptimizerContextRef, PlanRef, PlanRoot}; @@ -325,7 +324,6 @@ fn assemble_materialize( let logical_scan = LogicalScan::create( table_name, - ScanTableType::default(), table_desc.clone(), // Index table has no indexes. vec![], diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index c9b96065f297..e79571e5e0c9 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -55,7 +55,7 @@ use crate::handler::create_source::{ check_source_schema, validate_compatibility, UPSTREAM_SOURCE_KEY, }; use crate::handler::HandlerArgs; -use crate::optimizer::plan_node::{LogicalScan, LogicalSource}; +use crate::optimizer::plan_node::{LogicalCdcScan, LogicalSource}; use crate::optimizer::property::{Order, RequiredDist}; use crate::optimizer::{OptimizerContext, OptimizerContextRef, PlanRef, PlanRoot}; use crate::session::SessionImpl; @@ -863,7 +863,7 @@ pub(crate) fn gen_create_table_plan_for_cdc_source( tracing::debug!(?cdc_table_desc, "create cdc table"); - let logical_scan = LogicalScan::create_for_cdc( + let logical_scan = LogicalCdcScan::create( external_table_name, Rc::new(cdc_table_desc), context.clone(), diff --git a/src/frontend/src/optimizer/plan_node/generic/cdc_scan.rs b/src/frontend/src/optimizer/plan_node/generic/cdc_scan.rs new file mode 100644 index 000000000000..c33cdc3330f2 --- /dev/null +++ b/src/frontend/src/optimizer/plan_node/generic/cdc_scan.rs @@ -0,0 +1,173 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::rc::Rc; + +use educe::Educe; +use fixedbitset::FixedBitSet; +use pretty_xmlish::Pretty; +use risingwave_common::catalog::{CdcTableDesc, ColumnDesc, Field, Schema}; +use risingwave_common::util::column_index_mapping::ColIndexMapping; +use risingwave_common::util::sort_util::ColumnOrder; + +use super::GenericPlanNode; +use crate::catalog::ColumnId; +use crate::expr::ExprRewriter; +use crate::optimizer::optimizer_context::OptimizerContextRef; +use crate::optimizer::property::FunctionalDependencySet; + +/// [`CdcScan`] reads rows of a table from an external upstream database +#[derive(Debug, Clone, Educe)] +#[educe(PartialEq, Eq, Hash)] +pub struct CdcScan { + pub table_name: String, + /// Include `output_col_idx` and columns required in `predicate` + pub output_col_idx: Vec, + /// Descriptor of the external table for CDC + pub cdc_table_desc: Rc, + #[educe(PartialEq(ignore))] + #[educe(Hash(ignore))] + pub ctx: OptimizerContextRef, +} + +impl CdcScan { + pub fn rewrite_exprs(&self, _rewriter: &mut dyn ExprRewriter) {} + + /// Get the ids of the output columns. + pub fn output_column_ids(&self) -> Vec { + self.output_col_idx + .iter() + .map(|i| self.get_table_columns()[*i].column_id) + .collect() + } + + pub fn primary_key(&self) -> &[ColumnOrder] { + &self.cdc_table_desc.pk + } + + pub fn watermark_columns(&self) -> FixedBitSet { + FixedBitSet::with_capacity(self.get_table_columns().len()) + } + + pub(crate) fn column_names_with_table_prefix(&self) -> Vec { + self.output_col_idx + .iter() + .map(|&i| format!("{}.{}", self.table_name, self.get_table_columns()[i].name)) + .collect() + } + + pub(crate) fn column_names(&self) -> Vec { + self.output_col_idx + .iter() + .map(|&i| self.get_table_columns()[i].name.clone()) + .collect() + } + + /// get the Mapping of columnIndex from internal column index to output column index + pub fn i2o_col_mapping(&self) -> ColIndexMapping { + ColIndexMapping::with_remaining_columns( + &self.output_col_idx, + self.get_table_columns().len(), + ) + } + + /// Get the ids of the output columns and primary key columns. + pub fn output_and_pk_column_ids(&self) -> Vec { + let mut ids = self.output_column_ids(); + for column_order in self.primary_key() { + let id = self.get_table_columns()[column_order.column_index].column_id; + if !ids.contains(&id) { + ids.push(id); + } + } + ids + } + + /// Create a logical scan node for CDC backfill + pub(crate) fn new( + table_name: String, + output_col_idx: Vec, // the column index in the table + cdc_table_desc: Rc, + ctx: OptimizerContextRef, + ) -> Self { + Self { + table_name, + output_col_idx, + cdc_table_desc, + ctx, + } + } + + pub(crate) fn columns_pretty<'a>(&self, verbose: bool) -> Pretty<'a> { + Pretty::Array( + match verbose { + true => self.column_names_with_table_prefix(), + false => self.column_names(), + } + .into_iter() + .map(Pretty::from) + .collect(), + ) + } +} + +// TODO: extend for cdc table +impl GenericPlanNode for CdcScan { + fn schema(&self) -> Schema { + let fields = self + .output_col_idx + .iter() + .map(|tb_idx| { + let col = &self.get_table_columns()[*tb_idx]; + Field::from_with_table_name_prefix(col, &self.table_name) + }) + .collect(); + Schema { fields } + } + + fn stream_key(&self) -> Option> { + Some(self.cdc_table_desc.stream_key.clone()) + } + + fn ctx(&self) -> OptimizerContextRef { + self.ctx.clone() + } + + fn functional_dependency(&self) -> FunctionalDependencySet { + let pk_indices = self.stream_key(); + let col_num = self.output_col_idx.len(); + match &pk_indices { + Some(pk_indices) => FunctionalDependencySet::with_key(col_num, pk_indices), + None => FunctionalDependencySet::new(col_num), + } + } +} + +impl CdcScan { + pub fn get_table_columns(&self) -> &[ColumnDesc] { + &self.cdc_table_desc.columns + } + + pub fn append_only(&self) -> bool { + false + } + + /// Get the descs of the output columns. + pub fn column_descs(&self) -> Vec { + self.output_col_idx + .iter() + .map(|&i| self.get_table_columns()[i].clone()) + .collect() + } +} diff --git a/src/frontend/src/optimizer/plan_node/generic/mod.rs b/src/frontend/src/optimizer/plan_node/generic/mod.rs index c0baacffec8a..9da00cb05606 100644 --- a/src/frontend/src/optimizer/plan_node/generic/mod.rs +++ b/src/frontend/src/optimizer/plan_node/generic/mod.rs @@ -44,6 +44,10 @@ mod scan; pub use scan::*; mod sys_scan; pub use sys_scan::*; + +mod cdc_scan; +pub use cdc_scan::*; + mod union; pub use union::*; mod top_n; diff --git a/src/frontend/src/optimizer/plan_node/generic/scan.rs b/src/frontend/src/optimizer/plan_node/generic/scan.rs index 3c615183f9ad..d0890eb72ff3 100644 --- a/src/frontend/src/optimizer/plan_node/generic/scan.rs +++ b/src/frontend/src/optimizer/plan_node/generic/scan.rs @@ -18,7 +18,7 @@ use std::rc::Rc; use educe::Educe; use fixedbitset::FixedBitSet; use pretty_xmlish::Pretty; -use risingwave_common::catalog::{CdcTableDesc, ColumnDesc, Field, Schema, TableDesc}; +use risingwave_common::catalog::{ColumnDesc, Field, Schema, TableDesc}; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_common::util::sort_util::ColumnOrder; @@ -29,27 +29,16 @@ use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::optimizer::property::{Cardinality, FunctionalDependencySet, Order}; use crate::utils::{ColIndexMappingRewriteExt, Condition}; -#[derive(Debug, Default, Clone, Educe)] -#[educe(PartialEq, Eq, Hash)] -pub enum ScanTableType { - #[default] - General, - CdcTable, -} - /// [`Scan`] returns contents of a table or other equivalent object #[derive(Debug, Clone, Educe)] #[educe(PartialEq, Eq, Hash)] pub struct Scan { pub table_name: String, - pub scan_table_type: ScanTableType, /// Include `output_col_idx` and columns required in `predicate` pub required_col_idx: Vec, pub output_col_idx: Vec, /// Descriptor of the table pub table_desc: Rc, - /// Descriptor of the external table for CDC - pub cdc_table_desc: Rc, /// Descriptors of all indexes on this table pub indexes: Vec>, /// The pushed down predicates. It refers to column indexes of the table. @@ -77,10 +66,6 @@ impl Scan { /// /// Return `None` if the table's distribution key are not all in the `output_col_idx`. pub fn distribution_key(&self) -> Option> { - if self.is_cdc_table() { - return None; - } - let tb_idx_to_op_idx = self .output_col_idx .iter() @@ -103,20 +88,12 @@ impl Scan { } pub fn primary_key(&self) -> &[ColumnOrder] { - if self.is_cdc_table() { - &self.cdc_table_desc.pk - } else { - &self.table_desc.pk - } + &self.table_desc.pk } pub fn watermark_columns(&self) -> FixedBitSet { - if self.is_cdc_table() { - FixedBitSet::with_capacity(self.get_table_columns().len()) - } else { - let watermark_columns = &self.table_desc.watermark_columns; - self.i2o_col_mapping().rewrite_bitset(watermark_columns) - } + let watermark_columns = &self.table_desc.watermark_columns; + self.i2o_col_mapping().rewrite_bitset(watermark_columns) } pub(crate) fn column_names_with_table_prefix(&self) -> Vec { @@ -241,7 +218,6 @@ impl Scan { Self::new( index_name.to_string(), - ScanTableType::default(), new_output_col_idx, index_table_desc, vec![], @@ -256,7 +232,6 @@ impl Scan { #[allow(clippy::too_many_arguments)] pub(crate) fn new( table_name: String, - scan_table_type: ScanTableType, output_col_idx: Vec, // the column index in the table table_desc: Rc, indexes: Vec>, @@ -267,10 +242,8 @@ impl Scan { ) -> Self { Self::new_inner( table_name, - scan_table_type, output_col_idx, table_desc, - Rc::new(CdcTableDesc::default()), indexes, ctx, predicate, @@ -279,34 +252,11 @@ impl Scan { ) } - /// Create a logical scan node for CDC backfill - pub(crate) fn new_for_cdc( - table_name: String, - output_col_idx: Vec, // the column index in the table - cdc_table_desc: Rc, - ctx: OptimizerContextRef, - ) -> Self { - Self::new_inner( - table_name, - ScanTableType::CdcTable, - output_col_idx, - Rc::new(TableDesc::default()), - cdc_table_desc, - vec![], - ctx, - Condition::true_cond(), - false, - Cardinality::unknown(), - ) - } - #[allow(clippy::too_many_arguments)] pub(crate) fn new_inner( table_name: String, - scan_table_type: ScanTableType, output_col_idx: Vec, // the column index in the table table_desc: Rc, - cdc_table_desc: Rc, indexes: Vec>, ctx: OptimizerContextRef, predicate: Condition, // refers to column indexes of the table @@ -331,11 +281,9 @@ impl Scan { Self { table_name, - scan_table_type, required_col_idx, output_col_idx, table_desc, - cdc_table_desc, indexes, predicate, chunk_size: None, @@ -368,7 +316,6 @@ impl Scan { } } -// TODO: extend for cdc table impl GenericPlanNode for Scan { fn schema(&self) -> Schema { let fields = self @@ -383,21 +330,16 @@ impl GenericPlanNode for Scan { } fn stream_key(&self) -> Option> { - if self.is_cdc_table() { - Some(self.cdc_table_desc.stream_key.clone()) - } else { - let id_to_op_idx = - Self::get_id_to_op_idx_mapping(&self.output_col_idx, &self.table_desc); - self.table_desc - .stream_key - .iter() - .map(|&c| { - id_to_op_idx - .get(&self.table_desc.columns[c].column_id) - .copied() - }) - .collect::>>() - } + let id_to_op_idx = Self::get_id_to_op_idx_mapping(&self.output_col_idx, &self.table_desc); + self.table_desc + .stream_key + .iter() + .map(|&c| { + id_to_op_idx + .get(&self.table_desc.columns[c].column_id) + .copied() + }) + .collect::>>() } fn ctx(&self) -> OptimizerContextRef { @@ -416,21 +358,10 @@ impl GenericPlanNode for Scan { impl Scan { pub fn get_table_columns(&self) -> &[ColumnDesc] { - if self.is_cdc_table() { - &self.cdc_table_desc.columns - } else { - &self.table_desc.columns - } - } - - pub fn is_cdc_table(&self) -> bool { - matches!(self.scan_table_type, ScanTableType::CdcTable) + &self.table_desc.columns } pub fn append_only(&self) -> bool { - if self.is_cdc_table() { - return false; - } self.table_desc.append_only } diff --git a/src/frontend/src/optimizer/plan_node/logical_cdc_scan.rs b/src/frontend/src/optimizer/plan_node/logical_cdc_scan.rs new file mode 100644 index 000000000000..d155581f0196 --- /dev/null +++ b/src/frontend/src/optimizer/plan_node/logical_cdc_scan.rs @@ -0,0 +1,206 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::rc::Rc; + +use pretty_xmlish::{Pretty, XmlNode}; +use risingwave_common::catalog::{CdcTableDesc, ColumnDesc}; +use risingwave_common::error::Result; + +use super::generic::GenericPlanRef; +use super::utils::{childless_record, Distill}; +use super::{ + generic, ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, PredicatePushdown, ToBatch, + ToStream, +}; +use crate::catalog::ColumnId; +use crate::expr::ExprRewriter; +use crate::optimizer::optimizer_context::OptimizerContextRef; +use crate::optimizer::plan_node::{ + ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, StreamCdcTableScan, + ToStreamContext, +}; +use crate::optimizer::property::Order; +use crate::utils::{ColIndexMapping, Condition}; + +/// `LogicalCdcScan` reads rows of a table from an external upstream database +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct LogicalCdcScan { + pub base: PlanBase, + core: generic::CdcScan, +} + +impl From for LogicalCdcScan { + fn from(core: generic::CdcScan) -> Self { + let base = PlanBase::new_logical_with_core(&core); + Self { base, core } + } +} + +impl From for PlanRef { + fn from(core: generic::CdcScan) -> Self { + LogicalCdcScan::from(core).into() + } +} + +impl LogicalCdcScan { + pub fn create( + table_name: String, // explain-only + cdc_table_desc: Rc, + ctx: OptimizerContextRef, + ) -> Self { + generic::CdcScan::new( + table_name, + (0..cdc_table_desc.columns.len()).collect(), + cdc_table_desc, + ctx, + ) + .into() + } + + pub fn table_name(&self) -> &str { + &self.core.table_name + } + + pub fn cdc_table_desc(&self) -> &CdcTableDesc { + self.core.cdc_table_desc.as_ref() + } + + /// Get the descs of the output columns. + pub fn column_descs(&self) -> Vec { + self.core.column_descs() + } + + /// Get the ids of the output columns. + pub fn output_column_ids(&self) -> Vec { + self.core.output_column_ids() + } + + pub fn clone_with_output_indices(&self, output_col_idx: Vec) -> Self { + generic::CdcScan::new( + self.table_name().to_string(), + output_col_idx, + self.core.cdc_table_desc.clone(), + self.base.ctx().clone(), + ) + .into() + } + + pub fn output_col_idx(&self) -> &Vec { + &self.core.output_col_idx + } +} + +impl_plan_tree_node_for_leaf! {LogicalCdcScan} + +impl Distill for LogicalCdcScan { + fn distill<'a>(&self) -> XmlNode<'a> { + let verbose = self.base.ctx().is_explain_verbose(); + let mut vec = Vec::with_capacity(5); + vec.push(("table", Pretty::from(self.table_name().to_owned()))); + let key_is_columns = true; + let key = if key_is_columns { + "columns" + } else { + "output_columns" + }; + vec.push((key, self.core.columns_pretty(verbose))); + if !key_is_columns { + vec.push(( + "required_columns", + Pretty::Array( + self.output_col_idx() + .iter() + .map(|i| { + let col_name = &self.cdc_table_desc().columns[*i].name; + Pretty::from(if verbose { + format!("{}.{}", self.table_name(), col_name) + } else { + col_name.to_string() + }) + }) + .collect(), + ), + )); + } + + childless_record("LogicalCdcScan", vec) + } +} + +impl ColPrunable for LogicalCdcScan { + fn prune_col(&self, required_cols: &[usize], _ctx: &mut ColumnPruningContext) -> PlanRef { + let output_col_idx: Vec = required_cols + .iter() + .map(|i| self.output_col_idx()[*i]) + .collect(); + assert!(output_col_idx + .iter() + .all(|i| self.output_col_idx().contains(i))); + + self.clone_with_output_indices(output_col_idx).into() + } +} + +impl ExprRewritable for LogicalCdcScan { + fn has_rewritable_expr(&self) -> bool { + true + } + + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + let core = self.core.clone(); + core.rewrite_exprs(r); + Self { + base: self.base.clone_with_new_plan_id(), + core, + } + .into() + } +} + +impl PredicatePushdown for LogicalCdcScan { + fn predicate_pushdown( + &self, + _predicate: Condition, + _ctx: &mut PredicatePushdownContext, + ) -> PlanRef { + self.clone().into() + } +} + +impl ToBatch for LogicalCdcScan { + fn to_batch(&self) -> Result { + unreachable!() + } + + fn to_batch_with_order_required(&self, _required_order: &Order) -> Result { + unreachable!() + } +} + +impl ToStream for LogicalCdcScan { + fn to_stream(&self, _ctx: &mut ToStreamContext) -> Result { + Ok(StreamCdcTableScan::new(self.core.clone()).into()) + } + + fn logical_rewrite_for_stream( + &self, + _ctx: &mut RewriteStreamContext, + ) -> Result<(PlanRef, ColIndexMapping)> { + Ok(( + self.clone().into(), + ColIndexMapping::identity(self.schema().len()), + )) + } +} diff --git a/src/frontend/src/optimizer/plan_node/logical_scan.rs b/src/frontend/src/optimizer/plan_node/logical_scan.rs index bd1297a4cb92..8c4aedf52492 100644 --- a/src/frontend/src/optimizer/plan_node/logical_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_scan.rs @@ -18,7 +18,7 @@ use std::rc::Rc; use fixedbitset::FixedBitSet; use itertools::Itertools; use pretty_xmlish::{Pretty, XmlNode}; -use risingwave_common::catalog::{CdcTableDesc, ColumnDesc, TableDesc}; +use risingwave_common::catalog::{ColumnDesc, TableDesc}; use risingwave_common::error::Result; use risingwave_common::util::sort_util::ColumnOrder; @@ -31,10 +31,9 @@ use super::{ use crate::catalog::{ColumnId, IndexCatalog}; use crate::expr::{CorrelatedInputRef, ExprImpl, ExprRewriter, ExprVisitor, InputRef}; use crate::optimizer::optimizer_context::OptimizerContextRef; -use crate::optimizer::plan_node::generic::ScanTableType; use crate::optimizer::plan_node::{ BatchSeqScan, ColumnPruningContext, LogicalFilter, LogicalProject, LogicalValues, - PredicatePushdownContext, RewriteStreamContext, StreamCdcTableScan, ToStreamContext, + PredicatePushdownContext, RewriteStreamContext, ToStreamContext, }; use crate::optimizer::property::{Cardinality, Order}; use crate::optimizer::rule::IndexSelectionRule; @@ -64,7 +63,6 @@ impl LogicalScan { /// Create a [`LogicalScan`] node. Used by planner. pub fn create( table_name: String, // explain-only - scan_table_type: ScanTableType, table_desc: Rc, indexes: Vec>, ctx: OptimizerContextRef, @@ -73,7 +71,6 @@ impl LogicalScan { ) -> Self { generic::Scan::new( table_name, - scan_table_type, (0..table_desc.columns.len()).collect(), table_desc, indexes, @@ -85,32 +82,10 @@ impl LogicalScan { .into() } - pub fn create_for_cdc( - table_name: String, // explain-only - cdc_table_desc: Rc, - ctx: OptimizerContextRef, - ) -> Self { - generic::Scan::new_for_cdc( - table_name, - (0..cdc_table_desc.columns.len()).collect(), - cdc_table_desc, - ctx, - ) - .into() - } - pub fn table_name(&self) -> &str { &self.core.table_name } - pub fn scan_table_type(&self) -> &ScanTableType { - &self.core.scan_table_type - } - - pub fn is_cdc_table(&self) -> bool { - matches!(self.core.scan_table_type, ScanTableType::CdcTable) - } - pub fn for_system_time_as_of_proctime(&self) -> bool { self.core.for_system_time_as_of_proctime } @@ -125,10 +100,6 @@ impl LogicalScan { self.core.table_desc.as_ref() } - pub fn cdc_table_desc(&self) -> &CdcTableDesc { - self.core.cdc_table_desc.as_ref() - } - /// Get the descs of the output columns. pub fn column_descs(&self) -> Vec { self.core.column_descs() @@ -272,7 +243,6 @@ impl LogicalScan { let scan_without_predicate = generic::Scan::new( self.table_name().to_string(), - self.scan_table_type().clone(), self.required_col_idx().to_vec(), self.core.table_desc.clone(), self.indexes().to_vec(), @@ -292,10 +262,8 @@ impl LogicalScan { fn clone_with_predicate(&self, predicate: Condition) -> Self { generic::Scan::new_inner( self.table_name().to_string(), - self.scan_table_type().clone(), self.output_col_idx().to_vec(), self.core.table_desc.clone(), - self.core.cdc_table_desc.clone(), self.indexes().to_vec(), self.base.ctx().clone(), predicate, @@ -308,10 +276,8 @@ impl LogicalScan { pub fn clone_with_output_indices(&self, output_col_idx: Vec) -> Self { generic::Scan::new_inner( self.table_name().to_string(), - self.scan_table_type().clone(), output_col_idx, self.core.table_desc.clone(), - self.core.cdc_table_desc.clone(), self.indexes().to_vec(), self.base.ctx().clone(), self.predicate().clone(), @@ -419,11 +385,6 @@ impl PredicatePushdown for LogicalScan { mut predicate: Condition, _ctx: &mut PredicatePushdownContext, ) -> PlanRef { - // skip pushdown if the table is cdc table - if self.is_cdc_table() { - return self.clone().into(); - } - // If the predicate contains `CorrelatedInputRef` or `now()`. We don't push down. // This case could come from the predicate push down before the subquery unnesting. struct HasCorrelated {} @@ -554,11 +515,7 @@ impl ToBatch for LogicalScan { impl ToStream for LogicalScan { fn to_stream(&self, ctx: &mut ToStreamContext) -> Result { if self.predicate().always_true() { - if self.is_cdc_table() { - Ok(StreamCdcTableScan::new(self.core.clone()).into()) - } else { - Ok(StreamTableScan::new(self.core.clone()).into()) - } + Ok(StreamTableScan::new(self.core.clone()).into()) } else { let (scan, predicate, project_expr) = self.predicate_pull_up(); let mut plan = LogicalFilter::create(scan.into(), predicate); @@ -573,13 +530,6 @@ impl ToStream for LogicalScan { &self, _ctx: &mut RewriteStreamContext, ) -> Result<(PlanRef, ColIndexMapping)> { - if self.is_cdc_table() { - return Ok(( - self.clone().into(), - ColIndexMapping::identity(self.schema().len()), - )); - } - match self.base.stream_key().is_none() { true => { let mut col_ids = HashSet::new(); diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index 77f8fdb60607..a0be5132c92a 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -776,6 +776,7 @@ mod batch_update; mod batch_values; mod logical_agg; mod logical_apply; +mod logical_cdc_scan; mod logical_dedup; mod logical_delete; mod logical_except; @@ -864,6 +865,7 @@ pub use batch_update::BatchUpdate; pub use batch_values::BatchValues; pub use logical_agg::LogicalAgg; pub use logical_apply::LogicalApply; +pub use logical_cdc_scan::LogicalCdcScan; pub use logical_dedup::LogicalDedup; pub use logical_delete::LogicalDelete; pub use logical_except::LogicalExcept; @@ -948,6 +950,7 @@ macro_rules! for_all_plan_nodes { , { Logical, Filter } , { Logical, Project } , { Logical, Scan } + , { Logical, CdcScan } , { Logical, SysScan } , { Logical, Source } , { Logical, Insert } @@ -1041,6 +1044,7 @@ macro_rules! for_logical_plan_nodes { , { Logical, Filter } , { Logical, Project } , { Logical, Scan } + , { Logical, CdcScan } , { Logical, SysScan } , { Logical, Source } , { Logical, Insert } diff --git a/src/frontend/src/optimizer/plan_node/stream_cdc_table_scan.rs b/src/frontend/src/optimizer/plan_node/stream_cdc_table_scan.rs index 4731227bc54c..4ef394f3748f 100644 --- a/src/frontend/src/optimizer/plan_node/stream_cdc_table_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_cdc_table_scan.rs @@ -36,13 +36,13 @@ use crate::{Explain, TableCatalog}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamCdcTableScan { pub base: PlanBase, - core: generic::Scan, + core: generic::CdcScan, batch_plan_id: PlanNodeId, stream_scan_type: StreamScanType, } impl StreamCdcTableScan { - pub fn new(core: generic::Scan) -> Self { + pub fn new(core: generic::CdcScan) -> Self { let batch_plan_id = core.ctx.next_plan_node_id(); let distribution = Distribution::SomeShard; let base = PlanBase::new_stream_with_core( @@ -64,7 +64,7 @@ impl StreamCdcTableScan { &self.core.table_name } - pub fn core(&self) -> &generic::Scan { + pub fn core(&self) -> &generic::CdcScan { &self.core } @@ -263,7 +263,7 @@ impl ExprRewritable for StreamCdcTableScan { } fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - let mut core = self.core.clone(); + let core = self.core.clone(); core.rewrite_exprs(r); Self::new(core).into() } diff --git a/src/frontend/src/optimizer/rule/index_selection_rule.rs b/src/frontend/src/optimizer/rule/index_selection_rule.rs index 7aeefe4f9154..3920924d8146 100644 --- a/src/frontend/src/optimizer/rule/index_selection_rule.rs +++ b/src/frontend/src/optimizer/rule/index_selection_rule.rs @@ -66,7 +66,7 @@ use crate::expr::{ FunctionCall, InputRef, }; use crate::optimizer::optimizer_context::OptimizerContextRef; -use crate::optimizer::plan_node::generic::{GenericPlanRef, ScanTableType}; +use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::{ generic, ColumnPruningContext, LogicalJoin, LogicalScan, LogicalUnion, PlanTreeNode, PlanTreeNodeBinary, PredicatePushdown, PredicatePushdownContext, @@ -222,7 +222,6 @@ impl IndexSelectionRule { let index_scan = LogicalScan::create( index.index_table.name.clone(), - ScanTableType::default(), index.index_table.table_desc().into(), vec![], logical_scan.ctx(), @@ -232,7 +231,6 @@ impl IndexSelectionRule { let primary_table_scan = LogicalScan::create( index.primary_table.name.clone(), - ScanTableType::default(), index.primary_table.table_desc().into(), vec![], logical_scan.ctx(), @@ -332,7 +330,6 @@ impl IndexSelectionRule { let primary_table_scan = LogicalScan::create( logical_scan.table_name().to_string(), - ScanTableType::default(), primary_table_desc.clone().into(), vec![], logical_scan.ctx(), @@ -560,7 +557,6 @@ impl IndexSelectionRule { let primary_access = generic::Scan::new( logical_scan.table_name().to_string(), - ScanTableType::default(), primary_table_desc .pk .iter() @@ -603,7 +599,6 @@ impl IndexSelectionRule { Some( generic::Scan::new( index.index_table.name.to_string(), - ScanTableType::default(), index .primary_table_pk_ref_to_index_table() .iter() diff --git a/src/frontend/src/planner/relation.rs b/src/frontend/src/planner/relation.rs index b4987403a792..7ff5806eceb0 100644 --- a/src/frontend/src/planner/relation.rs +++ b/src/frontend/src/planner/relation.rs @@ -24,7 +24,6 @@ use crate::binder::{ BoundWindowTableFunction, Relation, WindowTableFunctionKind, }; use crate::expr::{Expr, ExprImpl, ExprType, FunctionCall, InputRef}; -use crate::optimizer::plan_node::generic::ScanTableType; use crate::optimizer::plan_node::{ LogicalApply, LogicalHopWindow, LogicalJoin, LogicalProject, LogicalScan, LogicalShare, LogicalSource, LogicalSysScan, LogicalTableFunction, LogicalValues, PlanRef, @@ -69,7 +68,6 @@ impl Planner { pub(super) fn plan_base_table(&mut self, base_table: &BoundBaseTable) -> Result { Ok(LogicalScan::create( base_table.table_catalog.name().to_string(), - ScanTableType::default(), Rc::new(base_table.table_catalog.table_desc()), base_table .table_indexes diff --git a/src/frontend/src/scheduler/distributed/query.rs b/src/frontend/src/scheduler/distributed/query.rs index f35b4771ba13..db5581aea856 100644 --- a/src/frontend/src/scheduler/distributed/query.rs +++ b/src/frontend/src/scheduler/distributed/query.rs @@ -449,7 +449,6 @@ pub(crate) mod tests { use crate::catalog::catalog_service::CatalogReader; use crate::catalog::root_catalog::Catalog; use crate::expr::InputRef; - use crate::optimizer::plan_node::generic::ScanTableType; use crate::optimizer::plan_node::{ generic, BatchExchange, BatchFilter, BatchHashJoin, EqJoinPredicate, LogicalScan, ToBatch, }; @@ -513,7 +512,6 @@ pub(crate) mod tests { let table_id = 0.into(); let batch_plan_node: PlanRef = LogicalScan::create( "".to_string(), - ScanTableType::default(), Rc::new(TableDesc { table_id, stream_key: vec![], From b289d384c74afa9fac3c415960b5e013b619b666 Mon Sep 17 00:00:00 2001 From: Bohan Zhang Date: Sat, 18 Nov 2023 00:46:36 +0800 Subject: [PATCH 16/16] fix: reject duplicate entries in with clause (#13488) --- e2e_test/ddl/throttle.slt | 4 +- e2e_test/schema_registry/pb.slt | 6 +-- e2e_test/source/basic/kafka.slt | 16 +++++--- .../basic/old_row_format_syntax/kafka.slt | 6 +-- src/frontend/src/utils/with_options.rs | 37 +++++++++++-------- 5 files changed, 38 insertions(+), 31 deletions(-) diff --git a/e2e_test/ddl/throttle.slt b/e2e_test/ddl/throttle.slt index 6395697f0df7..9b6c2f053bf6 100644 --- a/e2e_test/ddl/throttle.slt +++ b/e2e_test/ddl/throttle.slt @@ -6,8 +6,8 @@ create table t1 (v1 int); # tracked in https://github.com/risingwavelabs/risingwave/issues/13474 # create with duplicate streaming_rate_limit -# statement error -# create materialized view mv1 with (streaming_rate_limit = 1000, streaming_rate_limit = 2000) as select * from t1; +statement error Duplicated option +create materialized view mv1 with (streaming_rate_limit = 1000, streaming_rate_limit = 2000) as select * from t1; # create with unknown fields statement error unexpected options in WITH clause diff --git a/e2e_test/schema_registry/pb.slt b/e2e_test/schema_registry/pb.slt index 618ecd71e960..fb40759d34ad 100644 --- a/e2e_test/schema_registry/pb.slt +++ b/e2e_test/schema_registry/pb.slt @@ -7,8 +7,7 @@ create table sr_pb_test with ( connector = 'kafka', topic = 'sr_pb_test', properties.bootstrap.server = 'message_queue:29092', - scan.startup.mode = 'earliest', - message = 'test.User') + scan.startup.mode = 'earliest') FORMAT plain ENCODE protobuf( schema.registry = 'http://message_queue:8081', message = 'test.User' @@ -20,8 +19,7 @@ create table sr_pb_test_bk with ( connector = 'kafka', topic = 'sr_pb_test', properties.bootstrap.server = 'message_queue:29092', - scan.startup.mode = 'earliest', - message = 'test.User') + scan.startup.mode = 'earliest') FORMAT plain ENCODE protobuf( schema.registry = 'http://message_queue:8081,http://message_queue:8081', message = 'test.User' diff --git a/e2e_test/source/basic/kafka.slt b/e2e_test/source/basic/kafka.slt index 56097de50809..b1ad66bda7f9 100644 --- a/e2e_test/source/basic/kafka.slt +++ b/e2e_test/source/basic/kafka.slt @@ -184,9 +184,11 @@ create table s11 with ( connector = 'kafka', topic = 'proto_c_bin', properties.bootstrap.server = 'message_queue:29092', - scan.startup.mode = 'earliest', - proto.message = 'test.User' -) FORMAT PLAIN ENCODE PROTOBUF (message = 'test.User', schema.location = 'file:///risingwave/proto-complex-schema'); + scan.startup.mode = 'earliest') +FORMAT PLAIN ENCODE PROTOBUF ( + message = 'test.User', + schema.location = 'file:///risingwave/proto-complex-schema' +); statement ok CREATE TABLE s12( @@ -268,9 +270,11 @@ create source s17 with ( connector = 'kafka', topic = 'proto_c_bin', properties.bootstrap.server = 'message_queue:29092', - scan.startup.mode = 'earliest', - proto.message = 'test.User' -) FORMAT PLAIN ENCODE PROTOBUF (message = 'test.User', schema.location = 'file:///risingwave/proto-complex-schema'); + scan.startup.mode = 'earliest') +FORMAT PLAIN ENCODE PROTOBUF ( + message = 'test.User', + schema.location = 'file:///risingwave/proto-complex-schema' +); statement ok create source s18 with ( diff --git a/e2e_test/source/basic/old_row_format_syntax/kafka.slt b/e2e_test/source/basic/old_row_format_syntax/kafka.slt index 3d2e4719d744..05e0d55c28c4 100644 --- a/e2e_test/source/basic/old_row_format_syntax/kafka.slt +++ b/e2e_test/source/basic/old_row_format_syntax/kafka.slt @@ -176,8 +176,7 @@ create table s11 with ( connector = 'kafka', topic = 'proto_c_bin', properties.bootstrap.server = 'message_queue:29092', - scan.startup.mode = 'earliest', - proto.message = 'test.User' + scan.startup.mode = 'earliest' ) row format protobuf message 'test.User' row schema location 'file:///risingwave/proto-complex-schema' statement ok @@ -260,8 +259,7 @@ create source s17 with ( connector = 'kafka', topic = 'proto_c_bin', properties.bootstrap.server = 'message_queue:29092', - scan.startup.mode = 'earliest', - proto.message = 'test.User' + scan.startup.mode = 'earliest' ) row format protobuf message 'test.User' row schema location 'file:///risingwave/proto-complex-schema' statement ok diff --git a/src/frontend/src/utils/with_options.rs b/src/frontend/src/utils/with_options.rs index 4b0a70ef856d..77d418580421 100644 --- a/src/frontend/src/utils/with_options.rs +++ b/src/frontend/src/utils/with_options.rs @@ -16,7 +16,6 @@ use std::collections::{BTreeMap, HashMap}; use std::convert::TryFrom; use std::num::NonZeroU32; -use itertools::Itertools; use risingwave_common::error::{ErrorCode, Result as RwResult, RwError}; use risingwave_connector::source::kafka::{ insert_privatelink_broker_rewrite_map, PRIVATELINK_ENDPOINT_KEY, @@ -172,20 +171,28 @@ impl TryFrom<&[SqlOption]> for WithOptions { type Error = RwError; fn try_from(options: &[SqlOption]) -> Result { - let inner = options - .iter() - .cloned() - .map(|x| match x.value { - Value::CstyleEscapedString(s) => Ok((x.name.real_value(), s.value)), - Value::SingleQuotedString(s) => Ok((x.name.real_value(), s)), - Value::Number(n) => Ok((x.name.real_value(), n)), - Value::Boolean(b) => Ok((x.name.real_value(), b.to_string())), - _ => Err(ErrorCode::InvalidParameterValue( - "`with options` or `with properties` only support single quoted string value and C style escaped string" - .to_owned(), - )), - }) - .try_collect()?; + let mut inner: BTreeMap = BTreeMap::new(); + for option in options { + let key = option.name.real_value(); + let value: String = match option.value.clone() { + Value::CstyleEscapedString(s) => s.value, + Value::SingleQuotedString(s) => s, + Value::Number(n) => n, + Value::Boolean(b) => b.to_string(), + _ => { + return Err(RwError::from(ErrorCode::InvalidParameterValue( + "`with options` or `with properties` only support single quoted string value and C style escaped string" + .to_owned(), + ))) + } + }; + if inner.insert(key.clone(), value).is_some() { + return Err(RwError::from(ErrorCode::InvalidParameterValue(format!( + "Duplicated option: {}", + key + )))); + } + } Ok(Self { inner }) }